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/10/12 08:15:05 UTC

svn commit: r1021636 [2/3] - in /lucene/dev/branches/docvalues/lucene: contrib/benchmark/conf/ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ src/java/org/apache...

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,262 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+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.store.Directory;
+import org.apache.lucene.store.IndexInput;
+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.RamUsageEstimator;
+import org.apache.lucene.util.ByteBlockPool.Allocator;
+import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.packed.PackedInts;
+
+// Stores fixed-length byte[] by deref, ie when two docs
+// have the same value, they store only 1 byte[]
+
+class FixedDerefBytesImpl {
+
+  static final String CODEC_NAME = "FixedDerefBytes";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  static class Writer extends BytesWriterBase {
+    private int size = -1;
+    private int[] docToID;
+    private final BytesRefHash hash = new BytesRefHash(pool);
+    
+    public Writer(Directory dir, String id) throws IOException {
+      this(dir, id, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+          new AtomicLong());
+    }
+
+    public Writer(Directory dir, String id, Allocator allocator,
+        AtomicLong bytesUsed) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+          new ByteBlockPool(allocator), bytesUsed);
+      docToID = new int[1];
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
+    }
+
+    @Override
+    synchronized public void add(int docID, BytesRef bytes) throws IOException {
+      if(bytes.length == 0) // default value - skip it
+        return;
+      if (size == -1) {
+        size = bytes.length;
+        initDataOut();
+        datOut.writeInt(size);
+      } else if (bytes.length != size) {
+        throw new IllegalArgumentException("expected bytes size=" + size
+            + " but got " + bytes.length);
+      }
+      int ord = hash.add(bytes);
+
+      if (ord >= 0) {
+        // new added entry
+        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+      } else {
+        ord = (-ord)-1;
+      }
+
+      if (docID >= docToID.length) {
+        int size = docToID.length;
+        docToID = ArrayUtil.grow(docToID, 1 + docID);
+        bytesUsed.addAndGet((docToID.length - size) * RamUsageEstimator.NUM_BYTES_INT);
+      }
+      docToID[docID] = 1+ord;
+    }
+
+
+    // Important that we get docCount, in case there were
+    // some last docs that we didn't see
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if (datOut == null) // no added data
+        return;
+      initIndexOut();
+      final int count = 1+hash.size();
+      idxOut.writeInt(count - 1);
+      // write index
+      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+          PackedInts.bitsRequired(count - 1));
+      final int limit = docCount > docToID.length ? docToID.length : docCount;
+      for (int i = 0; i < limit; i++) {
+        w.add(docToID[i]);
+      }
+      // fill up remaining doc with zeros
+      for (int i = limit; i < docCount; i++) {
+        w.add(0);
+      }
+      w.finish();
+      hash.clear();
+
+      super.finish(docCount);
+    }
+  }
+
+  public static class Reader extends BytesReaderBase {
+    private final int size;
+
+    Reader(Directory dir, String id, int maxDoc) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_START, true);
+      try {
+        size = datIn.readInt();
+      } catch (IOException e) {
+        throw e;
+      }
+    }
+
+    @Override
+    public Source load() throws IOException {
+      return new Source(cloneData(), cloneIndex(), size);
+    }
+
+    private static class Source extends BytesBaseSource {
+      // TODO: paged data or mmap?
+      private final byte[] data;
+      private final BytesRef bytesRef = new BytesRef();
+      private final PackedInts.Reader index;
+      private final int numValue;
+      private final int size;
+
+      protected Source(IndexInput datIn, IndexInput idxIn, int size)
+          throws IOException {
+        super(datIn, idxIn);
+        this.size = size;
+        numValue = idxIn.readInt();
+        data = new byte[size * numValue];
+        datIn.readBytes(data, 0, size * numValue);
+        index = PackedInts.getReader(idxIn);
+        bytesRef.bytes = data;
+        bytesRef.length = size;
+      }
+
+      @Override
+      public BytesRef bytes(int docID) {
+        final int id = (int) index.get(docID);
+        if (id == 0) {
+          return defaultValue;
+        }
+        bytesRef.offset = ((id - 1) * size);
+        return bytesRef;
+      }
+
+      public long ramBytesUsed() {
+        // TODO(simonw): move ram calculation to PackedInts?!
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
+            + data.length
+            + (RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + index
+                .getBitsPerValue()
+                * index.size());
+      }
+
+      @Override
+      public int getValueCount() {
+        return numValue;
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new DerefBytesEnum(source, cloneData(), cloneIndex(), CODEC_NAME,
+          size);
+    }
+
+    static class DerefBytesEnum extends ValuesEnum {
+      protected final IndexInput datIn;
+      private final PackedInts.ReaderIterator idx;
+      protected final long fp;
+      private final int size;
+      protected final BytesRef ref;
+      private final int valueCount;
+      private int pos = -1;
+
+      public DerefBytesEnum(AttributeSource source, IndexInput datIn,
+          IndexInput idxIn, String codecName, int size) throws IOException {
+        this(source, datIn, idxIn, codecName, size, Values.BYTES_FIXED_DEREF);
+      }
+
+      protected DerefBytesEnum(AttributeSource source, IndexInput datIn,
+          IndexInput idxIn, String codecName, int size, Values enumType)
+          throws IOException {
+        super(source, enumType);
+        ref = attr.bytes();
+        this.datIn = datIn;
+        this.size = size == -1 ? 128 : size;
+        idxIn.readInt();// read valueCount
+        idx = PackedInts.getReaderIterator(idxIn);
+        fp = datIn.getFilePointer();
+        ref.grow(this.size);
+        ref.length = this.size;
+        ref.offset = 0;
+        valueCount = idx.size();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        if (target < valueCount) {
+          final long address = idx.advance(target);
+          pos = idx.ord();
+          if(address == 0) {
+            // default is empty
+            ref.length = 0;
+            ref.offset = 0;
+            return pos;
+          }
+          fill(address, ref);
+          return pos;
+        }
+        return pos = NO_MORE_DOCS;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        return advance(pos + 1);
+      }
+
+      public void close() throws IOException {
+        datIn.close();
+        idx.close();
+      }
+
+      protected void fill(long address, BytesRef ref) throws IOException {
+        datIn.seek(fp + ((address - 1) * size));
+        datIn.readBytes(ref.bytes, 0, size);
+        ref.length = size;
+        ref.offset = 0;
+      }
+
+      @Override
+      public int docID() {
+        return pos;
+      }
+
+    }
+  }
+
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,258 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
+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.Directory;
+import org.apache.lucene.store.IndexInput;
+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.RamUsageEstimator;
+import org.apache.lucene.util.ByteBlockPool.Allocator;
+import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.packed.PackedInts;
+
+// Stores fixed-length byte[] by deref, ie when two docs
+// have the same value, they store only 1 byte[]
+
+class FixedSortedBytesImpl {
+
+  static final String CODEC_NAME = "FixedSortedBytes";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  static class Writer extends BytesWriterBase {
+    private int size = -1;
+    private int[] docToEntry;
+    private final Comparator<BytesRef> comp;
+
+   
+    private final BytesRefHash hash = new BytesRefHash(pool);
+    
+    public Writer(Directory dir, String id,  Comparator<BytesRef> comp) throws IOException {
+      this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+          new AtomicLong());
+    }
+
+    public Writer(Directory dir, String id,  Comparator<BytesRef> comp, Allocator allocator, AtomicLong bytesUsed) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, new ByteBlockPool(allocator), bytesUsed);
+      docToEntry = new int[1];
+//      docToEntry[0] = -1;
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
+      this.comp = comp;
+    }
+
+    @Override
+    synchronized public void add(int docID, BytesRef bytes) throws IOException {
+      if(bytes.length == 0)
+        return; // default - skip it
+      if (size == -1) {
+        size = bytes.length;
+        initDataOut();
+        datOut.writeInt(size);
+      } else if (bytes.length != size) {
+        throw new IllegalArgumentException("expected bytes size=" + size + " but got " + bytes.length);
+      }
+      if (docID >= docToEntry.length) {
+        int[] newArray = new int[ArrayUtil.oversize(1 + docID,
+            RamUsageEstimator.NUM_BYTES_INT)];
+        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;
+      }
+      int e = hash.add(bytes);
+      docToEntry[docID] = 1+(e < 0? (-e)-1: e);
+    }
+
+
+    // Important that we get docCount, in case there were
+    // some last docs that we didn't see
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if(datOut == null)// no data added
+        return;
+      initIndexOut();
+      final int[] sortedEntries = hash.sort(comp);
+      final int count = hash.size();
+      int[] address= new int[count];
+      // first dump bytes data, recording address as we go
+      for(int i=0;i<count;i++) {
+        final int e = sortedEntries[i];
+        final BytesRef bytes = hash.get(e);
+        assert bytes.length == size;
+        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+        address[e] = 1+i;
+      }
+
+      idxOut.writeInt(count);
+
+      // next write index
+      PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount, PackedInts.bitsRequired(count));
+      final int limit;
+      if (docCount > docToEntry.length) {
+        limit = docToEntry.length;
+      } else {
+        limit = docCount;
+      }
+      for(int i=0;i<limit;i++) {
+        final int e = docToEntry[i];
+        if (e == 0) {
+          // null is encoded as zero
+          w.add(0);
+        } else {
+          assert e > 0 && e <= count: "index must  0 > && <= " + count + " was: " + e;
+          w.add(address[e-1]);
+        }
+      }
+
+      for(int i=limit;i<docCount;i++) {
+        w.add(0);
+      }
+      w.finish();
+
+      super.finish(docCount);
+      bytesUsed.addAndGet((-docToEntry.length)
+          * RamUsageEstimator.NUM_BYTES_INT);
+      docToEntry = null;
+    }
+  }
+  
+  public static class Reader extends BytesReaderBase {
+    // nocommit -- allow/require byte[] paging here?
+    private final int size;
+
+    public Reader(Directory dir, String id, int maxDoc)
+      throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_START, true);
+      size = datIn.readInt();
+    }
+
+    @Override
+    public org.apache.lucene.index.values.Reader.Source load() throws IOException {
+      return loadSorted(null);
+    }
+
+    @Override
+    public SortedSource loadSorted(Comparator<BytesRef> comp) throws IOException {
+      return new Source(cloneData(), cloneIndex(), size, comp);
+    }
+
+    private static class Source extends BytesBaseSortedSource {
+
+      // TODO: paged data
+      private final byte[] data;
+      private final BytesRef bytesRef = new BytesRef();
+      private final PackedInts.Reader index;
+      private final LookupResult lookupResult = new LookupResult();
+      private final int numValue;
+      private final Comparator<BytesRef> comp;
+      private final int size;
+
+      public Source(IndexInput datIn, IndexInput idxIn, int size, Comparator<BytesRef> comp) throws IOException {
+        super(datIn, idxIn);
+        this.size = size;
+        datIn.seek(CodecUtil.headerLength(CODEC_NAME) + 4);
+        idxIn.seek(CodecUtil.headerLength(CODEC_NAME));
+
+        numValue = idxIn.readInt();
+        data = new byte[size*numValue];
+        datIn.readBytes(data, 0, size*numValue);
+        datIn.close();
+
+        index = PackedInts.getReader(idxIn);
+        idxIn.close(); // do we need to close that here?
+
+        bytesRef.bytes = data;
+        bytesRef.length = size;
+        // default byte sort order 
+        this.comp = comp==null?BytesRef.getUTF8SortedAsUnicodeComparator():comp;
+      }
+
+      @Override
+      public int ord(int docID) {
+        return (int) index.get(docID);
+      }
+
+      @Override
+      public BytesRef getByOrd(int ord) {
+        if (ord == 0) {
+          return defaultValue;
+        } else {
+          bytesRef.offset = ((ord-1) * size);
+          return bytesRef;
+        }
+      }
+
+      @Override
+      public LookupResult getByValue(BytesRef bytes) {
+        return binarySearch(bytes, 0, numValue-1);
+      }
+
+      public long ramBytesUsed() {
+        // TODO(simonw): move ram calcultation to PackedInts?
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + data.length +
+            (RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + index.getBitsPerValue() * index.size());
+      }
+
+      @Override
+      public int getValueCount() {
+        return numValue;
+      }
+
+      private LookupResult binarySearch(BytesRef b, int low, int high) {
+        
+        while (low <= high) {
+          int mid = (low + high) >>> 1;
+          bytesRef.offset = mid * size;
+          int cmp = comp.compare(bytesRef, b);
+          if (cmp < 0) {
+            low = mid + 1;
+          } else if (cmp > 0) {
+            high = mid - 1;
+          } else {
+            lookupResult.ord = mid+1;
+            lookupResult.found = true;
+            return lookupResult;
+          }
+        }
+        lookupResult.ord = low;
+        lookupResult.found = false;
+        return lookupResult;
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+        // do unsorted
+        return new DerefBytesEnum(source, cloneData(), cloneIndex(), CODEC_NAME, size);
+    }
+  }
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,221 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.values.Bytes.BytesBaseSource;
+import org.apache.lucene.index.values.Bytes.BytesReaderBase;
+import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Simplest storage: stores fixed length byte[] per
+// document, with no dedup and no sorting.
+
+class FixedStraightBytesImpl {
+
+  static final String CODEC_NAME = "FixedStraightBytes";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  static class Writer extends BytesWriterBase {
+    private int size = -1;
+    // start at -1 if the first added value is > 0
+    private int lastDocID = -1;
+    private byte[] oneRecord;
+
+    protected Writer(Directory dir, String id) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, null, null);
+    }
+    
+    // nocommit - impl bulk copy here!
+
+    @Override
+    synchronized public void add(int docID, BytesRef bytes) throws IOException {
+      if (size == -1) {
+        size = bytes.length;
+        initDataOut();
+        datOut.writeInt(size);
+        oneRecord = new byte[size];
+      } else if (bytes.length != size) {
+        throw new IllegalArgumentException("expected bytes size=" + size + " but got " + bytes.length);
+      }
+      fill(docID);
+      assert bytes.bytes.length >= bytes.length;
+      datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.lucene.index.values.Writer#merge(org.apache.lucene.index.values.Writer.MergeState)
+     */
+    @Override
+    protected void merge(MergeState state) throws IOException {
+      if(state.bits == null && state.reader instanceof Reader){
+        Reader reader = (Reader) state.reader;
+        final int maxDocs = reader.maxDoc;
+        if(maxDocs == 0)
+          return;
+        if(size == -1) {
+          size = reader.size;
+          initDataOut();
+          datOut.writeInt(size);
+          oneRecord = new byte[size];
+        }
+       fill(state.docBase);
+       // nocommit should we add a transfer to API to each reader?
+       datOut.copyBytes(reader.cloneData(), size * maxDocs);
+       lastDocID += maxDocs-1;
+      } else
+        super.merge(state);
+    }
+
+    // Fills up to but not including this docID
+    private void fill(int docID) throws IOException {
+      assert size >= 0;
+      for(int i=lastDocID+1;i<docID;i++) {
+        datOut.writeBytes(oneRecord, size);
+      }
+      lastDocID = docID;
+    }
+
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if(datOut == null) // no data added
+        return;
+      fill(docCount);
+      super.finish(docCount);
+    }
+
+    public long ramBytesUsed() {
+      return 0;
+    }
+    
+  }
+
+  public static class Reader extends BytesReaderBase {
+    private final int size;
+    private final int maxDoc;
+
+    Reader(Directory dir, String id, int maxDoc)
+      throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_START, false);
+      size = datIn.readInt();
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public Source load() throws IOException {
+      return new Source(cloneData(), cloneIndex(), size, maxDoc);
+    }
+
+    @Override
+    public void close() throws IOException {
+      datIn.close();
+    }
+
+    private static class Source extends BytesBaseSource {
+      // TODO: paged data
+      private final byte[] data;
+      private final BytesRef bytesRef = new BytesRef();
+      private final int size;
+
+      public Source(IndexInput datIn, IndexInput idxIn, int size, int maxDoc) throws IOException {
+        super(datIn, idxIn);
+        this.size = size;
+        final int sizeInBytes = size*maxDoc;
+        data = new byte[sizeInBytes];
+        assert data.length <= datIn.length() : " file size is less than the expected size diff: " + (data.length - datIn.length()) + " size: " + size + " maxDoc " + maxDoc + " pos: " + datIn.getFilePointer();
+        datIn.readBytes(data, 0, sizeInBytes);
+        bytesRef.bytes = data;
+        bytesRef.length = size;
+      }
+
+      @Override
+      public BytesRef bytes(int docID) {
+        bytesRef.offset = docID * size;
+        return bytesRef;
+      }
+
+      public long ramBytesUsed() {
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + data.length;
+      }
+
+      @Override
+      public int getValueCount() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new FixedStraightBytesEnum(source, cloneData(), size, maxDoc);
+    }
+    
+    private static final class FixedStraightBytesEnum extends ValuesEnum {
+      private final IndexInput datIn;
+      private final int size;
+      private final int maxDoc;
+      private int pos = -1;
+      private final long fp;
+      private final BytesRef ref;
+
+      public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn, int size, int maxDoc) throws IOException{
+        super(source, Values.BYTES_FIXED_STRAIGHT);
+        this.datIn = datIn;
+        this.size = size;
+        this.maxDoc = maxDoc;
+        ref = attr.bytes();
+        ref.grow(size);
+        ref.length = size;
+        ref.offset = 0;
+        fp = datIn.getFilePointer();
+      }
+     
+      public void close() throws IOException {
+        datIn.close();
+      }
+  
+      @Override
+      public int advance(int target) throws IOException {
+        if(target >= maxDoc){
+          ref.length = 0;
+          ref.offset = 0;
+          return pos = NO_MORE_DOCS;
+        }
+        if((target-1) != pos) // pos inc == 1
+          datIn.seek(fp + target * size);
+        datIn.readBytes(ref.bytes, 0, size);
+        return pos = target;
+      }
+      
+      @Override
+      public int docID() {
+        return pos;
+      }
+      
+      @Override
+      public int nextDoc() throws IOException {
+        return advance(pos+1);
+      }
+    }
+  }
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,389 @@
+package org.apache.lucene.index.values;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.DoubleBuffer;
+import java.nio.FloatBuffer;
+import java.util.Collection;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.FloatsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Exposes writer/reader for floating point values. You can specify 4 (java
+ * float) or 8 (java double) byte precision.
+ */
+//nocommit - add mmap version
+//nocommti - add bulk copy where possible
+public class Floats {
+  private static final String CODEC_NAME = "SimpleFloats";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+  private static final int INT_ZERO = Float.floatToRawIntBits(0.0f);
+  private static final long LONG_ZERO = Double.doubleToRawLongBits(0.0);
+
+  public static void files(String id, Collection<String> files) {
+    files.add(id + "." + IndexFileNames.CSF_DATA_EXTENSION);
+  }
+
+  public static Writer getWriter(Directory dir, String id, int precisionBytes)
+      throws IOException {
+    if (precisionBytes != 4 && precisionBytes != 8) {
+      throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
+          + precisionBytes);
+    }
+    if (precisionBytes == 4) {
+      return new Float4Writer(dir, id);
+    } else {
+      return new Float8Writer(dir, id);
+    }
+  }
+
+  public static Reader getReader(Directory dir, String id, int maxDoc)
+      throws IOException {
+    return new FloatsReader(dir, id, maxDoc);
+  }
+
+  abstract static class FloatsWriter extends Writer {
+    private final Directory dir;
+    private final String id;
+    private FloatsRef floatsRef;
+    protected int lastDocId = -1;
+    protected IndexOutput datOut;
+    private final byte precision;
+
+    protected FloatsWriter(Directory dir, String id, int precision)
+        throws IOException {
+      this.dir = dir;
+      this.id = id;
+      this.precision = (byte) precision;
+    }
+
+    protected void initDatOut() throws IOException {
+      datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+          IndexFileNames.CSF_DATA_EXTENSION));
+      CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
+      assert datOut.getFilePointer() == CodecUtil.headerLength(CODEC_NAME);
+      datOut.writeByte(precision);
+    }
+
+    public long ramBytesUsed() {
+      return 0;
+    }
+
+    @Override
+    protected void add(int docID) throws IOException {
+      add(docID, floatsRef.get());
+    }
+
+    @Override
+    protected void setNextAttribute(ValuesAttribute attr) {
+      floatsRef = attr.floats();
+    }
+
+    protected abstract int fillDefault(int num) throws IOException;
+
+    @Override
+    protected void merge(MergeState state) throws IOException {
+      if (state.bits == null && state.reader instanceof FloatsReader) {
+        // no deletes - bulk copy
+        // nocommit - should be do bulks with deletes too?
+        final FloatsReader reader = (FloatsReader) state.reader;
+        assert reader.precisionBytes == (int) precision;
+        if (reader.maxDoc == 0)
+          return;
+        if (datOut == null)
+          initDatOut();
+        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);
+    }
+
+  }
+
+  // Writes 4 bytes (float) per value
+  static class Float4Writer extends FloatsWriter {
+
+    protected Float4Writer(Directory dir, String id) throws IOException {
+      super(dir, id, 4);
+    }
+
+    @Override
+    synchronized 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 (datOut == null) {
+        initDatOut();
+      }
+      if (docID - lastDocId > 1) {
+        // fill with default values
+        lastDocId += fillDefault(docID - lastDocId - 1);
+      }
+      assert datOut != null;
+      datOut.writeInt(Float.floatToRawIntBits((float) v));
+      ++lastDocId;
+    }
+
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if (datOut == null)
+        return; // no data added - don't create file!
+      if (docCount > lastDocId + 1)
+        for (int i = lastDocId; i < docCount; i++) {
+          datOut.writeInt(INT_ZERO); // default value
+        }
+      datOut.close();
+    }
+
+    @Override
+    protected int fillDefault(int numValues) throws IOException {
+      for (int i = 0; i < numValues; i++) {
+        datOut.writeInt(INT_ZERO);
+      }
+      return numValues;
+    }
+  }
+
+  // Writes 8 bytes (double) per value
+  static class Float8Writer extends FloatsWriter {
+
+    protected Float8Writer(Directory dir, String id) throws IOException {
+      super(dir, id, 8);
+    }
+
+    @Override
+    synchronized 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 (datOut == null) {
+        initDatOut();
+      }
+      if (docID - lastDocId > 1) {
+        // fill with default values
+        lastDocId += fillDefault(docID - lastDocId - 1);
+      }
+      assert datOut != null;
+      datOut.writeLong(Double.doubleToRawLongBits(v));
+      ++lastDocId;
+    }
+
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if (datOut == null)
+        return; // no data added - don't create file!
+      if (docCount > lastDocId + 1)
+        for (int i = lastDocId; i < docCount; i++) {
+          datOut.writeLong(LONG_ZERO); // default value
+        }
+      datOut.close();
+    }
+
+    @Override
+    protected int fillDefault(int numValues) throws IOException {
+      for (int i = 0; i < numValues; i++) {
+        datOut.writeLong(LONG_ZERO);
+      }
+      return numValues;
+    }
+  }
+
+  /**
+   * Opens all necessary files, but does not read any data in until you call
+   * {@link #load}.
+   */
+  static class FloatsReader extends Reader {
+
+    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)
+        throws IOException {
+      datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
+          IndexFileNames.CSF_DATA_EXTENSION));
+      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();
+      try {
+        indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
+        // skip precision:
+        indexInput.readByte();
+        out.copyBytes(indexInput, precisionBytes * 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 {
+      ByteBuffer buffer = ByteBuffer.allocate(precisionBytes * maxDoc);
+      IndexInput indexInput = (IndexInput) datIn.clone();
+      indexInput.seek(CodecUtil.headerLength(CODEC_NAME));
+      // skip precision:
+      indexInput.readByte();
+      assert buffer.hasArray() : "Buffer must support Array";
+      final byte[] arr = buffer.array();
+      indexInput.readBytes(arr, 0, arr.length);
+      return precisionBytes == 4 ? new Source4(buffer) : new Source8(buffer);
+    }
+
+    private class Source4 extends Source {
+      private final FloatBuffer values;
+
+      Source4(ByteBuffer buffer) {
+        values = buffer.asFloatBuffer();
+      }
+
+      @Override
+      public double floats(int docID) {
+        final float f = values.get(docID);
+        // nocommit should we return NaN as default instead of 0.0?
+        return Float.isNaN(f) ? 0.0f : f;
+      }
+
+      public long ramBytesUsed() {
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + values.limit()
+            * RamUsageEstimator.NUM_BYTES_FLOAT;
+      }
+    }
+
+    private class Source8 extends Source {
+      private final DoubleBuffer values;
+
+      Source8(ByteBuffer buffer) {
+        values = buffer.asDoubleBuffer();
+      }
+
+      @Override
+      public double floats(int docID) {
+        final double d = values.get(docID);
+        // nocommit should we return NaN as default instead of 0.0?
+        return Double.isNaN(d) ? 0.0d : d;
+      }
+
+      public long ramBytesUsed() {
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + values.limit()
+            * RamUsageEstimator.NUM_BYTES_DOUBLE;
+      }
+    }
+
+    public void close() throws IOException {
+      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);
+    }
+  }
+
+  static final class Floats4Enum extends FloatsEnumImpl {
+
+    Floats4Enum(AttributeSource source, IndexInput dataIn, int maxDoc)
+        throws IOException {
+      super(source, dataIn, 4, maxDoc, Values.SIMPLE_FLOAT_4BYTE);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc)
+        return pos = NO_MORE_DOCS;
+      dataIn.seek(fp + (target * precision));
+      ref.floats[0] = Float.intBitsToFloat(dataIn.readInt());
+      ref.offset = 0; // nocommit -- can we igore this?
+      return pos = target;
+    }
+
+    @Override
+    public int docID() {
+      return pos;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      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, Values.SIMPLE_FLOAT_8BYTE);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc)
+        return pos = NO_MORE_DOCS;
+      dataIn.seek(fp + (target * precision));
+      ref.floats[0] = Double.longBitsToDouble(dataIn.readLong());
+      ref.offset = 0; // nocommit -- can we igore this?
+      return pos = target;
+    }
+
+    @Override
+    public int docID() {
+      return pos;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      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;
+    protected final FloatsRef ref;
+
+    FloatsEnumImpl(AttributeSource source, IndexInput dataIn, int precision,
+        int maxDoc, Values type) throws IOException {
+      super(source, precision == 4 ? Values.SIMPLE_FLOAT_4BYTE
+          : Values.SIMPLE_FLOAT_8BYTE);
+      this.dataIn = dataIn;
+      this.precision = precision;
+      this.maxDoc = maxDoc;
+      fp = dataIn.getFilePointer();
+      this.ref = attr.floats();
+      this.ref.offset = 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      dataIn.close();
+    }
+  }
+}
\ No newline at end of file

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,32 @@
+package org.apache.lucene.index.values;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.values.PackedIntsImpl.IntsReader;
+import org.apache.lucene.index.values.PackedIntsImpl.IntsWriter;
+import org.apache.lucene.store.Directory;
+//nocommit - add mmap version 
+//nocommti - add bulk copy where possible
+public class Ints {
+
+  private Ints() {
+  }
+  
+  public static void files(String id, Collection<String> files)
+      throws IOException {
+    files.add(IndexFileNames.segmentFileName(id, "",
+        IndexFileNames.CSF_DATA_EXTENSION));
+  }
+
+  public static Writer getWriter(Directory dir, String id, boolean useFixedArray)
+      throws IOException {
+     //nocommit - implement fixed?!
+    return new IntsWriter(dir, id);
+  }
+
+  public static Reader getReader(Directory dir, String id, boolean useFixedArray) throws IOException {
+    return new IntsReader(dir, id);
+  }
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,240 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.Directory;
+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.LongsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.PackedInts;
+
+/** Stores ints packed with fixed-bit precision. */
+class PackedIntsImpl {
+
+  private static final String CODEC_NAME = "PackedInts";
+
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  static class IntsWriter extends Writer {
+    // nocommit - can we bulkcopy this on a merge?
+    private LongsRef intsRef;
+    private long[] docToValue;
+    private long minValue;
+    private long maxValue;
+    private boolean started;
+    private final Directory dir;
+    private final String id;
+    private int maxDocID;
+    private int minDocID;
+
+    protected IntsWriter(Directory dir, String id) throws IOException {
+      this.dir = dir;
+      this.id = id;
+      docToValue = new long[1];
+    }
+
+    @Override
+    synchronized public void add(int docID, long v) throws IOException {
+
+      if (!started) {
+        minValue = maxValue = v;
+        minDocID = maxDocID = docID;
+        started = true;
+
+      } else {
+        if (v < minValue) {
+          minValue = v;
+        } else if (v > maxValue) {
+          maxValue = v;
+        }
+        if (docID < minDocID) {
+          minDocID = docID;
+        } else if (docID > maxDocID) {
+          maxDocID = docID;
+        }
+      }
+      if (docID >= docToValue.length) {
+        docToValue = ArrayUtil.grow(docToValue, 1 + docID);
+      }
+      docToValue[docID] = v;
+    }
+
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if(!started)
+        return;
+      final IndexOutput datOut = dir.createOutput(IndexFileNames
+          .segmentFileName(id, "", IndexFileNames.CSF_DATA_EXTENSION));
+      CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
+
+      // nocommit -- long can't work right since it's signed
+      datOut.writeLong(minValue);
+      // write a default value to recognize docs without a value for that field
+      final long defaultValue = ++maxValue - minValue;
+      datOut.writeLong(defaultValue);
+      PackedInts.Writer w = PackedInts.getWriter(datOut, docCount, PackedInts.bitsRequired(maxValue-minValue));
+         
+      final int limit = maxDocID + 1;
+      for (int i = 0; i < minDocID; i++) {
+        w.add(defaultValue);
+      }
+      for (int i = minDocID; i < limit; i++) {
+        w.add(docToValue[i] - minValue);
+      }
+      for (int i = limit; i < docCount; i++) {
+        w.add(defaultValue);
+      }
+      w.finish();
+
+      datOut.close();
+    }
+
+    public long ramBytesUsed() {
+      return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + docToValue.length
+          * RamUsageEstimator.NUM_BYTES_LONG;
+    }
+
+    @Override
+    protected void add(int docID) throws IOException {
+      add(docID, intsRef.get());
+    }
+
+    @Override
+    protected void setNextAttribute(ValuesAttribute attr) {
+      intsRef = attr.ints();
+    }
+  }
+
+  /**
+   * Opens all necessary files, but does not read any data in until you call
+   * {@link #load}.
+   */
+  static class IntsReader extends Reader {
+    private final IndexInput datIn;
+
+    protected IntsReader(Directory dir, String id) throws IOException {
+      datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
+          IndexFileNames.CSF_DATA_EXTENSION));
+      CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
+    }
+
+    /**
+     * 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 {
+      return new IntsSource((IndexInput) datIn.clone());
+    }
+
+    private static class IntsSource extends Source {
+      private final long minValue;
+      private final long defaultValue;
+      private final PackedInts.Reader values;
+
+      public IntsSource(IndexInput dataIn) throws IOException {
+        dataIn.seek(CodecUtil.headerLength(CODEC_NAME));
+        minValue = dataIn.readLong();
+        defaultValue = dataIn.readLong();
+        values = PackedInts.getReader(dataIn);
+      }
+
+      @Override
+      public long ints(int docID) {
+        // nocommit -- can we somehow avoid 2X method calls
+        // on each get? must push minValue down, and make
+        // PackedInts implement Ints.Source
+        final long val = values.get(docID);
+        // docs not having a value for that field must return a default value
+        return val == defaultValue ? 0 : minValue + val;
+      }
+
+      public long ramBytesUsed() {
+        // TODO(simonw): move that to PackedInts?
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
+            + values.getBitsPerValue() * values.size();
+      }
+    }
+
+    public void close() throws IOException {
+      datIn.close();
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new IntsEnumImpl(source, (IndexInput) datIn.clone());
+    }
+
+  }
+
+  private static final class IntsEnumImpl extends ValuesEnum {
+    private final PackedInts.ReaderIterator ints;
+    private long minValue;
+    private final IndexInput dataIn;
+    private final long defaultValue;
+    private LongsRef ref;
+    private final int maxDoc;
+    private int pos = -1;
+
+    private IntsEnumImpl(AttributeSource source, IndexInput dataIn)
+        throws IOException {
+      super(source, Values.PACKED_INTS);
+      this.ref = attr.ints();
+      this.ref.offset = 0;
+      this.dataIn = dataIn;
+      dataIn.seek(CodecUtil.headerLength(CODEC_NAME));
+      minValue = dataIn.readLong();
+      defaultValue = dataIn.readLong();
+      this.ints = PackedInts.getReaderIterator(dataIn);
+      maxDoc = ints.size();
+    }
+
+    @Override
+    public void close() throws IOException {
+      ints.close();
+      dataIn.close();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc)
+        return pos = NO_MORE_DOCS;
+      final long val = ints.advance(target);
+      ref.ints[0] = val == defaultValue? 0:minValue + val;
+      ref.offset = 0; // can we skip this?
+      return pos = target;
+    }
+
+    @Override
+    public int docID() {
+      return pos;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(pos+1);
+    }
+  }
+}
\ No newline at end of file

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,109 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Comparator;
+
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+
+public abstract class Reader implements Closeable {
+  
+  
+  public ValuesEnum getEnum() throws IOException{
+    return getEnum(null);
+  }
+
+  public abstract ValuesEnum getEnum(AttributeSource attrSource) throws IOException;
+
+  public abstract Source load() throws IOException;
+
+  public SortedSource loadSorted(Comparator<BytesRef> comparator) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+
+  /**
+   * Source of integer (returned as java long), per document. The underlying
+   * implementation may use different numbers of bits per value; long is only
+   * used since it can handle all precisions.
+   */
+  public static abstract class Source {
+
+    public long ints(int docID) {
+      throw new UnsupportedOperationException("ints are not supported");
+    }
+
+    public double floats(int docID) {
+      throw new UnsupportedOperationException("floats are not supported");
+    }
+
+    public BytesRef bytes(int docID) {
+      throw new UnsupportedOperationException("bytes are not supported");
+    }
+    
+    /** Returns number of unique values.  Some impls may
+     * throw UnsupportedOperationException. */
+    public int getValueCount() {
+      throw new UnsupportedOperationException();
+    }
+    
+    public ValuesEnum getEnum() throws IOException{
+      return getEnum(null);
+    }
+    
+    // nocommit - enable obtaining enum from source since this is already in memory
+    public /*abstract*/ ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    public abstract long ramBytesUsed();
+  }
+
+  public static abstract class SortedSource extends Source {
+
+    @Override
+    public BytesRef bytes(int docID) {
+      return getByOrd(ord(docID));
+    }
+
+    /**
+     * Returns ord for specified docID. If this docID had not been added to the
+     * Writer, the ord is 0. Ord is dense, ie, starts at 0, then increments by 1
+     * for the next (as defined by {@link Comparator} value.
+     */
+    public abstract int ord(int docID);
+
+    /** Returns value for specified ord. */
+    public abstract BytesRef getByOrd(int ord);
+
+    public static class LookupResult {
+      public boolean found;
+      public int ord;
+    }
+
+    /**
+     * Finds the largest ord whose value is <= the requested value. If
+     * {@link LookupResult#found} is true, then ord is an exact match. The
+     * returned {@link LookupResult} may be reused across calls.
+     */
+    public abstract LookupResult getByValue(BytesRef value);
+  }
+  
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,48 @@
+package org.apache.lucene.index.values;
+/**
+ * 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.
+ */
+
+/** 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.
+ *
+ * <p><b>NOTE</b>: This feature is experimental and the
+ * API is free to change in non-backwards-compatible ways.  */
+public enum Values {
+
+  /** 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,
+  PACKED_INTS_FIXED,
+  SIMPLE_FLOAT_4BYTE,
+  SIMPLE_FLOAT_8BYTE,
+
+  // nocommit -- shouldn't lucene decide/detect straight vs
+  // deref, as well fixed vs var?
+  BYTES_FIXED_STRAIGHT,
+  BYTES_FIXED_DEREF,
+  BYTES_FIXED_SORTED,
+
+  BYTES_VAR_STRAIGHT,
+  BYTES_VAR_DEREF,
+  BYTES_VAR_SORTED
+
+  // nocommit -- need STRING variants as well
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,34 @@
+package org.apache.lucene.index.values;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.util.Comparator;
+
+import org.apache.lucene.util.Attribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FloatsRef;
+import org.apache.lucene.util.LongsRef;
+
+public interface ValuesAttribute extends Attribute {
+  public Values type();
+  public BytesRef bytes();
+  public FloatsRef floats();
+  public LongsRef ints();
+  public void setType(Values type);
+  public Comparator<BytesRef> bytesComparator();
+  public void setBytesComparator(Comparator<BytesRef> comp);
+  
+}
\ No newline at end of file

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,151 @@
+package org.apache.lucene.index.values;
+
+import java.util.Comparator;
+
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FloatsRef;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.SetOnce;
+
+public class ValuesAttributeImpl extends AttributeImpl implements ValuesAttribute {
+  private Values type;
+  private BytesRef bytes = null;
+  private FloatsRef floats = null;
+  private LongsRef ints = null;
+  private Comparator<BytesRef> bytesComp;
+
+  public BytesRef bytes() {
+    return bytes;
+  }
+
+  public FloatsRef floats() {
+    return floats;
+  }
+
+  public LongsRef ints() {
+    return ints;
+  }
+
+  public Values type() {
+    return type;
+  }
+
+  public void setType(Values type) {
+    this.type = type;
+    switch (type) {
+    case BYTES_FIXED_DEREF:
+    case BYTES_FIXED_SORTED:
+    case BYTES_FIXED_STRAIGHT:
+    case BYTES_VAR_DEREF:
+    case BYTES_VAR_SORTED:
+    case BYTES_VAR_STRAIGHT:
+      bytes = new BytesRef();
+      ints = null;
+      floats = null;
+      break;
+    case PACKED_INTS:
+    case PACKED_INTS_FIXED:
+      ints = new LongsRef(new long[1], 0, 1);
+      bytes = null;
+      floats = null;
+      break;
+    case SIMPLE_FLOAT_4BYTE:
+    case SIMPLE_FLOAT_8BYTE:
+      floats = new FloatsRef(new double[1], 0, 1);
+      ints = null;
+      bytes = null;
+      break;
+
+    }
+  }
+
+  @Override
+  public void clear() {
+    // TODO
+  }
+
+  @Override
+  public void copyTo(AttributeImpl target) {
+    ValuesAttributeImpl other = (ValuesAttributeImpl)target;
+    other.setType(type);
+    
+    switch (type) {
+    case BYTES_FIXED_DEREF:
+    case BYTES_FIXED_SORTED:
+    case BYTES_FIXED_STRAIGHT:
+    case BYTES_VAR_DEREF:
+    case BYTES_VAR_SORTED:
+    case BYTES_VAR_STRAIGHT:
+      bytes = (BytesRef) other.bytes.clone();
+      break;
+    case PACKED_INTS:
+    case PACKED_INTS_FIXED:
+      ints = (LongsRef) other.ints.clone();
+      break;
+    case SIMPLE_FLOAT_4BYTE:
+    case SIMPLE_FLOAT_8BYTE:
+      floats = (FloatsRef) other.floats.clone();
+      break;
+
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#hashCode()
+   */
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 0;
+    result = prime * result + ((bytes == null) ? 0 : bytes.hashCode());
+    result = prime * result + ((floats == null) ? 0 : floats.hashCode());
+    result = prime * result + ((ints == null) ? 0 : ints.hashCode());
+    result = prime * result + ((type == null) ? 0 : type.hashCode());
+    return result;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#equals(java.lang.Object)
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (getClass() != obj.getClass())
+      return false;
+    ValuesAttributeImpl other = (ValuesAttributeImpl) obj;
+    if (bytes == null) {
+      if (other.bytes != null)
+        return false;
+    } else if (!bytes.equals(other.bytes))
+      return false;
+    if (floats == null) {
+      if (other.floats != null)
+        return false;
+    } else if (!floats.equals(other.floats))
+      return false;
+    if (ints == null) {
+      if (other.ints != null)
+        return false;
+    } else if (!ints.equals(other.ints))
+      return false;
+    if (type == null) {
+      if (other.type != null)
+        return false;
+    } else if (!type.equals(other.type))
+      return false;
+    return true;
+  }
+
+  public Comparator<BytesRef> bytesComparator() {
+    return bytesComp;
+  }
+
+  public void setBytesComparator(Comparator<BytesRef> comp) {
+    bytesComp = comp;    
+  }
+
+
+
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,62 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Attribute;
+import org.apache.lucene.util.AttributeSource;
+
+public abstract class ValuesEnum extends DocIdSetIterator{
+  private AttributeSource source;
+  protected final ValuesAttribute attr;
+
+ 
+  protected ValuesEnum(Values enumType) {
+     this(null, enumType);
+  }
+
+  protected ValuesEnum(AttributeSource source, Values enumType) {
+    this.source = source;
+    boolean setType = !hasAttribute(ValuesAttribute.class);
+    attr = addAttribute(ValuesAttribute.class);
+    if (setType)
+      attr.setType(enumType);
+  }
+
+  public AttributeSource attributes() {
+    if (source == null)
+      source = new AttributeSource();
+    return source;
+  }
+
+  public <T extends Attribute> T addAttribute(Class<T> attr) {
+    return attributes().addAttribute(attr);
+  }
+
+  public <T extends Attribute> T getAttribute(Class<T> attr) {
+    return attributes().getAttribute(attr);
+  }
+
+  public <T extends Attribute> boolean hasAttribute(Class<T> attr) {
+    return attributes().hasAttribute(attr);
+  }
+
+  public abstract void close() throws IOException;
+
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: 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=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,255 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+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.Directory;
+import org.apache.lucene.store.IndexInput;
+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.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.ByteBlockPool.Allocator;
+import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.BytesRefHash.ParallelArrayBase;
+import org.apache.lucene.util.BytesRefHash.ParallelBytesStartArray;
+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
+// docs reference that single source
+
+class VarDerefBytesImpl {
+
+  static final String CODEC_NAME = "VarDerefBytes";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+  
+ 
+
+  private static class AddressParallelArray extends ParallelArrayBase<AddressParallelArray> {
+    final int[] address;
+    
+    AddressParallelArray(int size, AtomicLong bytesUsed) {
+      super(size, bytesUsed);
+      address = new int[size]; 
+    }
+    @Override
+    protected int bytesPerEntry() {
+      return RamUsageEstimator.NUM_BYTES_INT + super.bytesPerEntry();
+    }
+
+    @Override
+    protected void copyTo(AddressParallelArray toArray, int numToCopy) {
+      super.copyTo(toArray, numToCopy);
+      System.arraycopy(address, 0, toArray.address, 0, size);
+      
+    }
+
+    @Override
+    public AddressParallelArray newInstance(int size) {
+      return new AddressParallelArray(size, bytesUsed);
+    }
+    
+  }
+
+
+  static class Writer extends BytesWriterBase {
+    private int[] docToAddress;
+    private int address = 1;
+    
+    private final ParallelBytesStartArray<AddressParallelArray> array = new ParallelBytesStartArray<AddressParallelArray>(new AddressParallelArray(0, bytesUsed));
+    private final BytesRefHash hash  = new BytesRefHash(pool, 16, array) ;
+
+    public Writer(Directory dir, String id) throws IOException  {
+      this(dir, id, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+          new AtomicLong());
+    }
+    public Writer(Directory dir, String id, Allocator allocator, AtomicLong bytesUsed) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, new ByteBlockPool(allocator), bytesUsed);
+      docToAddress = new int[1];
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
+    }
+
+    @Override
+    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) {
+        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.array.address[e] = address;
+        address += IOUtils.writeLength(datOut, bytes);
+        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+        address += bytes.length;
+      } else {
+        docAddress = array.array.address[(-e)-1];
+      }
+      docToAddress[docID] = docAddress;
+    }
+
+    public long ramBytesUsed() {
+      return bytesUsed.get();
+    }
+
+    // Important that we get docCount, in case there were
+    // 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
+      // nocommit -- 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();
+      hash.clear(true);
+      super.finish(docCount);
+    }
+  }
+
+  public static class Reader extends BytesReaderBase {
+
+    Reader(Directory dir, String id, int maxDoc)
+      throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_START, true);
+    }
+
+    @Override
+    public Source load() throws IOException {
+      return new Source(cloneData(), cloneIndex());
+    }
+
+    private static class Source extends BytesBaseSource {
+      // TODO: paged data
+      private final byte[] data;
+      private final BytesRef bytesRef = new BytesRef();
+      private final PackedInts.Reader index;
+
+      public Source(IndexInput datIn, IndexInput idxIn) throws IOException {
+        super(datIn, idxIn);
+        datIn.seek(CodecUtil.headerLength(CODEC_NAME));
+        idxIn.seek(CodecUtil.headerLength(CODEC_NAME));
+
+        final int totBytes = idxIn.readInt();
+        data = new byte[totBytes];
+        datIn.readBytes(data, 0, totBytes);
+
+        index = PackedInts.getReader(idxIn);
+        bytesRef.bytes = data;
+      }
+
+      @Override
+      public BytesRef bytes(int docID) {
+        int address = (int) index.get(docID);
+        if (address == 0) {
+          assert defaultValue.length == 0: " default value manipulated";
+          return defaultValue;
+        } else {
+          address--;
+          if ((data[address] & 0x80) == 0) {
+            // length is 1 byte
+            bytesRef.length = data[address];
+            bytesRef.offset = address+1;
+          } else {
+            bytesRef.length = (data[address]&0x7f) + ((data[address+1]&0xff)<<7);
+            bytesRef.offset = address+2;
+          }
+          return bytesRef;
+        }
+      }
+      
+      @Override
+      public int getValueCount() {
+        return index.size();
+      }
+
+      public long ramBytesUsed() {
+        // TODO(simonw): move address ram usage to PackedInts? 
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + data.length + 
+        (RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + index.getBitsPerValue() * index.size());
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new VarDerefBytesEnum(source, cloneData(), cloneIndex(), CODEC_NAME);
+    }
+    
+    static class VarDerefBytesEnum extends DerefBytesEnum {
+
+      public VarDerefBytesEnum(AttributeSource source, IndexInput datIn, IndexInput idxIn,
+          String codecName) throws IOException {
+        super(source, datIn, idxIn, codecName, -1, Values.BYTES_VAR_DEREF);
+      }
+
+    
+      @Override
+      protected void fill(long address, BytesRef ref) throws IOException {
+        // TODO(simonw): use pages here
+        datIn.seek(fp + --address);
+        final byte sizeByte = datIn.readByte();
+        final int size;
+        if ((sizeByte & 0x80) == 0) {
+          // length is 1 byte
+          size = sizeByte;
+        } else {
+          size = (sizeByte & 0x7f) + ((datIn.readByte() & 0xff) << 7);
+        }
+        if(ref.bytes.length < size)
+          ref.grow(size);
+        ref.length = size;
+        ref.offset = 0;
+        datIn.readBytes(ref.bytes, 0, size);
+      }
+    }
+  }
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL