You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/10/01 05:05:07 UTC

svn commit: r1177888 [4/16] - in /lucene/dev/branches/lucene2621: ./ dev-tools/eclipse/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/demo/src/java/org/apache...

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Floats.java Sat Oct  1 03:04:53 2011
@@ -17,21 +17,14 @@ package org.apache.lucene.index.values;
  * limitations under the License.
  */
 import java.io.IOException;
-import java.util.Collection;
 
-import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FloatsRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Exposes {@link Writer} and reader ({@link Source}) for 32 bit and 64 bit
@@ -43,11 +36,6 @@ import org.apache.lucene.util.RamUsageEs
  * @lucene.experimental
  */
 public class Floats {
-  // TODO - add bulk copy where possible
-  private static final String CODEC_NAME = "SimpleFloats";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
-  private static final byte[] DEFAULTS = new byte[] {0,0,0,0,0,0,0,0};
   
   public static Writer getWriter(Directory dir, String id, int precisionBytes,
       Counter bytesUsed, IOContext context) throws IOException {
@@ -55,493 +43,73 @@ public class Floats {
       throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
           + precisionBytes);
     }
-    if (precisionBytes == 4) {
-      return new Float4Writer(dir, id, bytesUsed, context);
-    } else {
-      return new Float8Writer(dir, id, bytesUsed, context);
-    }
+    return new FloatsWriter(dir, id, bytesUsed, context, precisionBytes);
+
   }
 
   public static IndexDocValues getValues(Directory dir, String id, int maxDoc, IOContext context)
       throws IOException {
     return new FloatsReader(dir, id, maxDoc, context);
   }
-
-  abstract static class FloatsWriter extends Writer {
-    private final String id;
-    protected FloatsRef floatsRef;
-    protected int lastDocId = -1;
-    protected IndexOutput datOut;
-    private final byte precision;
-    private final Directory dir;
-    private final IOContext context; 
-
-    protected FloatsWriter(Directory dir, String id, int precision,
-        Counter bytesUsed, IOContext context) throws IOException {
-      super(bytesUsed);
-      this.id = id;
-      this.precision = (byte) precision;
-      this.dir = dir;
-      this.context = context;
-     
-    }
-
-    public long ramBytesUsed() {
-      return 0;
+  
+  final static class FloatsWriter extends FixedStraightBytesImpl.Writer {
+    private final int size; 
+    public FloatsWriter(Directory dir, String id, Counter bytesUsed,
+        IOContext context, int size) throws IOException {
+      super(dir, id, bytesUsed, context);
+      this.bytesRef = new BytesRef(size);
+      this.size = size;
+      bytesRef.length = size;
     }
     
-    final void initDataOut() throws IOException {
-      assert datOut == null;
-      datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
-          Writer.DATA_EXTENSION), context);
-      boolean success = false;
-      try {
-        CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
-        assert datOut.getFilePointer() == CodecUtil.headerLength(CODEC_NAME);
-        datOut.writeByte(this.precision);
-        success = true;
-      } finally {
-        if (!success) {
-          IOUtils.closeWhileHandlingException(datOut);
-        }
+    public void add(int docID, double v) throws IOException {
+      if (size == 8) {
+        bytesRef.copy(Double.doubleToRawLongBits(v));        
+      } else {
+        bytesRef.copy(Float.floatToRawIntBits((float)v));
       }
+      add(docID, bytesRef);
     }
-
-    @Override
-    protected void mergeDoc(int docID) throws IOException {
-      add(docID, floatsRef.get());
-    }
-
+    
     @Override
     public void add(int docID, PerDocFieldValues docValues) throws IOException {
       add(docID, docValues.getFloat());
     }
-
-    @Override
-    protected void setNextEnum(ValuesEnum valuesEnum) {
-      floatsRef = valuesEnum.getFloat();
-    }
-
-    protected final int fillDefault(int numValues) throws IOException {
-      for (int i = 0; i < numValues; i++) {
-        datOut.writeBytes(DEFAULTS, precision);
-      }
-      return numValues;
-    }
-
-    @Override
-    protected void merge(MergeState state) throws IOException {
-      if (datOut == null) {
-        initDataOut();
-      }
-      if (state.liveDocs == null && state.reader instanceof FloatsReader) {
-        // no deletes - bulk copy
-        final FloatsReader reader = (FloatsReader) state.reader;
-        assert reader.precisionBytes == (int) precision;
-        if (reader.maxDoc == 0)
-          return;
-        final int docBase = state.docBase;
-        if (docBase - lastDocId > 1) {
-          // fill with default values
-          lastDocId += fillDefault(docBase - lastDocId - 1);
-        }
-        lastDocId += reader.transferTo(datOut);
-      } else {
-        super.merge(state);        
-      }
-
-    }
-
-    @Override
-    public void files(Collection<String> files) throws IOException {
-      files.add(IndexFileNames.segmentFileName(id, "", Writer.DATA_EXTENSION));
-    }
-  }
-
-  // Writes 4 bytes (float) per value
-  static final class Float4Writer extends FloatsWriter {
-    private int[] values;
-    protected Float4Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
-        throws IOException {
-      super(dir, id, 4, bytesUsed, context);
-      values = new int[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
-    }
-
-    @Override
-    public void add(final int docID, final double v)
-        throws IOException {
-      assert docID > lastDocId : "docID: " + docID
-          + " must be greater than the last added doc id: " + lastDocId;
-      if (docID >= values.length) {
-        final long len = values.length;
-        values = ArrayUtil.grow(values, 1 + docID);
-        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-            * ((values.length) - len));
-      }
-      values[docID] = Float.floatToRawIntBits((float)v);
-      lastDocId = docID;
-    }
-
-    @Override
-    protected void mergeDoc(int docID) throws IOException {
-      assert datOut != null;
-      assert docID > lastDocId : "docID: " + docID
-      + " must be greater than the last added doc id: " + lastDocId;
-      if (docID - lastDocId > 1) {
-        // fill with default values
-        fillDefault(docID - lastDocId - 1);
-      }
-      assert datOut != null;
-      datOut.writeInt(Float.floatToRawIntBits((float) floatsRef.get()));
-      lastDocId = docID;
-    }
-
-    @Override
-    public void finish(int docCount) throws IOException {
-      boolean success = false;
-      try {
-        int numDefaultsToAppend = docCount - (lastDocId + 1);
-        if (datOut == null) {
-          initDataOut();
-          for (int i = 0; i <= lastDocId; i++) {
-            datOut.writeInt(values[i]);
-          }
-        }
-        fillDefault(numDefaultsToAppend);
-        success = true;
-      } finally {
-        bytesUsed.addAndGet(-(RamUsageEstimator.NUM_BYTES_INT
-            * ((values.length))));
-        values = null;
-        if (success) {
-          IOUtils.close(datOut);
-        } else {
-          IOUtils.closeWhileHandlingException(datOut);
-        }
-      }
-    }
   }
 
-  // Writes 8 bytes (double) per value
-  static final class Float8Writer extends FloatsWriter {
-    private long[] values;
-    protected Float8Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
-        throws IOException {
-      super(dir, id, 8, bytesUsed, context);
-      values = new long[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG);
-    }
-
-    @Override
-    public void add(int docID, double v) throws IOException {
-      assert docID > lastDocId : "docID: " + docID
-          + " must be greater than the last added doc id: " + lastDocId;
-      if (docID >= values.length) {
-        final long len = values.length;
-        values = ArrayUtil.grow(values, 1 + docID);
-        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG
-            * ((values.length) - len));
+  
+  final static class FloatsReader extends FixedStraightBytesImpl.Reader {
+    final IndexDocValuesArray arrayTemplate;
+    FloatsReader(Directory dir, String id, int maxDoc, IOContext context)
+        throws IOException {
+      super(dir, id, maxDoc, context);
+      assert size == 4 || size == 8;
+      if (size == 4) {
+        arrayTemplate = new IndexDocValuesArray.FloatValues();
+      } else {
+        arrayTemplate = new IndexDocValuesArray.DoubleValues();
       }
-      values[docID] = Double.doubleToLongBits(v);
-      lastDocId = docID;
     }
     
     @Override
-    protected void mergeDoc(int docID) throws IOException {
-      assert docID > lastDocId : "docID: " + docID
-      + " must be greater than the last added doc id: " + lastDocId;
-      if (docID - lastDocId > 1) {
-        // fill with default values
-        lastDocId += fillDefault(docID - lastDocId - 1);
-      }
-      assert datOut != null;
-      datOut.writeLong(Double.doubleToRawLongBits((float) floatsRef.get()));
-      lastDocId = docID;
-    }
-
-    @Override
-    public void finish(int docCount) throws IOException {
-      boolean success = false;
-      try {
-        int numDefaultsToAppend = docCount - (lastDocId + 1);
-        if (datOut == null) {
-          initDataOut();
-          for (int i = 0; i <= lastDocId; i++) {
-            datOut.writeLong(values[i]);
-          }
-        }
-        fillDefault(numDefaultsToAppend);
-        success = true;
-      } finally {
-        bytesUsed.addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG
-            * ((values.length))));
-        values = null;
-        if (success) {
-          IOUtils.close(datOut);
-        } else {
-          IOUtils.closeWhileHandlingException(datOut);
-        }
-      }
-    }
-  }
-
-  /**
-   * Opens all necessary files, but does not read any data in until you call
-   * {@link #load}.
-   */
-  static class FloatsReader extends IndexDocValues {
-
-    private final IndexInput datIn;
-    private final int precisionBytes;
-    // TODO(simonw) is ByteBuffer the way to go here?
-    private final int maxDoc;
-
-    protected FloatsReader(Directory dir, String id, int maxDoc, IOContext context)
-        throws IOException {
-      datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
-          Writer.DATA_EXTENSION), context);
-      CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
-      precisionBytes = datIn.readByte();
-      assert precisionBytes == 4 || precisionBytes == 8;
-      this.maxDoc = maxDoc;
-    }
-
-    int transferTo(IndexOutput out) throws IOException {
-      IndexInput indexInput = (IndexInput) datIn.clone();
+    public Source load() throws IOException {
+      final IndexInput indexInput = cloneData();
       try {
-        indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
-        // skip precision:
-        indexInput.readByte();
-        out.copyBytes(indexInput, precisionBytes * maxDoc);
+        return arrayTemplate.newFromInput(indexInput, maxDoc);
       } finally {
         indexInput.close();
       }
-      return maxDoc;
     }
-
-    /**
-     * Loads the actual values. You may call this more than once, eg if you
-     * already previously loaded but then discarded the Source.
-     */
-    @Override
-    public Source load() throws IOException {
-      /* we always read BIG_ENDIAN here since the writer uses
-       * DataOutput#writeInt() / writeLong() we can simply read the ints / longs
-       * back in using readInt / readLong */
-      final IndexInput indexInput = (IndexInput) datIn.clone();
-      indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
-      // skip precision:
-      indexInput.readByte();
-      if (precisionBytes == 4) {
-        final float[] values = new float[(4 * maxDoc) >> 2];
-        assert values.length == maxDoc;
-        for (int i = 0; i < values.length; i++) {
-          values[i] = Float.intBitsToFloat(indexInput.readInt());
-        }
-        return new Source4(values);
-      } else {
-        final double[] values = new double[(8 * maxDoc) >> 3];
-        assert values.length == maxDoc;
-        for (int i = 0; i < values.length; i++) {
-          values[i] = Double.longBitsToDouble(indexInput.readLong());
-        }
-        return new Source8(values);
-      }
-    }
-
-    private final class Source4 extends Source {
-      private final float[] values;
-
-      Source4(final float[] values ) throws IOException {
-        this.values = values;
-      }
-
-      @Override
-      public double getFloat(int docID) {
-        return values[docID];
-      }
-
-      @Override
-      public ValuesEnum getEnum(AttributeSource attrSource)
-          throws IOException {
-        return new SourceEnum(attrSource, ValueType.FLOAT_32, this, maxDoc) {
-          @Override
-          public int advance(int target) throws IOException {
-            if (target >= numDocs)
-              return pos = NO_MORE_DOCS;
-            floatsRef.floats[floatsRef.offset] = source.getFloat(target);
-            return pos = target;
-          }
-        };
-      }
-
-      @Override
-      public Object getArray() {
-        return this.values;
-      }
-
-      @Override
-      public boolean hasArray() {
-        return true;
-      }
-
-      @Override
-      public ValueType type() {
-        return ValueType.FLOAT_32;
-      }
-    }
-
-    private final class Source8 extends Source {
-      private final double[] values;
-
-      Source8(final double[] values) throws IOException {
-        this.values = values;
-      }
-
-      @Override
-      public double getFloat(int docID) {
-        return values[docID];
-      }
-
-      @Override
-      public ValuesEnum getEnum(AttributeSource attrSource)
-          throws IOException {
-        return new SourceEnum(attrSource, type(), this, maxDoc) {
-          @Override
-          public int advance(int target) throws IOException {
-            if (target >= numDocs)
-              return pos = NO_MORE_DOCS;
-            floatsRef.floats[floatsRef.offset] = source.getFloat(target);
-            return pos = target;
-          }
-        };
-      }
-
-      @Override
-      public ValueType type() {
-        return ValueType.FLOAT_64;
-      }
-      
-      @Override
-      public Object getArray() {
-        return this.values;
-      }
-
-      @Override
-      public boolean hasArray() {
-        return true;
-      }
-    }
-
-    @Override
-    public void close() throws IOException {
-      super.close();
-      datIn.close();
-    }
-
-    @Override
+    
     public ValuesEnum getEnum(AttributeSource source) throws IOException {
       IndexInput indexInput = (IndexInput) datIn.clone();
-      indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
-      // skip precision:
-      indexInput.readByte();
-      return precisionBytes == 4 ? new Floats4Enum(source, indexInput, maxDoc)
-          : new Floats8EnumImpl(source, indexInput, maxDoc);
+      return arrayTemplate.getDirectEnum(source, indexInput, maxDoc);
     }
 
     @Override
     public ValueType type() {
-      return precisionBytes == 4 ? ValueType.FLOAT_32
-          : ValueType.FLOAT_64;
+      return arrayTemplate.type();
     }
   }
 
-  static final class Floats4Enum extends FloatsEnumImpl {
-
-    Floats4Enum(AttributeSource source, IndexInput dataIn, int maxDoc)
-        throws IOException {
-      super(source, dataIn, 4, maxDoc, ValueType.FLOAT_32);
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      if (target >= maxDoc)
-        return pos = NO_MORE_DOCS;
-      dataIn.seek(fp + (target * precision));
-      final int intBits = dataIn.readInt();
-      floatsRef.floats[0] = Float.intBitsToFloat(intBits);
-      floatsRef.offset = 0;
-      return pos = target;
-    }
-
-    @Override
-    public int docID() {
-      return pos;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (pos >= maxDoc) {
-        return pos = NO_MORE_DOCS;
-      }
-      return advance(pos + 1);
-    }
-  }
-
-  private static final class Floats8EnumImpl extends FloatsEnumImpl {
-
-    Floats8EnumImpl(AttributeSource source, IndexInput dataIn, int maxDoc)
-        throws IOException {
-      super(source, dataIn, 8, maxDoc, ValueType.FLOAT_64);
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      if (target >= maxDoc) {
-        return pos = NO_MORE_DOCS;
-      }
-      dataIn.seek(fp + (target * precision));
-      final long value = dataIn.readLong();
-      floatsRef.floats[floatsRef.offset] = Double.longBitsToDouble(value);
-      return pos = target;
-    }
-
-    @Override
-    public int docID() {
-      return pos;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (pos >= maxDoc) {
-        return pos = NO_MORE_DOCS;
-      }
-      return advance(pos + 1);
-    }
-  }
-
-  static abstract class FloatsEnumImpl extends ValuesEnum {
-    protected final IndexInput dataIn;
-    protected int pos = -1;
-    protected final int precision;
-    protected final int maxDoc;
-    protected final long fp;
-
-    FloatsEnumImpl(AttributeSource source, IndexInput dataIn, int precision,
-        int maxDoc, ValueType type) throws IOException {
-      super(source, precision == 4 ? ValueType.FLOAT_32
-          : ValueType.FLOAT_64);
-      this.dataIn = dataIn;
-      this.precision = precision;
-      this.maxDoc = maxDoc;
-      fp = dataIn.getFilePointer();
-      floatsRef.offset = 0;
-    }
-
-    @Override
-    public void close() throws IOException {
-      dataIn.close();
-    }
-  }
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java Sat Oct  1 03:04:53 2011
@@ -2,14 +2,12 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.values.FixedStraightBytesImpl.FixedStraightBytesEnum;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
@@ -34,48 +32,33 @@ import org.apache.lucene.util.RamUsageEs
  */
 abstract class IndexDocValuesArray extends Source {
 
-  private final Counter bytesUsed;
-  private final int bytesPerValue;
-  private int size = 0;
+  protected final int bytesPerValue;
   private final ValueType type;
+  private final boolean isFloat;
   protected int maxDocID = -1;
 
-  IndexDocValuesArray(Counter bytesUsed, int bytesPerValue, ValueType type) {
-    this.bytesUsed = bytesUsed;
+  IndexDocValuesArray(int bytesPerValue, ValueType type) {
     this.bytesPerValue = bytesPerValue;
     this.type = type;
-  }
+    switch (type) {
+    case FIXED_INTS_16:
+    case FIXED_INTS_32:
+    case FIXED_INTS_64:
+    case FIXED_INTS_8:
+      isFloat = false;
+      break;
+    case FLOAT_32:
+    case FLOAT_64:
+      isFloat = true;
+      break;
+    default:
+      throw new IllegalStateException("illegal type: " + type);
 
-  void set(int docId, long value) {
-    if (docId >= size) {
-      adjustSize(grow(docId + 1));
-    }
-    if (docId > maxDocID) {
-      maxDocID = docId;
     }
-    setInternal(docId, value);
   }
 
-  protected final void adjustSize(int newSize) {
-    bytesUsed.addAndGet(bytesPerValue * (newSize - size));
-    size = newSize;
-  }
-
-  void clear() {
-    adjustSize(0);
-    maxDocID = -1;
-    size = 0;
-  }
-  
-  protected abstract void writeDirect(IndexOutput out, long value) throws IOException;
-  
-  protected abstract void writeDefaults(IndexOutput out, int num) throws IOException;
-
-  protected abstract void setInternal(int docId, long value);
-
-  protected abstract int grow(int numDocs);
-
-  abstract void write(IndexOutput output, int numDocs) throws IOException;
+  public abstract IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+      throws IOException;
 
   @Override
   public final int getValueCount() {
@@ -89,21 +72,38 @@ abstract class IndexDocValuesArray exten
 
   @Override
   public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
-    return new SourceEnum(attrSource, type(), this, maxDocID + 1) {
+    if (isFloat) {
+      return new SourceEnum(attrSource, type(), this, maxDocID + 1) {
 
-      @Override
-      public int advance(int target) throws IOException {
-        if (target >= numDocs) {
-          return pos = NO_MORE_DOCS;
+        @Override
+        public int advance(int target) throws IOException {
+          if (target >= numDocs) {
+            return pos = NO_MORE_DOCS;
+          }
+          floatsRef.floats[intsRef.offset] = IndexDocValuesArray.this
+              .getFloat(target);
+          return pos = target;
         }
-        intsRef.ints[intsRef.offset] = IndexDocValuesArray.this.getInt(target);
-        return pos = target;
-      }
-    };
+      };
+    } else {
+      return new SourceEnum(attrSource, type(), this, maxDocID + 1) {
+
+        @Override
+        public int advance(int target) throws IOException {
+          if (target >= numDocs) {
+            return pos = NO_MORE_DOCS;
+          }
+          intsRef.ints[intsRef.offset] = IndexDocValuesArray.this
+              .getInt(target);
+          return pos = target;
+        }
+
+      };
+    }
   }
 
-  abstract ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input, int maxDoc)
-      throws IOException;
+  abstract ValuesEnum getDirectEnum(AttributeSource attrSource,
+      IndexInput input, int maxDoc) throws IOException;
 
   @Override
   public final boolean hasArray() {
@@ -111,17 +111,16 @@ abstract class IndexDocValuesArray exten
   }
 
   final static class ByteValues extends IndexDocValuesArray {
-    private byte[] values;
+    private final byte[] values;
 
-    ByteValues(Counter bytesUsed) {
-      super(bytesUsed, 1, ValueType.FIXED_INTS_8);
+    ByteValues() {
+      super(1, ValueType.FIXED_INTS_8);
       values = new byte[0];
     }
 
-    ByteValues(IndexInput input, int numDocs) throws IOException {
-      super(Counter.newCounter(), 1, ValueType.FIXED_INTS_8);
+    private ByteValues(IndexInput input, int numDocs) throws IOException {
+      super(1, ValueType.FIXED_INTS_8);
       values = new byte[numDocs];
-      adjustSize(numDocs);
       input.readBytes(values, 0, values.length, false);
       maxDocID = numDocs - 1;
     }
@@ -138,69 +137,37 @@ abstract class IndexDocValuesArray exten
     }
 
     @Override
-    protected void setInternal(int docId, long value) {
-      values[docId] = (byte) (0xFFL & value);
-    }
+    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input,
+        int maxDoc) throws IOException {
+      return new FixedIntsEnum(attrSource, input, type(),
+          bytesPerValue, maxDoc) {
 
-    @Override
-    protected int grow(int numDocs) {
-      values = ArrayUtil.grow(values, numDocs);
-      return values.length;
-    }
-
-    @Override
-    void write(IndexOutput output, int numDocs) throws IOException {
-      assert maxDocID + 1 <= numDocs;
-      output.writeBytes(values, 0, maxDocID + 1);
-      writeDefaults(output,  numDocs - (maxDocID+1));
-    }
-
-    @Override
-    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input, int maxDoc)
-        throws IOException {
-      return new FixedIntsEnumImpl(attrSource, input, type(), maxDoc) {
         @Override
-        protected void fillNext(LongsRef ref, IndexInput dataIn)
-            throws IOException {
-          ref.ints[ref.offset] = dataIn.readByte();
+        protected final long toLong(BytesRef bytesRef) {
+          return bytesRef.bytes[bytesRef.offset];
         }
       };
     }
 
     @Override
-    void clear() {
-      super.clear();
-      values = new byte[0];
-    }
-
-    @Override
-    protected void writeDefaults(IndexOutput out, int num) throws IOException {
-      final byte zero = 0;
-      for (int i = 0; i < num; i++) {
-        out.writeByte(zero);
-      }
+    public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+        throws IOException {
+      return new ByteValues(input, numDocs);
     }
 
-    @Override
-    protected void writeDirect(IndexOutput out, long value) throws IOException {
-      out.writeByte((byte) (0xFFL & value));
-    }
   };
 
   final static class ShortValues extends IndexDocValuesArray {
-    private short[] values;
+    private final short[] values;
 
-    ShortValues(Counter bytesUsed) {
-      super(bytesUsed, RamUsageEstimator.NUM_BYTES_SHORT,
-          ValueType.FIXED_INTS_16);
+    ShortValues() {
+      super(RamUsageEstimator.NUM_BYTES_SHORT, ValueType.FIXED_INTS_16);
       values = new short[0];
     }
 
-    ShortValues(IndexInput input, int numDocs) throws IOException {
-      super(Counter.newCounter(), RamUsageEstimator.NUM_BYTES_SHORT,
-          ValueType.FIXED_INTS_16);
+    private ShortValues(IndexInput input, int numDocs) throws IOException {
+      super(RamUsageEstimator.NUM_BYTES_SHORT, ValueType.FIXED_INTS_16);
       values = new short[numDocs];
-      adjustSize(numDocs);
       for (int i = 0; i < values.length; i++) {
         values[i] = input.readShort();
       }
@@ -219,71 +186,37 @@ abstract class IndexDocValuesArray exten
     }
 
     @Override
-    protected void setInternal(int docId, long value) {
-      values[docId] = (short) (0xFFFFL & value);
-    }
+    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input,
+        int maxDoc) throws IOException {
+      return new FixedIntsEnum(attrSource, input, type(),
+          bytesPerValue, maxDoc) {
 
-    @Override
-    protected int grow(int numDocs) {
-      values = ArrayUtil.grow(values, numDocs);
-      return values.length;
-    }
-
-    @Override
-    void write(IndexOutput output, int numDocs) throws IOException {
-      assert maxDocID + 1 <= numDocs;
-      for (int i = 0; i < maxDocID + 1; i++) {
-        output.writeShort(values[i]);
-      }
-      writeDefaults(output,  numDocs - (maxDocID+1));
-    }
-
-    @Override
-    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input, int maxDoc)
-        throws IOException {
-      return new FixedIntsEnumImpl(attrSource, input, type(), maxDoc) {
         @Override
-        protected void fillNext(LongsRef ref, IndexInput dataIn)
-            throws IOException {
-          ref.ints[ref.offset] = dataIn.readShort();
+        protected final long toLong(BytesRef bytesRef) {
+          return bytesRef.asShort();
         }
       };
     }
 
     @Override
-    void clear() {
-      super.clear();
-      values = new short[0];
-    }
-
-    @Override
-    protected void writeDefaults(IndexOutput out, int num) throws IOException {
-      final short zero = 0;
-      for (int i = 0; i < num; i++) {
-        out.writeShort(zero);
-      }
-    }
-    
-    @Override
-    protected void writeDirect(IndexOutput out, long value) throws IOException {
-      out.writeShort((short) (0xFFFFL & value));
+    public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+        throws IOException {
+      return new ShortValues(input, numDocs);
     }
 
   };
 
   final static class IntValues extends IndexDocValuesArray {
-    private int[] values;
+    private final int[] values;
 
-    IntValues(Counter bytesUsed) {
-      super(bytesUsed, RamUsageEstimator.NUM_BYTES_INT, ValueType.FIXED_INTS_32);
+    IntValues() {
+      super(RamUsageEstimator.NUM_BYTES_INT, ValueType.FIXED_INTS_32);
       values = new int[0];
     }
 
-    IntValues(IndexInput input, int numDocs) throws IOException {
-      super(Counter.newCounter(), RamUsageEstimator.NUM_BYTES_INT,
-          ValueType.FIXED_INTS_32);
+    private IntValues(IndexInput input, int numDocs) throws IOException {
+      super(RamUsageEstimator.NUM_BYTES_INT, ValueType.FIXED_INTS_32);
       values = new int[numDocs];
-      adjustSize(numDocs);
       for (int i = 0; i < values.length; i++) {
         values[i] = input.readInt();
       }
@@ -302,205 +235,227 @@ abstract class IndexDocValuesArray exten
     }
 
     @Override
-    protected void setInternal(int docId, long value) {
-      values[docId] = (int) (0xFFFFFFFF & value);
+    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input,
+        int maxDoc) throws IOException {
+      return new FixedIntsEnum(attrSource, input, type(),
+          bytesPerValue, maxDoc) {
+        @Override
+        protected final long toLong(BytesRef bytesRef) {
+          return bytesRef.asInt();
+        }
+      };
     }
 
     @Override
-    protected int grow(int numDocs) {
-      values = ArrayUtil.grow(values, numDocs);
-      return values.length;
+    public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+        throws IOException {
+      return new IntValues(input, numDocs);
     }
 
-    @Override
-    void write(IndexOutput output, int numDocs) throws IOException {
-      assert maxDocID + 1 <= numDocs;
-      for (int i = 0; i < maxDocID + 1; i++) {
-        output.writeInt(values[i]);
+  };
+
+  final static class LongValues extends IndexDocValuesArray {
+    private final long[] values;
+
+    LongValues() {
+      super(RamUsageEstimator.NUM_BYTES_LONG, ValueType.FIXED_INTS_64);
+      values = new long[0];
+    }
+
+    private LongValues(IndexInput input, int numDocs) throws IOException {
+      super(RamUsageEstimator.NUM_BYTES_LONG, ValueType.FIXED_INTS_64);
+      values = new long[numDocs];
+      for (int i = 0; i < values.length; i++) {
+        values[i] = input.readLong();
       }
-      writeDefaults(output,  numDocs - (maxDocID+1));
+      maxDocID = numDocs - 1;
     }
 
     @Override
-    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input, int maxDoc)
-        throws IOException {
-      return new FixedIntsEnumImpl(attrSource, input, type(), maxDoc) {
-        @Override
-        protected void fillNext(LongsRef ref, IndexInput dataIn)
-            throws IOException {
-          ref.ints[ref.offset] = dataIn.readInt();
-        }
-      };
+    public long[] getArray() {
+      return values;
     }
 
     @Override
-    void clear() {
-      super.clear();
-      values = new int[0];
+    public long getInt(int docID) {
+      assert docID >= 0 && docID < values.length;
+      return values[docID];
     }
 
     @Override
-    protected void writeDefaults(IndexOutput out, int num) throws IOException {
-      for (int i = 0; i < num; i++) {
-        out.writeInt(0);
-      }
+    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input,
+        int maxDoc) throws IOException {
+      return new FixedIntsEnum(attrSource, input, type(),
+          bytesPerValue, maxDoc) {
+        @Override
+        protected final long toLong(BytesRef bytesRef) {
+          return bytesRef.asLong();
+        }
+      };
     }
-    
+
     @Override
-    protected void writeDirect(IndexOutput out, long value) throws IOException {
-      out.writeInt((int) (0xFFFFFFFFL & value));
+    public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+        throws IOException {
+      return new LongValues(input, numDocs);
     }
 
   };
 
-  final static class LongValues extends IndexDocValuesArray {
-    private long[] values;
+  final static class FloatValues extends IndexDocValuesArray {
+    private final float[] values;
 
-    LongValues(Counter bytesUsed) {
-      super(bytesUsed, RamUsageEstimator.NUM_BYTES_LONG,
-          ValueType.FIXED_INTS_64);
-      values = new long[0];
+    FloatValues() {
+      super(RamUsageEstimator.NUM_BYTES_FLOAT, ValueType.FLOAT_32);
+      values = new float[0];
     }
 
-    LongValues(IndexInput input, int numDocs) throws IOException {
-      super(Counter.newCounter(), RamUsageEstimator.NUM_BYTES_LONG,
-          ValueType.FIXED_INTS_64);
-      values = new long[numDocs];
-      adjustSize(numDocs);
+    private FloatValues(IndexInput input, int numDocs) throws IOException {
+      super(RamUsageEstimator.NUM_BYTES_FLOAT, ValueType.FLOAT_32);
+      values = new float[numDocs];
+      /* we always read BIG_ENDIAN here since the writer serialized plain bytes
+       * we can simply read the ints / longs
+       * back in using readInt / readLong */
       for (int i = 0; i < values.length; i++) {
-        values[i] = input.readLong();
+        values[i] = Float.intBitsToFloat(input.readInt());
       }
       maxDocID = numDocs - 1;
     }
 
     @Override
-    public long[] getArray() {
+    public float[] getArray() {
       return values;
     }
 
     @Override
-    public long getInt(int docID) {
+    public double getFloat(int docID) {
       assert docID >= 0 && docID < values.length;
       return values[docID];
     }
 
     @Override
-    protected void setInternal(int docId, long value) {
-      values[docId] = value;
+    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input,
+        int maxDoc) throws IOException {
+      return new FloatsEnum(attrSource, input, type(),
+          bytesPerValue, maxDoc) {
+            @Override
+            protected double toDouble(BytesRef bytesRef) {
+              return Float.intBitsToFloat(bytesRef.asInt());
+            }
+      };
     }
 
     @Override
-    protected int grow(int numDocs) {
-      values = ArrayUtil.grow(values, numDocs);
-      return values.length;
+    public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+        throws IOException {
+      return new FloatValues(input, numDocs);
     }
+  };
+  
+  final static class DoubleValues extends IndexDocValuesArray {
+    private final double[] values;
 
-    @Override
-    void write(IndexOutput output, int numDocs) throws IOException {
-      assert maxDocID + 1 <= numDocs;
-      for (int i = 0; i < maxDocID + 1; i++) {
-        output.writeLong(values[i]);
+    DoubleValues() {
+      super(RamUsageEstimator.NUM_BYTES_DOUBLE, ValueType.FLOAT_64);
+      values = new double[0];
+    }
+
+    private DoubleValues(IndexInput input, int numDocs) throws IOException {
+      super(RamUsageEstimator.NUM_BYTES_DOUBLE, ValueType.FLOAT_64);
+      values = new double[numDocs];
+      /* we always read BIG_ENDIAN here since the writer serialized plain bytes
+       * we can simply read the ints / longs
+       * back in using readInt / readLong */
+      for (int i = 0; i < values.length; i++) {
+        values[i] = Double.longBitsToDouble(input.readLong());
       }
-      writeDefaults(output, numDocs - (maxDocID+1));
-     
+      maxDocID = numDocs - 1;
     }
 
     @Override
-    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input, int maxDoc)
-        throws IOException {
-      return new FixedIntsEnumImpl(attrSource, input, type(), maxDoc) {
-        @Override
-        protected void fillNext(LongsRef ref, IndexInput dataIn)
-            throws IOException {
-          ref.ints[ref.offset] = dataIn.readLong();
-        }
-      };
+    public double[] getArray() {
+      return values;
     }
 
     @Override
-    void clear() {
-      super.clear();
-      values = new long[0];
+    public double getFloat(int docID) {
+      assert docID >= 0 && docID < values.length;
+      return values[docID];
     }
 
     @Override
-    protected void writeDefaults(IndexOutput out, int num) throws IOException {
-      for (int i = 0; i < num; i++) {
-        out.writeLong(0l);
-      }
+    ValuesEnum getDirectEnum(AttributeSource attrSource, IndexInput input,
+        int maxDoc) throws IOException {
+      return new FloatsEnum(attrSource, input, type(),
+          bytesPerValue, maxDoc) {
+            @Override
+            protected double toDouble(BytesRef bytesRef) {
+              return Double.longBitsToDouble(bytesRef.asLong());
+            }
+      };
     }
+
     @Override
-    protected void writeDirect(IndexOutput out, long value) throws IOException {
-      out.writeLong(value);
+    public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
+        throws IOException {
+      return new DoubleValues(input, numDocs);
     }
   };
 
-  private abstract static class FixedIntsEnumImpl extends ValuesEnum {
-    private final IndexInput dataIn;
-    private final int maxDoc;
-    private final int sizeInByte;
-    private int pos = -1;
-
-    private FixedIntsEnumImpl(AttributeSource source, IndexInput dataIn,
-        ValueType type, int maxDoc) throws IOException {
-      super(source, type);
-      switch (type) {
-      case FIXED_INTS_16:
-        sizeInByte = 2;
-        break;
-      case FIXED_INTS_32:
-        sizeInByte = 4;
-        break;
-      case FIXED_INTS_64:
-        sizeInByte = 8;
-        break;
-      case FIXED_INTS_8:
-        sizeInByte = 1;
-        break;
-      default:
-        throw new IllegalStateException("type " + type
-            + " is not a fixed int type");
-      }
-      intsRef.offset = 0;
-      this.dataIn = dataIn;
-      this.maxDoc = maxDoc;
+  private abstract static class FixedIntsEnum extends
+      FixedStraightBytesEnum {
+    private final ValueType type;
+
+    private FixedIntsEnum(AttributeSource source, IndexInput dataIn,
+        ValueType type, int bytesPerValue, int maxDoc) throws IOException {
+      super(source, dataIn, bytesPerValue, maxDoc);
+      this.type = type;
 
     }
 
     @Override
-    public void close() throws IOException {
-      dataIn.close();
+    public int advance(int target) throws IOException {
+      final int advance = super.advance(target);
+      if (advance != NO_MORE_DOCS) {
+        intsRef.ints[0] = toLong(this.bytesRef);
+      }
+      return advance;
     }
+    
+    protected abstract long toLong(BytesRef bytesRef);
 
     @Override
-    public int advance(int target) throws IOException {
-      if (target >= maxDoc) {
-        return pos = NO_MORE_DOCS;
-      }
-      assert target > pos;
-      if (target > pos + 1) {
-        dataIn
-            .seek(dataIn.getFilePointer() + ((target - pos - 1) * sizeInByte));
-      }
-      fillNext(intsRef, dataIn);
-      return pos = target;
+    public ValueType type() {
+      return type;
     }
 
-    protected abstract void fillNext(LongsRef ref, IndexInput input)
-        throws IOException;
+  }
+  
+  private abstract static class FloatsEnum extends FixedStraightBytesEnum {
 
+    private final ValueType type;
+    FloatsEnum(AttributeSource source, IndexInput dataIn, ValueType type, int bytePerValue, int maxDoc)
+        throws IOException {
+      super(source, dataIn, bytePerValue, maxDoc);
+      this.type = type;
+    }
+    
     @Override
-    public int docID() {
-      return pos;
+    public int advance(int target) throws IOException {
+      final int retVal = super.advance(target);
+      if (retVal != NO_MORE_DOCS) {
+        floatsRef.floats[floatsRef.offset] = toDouble(bytesRef);
+      }
+      return retVal;
     }
+    
+    protected abstract double toDouble(BytesRef bytesRef);
 
     @Override
-    public int nextDoc() throws IOException {
-      if (pos >= maxDoc) {
-        return pos = NO_MORE_DOCS;
-      }
-      return advance(pos + 1);
+    public ValueType type() {
+      return type;
     }
+
   }
 
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Ints.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Ints.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Ints.java Sat Oct  1 03:04:53 2011
@@ -19,28 +19,169 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.values.IntsImpl.IntsReader;
-import org.apache.lucene.index.values.IntsImpl.IntsWriter;
+import org.apache.lucene.index.values.IndexDocValuesArray.ByteValues;
+import org.apache.lucene.index.values.IndexDocValuesArray.IntValues;
+import org.apache.lucene.index.values.IndexDocValuesArray.LongValues;
+import org.apache.lucene.index.values.IndexDocValuesArray.ShortValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IOUtils;
 
 /**
+ * Stores ints packed and fixed with fixed-bit precision.
+ * 
  * @lucene.experimental
  */
-public class Ints {
-  // TODO - add bulk copy where possible
+public final class Ints {
 
   private Ints() {
   }
 
-  public static Writer getWriter(Directory dir, String id,
-      Counter bytesUsed, ValueType type, IOContext context) throws IOException {
-    return new IntsWriter(dir, id, bytesUsed, type, context);
+  public static Writer getWriter(Directory dir, String id, Counter bytesUsed,
+      ValueType type, IOContext context) throws IOException {
+    return type == ValueType.VAR_INTS ? new PackedIntValues.PackedIntsWriter(dir, id,
+        bytesUsed, context) : new IntsWriter(dir, id, bytesUsed, context, type);
   }
 
-  public static IndexDocValues getValues(Directory dir, String id,
-      int numDocs, IOContext context) throws IOException {
-    return new IntsReader(dir, id, numDocs, context);
+  public static IndexDocValues getValues(Directory dir, String id, int numDocs,
+      ValueType type, IOContext context) throws IOException {
+    return type == ValueType.VAR_INTS ? new PackedIntValues.PackedIntsReader(dir, id,
+        numDocs, context) : new IntsReader(dir, id, numDocs, context);
+  }
+
+  static class IntsWriter extends FixedStraightBytesImpl.Writer {
+    protected static final String CODEC_NAME = "Ints";
+    protected static final int VERSION_START = 0;
+    protected static final int VERSION_CURRENT = VERSION_START;
+
+    private final ValueType valueType;
+
+    public IntsWriter(Directory dir, String id, Counter bytesUsed,
+        IOContext context, ValueType valueType) throws IOException {
+      this(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, valueType);
+    }
+
+    protected IntsWriter(Directory dir, String id, String codecName,
+        int version, Counter bytesUsed, IOContext context, ValueType valueType) throws IOException {
+      super(dir, id, codecName, version, bytesUsed, context);
+      this.valueType = valueType;
+      final int expectedSize = getSize(valueType);
+      this.bytesRef = new BytesRef(expectedSize);
+      bytesRef.length = expectedSize;
+    }
+    
+    private static int getSize(ValueType type) {
+      switch (type) {
+      case FIXED_INTS_16:
+        return 2;
+      case FIXED_INTS_32:
+        return 4;
+      case FIXED_INTS_64:
+        return 8;
+      case FIXED_INTS_8:
+        return 1;
+      default:
+        throw new IllegalStateException("illegal type " + type);
+      }
+    }
+
+    @Override
+    public void add(int docID, long v) throws IOException {
+      switch (valueType) {
+      case FIXED_INTS_64:
+        bytesRef.copy(v);
+        break;
+      case FIXED_INTS_32:
+        bytesRef.copy((int) (0xFFFFFFFF & v));
+        break;
+      case FIXED_INTS_16:
+        bytesRef.copy((short) (0xFFFFL & v));
+        break;
+      case FIXED_INTS_8:
+        bytesRef.bytes[0] = (byte) (0xFFL & v);
+        break;
+      default:
+        throw new IllegalStateException("illegal type " + valueType);
+      }
+
+      add(docID, bytesRef);
+    }
+
+    @Override
+    public void add(int docID, PerDocFieldValues docValues) throws IOException {
+      add(docID, docValues.getInt());
+    }
+  }
+
+  final static class IntsReader extends FixedStraightBytesImpl.Reader {
+    private final ValueType type;
+    private final IndexDocValuesArray arrayTemplate;
+
+    IntsReader(Directory dir, String id, int maxDoc, IOContext context)
+        throws IOException {
+      super(dir, id, IntsWriter.CODEC_NAME, IntsWriter.VERSION_CURRENT, maxDoc,
+          context);
+      switch (size) {
+      case 8:
+        type = ValueType.FIXED_INTS_64;
+        arrayTemplate = new LongValues();
+        break;
+      case 4:
+        type = ValueType.FIXED_INTS_32;
+        arrayTemplate = new IntValues();
+        break;
+      case 2:
+        type = ValueType.FIXED_INTS_16;
+        arrayTemplate = new ShortValues();
+        break;
+      case 1:
+        type = ValueType.FIXED_INTS_8;
+        arrayTemplate = new ByteValues();
+        break;
+      default:
+        throw new IllegalStateException("illegal size: " + size);
+      }
+    }
+
+    @Override
+    public Source load() throws IOException {
+      boolean success = false;
+      IndexInput input = null;
+      try {
+        input = cloneData();
+        final Source source = arrayTemplate.newFromInput(input, maxDoc);
+        success = true;
+        return source;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(input, datIn);
+        }
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      final IndexInput input = cloneData();
+      boolean success = false;
+      try {
+        final ValuesEnum valuesEnum = arrayTemplate.getDirectEnum(source,
+            input, maxDoc);
+        success = true;
+        return valuesEnum;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(input);
+        }
+      }
+    }
+
+    @Override
+    public ValueType type() {
+      return type;
+    }
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java Sat Oct  1 03:04:53 2011
@@ -44,9 +44,9 @@ import org.apache.lucene.util.LongsRef;
 public abstract class ValuesEnum extends DocIdSetIterator {
   private AttributeSource source;
   private final ValueType enumType;
-  protected BytesRef bytesRef;
-  protected FloatsRef floatsRef;
-  protected LongsRef intsRef;
+  protected BytesRef bytesRef = new BytesRef(1);
+  protected FloatsRef floatsRef = new FloatsRef(1);
+  protected LongsRef intsRef = new LongsRef(1);
 
   /**
    * Creates a new {@link ValuesEnum} for the given type. The
@@ -62,28 +62,6 @@ public abstract class ValuesEnum extends
   protected ValuesEnum(AttributeSource source, ValueType enumType) {
     this.source = source;
     this.enumType = enumType;
-    switch (enumType) {
-    case BYTES_FIXED_DEREF:
-    case BYTES_FIXED_SORTED:
-    case BYTES_FIXED_STRAIGHT:
-    case BYTES_VAR_DEREF:
-    case BYTES_VAR_SORTED:
-    case BYTES_VAR_STRAIGHT:
-      bytesRef = new BytesRef();
-      break;
-    case FIXED_INTS_16:
-    case FIXED_INTS_32:
-    case FIXED_INTS_64:
-    case FIXED_INTS_8:
-    case VAR_INTS:
-      intsRef = new LongsRef(1);
-      break;
-    case FLOAT_32:
-    case FLOAT_64:
-      floatsRef = new FloatsRef(1);
-      break;
-    
-    }
   }
 
   /**

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Sat Oct  1 03:04:53 2011
@@ -19,29 +19,17 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
-import org.apache.lucene.index.values.Bytes.BytesWriterBase;
-import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
-import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.index.values.Bytes.DerefBytesSourceBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesEnumBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
-import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
-import org.apache.lucene.util.packed.PackedInts;
 
 // Stores variable-length byte[] by deref, ie when two docs
 // have the same value, they store only 1 byte[] and both
@@ -56,51 +44,6 @@ class VarDerefBytesImpl {
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 
-  private static final class AddressByteStartArray extends
-      TrackingDirectBytesStartArray {
-    int[] address;
-
-    AddressByteStartArray(int size, Counter bytesUsed) {
-      super(size, bytesUsed);
-    }
-
-    @Override
-    public Counter bytesUsed() {
-      return bytesUsed;
-    }
-
-    @Override
-    public int[] clear() {
-      if (address != null) {
-        bytesUsed.addAndGet(-address.length * RamUsageEstimator.NUM_BYTES_INT);
-        address = null;
-      }
-      return super.clear();
-    }
-
-    @Override
-    public int[] grow() {
-      assert address != null;
-      final int oldSize = address.length;
-      final int[] retVal = super.grow();
-      address = ArrayUtil.grow(address, retVal.length);
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-          * (address.length - oldSize));
-      return retVal;
-    }
-
-    @Override
-    public int[] init() {
-      if (address == null) {
-        address = new int[ArrayUtil.oversize(initSize,
-            RamUsageEstimator.NUM_BYTES_INT)];
-        bytesUsed.addAndGet((address.length) * RamUsageEstimator.NUM_BYTES_INT);
-      }
-      return super.init();
-    }
-
-  }
-
   /*
    * TODO: if impls like this are merged we are bound to the amount of memory we
    * can store into a BytesRefHash and therefore how much memory a ByteBlockPool
@@ -110,170 +53,66 @@ class VarDerefBytesImpl {
    * move the byte[] writing to #finish(int) and store the bytes in sorted
    * order and merge them in a streamed fashion. 
    */
-  static class Writer extends BytesWriterBase {
-    private int[] docToAddress;
-    private int address = 1;
-
-    private final AddressByteStartArray array = new AddressByteStartArray(1,
-        bytesUsed);
-    private final BytesRefHash hash;
-
+  static class Writer extends DerefBytesWriterBase {
     public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
-      this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
-          bytesUsed, context);
-    }
-
-    public Writer(Directory dir, String id, Allocator allocator,
-        Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
-      hash = new BytesRefHash(new ByteBlockPool(allocator), 16, array);
-      docToAddress = new int[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
     }
-
+    
     @Override
-    public void add(int docID, BytesRef bytes) throws IOException {
-      if (bytes.length == 0)
-        return; // default
-      final int e = hash.add(bytes);
-
-      if (docID >= docToAddress.length) {
-        final int oldSize = docToAddress.length;
-        docToAddress = ArrayUtil.grow(docToAddress, 1 + docID);
-        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-            * (docToAddress.length - oldSize));
-      }
-      final int docAddress;
-      if (e >= 0) {
-        docAddress = array.address[e] = address;
-        address += bytes.length < 128 ? 1 : 2;
-        address += bytes.length;
-      } else {
-        docAddress = array.address[(-e) - 1];
-      }
-      docToAddress[docID] = docAddress;
-    }
-
-    private static int writePrefixLength(DataOutput datOut, BytesRef bytes)
-        throws IOException {
-      if (bytes.length < 128) {
-        datOut.writeByte((byte) bytes.length);
-        return 1;
-      } else {
-        datOut.writeByte((byte) (0x80 | (bytes.length >> 8)));
-        datOut.writeByte((byte) (bytes.length & 0xff));
-        return 2;
-      }
+    protected void checkSize(BytesRef bytes) {
+      // allow var bytes sizes
     }
 
     // Important that we get docCount, in case there were
     // some last docs that we didn't see
     @Override
-    public void finish(int docCount) throws IOException {
-      final IndexOutput datOut = getDataOut();
-      boolean success = false;
-      try {
-        final int size = hash.size();
-        final BytesRef bytesRef = new BytesRef();
-        for (int i = 0; i < size; i++) {
-          hash.get(i, bytesRef);
-          writePrefixLength(datOut, bytesRef);
-          datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
-        }
-        success = true;
-      } finally {
-        hash.close();
-        if (success) {
-          IOUtils.close(datOut);
-        } else {
-          IOUtils.closeWhileHandlingException(datOut);
-        }
-      }
-      
-      final IndexOutput idxOut = getIndexOut();
-      success = false;
-      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();
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(idxOut);
-        } else {
-          IOUtils.closeWhileHandlingException(idxOut);
-        }
-        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
-            * (-docToAddress.length));
-        docToAddress = null;
-      }
+    public void finishInternal(int docCount) throws IOException {
+      final int size = hash.size();
+      final long[] addresses = new long[size+1];
+      final IndexOutput datOut = getOrCreateDataOut();
+      int addr = 1;
+      final BytesRef bytesRef = new BytesRef();
+      for (int i = 0; i < size; i++) {
+        hash.get(i, bytesRef);
+        addresses[i+1] = addr;
+        addr += writePrefixLength(datOut, bytesRef) + bytesRef.length;
+        datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
+      }
+
+      final IndexOutput idxOut = getOrCreateIndexOut();
+      // write the max address to read directly on source load
+      idxOut.writeLong(addr - 1);
+      writeIndex(idxOut, docCount, addresses[size], addresses, docToEntry);
     }
   }
 
   public static class Reader extends BytesReaderBase {
-
+    private final long totalBytes;
     Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_START, true, context);
+      totalBytes = idxIn.readLong();
     }
 
     @Override
     public Source load() throws IOException {
-      final IndexInput data = cloneData();
-      final IndexInput index = cloneIndex();
-      data.seek(CodecUtil.headerLength(CODEC_NAME));
-      index.seek(CodecUtil.headerLength(CODEC_NAME));
-      final long totalBytes = index.readInt(); // should be long
-      return new Source(data, index, totalBytes);
+      return new Source(cloneData(), cloneIndex(), totalBytes);
     }
 
-    private static class Source extends BytesBaseSource {
-      private final PackedInts.Reader index;
+    private final static class Source extends DerefBytesSourceBase {
 
       public Source(IndexInput datIn, IndexInput idxIn, long totalBytes)
           throws IOException {
-        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), totalBytes);
-        index = PackedInts.getReader(idxIn);
+        super(datIn, idxIn, totalBytes, ValueType.BYTES_VAR_DEREF);
       }
 
       @Override
       public BytesRef getBytes(int docID, BytesRef bytesRef) {
-        long address = index.get(docID);
+        long address = addresses.get(docID);
         bytesRef.length = 0;
         return address == 0 ? bytesRef : data.fillSliceWithPrefix(bytesRef,
             --address);
       }
-
-      @Override
-      public int getValueCount() {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public ValueType type() {
-        return ValueType.BYTES_VAR_DEREF;
-      }
-
-      @Override
-      protected int maxDoc() {
-        return index.size();
-      }
     }
 
     @Override
@@ -281,8 +120,8 @@ class VarDerefBytesImpl {
       return new VarDerefBytesEnum(source, cloneData(), cloneIndex());
     }
 
-    static class VarDerefBytesEnum extends DerefBytesEnum {
-
+    final static class VarDerefBytesEnum extends DerefBytesEnumBase {
+      
       public VarDerefBytesEnum(AttributeSource source, IndexInput datIn,
           IndexInput idxIn) throws IOException {
         super(source, datIn, idxIn, -1, ValueType.BYTES_VAR_DEREF);
@@ -299,8 +138,9 @@ class VarDerefBytesImpl {
         } else {
           size = ((sizeByte & 0x7f) << 8) | ((datIn.readByte() & 0xff));
         }
-        if (ref.bytes.length < size)
+        if (ref.bytes.length < size) {
           ref.grow(size);
+        }
         ref.length = size;
         ref.offset = 0;
         datIn.readBytes(ref.bytes, 0, size);

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Sat Oct  1 03:04:53 2011
@@ -18,28 +18,18 @@ package org.apache.lucene.index.values;
  */
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Comparator;
 
-import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
+import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
-import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
 import org.apache.lucene.util.packed.PackedInts;
 
 // Stores variable-length byte[] by deref, ie when two docs
@@ -55,130 +45,66 @@ class VarSortedBytesImpl {
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 
-  static class Writer extends BytesWriterBase {
-    private int[] docToEntry;
+  final static class Writer extends DerefBytesWriterBase {
     private final Comparator<BytesRef> comp;
 
-    private final BytesRefHash hash; 
-
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Counter bytesUsed, IOContext context) throws IOException {
-      this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
-          bytesUsed, context);
-    }
-
-    public Writer(Directory dir, String id, Comparator<BytesRef> comp,
-        Allocator allocator, Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
-      this.hash = new BytesRefHash(new ByteBlockPool(allocator),
-          BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
-              BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
       this.comp = comp;
-      docToEntry = new int[1];
-      docToEntry[0] = -1;
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
     }
-
+    
     @Override
-    public void add(int docID, BytesRef bytes) throws IOException {
-      if (bytes.length == 0)
-        return;// default
-      if (docID >= docToEntry.length) {
-        int[] newArray = new int[ArrayUtil.oversize(1 + docID,
-            RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-        System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
-        Arrays.fill(newArray, docToEntry.length, newArray.length, -1);
-        bytesUsed.addAndGet((newArray.length - docToEntry.length)
-            * RamUsageEstimator.NUM_BYTES_INT);
-        docToEntry = newArray;
-      }
-      final int e = hash.add(bytes);
-      docToEntry[docID] = e < 0 ? (-e) - 1 : e;
+    protected void checkSize(BytesRef bytes) {
+      // allow var bytes sizes
     }
 
     // Important that we get docCount, in case there were
     // some last docs that we didn't see
     @Override
-    public void finish(int docCount) throws IOException {
+    public void finishInternal(int docCount) throws IOException {
       final int count = hash.size();
-      final IndexOutput datOut = getDataOut();
+      final IndexOutput datOut = getOrCreateDataOut();
       long offset = 0;
       long lastOffset = 0;
-      final int[] index = new int[count];
+      final int[] index = new int[count+1];
       final long[] offsets = new long[count];
-      boolean success = false;
-      try {
-        final int[] sortedEntries = hash.sort(comp);
-        // first dump bytes data, recording index & offset as
-        // we go
-        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;
-        }
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(datOut);
-        } else {
-          IOUtils.closeWhileHandlingException(datOut);
-        }
-        hash.close();
-      }
-      final IndexOutput idxOut = getIndexOut();
-      success = false;
-      try {
-        // 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();
-        success = true;
-      } finally {
-        bytesUsed.addAndGet((-docToEntry.length)
-            * RamUsageEstimator.NUM_BYTES_INT);
-        docToEntry = null;
-        if (success) {
-          IOUtils.close(idxOut);
-        } else {
-          IOUtils.closeWhileHandlingException(idxOut);
-        }
+      final int[] sortedEntries = hash.sort(comp);
+      // first dump bytes data, recording index & offset as
+      // we go
+      for (int i = 0; i < count; i++) {
+        final int e = sortedEntries[i];
+        offsets[i] = offset;
+        index[e+1] = 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;
+      }
+
+      final IndexOutput idxOut = getOrCreateIndexOut();
+      // total bytes of data
+      idxOut.writeLong(offset);
+      // write index -- first doc -> 1+ord
+      writeIndex(idxOut, docCount, count, index, docToEntry);
+      // next ord (0-based) -> offset
+      PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count,
+          PackedInts.bitsRequired(lastOffset));
+      for (int i = 0; i < count; i++) {
+        offsetWriter.add(offsets[i]);
       }
+      offsetWriter.finish();
     }
   }
 
   public static class Reader extends BytesReaderBase {
 
     private final Comparator<BytesRef> defaultComp;
-        
-    Reader(Directory dir, String id, int maxDoc, Comparator<BytesRef> comparator, IOContext context) throws IOException {
+
+    Reader(Directory dir, String id, int maxDoc,
+        Comparator<BytesRef> comparator, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_START, true, context);
       this.defaultComp = comparator;
     }
@@ -196,32 +122,25 @@ class VarSortedBytesImpl {
       return new Source(cloneData(), indexIn, comp, indexIn.readLong());
     }
 
-    private static class Source extends BytesBaseSortedSource {
-      private final PackedInts.Reader docToOrdIndex;
+    private static class Source extends BytesSortedSourceBase {
       private final PackedInts.Reader ordToOffsetIndex; // 0-based
       private final long totBytes;
       private final int valueCount;
 
       public Source(IndexInput datIn, IndexInput idxIn,
           Comparator<BytesRef> comp, long dataLength) throws IOException {
-        super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), dataLength);
+        super(datIn, idxIn, comp, dataLength, ValueType.BYTES_VAR_SORTED);
         totBytes = dataLength;
-        docToOrdIndex = PackedInts.getReader(idxIn);
         ordToOffsetIndex = PackedInts.getReader(idxIn);
         valueCount = ordToOffsetIndex.size();
         closeIndexInput();
       }
 
       @Override
-      public int ord(int docID) {
-        return (int) docToOrdIndex.get(docID) - 1;
-      }
-
-      @Override
       public int getByValue(BytesRef bytes, BytesRef tmpRef) {
         return binarySearch(bytes, tmpRef, 0, valueCount - 1);
       }
-
+      
       @Override
       public int getValueCount() {
         return valueCount;
@@ -240,16 +159,6 @@ class VarSortedBytesImpl {
         data.fillSlice(bytesRef, offset, (int) (nextOffset - offset));
         return bytesRef;
       }
-
-      @Override
-      public ValueType type() {
-        return ValueType.BYTES_VAR_SORTED;
-      }
-
-      @Override
-      protected int maxDoc() {
-        return docToOrdIndex.size();
-      }
     }
 
     @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Sat Oct  1 03:04:53 2011
@@ -19,9 +19,9 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
 import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesSourceBase;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -32,7 +32,6 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.packed.PackedInts;
@@ -95,7 +94,7 @@ class VarStraightBytesImpl {
     @Override
     protected void merge(MergeState state) throws IOException {
       merge = true;
-      datOut = getDataOut();
+      datOut = getOrCreateDataOut();
       boolean success = false;
       try {
         if (state.liveDocs == null && state.reader instanceof Reader) {
@@ -166,7 +165,7 @@ class VarStraightBytesImpl {
     public void finish(int docCount) throws IOException {
       boolean success = false;
       assert (!merge && datOut == null) || (merge && datOut != null); 
-      final IndexOutput datOut = getDataOut();
+      final IndexOutput datOut = getOrCreateDataOut();
       try {
         if (!merge) {
           // header is already written in getDataOut()
@@ -183,7 +182,7 @@ class VarStraightBytesImpl {
       }
 
       success = false;
-      final IndexOutput idxOut = getIndexOut();
+      final IndexOutput idxOut = getOrCreateIndexOut();
       try {
         if (lastDocID == -1) {
           idxOut.writeVLong(0);
@@ -234,12 +233,10 @@ class VarStraightBytesImpl {
       return new Source(cloneData(), cloneIndex());
     }
 
-    private class Source extends BytesBaseSource {
-      private final PackedInts.Reader addresses;
+    private class Source extends DerefBytesSourceBase {
 
       public Source(IndexInput datIn, IndexInput idxIn) throws IOException {
-        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong());
-        addresses = PackedInts.getReader(idxIn);
+        super(datIn, idxIn, idxIn.readVLong(), ValueType.BYTES_VAR_STRAIGHT);
       }
 
       @Override
@@ -263,21 +260,6 @@ class VarStraightBytesImpl {
           }
         };
       }
-
-      @Override
-      public int getValueCount() {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public ValueType type() {
-        return ValueType.BYTES_VAR_STRAIGHT;
-      }
-
-      @Override
-      protected int maxDoc() {
-        return addresses.size();
-      }
     }
 
     @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Sat Oct  1 03:04:53 2011
@@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.ConjunctionTermScorer.DocsAndFreqs;
@@ -238,7 +239,7 @@ public class BooleanQuery extends Query 
       for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
         Weight w = wIter.next();
         BooleanClause c = cIter.next();
-        if (w.scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true)) == null) {
+        if (w.scorer(context, true, true, context.reader.getLiveDocs()) == null) {
           if (c.isRequired()) {
             fail = true;
             Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@@ -300,11 +301,12 @@ public class BooleanQuery extends Query 
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext)
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs)
         throws IOException {
       if (termConjunction) {
         // specialized scorer for term conjunctions
-        return createConjunctionTermScorer(context);
+        return createConjunctionTermScorer(context, acceptDocs);
       }
       List<Scorer> required = new ArrayList<Scorer>();
       List<Scorer> prohibited = new ArrayList<Scorer>();
@@ -312,7 +314,7 @@ public class BooleanQuery extends Query 
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
         BooleanClause c =  cIter.next();
-        Scorer subScorer = w.scorer(context, ScorerContext.def());
+        Scorer subScorer = w.scorer(context, true, false, acceptDocs);
         if (subScorer == null) {
           if (c.isRequired()) {
             return null;
@@ -327,7 +329,7 @@ public class BooleanQuery extends Query 
       }
       
       // Check if we can return a BooleanScorer
-      if (!scorerContext.scoreDocsInOrder && scorerContext.topScorer && required.size() == 0 && prohibited.size() < 32) {
+      if (!scoreDocsInOrder && topScorer && required.size() == 0 && prohibited.size() < 32) {
         return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
       }
       
@@ -345,7 +347,7 @@ public class BooleanQuery extends Query 
       return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord);
     }
 
-    private Scorer createConjunctionTermScorer(AtomicReaderContext context)
+    private Scorer createConjunctionTermScorer(AtomicReaderContext context, Bits acceptDocs)
         throws IOException {
       final DocsAndFreqs[] docsAndFreqs = new DocsAndFreqs[weights.size()];
       for (int i = 0; i < docsAndFreqs.length; i++) {
@@ -356,7 +358,7 @@ public class BooleanQuery extends Query 
         }
         final ExactDocScorer docScorer = weight.createDocScorer(context);
         docsAndFreqs[i] = new DocsAndFreqs(termsEnum.docs(
-            context.reader.getLiveDocs(), null), termsEnum.docFreq(), docScorer);
+            acceptDocs, null), termsEnum.docFreq(), docScorer);
       }
       return new ConjunctionTermScorer(this, disableCoord ? 1.0f : coord(
           docsAndFreqs.length, docsAndFreqs.length), docsAndFreqs);

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Sat Oct  1 03:04:53 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
@@ -126,7 +127,8 @@ public class ConstantScoreQuery extends 
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context,  ScorerContext scorerContext) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
@@ -137,7 +139,7 @@ public class ConstantScoreQuery extends 
         disi = dis.iterator();
       } else {
         assert query != null && innerWeight != null;
-        disi = innerWeight.scorer(context, scorerContext);
+        disi = innerWeight.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
       }
 
       if (disi == null) {
@@ -153,7 +155,7 @@ public class ConstantScoreQuery extends 
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      final Scorer cs = scorer(context, ScorerContext.def());
+      final Scorer cs = scorer(context, true, false, context.reader.getLiveDocs());
       final boolean exists = (cs != null && cs.advance(doc) == doc);
 
       final ComplexExplanation result = new ComplexExplanation();

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Sat Oct  1 03:04:53 2011
@@ -25,6 +25,7 @@ import java.util.Set;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
 
 /**
  * A query that generates the union of documents produced by its subqueries, and that scores each document with the maximum
@@ -135,11 +136,13 @@ public class DisjunctionMaxQuery extends
 
     /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
       Scorer[] scorers = new Scorer[weights.size()];
       int idx = 0;
       for (Weight w : weights) {
-        Scorer subScorer = w.scorer(context, ScorerContext.def());
+        // we will advance() subscorers
+        Scorer subScorer = w.scorer(context, true, false, acceptDocs);
         if (subScorer != null && subScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
           scorers[idx++] = subScorer;
         }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/FilteredQuery.java Sat Oct  1 03:04:53 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
@@ -105,9 +106,11 @@ extends Query {
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(AtomicReaderContext context, ScorerContext scoreContext)
+      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+          boolean topScorer, Bits acceptDocs)
           throws IOException {
-        final Scorer scorer = weight.scorer(context, ScorerContext.def());
+        // we will advance() the subscorer
+        final Scorer scorer = weight.scorer(context, true, false, acceptDocs);
         if (scorer == null) {
           return null;
         }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Sat Oct  1 03:04:53 2011
@@ -37,7 +37,6 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Weight.ScorerContext;
 import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
@@ -569,13 +568,11 @@ public class IndexSearcher implements Cl
 
     // TODO: should we make this
     // threaded...?  the Collector could be sync'd?
-    ScorerContext scorerContext =  ScorerContext.def().scoreDocsInOrder(true).topScorer(true);
     // always use single thread:
     if (filter == null) {
       for (int i = 0; i < leaves.length; i++) { // search each subreader
         collector.setNextReader(leaves[i]);
-        scorerContext = scorerContext.scoreDocsInOrder(!collector.acceptsDocsOutOfOrder());
-        Scorer scorer = weight.scorer(leaves[i], scorerContext);
+        Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader.getLiveDocs());
         if (scorer != null) {
           scorer.score(collector);
         }
@@ -593,7 +590,8 @@ public class IndexSearcher implements Cl
 
     assert filter != null;
     
-    Scorer scorer = weight.scorer(context, ScorerContext.def());
+    // we are gonna advance() this scorer, so we set inorder=true/toplevel=false 
+    Scorer scorer = weight.scorer(context, true, false, context.reader.getLiveDocs());
     if (scorer == null) {
       return;
     }
@@ -752,16 +750,17 @@ public class IndexSearcher implements Cl
     public TopDocs call() throws IOException {
       final TopDocs docs = searcher.search (slice.leaves, weight, filter, after, nDocs);
       final ScoreDoc[] scoreDocs = docs.scoreDocs;
-      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
-        final ScoreDoc scoreDoc = scoreDocs[j];
-        //it would be so nice if we had a thread-safe insert 
-        lock.lock();
-        try {
-          if (scoreDoc == hq.insertWithOverflow(scoreDoc))
+      //it would be so nice if we had a thread-safe insert 
+      lock.lock();
+      try {
+        for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
+          final ScoreDoc scoreDoc = scoreDocs[j];
+          if (scoreDoc == hq.insertWithOverflow(scoreDoc)) {
             break;
-        } finally {
-          lock.unlock();
+          }
         }
+      } finally {
+        lock.unlock();
       }
       return docs;
     }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Sat Oct  1 03:04:53 2011
@@ -38,9 +38,9 @@ public class MatchAllDocsQuery extends Q
     private final int maxDoc;
     private final Bits liveDocs;
 
-    MatchAllScorer(IndexReader reader, Weight w, float score) throws IOException {
+    MatchAllScorer(IndexReader reader, Bits liveDocs, Weight w, float score) throws IOException {
       super(w);
-      liveDocs = reader.getLiveDocs();
+      this.liveDocs = liveDocs;
       this.score = score;
       maxDoc = reader.maxDoc();
     }
@@ -104,8 +104,9 @@ public class MatchAllDocsQuery extends Q
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new MatchAllScorer(context.reader, this, queryWeight);
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
+      return new MatchAllScorer(context.reader, acceptDocs, this, queryWeight);
     }
 
     @Override