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 [3/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/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,344 @@
+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.Arrays;
+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.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 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 VarSortedBytesImpl {
+
+  static final String CODEC_NAME = "VarDerefBytes";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  static class Writer extends BytesWriterBase {
+    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);
+      this.comp = comp;
+      docToEntry = new int[1];
+      docToEntry[0] = -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 (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;
+    }
+
+    // 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 {
+      final int count = hash.size();
+      if (count == 0)
+        return;
+      initIndexOut();
+      initDataOut();
+      int[] sortedEntries = hash.sort(comp);
+
+      // first dump bytes data, recording index & offset as
+      // we go
+      long offset = 0;
+      long lastOffset = 0;
+      final int[] index = new int[count];
+      final long[] offsets = new long[count];
+      for (int i = 0; i < count; i++) {
+        final int e = sortedEntries[i];
+        offsets[i] = offset;
+        index[e] = 1 + i;
+
+        final BytesRef bytes = hash.get(e);
+        // TODO: we could prefix code...
+        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+        lastOffset = offset;
+        offset += bytes.length;
+      }
+
+      // total bytes of data
+      idxOut.writeLong(offset);
+
+      // write index -- first doc -> 1+ord
+      // nocommit -- 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
+      // nocommit -- allow not -1:
+      PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count,
+          PackedInts.bitsRequired(lastOffset));
+      for (int i = 0; i < count; i++) {
+        offsetWriter.add(offsets[i]);
+      }
+      offsetWriter.finish();
+
+      super.finish(docCount);
+      bytesUsed.addAndGet((-docToEntry.length)
+          * RamUsageEstimator.NUM_BYTES_INT);
+
+    }
+  }
+
+  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 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(), comp);
+    }
+
+    private static class Source extends BytesBaseSortedSource {
+      // TODO: paged data
+      private final byte[] data;
+      private final BytesRef bytesRef = new BytesRef();
+      private final PackedInts.Reader docToOrdIndex;
+      private final PackedInts.Reader ordToOffsetIndex; // 0-based
+      private final long totBytes;
+      private final int valueCount;
+      private final LookupResult lookupResult = new LookupResult();
+      private final Comparator<BytesRef> comp;
+
+      public Source(IndexInput datIn, IndexInput idxIn,
+          Comparator<BytesRef> comp) throws IOException {
+        super(datIn, idxIn);
+        totBytes = idxIn.readLong();
+        data = new byte[(int) totBytes];
+        datIn.readBytes(data, 0, (int) totBytes);
+        docToOrdIndex = PackedInts.getReader(idxIn);
+        ordToOffsetIndex = PackedInts.getReader(idxIn);
+        valueCount = ordToOffsetIndex.size();
+        bytesRef.bytes = data;
+        // default byte sort order
+        this.comp = comp == null ? BytesRef.getUTF8SortedAsUnicodeComparator()
+            : comp;
+
+      }
+
+      @Override
+      public BytesRef getByOrd(int ord) {
+        return ord == 0 ? defaultValue : deref(--ord);
+      }
+
+      @Override
+      public int ord(int docID) {
+        return (int) docToOrdIndex.get(docID);
+      }
+
+      @Override
+      public LookupResult getByValue(BytesRef bytes) {
+        return binarySearch(bytes, 0, valueCount - 1);
+      }
+
+      public long ramBytesUsed() {
+        // TODO(simonw): move ram usage to PackedInts?
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
+            + data.length
+            + (RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + docToOrdIndex
+                .getBitsPerValue()
+                * docToOrdIndex.getBitsPerValue())
+            + (RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + ordToOffsetIndex
+                .getBitsPerValue()
+                * ordToOffsetIndex.getBitsPerValue());
+      }
+
+      @Override
+      public int getValueCount() {
+        return valueCount;
+      }
+
+      // ord is 0-based
+      private BytesRef deref(int ord) {
+        bytesRef.offset = (int) ordToOffsetIndex.get(ord);
+        final long nextOffset;
+        if (ord == valueCount - 1) {
+          nextOffset = totBytes;
+        } else {
+          nextOffset = ordToOffsetIndex.get(1 + ord);
+        }
+        bytesRef.length = (int) (nextOffset - bytesRef.offset);
+        return bytesRef;
+      }
+
+      // TODO: share w/ FixedSortedBytesValues?
+      private LookupResult binarySearch(BytesRef b, int low, int high) {
+
+        while (low <= high) {
+          int mid = (low + high) >>> 1;
+          deref(mid);
+          final 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;
+          }
+        }
+        assert comp.compare(bytesRef, b) != 0;
+        lookupResult.ord = low;
+        lookupResult.found = false;
+        return lookupResult;
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new VarSortedBytesEnum(source, cloneData(), cloneIndex());
+    }
+
+    private static class VarSortedBytesEnum extends ValuesEnum {
+
+      private PackedInts.Reader docToOrdIndex;
+      private PackedInts.Reader ordToOffsetIndex;
+      private IndexInput idxIn;
+      private IndexInput datIn;
+      private final BytesRef bytesRef;
+      private int valueCount;
+      private long totBytes;
+      private int docCount;
+      private int pos = -1;
+      private final long fp;
+
+      protected VarSortedBytesEnum(AttributeSource source, IndexInput datIn,
+          IndexInput idxIn) throws IOException {
+        super(source, Values.BYTES_VAR_SORTED);
+        bytesRef = attr.bytes();
+        totBytes = idxIn.readLong();
+        // keep that in memory to prevent lots of disk seeks
+        docToOrdIndex = PackedInts.getReader(idxIn);
+        ordToOffsetIndex = PackedInts.getReader(idxIn);
+        valueCount = ordToOffsetIndex.size();
+        docCount = docToOrdIndex.size();
+        fp = datIn.getFilePointer();
+        this.idxIn = idxIn;
+        this.datIn = datIn;
+      }
+
+      @Override
+      public void close() throws IOException {
+        idxIn.close();
+        datIn.close();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        if (target >= docCount)
+          return pos = NO_MORE_DOCS;
+        final int ord = (int) docToOrdIndex.get(target) - 1;
+        if (ord == -1) {
+          bytesRef.length = 0;
+          bytesRef.offset = 0;
+          return pos = target;
+        }
+        final long offset = ordToOffsetIndex.get(ord);
+        final long nextOffset;
+        if (ord == valueCount - 1) {
+          nextOffset = totBytes;
+        } else {
+          nextOffset = ordToOffsetIndex.get(1 + ord);
+        }
+        final int length = (int) (nextOffset - offset);
+        datIn.seek(fp + offset);
+        if (bytesRef.bytes.length < length)
+          bytesRef.grow(length);
+        datIn.readBytes(bytesRef.bytes, 0, length);
+        bytesRef.length = length;
+        bytesRef.offset = 0;
+        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/VarSortedBytesImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

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

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,232 @@
+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.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.PackedInts;
+
+// Variable length byte[] per document, no sharing
+
+class VarStraightBytesImpl {
+
+  static final String CODEC_NAME = "VarStraightBytes";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  static class Writer extends BytesWriterBase {
+    private int address;
+    // start at -1 if the first added value is > 0
+    private int lastDocID = -1;
+    private int[] docToAddress;
+    
+    public Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, null, bytesUsed);
+      docToAddress = new int[1];
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
+    }
+
+    public Writer(Directory dir, String id) throws IOException {
+      this(dir, id, new AtomicLong());
+    }
+
+    // Fills up to but not including this docID
+    private void fill(final int docID) {
+      if (docID >= docToAddress.length) {
+        int oldSize = docToAddress.length;
+        docToAddress = ArrayUtil.grow(docToAddress, 1 + docID);
+        bytesUsed.addAndGet(-(docToAddress.length-oldSize)*RamUsageEstimator.NUM_BYTES_INT);
+      }
+      for (int i = lastDocID + 1; i < docID; i++) {
+        docToAddress[i] = address;
+      }
+      lastDocID = docID;
+    }
+
+    @Override
+    synchronized public void add(int docID, BytesRef bytes) throws IOException {
+      if(bytes.length == 0)
+        return; // default
+      if (datOut == null)
+        initDataOut();
+      fill(docID);
+      docToAddress[docID] = address;
+      datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+      address += bytes.length;
+    }
+
+    @Override
+    synchronized public void finish(int docCount) throws IOException {
+      if (datOut == null)
+        return;
+      initIndexOut();
+      // write all lengths to index
+      // write index
+      fill(docCount);
+      idxOut.writeVInt(address);
+      // nocommit -- allow not -1
+      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+          PackedInts.bitsRequired(address));
+      for (int i = 0; i < docCount; i++) {
+        w.add(docToAddress[i]);
+      }
+      w.finish();
+      bytesUsed.addAndGet(-(docToAddress.length)*RamUsageEstimator.NUM_BYTES_INT);
+      docToAddress = null;
+      super.finish(docCount);
+    }
+
+    public long ramBytesUsed() {
+      return bytesUsed.get();
+    }
+  }
+
+  public static class Reader extends BytesReaderBase {
+    private final int maxDoc;
+
+    Reader(Directory dir, String id, int maxDoc) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_START, true);
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public Source load() throws IOException {
+      return new Source(cloneData(), cloneIndex());
+    }
+
+    private class Source extends BytesBaseSource {
+      private final int totBytes;
+      // TODO: paged data
+      private final byte[] data;
+      private final BytesRef bytesRef = new BytesRef();
+      private final PackedInts.Reader addresses;
+
+      public Source(IndexInput datIn, IndexInput idxIn) throws IOException {
+        super(datIn, idxIn);
+        totBytes = idxIn.readVInt();
+        data = new byte[totBytes];
+        datIn.readBytes(data, 0, totBytes);
+        addresses = PackedInts.getReader(idxIn);
+        bytesRef.bytes = data;
+      }
+
+      @Override
+      public BytesRef bytes(int docID) {
+        final int address = (int) addresses.get(docID);
+        bytesRef.offset = address;
+        if (docID == maxDoc - 1) {
+          bytesRef.length = totBytes - bytesRef.offset;
+        } else {
+          bytesRef.length = (int) addresses.get(1 + docID) - bytesRef.offset;
+        }
+        return bytesRef;
+      }
+
+      @Override
+      public int getValueCount() {
+        throw new UnsupportedOperationException();
+      }
+
+      public long ramBytesUsed() {
+        // TODO(simonw): move address ram usage to PackedInts?
+        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
+            + data.length
+            + (RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + addresses
+                .getBitsPerValue()
+                * addresses.size());
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new VarStrainghtBytesEnum(source, cloneData(), cloneIndex());
+    }
+
+    private class VarStrainghtBytesEnum extends ValuesEnum {
+      private final PackedInts.Reader addresses;
+      private final IndexInput datIn;
+      private final IndexInput idxIn;
+      private final long fp;
+      private final int totBytes;
+      private final BytesRef ref;
+      private int pos = -1;
+
+      protected VarStrainghtBytesEnum(AttributeSource source, IndexInput datIn,
+          IndexInput idxIn) throws IOException {
+        super(source, Values.BYTES_VAR_STRAIGHT);
+        totBytes = idxIn.readVInt();
+        fp = datIn.getFilePointer();
+        addresses = PackedInts.getReader(idxIn);
+        this.datIn = datIn;
+        this.idxIn = idxIn;
+        ref = attr.bytes();
+
+      }
+
+      @Override
+      public void close() throws IOException {
+        datIn.close();
+        idxIn.close();
+      }
+
+      @Override
+      public int advance(final int target) throws IOException {
+        if (target >= maxDoc) {
+          ref.length = 0;
+          ref.offset = 0;
+          return pos = NO_MORE_DOCS;
+        }
+        final long addr = addresses.get(target);
+        if (addr == totBytes) {
+          // nocommit is that a valid default value
+          ref.length = 0;
+          ref.offset = 0;
+          return pos = target;
+        }
+        datIn.seek(fp + addr);
+        final int size = (int) (target == maxDoc - 1 ? totBytes - addr
+            : addresses.get(target + 1) - addr);
+        if (ref.bytes.length < size)
+          ref.grow(size);
+        ref.length = 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/VarStraightBytesImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

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

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,92 @@
+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.List;
+
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+public abstract class Writer {
+
+  /** Records the specfied value for the docID */
+  public void add(int docID, long value) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /** Records the specfied value for the docID */
+  public void add(int docID, double value) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /** Records the specfied value for the docID */
+  public void add(int docID, BytesRef value) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  /** Records the specfied value for the docID */
+  protected abstract void add(int docID) throws IOException;
+
+  protected abstract void setNextAttribute(ValuesAttribute attr);
+
+  /** Finish writing, close any files */
+  public abstract void finish(int docCount) throws IOException;
+
+  public static class MergeState {
+    public final Reader reader;
+    public final int docBase;
+    public final int docCount;
+    public final Bits bits;
+
+    public MergeState(Reader reader, int docBase, int docCount, Bits bits) {
+      assert reader != null;
+      this.reader = reader;
+      this.docBase = docBase;
+      this.docCount = docCount;
+      this.bits = bits;
+    }
+  }
+
+  public void add(List<MergeState> states) throws IOException {
+    for (MergeState state : states) {
+      merge(state);
+    }
+  }
+
+  // enables bulk copies in subclasses per MergeState
+  protected void merge(MergeState state) throws IOException {
+    final ValuesEnum valEnum = state.reader.getEnum();
+    assert valEnum != null;
+    try {
+      final ValuesAttribute attr = valEnum.addAttribute(ValuesAttribute.class);
+      setNextAttribute(attr);
+      int docID = state.docBase;
+      final Bits bits = state.bits;
+      final int docCount = state.docCount;
+      for (int i = 0; i < docCount; i++) {
+        if (bits == null || !bits.get(i)) {
+          if (valEnum.advance(i) == ValuesEnum.NO_MORE_DOCS)
+            break;
+          add(docID++);
+        }
+      }
+    } finally {
+      valEnum.close();
+    }
+  }
+}

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

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

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java Tue Oct 12 06:15:03 2010
@@ -22,8 +22,9 @@ import java.text.Collator;
 import java.util.Locale;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.FieldCache.DocTermsIndex;
+import org.apache.lucene.index.values.Reader.Source;
 import org.apache.lucene.search.FieldCache.DocTerms;
+import org.apache.lucene.search.FieldCache.DocTermsIndex;
 import org.apache.lucene.search.cache.ByteValuesCreator;
 import org.apache.lucene.search.cache.CachedArray;
 import org.apache.lucene.search.cache.CachedArrayCreator;
@@ -39,9 +40,9 @@ import org.apache.lucene.search.cache.Ca
 import org.apache.lucene.search.cache.CachedArray.LongValues;
 import org.apache.lucene.search.cache.CachedArray.ShortValues;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.packed.Direct8;
 import org.apache.lucene.util.packed.Direct16;
 import org.apache.lucene.util.packed.Direct32;
+import org.apache.lucene.util.packed.Direct8;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
@@ -159,7 +160,6 @@ public abstract class FieldComparator {
    *   comparators can just return "this" to reuse the same
    *   comparator across segments
    * @throws IOException
-   * @throws IOException
    */
   public abstract FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException;
 
@@ -309,6 +309,65 @@ public abstract class FieldComparator {
     }
   }
 
+  /** Uses float index values to sort by ascending value */
+  public static final class FloatIndexValuesComparator extends FieldComparator {
+    private final double[] values;
+    private Source currentReaderValues;
+    private final String field;
+    private double bottom;
+
+    FloatIndexValuesComparator(int numHits, String field) {
+      values = new double[numHits];
+      this.field = field;
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+      final double v1 = values[slot1];
+      final double v2 = values[slot2];
+      if (v1 > v2) {
+        return 1;
+      } else if (v1 < v2) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+
+    @Override
+    public int compareBottom(int doc) {
+      final double v2 = currentReaderValues.floats(doc);
+      if (bottom > v2) {
+        return 1;
+      } else if (bottom < v2) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+
+    @Override
+    public void copy(int slot, int doc) {
+      values[slot] = currentReaderValues.floats(doc);
+    }
+
+    @Override
+    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
+      currentReaderValues = reader.getIndexValuesCache().getFloats(field);
+      return this;
+    }
+    
+    @Override
+    public void setBottom(final int bottom) {
+      this.bottom = values[bottom];
+    }
+
+    @Override
+    public Comparable<Double> value(int slot) {
+      return Double.valueOf(values[slot]);
+    }
+  }
+
   /** Parses field's values as float (using {@link
    *  FieldCache#getFloats} and sorts by ascending value */
   public static final class FloatComparator extends NumericComparator<FloatValues> {
@@ -448,6 +507,69 @@ public abstract class FieldComparator {
     }
   }
 
+  /** Loads int index values and sorts by ascending value. */
+  public static final class IntIndexValuesComparator extends FieldComparator {
+    private final long[] values;
+    private Source currentReaderValues;
+    private final String field;
+    private long bottom;
+
+    IntIndexValuesComparator(int numHits, String field) {
+      values = new long[numHits];
+      this.field = field;
+    }
+
+    @Override
+    public int compare(int slot1, int slot2) {
+      // TODO: there are sneaky non-branch ways to compute
+      // -1/+1/0 sign
+      final long v1 = values[slot1];
+      final long v2 = values[slot2];
+      if (v1 > v2) {
+        return 1;
+      } else if (v1 < v2) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+
+    @Override
+    public int compareBottom(int doc) {
+      // TODO: there are sneaky non-branch ways to compute
+      // -1/+1/0 sign
+      final long v2 = currentReaderValues.ints(doc);
+      if (bottom > v2) {
+        return 1;
+      } else if (bottom < v2) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+
+    @Override
+    public void copy(int slot, int doc) {
+      values[slot] = currentReaderValues.ints(doc);
+    }
+
+    @Override
+    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
+      currentReaderValues = reader.getIndexValuesCache().getInts(field);
+      return this;
+    }
+    
+    @Override
+    public void setBottom(final int bottom) {
+      this.bottom = values[bottom];
+    }
+
+    @Override
+    public Comparable<Long> value(int slot) {
+      return Long.valueOf(values[slot]);
+    }
+  }
+
   /** Parses field's values as long (using {@link
    *  FieldCache#getLongs} and sorts by ascending value */
   public static final class LongComparator extends NumericComparator<LongValues> {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java Tue Oct 12 06:15:03 2010
@@ -23,7 +23,7 @@ import java.io.IOException;
 /** A Scorer for queries with a required subscorer
  * and an excluding (prohibited) sub DocIdSetIterator.
  * <br>
- * This <code>Scorer</code> implements {@link Scorer#skipTo(int)},
+ * This <code>Scorer</code> implements {@link Scorer#advance(int)},
  * and it uses the skipTo() on the given scorers.
  */
 class ReqExclScorer extends Scorer {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java Tue Oct 12 06:15:03 2010
@@ -21,7 +21,7 @@ import java.io.IOException;
 /** A Scorer for queries with a required part and an optional part.
  * Delays skipTo() on the optional part until a score() is needed.
  * <br>
- * This <code>Scorer</code> implements {@link Scorer#skipTo(int)}.
+ * This <code>Scorer</code> implements {@link Scorer#advance(int)}.
  */
 class ReqOptSumScorer extends Scorer {
   /** The scorers passed from the constructor.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java Tue Oct 12 06:15:03 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Comparator;
 import java.util.Locale;
 
 import org.apache.lucene.search.cache.ByteValuesCreator;
@@ -29,6 +30,11 @@ import org.apache.lucene.search.cache.In
 import org.apache.lucene.search.cache.LongValuesCreator;
 import org.apache.lucene.search.cache.ShortValuesCreator;
 import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.BytesRef;
+
+// nocommit -- for cleaner transition, maybe we should make
+// a new SortField that subclasses this one and always uses
+// index values?
 
 /**
  * Stores information about how to sort documents by terms in an individual
@@ -90,6 +96,9 @@ implements Serializable {
    * uses ordinals to do the sorting. */
   public static final int STRING_VAL = 11;
   
+  /** Sort use byte[] index values. */
+  public static final int BYTES = 12;
+  
   /** Represents sorting by document score (relevancy). */
   public static final SortField FIELD_SCORE = new SortField (null, SCORE);
 
@@ -440,6 +449,26 @@ implements Serializable {
       field = StringHelper.intern(field);
   }
 
+  private boolean useIndexValues;
+
+  public void setUseIndexValues(boolean b) {
+    useIndexValues = b;
+  }
+
+  public boolean getUseIndexValues() {
+    return useIndexValues;
+  }
+
+  private Comparator<BytesRef> bytesComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
+
+  public void setBytesComparator(Comparator<BytesRef> b) {
+    bytesComparator = b;
+  }
+
+  public Comparator<BytesRef> getBytesComparator() {
+    return bytesComparator;
+  }
+
   /** Returns the {@link FieldComparator} to use for
    * sorting.
    *
@@ -469,10 +498,18 @@ implements Serializable {
       return new FieldComparator.DocComparator(numHits);
 
     case SortField.INT:
-      return new FieldComparator.IntComparator(numHits, (IntValuesCreator)creator, (Integer)missingValue );
+      if (useIndexValues) {
+        return new FieldComparator.IntIndexValuesComparator(numHits, field);
+      } else {
+        return new FieldComparator.IntComparator(numHits, (IntValuesCreator)creator, (Integer) missingValue);
+      }
 
     case SortField.FLOAT:
-      return new FieldComparator.FloatComparator(numHits, (FloatValuesCreator)creator, (Float)missingValue );
+      if (useIndexValues) {
+        return new FieldComparator.FloatIndexValuesComparator(numHits, field);
+      } else {
+        return new FieldComparator.FloatComparator(numHits, (FloatValuesCreator) creator, (Float) missingValue);
+      }
 
     case SortField.LONG:
       return new FieldComparator.LongComparator(numHits, (LongValuesCreator)creator, (Long)missingValue );

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Tue Oct 12 06:15:03 2010
@@ -247,6 +247,19 @@ public final class ArrayUtil {
   public static short[] grow(short[] array) {
     return grow(array, 1 + array.length);
   }
+  
+  public static double[] grow(double[] array, int minSize) {
+    if (array.length < minSize) {
+      double[] newArray = new double[oversize(minSize, RamUsageEstimator.NUM_BYTES_DOUBLE)];
+      System.arraycopy(array, 0, newArray, 0, array.length);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static double[] grow(double[] array) {
+    return grow(array, 1 + array.length);
+  }
 
   public static short[] shrink(short[] array, int targetSize) {
     final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_SHORT);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java Tue Oct 12 06:15:03 2010
@@ -62,6 +62,18 @@ public final class ByteBlockPool {
       return new byte[blockSize];
     }
   }
+  
+  public static class DirectAllocator extends Allocator {
+
+    public DirectAllocator(int blockSize) {
+      super(blockSize);
+    }
+
+    @Override
+    public void recycleByteBlocks(byte[][] blocks, int start, int end) {
+    }
+    
+  }
 
   public byte[][] buffers = new byte[10][];
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java Tue Oct 12 06:15:03 2010
@@ -18,6 +18,7 @@ package org.apache.lucene.util;
  */
 
 import java.util.Comparator;
+import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
@@ -271,12 +272,13 @@ public final class BytesRef implements C
   }
 
   private final static Comparator<BytesRef> utf8SortedAsUnicodeSortOrder = new UTF8SortedAsUnicodeComparator();
-
+  
   public static Comparator<BytesRef> getUTF8SortedAsUnicodeComparator() {
     return utf8SortedAsUnicodeSortOrder;
   }
 
-  private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
+  @SuppressWarnings("serial") // serializable to work with contrib/remote
+  private static final class UTF8SortedAsUnicodeComparator implements Serializable, Comparator<BytesRef> {
     // Only singleton
     private UTF8SortedAsUnicodeComparator() {};
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Tue Oct 12 06:15:03 2010
@@ -353,6 +353,7 @@ public final class BytesRefHash {
         // 1 byte to store length
         buffer[bufferUpto] = (byte) length;
         pool.byteUpto += length + 1;
+        assert length >= 0: "Length must be positive: " + length;
         System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 1,
             length);
       } else {
@@ -569,4 +570,64 @@ public final class BytesRefHash {
     }
 
   }
+  
+  public static class ParallelBytesStartArray<T extends ParallelArrayBase<T>> extends BytesStartArray {
+    private final T prototype;
+    public T array;
+    
+    public ParallelBytesStartArray(T template) {
+      this.prototype = template;
+    }
+    @Override
+    public int[] init() {
+      if(array == null) { 
+        array = prototype.newInstance(2);
+      }
+      return array.textStart;
+    }
+
+    @Override
+    public int[] grow() {
+      array = array.grow();
+      return array.textStart;
+    }
+
+    @Override
+    public int[] clear() {
+      if(array != null) {
+        array.deref();
+        array = null;
+      }
+      return null;
+    }
+
+    @Override
+    public AtomicLong bytesUsed() {
+      return array.bytesUsed();
+    }
+    
+  }
+  
+  public abstract static class ParallelArrayBase<T extends ParallelArrayBase<T>> extends ParallelArray<T> {
+    final int[] textStart;
+    
+    protected ParallelArrayBase(int size, AtomicLong bytesUsed) {
+      super(size, bytesUsed);
+      textStart = new int[size];
+    }
+
+    @Override
+    protected int bytesPerEntry() {
+      return RamUsageEstimator.NUM_BYTES_INT;
+    }
+
+    @Override
+    protected void copyTo(T toArray, int numToCopy) {
+      System.arraycopy(textStart, 0, toArray.textStart, 0, size);
+    }
+
+    @Override
+    public abstract T newInstance(int size);
+    
+  }
 }

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/FloatsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/FloatsRef.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/FloatsRef.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/FloatsRef.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,91 @@
+/**
+ * 
+ */
+package org.apache.lucene.util;
+
+
+public final class FloatsRef implements Cloneable{
+  public double[] floats;
+  public int offset;
+  public int length;
+
+  public FloatsRef() {
+  }
+
+  public FloatsRef(int capacity) {
+    floats = new double[capacity];
+  }
+  
+  public void set(double value) {
+    floats[offset] = value;
+  }
+  
+  public double get() {
+    return floats[offset];
+  }
+
+  public FloatsRef(double[] floats, int offset, int length) {
+    this.floats = floats;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  public FloatsRef(FloatsRef other) {
+    copy(other);
+  }
+
+  @Override
+  public Object clone() {
+    return new FloatsRef(this);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 0;
+    final int end = offset + length;
+    for(int i = offset; i < end; i++) {
+      long value = Double.doubleToLongBits(floats[i]);
+      result = prime * result + (int) (value ^ (value >>> 32));
+    }
+    return result;
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    return other instanceof FloatsRef && this.floatsEquals((FloatsRef) other);
+  }
+
+  public boolean floatsEquals(FloatsRef other) {
+    if (length == other.length) {
+      int otherUpto = other.offset;
+      final double[] otherFloats = other.floats;
+      final int end = offset + length;
+      for(int upto=offset;upto<end;upto++,otherUpto++) {
+        if (floats[upto] != otherFloats[otherUpto]) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public void copy(FloatsRef other) {
+    if (floats == null) {
+      floats = new double[other.length];
+    } else {
+      floats = ArrayUtil.grow(floats, other.length);
+    }
+    System.arraycopy(other.floats, other.offset, floats, 0, other.length);
+    length = other.length;
+    offset = 0;
+  }
+
+  public void grow(int newLength) {
+    if (floats.length < newLength) {
+      floats = ArrayUtil.grow(floats, newLength);
+    }
+  }
+}
\ No newline at end of file

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/FloatsRef.java
------------------------------------------------------------------------------
    svn:eol-style = native

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

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java Tue Oct 12 06:15:03 2010
@@ -20,6 +20,8 @@ package org.apache.lucene.util;
 import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.lucene.store.DataOutput;
+
 /** @lucene.internal */
 public class IOUtils {
   /**
@@ -61,4 +63,32 @@ public class IOUtils {
     else if (firstIOE != null)
       throw firstIOE;
   }
+  
+  /**
+   * Writes the length of the {@link BytesRef} as either a one or two bytes to
+   * the {@link DataOutput} and returns the number of bytes used.
+   * 
+   * @param datOut
+   *          the output to write to
+   * @param bytes
+   *          the length to write
+   * @return the length of the {@link BytesRef} as either a one or two bytes to
+   *         the {@link DataOutput} and returns the number of bytes used.
+   * @throws IOException
+   *           if datOut throws an {@link IOException}
+   */
+  public static int writeLength(DataOutput datOut, BytesRef bytes)
+      throws IOException {
+    final int length = bytes.length;
+    if (length < 128) {
+      // 1 byte to store length
+      datOut.writeByte((byte) length);
+      return 1;
+    } else {
+      // 2 byte to store length
+      datOut.writeByte((byte) (0x80 | (length & 0x7f)));
+      datOut.writeByte((byte) ((length >> 7) & 0xff));
+      return 2;
+    }
+  }
 }

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/LongsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/LongsRef.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/LongsRef.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/LongsRef.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,91 @@
+/**
+ * 
+ */
+package org.apache.lucene.util;
+
+
+public final class LongsRef implements Cloneable {
+  public long[] ints;
+  public int offset;
+  public int length;
+
+  public LongsRef() {
+  }
+
+  public LongsRef(int capacity) {
+    ints = new long[capacity];
+  }
+
+  public LongsRef(long[] ints, int offset, int length) {
+    this.ints = ints;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  public LongsRef(LongsRef other) {
+    copy(other);
+  }
+
+  @Override
+  public Object clone() {
+    return new LongsRef(this);
+  }
+  
+  public void set(long value) {
+    ints[offset] = value;
+  }
+  
+  public long get() {
+    return ints[offset];
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 0;
+    final int end = offset + length;
+    for(int i = offset; i < end; i++) {
+      long value = ints[i];
+      result = prime * result + (int) (value ^ (value >>> 32));
+    }
+    return result;
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    return this.intsEquals((LongsRef) other);
+  }
+
+  public boolean intsEquals(LongsRef other) {
+    if (length == other.length) {
+      int otherUpto = other.offset;
+      final long[] otherInts = other.ints;
+      final int end = offset + length;
+      for(int upto=offset;upto<end;upto++,otherUpto++) {
+        if (ints[upto] != otherInts[otherUpto]) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public void copy(LongsRef other) {
+    if (ints == null) {
+      ints = new long[other.length];
+    } else {
+      ints = ArrayUtil.grow(ints, other.length);
+    }
+    System.arraycopy(other.ints, other.offset, ints, 0, other.length);
+    length = other.length;
+    offset = 0;
+  }
+
+  public void grow(int newLength) {
+    if (ints.length < newLength) {
+      ints = ArrayUtil.grow(ints, newLength);
+    }
+  }
+}
\ No newline at end of file

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/LongsRef.java
------------------------------------------------------------------------------
    svn:eol-style = native

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

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ParallelArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ParallelArray.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ParallelArray.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ParallelArray.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,57 @@
+package org.apache.lucene.util;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * 
+ * @lucene.internal
+ */
+public abstract class ParallelArray<T extends ParallelArray<?>> {
+
+    public final int size;
+    protected final AtomicLong bytesUsed;
+
+    protected ParallelArray(final int size, AtomicLong bytesUsed) {
+      this.size = size;
+      this.bytesUsed = bytesUsed;
+      bytesUsed.addAndGet((size) * bytesPerEntry());
+
+    }
+
+    protected abstract int bytesPerEntry();
+    
+    public AtomicLong bytesUsed() {
+      return bytesUsed;
+    }
+    
+    public void deref() {
+      bytesUsed.addAndGet((-size) * bytesPerEntry());
+    }
+
+    public abstract T newInstance(int size);
+    
+    public final T grow() {
+      int newSize = ArrayUtil.oversize(size + 1, bytesPerEntry());
+      T newArray = newInstance(newSize);
+      copyTo(newArray, size);
+      bytesUsed.addAndGet((newSize - size) * bytesPerEntry());
+      return newArray;
+    }
+
+    protected abstract void copyTo(T toArray, int numToCopy);
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ParallelArray.java
------------------------------------------------------------------------------
    svn:eol-style = native

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

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/packed/Packed64.java Tue Oct 12 06:15:03 2010
@@ -182,7 +182,7 @@ class Packed64 extends PackedInts.Reader
     final int bitPos =     (int)(majorBitPos & MOD_MASK); // % BLOCK_SIZE);
 
     final int base = bitPos * FAC_BITPOS;
-
+    assert elementPos < blocks.length : "elementPos: " + elementPos + "; blocks.len: " + blocks.length;
     return ((blocks[elementPos] << shifts[base]) >>> shifts[base+1]) |
             ((blocks[elementPos+1] >>> shifts[base+2]) & readMasks[bitPos]);
   }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Tue Oct 12 06:15:03 2010
@@ -47,7 +47,7 @@ public class TestIndexWriterConfig exten
     // Does not implement anything - used only for type checking on IndexWriterConfig.
 
     @Override
-    DocConsumer getChain(DocumentsWriter documentsWriter) {
+    public DocConsumer getChain(DocumentsWriter documentsWriter) {
       return null;
     }
     

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java Tue Oct 12 06:15:03 2010
@@ -61,7 +61,7 @@ final class TermInfosWriter {
   int indexInterval = 128;
 
   /** Expert: The fraction of {@link TermDocs} entries stored in skip tables,
-   * used to accelerate {@link TermDocs#skipTo(int)}.  Larger values result in
+   * used to accelerate {@link TermDocs#advance(int)}.  Larger values result in
    * smaller indexes, greater acceleration, but fewer accelerable cases, while
    * smaller values result in bigger indexes, less acceleration and more
    * accelerable cases. More detailed experiments would be useful here. */

Added: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,658 @@
+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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.document.ValuesField;
+import org.apache.lucene.document.Field.Index;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LogDocMergePolicy;
+import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.values.Reader.SortedSource;
+import org.apache.lucene.index.values.Reader.Source;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FloatsRef;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util._TestUtil;
+
+public class TestIndexValues extends LuceneTestCase {
+
+  public void testBytesStraight() throws IOException {
+    runTestBytes(Bytes.Mode.STRAIGHT, true);
+    runTestBytes(Bytes.Mode.STRAIGHT, false);
+  }
+
+  public void testBytesDeref() throws IOException {
+    runTestBytes(Bytes.Mode.DEREF, true);
+    runTestBytes(Bytes.Mode.DEREF, false);
+  }
+
+  public void testBytesSorted() throws IOException {
+    runTestBytes(Bytes.Mode.SORTED, true);
+    runTestBytes(Bytes.Mode.SORTED, false);
+  }
+
+  // nocommit -- for sorted test, do our own Sort of the
+  // values and verify it's identical
+  public void runTestBytes(final Bytes.Mode mode,
+      final boolean fixedSize) throws IOException {
+
+    final BytesRef bytesRef = new BytesRef();
+
+    final Comparator<BytesRef> comp = mode == Bytes.Mode.SORTED ? BytesRef
+        .getUTF8SortedAsUnicodeComparator()
+        : null;
+
+    Directory dir = newDirectory();
+    Writer w = Bytes
+        .getWriter(dir, "test", mode, comp, fixedSize);
+    int maxDoc = 220;
+    final String[] values = new String[maxDoc];
+    final int lenMin, lenMax;
+    if (fixedSize) {
+      lenMin = lenMax = 3 + random.nextInt(7);
+    } else {
+      lenMin = 1;
+      lenMax = 15 + random.nextInt(6);
+    }
+    for (int i = 0; i < 100; i++) {
+      final String s;
+      if (i > 0 && random.nextInt(5) <= 2) {
+        // use prior value
+        s = values[2 * random.nextInt(i)];
+      } else {
+        s = _TestUtil.randomUnicodeString(random, lenMin, lenMax);
+      }
+      values[2 * i] = s;
+
+      UnicodeUtil.UTF16toUTF8(s, 0, s.length(), bytesRef);
+      w.add(2 * i, bytesRef);
+    }
+    w.finish(maxDoc);
+
+    Reader r = Bytes.getReader(dir, "test", mode, fixedSize, maxDoc);
+    for (int iter = 0; iter < 2; iter++) {
+      ValuesEnum bytesEnum = r.getEnum();
+      assertNotNull("enum is null", bytesEnum);
+      ValuesAttribute attr = bytesEnum.addAttribute(ValuesAttribute.class);
+      assertNotNull("attribute is null", attr);
+      BytesRef ref = attr.bytes();
+      assertNotNull("BytesRef is null - enum not initialized to use bytes", attr);
+
+      for (int i = 0; i < 2; i++) {
+        final int idx = 2 * i;
+        assertEquals("doc: " + idx, idx, bytesEnum.advance(idx));
+        String utf8String = ref.utf8ToString();
+        assertEquals("doc: " + idx + " lenLeft: " + values[idx].length() + " lenRight: " + utf8String.length() , values[idx],  utf8String);
+      }
+      assertEquals(ValuesEnum.NO_MORE_DOCS, bytesEnum.advance(maxDoc));
+      assertEquals(ValuesEnum.NO_MORE_DOCS, bytesEnum.advance(maxDoc+1));
+
+      bytesEnum.close();
+    }
+    
+    
+    // Verify we can load source twice:
+    for (int iter = 0; iter < 2; iter++) {
+      Source s;
+      Reader.SortedSource ss;
+      if (mode == Bytes.Mode.SORTED) {
+        s = ss = r.loadSorted(comp);
+      } else {
+        s = r.load();
+        ss = null;
+      }
+
+      for (int i = 0; i < 100; i++) {
+        final int idx = 2 * i;
+        assertNotNull("doc " + idx + "; value=" + values[idx], s.bytes(idx));
+        assertEquals("doc " + idx, values[idx], s.bytes(idx).utf8ToString());
+        if (ss != null) {
+          assertEquals("doc " + idx, values[idx], ss.getByOrd(ss.ord(idx))
+              .utf8ToString());
+          Reader.SortedSource.LookupResult result = ss.getByValue(new BytesRef(
+              values[idx]));
+          assertTrue(result.found);
+          assertEquals(ss.ord(idx), result.ord);
+        }
+      }
+
+      // Lookup random strings:
+      if (mode == Bytes.Mode.SORTED) {
+        final int numValues = ss.getValueCount();
+        for (int i = 0; i < 1000; i++) {
+          BytesRef bytesValue = new BytesRef(_TestUtil.randomUnicodeString(
+              random, lenMin, lenMax));
+          SortedSource.LookupResult result = ss.getByValue(bytesValue);
+          if (result.found) {
+            assert result.ord > 0;
+            assertTrue(bytesValue.bytesEquals(ss.getByOrd(result.ord)));
+            int count = 0;
+            for (int k = 0; k < 100; k++) {
+              if (bytesValue.utf8ToString().equals(values[2 * k])) {
+                assertEquals(ss.ord(2 * k), result.ord);
+                count++;
+              }
+            }
+            assertTrue(count > 0);
+          } else {
+            assert result.ord >= 0;
+            if (result.ord == 0) {
+              final BytesRef firstRef = ss.getByOrd(1);
+              // random string was before our first
+              assertTrue(firstRef.compareTo(bytesValue) > 0);
+            } else if (result.ord == numValues) {
+              final BytesRef lastRef = ss.getByOrd(numValues);
+              // random string was after our last
+              assertTrue(lastRef.compareTo(bytesValue) < 0);
+            } else {
+              // random string fell between two of our values
+              final BytesRef before = (BytesRef) ss.getByOrd(result.ord)
+                  .clone();
+              final BytesRef after = ss.getByOrd(result.ord + 1);
+              assertTrue(before.compareTo(bytesValue) < 0);
+              assertTrue(bytesValue.compareTo(after) < 0);
+
+            }
+          }
+        }
+      }
+    }
+
+    r.close();
+    dir.close();
+  }
+
+  public void testInts() throws IOException {
+    long maxV = 1;
+    final int NUM_VALUES = 1000;
+    final long[] values = new long[NUM_VALUES];
+    for (int rx = 1; rx < 63; rx++, maxV *= 2) {
+      for (int b = 0; b < 2; b++) {
+        Directory dir = newDirectory();
+        boolean useFixedArrays = b == 0;
+        Writer w = Ints.getWriter(dir, "test", useFixedArrays);
+        for (int i = 0; i < NUM_VALUES; i++) {
+          final long v = random.nextLong() % (1 + maxV);
+          values[i] = v;
+          w.add(i, v);
+        }
+        final int additionalDocs = 1 + random.nextInt(9);
+        w.finish(NUM_VALUES + additionalDocs);
+
+        Reader r = Ints.getReader(dir, "test", useFixedArrays);
+        for (int iter = 0; iter < 2; iter++) {
+          Source s = r.load();
+          for (int i = 0; i < NUM_VALUES; i++) {
+            final long v = s.ints(i);
+            assertEquals("index " + i + " b: " + b, values[i], v);
+          }
+        }
+
+        for (int iter = 0; iter < 2; iter++) {
+          ValuesEnum iEnum = r.getEnum();
+          ValuesAttribute attr = iEnum.addAttribute(ValuesAttribute.class);
+          LongsRef ints = attr.ints();
+          for (int i = 0; i < NUM_VALUES; i++) {
+            assertEquals(i, iEnum.nextDoc());
+            assertEquals(values[i], ints.get());
+          }
+          for (int i = NUM_VALUES; i < NUM_VALUES + additionalDocs; i++) {
+            assertEquals(i, iEnum.nextDoc());
+            assertEquals("" + i, 0, ints.get());
+          }
+
+          iEnum.close();
+        }
+
+        for (int iter = 0; iter < 2; iter++) {
+          ValuesEnum iEnum = r.getEnum();
+          ValuesAttribute attr = iEnum.addAttribute(ValuesAttribute.class);
+          LongsRef ints = attr.ints();
+          for (int i = 0; i < NUM_VALUES; i += 1 + random.nextInt(25)) {
+            assertEquals(i, iEnum.advance(i));
+            assertEquals(values[i], ints.get());
+          }
+          for (int i = NUM_VALUES; i < NUM_VALUES + additionalDocs; i++) {
+            assertEquals(i, iEnum.advance(i));
+            assertEquals("" + i, 0, ints.get());
+          }
+          
+          iEnum.close();
+        }
+        r.close();
+        dir.close();
+      }
+    }
+  }
+
+  public void testFloats4() throws IOException {
+    runTestFloats(4, 0.00001);
+  }
+
+  private void runTestFloats(int precision, double delta)
+      throws IOException {
+    Directory dir = newDirectory();
+    Writer w = Floats.getWriter(dir, "test", precision);
+    final int NUM_VALUES = 1000;
+    final double[] values = new double[NUM_VALUES];
+    for (int i = 0; i < NUM_VALUES; i++) {
+      final double v = precision == 4 ? random.nextFloat() : random.nextDouble();
+      values[i] = v;
+      w.add(i, v);
+    }
+    final int additionalValues = 1 + random.nextInt(10);
+    w.finish(NUM_VALUES + additionalValues);
+
+    Reader r = Floats.getReader(dir, "test", NUM_VALUES
+        + additionalValues);
+    for (int iter = 0; iter < 2; iter++) {
+      Source s = r.load();
+      for (int i = 0; i < NUM_VALUES; i++) {
+        assertEquals(values[i], s.floats(i), 0.0f);
+      }
+    }
+
+    for (int iter = 0; iter < 2; iter++) {
+      ValuesEnum fEnum = r.getEnum();
+      ValuesAttribute attr = fEnum.addAttribute(ValuesAttribute.class);
+      FloatsRef floats = attr.floats();
+      for (int i = 0; i < NUM_VALUES; i++) {
+        assertEquals(i, fEnum.nextDoc());
+        assertEquals(values[i], floats.get(), delta);
+      }
+      for(int i = NUM_VALUES; i < NUM_VALUES + additionalValues; i++) {
+        assertEquals(i, fEnum.nextDoc());
+        assertEquals(0.0, floats.get(), delta);
+      }
+      fEnum.close();
+    }
+    for (int iter = 0; iter < 2; iter++) {
+      ValuesEnum fEnum = r.getEnum();
+      ValuesAttribute attr = fEnum.addAttribute(ValuesAttribute.class);
+      FloatsRef floats = attr.floats();
+      for (int i = 0; i < NUM_VALUES; i += 1 + random.nextInt(25)) {
+        assertEquals(i, fEnum.advance(i));
+        assertEquals(values[i], floats.get(), delta);
+      }
+      for(int i = NUM_VALUES; i < NUM_VALUES + additionalValues; i++) {
+        assertEquals(i, fEnum.advance(i));
+        assertEquals(0.0, floats.get(), delta);
+      }
+      fEnum.close();
+    }
+
+    r.close();
+    dir.close();
+  }
+
+  public void testFloats8() throws IOException {
+    runTestFloats(8, 0.0);
+  }
+
+  /**
+   * Tests complete indexing of {@link Values} including deletions, merging and
+   * sparse value fields on Compound-File
+   */
+  public void testCFSIndex() throws IOException {
+    // without deletions
+    IndexWriterConfig cfg = writerConfig(true);
+    // primitives - no deletes
+    runTestNumerics(cfg,false);
+
+    cfg = writerConfig(true);
+    // bytes - no deletes
+    runTestIndexBytes(cfg, false);
+
+    // with deletions
+    cfg = writerConfig(true);
+    // primitives
+    runTestNumerics(cfg, true);
+
+    cfg = writerConfig(true);
+    // bytes
+    runTestIndexBytes(cfg, true);
+  }
+
+  /**
+   * Tests complete indexing of {@link Values} including deletions, merging and
+   * sparse value fields on None-Compound-File
+   */
+  public void testIndex() throws IOException {
+    //
+    // without deletions
+    IndexWriterConfig cfg = writerConfig(false);
+    // primitives - no deletes
+    runTestNumerics(cfg, false);
+
+    cfg = writerConfig(false);
+    // bytes - no deletes
+    runTestIndexBytes(cfg, false);
+
+    // with deletions
+    cfg = writerConfig(false);
+    // primitives
+    runTestNumerics(cfg, true);
+
+    cfg = writerConfig(false);
+    // bytes
+    runTestIndexBytes(cfg, true);
+  }
+
+  private IndexWriterConfig writerConfig(boolean useCompoundFile) {
+    final IndexWriterConfig cfg = newIndexWriterConfig(
+        TEST_VERSION_CURRENT, new MockAnalyzer());
+    MergePolicy mergePolicy = cfg.getMergePolicy();
+    if(mergePolicy instanceof LogMergePolicy) {
+      ((LogMergePolicy)mergePolicy).setUseCompoundFile(useCompoundFile);
+    } else if(useCompoundFile) {
+      LogMergePolicy policy = new LogDocMergePolicy();
+      policy.setUseCompoundFile(useCompoundFile);
+      cfg.setMergePolicy(policy);
+    }
+    return cfg;
+  }
+
+  public void runTestNumerics(IndexWriterConfig cfg,
+      boolean withDeletions) throws IOException {
+    Directory d = newDirectory();
+    IndexWriter w = new IndexWriter(d, cfg);
+    final int numValues = 350;
+    final List<Values> numVariantList = new ArrayList<Values>(NUMERICS);
+
+    // run in random order to test if fill works correctly during merges
+    Collections.shuffle(numVariantList, random);
+    for (Values val : numVariantList) {
+      OpenBitSet deleted = indexValues(w, numValues, val, numVariantList,
+          withDeletions, 7);
+      List<Closeable> closeables = new ArrayList<Closeable>();
+      IndexReader r = IndexReader.open(w);
+      final int numRemainingValues = (int) (numValues - deleted.cardinality());
+      final int base = r.numDocs() - numRemainingValues;
+      switch (val) {
+      case PACKED_INTS:
+      case PACKED_INTS_FIXED: {
+        Reader intsReader = r.getIndexValues(val.name());
+        Source ints = intsReader.load();
+        ValuesEnum intsEnum = intsReader.getEnum();
+        assertNotNull(intsEnum);
+        LongsRef enumRef = intsEnum.addAttribute(ValuesAttribute.class).ints();
+        for (int i = 0; i < base; i++) {
+          assertEquals(0, ints.ints(i));
+          assertEquals(val.name() + " base: " + base + " index: " + i, i, random.nextBoolean()?intsEnum.advance(i): intsEnum.nextDoc());
+          assertEquals(0, enumRef.get());
+        }
+        int expected = 0;
+        for (int i = base; i < r.numDocs(); i++, expected++) {
+          while (deleted.get(expected)) {
+            expected++;
+          }
+          assertEquals("advance failed at index: " + i + " of " + r.numDocs() + " docs", i, intsEnum.advance(i));
+          assertEquals(expected, ints.ints(i));
+          assertEquals(expected, enumRef.get());
+
+        }
+      }
+        break;
+      case SIMPLE_FLOAT_4BYTE:
+      case SIMPLE_FLOAT_8BYTE: {
+        Reader floatReader = r.getIndexValues(val.name());
+        Source floats = floatReader.load();
+        ValuesEnum floatEnum = floatReader.getEnum();
+        assertNotNull(floatEnum);
+        FloatsRef enumRef = floatEnum.addAttribute(ValuesAttribute.class).floats();
+
+        for (int i = 0; i < base; i++) {
+          assertEquals(0.0d, floats.floats(i), 0.0d);
+          assertEquals(i, random.nextBoolean()?floatEnum.advance(i): floatEnum.nextDoc());
+          assertEquals("index " + i, 0.0 ,enumRef.get(), 0.0);
+        }
+        int expected = 0;
+        for (int i = base; i < r.numDocs(); i++, expected++) {
+          while (deleted.get(expected)) {
+            expected++;
+          }
+          assertEquals("advance failed at index: " + i + " of " + r.numDocs() + " docs base:" + base, i, floatEnum.advance(i));
+          assertEquals("index " + i, 2.0 * expected ,enumRef.get() , 0.00001);
+          assertEquals("index " + i, 2.0 * expected, floats.floats(i), 0.00001);
+        }
+      }
+        break;
+      default:
+        fail("unexpected value " + val);
+      }
+
+      closeables.add(r);
+      for (Closeable toClose : closeables) {
+        toClose.close();
+      }
+    }
+    w.close();
+    d.close();
+  }
+
+  private static EnumSet<Values> BYTES = EnumSet.of(
+      Values.BYTES_FIXED_DEREF,
+      Values.BYTES_FIXED_SORTED, 
+      Values.BYTES_FIXED_STRAIGHT,
+      Values.BYTES_VAR_DEREF ,
+      Values.BYTES_VAR_SORTED,
+      Values.BYTES_VAR_STRAIGHT
+      );
+  
+  private static EnumSet<Values> STRAIGHT_BYTES = EnumSet.of(
+      Values.BYTES_FIXED_STRAIGHT,
+      Values.BYTES_VAR_STRAIGHT
+      );
+
+  private static EnumSet<Values> NUMERICS = EnumSet.of(Values.PACKED_INTS, Values.PACKED_INTS_FIXED, Values.SIMPLE_FLOAT_4BYTE, Values.SIMPLE_FLOAT_8BYTE);
+  
+  private static Index[] IDX_VALUES = new Index[] { Index.ANALYZED, Index.ANALYZED_NO_NORMS, Index.NOT_ANALYZED, Index.NOT_ANALYZED_NO_NORMS};
+  private OpenBitSet indexValues(IndexWriter w, int numValues,
+      Values value, List<Values> valueVarList, boolean withDeletions,
+      int multOfSeven) throws CorruptIndexException, IOException {
+    final boolean isNumeric = NUMERICS.contains(value);
+    OpenBitSet deleted = new OpenBitSet(numValues);
+    Document doc = new Document();
+    Fieldable field = random.nextBoolean()? new ValuesField(value.name()):newField(value.name(), _TestUtil.randomRealisticUnicodeString(random, 10), IDX_VALUES[random.nextInt(IDX_VALUES.length)]);
+    doc.add(field);
+
+    ValuesAttribute valuesAttribute = ValuesField.values(field);
+    valuesAttribute.setType(value);
+    final LongsRef intsRef = valuesAttribute.ints();
+    final FloatsRef floatsRef = valuesAttribute.floats();
+    final BytesRef bytesRef = valuesAttribute.bytes();
+
+    final String idBase = value.name() + "_";
+    final byte[] b = new byte[multOfSeven];
+    if (bytesRef != null) {
+      bytesRef.bytes = b;
+      bytesRef.length = b.length;
+      bytesRef.offset = 0;
+    }
+    // 
+    byte upto = 0;
+    for (int i = 0; i < numValues; i++) {
+      if (isNumeric) {
+        switch (value) {
+        case PACKED_INTS:
+        case PACKED_INTS_FIXED:
+          intsRef.set(i);
+          break;
+        case SIMPLE_FLOAT_4BYTE:
+        case SIMPLE_FLOAT_8BYTE:
+          floatsRef.set(2.0f * i);
+          break;
+        default:
+          fail("unexpected value " + value);
+        }
+      } else {
+        for (int j = 0; j < b.length; j++) {
+          b[j] = upto++;
+        }
+      }
+      doc.removeFields("id");
+      doc.add(new Field("id", idBase + i, Store.YES,
+          Index.NOT_ANALYZED_NO_NORMS));
+      w.addDocument(doc);
+
+      if (i % 7 == 0) {
+        if (withDeletions && random.nextBoolean()) {
+          Values val = valueVarList.get(random.nextInt(1 + valueVarList
+              .indexOf(value)));
+          final int randInt = val == value ? random.nextInt(1 + i) : random
+              .nextInt(numValues);
+          w.deleteDocuments(new Term("id", val.name() + "_" + randInt));
+          if (val == value) {
+            deleted.set(randInt);
+          }
+        }
+        w.commit();
+
+      }
+    }
+    w.commit();
+    
+    // nocommit test unoptimized with deletions
+    if(withDeletions || random.nextBoolean())
+      w.optimize();
+    return deleted;
+  }
+
+  public void runTestIndexBytes(IndexWriterConfig cfg,
+      boolean withDeletions) throws CorruptIndexException,
+      LockObtainFailedException, IOException {
+    Directory d = newDirectory();
+    IndexWriter w = new IndexWriter(d, cfg);
+    final List<Values> byteVariantList = new ArrayList<Values>(BYTES);
+
+    // run in random order to test if fill works correctly during merges
+    Collections.shuffle(byteVariantList, random);
+    final int numValues = 350;
+    for (Values byteIndexValue : byteVariantList) {
+      List<Closeable> closeables = new ArrayList<Closeable>();
+
+      int bytesSize = 7 + random.nextInt(128);
+      OpenBitSet deleted = indexValues(w, numValues, byteIndexValue,
+          byteVariantList, withDeletions, bytesSize);
+      final IndexReader r = IndexReader.open(w);
+      assertEquals(0, r.numDeletedDocs());
+      final int numRemainingValues = (int) (numValues - deleted.cardinality());
+      final int base = r.numDocs() - numRemainingValues;
+
+      Reader bytesReader = r.getIndexValues(byteIndexValue.name());
+//      closeables.add(bytesReader);
+      assertNotNull("field " + byteIndexValue.name()
+          + " returned null reader - maybe merged failed", bytesReader);
+      Source bytes = bytesReader.load();
+      ValuesEnum bytesEnum = bytesReader.getEnum();
+      assertNotNull(bytesEnum);
+      final ValuesAttribute attr = bytesEnum.addAttribute(ValuesAttribute.class);
+      byte upto = 0;
+      // test the filled up slots for correctness
+      for (int i = 0; i < base; i++) {
+        final BytesRef br = bytes.bytes(i);
+        String msg = " field: " + byteIndexValue.name() + " at index: " + i
+            + " base: " + base + " numDocs:" + r.numDocs();
+        switch (byteIndexValue) {
+        case BYTES_VAR_STRAIGHT:
+        case BYTES_FIXED_STRAIGHT:
+          assertEquals(i, bytesEnum.advance(i));
+          // fixed straight returns bytesref with zero bytes all of fixed
+          // length
+          assertNotNull("expected none null - " + msg, br);
+          if(br.length != 0) {
+            assertEquals("expected zero bytes of length " + bytesSize + " - "
+                + msg, bytesSize, br.length);
+            for (int j = 0; j < br.length; j++) {
+              assertEquals("Byte at index " + j + " doesn't match - " + msg, 0,
+                  br.bytes[br.offset + j]);
+            }
+          }
+          break;
+        case BYTES_VAR_SORTED:
+        case BYTES_FIXED_SORTED:
+        case BYTES_VAR_DEREF:
+        case BYTES_FIXED_DEREF:
+        default:
+          assertNotNull("expected none null - " + msg, br);
+          if(br.length != 0){
+            bytes.bytes(i);
+          }
+          assertEquals("expected empty bytes - " + br.utf8ToString() + msg, 0, br.length);
+        }
+      }
+      final BytesRef enumRef = attr.bytes();
+
+     
+      // test the actual doc values added in this iteration
+      assertEquals(base + numRemainingValues, r.numDocs());
+      int v = 0;
+      for (int i = base; i < r.numDocs(); i++) {
+        
+        String msg = " field: " + byteIndexValue.name() + " at index: " + i
+            + " base: " + base + " numDocs:" + r.numDocs() + " bytesSize: " + bytesSize;
+        while (withDeletions && deleted.get(v++)) {
+          upto += bytesSize;
+        }
+        
+        BytesRef br = bytes.bytes(i);
+        if(bytesEnum.docID() != i)
+          assertEquals("seek failed for index " + i + " " + msg, i, bytesEnum.advance(i));
+        for (int j = 0; j < br.length; j++, upto++) {
+          assertEquals("EnumRef Byte at index " + j + " doesn't match - " + msg,
+              upto, enumRef.bytes[enumRef.offset + j]);      
+          assertEquals("SourceRef Byte at index " + j + " doesn't match - " + msg,
+              upto, br.bytes[br.offset + j]);
+           }
+      }
+
+      // clean up
+      closeables.add(r);
+      for (Closeable toClose : closeables) {
+        toClose.close();
+      }
+    }
+    
+    w.close();
+    d.close();
+  }
+  
+}

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

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

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java Tue Oct 12 06:15:03 2010
@@ -117,6 +117,37 @@ public class _TestUtil {
     }
     return new String(buffer, 0, end);
   }
+  
+  public static String randomUnicodeString(Random r, int minLength, int maxLength) {
+    if(minLength > maxLength)
+      throw new IllegalArgumentException("minLength must be >= maxLength");
+    final boolean lenEqual = minLength==maxLength;
+    final int end = lenEqual?minLength:minLength + r.nextInt(maxLength-minLength+1);
+    if (end == 0) {
+      // allow 0 length
+      return "";
+    }
+    
+    // TODO(simonw): check this
+    final int fixedPlane = 5;//minLength % 5;
+    final char[] buffer = new char[end];
+    for (int i = 0; i < end; i++) {
+      int t = lenEqual? fixedPlane: r.nextInt(5);
+      //buffer[i] = (char) (97 + r.nextInt(26));
+      if (0 == t && i < end - 1 && !lenEqual) {
+        // Make a surrogate pair
+        // High surrogate
+        buffer[i++] = (char) nextInt(r, 0xd800, 0xdbff);
+        // Low surrogate
+        buffer[i] = (char) nextInt(r, 0xdc00, 0xdfff);
+      }
+      else if (t <= 1) buffer[i] = (char) r.nextInt(0x80);
+      else if (2 == t) buffer[i] = (char) nextInt(r, 0x80, 0x800);
+      else if (3 == t) buffer[i] = (char) nextInt(r, 0x800, 0xd7ff);
+      else if (4 == t) buffer[i] = (char) nextInt(r, 0xe000, 0xffff);
+    }
+    return new String(buffer, 0, end);
+  }
 
   private static final int[] blockStarts = {
     0x0000, 0x0080, 0x0100, 0x0180, 0x0250, 0x02B0, 0x0300, 0x0370, 0x0400,