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 [1/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...

Author: mikemccand
Date: Sun May 29 12:30:14 2011
New Revision: 1128870

URL: http://svn.apache.org/viewvc?rev=1128870&view=rev
Log:
LUCENE-3140: port FST impl to 3.x

Added:
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataInput.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataOutput.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/BytesRef.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/CodecUtil.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/IntsRef.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FST.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/IntsRefFSTEnum.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NoOutputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Outputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/TODO
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Util.java   (with props)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/package.html   (with props)
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java   (with props)
Removed:
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/store/TestCopyBytes.java
Modified:
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexInput.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexOutput.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java
    lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/store/MockIndexOutputWrapper.java
    lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1128870&r1=1128869&r2=1128870&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Sun May 29 12:30:14 2011
@@ -2,6 +2,12 @@ Lucene Change Log
 
 ======================= Lucene 3.x (not yet released) ================
 
+Changes in backwards compatibility policy
+
+* LUCENE-3140: IndexOutput.copyBytes now takes a DataInput (superclass
+  of IndexInput) as its first argument.  (Robert Muir, Dawid Weiss,
+  Mike McCandless)
+
 Changes in runtime behavior
 
 * LUCENE-2834: the hash used to compute the lock file name when the
@@ -9,6 +15,12 @@ Changes in runtime behavior
   will see a different lucene-XXX-write.lock in your lock directory.
   (Robert Muir, Uwe Schindler, Mike McCandless)
 
+New Features
+
+* LUCENE-3140: Added experimental FST implementation to Lucene.
+  (Robert Muir, Dawid Weiss, Mike McCandless)
+
+
 ======================= Lucene 3.2.0 =======================
 
 Changes in backwards compatibility policy
@@ -67,7 +79,7 @@ New features
 * LUCENE-1421, LUCENE-3102: added CachingCollector which allow you to cache 
   document IDs and scores encountered during the search, and "replay" them to 
   another Collector. (Mike McCandless, Shai Erera)
-  
+
 * LUCENE-3112: Added experimental IndexWriter.add/updateDocuments,
   enabling a block of documents to be indexed, atomically, with
   guaranteed sequential docIDs.  (Mike McCandless)

Modified: lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1128870&r1=1128869&r2=1128870&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Sun May 29 12:30:14 2011
@@ -929,45 +929,6 @@ public class TestIndexWriterExceptions e
     dir.close();
   }
   
-  // LUCENE-1044: Simulate checksum error in segments_N
-  public void testSegmentsChecksumError() throws IOException {
-    Directory dir = newDirectory();
-
-    IndexWriter writer = null;
-
-    writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
-
-    // add 100 documents
-    for (int i = 0; i < 100; i++) {
-      addDoc(writer);
-    }
-
-    // close
-    writer.close();
-
-    long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
-    assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
-
-    final String segmentsFileName = SegmentInfos.getCurrentSegmentFileName(dir);
-    IndexInput in = dir.openInput(segmentsFileName);
-    IndexOutput out = dir.createOutput(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", 1+gen));
-    out.copyBytes(in, in.length()-1);
-    byte b = in.readByte();
-    out.writeByte((byte) (1+b));
-    out.close();
-    in.close();
-
-    IndexReader reader = null;
-    try {
-      reader = IndexReader.open(dir, true);
-    } catch (IOException e) {
-      e.printStackTrace(System.out);
-      fail("segmentInfos failed to retry fallback to correct segments_N file");
-    }
-    reader.close();
-    dir.close();
-  }
-  
   // Simulate a corrupt index by removing last byte of
   // latest segments file and make sure we get an
   // IOException trying to open the index:

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java Sun May 29 12:30:14 2011
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+/**
+ * This exception is thrown when Lucene detects
+ * an index that is newer than this Lucene version.
+ */
+public class IndexFormatTooNewException extends CorruptIndexException {
+
+  public IndexFormatTooNewException(String filename, int version, int minVersion, int maxVersion) {
+    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
+      ": " + version + " (needs to be between " + minVersion + " and " + maxVersion + ")");
+  }
+
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java Sun May 29 12:30:14 2011
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+/**
+ * This exception is thrown when Lucene detects
+ * an index that is too old for this Lucene version
+ */
+public class IndexFormatTooOldException extends CorruptIndexException {
+
+  public IndexFormatTooOldException(String filename, String version) {
+    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
+        ": " + version + ". This version of Lucene only supports indexes created with release 3.0 and later.");
+  }
+  
+  public IndexFormatTooOldException(String filename, int version, int minVersion, int maxVersion) {
+    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
+        ": " + version + " (needs to be between " + minVersion + " and " + maxVersion +
+    "). This version of Lucene only supports indexes created with release 3.0 and later.");
+  }
+
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataInput.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataInput.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataInput.java Sun May 29 12:30:14 2011
@@ -0,0 +1,251 @@
+package org.apache.lucene.store;
+
+/**
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Abstract base class for performing read operations of Lucene's low-level
+ * data types.
+ */
+public abstract class DataInput implements Cloneable {
+
+  private boolean preUTF8Strings;                 // true if we are reading old (modified UTF8) string format
+
+  /** Call this if readString should read characters stored
+   *  in the old modified UTF8 format (length in java chars
+   *  and java's modified UTF8 encoding).  This is used for
+   *  indices written pre-2.4 See LUCENE-510 for details. */
+  public void setModifiedUTF8StringsMode() {
+    preUTF8Strings = true;
+  }
+
+  /** Reads and returns a single byte.
+   * @see DataOutput#writeByte(byte)
+   */
+  public abstract byte readByte() throws IOException;
+
+  /** Reads a specified number of bytes into an array at the specified offset.
+   * @param b the array to read bytes into
+   * @param offset the offset in the array to start storing bytes
+   * @param len the number of bytes to read
+   * @see DataOutput#writeBytes(byte[],int)
+   */
+  public abstract void readBytes(byte[] b, int offset, int len)
+    throws IOException;
+
+  /** Reads a specified number of bytes into an array at the
+   * specified offset with control over whether the read
+   * should be buffered (callers who have their own buffer
+   * should pass in "false" for useBuffer).  Currently only
+   * {@link BufferedIndexInput} respects this parameter.
+   * @param b the array to read bytes into
+   * @param offset the offset in the array to start storing bytes
+   * @param len the number of bytes to read
+   * @param useBuffer set to false if the caller will handle
+   * buffering.
+   * @see DataOutput#writeBytes(byte[],int)
+   */
+  public void readBytes(byte[] b, int offset, int len, boolean useBuffer)
+    throws IOException
+  {
+    // Default to ignoring useBuffer entirely
+    readBytes(b, offset, len);
+  }
+
+  /** Reads two bytes and returns a short.
+   * @see DataOutput#writeByte(byte)
+   */
+  public short readShort() throws IOException {
+    return (short) (((readByte() & 0xFF) <<  8) |  (readByte() & 0xFF));
+  }
+
+  /** Reads four bytes and returns an int.
+   * @see DataOutput#writeInt(int)
+   */
+  public int readInt() throws IOException {
+    return ((readByte() & 0xFF) << 24) | ((readByte() & 0xFF) << 16)
+         | ((readByte() & 0xFF) <<  8) |  (readByte() & 0xFF);
+  }
+
+  /** Reads an int stored in variable-length format.  Reads between one and
+   * five bytes.  Smaller values take fewer bytes.  Negative numbers are not
+   * supported.
+   * @see DataOutput#writeVInt(int)
+   */
+  public int readVInt() throws IOException {
+    /* This is the original code of this method,
+     * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
+     * readByte() is inlined. So the loop was unwinded!
+    byte b = readByte();
+    int i = b & 0x7F;
+    for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+      b = readByte();
+      i |= (b & 0x7F) << shift;
+    }
+    return i;
+    */
+    byte b = readByte();
+    int i = b & 0x7F;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7F) << 7;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7F) << 14;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7F) << 21;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    assert (b & 0x80) == 0;
+    return i | ((b & 0x7F) << 28);
+  }
+
+  /** Reads eight bytes and returns a long.
+   * @see DataOutput#writeLong(long)
+   */
+  public long readLong() throws IOException {
+    return (((long)readInt()) << 32) | (readInt() & 0xFFFFFFFFL);
+  }
+
+  /** Reads a long stored in variable-length format.  Reads between one and
+   * nine bytes.  Smaller values take fewer bytes.  Negative numbers are not
+   * supported. */
+  public long readVLong() throws IOException {
+    /* This is the original code of this method,
+     * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
+     * readByte() is inlined. So the loop was unwinded!
+    byte b = readByte();
+    long i = b & 0x7F;
+    for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+      b = readByte();
+      i |= (b & 0x7FL) << shift;
+    }
+    return i;
+    */
+    byte b = readByte();
+    long i = b & 0x7FL;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 7;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 14;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 21;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 28;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 35;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 42;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 49;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    assert (b & 0x80) == 0;
+    return i | ((b & 0x7FL) << 56);
+  }
+
+  /** Reads a string.
+   * @see DataOutput#writeString(String)
+   */
+  public String readString() throws IOException {
+    if (preUTF8Strings)
+      return readModifiedUTF8String();
+    int length = readVInt();
+    final byte[] bytes = new byte[length];
+    readBytes(bytes, 0, length);
+    return new String(bytes, 0, length, "UTF-8");
+  }
+
+  private String readModifiedUTF8String() throws IOException {
+    int length = readVInt();
+    final char[] chars = new char[length];
+    readChars(chars, 0, length);
+    return new String(chars, 0, length);
+  }
+
+  /** Reads Lucene's old "modified UTF-8" encoded
+   *  characters into an array.
+   * @param buffer the array to read characters into
+   * @param start the offset in the array to start storing characters
+   * @param length the number of characters to read
+   * @see DataOutput#writeChars(String,int,int)
+   * @deprecated -- please use readString or readBytes
+   *                instead, and construct the string
+   *                from those utf8 bytes
+   */
+  @Deprecated
+  public void readChars(char[] buffer, int start, int length)
+       throws IOException {
+    final int end = start + length;
+    for (int i = start; i < end; i++) {
+      byte b = readByte();
+      if ((b & 0x80) == 0)
+	buffer[i] = (char)(b & 0x7F);
+      else if ((b & 0xE0) != 0xE0) {
+	buffer[i] = (char)(((b & 0x1F) << 6)
+		 | (readByte() & 0x3F));
+      } else {
+	buffer[i] = (char)(((b & 0x0F) << 12)
+		| ((readByte() & 0x3F) << 6)
+	        |  (readByte() & 0x3F));
+      }
+    }
+  }
+
+  /** Returns a clone of this stream.
+   *
+   * <p>Clones of a stream access the same data, and are positioned at the same
+   * point as the stream they were cloned from.
+   *
+   * <p>Expert: Subclasses must ensure that clones may be positioned at
+   * different points in the input from each other and from the stream they
+   * were cloned from.
+   */
+  @Override
+  public Object clone() {
+    DataInput clone = null;
+    try {
+      clone = (DataInput)super.clone();
+    } catch (CloneNotSupportedException e) {}
+
+    return clone;
+  }
+
+  public Map<String,String> readStringStringMap() throws IOException {
+    final Map<String,String> map = new HashMap<String,String>();
+    final int count = readInt();
+    for(int i=0;i<count;i++) {
+      final String key = readString();
+      final String val = readString();
+      map.put(key, val);
+    }
+
+    return map;
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataOutput.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataOutput.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/DataOutput.java Sun May 29 12:30:14 2011
@@ -0,0 +1,193 @@
+package org.apache.lucene.store;
+
+/**
+ * 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 java.util.Map;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.UnicodeUtil;
+
+/**
+ * Abstract base class for performing write operations of Lucene's low-level
+ * data types.
+ */
+public abstract class DataOutput {
+
+  /** Writes a single byte.
+   * @see IndexInput#readByte()
+   */
+  public abstract void writeByte(byte b) throws IOException;
+
+  /** Writes an array of bytes.
+   * @param b the bytes to write
+   * @param length the number of bytes to write
+   * @see DataInput#readBytes(byte[],int,int)
+   */
+  public void writeBytes(byte[] b, int length) throws IOException {
+    writeBytes(b, 0, length);
+  }
+
+  /** Writes an array of bytes.
+   * @param b the bytes to write
+   * @param offset the offset in the byte array
+   * @param length the number of bytes to write
+   * @see DataInput#readBytes(byte[],int,int)
+   */
+  public abstract void writeBytes(byte[] b, int offset, int length) throws IOException;
+
+  /** Writes an int as four bytes.
+   * @see DataInput#readInt()
+   */
+  public void writeInt(int i) throws IOException {
+    writeByte((byte)(i >> 24));
+    writeByte((byte)(i >> 16));
+    writeByte((byte)(i >>  8));
+    writeByte((byte) i);
+  }
+
+  /** Writes an int in a variable-length format.  Writes between one and
+   * five bytes.  Smaller values take fewer bytes.  Negative numbers are not
+   * supported.
+   * @see DataInput#readVInt()
+   */
+  public final void writeVInt(int i) throws IOException {
+    while ((i & ~0x7F) != 0) {
+      writeByte((byte)((i & 0x7f) | 0x80));
+      i >>>= 7;
+    }
+    writeByte((byte)i);
+  }
+
+  /** Writes a long as eight bytes.
+   * @see DataInput#readLong()
+   */
+  public void writeLong(long i) throws IOException {
+    writeInt((int) (i >> 32));
+    writeInt((int) i);
+  }
+
+  /** Writes an long in a variable-length format.  Writes between one and nine
+   * bytes.  Smaller values take fewer bytes.  Negative numbers are not
+   * supported.
+   * @see DataInput#readVLong()
+   */
+  public final void writeVLong(long i) throws IOException {
+    while ((i & ~0x7F) != 0) {
+      writeByte((byte)((i & 0x7f) | 0x80));
+      i >>>= 7;
+    }
+    writeByte((byte)i);
+  }
+
+  /** Writes a string.
+   * @see DataInput#readString()
+   */
+  public void writeString(String s) throws IOException {
+    final BytesRef utf8Result = new BytesRef(10);
+    UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8Result);
+    writeVInt(utf8Result.length);
+    writeBytes(utf8Result.bytes, 0, utf8Result.length);
+  }
+
+  private static int COPY_BUFFER_SIZE = 16384;
+  private byte[] copyBuffer;
+
+  /** Copy numBytes bytes from input to ourself. */
+  public void copyBytes(DataInput input, long numBytes) throws IOException {
+    assert numBytes >= 0: "numBytes=" + numBytes;
+    long left = numBytes;
+    if (copyBuffer == null)
+      copyBuffer = new byte[COPY_BUFFER_SIZE];
+    while(left > 0) {
+      final int toCopy;
+      if (left > COPY_BUFFER_SIZE)
+        toCopy = COPY_BUFFER_SIZE;
+      else
+        toCopy = (int) left;
+      input.readBytes(copyBuffer, 0, toCopy);
+      writeBytes(copyBuffer, 0, toCopy);
+      left -= toCopy;
+    }
+  }
+
+  /** Writes a sub sequence of characters from s as the old
+   *  format (modified UTF-8 encoded bytes).
+   * @param s the source of the characters
+   * @param start the first character in the sequence
+   * @param length the number of characters in the sequence
+   * @deprecated -- please pre-convert to utf8 bytes
+   * instead or use {@link #writeString}
+   */
+  @Deprecated
+  public void writeChars(String s, int start, int length)
+       throws IOException {
+    final int end = start + length;
+    for (int i = start; i < end; i++) {
+      final int code = s.charAt(i);
+      if (code >= 0x01 && code <= 0x7F)
+	writeByte((byte)code);
+      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
+	writeByte((byte)(0xC0 | (code >> 6)));
+	writeByte((byte)(0x80 | (code & 0x3F)));
+      } else {
+	writeByte((byte)(0xE0 | (code >>> 12)));
+	writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
+	writeByte((byte)(0x80 | (code & 0x3F)));
+      }
+    }
+  }
+
+  /** Writes a sub sequence of characters from char[] as
+   *  the old format (modified UTF-8 encoded bytes).
+   * @param s the source of the characters
+   * @param start the first character in the sequence
+   * @param length the number of characters in the sequence
+   * @deprecated -- please pre-convert to utf8 bytes instead or use {@link #writeString}
+   */
+  @Deprecated
+  public void writeChars(char[] s, int start, int length)
+    throws IOException {
+    final int end = start + length;
+    for (int i = start; i < end; i++) {
+      final int code = s[i];
+      if (code >= 0x01 && code <= 0x7F)
+	writeByte((byte)code);
+      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
+	writeByte((byte)(0xC0 | (code >> 6)));
+	writeByte((byte)(0x80 | (code & 0x3F)));
+      } else {
+	writeByte((byte)(0xE0 | (code >>> 12)));
+	writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
+	writeByte((byte)(0x80 | (code & 0x3F)));
+      }
+    }
+  }
+
+  public void writeStringStringMap(Map<String,String> map) throws IOException {
+    if (map == null) {
+      writeInt(0);
+    } else {
+      writeInt(map.size());
+      for(final Map.Entry<String, String> entry: map.entrySet()) {
+        writeString(entry.getKey());
+        writeString(entry.getValue());
+      }
+    }
+  }
+}

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexInput.java?rev=1128870&r1=1128869&r2=1128870&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexInput.java Sun May 29 12:30:14 2011
@@ -19,200 +19,15 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.io.Closeable;
-import java.util.Map;
-import java.util.HashMap;
 
 /** Abstract base class for input from a file in a {@link Directory}.  A
  * random-access input stream.  Used for all Lucene index input operations.
  * @see Directory
  */
-public abstract class IndexInput implements Cloneable,Closeable {
-
-  private boolean preUTF8Strings;                 // true if we are reading old (modified UTF8) string format
+public abstract class IndexInput extends DataInput implements Cloneable,Closeable {
 
   protected byte[] copyBuf = null;
 
-  /** Reads and returns a single byte.
-   * @see IndexOutput#writeByte(byte)
-   */
-  public abstract byte readByte() throws IOException;
-
-  /** Reads a specified number of bytes into an array at the specified offset.
-   * @param b the array to read bytes into
-   * @param offset the offset in the array to start storing bytes
-   * @param len the number of bytes to read
-   * @see IndexOutput#writeBytes(byte[],int)
-   */
-  public abstract void readBytes(byte[] b, int offset, int len)
-    throws IOException;
-
-  /** Reads a specified number of bytes into an array at the
-   * specified offset with control over whether the read
-   * should be buffered (callers who have their own buffer
-   * should pass in "false" for useBuffer).  Currently only
-   * {@link BufferedIndexInput} respects this parameter.
-   * @param b the array to read bytes into
-   * @param offset the offset in the array to start storing bytes
-   * @param len the number of bytes to read
-   * @param useBuffer set to false if the caller will handle
-   * buffering.
-   * @see IndexOutput#writeBytes(byte[],int)
-   */
-  public void readBytes(byte[] b, int offset, int len, boolean useBuffer)
-    throws IOException {
-    // Default to ignoring useBuffer entirely
-    readBytes(b, offset, len);
-  }
-
-  /** Reads four bytes and returns an int.
-   * @see IndexOutput#writeInt(int)
-   */
-  public int readInt() throws IOException {
-    return ((readByte() & 0xFF) << 24) | ((readByte() & 0xFF) << 16)
-         | ((readByte() & 0xFF) <<  8) |  (readByte() & 0xFF);
-  }
-
-  /** Reads an int stored in variable-length format.  Reads between one and
-   * five bytes.  Smaller values take fewer bytes.  Negative numbers are not
-   * supported.
-   * @see IndexOutput#writeVInt(int)
-   */
-  public int readVInt() throws IOException {
-    /* This is the original code of this method,
-     * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
-     * readByte() is inlined. So the loop was unwinded!
-    byte b = readByte();
-    int i = b & 0x7F;
-    for (int shift = 7; (b & 0x80) != 0; shift += 7) {
-      b = readByte();
-      i |= (b & 0x7F) << shift;
-    }
-    return i;
-    */
-    byte b = readByte();
-    int i = b & 0x7F;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7F) << 7;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7F) << 14;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7F) << 21;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    assert (b & 0x80) == 0;
-    return i | ((b & 0x7F) << 28);
-  }
-
-  /** Reads eight bytes and returns a long.
-   * @see IndexOutput#writeLong(long)
-   */
-  public long readLong() throws IOException {
-    return (((long)readInt()) << 32) | (readInt() & 0xFFFFFFFFL);
-  }
-
-  /** Reads a long stored in variable-length format.  Reads between one and
-   * nine bytes.  Smaller values take fewer bytes.  Negative numbers are not
-   * supported. */
-  public long readVLong() throws IOException {
-    /* This is the original code of this method,
-     * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
-     * readByte() is inlined. So the loop was unwinded!
-    byte b = readByte();
-    long i = b & 0x7F;
-    for (int shift = 7; (b & 0x80) != 0; shift += 7) {
-      b = readByte();
-      i |= (b & 0x7FL) << shift;
-    }
-    return i;
-    */
-    byte b = readByte();
-    long i = b & 0x7FL;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 7;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 14;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 21;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 28;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 35;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 42;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    i |= (b & 0x7FL) << 49;
-    if ((b & 0x80) == 0) return i;
-    b = readByte();
-    assert (b & 0x80) == 0;
-    return i | ((b & 0x7FL) << 56);
-  }
-
-  /** Call this if readString should read characters stored
-   *  in the old modified UTF8 format (length in java chars
-   *  and java's modified UTF8 encoding).  This is used for
-   *  indices written pre-2.4 See LUCENE-510 for details. */
-  public void setModifiedUTF8StringsMode() {
-    preUTF8Strings = true;
-  }
-
-  /** Reads a string.
-   * @see IndexOutput#writeString(String)
-   */
-  public String readString() throws IOException {
-    if (preUTF8Strings)
-      return readModifiedUTF8String();
-    int length = readVInt();
-    final byte[] bytes = new byte[length];
-    readBytes(bytes, 0, length);
-    return new String(bytes, 0, length, "UTF-8");
-  }
-
-  private String readModifiedUTF8String() throws IOException {
-    int length = readVInt();
-    final char[] chars = new char[length];
-    readChars(chars, 0, length);
-    return new String(chars, 0, length);
-  }
-
-  /** Reads Lucene's old "modified UTF-8" encoded
-   *  characters into an array.
-   * @param buffer the array to read characters into
-   * @param start the offset in the array to start storing characters
-   * @param length the number of characters to read
-   * @see IndexOutput#writeChars(String,int,int)
-   * @deprecated -- please use readString or readBytes
-   *                instead, and construct the string
-   *                from those utf8 bytes
-   */
-  @Deprecated
-  public void readChars(char[] buffer, int start, int length)
-       throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      byte b = readByte();
-      if ((b & 0x80) == 0)
-	buffer[i] = (char)(b & 0x7F);
-      else if ((b & 0xE0) != 0xE0) {
-	buffer[i] = (char)(((b & 0x1F) << 6)
-		 | (readByte() & 0x3F));
-      } else {
-	buffer[i] = (char)(((b & 0x0F) << 12)
-		| ((readByte() & 0x3F) << 6)
-	        |  (readByte() & 0x3F));
-      }
-    }
-  }
-
   /**
    * Expert
    * 
@@ -239,7 +54,6 @@ public abstract class IndexInput impleme
       }
     }
   }
-  
 
   /** Closes the stream to further operations. */
   public abstract void close() throws IOException;
@@ -258,37 +72,6 @@ public abstract class IndexInput impleme
   /** The number of bytes in the file. */
   public abstract long length();
 
-  /** Returns a clone of this stream.
-   *
-   * <p>Clones of a stream access the same data, and are positioned at the same
-   * point as the stream they were cloned from.
-   *
-   * <p>Expert: Subclasses must ensure that clones may be positioned at
-   * different points in the input from each other and from the stream they
-   * were cloned from.
-   */
-  @Override
-  public Object clone() {
-    IndexInput clone = null;
-    try {
-      clone = (IndexInput)super.clone();
-    } catch (CloneNotSupportedException e) {}
-
-    return clone;
-  }
-
-  public Map<String,String> readStringStringMap() throws IOException {
-    final Map<String,String> map = new HashMap<String,String>();
-    final int count = readInt();
-    for(int i=0;i<count;i++) {
-      final String key = readString();
-      final String val = readString();
-      map.put(key, val);
-    }
-
-    return map;
-  }
-
   /**
    * Copies <code>numBytes</code> bytes to the given {@link IndexOutput}.
    * <p>

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexOutput.java?rev=1128870&r1=1128869&r2=1128870&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/IndexOutput.java Sun May 29 12:30:14 2011
@@ -19,165 +19,13 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.io.Closeable;
-import java.util.Map;
-import org.apache.lucene.util.UnicodeUtil;
 
 /** Abstract base class for output to a file in a Directory.  A random-access
  * output stream.  Used for all Lucene index output operations.
  * @see Directory
  * @see IndexInput
  */
-public abstract class IndexOutput implements Closeable {
-
-  /** Writes a single byte.
-   * @see IndexInput#readByte()
-   */
-  public abstract void writeByte(byte b) throws IOException;
-
-  /** Writes an array of bytes.
-   * @param b the bytes to write
-   * @param length the number of bytes to write
-   * @see IndexInput#readBytes(byte[],int,int)
-   */
-  public void writeBytes(byte[] b, int length) throws IOException {
-    writeBytes(b, 0, length);
-  }
-
-  /** Writes an array of bytes.
-   * @param b the bytes to write
-   * @param offset the offset in the byte array
-   * @param length the number of bytes to write
-   * @see IndexInput#readBytes(byte[],int,int)
-   */
-  public abstract void writeBytes(byte[] b, int offset, int length) throws IOException;
-
-  /** Writes an int as four bytes.
-   * @see IndexInput#readInt()
-   */
-  public void writeInt(int i) throws IOException {
-    writeByte((byte)(i >> 24));
-    writeByte((byte)(i >> 16));
-    writeByte((byte)(i >>  8));
-    writeByte((byte) i);
-  }
-
-  /** Writes an int in a variable-length format.  Writes between one and
-   * five bytes.  Smaller values take fewer bytes.  Negative numbers are not
-   * supported.
-   * @see IndexInput#readVInt()
-   */
-  public void writeVInt(int i) throws IOException {
-    while ((i & ~0x7F) != 0) {
-      writeByte((byte)((i & 0x7f) | 0x80));
-      i >>>= 7;
-    }
-    writeByte((byte)i);
-  }
-
-  /** Writes a long as eight bytes.
-   * @see IndexInput#readLong()
-   */
-  public void writeLong(long i) throws IOException {
-    writeInt((int) (i >> 32));
-    writeInt((int) i);
-  }
-
-  /** Writes an long in a variable-length format.  Writes between one and nine
-   * bytes.  Smaller values take fewer bytes.  Negative numbers are not
-   * supported.
-   * @see IndexInput#readVLong()
-   */
-  public void writeVLong(long i) throws IOException {
-    while ((i & ~0x7F) != 0) {
-      writeByte((byte)((i & 0x7f) | 0x80));
-      i >>>= 7;
-    }
-    writeByte((byte)i);
-  }
-
-  /** Writes a string.
-   * @see IndexInput#readString()
-   */
-  public void writeString(String s) throws IOException {
-    final UnicodeUtil.UTF8Result utf8Result = new UnicodeUtil.UTF8Result();
-    UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8Result);
-    writeVInt(utf8Result.length);
-    writeBytes(utf8Result.result, 0, utf8Result.length);
-  }
-
-  /** Writes a sub sequence of characters from s as the old
-   *  format (modified UTF-8 encoded bytes).
-   * @param s the source of the characters
-   * @param start the first character in the sequence
-   * @param length the number of characters in the sequence
-   * @deprecated -- please pre-convert to utf8 bytes
-   * instead or use {@link #writeString}
-   */
-  @Deprecated
-  public void writeChars(String s, int start, int length)
-       throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      final int code = s.charAt(i);
-      if (code >= 0x01 && code <= 0x7F)
-	writeByte((byte)code);
-      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
-	writeByte((byte)(0xC0 | (code >> 6)));
-	writeByte((byte)(0x80 | (code & 0x3F)));
-      } else {
-	writeByte((byte)(0xE0 | (code >>> 12)));
-	writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
-	writeByte((byte)(0x80 | (code & 0x3F)));
-      }
-    }
-  }
-
-  /** Writes a sub sequence of characters from char[] as
-   *  the old format (modified UTF-8 encoded bytes).
-   * @param s the source of the characters
-   * @param start the first character in the sequence
-   * @param length the number of characters in the sequence
-   * @deprecated -- please pre-convert to utf8 bytes instead or use {@link #writeString}
-   */
-  @Deprecated
-  public void writeChars(char[] s, int start, int length)
-    throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      final int code = s[i];
-      if (code >= 0x01 && code <= 0x7F)
-	writeByte((byte)code);
-      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
-	writeByte((byte)(0xC0 | (code >> 6)));
-	writeByte((byte)(0x80 | (code & 0x3F)));
-      } else {
-	writeByte((byte)(0xE0 | (code >>> 12)));
-	writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
-	writeByte((byte)(0x80 | (code & 0x3F)));
-      }
-    }
-  }
-
-  private static int COPY_BUFFER_SIZE = 16384;
-  private byte[] copyBuffer;
-
-  /** Copy numBytes bytes from input to ourself. */
-  public void copyBytes(IndexInput input, long numBytes) throws IOException {
-    assert numBytes >= 0: "numBytes=" + numBytes;
-    long left = numBytes;
-    if (copyBuffer == null)
-      copyBuffer = new byte[COPY_BUFFER_SIZE];
-    while(left > 0) {
-      final int toCopy;
-      if (left > COPY_BUFFER_SIZE)
-        toCopy = COPY_BUFFER_SIZE;
-      else
-        toCopy = (int) left;
-      input.readBytes(copyBuffer, 0, toCopy);
-      writeBytes(copyBuffer, 0, toCopy);
-      left -= toCopy;
-    }
-  }
+public abstract class IndexOutput extends DataOutput implements Closeable {
 
   /** Forces any buffered output to be written. */
   public abstract void flush() throws IOException;
@@ -210,16 +58,4 @@ public abstract class IndexOutput implem
    * @param length file length
    */
   public void setLength(long length) throws IOException {}
-
-  public void writeStringStringMap(Map<String,String> map) throws IOException {
-    if (map == null) {
-      writeInt(0);
-    } else {
-      writeInt(map.size());
-      for(final Map.Entry<String, String> entry: map.entrySet()) {
-        writeString(entry.getKey());
-        writeString(entry.getValue());
-      }
-    }
-  }
 }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1128870&r1=1128869&r2=1128870&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java Sun May 29 12:30:14 2011
@@ -162,7 +162,7 @@ public class RAMOutputStream extends Ind
   }
   
   @Override
-  public void copyBytes(IndexInput input, long numBytes) throws IOException {
+  public void copyBytes(DataInput input, long numBytes) throws IOException {
     assert numBytes >= 0: "numBytes=" + numBytes;
 
     while (numBytes > 0) {

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/BytesRef.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/BytesRef.java Sun May 29 12:30:14 2011
@@ -0,0 +1,368 @@
+package org.apache.lucene.util;
+
+/**
+ * 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.util.Comparator;
+import java.io.UnsupportedEncodingException;
+
+/** Represents byte[], as a slice (offset + length) into an
+ *  existing byte[].
+ *
+ *  @lucene.experimental */
+public final class BytesRef implements Comparable<BytesRef> {
+
+  static final int HASH_PRIME = 31;
+  public static final byte[] EMPTY_BYTES = new byte[0]; 
+
+  /** The contents of the BytesRef. Should never be {@code null}. */
+  public byte[] bytes;
+
+  /** Offset of first valid byte. */
+  public int offset;
+
+  /** Length of used bytes. */
+  public int length;
+
+  public BytesRef() {
+    bytes = EMPTY_BYTES;
+  }
+
+  /** This instance will directly reference bytes w/o making a copy.
+   * bytes should not be null.
+   */
+  public BytesRef(byte[] bytes, int offset, int length) {
+    assert bytes != null;
+    this.bytes = bytes;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  /** This instance will directly reference bytes w/o making a copy.
+   * bytes should not be null */
+  public BytesRef(byte[] bytes) {
+    assert bytes != null;
+    this.bytes = bytes;
+    this.offset = 0;
+    this.length = bytes.length;
+  }
+
+  public BytesRef(int capacity) {
+    this.bytes = new byte[capacity];
+  }
+
+  /**
+   * @param text Initialize the byte[] from the UTF8 bytes
+   * for the provided Sring.  This must be well-formed
+   * unicode text, with no unpaired surrogates or U+FFFF.
+   */
+  public BytesRef(CharSequence text) {
+    this();
+    copy(text);
+  }
+  
+  /**
+   * @param text Initialize the byte[] from the UTF8 bytes
+   * for the provided array.  This must be well-formed
+   * unicode text, with no unpaired surrogates or U+FFFF.
+   */
+  public BytesRef(char text[], int offset, int length) {
+    this(length * 4);
+    copy(text, offset, length);
+  }
+
+  public BytesRef(BytesRef other) {
+    this();
+    copy(other);
+  }
+
+  /* // maybe?
+  public BytesRef(BytesRef other, boolean shallow) {
+    this();
+    if (shallow) {
+      offset = other.offset;
+      length = other.length;
+      bytes = other.bytes;
+    } else {
+      copy(other);
+    }
+  }
+  */
+
+  /**
+   * Copies the UTF8 bytes for this string.
+   * 
+   * @param text Must be well-formed unicode text, with no
+   * unpaired surrogates or invalid UTF16 code units.
+   */
+  public void copy(CharSequence text) {
+    UnicodeUtil.UTF16toUTF8(text, 0, text.length(), this);
+  }
+
+  /**
+   * Copies the UTF8 bytes for this string.
+   * 
+   * @param text Must be well-formed unicode text, with no
+   * unpaired surrogates or invalid UTF16 code units.
+   */
+  public void copy(char text[], int offset, int length) {
+    UnicodeUtil.UTF16toUTF8(text, offset, length, this);
+  }
+
+  public boolean bytesEquals(BytesRef other) {
+    if (length == other.length) {
+      int otherUpto = other.offset;
+      final byte[] otherBytes = other.bytes;
+      final int end = offset + length;
+      for(int upto=offset;upto<end;upto++,otherUpto++) {
+        if (bytes[upto] != otherBytes[otherUpto]) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Object clone() {
+    return new BytesRef(this);
+  }
+
+  private boolean sliceEquals(BytesRef other, int pos) {
+    if (pos < 0 || length - pos < other.length) {
+      return false;
+    }
+    int i = offset + pos;
+    int j = other.offset;
+    final int k = other.offset + other.length;
+    
+    while (j < k) {
+      if (bytes[i++] != other.bytes[j++]) {
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  public boolean startsWith(BytesRef other) {
+    return sliceEquals(other, 0);
+  }
+
+  public boolean endsWith(BytesRef other) {
+    return sliceEquals(other, length - other.length);
+  }
+  
+  /** Calculates the hash code as required by TermsHash during indexing.
+   * <p>It is defined as:
+   * <pre>
+   *  int hash = 0;
+   *  for (int i = offset; i &lt; offset + length; i++) {
+   *    hash = 31*hash + bytes[i];
+   *  }
+   * </pre>
+   */
+  @Override
+  public int hashCode() {
+    int result = 0;
+    final int end = offset + length;
+    for(int i=offset;i<end;i++) {
+      result = HASH_PRIME * result + bytes[i];
+    }
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    return this.bytesEquals((BytesRef) other);
+  }
+
+  /** Interprets stored bytes as UTF8 bytes, returning the
+   *  resulting string */
+  public String utf8ToString() {
+    try {
+      return new String(bytes, offset, length, "UTF-8");
+    } catch (UnsupportedEncodingException uee) {
+      // should not happen -- UTF8 is presumably supported
+      // by all JREs
+      throw new RuntimeException(uee);
+    }
+  }
+
+  /** Returns hex encoded bytes, eg [0x6c 0x75 0x63 0x65 0x6e 0x65] */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append('[');
+    final int end = offset + length;
+    for(int i=offset;i<end;i++) {
+      if (i > offset) {
+        sb.append(' ');
+      }
+      sb.append(Integer.toHexString(bytes[i]&0xff));
+    }
+    sb.append(']');
+    return sb.toString();
+  }
+
+  public void copy(BytesRef other) {
+    if (bytes.length < other.length) {
+      bytes = new byte[other.length];
+    }
+    System.arraycopy(other.bytes, other.offset, bytes, 0, other.length);
+    length = other.length;
+    offset = 0;
+  }
+
+  public void append(BytesRef other) {
+    int newLen = length + other.length;
+    if (bytes.length < newLen) {
+      byte[] newBytes = new byte[newLen];
+      System.arraycopy(bytes, offset, newBytes, 0, length);
+      offset = 0;
+      bytes = newBytes;
+    }
+    System.arraycopy(other.bytes, other.offset, bytes, length+offset, other.length);
+    length = newLen;
+  }
+
+  public void grow(int newLength) {
+    bytes = ArrayUtil.grow(bytes, newLength);
+  }
+
+  /** Unsigned byte order comparison */
+  public int compareTo(BytesRef other) {
+    if (this == other) return 0;
+
+    final byte[] aBytes = this.bytes;
+    int aUpto = this.offset;
+    final byte[] bBytes = other.bytes;
+    int bUpto = other.offset;
+
+    final int aStop = aUpto + Math.min(this.length, other.length);
+
+    while(aUpto < aStop) {
+      int aByte = aBytes[aUpto++] & 0xff;
+      int bByte = bBytes[bUpto++] & 0xff;
+      int diff = aByte - bByte;
+      if (diff != 0) return diff;
+    }
+
+    // One is a prefix of the other, or, they are equal:
+    return this.length - other.length;
+  }
+
+  private final static Comparator<BytesRef> utf8SortedAsUnicodeSortOrder = new UTF8SortedAsUnicodeComparator();
+
+  public static Comparator<BytesRef> getUTF8SortedAsUnicodeComparator() {
+    return utf8SortedAsUnicodeSortOrder;
+  }
+
+  private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
+    // Only singleton
+    private UTF8SortedAsUnicodeComparator() {};
+
+    public int compare(BytesRef a, BytesRef b) {
+      final byte[] aBytes = a.bytes;
+      int aUpto = a.offset;
+      final byte[] bBytes = b.bytes;
+      int bUpto = b.offset;
+      
+      final int aStop;
+      if (a.length < b.length) {
+        aStop = aUpto + a.length;
+      } else {
+        aStop = aUpto + b.length;
+      }
+
+      while(aUpto < aStop) {
+        int aByte = aBytes[aUpto++] & 0xff;
+        int bByte = bBytes[bUpto++] & 0xff;
+
+        int diff = aByte - bByte;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+
+      // One is a prefix of the other, or, they are equal:
+      return a.length - b.length;
+    }    
+  }
+
+  private final static Comparator<BytesRef> utf8SortedAsUTF16SortOrder = new UTF8SortedAsUTF16Comparator();
+
+  public static Comparator<BytesRef> getUTF8SortedAsUTF16Comparator() {
+    return utf8SortedAsUTF16SortOrder;
+  }
+
+  private static class UTF8SortedAsUTF16Comparator implements Comparator<BytesRef> {
+    // Only singleton
+    private UTF8SortedAsUTF16Comparator() {};
+
+    public int compare(BytesRef a, BytesRef b) {
+
+      final byte[] aBytes = a.bytes;
+      int aUpto = a.offset;
+      final byte[] bBytes = b.bytes;
+      int bUpto = b.offset;
+      
+      final int aStop;
+      if (a.length < b.length) {
+        aStop = aUpto + a.length;
+      } else {
+        aStop = aUpto + b.length;
+      }
+
+      while(aUpto < aStop) {
+        int aByte = aBytes[aUpto++] & 0xff;
+        int bByte = bBytes[bUpto++] & 0xff;
+
+        if (aByte != bByte) {
+
+          // See http://icu-project.org/docs/papers/utf16_code_point_order.html#utf-8-in-utf-16-order
+
+          // We know the terms are not equal, but, we may
+          // have to carefully fixup the bytes at the
+          // difference to match UTF16's sort order:
+          
+          // NOTE: instead of moving supplementary code points (0xee and 0xef) to the unused 0xfe and 0xff, 
+          // we move them to the unused 0xfc and 0xfd [reserved for future 6-byte character sequences]
+          // this reserves 0xff for preflex's term reordering (surrogate dance), and if unicode grows such
+          // that 6-byte sequences are needed we have much bigger problems anyway.
+          if (aByte >= 0xee && bByte >= 0xee) {
+            if ((aByte & 0xfe) == 0xee) {
+              aByte += 0xe;
+            }
+            if ((bByte&0xfe) == 0xee) {
+              bByte += 0xe;
+            }
+          }
+          return aByte - bByte;
+        }
+      }
+
+      // One is a prefix of the other, or, they are equal:
+      return a.length - b.length;
+    }
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/CodecUtil.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/CodecUtil.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/CodecUtil.java Sun May 29 12:30:14 2011
@@ -0,0 +1,79 @@
+package org.apache.lucene.util;
+
+/**
+ * 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;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFormatTooNewException;
+import org.apache.lucene.index.IndexFormatTooOldException;
+
+import java.io.IOException;
+
+/**
+ * @lucene.experimental
+ */
+
+public final class CodecUtil {
+  private CodecUtil() {} // no instance
+
+  private final static int CODEC_MAGIC = 0x3fd76c17;
+
+  public static DataOutput writeHeader(DataOutput out, String codec, int version)
+    throws IOException {
+    BytesRef bytes = new BytesRef(codec);
+    if (bytes.length != codec.length() || bytes.length >= 128) {
+      throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]");
+    }
+    out.writeInt(CODEC_MAGIC);
+    out.writeString(codec);
+    out.writeInt(version);
+
+    return out;
+  }
+
+  public static int headerLength(String codec) {
+    return 9+codec.length();
+  }
+
+  public static int checkHeader(DataInput in, String codec, int minVersion, int maxVersion)
+    throws IOException {
+
+    // Safety to guard against reading a bogus string:
+    final int actualHeader = in.readInt();
+    if (actualHeader != CODEC_MAGIC) {
+      throw new CorruptIndexException("codec header mismatch: actual header=" + actualHeader + " vs expected header=" + CODEC_MAGIC);
+    }
+
+    final String actualCodec = in.readString();
+    if (!actualCodec.equals(codec)) {
+      throw new CorruptIndexException("codec mismatch: actual codec=" + actualCodec + " vs expected codec=" + codec);
+    }
+
+    final int actualVersion = in.readInt();
+    if (actualVersion < minVersion) {
+      throw new IndexFormatTooOldException(null, actualVersion, minVersion, maxVersion);
+    }
+    if (actualVersion > maxVersion) {
+      throw new IndexFormatTooNewException(null, actualVersion, minVersion, maxVersion);
+    }
+
+    return actualVersion;
+  }
+}

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/IntsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/IntsRef.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/IntsRef.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/IntsRef.java Sun May 29 12:30:14 2011
@@ -0,0 +1,140 @@
+package org.apache.lucene.util;
+
+/**
+ * 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.
+ */
+
+/** Represents int[], as a slice (offset + length) into an
+ *  existing int[].
+ *
+ *  @lucene.internal */
+public final class IntsRef implements Comparable<IntsRef> {
+
+  public int[] ints;
+  public int offset;
+  public int length;
+
+  public IntsRef() {
+  }
+
+  public IntsRef(int capacity) {
+    ints = new int[capacity];
+  }
+
+  public IntsRef(int[] ints, int offset, int length) {
+    this.ints = ints;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  public IntsRef(IntsRef other) {
+    copy(other);
+  }
+
+  @Override
+  public Object clone() {
+    return new IntsRef(this);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 0;
+    final int end = offset + length;
+    for(int i = offset; i < end; i++) {
+      result = prime * result + ints[i];
+    }
+    return result;
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    return this.intsEquals((IntsRef) other);
+  }
+
+  public boolean intsEquals(IntsRef other) {
+    if (length == other.length) {
+      int otherUpto = other.offset;
+      final int[] otherInts = other.ints;
+      final int end = offset + length;
+      for(int upto=offset;upto<end;upto++,otherUpto++) {
+        if (ints[upto] != otherInts[otherUpto]) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /** Signed int order comparison */
+  public int compareTo(IntsRef other) {
+    if (this == other) return 0;
+
+    final int[] aInts = this.ints;
+    int aUpto = this.offset;
+    final int[] bInts = other.ints;
+    int bUpto = other.offset;
+
+    final int aStop = aUpto + Math.min(this.length, other.length);
+
+    while(aUpto < aStop) {
+      int aInt = aInts[aUpto++];
+      int bInt = bInts[bUpto++];
+      if (aInt > bInt) {
+        return 1;
+      } else if (aInt < bInt) {
+        return -1;
+      }
+    }
+
+    // One is a prefix of the other, or, they are equal:
+    return this.length - other.length;
+  }
+
+  public void copy(IntsRef other) {
+    if (ints == null) {
+      ints = new int[other.length];
+    } else {
+      ints = ArrayUtil.grow(ints, other.length);
+    }
+    System.arraycopy(other.ints, other.offset, ints, 0, other.length);
+    length = other.length;
+    offset = 0;
+  }
+
+  public void grow(int newLength) {
+    if (ints.length < newLength) {
+      ints = ArrayUtil.grow(ints, newLength);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append('[');
+    final int end = offset + length;
+    for(int i=offset;i<end;i++) {
+      if (i > offset) {
+        sb.append(' ');
+      }
+      sb.append(Integer.toHexString(ints[i]));
+    }
+    sb.append(']');
+    return sb.toString();
+  }
+}

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=1128870&r1=1128869&r2=1128870&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java Sun May 29 12:30:14 2011
@@ -109,6 +109,10 @@ public final class UnicodeUtil {
   private static final long HALF_SHIFT = 10;
   private static final long HALF_MASK = 0x3FFL;
 
+  private static final int SURROGATE_OFFSET = 
+    Character.MIN_SUPPLEMENTARY_CODE_POINT - 
+    (UNI_SUR_HIGH_START << HALF_SHIFT) - UNI_SUR_LOW_START;
+
   /**
    * @lucene.internal
    */
@@ -304,6 +308,114 @@ public final class UnicodeUtil {
     result.length = upto;
   }
 
+  /** Encode characters from this String, starting at offset
+   *  for length characters. After encoding, result.offset will always be 0.
+   */
+  public static void UTF16toUTF8(final CharSequence s, final int offset, final int length, BytesRef result) {
+    final int end = offset + length;
+
+    byte[] out = result.bytes;
+    result.offset = 0;
+    // Pre-allocate for worst case 4-for-1
+    final int maxLen = length * 4;
+    if (out.length < maxLen)
+      out = result.bytes = new byte[maxLen];
+
+    int upto = 0;
+    for(int i=offset;i<end;i++) {
+      final int code = (int) s.charAt(i);
+
+      if (code < 0x80)
+        out[upto++] = (byte) code;
+      else if (code < 0x800) {
+        out[upto++] = (byte) (0xC0 | (code >> 6));
+        out[upto++] = (byte)(0x80 | (code & 0x3F));
+      } else if (code < 0xD800 || code > 0xDFFF) {
+        out[upto++] = (byte)(0xE0 | (code >> 12));
+        out[upto++] = (byte)(0x80 | ((code >> 6) & 0x3F));
+        out[upto++] = (byte)(0x80 | (code & 0x3F));
+      } else {
+        // surrogate pair
+        // confirm valid high surrogate
+        if (code < 0xDC00 && (i < end-1)) {
+          int utf32 = (int) s.charAt(i+1);
+          // confirm valid low surrogate and write pair
+          if (utf32 >= 0xDC00 && utf32 <= 0xDFFF) { 
+            utf32 = (code << 10) + utf32 + SURROGATE_OFFSET;
+            i++;
+            out[upto++] = (byte)(0xF0 | (utf32 >> 18));
+            out[upto++] = (byte)(0x80 | ((utf32 >> 12) & 0x3F));
+            out[upto++] = (byte)(0x80 | ((utf32 >> 6) & 0x3F));
+            out[upto++] = (byte)(0x80 | (utf32 & 0x3F));
+            continue;
+          }
+        }
+        // replace unpaired surrogate or out-of-order low surrogate
+        // with substitution character
+        out[upto++] = (byte) 0xEF;
+        out[upto++] = (byte) 0xBF;
+        out[upto++] = (byte) 0xBD;
+      }
+    }
+    //assert matches(s, offset, length, out, upto);
+    result.length = upto;
+  }
+
+  /** Encode characters from a char[] source, starting at
+   *  offset for length chars. After encoding, result.offset will always be 0.
+   */
+  public static void UTF16toUTF8(final char[] source, final int offset, final int length, BytesRef result) {
+
+    int upto = 0;
+    int i = offset;
+    final int end = offset + length;
+    byte[] out = result.bytes;
+    // Pre-allocate for worst case 4-for-1
+    final int maxLen = length * 4;
+    if (out.length < maxLen)
+      out = result.bytes = new byte[maxLen];
+    result.offset = 0;
+
+    while(i < end) {
+      
+      final int code = (int) source[i++];
+
+      if (code < 0x80)
+        out[upto++] = (byte) code;
+      else if (code < 0x800) {
+        out[upto++] = (byte) (0xC0 | (code >> 6));
+        out[upto++] = (byte)(0x80 | (code & 0x3F));
+      } else if (code < 0xD800 || code > 0xDFFF) {
+        out[upto++] = (byte)(0xE0 | (code >> 12));
+        out[upto++] = (byte)(0x80 | ((code >> 6) & 0x3F));
+        out[upto++] = (byte)(0x80 | (code & 0x3F));
+      } else {
+        // surrogate pair
+        // confirm valid high surrogate
+        if (code < 0xDC00 && i < end) {
+          int utf32 = (int) source[i];
+          // confirm valid low surrogate and write pair
+          if (utf32 >= 0xDC00 && utf32 <= 0xDFFF) { 
+            utf32 = (code << 10) + utf32 + SURROGATE_OFFSET;
+            i++;
+            out[upto++] = (byte)(0xF0 | (utf32 >> 18));
+            out[upto++] = (byte)(0x80 | ((utf32 >> 12) & 0x3F));
+            out[upto++] = (byte)(0x80 | ((utf32 >> 6) & 0x3F));
+            out[upto++] = (byte)(0x80 | (utf32 & 0x3F));
+            continue;
+          }
+        }
+        // replace unpaired surrogate or out-of-order low surrogate
+        // with substitution character
+        out[upto++] = (byte) 0xEF;
+        out[upto++] = (byte) 0xBF;
+        out[upto++] = (byte) 0xBD;
+      }
+    }
+    //assert matches(source, offset, length, out, upto);
+    result.length = upto;
+  }
+
   /** Convert UTF8 bytes into UTF16 characters.  If offset
    *  is non-zero, conversion starts at that starting point
    *  in utf8, re-using the results from the previous call

Added: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java?rev=1128870&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java (added)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/fst/Builder.java Sun May 29 12:30:14 2011
@@ -0,0 +1,545 @@
+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 org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+
+import java.io.IOException;
+
+/**
+ * Builds a compact FST (maps an IntsRef term to an arbitrary
+ * output) from pre-sorted terms with outputs (the FST
+ * becomes an FSA if you use NoOutputs).  The FST is written
+ * on-the-fly into a compact serialized format byte array, which can
+ * be saved to / loaded from a Directory or used directly
+ * for traversal.  The FST is always finite (no cycles).
+ *
+ * <p>NOTE: The algorithm is described at
+ * http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.24.3698</p>
+ *
+ * If your outputs are ByteSequenceOutput then the final FST
+ * will be minimal, but if you use PositiveIntOutput then
+ * it's only "near minimal".  For example, aa/0, aab/1, bbb/2
+ * will produce 6 states when a 5 state fst is also
+ * possible.
+ *
+ * The parameterized type T is the output type.  See the
+ * subclasses of {@link Outputs}.
+ *
+ * @lucene.experimental
+ */
+
+public class Builder<T> {
+  private final NodeHash<T> dedupHash;
+  private final FST<T> fst;
+  private final T NO_OUTPUT;
+
+  // simplistic pruning: we prune node (and all following
+  // nodes) if less than this number of terms go through it:
+  private final int minSuffixCount1;
+
+  // better pruning: we prune node (and all following
+  // nodes) if the prior node has less than this number of
+  // terms go through it:
+  private final int minSuffixCount2;
+
+  private final IntsRef lastInput = new IntsRef();
+
+  // NOTE: cutting this over to ArrayList instead loses ~6%
+  // in build performance on 9.8M Wikipedia terms; so we
+  // left this as an array:
+  // current "frontier"
+  private UnCompiledNode<T>[] frontier;
+
+  public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doMinSuffix, Outputs<T> outputs) {
+    this.minSuffixCount1 = minSuffixCount1;
+    this.minSuffixCount2 = minSuffixCount2;
+    fst = new FST<T>(inputType, outputs);
+    if (doMinSuffix) {
+      dedupHash = new NodeHash<T>(fst);
+    } else {
+      dedupHash = null;
+    }
+    NO_OUTPUT = outputs.getNoOutput();
+
+    @SuppressWarnings("unchecked") final UnCompiledNode<T>[] f = (UnCompiledNode<T>[]) new UnCompiledNode[10];
+    frontier = f;
+    for(int idx=0;idx<frontier.length;idx++) {
+      frontier[idx] = new UnCompiledNode<T>(this, idx);
+    }
+  }
+
+  public int getTotStateCount() {
+    return fst.nodeCount;
+  }
+
+  public long getTermCount() {
+    return frontier[0].inputCount;
+  }
+
+  public int getMappedStateCount() {
+    return dedupHash == null ? 0 : fst.nodeCount;
+  }
+
+  private CompiledNode compileNode(UnCompiledNode<T> n) throws IOException {
+
+    final int address;
+    if (dedupHash != null) {
+      if (n.numArcs == 0) {
+        address = fst.addNode(n);
+      } else {
+        address = dedupHash.add(n);
+      }
+    } else {
+      address = fst.addNode(n);
+    }
+    assert address != -2;
+
+    n.clear();
+
+    final CompiledNode fn = new CompiledNode();
+    fn.address = address;
+    return fn;
+  }
+
+  private void compilePrevTail(int prefixLenPlus1) throws IOException {
+    assert prefixLenPlus1 >= 1;
+    //System.out.println("  compileTail " + prefixLenPlus1);
+    for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
+      boolean doPrune = false;
+      boolean doCompile = false;
+
+      final UnCompiledNode<T> node = frontier[idx];
+      final UnCompiledNode<T> parent = frontier[idx-1];
+
+      if (node.inputCount < minSuffixCount1) {
+        doPrune = true;
+        doCompile = true;
+      } else if (idx > prefixLenPlus1) {
+        // prune if parent's inputCount is less than suffixMinCount2
+        if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) {
+          // my parent, about to be compiled, doesn't make the cut, so
+          // I'm definitely pruned 
+
+          // if pruneCount2 is 1, we keep only up
+          // until the 'distinguished edge', ie we keep only the
+          // 'divergent' part of the FST. if my parent, about to be
+          // compiled, has inputCount 1 then we are already past the
+          // distinguished edge.  NOTE: this only works if
+          // the FST outputs are not "compressible" (simple
+          // ords ARE compressible).
+          doPrune = true;
+        } else {
+          // my parent, about to be compiled, does make the cut, so
+          // I'm definitely not pruned 
+          doPrune = false;
+        }
+        doCompile = true;
+      } else {
+        // if pruning is disabled (count is 0) we can always
+        // compile current node
+        doCompile = minSuffixCount2 == 0;
+      }
+
+      //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
+
+      if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) {
+        // drop all arcs
+        for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+          @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
+          target.clear();
+        }
+        node.numArcs = 0;
+      }
+
+      if (doPrune) {
+        // this node doesn't make it -- deref it
+        node.clear();
+        parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
+      } else {
+
+        if (minSuffixCount2 != 0) {
+          compileAllTargets(node);
+        }
+        final T nextFinalOutput = node.output;
+
+        // We "fake" the node as being final if it has no
+        // outgoing arcs; in theory we could leave it
+        // as non-final (the FST can represent this), but
+        // FSTEnum, Util, etc., have trouble w/ non-final
+        // dead-end states:
+        final boolean isFinal = node.isFinal || node.numArcs == 0;
+
+        if (doCompile) {
+          // this node makes it and we now compile it.  first,
+          // compile any targets that were previously
+          // undecided:
+          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                             compileNode(node),
+                             nextFinalOutput,
+                             isFinal);
+        } else {
+          // replaceLast just to install
+          // nextFinalOutput/isFinal onto the arc
+          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                             node,
+                             nextFinalOutput,
+                             isFinal);
+          // this node will stay in play for now, since we are
+          // undecided on whether to prune it.  later, it
+          // will be either compiled or pruned, so we must
+          // allocate a new node:
+          frontier[idx] = new UnCompiledNode<T>(this, idx);
+        }
+      }
+    }
+  }
+
+  private final IntsRef scratchIntsRef = new IntsRef(10);
+
+  public void add(BytesRef input, T output) throws IOException {
+    assert fst.getInputType() == FST.INPUT_TYPE.BYTE1;
+    scratchIntsRef.grow(input.length);
+    for(int i=0;i<input.length;i++) {
+      scratchIntsRef.ints[i] = input.bytes[i+input.offset] & 0xFF;
+    }
+    scratchIntsRef.length = input.length;
+    add(scratchIntsRef, output);
+  }
+
+  /** Sugar: adds the UTF32 codepoints from char[] slice.  FST
+   *  must be FST.INPUT_TYPE.BYTE4! */
+  public void add(char[] s, int offset, int length, T output) throws IOException {
+    assert fst.getInputType() == FST.INPUT_TYPE.BYTE4;
+    int charIdx = offset;
+    int intIdx = 0;
+    final int charLimit = offset + length;
+    while(charIdx < charLimit) {
+      scratchIntsRef.grow(intIdx+1);
+      final int utf32 = Character.codePointAt(s, charIdx);
+      scratchIntsRef.ints[intIdx] = utf32;
+      charIdx += Character.charCount(utf32);
+      intIdx++;
+    }
+    scratchIntsRef.length = intIdx;
+    add(scratchIntsRef, output);
+  }
+
+  /** Sugar: adds the UTF32 codepoints from CharSequence.  FST
+   *  must be FST.INPUT_TYPE.BYTE4! */
+  public void add(CharSequence s, T output) throws IOException {
+    assert fst.getInputType() == FST.INPUT_TYPE.BYTE4;
+    int charIdx = 0;
+    int intIdx = 0;
+    final int charLimit = s.length();
+    while(charIdx < charLimit) {
+      scratchIntsRef.grow(intIdx+1);
+      final int utf32 = Character.codePointAt(s, charIdx);
+      scratchIntsRef.ints[intIdx] = utf32;
+      charIdx += Character.charCount(utf32);
+      intIdx++;
+    }
+    scratchIntsRef.length = intIdx;
+    add(scratchIntsRef, output);
+  }
+
+  /** It's OK to add the same input twice in a row with
+   *  different outputs, as long as outputs impls the merge
+   *  method. */
+  public void add(IntsRef input, T output) throws IOException {
+    //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output));
+    assert lastInput.length == 0 || input.compareTo(lastInput) >= 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
+    assert validOutput(output);
+
+    //System.out.println("\nadd: " + input);
+    if (input.length == 0) {
+      // empty input: only allowed as first input.  we have
+      // to special case this because the packed FST
+      // format cannot represent the empty input since
+      // 'finalness' is stored on the incoming arc, not on
+      // the node
+      frontier[0].inputCount++;
+      frontier[0].isFinal = true;
+      fst.setEmptyOutput(output);
+      return;
+    }
+
+    // compare shared prefix length
+    int pos1 = 0;
+    int pos2 = input.offset;
+    final int pos1Stop = Math.min(lastInput.length, input.length);
+    while(true) {
+      //System.out.println("  incr " + pos1);
+      frontier[pos1].inputCount++;
+      if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) {
+        break;
+      }
+      pos1++;
+      pos2++;
+    }
+    final int prefixLenPlus1 = pos1+1;
+      
+    if (frontier.length < input.length+1) {
+      @SuppressWarnings("unchecked") final UnCompiledNode<T>[] next =
+        new UnCompiledNode[ArrayUtil.oversize(input.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      System.arraycopy(frontier, 0, next, 0, frontier.length);
+      for(int idx=frontier.length;idx<next.length;idx++) {
+        next[idx] = new UnCompiledNode<T>(this, idx);
+      }
+      frontier = next;
+    }
+
+    // minimize/compile states from previous input's
+    // orphan'd suffix
+    compilePrevTail(prefixLenPlus1);
+
+    // init tail states for current input
+    for(int idx=prefixLenPlus1;idx<=input.length;idx++) {
+      frontier[idx-1].addArc(input.ints[input.offset + idx - 1],
+                             frontier[idx]);
+      //System.out.println("  incr tail " + idx);
+      frontier[idx].inputCount++;
+    }
+
+    final UnCompiledNode<T> lastNode = frontier[input.length];
+    lastNode.isFinal = true;
+    lastNode.output = NO_OUTPUT;
+
+    // push conflicting outputs forward, only as far as
+    // needed
+    for(int idx=1;idx<prefixLenPlus1;idx++) {
+      final UnCompiledNode<T> node = frontier[idx];
+      final UnCompiledNode<T> parentNode = frontier[idx-1];
+
+      final T lastOutput = parentNode.getLastOutput(input.ints[input.offset + idx - 1]);
+      assert validOutput(lastOutput);
+
+      final T commonOutputPrefix;
+      final T wordSuffix;
+
+      if (lastOutput != NO_OUTPUT) {
+        commonOutputPrefix = fst.outputs.common(output, lastOutput);
+        assert validOutput(commonOutputPrefix);
+        wordSuffix = fst.outputs.subtract(lastOutput, commonOutputPrefix);
+        assert validOutput(wordSuffix);
+        parentNode.setLastOutput(input.ints[input.offset + idx - 1], commonOutputPrefix);
+        node.prependOutput(wordSuffix);
+      } else {
+        commonOutputPrefix = wordSuffix = NO_OUTPUT;
+      }
+
+      output = fst.outputs.subtract(output, commonOutputPrefix);
+      assert validOutput(output);
+    }
+
+    if (lastInput.length == input.length && prefixLenPlus1 == 1+input.length) {
+      // same input more than 1 time in a row, mapping to
+      // multiple outputs
+      lastNode.output = fst.outputs.merge(lastNode.output, output);
+    } else {
+      // this new arc is private to this new input; set its
+      // arc output to the leftover output:
+      frontier[prefixLenPlus1-1].setLastOutput(input.ints[input.offset + prefixLenPlus1-1], output);
+    }
+
+    // save last input
+    lastInput.copy(input);
+
+    //System.out.println("  count[0]=" + frontier[0].inputCount);
+  }
+
+  private boolean validOutput(T output) {
+    return output == NO_OUTPUT || !output.equals(NO_OUTPUT);
+  }
+
+  /** Returns final FST.  NOTE: this will return null if
+   *  nothing is accepted by the FST. */
+  public FST<T> finish() throws IOException {
+
+    // minimize nodes in the last word's suffix
+    compilePrevTail(1);
+    //System.out.println("finish: inputCount=" + frontier[0].inputCount);
+    if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) {
+      if (fst.emptyOutput == null) {
+        return null;
+      } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) {
+        // empty string got pruned
+        return null;
+      } else {
+        fst.finish(compileNode(frontier[0]).address);
+        //System.out.println("compile addr = " + fst.getStartNode());
+        return fst;
+      }
+    } else {
+      if (minSuffixCount2 != 0) {
+        compileAllTargets(frontier[0]);
+      }
+      //System.out.println("NOW: " + frontier[0].numArcs);
+      fst.finish(compileNode(frontier[0]).address);
+    }
+    
+    return fst;
+  }
+
+  private void compileAllTargets(UnCompiledNode<T> node) throws IOException {
+    for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+      final Arc<T> arc = node.arcs[arcIdx];
+      if (!arc.target.isCompiled()) {
+        // not yet compiled
+        @SuppressWarnings("unchecked") final UnCompiledNode<T> n = (UnCompiledNode<T>) arc.target;
+        if (n.numArcs == 0) {
+          //System.out.println("seg=" + segment + "        FORCE final arc=" + (char) arc.label);
+          arc.isFinal = n.isFinal = true;
+        }
+        arc.target = compileNode(n);
+      }
+    }
+  }
+
+  static class Arc<T> {
+    public int label;                             // really an "unsigned" byte
+    public Node target;
+    public boolean isFinal;
+    public T output;
+    public T nextFinalOutput;
+  }
+
+  // NOTE: not many instances of Node or CompiledNode are in
+  // memory while the FST is being built; it's only the
+  // current "frontier":
+
+  static interface Node {
+    boolean isCompiled();
+  }
+
+  static final class CompiledNode implements Node {
+    int address;
+    public boolean isCompiled() {
+      return true;
+    }
+  }
+
+  static final class UnCompiledNode<T> implements Node {
+    final Builder<T> owner;
+    int numArcs;
+    Arc<T>[] arcs;
+    T output;
+    boolean isFinal;
+    long inputCount;
+
+    /** This node's depth, starting from the automaton root. */
+    final int depth;
+
+    /**
+     * @param depth
+     *          The node's depth starting from the automaton root. Needed for
+     *          LUCENE-2934 (node expansion based on conditions other than the
+     *          fanout size).
+     */
+    @SuppressWarnings("unchecked")
+    public UnCompiledNode(Builder<T> owner, int depth) {
+      this.owner = owner;
+      arcs = (Arc<T>[]) new Arc[1];
+      arcs[0] = new Arc<T>();
+      output = owner.NO_OUTPUT;
+      this.depth = depth;
+    }
+
+    public boolean isCompiled() {
+      return false;
+    }
+
+    public void clear() {
+      numArcs = 0;
+      isFinal = false;
+      output = owner.NO_OUTPUT;
+      inputCount = 0;
+
+      // We don't clear the depth here because it never changes 
+      // for nodes on the frontier (even when reused).
+    }
+
+    public T getLastOutput(int labelToMatch) {
+      assert numArcs > 0;
+      assert arcs[numArcs-1].label == labelToMatch;
+      return arcs[numArcs-1].output;
+    }
+
+    public void addArc(int label, Node target) {
+      assert label >= 0;
+      assert numArcs == 0 || label > arcs[numArcs-1].label: "arc[-1].label=" + arcs[numArcs-1].label + " new label=" + label + " numArcs=" + numArcs;
+      if (numArcs == arcs.length) {
+        @SuppressWarnings("unchecked") final Arc<T>[] newArcs =
+          new Arc[ArrayUtil.oversize(numArcs+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+        System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
+        for(int arcIdx=numArcs;arcIdx<newArcs.length;arcIdx++) {
+          newArcs[arcIdx] = new Arc<T>();
+        }
+        arcs = newArcs;
+      }
+      final Arc<T> arc = arcs[numArcs++];
+      arc.label = label;
+      arc.target = target;
+      arc.output = arc.nextFinalOutput = owner.NO_OUTPUT;
+      arc.isFinal = false;
+    }
+
+    public void replaceLast(int labelToMatch, Node target, T nextFinalOutput, boolean isFinal) {
+      assert numArcs > 0;
+      final Arc<T> arc = arcs[numArcs-1];
+      assert arc.label == labelToMatch: "arc.label=" + arc.label + " vs " + labelToMatch;
+      arc.target = target;
+      //assert target.address != -2;
+      arc.nextFinalOutput = nextFinalOutput;
+      arc.isFinal = isFinal;
+    }
+
+    public void deleteLast(int label, Node target) {
+      assert numArcs > 0;
+      assert label == arcs[numArcs-1].label;
+      assert target == arcs[numArcs-1].target;
+      numArcs--;
+    }
+
+    public void setLastOutput(int labelToMatch, T newOutput) {
+      assert owner.validOutput(newOutput);
+      assert numArcs > 0;
+      final Arc<T> arc = arcs[numArcs-1];
+      assert arc.label == labelToMatch;
+      arc.output = newOutput;
+    }
+
+    // pushes an output prefix forward onto all arcs
+    public void prependOutput(T outputPrefix) {
+      assert owner.validOutput(outputPrefix);
+
+      for(int arcIdx=0;arcIdx<numArcs;arcIdx++) {
+        arcs[arcIdx].output = owner.fst.outputs.add(outputPrefix, arcs[arcIdx].output);
+        assert owner.validOutput(arcs[arcIdx].output);
+      }
+
+      if (isFinal) {
+        output = owner.fst.outputs.add(outputPrefix, output);
+        assert owner.validOutput(output);
+      }
+    }
+  }
+}