You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2010/12/20 11:53:33 UTC

svn commit: r1051056 [5/8] - in /lucene/dev/branches/docvalues: ./ lucene/ lucene/contrib/ant/src/java/org/apache/lucene/ant/ lucene/contrib/ant/src/test/org/apache/lucene/ant/ lucene/contrib/benchmark/src/test/org/apache/lucene/benchmark/quality/ luce...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Type.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Type.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Type.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Type.java Mon Dec 20 10:53:27 2010
@@ -1,4 +1,7 @@
 package org.apache.lucene.index.values;
+
+import org.apache.lucene.index.values.DocValues.SortedSource;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,32 +19,62 @@ package org.apache.lucene.index.values;
  * limitations under the License.
  */
 
-/** Controls whether per-field values are stored into
- *  index.  This storage is non-sparse, so it's best to
- *  use this when all docs have the field, and loads all
- *  values into RAM, exposing a random access API, when
- *  loaded.
- *
- * @lucene.experimental 
+/**
+ * {@link Type} specifies the type of the {@link DocValues} for a certain field.
+ * A {@link Type} can specify the actual data type for a field, used compression
+ * schemes and high-level data-structures.
+ * 
+ * @lucene.experimental
  */
 public enum Type {
 
-  /** Integral value is stored as packed ints.  The bit
-   *  precision is fixed across the segment, and
-   *  determined by the min/max values in the field. */
+  /**
+   * Integral value is stored as packed ints. The bit precision is fixed across
+   * the segment, and determined by the min/max values in the field.
+   */
   PACKED_INTS,
+  /**
+   * 32 bit floating point value stored without modification or compression.
+   */
   SIMPLE_FLOAT_4BYTE,
+  /**
+   * 64 bit floating point value stored without modification or compression.
+   */
   SIMPLE_FLOAT_8BYTE,
 
   // TODO(simonw): -- shouldn't lucene decide/detect straight vs
   // deref, as well fixed vs var?
+  /**
+   * Fixed length straight stored byte variant
+   */
   BYTES_FIXED_STRAIGHT,
+
+  /**
+   * Fixed length dereferenced (indexed) byte variant
+   */
   BYTES_FIXED_DEREF,
+
+  /**
+   * Fixed length pre-sorted byte variant
+   * 
+   * @see SortedSource
+   */
   BYTES_FIXED_SORTED,
 
+  /**
+   * Variable length straight stored byte variant
+   */
   BYTES_VAR_STRAIGHT,
+
+  /**
+   * Variable length dereferenced (indexed) byte variant
+   */
   BYTES_VAR_DEREF,
-  BYTES_VAR_SORTED
 
-  // TODO(simonw): -- need STRING variants as well
+  /**
+   * Variable length pre-sorted byte variant
+   * 
+   * @see SortedSource
+   */
+  BYTES_VAR_SORTED
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -114,7 +114,7 @@ class VarDerefBytesImpl {
 
     public Writer(Directory dir, String id, Allocator allocator,
         AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
           new ByteBlockPool(allocator), bytesUsed);
       docToAddress = new int[1];
       bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
@@ -124,8 +124,6 @@ class VarDerefBytesImpl {
     synchronized public void add(int docID, BytesRef bytes) throws IOException {
       if (bytes.length == 0)
         return; // default
-      if (datOut == null)
-        initDataOut();
       final int e = hash.add(bytes);
 
       if (docID >= docToAddress.length) {
@@ -162,34 +160,33 @@ class VarDerefBytesImpl {
     // some last docs that we didn't see
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if (datOut == null)
-        return;
-      initIndexOut();
-      idxOut.writeInt(address - 1);
-
-      // write index
-      // TODO(simonw): -- allow forcing fixed array (not -1)
-      // TODO(simonw): check the address calculation / make it more intuitive
-      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
-          PackedInts.bitsRequired(address - 1));
-      final int limit;
-      if (docCount > docToAddress.length) {
-        limit = docToAddress.length;
-      } else {
-        limit = docCount;
-      }
-      for (int i = 0; i < limit; i++) {
-        w.add(docToAddress[i]);
-      }
-      for (int i = limit; i < docCount; i++) {
-        w.add(0);
+      try {
+        idxOut.writeInt(address - 1);
+        // write index
+        // TODO(simonw): -- allow forcing fixed array (not -1)
+        // TODO(simonw): check the address calculation / make it more intuitive
+        final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+            PackedInts.bitsRequired(address - 1));
+        final int limit;
+        if (docCount > docToAddress.length) {
+          limit = docToAddress.length;
+        } else {
+          limit = docCount;
+        }
+        for (int i = 0; i < limit; i++) {
+          w.add(docToAddress[i]);
+        }
+        for (int i = limit; i < docCount; i++) {
+          w.add(0);
+        }
+        w.finish();
+      } finally {
+        hash.close();
+        super.finish(docCount);
+        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
+            * (-docToAddress.length));
+        docToAddress = null;
       }
-      w.finish();
-      hash.close();
-      super.finish(docCount);
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-          * (-docToAddress.length));
-      docToAddress = null;
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -56,17 +56,19 @@ class VarSortedBytesImpl {
     private int[] docToEntry;
     private final Comparator<BytesRef> comp;
 
-    private final BytesRefHash hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
-        new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+    private final BytesRefHash hash = new BytesRefHash(pool,
+        BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+            BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
 
-    public Writer(Directory dir, String id, Comparator<BytesRef> comp, AtomicLong bytesUsed)
-        throws IOException {
-      this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE), bytesUsed);
+    public Writer(Directory dir, String id, Comparator<BytesRef> comp,
+        AtomicLong bytesUsed) throws IOException {
+      this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+          bytesUsed);
     }
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Allocator allocator, AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
           new ByteBlockPool(allocator), bytesUsed);
       this.comp = comp;
       docToEntry = new int[1];
@@ -97,62 +99,59 @@ class VarSortedBytesImpl {
     @Override
     synchronized public void finish(int docCount) throws IOException {
       final int count = hash.size();
-      if (count == 0)
-        return;
-      initIndexOut();
-      initDataOut();
-      int[] sortedEntries = hash.sort(comp);
-
-      // first dump bytes data, recording index & offset as
-      // we go
-      long offset = 0;
-      long lastOffset = 0;
-      final int[] index = new int[count];
-      final long[] offsets = new long[count];
-      for (int i = 0; i < count; i++) {
-        final int e = sortedEntries[i];
-        offsets[i] = offset;
-        index[e] = 1 + i;
-
-        final BytesRef bytes = hash.get(e, new BytesRef());
-        // TODO: we could prefix code...
-        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-        lastOffset = offset;
-        offset += bytes.length;
-      }
-
-      // total bytes of data
-      idxOut.writeLong(offset);
-
-      // write index -- first doc -> 1+ord
-      // TODO(simonw): allow not -1:
-      final PackedInts.Writer indexWriter = PackedInts.getWriter(idxOut,
-          docCount, PackedInts.bitsRequired(count));
-      final int limit = docCount > docToEntry.length ? docToEntry.length
-          : docCount;
-      for (int i = 0; i < limit; i++) {
-        final int e = docToEntry[i];
-        indexWriter.add(e == -1 ? 0 : index[e]);
-      }
-      for (int i = limit; i < docCount; i++) {
-        indexWriter.add(0);
-      }
-      indexWriter.finish();
-
-      // next ord (0-based) -> offset
-      // TODO(simonw): -- allow not -1:
-      PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count,
-          PackedInts.bitsRequired(lastOffset));
-      for (int i = 0; i < count; i++) {
-        offsetWriter.add(offsets[i]);
-      }
-      offsetWriter.finish();
-
-      super.finish(docCount);
-      bytesUsed.addAndGet((-docToEntry.length)
-          * RamUsageEstimator.NUM_BYTES_INT);
-      hash.close();
+      try {
+        final int[] sortedEntries = hash.sort(comp);
+
+        // first dump bytes data, recording index & offset as
+        // we go
+        long offset = 0;
+        long lastOffset = 0;
+        final int[] index = new int[count];
+        final long[] offsets = new long[count];
+        for (int i = 0; i < count; i++) {
+          final int e = sortedEntries[i];
+          offsets[i] = offset;
+          index[e] = 1 + i;
+
+          final BytesRef bytes = hash.get(e, new BytesRef());
+          // TODO: we could prefix code...
+          datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+          lastOffset = offset;
+          offset += bytes.length;
+        }
+
+        // total bytes of data
+        idxOut.writeLong(offset);
+
+        // write index -- first doc -> 1+ord
+        // TODO(simonw): allow not -1:
+        final PackedInts.Writer indexWriter = PackedInts.getWriter(idxOut,
+            docCount, PackedInts.bitsRequired(count));
+        final int limit = docCount > docToEntry.length ? docToEntry.length
+            : docCount;
+        for (int i = 0; i < limit; i++) {
+          final int e = docToEntry[i];
+          indexWriter.add(e == -1 ? 0 : index[e]);
+        }
+        for (int i = limit; i < docCount; i++) {
+          indexWriter.add(0);
+        }
+        indexWriter.finish();
 
+        // next ord (0-based) -> offset
+        // TODO(simonw): -- allow not -1:
+        PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count,
+            PackedInts.bitsRequired(lastOffset));
+        for (int i = 0; i < count; i++) {
+          offsetWriter.add(offsets[i]);
+        }
+        offsetWriter.finish();
+      } finally {
+        super.finish(docCount);
+        bytesUsed.addAndGet((-docToEntry.length)
+            * RamUsageEstimator.NUM_BYTES_INT);
+        hash.close();
+      }
     }
   }
 
@@ -172,7 +171,7 @@ class VarSortedBytesImpl {
     public SortedSource loadSorted(Comparator<BytesRef> comp)
         throws IOException {
       IndexInput indexIn = cloneIndex();
-      return new Source(cloneData(), indexIn , comp, indexIn.readLong());
+      return new Source(cloneData(), indexIn, comp, indexIn.readLong());
     }
 
     private static class Source extends BytesBaseSortedSource {
@@ -182,7 +181,7 @@ class VarSortedBytesImpl {
       private final int valueCount;
 
       public Source(IndexInput datIn, IndexInput idxIn,
-          Comparator<BytesRef> comp,  long dataLength) throws IOException {
+          Comparator<BytesRef> comp, long dataLength) throws IOException {
         super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), dataLength);
         totBytes = dataLength;
         docToOrdIndex = PackedInts.getReader(idxIn);
@@ -215,7 +214,7 @@ class VarSortedBytesImpl {
           nextOffset = ordToOffsetIndex.get(1 + ord);
         }
         final long offset = ordToOffsetIndex.get(ord);
-        data.fillSlice(bytesRef, offset , (int)(nextOffset - offset));
+        data.fillSlice(bytesRef, offset, (int) (nextOffset - offset));
         return bytesRef;
       }
 
@@ -272,8 +271,8 @@ class VarSortedBytesImpl {
           return pos = NO_MORE_DOCS;
         }
         int ord;
-        while((ord =(int) docToOrdIndex.get(target)) == 0) {
-          if(++target >= docCount) {
+        while ((ord = (int) docToOrdIndex.get(target)) == 0) {
+          if (++target >= docCount) {
             return pos = NO_MORE_DOCS;
           }
         }
@@ -307,7 +306,7 @@ class VarSortedBytesImpl {
         return advance(pos + 1);
       }
     }
-    
+
     @Override
     public Type type() {
       return Type.BYTES_VAR_SORTED;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -51,7 +51,7 @@ class VarStraightBytesImpl {
 
     public Writer(Directory dir, String id, AtomicLong bytesUsed)
         throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, null, bytesUsed);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true, null, bytesUsed);
       docToAddress = new long[1];
       bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
     }
@@ -78,8 +78,6 @@ class VarStraightBytesImpl {
     synchronized public void add(int docID, BytesRef bytes) throws IOException {
       if (bytes.length == 0)
         return; // default
-      if (datOut == null)
-        initDataOut();
       fill(docID);
       docToAddress[docID] = address;
       datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
@@ -88,22 +86,31 @@ class VarStraightBytesImpl {
 
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if (datOut == null) {
-        return;
+      try {
+        if (lastDocID == -1) {
+          idxOut.writeVLong(0);
+          final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+              PackedInts.bitsRequired(0));
+          for (int i = 0; i < docCount; i++) {
+            w.add(0);
+          }
+          w.finish();
+        } else {
+          fill(docCount);
+          idxOut.writeVLong(address);
+          final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+              PackedInts.bitsRequired(address));
+          for (int i = 0; i < docCount; i++) {
+            w.add(docToAddress[i]);
+          }
+          w.finish();
+        }
+      } finally {
+        bytesUsed.addAndGet(-(docToAddress.length)
+            * RamUsageEstimator.NUM_BYTES_INT);
+        docToAddress = null;
+        super.finish(docCount);
       }
-      initIndexOut();
-      fill(docCount);
-      idxOut.writeVLong(address);
-      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
-          PackedInts.bitsRequired(address));
-      for (int i = 0; i < docCount; i++) {
-        w.add(docToAddress[i]);
-      }
-      w.finish();
-      bytesUsed.addAndGet(-(docToAddress.length)
-          * RamUsageEstimator.NUM_BYTES_INT);
-      docToAddress = null;
-      super.finish(docCount);
     }
 
     public long ramBytesUsed() {
@@ -128,7 +135,7 @@ class VarStraightBytesImpl {
       private final PackedInts.Reader addresses;
 
       public Source(IndexInput datIn, IndexInput idxIn) throws IOException {
-        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong()); 
+        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong());
         addresses = PackedInts.getReader(idxIn);
         missingValue.bytesValue = new BytesRef(0); // empty
       }
@@ -167,13 +174,13 @@ class VarStraightBytesImpl {
       private final IndexInput datIn;
       private final IndexInput idxIn;
       private final long fp;
-      private final int totBytes;
+      private final long totBytes;
       private int pos = -1;
 
       protected VarStraightBytesEnum(AttributeSource source, IndexInput datIn,
           IndexInput idxIn) throws IOException {
         super(source, Type.BYTES_VAR_STRAIGHT);
-        totBytes = idxIn.readVInt();
+        totBytes = idxIn.readVLong();
         fp = datIn.getFilePointer();
         addresses = PackedInts.getReader(idxIn);
         this.datIn = datIn;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java Mon Dec 20 10:53:27 2010
@@ -39,6 +39,7 @@ import org.apache.lucene.search.cache.Ca
 import org.apache.lucene.search.cache.CachedArray.IntValues;
 import org.apache.lucene.search.cache.CachedArray.LongValues;
 import org.apache.lucene.search.cache.CachedArray.ShortValues;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.packed.Direct16;
 import org.apache.lucene.util.packed.Direct32;
@@ -187,16 +188,25 @@ public abstract class FieldComparator {
     protected final CachedArrayCreator<T> creator;
     protected T cached;
     protected final boolean checkMissing;
+    protected Bits valid;
     
     public NumericComparator( CachedArrayCreator<T> c, boolean checkMissing ) {
       this.creator = c;
       this.checkMissing = checkMissing;
     }
+
+    protected FieldComparator setup(T cached) {
+      this.cached = cached;
+      if (checkMissing)
+        valid = cached.valid;
+      return this;
+    }
   }
 
   /** Parses field's values as byte (using {@link
    *  FieldCache#getBytes} and sorts by ascending value */
   public static final class ByteComparator extends NumericComparator<ByteValues> {
+    private byte[] docValues;
     private final byte[] values;
     private final byte missingValue;
     private byte bottom;
@@ -215,20 +225,26 @@ public abstract class FieldComparator {
 
     @Override
     public int compareBottom(int doc) {
-      final byte v2 = (checkMissing && !cached.valid.get(doc)) 
-        ? missingValue : cached.values[doc];
+      byte v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
       return bottom - v2;
     }
 
     @Override
     public void copy(int slot, int doc) {
-      values[slot] = ( checkMissing && cached.valid != null && !cached.valid.get(doc) )
-        ? missingValue : cached.values[doc];
+      byte v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      values[slot] = v2;
     }
 
     @Override
     public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      cached = FieldCache.DEFAULT.getBytes(reader, creator.field, creator );
+      setup(FieldCache.DEFAULT.getBytes(reader, creator.field, creator));
+      docValues = cached.values;
       return this;
     }
     
@@ -247,6 +263,7 @@ public abstract class FieldComparator {
   /** Parses field's values as double (using {@link
    *  FieldCache#getDoubles} and sorts by ascending value */
   public static final class DoubleComparator extends NumericComparator<DoubleValues> {
+    private double[] docValues;
     private final double[] values;
     private final double missingValue;
     private double bottom;
@@ -274,9 +291,10 @@ public abstract class FieldComparator {
 
     @Override
     public int compareBottom(int doc) {
-      final double v2 = (checkMissing && !cached.valid.get(doc)) 
-        ? missingValue : cached.values[doc];
-      
+      double v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
       if (bottom > v2) {
         return 1;
       } else if (bottom < v2) {
@@ -288,13 +306,17 @@ public abstract class FieldComparator {
 
     @Override
     public void copy(int slot, int doc) {
-      values[slot] = ( checkMissing && cached.valid != null && !cached.valid.get(doc) )
-        ? missingValue : cached.values[doc];
+      double v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      values[slot] = v2;
     }
 
     @Override
     public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      cached = FieldCache.DEFAULT.getDoubles(reader, creator.field, creator );
+      setup(FieldCache.DEFAULT.getDoubles(reader, creator.field, creator));
+      docValues = cached.values;
       return this;
     }
     
@@ -371,6 +393,7 @@ public abstract class FieldComparator {
   /** Parses field's values as float (using {@link
    *  FieldCache#getFloats} and sorts by ascending value */
   public static final class FloatComparator extends NumericComparator<FloatValues> {
+    private float[] docValues;
     private final float[] values;
     private final float missingValue;
     private float bottom;
@@ -400,8 +423,10 @@ public abstract class FieldComparator {
     @Override
     public int compareBottom(int doc) {
       // TODO: are there sneaky non-branch ways to compute sign of float?
-      final float v2 = (checkMissing && !cached.valid.get(doc)) 
-        ? missingValue : cached.values[doc];
+      float v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
       
       if (bottom > v2) {
         return 1;
@@ -414,13 +439,17 @@ public abstract class FieldComparator {
 
     @Override
     public void copy(int slot, int doc) {
-      values[slot] = ( checkMissing && cached.valid != null && !cached.valid.get(doc) )
-        ? missingValue : cached.values[doc];
+      float v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      values[slot] = v2;
     }
 
     @Override
     public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      cached = FieldCache.DEFAULT.getFloats(reader, creator.field, creator );
+      setup(FieldCache.DEFAULT.getFloats(reader, creator.field, creator));
+      docValues = cached.values;
       return this;
     }
     
@@ -435,9 +464,66 @@ public abstract class FieldComparator {
     }
   }
 
+  /** Parses field's values as short (using {@link
+   *  FieldCache#getShorts} and sorts by ascending value */
+  public static final class ShortComparator extends NumericComparator<ShortValues> {
+    private short[] docValues;
+    private final short[] values;
+    private short bottom;
+    private final short missingValue;
+
+    ShortComparator(int numHits, ShortValuesCreator creator, Short missingValue ) {
+      super( creator, missingValue != null );
+      values = new short[numHits];
+      this.missingValue = checkMissing
+        ? missingValue.shortValue() : 0;
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+      return values[slot1] - values[slot2];
+    }
+
+    @Override
+    public int compareBottom(int doc) {
+      short v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      return bottom - v2;
+    }
+
+    @Override
+    public void copy(int slot, int doc) {
+      short v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      values[slot] = v2;
+    }
+
+    @Override
+    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
+      setup( FieldCache.DEFAULT.getShorts(reader, creator.field, creator));
+      docValues = cached.values;
+      return this;
+    }
+
+    @Override
+    public void setBottom(final int bottom) {
+      this.bottom = values[bottom];
+    }
+
+    @Override
+    public Comparable<?> value(int slot) {
+      return Short.valueOf(values[slot]);
+    }
+  }
+
   /** Parses field's values as int (using {@link
    *  FieldCache#getInts} and sorts by ascending value */
   public static final class IntComparator extends NumericComparator<IntValues> {
+    private int[] docValues;
     private final int[] values;
     private int bottom;                           // Value of bottom of queue
     final int missingValue;
@@ -472,9 +558,10 @@ public abstract class FieldComparator {
       // -1/+1/0 sign
       // Cannot return bottom - values[slot2] because that
       // may overflow
-      final int v2 = (checkMissing && !cached.valid.get(doc)) 
-        ? missingValue : cached.values[doc];
-      
+      int v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
       if (bottom > v2) {
         return 1;
       } else if (bottom < v2) {
@@ -486,13 +573,17 @@ public abstract class FieldComparator {
 
     @Override
     public void copy(int slot, int doc) {
-      values[slot] = ( checkMissing && cached.valid != null && !cached.valid.get(doc) )
-        ? missingValue : cached.values[doc];
+      int v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      values[slot] = v2;
     }
 
     @Override
     public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      cached = FieldCache.DEFAULT.getInts(reader, creator.field, creator);  
+      setup(FieldCache.DEFAULT.getInts(reader, creator.field, creator));
+      docValues = cached.values;
       return this;
     }
     
@@ -573,6 +664,7 @@ public abstract class FieldComparator {
   /** Parses field's values as long (using {@link
    *  FieldCache#getLongs} and sorts by ascending value */
   public static final class LongComparator extends NumericComparator<LongValues> {
+    private long[] docValues;
     private final long[] values;
     private long bottom;
     private final long missingValue;
@@ -603,8 +695,10 @@ public abstract class FieldComparator {
     public int compareBottom(int doc) {
       // TODO: there are sneaky non-branch ways to compute
       // -1/+1/0 sign
-      final long v2 = (checkMissing && !cached.valid.get(doc)) 
-        ? missingValue : cached.values[doc];
+      long v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
       
       if (bottom > v2) {
         return 1;
@@ -617,13 +711,17 @@ public abstract class FieldComparator {
 
     @Override
     public void copy(int slot, int doc) {
-      values[slot] = ( checkMissing && cached.valid != null && !cached.valid.get(doc) )
-        ? missingValue : cached.values[doc];
+      long v2 = docValues[doc];
+      if (valid != null && v2==0 && !valid.get(doc))
+        v2 = missingValue;
+
+      values[slot] = v2;
     }
 
     @Override
     public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      cached = FieldCache.DEFAULT.getLongs(reader, creator.field, creator);
+      setup(FieldCache.DEFAULT.getLongs(reader, creator.field, creator));
+      docValues = cached.values;
       return this;
     }
     
@@ -694,55 +792,6 @@ public abstract class FieldComparator {
     }
   }
 
-  /** Parses field's values as short (using {@link
-   *  FieldCache#getShorts} and sorts by ascending value */
-  public static final class ShortComparator extends NumericComparator<ShortValues> {
-    private final short[] values;
-    private short bottom;
-    private final short missingValue;
-    
-    ShortComparator(int numHits, ShortValuesCreator creator, Short missingValue ) {
-      super( creator, missingValue != null );
-      values = new short[numHits];
-      this.missingValue = checkMissing
-        ? missingValue.shortValue() : 0;
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      return values[slot1] - values[slot2];
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      final short v2 = (checkMissing && !cached.valid.get(doc)) 
-        ? missingValue : cached.values[doc];
-      
-      return bottom - v2;
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      values[slot] = ( checkMissing && cached.valid != null && !cached.valid.get(doc) )
-        ? missingValue : cached.values[doc];
-    }
-
-    @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      cached = FieldCache.DEFAULT.getShorts(reader, creator.field, creator );
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public Comparable<?> value(int slot) {
-      return Short.valueOf(values[slot]);
-    }
-  }
 
 
   /** Sorts by ascending docID */
@@ -881,20 +930,28 @@ public abstract class FieldComparator {
    *  than {@link TermValComparator}.  For very small
    *  result sets it may be slower. */
   public static final class TermOrdValComparator extends FieldComparator {
+    /** @lucene.internal */
+    final int[] ords;
+    /** @lucene.internal */
+    final BytesRef[] values;
+    /** @lucene.internal */
+    final int[] readerGen;
 
-    private final int[] ords;
-    private final BytesRef[] values;
-    private final int[] readerGen;
-
-    private int currentReaderGen = -1;
+    /** @lucene.internal */
+    int currentReaderGen = -1;
     private DocTermsIndex termsIndex;
     private final String field;
 
-    private int bottomSlot = -1;
-    private int bottomOrd;
-    private boolean bottomSameReader;
-    private BytesRef bottomValue;
-    private final BytesRef tempBR = new BytesRef();
+    /** @lucene.internal */
+    int bottomSlot = -1;
+    /** @lucene.internal */
+    int bottomOrd;
+    /** @lucene.internal */
+    boolean bottomSameReader;
+    /** @lucene.internal */
+    BytesRef bottomValue;
+    /** @lucene.internal */
+    final BytesRef tempBR = new BytesRef();
 
     public TermOrdValComparator(int numHits, String field, int sortPos, boolean reversed) {
       ords = new int[numHits];
@@ -932,11 +989,13 @@ public abstract class FieldComparator {
       throw new UnsupportedOperationException();
     }
 
-    // Base class for specialized (per bit width of the
-    // ords) per-segment comparator.  NOTE: this is messy;
-    // we do this only because hotspot can't reliably inline
-    // the underlying array access when looking up doc->ord
-    private abstract class PerSegmentComparator extends FieldComparator {
+    /** Base class for specialized (per bit width of the
+     * ords) per-segment comparator.  NOTE: this is messy;
+     * we do this only because hotspot can't reliably inline
+     * the underlying array access when looking up doc->ord
+     * @lucene.internal
+     */
+    abstract class PerSegmentComparator extends FieldComparator {
       
       @Override
       public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java Mon Dec 20 10:53:27 2010
@@ -66,7 +66,7 @@ abstract class TermCollectingRewrite<Q e
       
       // Check comparator compatibility:
       final Comparator<BytesRef> newTermComp = termsEnum.getComparator();
-      if (lastTermComp != null && newTermComp != lastTermComp)
+      if (lastTermComp != null && newTermComp != null && newTermComp != lastTermComp)
         throw new RuntimeException("term comparator should not change between segments: "+lastTermComp+" != "+newTermComp);
       lastTermComp = newTermComp;
       

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java Mon Dec 20 10:53:27 2010
@@ -153,7 +153,7 @@ public abstract class Directory implemen
    *
    * @param lockFactory instance of {@link LockFactory}.
    */
-  public void setLockFactory(LockFactory lockFactory) {
+  public void setLockFactory(LockFactory lockFactory) throws IOException {
     assert lockFactory != null;
     this.lockFactory = lockFactory;
     lockFactory.setLockPrefix(this.getLockID());

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java Mon Dec 20 10:53:27 2010
@@ -156,20 +156,6 @@ public abstract class FSDirectory extend
       throw new NoSuchDirectoryException("file '" + directory + "' exists but is not a directory");
 
     setLockFactory(lockFactory);
-    
-    // for filesystem based LockFactory, delete the lockPrefix, if the locks are placed
-    // in index dir. If no index dir is given, set ourselves
-    if (lockFactory instanceof FSLockFactory) {
-      final FSLockFactory lf = (FSLockFactory) lockFactory;
-      final File dir = lf.getLockDir();
-      // if the lock factory has no lockDir set, use the this directory as lockDir
-      if (dir == null) {
-        lf.setLockDir(directory);
-        lf.setLockPrefix(null);
-      } else if (dir.getCanonicalPath().equals(directory.getCanonicalPath())) {
-        lf.setLockPrefix(null);
-      }
-    }
   }
 
   /** Creates an FSDirectory instance, trying to pick the
@@ -209,6 +195,26 @@ public abstract class FSDirectory extend
     }
   }
 
+  @Override
+  public void setLockFactory(LockFactory lockFactory) throws IOException {
+    super.setLockFactory(lockFactory);
+
+    // for filesystem based LockFactory, delete the lockPrefix, if the locks are placed
+    // in index dir. If no index dir is given, set ourselves
+    if (lockFactory instanceof FSLockFactory) {
+      final FSLockFactory lf = (FSLockFactory) lockFactory;
+      final File dir = lf.getLockDir();
+      // if the lock factory has no lockDir set, use the this directory as lockDir
+      if (dir == null) {
+        lf.setLockDir(directory);
+        lf.setLockPrefix(null);
+      } else if (dir.getCanonicalPath().equals(directory.getCanonicalPath())) {
+        lf.setLockPrefix(null);
+      }
+    }
+
+  }
+  
   /** Lists all files (not subdirectories) in the
    *  directory.  This method never returns null (throws
    *  {@link IOException} instead).

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/MMapDirectory.java Mon Dec 20 10:53:27 2010
@@ -242,8 +242,35 @@ public class MMapDirectory extends FSDir
         throw new IOException("read past EOF");
       }
     }
+    
+    @Override
+    public short readShort() throws IOException {
+      try {
+        return buffer.getShort();
+      } catch (BufferUnderflowException e) {
+        throw new IOException("read past EOF");
+      }
+    }
 
     @Override
+    public int readInt() throws IOException {
+      try {
+        return buffer.getInt();
+      } catch (BufferUnderflowException e) {
+        throw new IOException("read past EOF");
+      }
+    }
+
+    @Override
+    public long readLong() throws IOException {
+      try {
+        return buffer.getLong();
+      } catch (BufferUnderflowException e) {
+        throw new IOException("read past EOF");
+      }
+    }
+    
+    @Override
     public long getFilePointer() {
       return buffer.position();
     }
@@ -294,7 +321,6 @@ public class MMapDirectory extends FSDir
     private final int maxBufSize;
   
     private ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
-    private int curAvail; // redundant for speed: (bufSizes[curBufIndex] - curBuf.position())
   
     private boolean isClone = false;
     
@@ -333,38 +359,67 @@ public class MMapDirectory extends FSDir
   
     @Override
     public byte readByte() throws IOException {
-      // Performance might be improved by reading ahead into an array of
-      // e.g. 128 bytes and readByte() from there.
-      if (curAvail == 0) {
+      try {
+        return curBuf.get();
+      } catch (BufferUnderflowException e) {
         curBufIndex++;
         if (curBufIndex >= buffers.length)
           throw new IOException("read past EOF");
         curBuf = buffers[curBufIndex];
         curBuf.position(0);
-        curAvail = bufSizes[curBufIndex];
+        return curBuf.get();
       }
-      curAvail--;
-      return curBuf.get();
     }
   
     @Override
     public void readBytes(byte[] b, int offset, int len) throws IOException {
-      while (len > curAvail) {
-        curBuf.get(b, offset, curAvail);
-        len -= curAvail;
-        offset += curAvail;
-        curBufIndex++;
-        if (curBufIndex >= buffers.length)
-          throw new IOException("read past EOF");
-        curBuf = buffers[curBufIndex];
-        curBuf.position(0);
-        curAvail = bufSizes[curBufIndex];
+      try {
+        curBuf.get(b, offset, len);
+      } catch (BufferUnderflowException e) {
+        int curAvail = curBuf.remaining();
+        while (len > curAvail) {
+          curBuf.get(b, offset, curAvail);
+          len -= curAvail;
+          offset += curAvail;
+          curBufIndex++;
+          if (curBufIndex >= buffers.length)
+            throw new IOException("read past EOF");
+          curBuf = buffers[curBufIndex];
+          curBuf.position(0);
+          curAvail = curBuf.remaining();
+        }
+        curBuf.get(b, offset, len);
       }
-      curBuf.get(b, offset, len);
-      curAvail -= len;
     }
   
     @Override
+    public short readShort() throws IOException {
+      try {
+        return curBuf.getShort();
+      } catch (BufferUnderflowException e) {
+        return super.readShort();
+      }
+    }
+
+    @Override
+    public int readInt() throws IOException {
+      try {
+        return curBuf.getInt();
+      } catch (BufferUnderflowException e) {
+        return super.readInt();
+      }
+    }
+
+    @Override
+    public long readLong() throws IOException {
+      try {
+        return curBuf.getLong();
+      } catch (BufferUnderflowException e) {
+        return super.readLong();
+      }
+    }
+    
+    @Override
     public long getFilePointer() {
       return ((long) curBufIndex * maxBufSize) + curBuf.position();
     }
@@ -375,7 +430,6 @@ public class MMapDirectory extends FSDir
       curBuf = buffers[curBufIndex];
       int bufOffset = (int) (pos - ((long) curBufIndex * maxBufSize));
       curBuf.position(bufOffset);
-      curAvail = bufSizes[curBufIndex] - bufOffset;
     }
   
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Mon Dec 20 10:53:27 2010
@@ -48,7 +48,11 @@ public class RAMDirectory extends Direct
 
   /** Constructs an empty {@link Directory}. */
   public RAMDirectory() {
-    setLockFactory(new SingleInstanceLockFactory());
+    try {
+      setLockFactory(new SingleInstanceLockFactory());
+    } catch (IOException e) {
+      // Cannot happen
+    }
   }
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Mon Dec 20 10:53:27 2010
@@ -405,7 +405,7 @@ public final class ArrayUtil {
   }
 
   /**
-   * Returns hash of chars in range start (inclusive) to
+   * Returns hash of bytes in range start (inclusive) to
    * end (inclusive)
    */
   public static int hashCode(byte[] array, int start, int end) {
@@ -442,6 +442,33 @@ public final class ArrayUtil {
     return false;
   }
 
+  /* DISABLE THIS FOR NOW: This has performance problems until Java creates intrinsics for Class#getComponentType() and Array.newInstance()
+  public static <T> T[] grow(T[] array, int minSize) {
+    if (array.length < minSize) {
+      @SuppressWarnings("unchecked") final T[] newArray =
+        (T[]) Array.newInstance(array.getClass().getComponentType(), oversize(minSize, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
+      System.arraycopy(array, 0, newArray, 0, array.length);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static <T> T[] grow(T[] array) {
+    return grow(array, 1 + array.length);
+  }
+
+  public static <T> T[] shrink(T[] array, int targetSize) {
+    final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+    if (newSize != array.length) {
+      @SuppressWarnings("unchecked") final T[] newArray =
+        (T[]) Array.newInstance(array.getClass().getComponentType(), newSize);
+      System.arraycopy(array, 0, newArray, 0, newSize);
+      return newArray;
+    } else
+      return array;
+  }
+  */
+
   // Since Arrays.equals doesn't implement offsets for equals
   /**
    * See if two array slices are the same.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java Mon Dec 20 10:53:27 2010
@@ -65,4 +65,27 @@ public final class IOUtils {
       throw firstIOE;
     }
   }
+
+  /**
+   * <p>Closes all given <tt>Closeable</tt>s, suppressing all thrown exceptions. Some of the <tt>Closeable</tt>s
+   * may be null, they are ignored. After everything is closed, method either throws the first of suppressed exceptions,
+   * or completes normally.</p>
+   * @param objects         objects to call <tt>close()</tt> on
+   */
+  public static void closeSafely(Closeable... objects) throws IOException {
+    IOException firstIOE = null;
+
+    for (Closeable object : objects) {
+      try {
+        if (object != null)
+          object.close();
+      } catch (IOException ioe) {
+        if (firstIOE == null)
+          firstIOE = ioe;
+      }
+    }
+
+    if (firstIOE != null)
+      throw firstIOE;
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IntsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IntsRef.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IntsRef.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IntsRef.java Mon Dec 20 10:53:27 2010
@@ -21,7 +21,7 @@ package org.apache.lucene.util;
  *  existing int[].
  *
  *  @lucene.internal */
-public final class IntsRef {
+public final class IntsRef implements Comparable<IntsRef> {
 
   public int[] ints;
   public int offset;
@@ -81,6 +81,31 @@ public final class IntsRef {
     }
   }
 
+  /** 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];
@@ -97,4 +122,18 @@ public final class IntsRef {
       ints = ArrayUtil.grow(ints, newLength);
     }
   }
+
+  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/docvalues/lucene/src/java/org/apache/lucene/util/RamUsageEstimator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RamUsageEstimator.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RamUsageEstimator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RamUsageEstimator.java Mon Dec 20 10:53:27 2010
@@ -43,9 +43,10 @@ public final class RamUsageEstimator {
   public final static int NUM_BYTES_LONG = 8;
   public final static int NUM_BYTES_FLOAT = 4;
   public final static int NUM_BYTES_DOUBLE = 8;
-  public final static int NUM_BYTES_OBJ_HEADER = 8;
-  public final static int NUM_BYTES_OBJ_REF = Constants.JRE_IS_64BIT ? 8 : 4;
-  public final static int NUM_BYTES_ARRAY_HEADER = NUM_BYTES_OBJ_HEADER + NUM_BYTES_INT + NUM_BYTES_OBJ_REF;
+  public final static int NUM_BYTES_CHAR = 2;
+  public final static int NUM_BYTES_OBJECT_HEADER = 8;
+  public final static int NUM_BYTES_OBJECT_REF = Constants.JRE_IS_64BIT ? 8 : 4;
+  public final static int NUM_BYTES_ARRAY_HEADER = NUM_BYTES_OBJECT_HEADER + NUM_BYTES_INT + NUM_BYTES_OBJECT_REF;
 
   private MemoryModel memoryModel;
 
@@ -55,9 +56,6 @@ public final class RamUsageEstimator {
   private int arraySize;
   private int classSize;
 
-  public final static int NUM_BYTES_OBJECT_REF = Constants.JRE_IS_64BIT ? 8 : 4;
-  public final static int NUM_BYTES_CHAR = 2;
-
   private boolean checkInterned;
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java Mon Dec 20 10:53:27 2010
@@ -96,7 +96,7 @@ public final class RecyclingByteBlockAll
     final int size = freeBlocks + numBlocks;
     if (size >= freeByteBlocks.length) {
       final byte[][] newBlocks = new byte[ArrayUtil.oversize(size,
-          RamUsageEstimator.NUM_BYTES_OBJ_REF)][];
+          RamUsageEstimator.NUM_BYTES_OBJECT_REF)][];
       System.arraycopy(freeByteBlocks, 0, newBlocks, 0, freeBlocks);
       freeByteBlocks = newBlocks;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java Mon Dec 20 10:53:27 2010
@@ -281,7 +281,7 @@ public class Automaton implements Serial
             worklist.add(t.to);
             t.to.number = upto;
             if (upto == numberedStates.length) {
-              final State[] newArray = new State[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJ_REF)];
+              final State[] newArray = new State[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
               System.arraycopy(numberedStates, 0, newArray, 0, upto);
               numberedStates = newArray;
             }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java Mon Dec 20 10:53:27 2010
@@ -459,7 +459,7 @@ final public class BasicOperations {
 
     public void add(Transition t) {
       if (transitions.length == count) {
-        Transition[] newArray = new Transition[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJ_REF)];
+        Transition[] newArray = new Transition[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
         System.arraycopy(transitions, 0, newArray, 0, count);
         transitions = newArray;
       }
@@ -503,7 +503,7 @@ final public class BasicOperations {
     private PointTransitions next(int point) {
       // 1st time we are seeing this point
       if (count == points.length) {
-        final PointTransitions[] newArray = new PointTransitions[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJ_REF)];
+        final PointTransitions[] newArray = new PointTransitions[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
         System.arraycopy(points, 0, newArray, 0, count);
         points = newArray;
       }
@@ -650,7 +650,7 @@ final public class BasicOperations {
             final SortedIntSet.FrozenIntSet p = statesSet.freeze(q);
             worklist.add(p);
             if (newStateUpto == newStatesArray.length) {
-              final State[] newArray = new State[ArrayUtil.oversize(1+newStateUpto, RamUsageEstimator.NUM_BYTES_OBJ_REF)];
+              final State[] newArray = new State[ArrayUtil.oversize(1+newStateUpto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
               System.arraycopy(newStatesArray, 0, newArray, 0, newStateUpto);
               newStatesArray = newArray;
             }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java Mon Dec 20 10:53:27 2010
@@ -111,7 +111,7 @@ public class State implements Serializab
    */
   public void addTransition(Transition t) {
     if (numTransitions == transitionsArray.length) {
-      final Transition[] newArray = new Transition[ArrayUtil.oversize(1+numTransitions, RamUsageEstimator.NUM_BYTES_OBJ_REF)];
+      final Transition[] newArray = new Transition[ArrayUtil.oversize(1+numTransitions, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(transitionsArray, 0, newArray, 0, numTransitions);
       transitionsArray = newArray;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java Mon Dec 20 10:53:27 2010
@@ -299,7 +299,7 @@ final class UTF32ToUTF8 {
   private State newUTF8State() {
     State s = new State();
     if (utf8StateCount == utf8States.length) {
-      final State[] newArray = new State[ArrayUtil.oversize(1+utf8StateCount, RamUsageEstimator.NUM_BYTES_OBJ_REF)];
+      final State[] newArray = new State[ArrayUtil.oversize(1+utf8StateCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(utf8States, 0, newArray, 0, utf8StateCount);
       utf8States = newArray;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py Mon Dec 20 10:53:27 2010
@@ -1,3 +1,18 @@
+# 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 types
 import os
 import sys

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestDemo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestDemo.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestDemo.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestDemo.java Mon Dec 20 10:53:27 2010
@@ -50,6 +50,7 @@ public class TestDemo extends LuceneTest
     // To store an index on disk, use this instead:
     //Directory directory = FSDirectory.open("/tmp/testindex");
     RandomIndexWriter iwriter = new RandomIndexWriter(random, directory);
+    iwriter.w.setInfoStream(VERBOSE ? System.out : null);
     Document doc = new Document();
     String longTerm = "longtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongterm";
     String text = "This is the text to be indexed. " + longTerm;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestMergeSchedulerExternal.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestMergeSchedulerExternal.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestMergeSchedulerExternal.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestMergeSchedulerExternal.java Mon Dec 20 10:53:27 2010
@@ -105,6 +105,5 @@ public class TestMergeSchedulerExternal 
     assertTrue(mergeCalled);
     assertTrue(excCalled);
     dir.close();
-    assertTrue(ConcurrentMergeScheduler.anyUnhandledExceptions());
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java Mon Dec 20 10:53:27 2010
@@ -17,7 +17,6 @@ package org.apache.lucene;
  * limitations under the License.
  */
 
-import java.util.GregorianCalendar;
 import java.util.Random;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -77,7 +76,6 @@ public class TestSearch extends LuceneTe
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
       LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       IndexWriter writer = new IndexWriter(directory, conf);
 
       String[] docs = {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java Mon Dec 20 10:53:27 2010
@@ -82,8 +82,11 @@ public class TestSearchForDuplicates ext
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
       LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFiles);
-      lmp.setUseCompoundDocStore(useCompoundFiles);
       IndexWriter writer = new IndexWriter(directory, conf);
+      if (VERBOSE) {
+        System.out.println("TEST: now build index");
+        writer.setInfoStream(System.out);
+      }
 
       final int MAX_DOCS = 225;
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Mon Dec 20 10:53:27 2010
@@ -58,6 +58,7 @@ public class Test2BTerms extends LuceneT
       bytes.length = TOKEN_LEN;
     }
     
+    @Override
     public boolean incrementToken() {
       if (tokenCount >= tokensPerDoc) {
         return false;
@@ -67,6 +68,7 @@ public class Test2BTerms extends LuceneT
       return true;
     }
 
+    @Override
     public void reset() {
       tokenCount = 0;
     }
@@ -131,7 +133,7 @@ public class Test2BTerms extends LuceneT
 
     int TERMS_PER_DOC = 1000000;
 
-    Directory dir = FSDirectory.open(_TestUtil.getTempDir("2BTerms"));
+    Directory dir = newFSDirectory(_TestUtil.getTempDir("2BTerms"));
     IndexWriter w = new IndexWriter(
         dir,
         new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Mon Dec 20 10:53:27 2010
@@ -39,7 +39,6 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.Version;
 import org.apache.lucene.util._TestUtil;
 
 public class TestAddIndexes extends LuceneTestCase {
@@ -571,7 +570,6 @@ public class TestAddIndexes extends Luce
     Directory dir = newDirectory();
     LogByteSizeMergePolicy lmp = new LogByteSizeMergePolicy();
     lmp.setUseCompoundFile(false);
-    lmp.setUseCompoundDocStore(false);
     lmp.setMergeFactor(100);
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer())
@@ -600,7 +598,6 @@ public class TestAddIndexes extends Luce
     lmp = new LogByteSizeMergePolicy();
     lmp.setMinMergeMB(0.0001);
     lmp.setUseCompoundFile(false);
-    lmp.setUseCompoundDocStore(false);
     lmp.setMergeFactor(4);
     writer = new IndexWriter(dir2, newIndexWriterConfig(TEST_VERSION_CURRENT,
         new MockAnalyzer())
@@ -642,9 +639,11 @@ public class TestAddIndexes extends Luce
         addDoc(writer);
       writer.close();
 
-      dir2 = new MockDirectoryWrapper(random, new RAMDirectory());
+      dir2 = newDirectory();
       writer2 = new IndexWriter(dir2, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+      writer2.setInfoStream(VERBOSE ? System.out : null);
       writer2.commit();
+      
 
       readers = new IndexReader[NUM_COPY];
       for(int i=0;i<NUM_COPY;i++)
@@ -914,14 +913,17 @@ public class TestAddIndexes extends Luce
 
   // LUCENE-1335: test simultaneous addIndexes & close
   public void testAddIndexesWithRollback() throws Throwable {
-    
+
     final int NUM_COPY = 50;
     CommitAndAddIndexes3 c = new CommitAndAddIndexes3(NUM_COPY);
     c.launchThreads(-1);
 
-    Thread.sleep(500);
+    Thread.sleep(_TestUtil.nextInt(random, 100, 500));
 
     // Close w/o first stopping/joining the threads
+    if (VERBOSE) {
+      System.out.println("TEST: now force rollback");
+    }
     c.didClose = true;
     c.writer2.rollback();
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java Mon Dec 20 10:53:27 2010
@@ -195,7 +195,7 @@ public class TestAtomicUpdate extends Lu
 
     // Second in an FSDirectory:
     File dirPath = _TestUtil.getTempDir("lucene.test.atomic");
-    directory = FSDirectory.open(dirPath);
+    directory = newFSDirectory(dirPath);
     runTest(directory);
     directory.close();
     _TestUtil.rmDir(dirPath);

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Dec 20 10:53:27 2010
@@ -18,12 +18,12 @@ package org.apache.lucene.index;
  */
 
 import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.util.Arrays;
 import java.util.Enumeration;
@@ -36,20 +36,19 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.document.NumericField;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Bits;
 
 /*
   Verify we can read the pre-4.0 file format, do searches
@@ -149,10 +148,13 @@ public class TestBackwardsCompatibility 
   /** This test checks that *only* IndexFormatTooOldExceptions are throws when you open and operate on too old indexes! */
   public void testUnsupportedOldIndexes() throws Exception {
     for(int i=0;i<unsupportedNames.length;i++) {
+      if (VERBOSE) {
+        System.out.println("TEST: index " + unsupportedNames[i]);
+      }
       unzip(getDataFile("unsupported." + unsupportedNames[i] + ".zip"), unsupportedNames[i]);
 
       String fullPath = fullDir(unsupportedNames[i]);
-      Directory dir = FSDirectory.open(new File(fullPath));
+      Directory dir = newFSDirectory(new File(fullPath));
 
       IndexReader reader = null;
       IndexWriter writer = null;
@@ -180,6 +182,10 @@ public class TestBackwardsCompatibility 
         fail("IndexWriter creation should not pass for "+unsupportedNames[i]);
       } catch (IndexFormatTooOldException e) {
         // pass
+        if (VERBOSE) {
+          System.out.println("TEST: got expected exc:");
+          e.printStackTrace(System.out);
+        }
       } finally {
         if (reader != null) reader.close();
         reader = null;
@@ -190,6 +196,7 @@ public class TestBackwardsCompatibility 
             // OK -- since IW gives merge scheduler a chance
             // to merge at close, it's possible and fine to
             // hit this exc here
+            writer.close(false);
           }
         }
         writer = null;
@@ -212,7 +219,7 @@ public class TestBackwardsCompatibility 
       unzip(getDataFile("index." + oldNames[i] + ".zip"), oldNames[i]);
 
       String fullPath = fullDir(oldNames[i]);
-      Directory dir = FSDirectory.open(new File(fullPath));
+      Directory dir = newFSDirectory(new File(fullPath));
 
       IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(
           TEST_VERSION_CURRENT, new MockAnalyzer()));
@@ -230,7 +237,7 @@ public class TestBackwardsCompatibility 
     for (String name : oldNames) {
       unzip(getDataFile("index." + name + ".zip"), name);
       String fullPath = fullDir(name);
-      Directory dir = FSDirectory.open(new File(fullPath));
+      Directory dir = newFSDirectory(new File(fullPath));
 
       Directory targetDir = newDirectory();
       IndexWriter w = new IndexWriter(targetDir, newIndexWriterConfig(
@@ -250,7 +257,7 @@ public class TestBackwardsCompatibility 
     for (String name : oldNames) {
       unzip(getDataFile("index." + name + ".zip"), name);
       String fullPath = fullDir(name);
-      Directory dir = FSDirectory.open(new File(fullPath));
+      Directory dir = newFSDirectory(new File(fullPath));
       IndexReader reader = IndexReader.open(dir);
       
       Directory targetDir = newDirectory();
@@ -286,6 +293,9 @@ public class TestBackwardsCompatibility 
 
   public void testIndexOldIndex() throws IOException {
     for(int i=0;i<oldNames.length;i++) {
+      if (VERBOSE) {
+        System.out.println("TEST: oldName=" + oldNames[i]);
+      }
       unzip(getDataFile("index." + oldNames[i] + ".zip"), oldNames[i]);
       changeIndexWithAdds(random, oldNames[i]);
       rmDir(oldNames[i]);
@@ -307,7 +317,7 @@ public class TestBackwardsCompatibility 
 
     dirName = fullDir(dirName);
 
-    Directory dir = FSDirectory.open(new File(dirName));
+    Directory dir = newFSDirectory(new File(dirName));
     IndexSearcher searcher = new IndexSearcher(dir, true);
     IndexReader reader = searcher.getIndexReader();
 
@@ -337,6 +347,10 @@ public class TestBackwardsCompatibility 
           f = d.getField("fie\u2C77ld");
           assertEquals("field with non-ascii name", f.stringValue());
         }
+
+        TermFreqVector tfv = reader.getTermFreqVector(i, "utf8");
+        assertNotNull("docID=" + i + " index=" + dirName, tfv);
+        assertTrue(tfv instanceof TermPositionVector);
       } else
         // Only ID 7 is deleted
         assertEquals(7, i);
@@ -372,9 +386,10 @@ public class TestBackwardsCompatibility 
     String origDirName = dirName;
     dirName = fullDir(dirName);
 
-    Directory dir = FSDirectory.open(new File(dirName));
+    Directory dir = newFSDirectory(new File(dirName));
     // open writer
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.APPEND));
+    writer.setInfoStream(VERBOSE ? System.out : null);
     // add 10 docs
     for(int i=0;i<10;i++) {
       addDoc(writer, 35+i);
@@ -437,7 +452,7 @@ public class TestBackwardsCompatibility 
 
     dirName = fullDir(dirName);
 
-    Directory dir = FSDirectory.open(new File(dirName));
+    Directory dir = newFSDirectory(new File(dirName));
 
     // make sure searching sees right # hits
     IndexSearcher searcher = new IndexSearcher(dir, true);
@@ -486,10 +501,9 @@ public class TestBackwardsCompatibility 
 
     dirName = fullDir(dirName);
 
-    Directory dir = FSDirectory.open(new File(dirName));
+    Directory dir = newFSDirectory(new File(dirName));
     IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(10);
     ((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(doCFS);
-    ((LogMergePolicy) conf.getMergePolicy()).setUseCompoundDocStore(doCFS);
     IndexWriter writer = new IndexWriter(dir, conf);
     
     for(int i=0;i<35;i++) {
@@ -501,7 +515,6 @@ public class TestBackwardsCompatibility 
     // open fresh writer so we get no prx file in the added segment
     conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(10);
     ((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(doCFS);
-    ((LogMergePolicy) conf.getMergePolicy()).setUseCompoundDocStore(doCFS);
     writer = new IndexWriter(dir, conf);
     addNoProxDoc(writer);
     writer.close();
@@ -525,7 +538,7 @@ public class TestBackwardsCompatibility 
     rmDir(outputDir);
 
     try {
-      Directory dir = FSDirectory.open(new File(fullDir(outputDir)));
+      Directory dir = newFSDirectory(new File(fullDir(outputDir)));
 
       LogMergePolicy mergePolicy = newLogMergePolicy(true, 10);
       mergePolicy.setNoCFSRatio(1); // This test expects all of its segments to be in CFS
@@ -658,7 +671,7 @@ public class TestBackwardsCompatibility 
     for(int i=0;i<oldNames.length;i++) {
       unzip(getDataFile("index." + oldNames[i] + ".zip"), oldNames[i]);
       String fullPath = fullDir(oldNames[i]);
-      Directory dir = FSDirectory.open(new File(fullPath));
+      Directory dir = newFSDirectory(new File(fullPath));
       IndexReader r = IndexReader.open(dir);
       TermsEnum terms = MultiFields.getFields(r).terms("content").iterator();
       BytesRef t = terms.next();
@@ -704,7 +717,7 @@ public class TestBackwardsCompatibility 
       
       unzip(getDataFile("index." + oldNames[i] + ".zip"), oldNames[i]);
       String fullPath = fullDir(oldNames[i]);
-      Directory dir = FSDirectory.open(new File(fullPath));
+      Directory dir = newFSDirectory(new File(fullPath));
       IndexSearcher searcher = new IndexSearcher(dir, true);
       
       for (int id=10; id<15; id++) {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestCodecs.java Mon Dec 20 10:53:27 2010
@@ -271,7 +271,7 @@ public class TestCodecs extends LuceneTe
 
     final Directory dir = newDirectory();
     this.write(fieldInfos, dir, fields);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()));
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
     si.setHasProx(false);
 
     final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
@@ -319,7 +319,7 @@ public class TestCodecs extends LuceneTe
     final Directory dir = newDirectory();
 
     this.write(fieldInfos, dir, fields);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()));
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
 
     final FieldsProducer terms = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
 
@@ -608,8 +608,7 @@ public class TestCodecs extends LuceneTe
 
     final int termIndexInterval = this.nextInt(13, 27);
     final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, CodecProvider.getDefault());
-    final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, null, 10000, 10000, termIndexInterval, codecInfo, new AtomicLong());
-
+    final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo, new AtomicLong());
     final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
     Arrays.sort(fields);
     for (final FieldData field : fields) {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Mon Dec 20 10:53:27 2010
@@ -57,9 +57,9 @@ public class TestConcurrentMergeSchedule
             isClose = true;
           }
         }
-        if (isDoFlush && !isClose) {
+        if (isDoFlush && !isClose && random.nextBoolean()) {
           hitExc = true;
-          throw new IOException("now failing during flush");
+          throw new IOException(Thread.currentThread().getName() + ": now failing during flush");
         }
       }
     }
@@ -73,12 +73,17 @@ public class TestConcurrentMergeSchedule
     directory.failOn(failure);
 
     IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(2));
+    writer.setInfoStream(VERBOSE ? System.out : null);
     Document doc = new Document();
     Field idField = newField("id", "", Field.Store.YES, Field.Index.NOT_ANALYZED);
     doc.add(idField);
     int extraCount = 0;
 
     for(int i=0;i<10;i++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + i);
+      }
+
       for(int j=0;j<20;j++) {
         idField.setValue(Integer.toString(i*20+j));
         writer.addDocument(doc);
@@ -91,16 +96,20 @@ public class TestConcurrentMergeSchedule
         writer.addDocument(doc);
         failure.setDoFail();
         try {
-          writer.flush(true, false, true);
+          writer.flush(true, true);
           if (failure.hitExc) {
             fail("failed to hit IOException");
           }
           extraCount++;
         } catch (IOException ioe) {
+          if (VERBOSE) {
+            ioe.printStackTrace(System.out);
+          }
           failure.clearDoFail();
           break;
         }
       }
+      assertEquals(20*(i+1)+extraCount, writer.numDocs());
     }
 
     writer.close();
@@ -155,8 +164,12 @@ public class TestConcurrentMergeSchedule
     IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer())
         .setMaxBufferedDocs(2));
+    writer.setInfoStream(VERBOSE ? System.out : null);
 
     for(int iter=0;iter<7;iter++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter);
+      }
 
       for(int j=0;j<21;j++) {
         Document doc = new Document();
@@ -171,6 +184,7 @@ public class TestConcurrentMergeSchedule
       writer = new IndexWriter(directory, newIndexWriterConfig(
           TEST_VERSION_CURRENT, new MockAnalyzer())
           .setOpenMode(OpenMode.APPEND).setMaxBufferedDocs(2));
+      writer.setInfoStream(VERBOSE ? System.out : null);
     }
 
     writer.close();

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java Mon Dec 20 10:53:27 2010
@@ -195,16 +195,13 @@ public class TestDeletionPolicy extends 
 
     final double SECONDS = 2.0;
 
-    boolean useCompoundFile = true;
-    
     Directory dir = newDirectory();
     ExpirationTimeDeletionPolicy policy = new ExpirationTimeDeletionPolicy(dir, SECONDS);
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
         new MockAnalyzer())
         .setIndexDeletionPolicy(policy);
     LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
-    lmp.setUseCompoundFile(useCompoundFile);
-    lmp.setUseCompoundDocStore(useCompoundFile);
+    lmp.setUseCompoundFile(true);
     IndexWriter writer = new IndexWriter(dir, conf);
     writer.close();
 
@@ -219,8 +216,7 @@ public class TestDeletionPolicy extends 
           new MockAnalyzer()).setOpenMode(
           OpenMode.APPEND).setIndexDeletionPolicy(policy);
       lmp = (LogMergePolicy) conf.getMergePolicy();
-      lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
+      lmp.setUseCompoundFile(true);
       writer = new IndexWriter(dir, conf);
       for(int j=0;j<17;j++) {
         addDoc(writer);
@@ -298,7 +294,6 @@ public class TestDeletionPolicy extends 
           .setMergeScheduler(new SerialMergeScheduler());
       LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       lmp.setMergeFactor(10);
       IndexWriter writer = new IndexWriter(dir, conf);
       for(int i=0;i<107;i++) {
@@ -311,7 +306,6 @@ public class TestDeletionPolicy extends 
           OpenMode.APPEND).setIndexDeletionPolicy(policy);
       lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       writer = new IndexWriter(dir, conf);
       writer.optimize();
       writer.close();
@@ -488,7 +482,6 @@ public class TestDeletionPolicy extends 
           .setMaxBufferedDocs(10);
       LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       IndexWriter writer = new IndexWriter(dir, conf);
       for(int i=0;i<107;i++) {
         addDoc(writer);
@@ -499,7 +492,6 @@ public class TestDeletionPolicy extends 
           .setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
       lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       writer = new IndexWriter(dir, conf);
       writer.optimize();
       writer.close();
@@ -539,7 +531,6 @@ public class TestDeletionPolicy extends 
             .setMaxBufferedDocs(10);
         LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
         lmp.setUseCompoundFile(useCompoundFile);
-        lmp.setUseCompoundDocStore(useCompoundFile);
         IndexWriter writer = new IndexWriter(dir, conf);
         for(int i=0;i<17;i++) {
           addDoc(writer);
@@ -597,7 +588,6 @@ public class TestDeletionPolicy extends 
           .setOpenMode(OpenMode.CREATE).setIndexDeletionPolicy(policy);
       LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       IndexWriter writer = new IndexWriter(dir, conf);
       writer.close();
       Term searchTerm = new Term("content", "aaa");        
@@ -609,7 +599,6 @@ public class TestDeletionPolicy extends 
             .setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
         lmp = (LogMergePolicy) conf.getMergePolicy();
         lmp.setUseCompoundFile(useCompoundFile);
-        lmp.setUseCompoundDocStore(useCompoundFile);
         writer = new IndexWriter(dir, conf);
         for(int j=0;j<17;j++) {
           addDoc(writer);
@@ -630,7 +619,6 @@ public class TestDeletionPolicy extends 
           .setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(policy);
       lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       writer = new IndexWriter(dir, conf);
       writer.optimize();
       // this is a commit
@@ -706,7 +694,6 @@ public class TestDeletionPolicy extends 
           .setMaxBufferedDocs(10);
       LogMergePolicy lmp = (LogMergePolicy) conf.getMergePolicy();
       lmp.setUseCompoundFile(useCompoundFile);
-      lmp.setUseCompoundDocStore(useCompoundFile);
       IndexWriter writer = new IndexWriter(dir, conf);
       writer.close();
       Term searchTerm = new Term("content", "aaa");        
@@ -720,7 +707,6 @@ public class TestDeletionPolicy extends 
             .setMaxBufferedDocs(10);
         lmp = (LogMergePolicy) conf.getMergePolicy();
         lmp.setUseCompoundFile(useCompoundFile);
-        lmp.setUseCompoundDocStore(useCompoundFile);
         writer = new IndexWriter(dir, conf);
         for(int j=0;j<17;j++) {
           addDoc(writer);