You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC

svn commit: r1512909 [6/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-tool...

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java Sun Aug 11 12:19:13 2013
@@ -32,8 +32,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
@@ -44,12 +43,7 @@ import org.apache.lucene.util.fst.Util; 
  * @lucene.experimental */
 public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
 
-  private final PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton(true);
-  private int indexDivisor;
-
-  // Closed if indexLoaded is true:
-  private IndexInput in;
-  private volatile boolean indexLoaded;
+  private final PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
 
   final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<FieldInfo,FieldIndexData>();
   
@@ -59,17 +53,15 @@ public class VariableGapTermsIndexReader
   private final int version;
 
   final String segment;
-  public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String segmentSuffix, IOContext context)
+  public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, String segmentSuffix, IOContext context)
     throws IOException {
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
+    final IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
     this.segment = segment;
     boolean success = false;
-    assert indexDivisor == -1 || indexDivisor > 0;
 
     try {
       
       version = readHeader(in);
-      this.indexDivisor = indexDivisor;
 
       seekDir(in, dirOffset);
 
@@ -83,27 +75,20 @@ public class VariableGapTermsIndexReader
         final int field = in.readVInt();
         final long indexStart = in.readVLong();
         final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-        FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(fieldInfo, indexStart));
+        FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, fieldInfo, indexStart));
         if (previous != null) {
           throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
         }
       }
       success = true;
     } finally {
-      if (indexDivisor > 0) {
-        in.close();
-        in = null;
-        if (success) {
-          indexLoaded = true;
-        }
+      if (success) {
+        IOUtils.close(in);
+      } else {
+        IOUtils.closeWhileHandlingException(in);
       }
     }
   }
-
-  @Override
-  public int getDivisor() {
-    return indexDivisor;
-  }
   
   private int readHeader(IndexInput input) throws IOException {
     int version = CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME,
@@ -168,52 +153,21 @@ public class VariableGapTermsIndexReader
   }
 
   private final class FieldIndexData {
+    private final FST<Long> fst;
 
-    private final long indexStart;
-    // Set only if terms index is loaded:
-    private volatile FST<Long> fst;
-
-    public FieldIndexData(FieldInfo fieldInfo, long indexStart) throws IOException {
-      this.indexStart = indexStart;
-
-      if (indexDivisor > 0) {
-        loadTermsIndex();
-      }
-    }
-
-    private void loadTermsIndex() throws IOException {
-      if (fst == null) {
-        IndexInput clone = in.clone();
-        clone.seek(indexStart);
-        fst = new FST<Long>(clone, fstOutputs);
-        clone.close();
-
-        /*
-        final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
-        Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
-        Util.toDot(fst, w, false, false);
-        System.out.println("FST INDEX: SAVED to " + dotFileName);
-        w.close();
-        */
-
-        if (indexDivisor > 1) {
-          // subsample
-          final IntsRef scratchIntsRef = new IntsRef();
-          final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
-          final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
-          final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
-          BytesRefFSTEnum.InputOutput<Long> result;
-          int count = indexDivisor;
-          while((result = fstEnum.next()) != null) {
-            if (count == indexDivisor) {
-              builder.add(Util.toIntsRef(result.input, scratchIntsRef), result.output);
-              count = 0;
-            }
-            count++;
-          }
-          fst = builder.finish();
-        }
-      }
+    public FieldIndexData(IndexInput in, FieldInfo fieldInfo, long indexStart) throws IOException {
+      IndexInput clone = in.clone();
+      clone.seek(indexStart);
+      fst = new FST<Long>(clone, fstOutputs);
+      clone.close();
+
+      /*
+      final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+      Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+      Util.toDot(fst, w, false, false);
+      System.out.println("FST INDEX: SAVED to " + dotFileName);
+      w.close();
+      */
     }
   }
 
@@ -228,11 +182,7 @@ public class VariableGapTermsIndexReader
   }
 
   @Override
-  public void close() throws IOException {
-    if (in != null && !indexLoaded) {
-      in.close();
-    }
-  }
+  public void close() throws IOException {}
 
   private void seekDir(IndexInput input, long dirOffset) throws IOException {
     if (version >= VariableGapTermsIndexWriter.VERSION_APPEND_ONLY) {

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java Sun Aug 11 12:19:13 2013
@@ -235,7 +235,7 @@ public class VariableGapTermsIndexWriter
 
     public FSTFieldWriter(FieldInfo fieldInfo, long termsFilePointer) throws IOException {
       this.fieldInfo = fieldInfo;
-      fstOutputs = PositiveIntOutputs.getSingleton(true);
+      fstOutputs = PositiveIntOutputs.getSingleton();
       fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, fstOutputs);
       indexStart = out.getFilePointer();
       ////System.out.println("VGW: field=" + fieldInfo.name);

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -331,7 +331,7 @@ public final class BloomFilteringPosting
       }
       
       @Override
-      public final boolean seekExact(BytesRef text, boolean useCache)
+      public final boolean seekExact(BytesRef text)
           throws IOException {
         // The magical fail-fast speed up that is the entire point of all of
         // this code - save a disk seek if there is a match on an in-memory
@@ -341,13 +341,13 @@ public final class BloomFilteringPosting
         if (filter.contains(text) == ContainsResult.NO) {
           return false;
         }
-        return delegate().seekExact(text, useCache);
+        return delegate().seekExact(text);
       }
       
       @Override
-      public final SeekStatus seekCeil(BytesRef text, boolean useCache)
+      public final SeekStatus seekCeil(BytesRef text)
           throws IOException {
-        return delegate().seekCeil(text, useCache);
+        return delegate().seekCeil(text);
       }
       
       @Override

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java Sun Aug 11 12:19:13 2013
@@ -27,9 +27,11 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.packed.BlockPackedWriter;
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
@@ -38,6 +40,7 @@ import org.apache.lucene.util.packed.Pac
 public class DiskDocValuesConsumer extends DocValuesConsumer {
 
   static final int BLOCK_SIZE = 16384;
+  static final int ADDRESS_INTERVAL = 16;
 
   /** Compressed using packed blocks of ints. */
   public static final int DELTA_COMPRESSED = 0;
@@ -45,6 +48,13 @@ public class DiskDocValuesConsumer exten
   public static final int GCD_COMPRESSED = 1;
   /** Compressed by giving IDs to unique values. */
   public static final int TABLE_COMPRESSED = 2;
+  
+  /** Uncompressed binary, written directly (fixed length). */
+  public static final int BINARY_FIXED_UNCOMPRESSED = 0;
+  /** Uncompressed binary, written directly (variable length). */
+  public static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+  /** Compressed binary with shared prefixes */
+  public static final int BINARY_PREFIX_COMPRESSED = 2;
 
   final IndexOutput data, meta;
   final int maxDoc;
@@ -173,7 +183,7 @@ public class DiskDocValuesConsumer exten
   }
 
   @Override
-  public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+  public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
     // write the byte[] data
     meta.writeVInt(field.number);
     meta.writeByte(DiskDocValuesFormat.BINARY);
@@ -187,6 +197,7 @@ public class DiskDocValuesConsumer exten
       data.writeBytes(v.bytes, v.offset, v.length);
       count++;
     }
+    meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
     meta.writeVInt(minLength);
     meta.writeVInt(maxLength);
     meta.writeVLong(count);
@@ -208,12 +219,68 @@ public class DiskDocValuesConsumer exten
       writer.finish();
     }
   }
+  
+  protected void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // first check if its a "fixed-length" terms dict
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    for (BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+    }
+    if (minLength == maxLength) {
+      // no index needed: direct addressing by mult
+      addBinaryField(field, values);
+    } else {
+      // header
+      meta.writeVInt(field.number);
+      meta.writeByte(DiskDocValuesFormat.BINARY);
+      meta.writeVInt(BINARY_PREFIX_COMPRESSED);
+      // now write the bytes: sharing prefixes within a block
+      final long startFP = data.getFilePointer();
+      // currently, we have to store the delta from expected for every 1/nth term
+      // we could avoid this, but its not much and less overall RAM than the previous approach!
+      RAMOutputStream addressBuffer = new RAMOutputStream();
+      MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCK_SIZE);
+      BytesRef lastTerm = new BytesRef();
+      long count = 0;
+      for (BytesRef v : values) {
+        if (count % ADDRESS_INTERVAL == 0) {
+          termAddresses.add(data.getFilePointer() - startFP);
+          // force the first term in a block to be abs-encoded
+          lastTerm.length = 0;
+        }
+        
+        // prefix-code
+        int sharedPrefix = StringHelper.bytesDifference(lastTerm, v);
+        data.writeVInt(sharedPrefix);
+        data.writeVInt(v.length - sharedPrefix);
+        data.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
+        lastTerm.copyBytes(v);
+        count++;
+      }
+      final long indexStartFP = data.getFilePointer();
+      // write addresses of indexed terms
+      termAddresses.finish();
+      addressBuffer.writeTo(data);
+      addressBuffer = null;
+      termAddresses = null;
+      meta.writeVInt(minLength);
+      meta.writeVInt(maxLength);
+      meta.writeVLong(count);
+      meta.writeLong(startFP);
+      meta.writeVInt(ADDRESS_INTERVAL);
+      meta.writeLong(indexStartFP);
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(BLOCK_SIZE);
+    }
+  }
 
   @Override
   public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
     meta.writeVInt(field.number);
     meta.writeByte(DiskDocValuesFormat.SORTED);
-    addBinaryField(field, values);
+    addTermsDict(field, values);
     addNumericField(field, docToOrd, false);
   }
   
@@ -222,7 +289,7 @@ public class DiskDocValuesConsumer exten
     meta.writeVInt(field.number);
     meta.writeByte(DiskDocValuesFormat.SORTED_SET);
     // write the ord -> byte[] as a binary field
-    addBinaryField(field, values);
+    addTermsDict(field, values);
     // write the stream of ords as a numeric field
     // NOTE: we could return an iterator that delta-encodes these within a doc
     addNumericField(field, ords, false);

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java Sun Aug 11 12:19:13 2013
@@ -28,8 +28,7 @@ import org.apache.lucene.index.SegmentWr
 /**
  * DocValues format that keeps most things on disk.
  * <p>
- * Things like ordinals and disk offsets are loaded into ram,
- * for single-seek access to all the types.
+ * Only things like disk offsets are loaded into ram.
  * <p>
  * @lucene.experimental
  */
@@ -54,7 +53,8 @@ public final class DiskDocValuesFormat e
   public static final String META_CODEC = "DiskDocValuesMetadata";
   public static final String META_EXTENSION = "dvdm";
   public static final int VERSION_START = 0;
-  public static final int VERSION_CURRENT = VERSION_START;
+  public static final int VERSION_COMPRESSED_TERMS = 1;
+  public static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS;
   public static final byte NUMERIC = 0;
   public static final byte BINARY = 1;
   public static final byte SORTED = 2;

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Sun Aug 11 12:19:13 2013
@@ -21,7 +21,12 @@ import static org.apache.lucene.codecs.d
 import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.GCD_COMPRESSED;
 import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.TABLE_COMPRESSED;
 
+import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
+import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
+import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -29,6 +34,8 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
@@ -36,7 +43,10 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.BlockPackedReader;
@@ -51,7 +61,6 @@ class DiskDocValuesProducer extends DocV
   private final IndexInput data;
 
   // memory-resident structures
-  private final Map<Integer,BlockPackedReader> ordinalInstances = new HashMap<Integer,BlockPackedReader>();
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   
@@ -63,7 +72,7 @@ class DiskDocValuesProducer extends DocV
     final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
-                                      DiskDocValuesFormat.VERSION_START,
+                                      DiskDocValuesFormat.VERSION_CURRENT,
                                       DiskDocValuesFormat.VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       ords = new HashMap<Integer,NumericEntry>();
@@ -85,7 +94,7 @@ class DiskDocValuesProducer extends DocV
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.openInput(dataName, state.context);
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 DiskDocValuesFormat.VERSION_START,
+                                                 DiskDocValuesFormat.VERSION_CURRENT,
                                                  DiskDocValuesFormat.VERSION_CURRENT);
       if (version != version2) {
         throw new CorruptIndexException("Format versions mismatch");
@@ -197,14 +206,27 @@ class DiskDocValuesProducer extends DocV
   
   static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
     BinaryEntry entry = new BinaryEntry();
+    entry.format = meta.readVInt();
     entry.minLength = meta.readVInt();
     entry.maxLength = meta.readVInt();
     entry.count = meta.readVLong();
     entry.offset = meta.readLong();
-    if (entry.minLength != entry.maxLength) {
-      entry.addressesOffset = meta.readLong();
-      entry.packedIntsVersion = meta.readVInt();
-      entry.blockSize = meta.readVInt();
+    switch(entry.format) {
+      case BINARY_FIXED_UNCOMPRESSED:
+        break;
+      case BINARY_PREFIX_COMPRESSED:
+        entry.addressInterval = meta.readVInt();
+        entry.addressesOffset = meta.readLong();
+        entry.packedIntsVersion = meta.readVInt();
+        entry.blockSize = meta.readVInt();
+        break;
+      case BINARY_VARIABLE_UNCOMPRESSED:
+        entry.addressesOffset = meta.readLong();
+        entry.packedIntsVersion = meta.readVInt();
+        entry.blockSize = meta.readVInt();
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
     }
     return entry;
   }
@@ -256,10 +278,15 @@ class DiskDocValuesProducer extends DocV
   @Override
   public BinaryDocValues getBinary(FieldInfo field) throws IOException {
     BinaryEntry bytes = binaries.get(field.number);
-    if (bytes.minLength == bytes.maxLength) {
-      return getFixedBinary(field, bytes);
-    } else {
-      return getVariableBinary(field, bytes);
+    switch(bytes.format) {
+      case BINARY_FIXED_UNCOMPRESSED:
+        return getFixedBinary(field, bytes);
+      case BINARY_VARIABLE_UNCOMPRESSED:
+        return getVariableBinary(field, bytes);
+      case BINARY_PREFIX_COMPRESSED:
+        return getCompressedBinary(field, bytes);
+      default:
+        throw new AssertionError();
     }
   }
   
@@ -322,22 +349,39 @@ class DiskDocValuesProducer extends DocV
     };
   }
 
+  private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final IndexInput data = this.data.clone();
+    final long interval = bytes.addressInterval;
+
+    final MonotonicBlockPackedReader addresses;
+    synchronized (addressInstances) {
+      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
+      if (addrInstance == null) {
+        data.seek(bytes.addressesOffset);
+        final long size;
+        if (bytes.count % interval == 0) {
+          size = bytes.count / interval;
+        } else {
+          size = 1L + bytes.count / interval;
+        }
+        addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+        addressInstances.put(field.number, addrInstance);
+      }
+      addresses = addrInstance;
+    }
+    
+    return new CompressedBinaryDocValues(bytes, addresses, data);
+  }
+
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
     final int valueCount = (int) binaries.get(field.number).count;
     final BinaryDocValues binary = getBinary(field);
-    final BlockPackedReader ordinals;
-    synchronized (ordinalInstances) {
-      BlockPackedReader ordsInstance = ordinalInstances.get(field.number);
-      if (ordsInstance == null) {
-        NumericEntry entry = ords.get(field.number);
-        IndexInput data = this.data.clone();
-        data.seek(entry.offset);
-        ordsInstance = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
-        ordinalInstances.put(field.number, ordsInstance);
-      }
-      ordinals = ordsInstance;
-    }
+    NumericEntry entry = ords.get(field.number);
+    IndexInput data = this.data.clone();
+    data.seek(entry.offset);
+    final BlockPackedReader ordinals = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
+    
     return new SortedDocValues() {
 
       @Override
@@ -354,6 +398,24 @@ class DiskDocValuesProducer extends DocV
       public int getValueCount() {
         return valueCount;
       }
+
+      @Override
+      public int lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+        return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
     };
   }
 
@@ -407,6 +469,24 @@ class DiskDocValuesProducer extends DocV
       public long getValueCount() {
         return valueCount;
       }
+      
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
     };
   }
 
@@ -431,10 +511,12 @@ class DiskDocValuesProducer extends DocV
   static class BinaryEntry {
     long offset;
 
+    int format;
     long count;
     int minLength;
     int maxLength;
     long addressesOffset;
+    long addressInterval;
     int packedIntsVersion;
     int blockSize;
   }
@@ -457,4 +539,204 @@ class DiskDocValuesProducer extends DocV
     
     abstract void get(long id, BytesRef Result);
   }
+  
+  // in the compressed case, we add a few additional operations for
+  // more efficient reverse lookup and enumeration
+  static class CompressedBinaryDocValues extends LongBinaryDocValues {
+    final BinaryEntry bytes;
+    final long interval;
+    final long numValues;
+    final long numIndexValues;
+    final MonotonicBlockPackedReader addresses;
+    final IndexInput data;
+    final TermsEnum termsEnum;
+    
+    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, IndexInput data) throws IOException {
+      this.bytes = bytes;
+      this.interval = bytes.addressInterval;
+      this.addresses = addresses;
+      this.data = data;
+      this.numValues = bytes.count;
+      this.numIndexValues = addresses.size();
+      this.termsEnum = getTermsEnum(data);
+    }
+    
+    @Override
+    public void get(long id, BytesRef result) {
+      try {
+        termsEnum.seekExact(id);
+        BytesRef term = termsEnum.term();
+        result.bytes = term.bytes;
+        result.offset = term.offset;
+        result.length = term.length;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    long lookupTerm(BytesRef key) {
+      try {
+        SeekStatus status = termsEnum.seekCeil(key);
+        if (status == SeekStatus.END) {
+          return -numValues-1;
+        } else if (status == SeekStatus.FOUND) {
+          return termsEnum.ord();
+        } else {
+          return -termsEnum.ord()-1;
+        }
+      } catch (IOException bogus) {
+        throw new RuntimeException(bogus);
+      }
+    }
+    
+    TermsEnum getTermsEnum() {
+      try {
+        return getTermsEnum(data.clone());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    private TermsEnum getTermsEnum(final IndexInput input) throws IOException {
+      input.seek(bytes.offset);
+      
+      return new TermsEnum() {
+        private long currentOrd = -1;
+        // TODO: maxLength is negative when all terms are merged away...
+        private final BytesRef termBuffer = new BytesRef(bytes.maxLength < 0 ? 0 : bytes.maxLength);
+        private final BytesRef term = new BytesRef(); // TODO: paranoia?
+
+        @Override
+        public BytesRef next() throws IOException {
+          if (doNext() == null) {
+            return null;
+          } else {
+            setTerm();
+            return term;
+          }
+        }
+        
+        private BytesRef doNext() throws IOException {
+          if (++currentOrd >= numValues) {
+            return null;
+          } else {
+            int start = input.readVInt();
+            int suffix = input.readVInt();
+            input.readBytes(termBuffer.bytes, start, suffix);
+            termBuffer.length = start + suffix;
+            return termBuffer;
+          }
+        }
+
+        @Override
+        public SeekStatus seekCeil(BytesRef text) throws IOException {
+          // binary-search just the index values to find the block,
+          // then scan within the block
+          long low = 0;
+          long high = numIndexValues-1;
+
+          while (low <= high) {
+            long mid = (low + high) >>> 1;
+            doSeek(mid * interval);
+            int cmp = termBuffer.compareTo(text);
+
+            if (cmp < 0) {
+              low = mid + 1;
+            } else if (cmp > 0) {
+              high = mid - 1;
+            } else {
+              // we got lucky, found an indexed term
+              setTerm();
+              return SeekStatus.FOUND;
+            }
+          }
+          
+          if (numIndexValues == 0) {
+            return SeekStatus.END;
+          }
+          
+          // block before insertion point
+          long block = low-1;
+          doSeek(block < 0 ? -1 : block * interval);
+          
+          while (doNext() != null) {
+            int cmp = termBuffer.compareTo(text);
+            if (cmp == 0) {
+              setTerm();
+              return SeekStatus.FOUND;
+            } else if (cmp > 0) {
+              setTerm();
+              return SeekStatus.NOT_FOUND;
+            }
+          }
+          
+          return SeekStatus.END;
+        }
+
+        @Override
+        public void seekExact(long ord) throws IOException {
+          doSeek(ord);
+          setTerm();
+        }
+        
+        private void doSeek(long ord) throws IOException {
+          long block = ord / interval;
+
+          if (ord >= currentOrd && block == currentOrd / interval) {
+            // seek within current block
+          } else {
+            // position before start of block
+            currentOrd = ord - ord % interval - 1;
+            input.seek(bytes.offset + addresses.get(block));
+          }
+          
+          while (currentOrd < ord) {
+            doNext();
+          }
+        }
+        
+        private void setTerm() {
+          // TODO: is there a cleaner way
+          term.bytes = new byte[termBuffer.length];
+          term.offset = 0;
+          term.copyBytes(termBuffer);
+        }
+
+        @Override
+        public BytesRef term() throws IOException {
+          return term;
+        }
+
+        @Override
+        public long ord() throws IOException {
+          return currentOrd;
+        }
+        
+        @Override
+        public Comparator<BytesRef> getComparator() {
+          return BytesRef.getUTF8SortedAsUnicodeComparator();
+        }
+
+        @Override
+        public int docFreq() throws IOException {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public long totalTermFreq() throws IOException {
+          return -1;
+        }
+
+        @Override
+        public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -684,7 +684,7 @@ public final class DirectPostingsFormat 
       }
 
       @Override
-      public SeekStatus seekCeil(BytesRef term, boolean useCache) {
+      public SeekStatus seekCeil(BytesRef term) {
         // TODO: we should use the skip pointers; should be
         // faster than bin search; we should also hold
         // & reuse current state so seeking forwards is
@@ -707,7 +707,7 @@ public final class DirectPostingsFormat 
       }
 
       @Override
-      public boolean seekExact(BytesRef term, boolean useCache) {
+      public boolean seekExact(BytesRef term) {
         // TODO: we should use the skip pointers; should be
         // faster than bin search; we should also hold
         // & reuse current state so seeking forwards is
@@ -1413,7 +1413,7 @@ public final class DirectPostingsFormat 
       }
 
       @Override
-      public SeekStatus seekCeil(BytesRef term, boolean useCache) {
+      public SeekStatus seekCeil(BytesRef term) {
         throw new UnsupportedOperationException();
       }
 

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -335,11 +335,11 @@ public final class MemoryPostingsFormat 
     
     public FSTDocsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
       assert numDocs > 0;
-      if (buffer.length < bufferIn.length - bufferIn.offset) {
-        buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
+      if (buffer.length < bufferIn.length) {
+        buffer = ArrayUtil.grow(buffer, bufferIn.length);
       }
-      in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
-      System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
+      in.reset(buffer, 0, bufferIn.length);
+      System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length);
       this.liveDocs = liveDocs;
       docID = -1;
       accum = 0;
@@ -472,11 +472,11 @@ public final class MemoryPostingsFormat 
       //   System.out.println("  " + Integer.toHexString(bufferIn.bytes[i]&0xFF));
       // }
 
-      if (buffer.length < bufferIn.length - bufferIn.offset) {
-        buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
+      if (buffer.length < bufferIn.length) {
+        buffer = ArrayUtil.grow(buffer, bufferIn.length);
       }
       in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
-      System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
+      System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length);
       this.liveDocs = liveDocs;
       docID = -1;
       accum = 0;
@@ -632,6 +632,7 @@ public final class MemoryPostingsFormat 
     private int docFreq;
     private long totalTermFreq;
     private BytesRefFSTEnum.InputOutput<BytesRef> current;
+    private BytesRef postingsSpare = new BytesRef();
 
     public FSTTermsEnum(FieldInfo field, FST<BytesRef> fst) {
       this.field = field;
@@ -640,21 +641,23 @@ public final class MemoryPostingsFormat 
 
     private void decodeMetaData() {
       if (!didDecode) {
-        buffer.reset(current.output.bytes, 0, current.output.length);
+        buffer.reset(current.output.bytes, current.output.offset, current.output.length);
         docFreq = buffer.readVInt();
         if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
           totalTermFreq = docFreq + buffer.readVLong();
         } else {
           totalTermFreq = -1;
         }
-        current.output.offset = buffer.getPosition();
+        postingsSpare.bytes = current.output.bytes;
+        postingsSpare.offset = buffer.getPosition();
+        postingsSpare.length = current.output.length - (buffer.getPosition() - current.output.offset);
         //System.out.println("  df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length);
         didDecode = true;
       }
     }
 
     @Override
-    public boolean seekExact(BytesRef text, boolean useCache /* ignored */) throws IOException {
+    public boolean seekExact(BytesRef text) throws IOException {
       //System.out.println("te.seekExact text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
       current = fstEnum.seekExact(text);
       didDecode = false;
@@ -662,7 +665,7 @@ public final class MemoryPostingsFormat 
     }
 
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
       //System.out.println("te.seek text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
       current = fstEnum.seekCeil(text);
       if (current == null) {
@@ -699,7 +702,7 @@ public final class MemoryPostingsFormat 
           docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
         }
       }
-      return docsEnum.reset(current.output, liveDocs, docFreq);
+      return docsEnum.reset(this.postingsSpare, liveDocs, docFreq);
     }
 
     @Override
@@ -720,7 +723,7 @@ public final class MemoryPostingsFormat 
         }
       }
       //System.out.println("D&P reset this=" + this);
-      return docsAndPositionsEnum.reset(current.output, liveDocs, docFreq);
+      return docsAndPositionsEnum.reset(postingsSpare, liveDocs, docFreq);
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -103,8 +103,7 @@ public abstract class PulsingPostingsFor
                                                     state.directory, state.fieldInfos, state.segmentInfo,
                                                     pulsingReader,
                                                     state.context,
-                                                    state.segmentSuffix,
-                                                    state.termsIndexDivisor);
+                                                    state.segmentSuffix);
       success = true;
       return ret;
     } finally {

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Sun Aug 11 12:19:13 2013
@@ -109,7 +109,7 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
-    public boolean seekExact(BytesRef text, boolean useCache /* ignored */) throws IOException {
+    public boolean seekExact(BytesRef text) throws IOException {
 
       final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> result = fstEnum.seekExact(text);
       if (result != null) {
@@ -125,7 +125,7 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
 
       //System.out.println("seek to text=" + text.utf8ToString());
       final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> result = fstEnum.seekCeil(text);
@@ -513,7 +513,7 @@ class SimpleTextFieldsReader extends Fie
     }
 
     private void loadTerms() throws IOException {
-      PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton(false);
+      PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton();
       final Builder<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> b;
       final PairOutputs<Long,Long> outputsInner = new PairOutputs<Long,Long>(posIntOutputs, posIntOutputs);
       final PairOutputs<Long,PairOutputs.Pair<Long,Long>> outputs = new PairOutputs<Long,PairOutputs.Pair<Long,Long>>(posIntOutputs,

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Sun Aug 11 12:19:13 2013
@@ -331,7 +331,7 @@ public class SimpleTextTermVectorsReader
     }
     
     @Override
-    public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
       iterator = terms.tailMap(text).entrySet().iterator();
       if (!iterator.hasNext()) {
         return SeekStatus.END;

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -25,10 +25,8 @@ import org.apache.lucene.util._TestUtil;
 /**
  * Basic tests of a PF using FixedGap terms dictionary
  */
-// TODO: we should add an instantiation for VarGap too to TestFramework, and a test in this package
-// TODO: ensure both of these are also in rotation in RandomCodec
 public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
-  private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41WithOrds());
+  private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41WithOrds(_TestUtil.nextInt(random(), 1, 1000)));
 
   @Override
   protected Codec getCodec() {

Modified: lucene/dev/branches/lucene4956/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/common-build.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/common-build.xml (original)
+++ lucene/dev/branches/lucene4956/lucene/common-build.xml Sun Aug 11 12:19:13 2013
@@ -186,6 +186,7 @@
   <property name="build.encoding" value="utf-8"/>
 
   <property name="src.dir" location="src/java"/>
+  <property name="resources.dir" location="${src.dir}/../resources"/>
   <property name="tests.src.dir" location="src/test"/>
   <available property="module.has.tests" type="dir" file="${tests.src.dir}"/>
   <property name="build.dir" location="build"/>
@@ -229,9 +230,6 @@
   <property name="svn.exe" value="svn" />
   <property name="perl.exe" value="perl" />
   
-  <property name="hg.exe" value="hg" />
-  <property name="moman.url" value="https://bitbucket.org/jpbarrette/moman" />
-  <property name="moman.rev" value="120" />
   <property name="python.exe" value="python" />
   <property name="python32.exe" value="python3.2" />
 
@@ -255,6 +253,10 @@
 
   <!-- a reasonable default exclusion set, can be overridden for special cases -->
   <property name="rat.excludes" value="**/TODO,**/*.txt,**/*.iml"/>
+  
+  <!-- These patterns can be defined to add additional files for checks, relative to module's home dir -->
+  <property name="rat.additional-includes" value=""/>
+  <property name="rat.additional-excludes" value=""/>
 
   <propertyset id="uptodate.and.compiled.properties" dynamic="true">
     <propertyref regex=".*\.uptodate$$"/>
@@ -351,7 +353,7 @@
   <target name="resolve" depends="ivy-availability-check,ivy-configure">
     <!-- todo, make this a property or something. 
          only special cases need bundles -->
-    <ivy:retrieve type="jar,bundle" log="download-only" 
+    <ivy:retrieve type="jar,bundle,tests" log="download-only"
                   conf="${ivy.default.configuration}" sync="${ivy.sync}"/>
   </target>
 
@@ -440,20 +442,6 @@
     </sequential>
   </macrodef>
 
-  <target name="jflex-uptodate-check">
-    <uptodate property="jflex.files.uptodate">
-      <srcfiles dir="${src.dir}" includes="**/*.jflex" />
-      <mapper type="glob" from="*.jflex" to="*.java"/>
-    </uptodate>
-  </target>
- 
-  <target name="jflex-notice" depends="jflex-uptodate-check" unless="jflex.files.uptodate">
-    <echo>
-      One or more of the JFlex .jflex files is newer than its corresponding
-      .java file.  Run the "jflex" target to regenerate the artifacts.
-    </echo>
-  </target>
-
   <target name="jflex-check">
     <available property="jflex.present" classname="jflex.anttask.JFlexTask">
       <classpath refid="jflex.classpath"/>
@@ -466,7 +454,7 @@
       Please install the jFlex 1.5 version (currently not released)
       from its SVN repository:
 
-       svn co http://jflex.svn.sourceforge.net/svnroot/jflex/trunk jflex
+       svn co -r 623 http://jflex.svn.sourceforge.net/svnroot/jflex/trunk jflex
        cd jflex
        mvn install
 
@@ -489,7 +477,7 @@
 
     <!-- Copy the resources folder (if existent) -->
     <copy todir="${build.dir}/classes/java">
-      <fileset dir="${src.dir}/../resources" erroronmissingdir="no"/>
+      <fileset dir="${resources.dir}" erroronmissingdir="no"/>
     </copy>
   </target>
 
@@ -864,6 +852,7 @@
         <mkdir dir="${tests.cachedir}/${name}" />
 
         <junit4:junit4
+            taskName="junit4"
             dir="@{workDir}"
             tempdir="@{workDir}/temp"
             maxmemory="${tests.heapsize}" 
@@ -962,6 +951,7 @@
                 <propertyref prefix="tests.badapples" />
                 <propertyref prefix="tests.timeoutSuite" />
                 <propertyref prefix="tests.jettyConnector" />
+                <propertyref prefix="tests.disableHdfs" />
             </syspropertyset>
 
             <!-- Pass randomized settings to the forked JVM. -->
@@ -1339,7 +1329,7 @@ ${tests-output}/junit4-*.suites     - pe
     <sequential>
       <mkdir dir="${build.dir}" />
       <jarify basedir="${src.dir}" destfile="${build.dir}/${final.name}-src.jar">
-        <fileset dir="${src.dir}/../resources" erroronmissingdir="no"/>
+        <fileset dir="${resources.dir}" erroronmissingdir="no"/>
       </jarify>
     </sequential>
   </target>
@@ -1454,7 +1444,7 @@ ${tests-output}/junit4-*.suites     - pe
 
   <target name="filter-pom-templates" unless="filtered.pom.templates.uptodate">
     <mkdir dir="${filtered.pom.templates.dir}"/>
-    <copy todir="${common.dir}/build/poms" overwrite="true">
+    <copy todir="${common.dir}/build/poms" overwrite="true" encoding="UTF-8">
       <fileset dir="${common.dir}/../dev-tools/maven"/>
       <filterset begintoken="@" endtoken="@">
         <filter token="version" value="${version}"/>
@@ -1508,28 +1498,34 @@ ${tests-output}/junit4-*.suites     - pe
   </target>
 
   <target name="rat-sources-typedef" unless="rat.loaded">
-    <ivy:cachepath organisation="org.apache.rat" module="apache-rat" revision="0.8" transitive="false" inline="true" conf="master" type="jar" pathid="rat.classpath"/>
+    <ivy:cachepath organisation="org.apache.rat" module="apache-rat" revision="0.9" transitive="false" inline="true" conf="master" type="jar" pathid="rat.classpath"/>
     <typedef resource="org/apache/rat/anttasks/antlib.xml" uri="antlib:org.apache.rat.anttasks" classpathref="rat.classpath"/>
     <property name="rat.loaded" value="true"/>
   </target>
 
   <target name="rat-sources" depends="rat-sources-typedef"
 	  description="runs the tasks over source and test files">
-    <sequential>
     <!-- create a temp file for the log to go to -->
     <tempfile property="rat.sources.logfile"
               prefix="rat"
               destdir="${java.io.tmpdir}"/>
     <!-- run rat, going to the file -->
     <rat:report xmlns:rat="antlib:org.apache.rat.anttasks" 
-                reportFile="${rat.sources.logfile}">
-      <fileset dir="${src.dir}" excludes="${rat.excludes}"/>
+                reportFile="${rat.sources.logfile}" addDefaultLicenseMatchers="true">
+      <fileset dir="." includes="*.xml ${rat.additional-includes}" excludes="${rat.additional-excludes}"/>
+      <fileset dir="${src.dir}" excludes="${rat.excludes}" erroronmissingdir="false"/>
       <fileset dir="${tests.src.dir}" excludes="${rat.excludes}" erroronmissingdir="false"/>
-      <!-- some modules have a src/tools/[java,test] -->
-      <fileset dir="src/tools/java" excludes="${rat.excludes}" erroronmissingdir="false"/>
-      <fileset dir="src/tools/test" excludes="${rat.excludes}" erroronmissingdir="false"/>
+
+      <!-- TODO: Check all resource files. Currently not all stopword and similar files have no header! -->
+      <fileset dir="${resources.dir}" includes="META-INF/**" erroronmissingdir="false"/>
       
-      <!-- bsd-like stuff -->
+      <!-- BSD 4-clause stuff (is disallowed below) -->
+      <rat:substringMatcher licenseFamilyCategory="BSD4 "
+             licenseFamilyName="Original BSD License (with advertising clause)">
+        <pattern substring="All advertising materials"/>
+      </rat:substringMatcher>
+
+      <!-- BSD-like stuff -->
       <rat:substringMatcher licenseFamilyCategory="BSD  "
              licenseFamilyName="Modified BSD License">
       <!-- brics automaton -->
@@ -1542,16 +1538,20 @@ ${tests-output}/junit4-*.suites     - pe
         <pattern substring="Egothor Software License version 1.00"/>
       <!-- JaSpell -->
         <pattern substring="Copyright (c) 2005 Bruno Martins"/>
+      <!-- d3.js -->
+        <pattern substring="THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS"/>
+      <!-- highlight.js -->
+        <pattern substring="THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS"/>
       </rat:substringMatcher>
 
-      <!-- mit-like -->
+      <!-- MIT-like -->
       <rat:substringMatcher licenseFamilyCategory="MIT  "
              licenseFamilyName="The MIT License">
       <!-- ICU license -->
         <pattern substring="Permission is hereby granted, free of charge, to any person obtaining a copy"/>
       </rat:substringMatcher>
 
-      <!-- apache -->
+      <!-- Apache -->
       <rat:substringMatcher licenseFamilyCategory="AL   "
              licenseFamilyName="Apache">
         <pattern substring="Licensed to the Apache Software Foundation (ASF) under"/>
@@ -1577,13 +1577,13 @@ ${tests-output}/junit4-*.suites     - pe
     </rat:report>
     <!-- now print the output, for review -->
     <loadfile property="rat.output" srcFile="${rat.sources.logfile}"/>
-    <echo>${rat.output}</echo>
+    <echo taskname="rat">${rat.output}</echo>
     <delete>
       <fileset file="${rat.sources.logfile}">
         <and>
           <containsregexp expression="^0 Unknown Licenses"/>
           <not>
-            <containsregexp expression="^\s+!AL"/>
+            <containsregexp expression="^\s+!"/>
           </not>
         </and>
       </fileset>
@@ -1594,7 +1594,6 @@ ${tests-output}/junit4-*.suites     - pe
         <available file="${rat.sources.logfile}"/>
       </condition>
     </fail>
-    </sequential>
   </target>
 
   <!--+
@@ -1814,10 +1813,71 @@ ${tests-output}/junit4-*.suites     - pe
         </condition>
       </fail>
 
-
+      <patch-javadoc dir="@{destdir}" docencoding="${javadoc.charset}"/>
    </sequential>
   </macrodef>
 
+  <!--
+    Patch frame injection bugs in javadoc generated files - see CVE-2013-1571, http://www.kb.cert.org/vuls/id/225657
+    
+    Feel free to use this macro in your own Ant build file. This macro works together with the javadoc task on Ant
+    and should be invoked directly after its execution to patch broken javadocs, e.g.:
+      <patch-javadoc dir="..." docencoding="UTF-8"/>
+    Please make sure that the docencoding parameter uses the same charset like javadoc's docencoding. Default
+    is the platform default encoding (like the javadoc task).
+    The specified dir is the destination directory of the javadoc task.
+  -->
+  <macrodef name="patch-javadoc">
+    <attribute name="dir"/>
+    <attribute name="docencoding" default="${file.encoding}"/>
+    <sequential>
+      <replace encoding="@{docencoding}" summary="true" taskname="patch-javadoc">
+        <fileset dir="@{dir}" casesensitive="false" includes="**/index.html,**/index.htm,**/toc.html,**/toc.htm">
+          <!-- TODO: add encoding="@{docencoding}" to contains check, when we are on ANT 1.9.0: -->
+          <not><contains text="function validURL(url) {" casesensitive="true" /></not>
+        </fileset>
+        <replacetoken><![CDATA[function loadFrames() {]]></replacetoken>
+        <replacevalue expandProperties="false"><![CDATA[if (targetPage != "" && !validURL(targetPage))
+        targetPage = "undefined";
+    function validURL(url) {
+        var pos = url.indexOf(".html");
+        if (pos == -1 || pos != url.length - 5)
+            return false;
+        var allowNumber = false;
+        var allowSep = false;
+        var seenDot = false;
+        for (var i = 0; i < url.length - 5; i++) {
+            var ch = url.charAt(i);
+            if ('a' <= ch && ch <= 'z' ||
+                    'A' <= ch && ch <= 'Z' ||
+                    ch == '$' ||
+                    ch == '_') {
+                allowNumber = true;
+                allowSep = true;
+            } else if ('0' <= ch && ch <= '9'
+                    || ch == '-') {
+                if (!allowNumber)
+                     return false;
+            } else if (ch == '/' || ch == '.') {
+                if (!allowSep)
+                    return false;
+                allowNumber = false;
+                allowSep = false;
+                if (ch == '.')
+                     seenDot = true;
+                if (ch == '/' && seenDot)
+                     return false;
+            } else {
+                return false;
+            }
+        }
+        return true;
+    }
+    function loadFrames() {]]></replacevalue>
+      </replace>
+    </sequential>
+  </macrodef>
+
   <macrodef name="modules-crawl">
     <attribute name="target" default=""/>
     <attribute name="failonerror" default="true"/>
@@ -2002,7 +2062,7 @@ ${tests-output}/junit4-*.suites     - pe
   
   <!-- GROOVY scripting engine for ANT tasks -->
   <target name="resolve-groovy" unless="groovy.loaded" depends="ivy-availability-check,ivy-configure">
-    <ivy:cachepath organisation="org.codehaus.groovy" module="groovy-all" revision="2.0.4"
+    <ivy:cachepath organisation="org.codehaus.groovy" module="groovy-all" revision="2.1.5"
       inline="true" conf="default" type="jar" transitive="true" pathid="groovy.classpath"/>
     <property name="groovy.loaded" value="true"/>
   </target>
@@ -2015,14 +2075,16 @@ ${tests-output}/junit4-*.suites     - pe
     <property name="forbidden-apis.loaded" value="true"/>
   </target>
   
-  <!-- PEGDOWN macro: Before using depend on the target "resolve-pegdown" -->
+  <!-- PEGDOWN macro: Before using depend on the target "resolve-pegdown,resolve-groovy" -->
   
   <target name="resolve-pegdown" unless="pegdown.loaded" depends="ivy-availability-check,ivy-configure">
-    <ivy:cachepath organisation="org.pegdown" module="pegdown" revision="1.2.1"
-      inline="true" conf="default" type="jar" transitive="true" pathid="pegdown.classpath"/>
+    <ivy:cachepath organisation="org.pegdown" module="pegdown" revision="1.4.0"
+      inline="true" conf="default" transitive="true" pathid="pegdown.classpath"/>
     <property name="pegdown.loaded" value="true"/>
   </target>
   
+  <target name="regenerate"/>
+	
   <macrodef name="pegdown">
     <attribute name="todir"/>
     <attribute name="flatten" default="false"/>
@@ -2030,7 +2092,7 @@ ${tests-output}/junit4-*.suites     - pe
     <element name="nested" optional="false" implicit="true"/>
     <sequential>
       <copy todir="@{todir}" flatten="@{flatten}" overwrite="@{overwrite}" verbose="true"
-        preservelastmodified="false" encoding="UTF-8" outputencoding="UTF-8" taskname="pegdown"
+        preservelastmodified="false" encoding="UTF-8" taskname="pegdown"
       >
         <filterchain>
           <tokenfilter>

Modified: lucene/dev/branches/lucene4956/lucene/core/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/build.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/build.xml (original)
+++ lucene/dev/branches/lucene4956/lucene/core/build.xml Sun Aug 11 12:19:13 2013
@@ -24,6 +24,9 @@
 
   <import file="../common-build.xml"/>
 
+  <property name="moman.commit-hash" value="5c5c2a1e4dea" />
+  <property name="moman.url" value="https://bitbucket.org/jpbarrette/moman/get/${moman.commit-hash}.zip" />
+
   <path id="classpath"/>
   
   <path id="test.classpath">
@@ -109,36 +112,24 @@
     <fixcrlf srcdir="src/java/org/apache/lucene/util/packed" includes="BulkOperation*.java,Direct*.java,Packed64SingleBlock.java,Packed*ThreeBlocks.py" encoding="UTF-8"/>
   </target>
 
-  <target name="createLevAutomata" depends="check-moman,clone-moman,pull-moman">
+  <target name="createLevAutomata" depends="check-moman,download-moman">
     <createLevAutomaton n="1"/>
     <createLevAutomaton n="2"/>
   </target>
   
   <target name="check-moman">
-    <condition property="moman.cloned">
-      <available file="${build.dir}/moman"/>
-    </condition>
+    <available file="${build.dir}/moman" property="moman.downloaded"/>
   </target>
 
-  <target name="clone-moman" unless="moman.cloned">
-    <mkdir dir="${build.dir}"/>
-    <exec dir="${build.dir}" 
-          executable="${hg.exe}" failonerror="true">
-      <arg value="clone"/>
-      <arg value="-r"/>
-      <arg value="${moman.rev}"/>
-      <arg value="${moman.url}"/>
-      <arg value="moman"/>
-    </exec>
+  <target name="download-moman" unless="moman.downloaded">
+    <mkdir dir="${build.dir}/moman"/>
+    <get src="${moman.url}" dest="${build.dir}/moman.zip"/>
+    <unzip dest="${build.dir}/moman" src="${build.dir}/moman.zip">
+      <cutdirsmapper dirs="1"/>
+    </unzip>
+    <delete file="${build.dir}/moman.zip"/>
   </target>
 
-  <target name="pull-moman" if="moman.cloned">
-    <exec dir="${build.dir}/moman" 
-          executable="${hg.exe}" failonerror="true">
-      <arg value="pull"/>
-      <arg value="-f"/>
-      <arg value="-r"/>
-      <arg value="${moman.rev}"/>
-    </exec>
-  </target>
+  <target name="regenerate" depends="createLevAutomata,createPackedIntSources"/>
+
 </project>

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java Sun Aug 11 12:19:13 2013
@@ -120,11 +120,16 @@ public abstract class Analyzer implement
    * See the {@link org.apache.lucene.analysis Analysis package documentation} for
    * some examples demonstrating this.
    * 
+   * <b>NOTE:</b> If your data is available as a {@code String}, use
+   * {@link #tokenStream(String, String)} which reuses a {@code StringReader}-like
+   * instance internally.
+   * 
    * @param fieldName the name of the field the created TokenStream is used for
    * @param reader the reader the streams source reads from
    * @return TokenStream for iterating the analyzed content of <code>reader</code>
    * @throws AlreadyClosedException if the Analyzer is closed.
    * @throws IOException if an i/o error occurs.
+   * @see #tokenStream(String, String)
    */
   public final TokenStream tokenStream(final String fieldName,
                                        final Reader reader) throws IOException {
@@ -140,6 +145,45 @@ public abstract class Analyzer implement
   }
   
   /**
+   * Returns a TokenStream suitable for <code>fieldName</code>, tokenizing
+   * the contents of <code>text</code>.
+   * <p>
+   * This method uses {@link #createComponents(String, Reader)} to obtain an
+   * instance of {@link TokenStreamComponents}. It returns the sink of the
+   * components and stores the components internally. Subsequent calls to this
+   * method will reuse the previously stored components after resetting them
+   * through {@link TokenStreamComponents#setReader(Reader)}.
+   * <p>
+   * <b>NOTE:</b> After calling this method, the consumer must follow the 
+   * workflow described in {@link TokenStream} to properly consume its contents.
+   * See the {@link org.apache.lucene.analysis Analysis package documentation} for
+   * some examples demonstrating this.
+   * 
+   * @param fieldName the name of the field the created TokenStream is used for
+   * @param text the String the streams source reads from
+   * @return TokenStream for iterating the analyzed content of <code>reader</code>
+   * @throws AlreadyClosedException if the Analyzer is closed.
+   * @throws IOException if an i/o error occurs (may rarely happen for strings).
+   * @see #tokenStream(String, Reader)
+   */
+  public final TokenStream tokenStream(final String fieldName, final String text) throws IOException {
+    TokenStreamComponents components = reuseStrategy.getReusableComponents(fieldName);
+    @SuppressWarnings("resource") final ReusableStringReader strReader = 
+        (components == null || components.reusableStringReader == null) ?
+        new ReusableStringReader() : components.reusableStringReader;
+    strReader.setValue(text);
+    final Reader r = initReader(fieldName, strReader);
+    if (components == null) {
+      components = createComponents(fieldName, r);
+      reuseStrategy.setReusableComponents(fieldName, components);
+    } else {
+      components.setReader(r);
+    }
+    components.reusableStringReader = strReader;
+    return components.getTokenStream();
+  }
+    
+  /**
    * Override this if you want to add a CharFilter chain.
    * <p>
    * The default implementation returns <code>reader</code>
@@ -208,6 +252,9 @@ public abstract class Analyzer implement
      * the chain. This can be the source if there are no filters.
      */
     protected final TokenStream sink;
+    
+    /** Internal cache only used by {@link Analyzer#tokenStream(String, String)}. */
+    transient ReusableStringReader reusableStringReader;
 
     /**
      * Creates a new {@link TokenStreamComponents} instance.

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java Sun Aug 11 12:19:13 2013
@@ -51,16 +51,35 @@ public abstract class AnalyzerWrapper ex
 
   /**
    * Wraps / alters the given TokenStreamComponents, taken from the wrapped
-   * Analyzer, to form new components.  It is through this method that new
-   * TokenFilters can be added by AnalyzerWrappers.
-   *
-   *
-   * @param fieldName Name of the field which is to be analyzed
-   * @param components TokenStreamComponents taken from the wrapped Analyzer
+   * Analyzer, to form new components. It is through this method that new
+   * TokenFilters can be added by AnalyzerWrappers. By default, the given
+   * components are returned.
+   * 
+   * @param fieldName
+   *          Name of the field which is to be analyzed
+   * @param components
+   *          TokenStreamComponents taken from the wrapped Analyzer
    * @return Wrapped / altered TokenStreamComponents.
    */
-  protected abstract TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components);
+  protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
+    return components;
+  }
 
+  /**
+   * Wraps / alters the given Reader. Through this method AnalyzerWrappers can
+   * implement {@link #initReader(String, Reader)}. By default, the given reader
+   * is returned.
+   * 
+   * @param fieldName
+   *          name of the field which is to be analyzed
+   * @param reader
+   *          the reader to wrap
+   * @return the wrapped reader
+   */
+  protected Reader wrapReader(String fieldName, Reader reader) {
+    return reader;
+  }
+  
   @Override
   protected final TokenStreamComponents createComponents(String fieldName, Reader aReader) {
     return wrapComponents(fieldName, getWrappedAnalyzer(fieldName).createComponents(fieldName, aReader));
@@ -78,6 +97,6 @@ public abstract class AnalyzerWrapper ex
 
   @Override
   public final Reader initReader(String fieldName, Reader reader) {
-    return getWrappedAnalyzer(fieldName).initReader(fieldName, reader);
+    return getWrappedAnalyzer(fieldName).initReader(fieldName, wrapReader(fieldName, reader));
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java Sun Aug 11 12:19:13 2013
@@ -32,7 +32,8 @@ import org.apache.lucene.util.automaton.
 // TODO: maybe also toFST?  then we can translate atts into FST outputs/weights
 
 /** Consumes a TokenStream and creates an {@link Automaton}
- *  where the transition labels are UTF8 bytes from the {@link
+ *  where the transition labels are UTF8 bytes (or Unicode 
+ *  code points if unicodeArcs is true) from the {@link
  *  TermToBytesRefAttribute}.  Between tokens we insert
  *  POS_SEP and for holes we insert HOLE.
  *
@@ -40,6 +41,7 @@ import org.apache.lucene.util.automaton.
 public class TokenStreamToAutomaton {
 
   private boolean preservePositionIncrements;
+  private boolean unicodeArcs;
 
   /** Sole constructor. */
   public TokenStreamToAutomaton() {
@@ -51,6 +53,12 @@ public class TokenStreamToAutomaton {
     this.preservePositionIncrements = enablePositionIncrements;
   }
 
+  /** Whether to make transition labels Unicode code points instead of UTF8 bytes, 
+   *  <code>false</code> by default */
+  public void setUnicodeArcs(boolean unicodeArcs) {
+    this.unicodeArcs = unicodeArcs;
+  }
+
   private static class Position implements RollingBuffer.Resettable {
     // Any tokens that ended at our position arrive to this state:
     State arriving;
@@ -80,15 +88,16 @@ public class TokenStreamToAutomaton {
   }
 
   /** We create transition between two adjacent tokens. */
-  public static final int POS_SEP = 256;
+  public static final int POS_SEP = 0x001f;
 
   /** We add this arc to represent a hole. */
-  public static final int HOLE = 257;
+  public static final int HOLE = 0x001e;
 
   /** Pulls the graph (including {@link
    *  PositionLengthAttribute}) from the provided {@link
    *  TokenStream}, and creates the corresponding
-   *  automaton where arcs are bytes from each term. */
+   *  automaton where arcs are bytes (or Unicode code points 
+   *  if unicodeArcs = true) from each term. */
   public Automaton toAutomaton(TokenStream in) throws IOException {
     final Automaton a = new Automaton();
     boolean deterministic = true;
@@ -156,16 +165,34 @@ public class TokenStreamToAutomaton {
       final int endPos = pos + posLengthAtt.getPositionLength();
 
       termBytesAtt.fillBytesRef();
-      final BytesRef term2 = changeToken(term);
+      final BytesRef termUTF8 = changeToken(term);
+      int[] termUnicode = null;
       final Position endPosData = positions.get(endPos);
       if (endPosData.arriving == null) {
         endPosData.arriving = new State();
       }
 
       State state = posData.leaving;
-      for(int byteIDX=0;byteIDX<term2.length;byteIDX++) {
-        final State nextState = byteIDX == term2.length-1 ? endPosData.arriving : new State();
-        state.addTransition(new Transition(term2.bytes[term2.offset + byteIDX] & 0xff, nextState));
+      int termLen;
+      if (unicodeArcs) {
+        final String utf16 = termUTF8.utf8ToString();
+        termUnicode = new int[utf16.codePointCount(0, utf16.length())];
+        termLen = termUnicode.length;
+        for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp))
+          termUnicode[j++] = cp = utf16.codePointAt(i);
+      } else {
+        termLen = termUTF8.length;
+      }
+
+      for(int byteIDX=0;byteIDX<termLen;byteIDX++) {
+        final State nextState = byteIDX == termLen-1 ? endPosData.arriving : new State();
+        int c;
+        if (unicodeArcs) {
+          c = termUnicode[byteIDX];
+        } else {
+          c = termUTF8.bytes[termUTF8.offset + byteIDX] & 0xff;
+        }
+        state.addTransition(new Transition(c, nextState));
         state = nextState;
       }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java Sun Aug 11 12:19:13 2013
@@ -50,10 +50,6 @@ public class BlockTermState extends OrdT
     totalTermFreq = other.totalTermFreq;
     termBlockOrd = other.termBlockOrd;
     blockFilePointer = other.blockFilePointer;
-
-    // NOTE: don't copy blockTermCount;
-    // it's "transient": used only by the "primary"
-    // termState, and regenerated on seek by TermState
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Sun Aug 11 12:19:13 2013
@@ -67,9 +67,9 @@ import org.apache.lucene.util.fst.Util;
  *  does not support a pluggable terms index
  *  implementation).
  *
- *  <p><b>NOTE</b>: this terms dictionary does not support
- *  index divisor when opening an IndexReader.  Instead, you
- *  can change the min/maxItemsPerBlock during indexing.</p>
+ *  <p><b>NOTE</b>: this terms dictionary supports
+ *  min/maxItemsPerBlock during indexing to control how
+ *  much memory the terms index uses.</p>
  *
  *  <p>The data structure used by this implementation is very
  *  similar to a burst trie
@@ -112,7 +112,7 @@ public class BlockTreeTermsReader extend
   /** Sole constructor. */
   public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
                               PostingsReaderBase postingsReader, IOContext ioContext,
-                              String segmentSuffix, int indexDivisor)
+                              String segmentSuffix)
     throws IOException {
     
     this.postingsReader = postingsReader;
@@ -126,13 +126,11 @@ public class BlockTreeTermsReader extend
 
     try {
       version = readHeader(in);
-      if (indexDivisor != -1) {
-        indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
+      indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
                                 ioContext);
-        int indexVersion = readIndexHeader(indexIn);
-        if (indexVersion != version) {
-          throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
-        }
+      int indexVersion = readIndexHeader(indexIn);
+      if (indexVersion != version) {
+        throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
       }
 
       // Have PostingsReader init itself
@@ -140,9 +138,7 @@ public class BlockTreeTermsReader extend
 
       // Read per-field details
       seekDir(in, dirOffset);
-      if (indexDivisor != -1) {
-        seekDir(indexIn, indexDirOffset);
-      }
+      seekDir(indexIn, indexDirOffset);
 
       final int numFields = in.readVInt();
       if (numFields < 0) {
@@ -171,15 +167,13 @@ public class BlockTreeTermsReader extend
         if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
           throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
         }
-        final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
+        final long indexStartFP = indexIn.readVLong();
         FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, indexIn));
         if (previous != null) {
           throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
         }
       }
-      if (indexDivisor != -1) {
-        indexIn.close();
-      }
+      indexIn.close();
 
       success = true;
     } finally {
@@ -1222,7 +1216,7 @@ public class BlockTreeTermsReader extend
       }
 
       @Override
-      public boolean seekExact(BytesRef text, boolean useCache) {
+      public boolean seekExact(BytesRef text) {
         throw new UnsupportedOperationException();
       }
 
@@ -1237,7 +1231,7 @@ public class BlockTreeTermsReader extend
       }
 
       @Override
-      public SeekStatus seekCeil(BytesRef text, boolean useCache) {
+      public SeekStatus seekCeil(BytesRef text) {
         throw new UnsupportedOperationException();
       }
     }
@@ -1499,7 +1493,7 @@ public class BlockTreeTermsReader extend
       }
 
       @Override
-      public boolean seekExact(final BytesRef target, final boolean useCache) throws IOException {
+      public boolean seekExact(final BytesRef target) throws IOException {
 
         if (index == null) {
           throw new IllegalStateException("terms index was not loaded");
@@ -1713,7 +1707,6 @@ public class BlockTreeTermsReader extend
             if (arc.output != NO_OUTPUT) {
               output = fstOutputs.add(output, arc.output);
             }
-
             // if (DEBUG) {
             //   System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
             // }
@@ -1760,7 +1753,7 @@ public class BlockTreeTermsReader extend
       }
 
       @Override
-      public SeekStatus seekCeil(final BytesRef target, final boolean useCache) throws IOException {
+      public SeekStatus seekCeil(final BytesRef target) throws IOException {
         if (index == null) {
           throw new IllegalStateException("terms index was not loaded");
         }
@@ -2096,7 +2089,7 @@ public class BlockTreeTermsReader extend
           // this method catches up all internal state so next()
           // works properly:
           //if (DEBUG) System.out.println("  re-seek to pending term=" + term.utf8ToString() + " " + term);
-          final boolean result = seekExact(term, false);
+          final boolean result = seekExact(term);
           assert result;
         }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java Sun Aug 11 12:19:13 2013
@@ -17,7 +17,6 @@ package org.apache.lucene.codecs.compres
  * limitations under the License.
  */
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.Arrays;
 
@@ -25,16 +24,13 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Random-access reader for {@link CompressingStoredFieldsIndexWriter}.
  * @lucene.internal
  */
-public final class CompressingStoredFieldsIndexReader implements Closeable, Cloneable {
-
-  final IndexInput fieldsIndexIn;
+public final class CompressingStoredFieldsIndexReader implements Cloneable {
 
   static long moveLowOrderBitToSign(long n) {
     return ((n >>> 1) ^ -(n & 1));
@@ -48,8 +44,9 @@ public final class CompressingStoredFiel
   final PackedInts.Reader[] docBasesDeltas; // delta from the avg
   final PackedInts.Reader[] startPointersDeltas; // delta from the avg
 
+  // It is the responsibility of the caller to close fieldsIndexIn after this constructor
+  // has been called
   CompressingStoredFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
-    this.fieldsIndexIn = fieldsIndexIn;
     maxDoc = si.getDocCount();
     int[] docBases = new int[16];
     long[] startPointers = new long[16];
@@ -106,17 +103,6 @@ public final class CompressingStoredFiel
     this.startPointersDeltas = Arrays.copyOf(startPointersDeltas, blockCount);
   }
 
-  private CompressingStoredFieldsIndexReader(CompressingStoredFieldsIndexReader other) {
-    this.fieldsIndexIn = null;
-    this.maxDoc = other.maxDoc;
-    this.docBases = other.docBases;
-    this.startPointers = other.startPointers;
-    this.avgChunkDocs = other.avgChunkDocs;
-    this.avgChunkSizes = other.avgChunkSizes;
-    this.docBasesDeltas = other.docBasesDeltas;
-    this.startPointersDeltas = other.startPointersDeltas;
-  }
-
   private int block(int docID) {
     int lo = 0, hi = docBases.length - 1;
     while (lo <= hi) {
@@ -172,16 +158,7 @@ public final class CompressingStoredFiel
 
   @Override
   public CompressingStoredFieldsIndexReader clone() {
-    if (fieldsIndexIn == null) {
-      return this;
-    } else {
-      return new CompressingStoredFieldsIndexReader(this);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    IOUtils.close(fieldsIndexIn);
+    return this;
   }
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java Sun Aug 11 12:19:13 2013
@@ -21,7 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.packed.PackedInts;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java Sun Aug 11 12:19:13 2013
@@ -60,6 +60,9 @@ import org.apache.lucene.util.packed.Pac
  */
 public final class CompressingStoredFieldsReader extends StoredFieldsReader {
 
+  // Do not reuse the decompression buffer when there is more than 32kb to decompress
+  private static final int BUFFER_REUSE_THRESHOLD = 1 << 15;
+
   private final FieldInfos fieldInfos;
   private final CompressingStoredFieldsIndexReader indexReader;
   private final IndexInput fieldsStream;
@@ -93,20 +96,23 @@ public final class CompressingStoredFiel
     numDocs = si.getDocCount();
     IndexInput indexStream = null;
     try {
-      fieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION), context);
+      // Load the index into memory
       final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION);
       indexStream = d.openInput(indexStreamFN, context);
-
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
-      final String codecNameDat = formatName + CODEC_SFX_DAT;
       CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
-      CodecUtil.checkHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
-      assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
       assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
-
       indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
+      indexStream.close();
       indexStream = null;
 
+      // Open the data file and read metadata
+      final String fieldsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION);
+      fieldsStream = d.openInput(fieldsStreamFN, context);
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      CodecUtil.checkHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
+
       packedIntsVersion = fieldsStream.readVInt();
       decompressor = compressionMode.newDecompressor();
       this.bytes = new BytesRef();
@@ -134,7 +140,7 @@ public final class CompressingStoredFiel
   @Override
   public void close() throws IOException {
     if (!closed) {
-      IOUtils.close(fieldsStream, indexReader);
+      IOUtils.close(fieldsStream);
       closed = true;
     }
   }
@@ -255,6 +261,7 @@ public final class CompressingStoredFiel
       return;
     }
 
+    final BytesRef bytes = totalLength <= BUFFER_REUSE_THRESHOLD ? this.bytes : new BytesRef();
     decompressor.decompress(fieldsStream, totalLength, offset, length, bytes);
     assert bytes.length == length;
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java Sun Aug 11 12:19:13 2013
@@ -44,6 +44,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GrowableByteArrayDataOutput;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;