You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2011/05/29 14:30:15 UTC

svn commit: r1128870 [2/3] - in /lucene/dev/branches/branch_3x/lucene: ./ backwards/src/test/org/apache/lucene/index/ backwards/src/test/org/apache/lucene/store/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/store/ src/java/org/apache/lu...

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Sun May 29 12:30:14 2011
@@ -0,0 +1,138 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Output is a sequence of bytes, for each input term.
+ *
+ * @lucene.experimental
+ */
+
+public final class ByteSequenceOutputs extends Outputs<BytesRef> {
+
+  private final static BytesRef NO_OUTPUT = new BytesRef();
+
+  private ByteSequenceOutputs() {
+  }
+
+  public static ByteSequenceOutputs getSingleton() {
+    return new ByteSequenceOutputs();
+  }
+
+  @Override
+  public BytesRef common(BytesRef output1, BytesRef output2) {
+    assert output1 != null;
+    assert output2 != null;
+
+    int pos1 = output1.offset;
+    int pos2 = output2.offset;
+    int stopAt1 = pos1 + Math.min(output1.length, output2.length);
+    while(pos1 < stopAt1) {
+      if (output1.bytes[pos1] != output2.bytes[pos2]) {
+        break;
+      }
+      pos1++;
+      pos2++;
+    }
+
+    if (pos1 == output1.offset) {
+      // no common prefix
+      return NO_OUTPUT;
+    } else if (pos1 == output1.offset + output1.length) {
+      // output1 is a prefix of output2
+      return output1;
+    } else if (pos2 == output2.offset + output2.length) {
+      // output2 is a prefix of output1
+      return output2;
+    } else {
+      return new BytesRef(output1.bytes, output1.offset, pos1-output1.offset);
+    }
+  }
+
+  @Override
+  public BytesRef subtract(BytesRef output, BytesRef inc) {
+    assert output != null;
+    assert inc != null;
+    if (inc == NO_OUTPUT) {
+      // no prefix removed
+      return output;
+    } else if (inc.length == output.length) {
+      // entire output removed
+      return NO_OUTPUT;
+    } else {
+      assert inc.length < output.length: "inc.length=" + inc.length + " vs output.length=" + output.length;
+      assert inc.length > 0;
+      return new BytesRef(output.bytes, output.offset + inc.length, output.length-inc.length);
+    }
+  }
+
+  @Override
+  public BytesRef add(BytesRef prefix, BytesRef output) {
+    assert prefix != null;
+    assert output != null;
+    if (prefix == NO_OUTPUT) {
+      return output;
+    } else if (output == NO_OUTPUT) {
+      return prefix;
+    } else {
+      assert prefix.length > 0;
+      assert output.length > 0;
+      BytesRef result = new BytesRef(prefix.length + output.length);
+      System.arraycopy(prefix.bytes, prefix.offset, result.bytes, 0, prefix.length);
+      System.arraycopy(output.bytes, output.offset, result.bytes, prefix.length, output.length);
+      result.length = prefix.length + output.length;
+      return result;
+    }
+  }
+
+  @Override
+  public void write(BytesRef prefix, DataOutput out) throws IOException {
+    assert prefix != null;
+    out.writeVInt(prefix.length);
+    out.writeBytes(prefix.bytes, prefix.offset, prefix.length);
+  }
+
+  @Override
+  public BytesRef read(DataInput in) throws IOException {
+    final int len = in.readVInt();
+    if (len == 0) {
+      return NO_OUTPUT;
+    } else {
+      final BytesRef output = new BytesRef(len);
+      in.readBytes(output.bytes, 0, len);
+      output.length = len;
+      return output;
+    }
+  }
+
+  @Override
+  public BytesRef getNoOutput() {
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public String outputToString(BytesRef output) {
+    return output.utf8ToString();
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java Sun May 29 12:30:14 2011
@@ -0,0 +1,108 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+
+/** Can next() and advance() through the terms in an FST
+ *
+ * @lucene.experimental
+*/
+
+public final class BytesRefFSTEnum<T> extends FSTEnum<T> {
+  private final BytesRef current = new BytesRef(10);
+  private final InputOutput<T> result = new InputOutput<T>();
+  private BytesRef target;
+
+  public static class InputOutput<T> {
+    public BytesRef input;
+    public T output;
+  }
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
+  public BytesRefFSTEnum(FST<T> fst) {
+    super(fst);
+    result.input = current;
+    current.offset = 1;
+  }
+
+  public InputOutput<T> current() {
+    return result;
+  }
+
+  public InputOutput<T> next() throws IOException {
+    //System.out.println("  enum.next");
+    doNext();
+    return setResult();
+  }
+
+  /** Seeks to smallest term that's >= target. */
+  public InputOutput<T> seekCeil(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekCeil();
+    return setResult();
+  }
+
+  /** Seeks to biggest term that's <= target. */
+  public InputOutput<T> seekFloor(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekFloor();
+    return setResult();
+  }
+
+  @Override
+  protected int getTargetLabel() {
+    if (upto-1 == target.length) {
+      return FST.END_LABEL;
+    } else {
+      return target.bytes[target.offset + upto - 1] & 0xFF;
+    }
+  }
+
+  @Override
+  protected int getCurrentLabel() {
+    // current.offset fixed at 1
+    return current.bytes[upto] & 0xFF;
+  }
+
+  @Override
+  protected void setCurrentLabel(int label) {
+    current.bytes[upto] = (byte) label;
+  }
+
+  @Override
+  protected void grow() {
+    current.grow(upto+1);
+  }
+
+  private InputOutput<T> setResult() {
+    if (upto == 0) {
+      return null;
+    } else {
+      current.length = upto-1;
+      result.output = output[upto];
+      return result;
+    }
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java Sun May 29 12:30:14 2011
@@ -0,0 +1,872 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.fst.Builder.UnCompiledNode;
+
+// NOTE: while the FST is able to represent a non-final
+// dead-end state (NON_FINAL_END_NODE=0), the layres above
+// (FSTEnum, Util) have problems with this!!
+
+/** Represents an FST using a compact byte[] format.
+ *  <p> The format is similar to what's used by Morfologik
+ *  (http://sourceforge.net/projects/morfologik).
+ *
+ * @lucene.experimental
+ */
+public 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;
+  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;
+
+  // 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;
+
+  /**
+   * @see #shouldExpand(UnCompiledNode)
+   */
+  final static int FIXED_ARRAY_SHALLOW_DISTANCE = 3; // 0 => only root node.
+
+  /**
+   * @see #shouldExpand(UnCompiledNode)
+   */
+  final static int FIXED_ARRAY_NUM_ARCS_SHALLOW = 5;
+
+  /**
+   * @see #shouldExpand(UnCompiledNode)
+   */
+  final static int FIXED_ARRAY_NUM_ARCS_DEEP = 10;
+
+  private int[] bytesPerArc = new int[0];
+
+  // Increment version to change it
+  private final static String FILE_FORMAT_NAME = "FST";
+  private final static int VERSION_START = 0;
+  private final static int VERSION_CURRENT = VERSION_START;
+
+  // Never serialized; just used to represent the virtual
+  // final node w/ no arcs:
+  private final static int FINAL_END_NODE = -1;
+
+  // Never serialized; just used to represent the virtual
+  // non-final node w/ no arcs:
+  private final static int NON_FINAL_END_NODE = 0;
+
+  // if non-null, this FST accepts the empty string and
+  // produces this output
+  T emptyOutput;
+  private byte[] emptyOutputBytes;
+
+  private byte[] bytes;
+  int byteUpto = 0;
+
+  private int startNode = -1;
+
+  public final Outputs<T> outputs;
+
+  private int lastFrozenNode;
+
+  private final T NO_OUTPUT;
+
+  public int nodeCount;
+  public int arcCount;
+  public int arcWithOutputCount;
+
+  // If arc has this label then that arc is final/accepted
+  public static final int END_LABEL = -1;
+
+  public final static class Arc<T> {
+    public int label;
+    public T output;
+
+    int target;
+
+    byte flags;
+    T nextFinalOutput;
+    int nextArc;
+
+    // This is non-zero if current arcs are fixed array:
+    int posArcsStart;
+    int bytesPerArc;
+    int arcIdx;
+    int numArcs;
+
+    /** Returns this */
+    public Arc<T> copyFrom(Arc<T> other) {
+      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;
+        posArcsStart = other.posArcsStart;
+        arcIdx = other.arcIdx;
+        numArcs = other.numArcs;
+      } else {
+        bytesPerArc = 0;
+      }
+      return this;
+    }
+
+    boolean flag(int flag) {
+      return FST.flag(flags, flag);
+    }
+
+    public boolean isLast() {
+      return flag(BIT_LAST_ARC);
+    }
+
+    boolean isFinal() {
+      return flag(BIT_FINAL_ARC);
+    }
+  };
+
+  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) {
+    this.inputType = inputType;
+    this.outputs = outputs;
+    bytes = new byte[128];
+    NO_OUTPUT = outputs.getNoOutput();
+    
+    writer = new BytesWriter();
+
+    emptyOutput = null;
+  }
+
+  // create an existing FST
+  public FST(DataInput in, Outputs<T> outputs) throws IOException {
+    this.outputs = outputs;
+    writer = null;
+    CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_START, VERSION_START);
+    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));
+    } else {
+      emptyOutput = null;
+    }
+    final byte t = in.readByte();
+    switch(t) {
+      case 0:
+        inputType = INPUT_TYPE.BYTE1;
+        break;
+      case 1:
+        inputType = INPUT_TYPE.BYTE2;
+        break;
+      case 2:
+        inputType = INPUT_TYPE.BYTE4;
+        break;
+    default:
+      throw new IllegalStateException("invalid input type " + t);
+    }
+    startNode = in.readVInt();
+    nodeCount = in.readVInt();
+    arcCount = in.readVInt();
+    arcWithOutputCount = in.readVInt();
+
+    bytes = new byte[in.readVInt()];
+    in.readBytes(bytes, 0, bytes.length);
+    NO_OUTPUT = outputs.getNoOutput();
+  }
+
+  public INPUT_TYPE getInputType() {
+    return inputType;
+  }
+
+  /** Returns bytes used to represent the FST */
+  public int sizeInBytes() {
+    return bytes.length;
+  }
+
+  void finish(int startNode) {
+    if (startNode == FINAL_END_NODE && emptyOutput != null) {
+      startNode = 0;
+    }
+    if (this.startNode != -1) {
+      throw new IllegalStateException("already finished");
+    }
+    byte[] finalBytes = new byte[writer.posWrite];
+    System.arraycopy(bytes, 0, finalBytes, 0, writer.posWrite);
+    bytes = finalBytes;
+    this.startNode = startNode;
+  }
+
+  void setEmptyOutput(T v) throws IOException {
+    if (emptyOutput != null) {
+      emptyOutput = outputs.merge(emptyOutput, v);
+    } else {
+      emptyOutput = v;
+    }
+
+    // TODO: this is messy -- replace with sillyBytesWriter; maybe make
+    // bytes private
+    final int posSave = writer.posWrite;
+    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++;
+    }
+    System.arraycopy(bytes, posSave, emptyOutputBytes, 0, writer.posWrite-posSave);
+    writer.posWrite = posSave;
+  }
+
+  public void save(DataOutput out) throws IOException {
+    if (startNode == -1) {
+      throw new IllegalStateException("call finish first");
+    }
+    CodecUtil.writeHeader(out, FILE_FORMAT_NAME, VERSION_CURRENT);
+    // TODO: really we should encode this as an arc, arriving
+    // to the root node, instead of special casing here:
+    if (emptyOutput != null) {
+      out.writeByte((byte) 1);
+      out.writeVInt(emptyOutputBytes.length);
+      out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length);
+    } else {
+      out.writeByte((byte) 0);
+    }
+    final byte t;
+    if (inputType == INPUT_TYPE.BYTE1) {
+      t = 0;
+    } else if (inputType == INPUT_TYPE.BYTE2) {
+      t = 1;
+    } else {
+      t = 2;
+    }
+    out.writeByte(t);
+    out.writeVInt(startNode);
+    out.writeVInt(nodeCount);
+    out.writeVInt(arcCount);
+    out.writeVInt(arcWithOutputCount);
+    out.writeVInt(bytes.length);
+    out.writeBytes(bytes, 0, bytes.length);
+  }
+
+  private void writeLabel(int v) throws IOException {
+    assert v >= 0: "v=" + v;
+    if (inputType == INPUT_TYPE.BYTE1) {
+      assert v <= 255: "v=" + v;
+      writer.writeByte((byte) v);
+    } else if (inputType == INPUT_TYPE.BYTE2) {
+      assert v <= 65535: "v=" + v;
+      writer.writeVInt(v);
+    } else {
+      //writeInt(v);
+      writer.writeVInt(v);
+    }
+  }
+
+  int readLabel(DataInput in) throws IOException {
+    final int v;
+    if (inputType == INPUT_TYPE.BYTE1) {
+      v = in.readByte()&0xFF;
+    } else { 
+      v = in.readVInt();
+    }
+    return v;
+  }
+
+  // returns true if the node at this address has any
+  // outgoing arcs
+  public 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) {
+        return FINAL_END_NODE;
+      } else {
+        return NON_FINAL_END_NODE;
+      }
+    }
+
+    int startAddress = writer.posWrite;
+    //System.out.println("  startAddr=" + startAddress);
+
+    final boolean doFixedArray = shouldExpand(node);
+    final int fixedArrayStart;
+    if (doFixedArray) {
+      if (bytesPerArc.length < node.numArcs) {
+        bytesPerArc = new int[ArrayUtil.oversize(node.numArcs, 1)];
+      }
+      // write a "false" first arc:
+      writer.writeByte((byte) BIT_ARCS_AS_FIXED_ARRAY);
+      writer.writeVInt(node.numArcs);
+      // placeholder -- we'll come back and write the number
+      // of bytes per arc here:
+      writer.writeByte((byte) 0);
+      fixedArrayStart = writer.posWrite;
+      //System.out.println("  do fixed arcs array arcsStart=" + fixedArrayStart);
+    } else {
+      fixedArrayStart = 0;
+    }
+
+    nodeCount++;
+    arcCount += node.numArcs;
+    
+    final int lastArc = node.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];
+      final Builder.CompiledNode target = (Builder.CompiledNode) arc.target;
+      int flags = 0;
+
+      if (arcIdx == lastArc) {
+        flags += BIT_LAST_ARC;
+      }
+
+      if (lastFrozenNode == target.address && !doFixedArray) {
+        flags += BIT_TARGET_NEXT;
+      }
+
+      if (arc.isFinal) {
+        flags += BIT_FINAL_ARC;
+        if (arc.nextFinalOutput != NO_OUTPUT) {
+          flags += BIT_ARC_HAS_FINAL_OUTPUT;
+        }
+      } else {
+        assert arc.nextFinalOutput == NO_OUTPUT;
+      }
+
+      boolean targetHasArcs = target.address > 0;
+
+      if (!targetHasArcs) {
+        flags += BIT_STOP_NODE;
+      }
+
+      if (arc.output != NO_OUTPUT) {
+        flags += BIT_ARC_HAS_OUTPUT;
+      }
+
+      writer.writeByte((byte) flags);
+      writeLabel(arc.label);
+
+      //System.out.println("  write arc: label=" + arc.label + " flags=" + flags);
+
+      if (arc.output != NO_OUTPUT) {
+        outputs.write(arc.output, writer);
+        arcWithOutputCount++;
+      }
+      if (arc.nextFinalOutput != NO_OUTPUT) {
+        outputs.write(arc.nextFinalOutput, writer);
+      }
+
+      if (targetHasArcs && (doFixedArray || lastFrozenNode != target.address)) {
+        assert target.address > 0;
+        writer.writeInt(target.address);
+      }
+
+      // just write the arcs "like normal" on first pass,
+      // but record how many bytes each one took, and max
+      // byte size:
+      if (doFixedArray) {
+        bytesPerArc[arcIdx] = writer.posWrite - lastArcStart;
+        lastArcStart = writer.posWrite;
+        maxBytesPerArc = Math.max(maxBytesPerArc, bytesPerArc[arcIdx]);
+        //System.out.println("    bytes=" + bytesPerArc[arcIdx]);
+      }
+    }
+
+    if (doFixedArray) {
+      assert maxBytesPerArc > 0;
+      // 2nd pass just "expands" all arcs to take up a fixed
+      // byte size
+      final int sizeNeeded = fixedArrayStart + node.numArcs * maxBytesPerArc;
+      bytes = ArrayUtil.grow(bytes, sizeNeeded);
+      if (maxBytesPerArc > 255) {
+        throw new IllegalStateException("max arc size is too large (" + maxBytesPerArc + ")");
+      }
+      bytes[fixedArrayStart-1] = (byte) maxBytesPerArc;
+
+      // expand the arcs in place, backwards
+      int srcPos = writer.posWrite;
+      int destPos = fixedArrayStart + node.numArcs*maxBytesPerArc;
+      writer.posWrite = destPos;
+      for(int arcIdx=node.numArcs-1;arcIdx>=0;arcIdx--) {
+        //System.out.println("  repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
+        destPos -= maxBytesPerArc;
+        srcPos -= bytesPerArc[arcIdx];
+        if (srcPos != destPos) {
+          assert destPos > srcPos;
+          System.arraycopy(bytes, srcPos, bytes, destPos, bytesPerArc[arcIdx]);
+        }
+      }
+    }
+
+    // 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;
+
+    int left = startAddress;
+    int right = endAddress;
+    while (left < right) {
+      final byte b = bytes[left];
+      bytes[left++] = bytes[right];
+      bytes[right--] = b;
+    }
+
+    return endAddress;
+  }
+
+  /** 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;
+    } else {
+      arc.flags = BIT_LAST_ARC;
+      arc.nextFinalOutput = NO_OUTPUT;
+    }
+    arc.output = NO_OUTPUT;
+
+    // If there are no nodes, ie, the FST only accepts the
+    // empty string, then startNode is 0, and then readFirstTargetArc
+    arc.target = startNode;
+    return arc;
+  }
+
+  /** Follows the <code>follow</code> arc and reads the last
+   *  arc of its target; this changes the provided
+   *  <code>arc</code> (2nd arg) in-place and returns it.
+   * 
+   * @return Returns the second argument
+   * (<code>arc</code>). */
+  public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
+    //System.out.println("readLast");
+    if (!targetHasArcs(follow)) {
+      //System.out.println("  end node");
+      assert follow.isFinal();
+      arc.label = -1;
+      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)) {
+        // array: jump straight to end
+        arc.numArcs = in.readVInt();
+        arc.bytesPerArc = in.readByte() & 0xFF;
+        //System.out.println("  array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
+        arc.posArcsStart = in.pos;
+        arc.arcIdx = arc.numArcs - 2;
+      } else {
+        // non-array: linear scan
+        arc.bytesPerArc = 0;
+        //System.out.println("  scan");
+        while(!arc.isLast()) {
+          // skip this arc:
+          readLabel(in);
+          if (arc.flag(BIT_ARC_HAS_OUTPUT)) {
+            outputs.read(in);
+          }
+          if (arc.flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
+            outputs.read(in);
+          }
+          if (arc.flag(BIT_STOP_NODE)) {
+          } else if (arc.flag(BIT_TARGET_NEXT)) {
+          } else {
+            in.pos -= 4;
+          }
+          arc.flags = in.readByte();
+        }
+        arc.nextArc = in.pos+1;
+      }
+      readNextRealArc(arc);
+      assert arc.isLast();
+      return arc;
+    }
+  }
+
+  /**
+   * Follow the <code>follow</code> arc and read the first arc of its target;
+   * this changes the provided <code>arc</code> (2nd arg) in-place and returns
+   * it.
+   * 
+   * @return Returns the second argument (<code>arc</code>).
+   */
+  public Arc<T> readFirstTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
+    //int pos = address;
+    //System.out.println("    readFirstTarget follow.target=" + follow.target + " isFinal=" + follow.isFinal());
+    if (follow.isFinal()) {
+      // Insert "fake" final first arc:
+      arc.label = -1;
+      arc.output = follow.nextFinalOutput;
+      if (follow.target <= 0) {
+        arc.flags = BIT_LAST_ARC;
+      } else {
+        arc.flags = 0;
+        arc.nextArc = follow.target;
+      }
+      //System.out.println("    insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
+      return arc;
+    } else {
+      return readFirstRealArc(follow.target, arc);
+    }
+  }
+
+  // Not private because NodeHash needs access:
+  Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
+
+    final BytesReader in = getBytesReader(address);
+
+    arc.flags = in.readByte();
+
+    if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+      //System.out.println("  fixedArray");
+      // this is first arc in a fixed-array
+      arc.numArcs = in.readVInt();
+      arc.bytesPerArc = in.readByte() & 0xFF;
+      arc.arcIdx = -1;
+      arc.nextArc = arc.posArcsStart = in.pos;
+      //System.out.println("  bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
+    } else {
+      arc.nextArc = address;
+      arc.bytesPerArc = 0;
+    }
+    return readNextRealArc(arc);
+  }
+
+  /**
+   * Checks if <code>arc</code>'s target state is in expanded (or vector) format. 
+   * 
+   * @return Returns <code>true</code> if <code>arc</code> points to a state in an
+   * expanded array format.
+   */
+  boolean isExpandedTarget(Arc<T> follow) throws IOException {
+    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;
+    }
+  }
+
+  /** In-place read; returns the arc. */
+  public Arc<T> readNextArc(Arc<T> arc) throws IOException {
+    if (arc.label == -1) {
+      // 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;
+      }
+      return readFirstRealArc(arc.nextArc, arc);
+    } else {
+      return readNextRealArc(arc);
+    }
+  }
+
+  /** Peeks at next arc's label; does not alter arc.  Do
+   *  not call this if arc.isLast()! */
+  public int readNextArcLabel(Arc<T> arc) throws IOException {
+    assert !arc.isLast();
+
+    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)) {
+        //System.out.println("    nextArc fake array");
+        in.pos--;
+        in.readVInt();
+        in.readByte();
+      }
+    } 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);
+      } else {
+        // arcs are packed
+        //System.out.println("    nextArc real packed");
+        in = getBytesReader(arc.nextArc);
+      }
+    }
+    // skip flags
+    in.readByte();
+    return readLabel(in);
+  }
+
+  Arc<T> readNextRealArc(Arc<T> arc) throws IOException {
+    // this is a continuing arc in a fixed array
+    final BytesReader in;
+    if (arc.bytesPerArc != 0) {
+      // arcs are at fixed entries
+      arc.arcIdx++;
+      assert arc.arcIdx < arc.numArcs;
+      in = getBytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
+    } else {
+      // arcs are packed
+      in = getBytesReader(arc.nextArc);
+    }
+    arc.flags = in.readByte();
+    arc.label = readLabel(in);
+
+    if (arc.flag(BIT_ARC_HAS_OUTPUT)) {
+      arc.output = outputs.read(in);
+    } else {
+      arc.output = outputs.getNoOutput();
+    }
+
+    if (arc.flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
+      arc.nextFinalOutput = outputs.read(in);
+    } else {
+      arc.nextFinalOutput = outputs.getNoOutput();
+    }
+
+    if (arc.flag(BIT_STOP_NODE)) {
+      if (arc.flag(BIT_FINAL_ARC)) {
+        arc.target = FINAL_END_NODE;
+      } else {
+        arc.target = NON_FINAL_END_NODE;
+      }
+      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;
+        }
+      }
+      arc.target = in.pos;
+    } 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 {
+
+    if (labelToMatch == END_LABEL) {
+      if (follow.isFinal()) {
+        arc.output = follow.nextFinalOutput;
+        arc.label = END_LABEL;
+        return arc;
+      } else {
+        return null;
+      }
+    }
+
+    if (!targetHasArcs(follow)) {
+      return null;
+    }
+
+    // TODO: maybe make an explicit thread state that holds
+    // reusable stuff eg BytesReader:
+    final BytesReader in = getBytesReader(follow.target);
+
+    if ((in.readByte() & BIT_ARCS_AS_FIXED_ARRAY) != 0) {
+      // Arcs are full array; do binary search:
+      arc.numArcs = in.readVInt();
+      arc.bytesPerArc = in.readByte() & 0xFF;
+      arc.posArcsStart = in.pos;
+      int low = 0;
+      int high = arc.numArcs-1;
+      while (low <= high) {
+        int mid = (low + high) >>> 1;
+        in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+        int midLabel = readLabel(in);
+        final int cmp = midLabel - labelToMatch;
+        if (cmp < 0)
+          low = mid + 1;
+        else if (cmp > 0)
+          high = mid - 1;
+        else {
+          arc.arcIdx = mid-1;
+          return readNextRealArc(arc);
+        }
+      }
+
+      return null;
+    }
+
+    // Linear scan
+    readFirstTargetArc(follow, arc);
+    while(true) {
+      if (arc.label == labelToMatch) {
+        return arc;
+      } else if (arc.label > labelToMatch) {
+        return null;
+      } else if (arc.isLast()) {
+        return null;
+      } else {
+        readNextArc(arc);
+      }
+    }
+  }
+
+  private void seekToNextNode(BytesReader in) throws IOException {
+
+    while(true) {
+
+      final int flags = in.readByte();
+      readLabel(in);
+
+      if (flag(flags, BIT_ARC_HAS_OUTPUT)) {
+        outputs.read(in);
+      }
+
+      if (flag(flags, BIT_ARC_HAS_FINAL_OUTPUT)) {
+        outputs.read(in);
+      }
+
+      if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
+        in.readInt();
+      }
+
+      if (flag(flags, BIT_LAST_ARC)) {
+        return;
+      }
+    }
+  }
+
+  public int getNodeCount() {
+    // 1+ in order to count the -1 implicit final node
+    return 1+nodeCount;
+  }
+  
+  public int getArcCount() {
+    return arcCount;
+  }
+
+  public int getArcWithOutputCount() {
+    return arcWithOutputCount;
+  }
+  
+  /**
+   * Nodes will be expanded if their depth (distance from the root node) is
+   * &lt;= this value and their number of arcs is &gt;=
+   * {@link #FIXED_ARRAY_NUM_ARCS_SHALLOW}.
+   * 
+   * <p>
+   * Fixed array consumes more RAM but enables binary search on the arcs
+   * (instead of a linear scan) on lookup by arc label.
+   * 
+   * @return <code>true</code> if <code>node</code> should be stored in an
+   *         expanded (array) form.
+   * 
+   * @see #FIXED_ARRAY_NUM_ARCS_DEEP
+   * @see Builder.UnCompiledNode#depth
+   */
+  private boolean shouldExpand(UnCompiledNode<T> node) {
+    return (node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) || 
+            node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP;
+  }
+
+  // Non-static: writes to FST's byte[]
+  class BytesWriter extends DataOutput {
+    int posWrite;
+
+    public BytesWriter() {
+      // pad: ensure no node gets address 0 which is reserved to mean
+      // the stop state w/ no arcs
+      posWrite = 1;
+    }
+
+    @Override
+    public void writeByte(byte b) {
+      if (bytes.length == posWrite) {
+        bytes = ArrayUtil.grow(bytes);
+      }
+      assert posWrite < bytes.length: "posWrite=" + posWrite + " bytes.length=" + bytes.length;
+      bytes[posWrite++] = b;
+    }
+
+    @Override
+    public void writeBytes(byte[] b, int offset, int length) {
+      final int size = posWrite + length;
+      bytes = ArrayUtil.grow(bytes, size);
+      System.arraycopy(b, offset, bytes, posWrite, length);
+      posWrite += length;
+    }
+  }
+
+  final BytesReader getBytesReader(int pos) {
+    // TODO: maybe re-use via ThreadLocal?
+    return new BytesReader(pos);
+  }
+
+  // Non-static: reads byte[] from FST
+  class BytesReader extends DataInput {
+    int pos;
+
+    public BytesReader(int pos) {
+      this.pos = pos;
+    }
+
+    @Override
+    public byte readByte() {
+      return bytes[pos--];
+    }
+
+    @Override
+    public void readBytes(byte[] b, int offset, int len) {
+      for(int i=0;i<len;i++) {
+        b[offset+i] = bytes[pos--];
+      }
+    }
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java Sun May 29 12:30:14 2011
@@ -0,0 +1,479 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.io.IOException;
+
+/** Can next() and advance() through the terms in an FST
+ *
+  * @lucene.experimental
+*/
+
+abstract class FSTEnum<T> {
+  protected final FST<T> fst;
+
+  @SuppressWarnings("unchecked") protected FST.Arc<T>[] arcs = new FST.Arc[10];
+  // outputs are cumulative
+  @SuppressWarnings("unchecked") protected T[] output = (T[]) new Object[10];
+
+  protected final T NO_OUTPUT;
+  protected final FST.Arc<T> scratchArc = new FST.Arc<T>();
+
+  protected int upto;
+  protected int targetLength;
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
+  protected FSTEnum(FST<T> fst) {
+    this.fst = fst;
+    NO_OUTPUT = fst.outputs.getNoOutput();
+    fst.getFirstArc(getArc(0));
+    output[0] = NO_OUTPUT;
+  }
+
+  protected abstract int getTargetLabel();
+  protected abstract int getCurrentLabel();
+
+  protected abstract void setCurrentLabel(int label);
+  protected abstract void grow();
+
+  /** Rewinds enum state to match the shared prefix between
+   *  current term and target term */
+  protected final void rewindPrefix() throws IOException {
+    if (upto == 0) {
+      //System.out.println("  init");
+      upto = 1;
+      fst.readFirstTargetArc(getArc(0), getArc(1));
+      return;
+    }
+    //System.out.println("  rewind upto=" + upto + " vs targetLength=" + targetLength);
+
+    final int currentLimit = upto;
+    upto = 1;
+    while (upto < currentLimit && upto <= targetLength+1) {
+      final int cmp = getCurrentLabel() - getTargetLabel();
+      if (cmp < 0) {
+        // seek forward
+        break;
+      } else if (cmp > 0) {
+        // seek backwards -- reset this arc to the first arc
+        final FST.Arc<T> arc = getArc(upto);
+        fst.readFirstTargetArc(getArc(upto-1), arc);
+        //System.out.println("    seek first arc");
+        break;
+      }
+      upto++;
+    }
+  }
+
+  protected void doNext() throws IOException {
+    //System.out.println("FE: next upto=" + upto);
+    if (upto == 0) {
+      //System.out.println("  init");
+      upto = 1;
+      fst.readFirstTargetArc(getArc(0), getArc(1));
+    } else {
+      // pop
+      //System.out.println("  check pop curArc target=" + arcs[upto].target + " label=" + arcs[upto].label + " isLast?=" + arcs[upto].isLast());
+      while (arcs[upto].isLast()) {
+        upto--;
+        if (upto == 0) {
+          //System.out.println("  eof");
+          return;
+        }
+      }
+      fst.readNextArc(arcs[upto]);
+    }
+
+    pushFirst();
+  }
+
+  // TODO: should we return a status here (SEEK_FOUND / SEEK_NOT_FOUND /
+  // SEEK_END)?  saves the eq check above?
+
+  /** Seeks to smallest term that's >= target. */
+  protected void doSeekCeil() throws IOException {
+
+    //System.out.println("    advance len=" + target.length + " curlen=" + current.length);
+
+    // TODO: possibly caller could/should provide common
+    // prefix length?  ie this work may be redundant if
+    // caller is in fact intersecting against its own
+    // automaton
+
+    //System.out.println("FE.seekCeil upto=" + upto);
+
+    // Save time by starting at the end of the shared prefix
+    // b/w our current term & the target:
+    rewindPrefix();
+    //System.out.println("  after rewind upto=" + upto);
+
+    FST.Arc<T> arc = getArc(upto);
+    int targetLabel = getTargetLabel();
+    //System.out.println("  init targetLabel=" + targetLabel);
+
+    // 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 + ") vs targetLabel=" + targetLabel);
+
+      if (arc.bytesPerArc != 0 && arc.label != -1) {
+
+        // Arcs are fixed array -- use binary search to find
+        // the target.
+
+        final FST<T>.BytesReader in = fst.getBytesReader(0);
+        int low = arc.arcIdx;
+        int high = arc.numArcs-1;
+        int mid = 0;
+        //System.out.println("do arc array low=" + low + " high=" + high + " targetLabel=" + targetLabel);
+        boolean found = false;
+        while (low <= high) {
+          mid = (low + high) >>> 1;
+          in.pos = arc.posArcsStart - 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)
+            low = mid + 1;
+          else if (cmp > 0)
+            high = mid - 1;
+          else {
+            found = true;
+            break;
+          }
+        }
+
+        // NOTE: this code is dup'd w/ the code below (in
+        // the outer else clause):
+        if (found) {
+          // Match
+          arc.arcIdx = mid-1;
+          fst.readNextRealArc(arc);
+          assert arc.arcIdx == mid;
+          assert arc.label == targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel + " mid=" + mid;
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+          continue;
+        } else if (low == arc.numArcs) {
+          // Dead end
+          arc.arcIdx = arc.numArcs-2;
+          fst.readNextRealArc(arc);
+          assert arc.isLast();
+          // Dead end (target is after the last arc);
+          // rollback to last fork then push
+          upto--;
+          while(true) {
+            if (upto == 0) {
+              return;
+            }
+            final FST.Arc<T> prevArc = getArc(upto);
+            //System.out.println("  rollback upto=" + upto + " arc.label=" + prevArc.label + " isLast?=" + prevArc.isLast());
+            if (!prevArc.isLast()) {
+              fst.readNextArc(prevArc);
+              pushFirst();
+              return;
+            }
+            upto--;
+          }
+        } else {
+          arc.arcIdx = (low > high ? low : high)-1;
+          fst.readNextRealArc(arc);
+          assert arc.label > targetLabel;
+          pushFirst();
+          return;
+        }
+      } else {
+        // Arcs are not array'd -- must do linear scan:
+        if (arc.label == targetLabel) {
+          // recurse
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+        } else if (arc.label > targetLabel) {
+          pushFirst();
+          return;
+        } else if (arc.isLast()) {
+          // Dead end (target is after the last arc);
+          // rollback to last fork then push
+          upto--;
+          while(true) {
+            if (upto == 0) {
+              return;
+            }
+            final FST.Arc<T> prevArc = getArc(upto);
+            //System.out.println("  rollback upto=" + upto + " arc.label=" + prevArc.label + " isLast?=" + prevArc.isLast());
+            if (!prevArc.isLast()) {
+              fst.readNextArc(prevArc);
+              pushFirst();
+              return;
+            }
+            upto--;
+          }
+        } else {
+          // keep scanning
+          //System.out.println("    next scan");
+          fst.readNextArc(arc);
+        }
+      }
+    }
+  }
+
+  // TODO: should we return a status here (SEEK_FOUND / SEEK_NOT_FOUND /
+  // SEEK_END)?  saves the eq check above?
+  /** Seeks to largest term that's <= target. */
+  protected void doSeekFloor() throws IOException {
+
+    // TODO: possibly caller could/should provide common
+    // prefix length?  ie this work may be redundant if
+    // caller is in fact intersecting against its own
+    // automaton
+    //System.out.println("FE: seek floor upto=" + upto);
+
+    // Save CPU by starting at the end of the shared prefix
+    // b/w our current term & the target:
+    rewindPrefix();
+
+    //System.out.println("FE: after rewind upto=" + upto);
+
+    FST.Arc<T> arc = getArc(upto);
+    int targetLabel = getTargetLabel();
+
+    //System.out.println("FE: init targetLabel=" + targetLabel);
+
+    // 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());
+
+      if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
+        // Arcs are fixed array -- use binary search to find
+        // the target.
+
+        final FST<T>.BytesReader in = fst.getBytesReader(0);
+        int low = arc.arcIdx;
+        int high = arc.numArcs-1;
+        int mid = 0;
+        //System.out.println("do arc array low=" + low + " high=" + high + " targetLabel=" + targetLabel);
+        boolean found = false;
+        while (low <= high) {
+          mid = (low + high) >>> 1;
+          in.pos = arc.posArcsStart - 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)
+            low = mid + 1;
+          else if (cmp > 0)
+            high = mid - 1;
+          else {
+            found = true;
+            break;
+          }
+        }
+
+        // NOTE: this code is dup'd w/ the code below (in
+        // the outer else clause):
+        if (found) {
+          // Match -- recurse
+          //System.out.println("  match!  arcIdx=" + mid);
+          arc.arcIdx = mid-1;
+          fst.readNextRealArc(arc);
+          assert arc.arcIdx == mid;
+          assert arc.label == targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel + " mid=" + mid;
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+          continue;
+        } else if (high == -1) {
+          //System.out.println("  before first");
+          // Very first arc is after our target
+          // TODO: if each arc could somehow read the arc just
+          // before, we can save this re-scan.  The ceil case
+          // doesn't need this because it reads the next arc
+          // instead:
+          while(true) {
+            // First, walk backwards until we find a first arc
+            // that's before our target label:
+            fst.readFirstTargetArc(getArc(upto-1), arc);
+            if (arc.label < targetLabel) {
+              // Then, scan forwards to the arc just before
+              // the targetLabel:
+              while(!arc.isLast() && fst.readNextArcLabel(arc) < targetLabel) {
+                fst.readNextArc(arc);
+              }
+              pushLast();
+              return;
+            }
+            upto--;
+            if (upto == 0) {
+              return;
+            }
+            targetLabel = getTargetLabel();
+            arc = getArc(upto);
+          }
+        } else {
+          // There is a floor arc:
+          arc.arcIdx = (low > high ? high : low)-1;
+          //System.out.println(" hasFloor arcIdx=" + (arc.arcIdx+1));
+          fst.readNextRealArc(arc);
+          assert arc.isLast() || fst.readNextArcLabel(arc) > targetLabel;
+          assert arc.label < targetLabel;
+          pushLast();
+          return;
+        }        
+      } else {
+
+        if (arc.label == targetLabel) {
+          // Match -- recurse
+          output[upto] = fst.outputs.add(output[upto-1], arc.output);
+          if (targetLabel == FST.END_LABEL) {
+            return;
+          }
+          setCurrentLabel(arc.label);
+          incr();
+          arc = fst.readFirstTargetArc(arc, getArc(upto));
+          targetLabel = getTargetLabel();
+        } else if (arc.label > targetLabel) {
+          // TODO: if each arc could somehow read the arc just
+          // before, we can save this re-scan.  The ceil case
+          // doesn't need this because it reads the next arc
+          // instead:
+          while(true) {
+            // First, walk backwards until we find a first arc
+            // that's before our target label:
+            fst.readFirstTargetArc(getArc(upto-1), arc);
+            if (arc.label < targetLabel) {
+              // Then, scan forwards to the arc just before
+              // the targetLabel:
+              while(!arc.isLast() && fst.readNextArcLabel(arc) < targetLabel) {
+                fst.readNextArc(arc);
+              }
+              pushLast();
+              return;
+            }
+            upto--;
+            if (upto == 0) {
+              return;
+            }
+            targetLabel = getTargetLabel();
+            arc = getArc(upto);
+          }
+        } else if (!arc.isLast()) {
+          //System.out.println("  check next label=" + fst.readNextArcLabel(arc) + " (" + (char) fst.readNextArcLabel(arc) + ")");
+          if (fst.readNextArcLabel(arc) > targetLabel) {
+            pushLast();
+            return;
+          } else {
+            // keep scanning
+            fst.readNextArc(arc);
+          }
+        } else {
+          pushLast();
+          return;
+        }
+      }
+    }
+  }
+
+  private void incr() {
+    upto++;
+    grow();
+    if (arcs.length <= upto) {
+      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
+        new FST.Arc[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
+      arcs = newArcs;
+    }
+    if (output.length <= upto) {
+      @SuppressWarnings("unchecked") final T[] newOutput =
+        (T[]) new Object[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      System.arraycopy(output, 0, newOutput, 0, output.length);
+      output = newOutput;
+    }
+  }
+
+  // Appends current arc, and then recurses from its target,
+  // appending first arc all the way to the final node
+  private void pushFirst() throws IOException {
+
+    FST.Arc<T> arc = arcs[upto];
+    assert arc != null;
+
+    while (true) {
+      output[upto] = fst.outputs.add(output[upto-1], arc.output);
+      if (arc.label == FST.END_LABEL) {
+        // Final node
+        break;
+      }
+      //System.out.println("  pushFirst label=" + (char) arc.label + " upto=" + upto + " output=" + fst.outputs.outputToString(output[upto]));
+      setCurrentLabel(arc.label);
+      incr();
+      
+      final FST.Arc<T> nextArc = getArc(upto);
+      fst.readFirstTargetArc(arc, nextArc);
+      arc = nextArc;
+    }
+  }
+
+  // Recurses from current arc, appending last arc all the
+  // way to the first final node
+  private void pushLast() throws IOException {
+
+    FST.Arc<T> arc = arcs[upto];
+    assert arc != null;
+
+    while (true) {
+      setCurrentLabel(arc.label);
+      output[upto] = fst.outputs.add(output[upto-1], arc.output);
+      if (arc.label == FST.END_LABEL) {
+        // Final node
+        break;
+      }
+      incr();
+
+      arc = fst.readLastTargetArc(arc, getArc(upto));
+    }
+  }
+
+  private FST.Arc<T> getArc(int idx) {
+    if (arcs[idx] == null) {
+      arcs[idx] = new FST.Arc<T>();
+    }
+    return arcs[idx];
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java Sun May 29 12:30:14 2011
@@ -0,0 +1,142 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.IntsRef;
+
+/**
+ * Output is a sequence of ints, for each input term.
+ *
+ * @lucene.experimental
+ */
+
+public final class IntSequenceOutputs extends Outputs<IntsRef> {
+
+  private final static IntsRef NO_OUTPUT = new IntsRef();
+
+  private IntSequenceOutputs() {
+  }
+
+  public static IntSequenceOutputs getSingleton() {
+    return new IntSequenceOutputs();
+  }
+
+  @Override
+  public IntsRef common(IntsRef output1, IntsRef output2) {
+    assert output1 != null;
+    assert output2 != null;
+
+    int pos1 = output1.offset;
+    int pos2 = output2.offset;
+    int stopAt1 = pos1 + Math.min(output1.length, output2.length);
+    while(pos1 < stopAt1) {
+      if (output1.ints[pos1] != output2.ints[pos2]) {
+        break;
+      }
+      pos1++;
+      pos2++;
+    }
+
+    if (pos1 == output1.offset) {
+      // no common prefix
+      return NO_OUTPUT;
+    } else if (pos1 == output1.offset + output1.length) {
+      // output1 is a prefix of output2
+      return output1;
+    } else if (pos2 == output2.offset + output2.length) {
+      // output2 is a prefix of output1
+      return output2;
+    } else {
+      return new IntsRef(output1.ints, output1.offset, pos1-output1.offset);
+    }
+  }
+
+  @Override
+  public IntsRef subtract(IntsRef output, IntsRef inc) {
+    assert output != null;
+    assert inc != null;
+    if (inc == NO_OUTPUT) {
+      // no prefix removed
+      return output;
+    } else if (inc.length == output.length) {
+      // entire output removed
+      return NO_OUTPUT;
+    } else {
+      assert inc.length < output.length: "inc.length=" + inc.length + " vs output.length=" + output.length;
+      assert inc.length > 0;
+      return new IntsRef(output.ints, output.offset + inc.length, output.length-inc.length);
+    }
+  }
+
+  @Override
+  public IntsRef add(IntsRef prefix, IntsRef output) {
+    assert prefix != null;
+    assert output != null;
+    if (prefix == NO_OUTPUT) {
+      return output;
+    } else if (output == NO_OUTPUT) {
+      return prefix;
+    } else {
+      assert prefix.length > 0;
+      assert output.length > 0;
+      IntsRef result = new IntsRef(prefix.length + output.length);
+      System.arraycopy(prefix.ints, prefix.offset, result.ints, 0, prefix.length);
+      System.arraycopy(output.ints, output.offset, result.ints, prefix.length, output.length);
+      result.length = prefix.length + output.length;
+      return result;
+    }
+  }
+
+  @Override
+  public void write(IntsRef prefix, DataOutput out) throws IOException {
+    assert prefix != null;
+    out.writeVInt(prefix.length);
+    for(int idx=0;idx<prefix.length;idx++) {
+      out.writeVInt(prefix.ints[prefix.offset+idx]);
+    }
+  }
+
+  @Override
+  public IntsRef read(DataInput in) throws IOException {
+    final int len = in.readVInt();
+    if (len == 0) {
+      return NO_OUTPUT;
+    } else {
+      final IntsRef output = new IntsRef(len);
+      for(int idx=0;idx<len;idx++) {
+        output.ints[idx] = in.readVInt();
+      }
+      output.length = len;
+      return output;
+    }
+  }
+
+  @Override
+  public IntsRef getNoOutput() {
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public String outputToString(IntsRef output) {
+    return output.toString();
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java Sun May 29 12:30:14 2011
@@ -0,0 +1,108 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.IntsRef;
+
+import java.io.IOException;
+
+/** Can next() and advance() through the terms in an FST
+ *
+  * @lucene.experimental
+*/
+
+public final class IntsRefFSTEnum<T> extends FSTEnum<T> {
+  private final IntsRef current = new IntsRef(10);
+  private final InputOutput<T> result = new InputOutput<T>();
+  private IntsRef target;
+
+  public static class InputOutput<T> {
+    public IntsRef input;
+    public T output;
+  }
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
+  public IntsRefFSTEnum(FST<T> fst) {
+    super(fst);
+    result.input = current;
+    current.offset = 1;
+  }
+
+  public InputOutput<T> current() {
+    return result;
+  }
+
+  public InputOutput<T> next() throws IOException {
+    //System.out.println("  enum.next");
+    doNext();
+    return setResult();
+  }
+
+  /** Seeks to smallest term that's >= target. */
+  public InputOutput<T> seekCeil(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekCeil();
+    return setResult();
+  }
+
+  /** Seeks to biggest term that's <= target. */
+  public InputOutput<T> seekFloor(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekFloor();
+    return setResult();
+  }
+
+  @Override
+  protected int getTargetLabel() {
+    if (upto-1 == target.length) {
+      return FST.END_LABEL;
+    } else {
+      return target.ints[target.offset + upto - 1];
+    }
+  }
+
+  @Override
+  protected int getCurrentLabel() {
+    // current.offset fixed at 1
+    return current.ints[upto];
+  }
+
+  @Override
+  protected void setCurrentLabel(int label) {
+    current.ints[upto] = label;
+  }
+
+  @Override
+  protected void grow() {
+    current.grow(upto+1);
+  }
+
+  private InputOutput<T> setResult() {
+    if (upto == 0) {
+      return null;
+    } else {
+      current.length = upto-1;
+      result.output = output[upto];
+      return result;
+    }
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NoOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NoOutputs.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NoOutputs.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NoOutputs.java Sun May 29 12:30:14 2011
@@ -0,0 +1,96 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * Use this if you just want to build an FSA.
+ *
+ * @lucene.experimental
+ */
+
+public final class NoOutputs extends Outputs<Object> {
+
+  final Object NO_OUTPUT = new Object() {
+    // NodeHash calls hashCode for this output; we fix this
+    // so we get deterministic hashing.
+    @Override
+    public int hashCode() {
+      return 42;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      return other == this;
+    }
+  };
+
+  private static final NoOutputs singleton = new NoOutputs();
+
+  private NoOutputs() {
+  }
+
+  public static NoOutputs getSingleton() {
+    return singleton;
+  }
+
+  @Override
+  public Object common(Object output1, Object output2) {
+    assert output1 == NO_OUTPUT;
+    assert output2 == NO_OUTPUT;
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public Object subtract(Object output, Object inc) {
+    assert output == NO_OUTPUT;
+    assert inc == NO_OUTPUT;
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public Object add(Object prefix, Object output) {
+    assert prefix == NO_OUTPUT: "got " + prefix;
+    assert output == NO_OUTPUT;
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public void write(Object prefix, DataOutput out) {
+    //assert false;
+  }
+
+  @Override
+  public Object read(DataInput in) {
+    //assert false;
+    //return null;
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public Object getNoOutput() {
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public String outputToString(Object output) {
+    return "";
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java Sun May 29 12:30:14 2011
@@ -0,0 +1,165 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+// Used to dedup states (lookup already-frozen states)
+final class NodeHash<T> {
+
+  private int[] table;
+  private int count;
+  private int mask;
+  private final FST<T> fst;
+  private final FST.Arc<T> scratchArc = new FST.Arc<T>();
+
+  public NodeHash(FST<T> fst) {
+    table = new int[16];
+    mask = 15;
+    this.fst = fst;
+  }
+
+  private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address) throws IOException {
+    fst.readFirstRealArc(address, scratchArc);
+    if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
+      return false;
+    }
+    for(int arcUpto=0;arcUpto<node.numArcs;arcUpto++) {
+      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 ||
+          !arc.nextFinalOutput.equals(scratchArc.nextFinalOutput) ||
+          arc.isFinal != scratchArc.isFinal()) {
+        return false;
+      }
+
+      if (scratchArc.isLast()) {
+        if (arcUpto == node.numArcs-1) {
+          return true;
+        } else {
+          return false;
+        }
+      }
+      fst.readNextRealArc(scratchArc);
+    }
+
+    return false;
+  }
+
+  // hash code for an unfrozen node.  This must be identical
+  // to the un-frozen case (below)!!
+  private int hash(Builder.UnCompiledNode<T> node) {
+    final int PRIME = 31;
+    //System.out.println("hash unfrozen");
+    int h = 0;
+    // 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);
+      h = PRIME * h + arc.label;
+      h = PRIME * h + ((Builder.CompiledNode) arc.target).address;
+      h = PRIME * h + arc.output.hashCode();
+      h = PRIME * h + arc.nextFinalOutput.hashCode();
+      if (arc.isFinal) {
+        h += 17;
+      }
+    }
+    //System.out.println("  ret " + (h&Integer.MAX_VALUE));
+    return h & Integer.MAX_VALUE;
+  }
+
+  // hash code for a frozen node
+  private int hash(int node) throws IOException {
+    final int PRIME = 31;
+    //System.out.println("hash frozen");
+    int h = 0;
+    fst.readFirstRealArc(node, scratchArc);
+    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;
+      h = PRIME * h + scratchArc.target;
+      h = PRIME * h + scratchArc.output.hashCode();
+      h = PRIME * h + scratchArc.nextFinalOutput.hashCode();
+      if (scratchArc.isFinal()) {
+        h += 17;
+      }
+      if (scratchArc.isLast()) {
+        break;
+      }
+      fst.readNextRealArc(scratchArc);
+    }
+    //System.out.println("  ret " + (h&Integer.MAX_VALUE));
+    return h & Integer.MAX_VALUE;
+  }
+
+  public int add(Builder.UnCompiledNode<T> node) throws IOException {
+    // System.out.println("hash: add count=" + count + " vs " + table.length);
+    final int h = hash(node);
+    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;
+        count++;
+        table[pos] = address;
+        if (table.length < 2*count) {
+          rehash();
+        }
+        return address;
+      } else if (nodesEqual(node, v)) {
+        // same node is already here
+        return v;
+      }
+
+      // quadratic probe
+      pos = (pos + (++c)) & mask;
+    }
+  }
+
+  // called only by rehash
+  private void addNew(int address) throws IOException {
+    int pos = hash(address) & mask;
+    int c = 0;
+    while(true) {
+      if (table[pos] == 0) {
+        table[pos] = address;
+        break;
+      }
+
+      // quadratic probe
+      pos = (pos + (++c)) & mask;
+    }
+  }
+
+  private void rehash() throws IOException {
+    final int[] oldTable = table;
+    table = new int[2*table.length];
+    mask = table.length-1;
+    for(int idx=0;idx<oldTable.length;idx++) {
+      final int address = oldTable[idx];
+      if (address != 0) {
+        addNew(address);
+      }
+    }
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Outputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Outputs.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Outputs.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Outputs.java Sun May 29 12:30:14 2011
@@ -0,0 +1,62 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * Represents the outputs for an FST, providing the basic
+ * algebra needed for the FST.
+ *
+ * @lucene.experimental
+ */
+
+public abstract class Outputs<T> {
+
+  // TODO: maybe change this API to allow for re-use of the
+  // output instances -- this is an insane amount of garbage
+  // (new object per byte/char/int) if eg used during
+  // analysis
+
+  /** Eg common("foo", "foobar") -> "foo" */
+  public abstract T common(T output1, T output2);
+
+  /** Eg subtract("foobar", "foo") -> "bar" */
+  public abstract T subtract(T output, T inc);
+
+  /** Eg add("foo", "bar") -> "foobar" */
+  public abstract T add(T prefix, T output);
+
+  public abstract void write(T output, DataOutput out) throws IOException;
+
+  public abstract T read(DataInput in) throws IOException;
+
+  /** NOTE: this output is compared with == so you must
+   *  ensure that all methods return the single object if
+   *  it's really no output */
+  public abstract T getNoOutput();
+
+  public abstract String outputToString(T output);
+
+  public T merge(T first, T second) {
+    throw new UnsupportedOperationException();
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java Sun May 29 12:30:14 2011
@@ -0,0 +1,118 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * Pairs up two outputs into one.
+ *
+ * @lucene.experimental
+ */
+
+public class PairOutputs<A,B> extends Outputs<PairOutputs.Pair<A,B>> {
+
+  private final Pair<A,B> NO_OUTPUT;
+  private final Outputs<A> outputs1;
+  private final Outputs<B> outputs2;
+
+  public static class Pair<A,B> {
+    public final A output1;
+    public final B output2;
+
+    public Pair(A output1, B output2) {
+      this.output1 = output1;
+      this.output2 = output2;
+    }
+
+    @Override @SuppressWarnings("rawtypes")
+    public boolean equals(Object other) {
+      if (other == this) {
+        return true;
+      } else if (other instanceof Pair) {
+        Pair pair = (Pair) other;
+        return output1.equals(pair.output1) && output2.equals(pair.output2);
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      return output1.hashCode() + output2.hashCode();
+    }
+  };
+
+  public PairOutputs(Outputs<A> outputs1, Outputs<B> outputs2) {
+    this.outputs1 = outputs1;
+    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()) {
+      return NO_OUTPUT;
+    } else {
+      return new Pair<A,B>(output1, output2);
+    }
+  }
+ 
+  @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));
+  }
+
+  @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));
+  }
+
+  @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));
+  }
+
+  @Override
+  public void write(Pair<A,B> output, DataOutput writer) throws IOException {
+    outputs1.write(output.output1, writer);
+    outputs2.write(output.output2, writer);
+  }
+
+  @Override
+  public Pair<A,B> read(DataInput in) throws IOException {
+    A output1 = outputs1.read(in);
+    B output2 = outputs2.read(in);
+    return get(output1, output2);
+  }
+
+  @Override
+  public Pair<A,B> getNoOutput() {
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public String outputToString(Pair<A,B> output) {
+    return "<pair:" + outputs1.outputToString(output.output1) + "," + outputs2.outputToString(output.output2) + ">";
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java Sun May 29 12:30:14 2011
@@ -0,0 +1,136 @@
+package org.apache.lucene.util.fst;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * Output is a long, for each input term.  NOTE: the
+ * resulting FST is not guaranteed to be minimal!  See
+ * {@link Builder}.  You cannot store 0 output with this
+ * (that's reserved to mean "no output")!
+ *
+ * @lucene.experimental
+ */
+
+public final class PositiveIntOutputs extends Outputs<Long> {
+  
+  private final static Long NO_OUTPUT = new Long(0);
+
+  private final boolean doShare;
+
+  private final static PositiveIntOutputs singletonShare = new PositiveIntOutputs(true);
+  private final static PositiveIntOutputs singletonNoShare = new PositiveIntOutputs(false);
+
+  private PositiveIntOutputs(boolean doShare) {
+    this.doShare = doShare;
+  }
+
+  public static PositiveIntOutputs getSingleton(boolean doShare) {
+    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);
+    assert valid(output2);
+    if (output1 == NO_OUTPUT || output2 == NO_OUTPUT) {
+      return NO_OUTPUT;
+    } else if (doShare) {
+      assert output1 > 0;
+      assert output2 > 0;
+      return Math.min(output1, output2);
+    } else if (output1.equals(output2)) {
+      return output1;
+    } else {
+      return NO_OUTPUT;
+    }
+  }
+
+  @Override
+  public Long subtract(Long output, Long inc) {
+    assert valid(output);
+    assert valid(inc);
+    assert output >= inc;
+
+    if (inc == NO_OUTPUT) {
+      return output;
+    } else if (output.equals(inc)) {
+      return NO_OUTPUT;
+    } else {
+      return output - inc;
+    }
+  }
+
+  @Override
+  public Long add(Long prefix, Long output) {
+    assert valid(prefix);
+    assert valid(output);
+    if (prefix == NO_OUTPUT) {
+      return output;
+    } else if (output == NO_OUTPUT) {
+      return prefix;
+    } else {
+      return prefix + output;
+    }
+  }
+
+  @Override
+  public void write(Long output, DataOutput out) throws IOException {
+    assert valid(output);
+    out.writeVLong(output);
+  }
+
+  @Override
+  public Long read(DataInput in) throws IOException {
+    long v = in.readVLong();
+    if (v == 0) {
+      return NO_OUTPUT;
+    } else {
+      return v;
+    }
+  }
+
+  private boolean valid(Long o) {
+    assert o != null;
+    assert o instanceof Long;
+    assert o == NO_OUTPUT || o > 0;
+    return true;
+  }
+
+  @Override
+  public Long getNoOutput() {
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public String outputToString(Long output) {
+    return output.toString();
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/TODO
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/TODO?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/TODO (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/TODO Sun May 29 12:30:14 2011
@@ -0,0 +1,39 @@
+is threadlocal.get costly?  if so maybe make an FSTReader?  would hold this "relative" pos, and each thread'd use it for reading, instead of PosRef
+
+maybe changed Outputs class to "reuse" stuff?  eg this new BytesRef in ByteSequenceOutputs..
+
+do i even "need" both non_final_end_state and final_end_state?
+
+hmm -- can I get weights working here?
+
+can FST be used to index all internal substrings, mapping to term?
+  - maybe put back ability to add multiple outputs per input...?
+
+make this work w/ char...?
+  - then FSTCharFilter/FSTTokenFilter
+  - syn filter?
+
+experiment: try reversing terms before compressing -- how much smaller?
+
+maybe seprate out a 'writable/growing fst' from a read-only one?
+
+can we somehow [partially] tableize lookups like oal.util.automaton?
+
+make an FST terms index option for codecs...?
+
+make an FSTCharsMap?
+
+need a benchmark testing FST traversal -- just fix the static main to rewind & visit all terms
+
+thread state
+
+when writing FST to disk:
+- Sequentially writing (would save memory in codec during indexing). We are now using DataOutput, which could also go directly to disk
+- problem: size of BytesRef must be known before
+
+later
+  - maybe don't require FSTEnum.advance to be forward only?
+  - should i make a posIntOutputs separate from posLongOutputs?
+  - mv randomAccpetedWord / run / etc. from test into FST?
+  - hmm get multi-outputs working again?  do we ever need this?
+