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 2012/11/16 15:45:13 UTC

svn commit: r1410376 - in /lucene/dev/branches/lucene4547/lucene: codecs/src/java/org/apache/lucene/codecs/simpletext/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/codecs/lucene41/ core/src/java/org/apache/lucene/codecs/lucen...

Author: simonw
Date: Fri Nov 16 14:45:10 2012
New Revision: 1410376

URL: http://svn.apache.org/viewvc?rev=1410376&view=rev
Log:
removed several no-commits and added basic lucene 41 binary & numeric consumer & producers

Added:
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41DocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SimpleDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SortedDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
      - copied, changed from r1410243, lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
      - copied, changed from r1410243, lucene/dev/branches/lucene4547/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefList.java
Removed:
    lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java
    lucene/dev/branches/lucene4547/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefList.java
Modified:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/BinaryDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/NumericDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SimpleDVConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SortedDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/BytesDVWriter.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java
    lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java
    lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java
    lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSimpleDocValuesFormat.java Fri Nov 16 14:45:10 2012
@@ -142,7 +142,7 @@ public class SimpleTextSimpleDocValuesFo
     }
 
     @Override
-    public NumericDocValuesConsumer addNumericField(FieldInfo field, final long minValue, long maxValue) throws IOException {
+    public NumericDocValuesConsumer addNumericField(FieldInfo field, final long minValue, long maxValue, final int numDocs) throws IOException {
       writeFieldEntry(field);
       
       // write our minimum value to the .dat, all entries are deltas from that
@@ -175,15 +175,14 @@ public class SimpleTextSimpleDocValuesFo
         }
 
         @Override
-        public void finish(FieldInfos fieldInfos, int numDocs) throws IOException {
+        public void finish() throws IOException {
           assert numDocs == numDocsWritten;
-          // nocommit: hopefully indexwriter is responsible for "filling" like it does stored fields!
         }
       };
     }
 
     @Override
-    public BinaryDocValuesConsumer addBinaryField(FieldInfo field, boolean fixedLength, final int maxLength) throws IOException {
+    public BinaryDocValuesConsumer addBinaryField(FieldInfo field, boolean fixedLength, final int maxLength, final int numDocs) throws IOException {
       writeFieldEntry(field);
       // write maxLength
       SimpleTextUtil.write(data, MAXLENGTH);
@@ -222,16 +221,15 @@ public class SimpleTextSimpleDocValuesFo
         }
 
         @Override
-        public void finish(FieldInfos fis, int numDocs) throws IOException {
+        public void finish() throws IOException {
           assert numDocs == numDocsWritten;
-          // nocommit: hopefully indexwriter is responsible for "filling" like it does stored fields!
         }
       };
     }
     
     // nocommit
     @Override
-    public SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, final int maxLength) throws IOException {
+    public SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, final int maxLength, final int numDocs) throws IOException {
       writeFieldEntry(field);
       // write numValues
       SimpleTextUtil.write(data, NUMVALUES);
@@ -290,6 +288,9 @@ public class SimpleTextSimpleDocValuesFo
           SimpleTextUtil.write(data, ordEncoder.format(ord), scratch);
           SimpleTextUtil.writeNewline(data);
         }
+
+        @Override
+        public void finish() throws IOException {}
       };
     }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/BinaryDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/BinaryDocValuesConsumer.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/BinaryDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/BinaryDocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -28,7 +28,7 @@ import org.apache.lucene.util.BytesRef;
 
 public abstract class BinaryDocValuesConsumer {
   public abstract void add(BytesRef value) throws IOException;
-  public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
+  public abstract void finish() throws IOException;
   
   public int merge(MergeState mergeState) throws IOException {
     int docCount = 0;
@@ -45,7 +45,7 @@ public abstract class BinaryDocValuesCon
         mergeState.checkAbort.work(300);
       }
     }
-    finish(mergeState.fieldInfos, docCount);
+    finish();
     return docCount;
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/NumericDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/NumericDocValuesConsumer.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/NumericDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/NumericDocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs;
 import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.DocValues.Source;
@@ -27,7 +28,7 @@ import org.apache.lucene.util.Bits;
 
 public abstract class NumericDocValuesConsumer {
   public abstract void add(long value) throws IOException;
-  public abstract void finish(FieldInfos fieldInfos, int numDocs) throws IOException;
+  public abstract void finish() throws IOException;
   
   public int merge(MergeState mergeState) throws IOException {
     int docCount = 0;
@@ -43,7 +44,7 @@ public abstract class NumericDocValuesCo
         mergeState.checkAbort.work(300);
       }
     }
-    finish(mergeState.fieldInfos, docCount);
+    finish();
     return docCount;
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SimpleDVConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SimpleDVConsumer.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SimpleDVConsumer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SimpleDVConsumer.java Fri Nov 16 14:45:10 2012
@@ -35,10 +35,10 @@ public abstract class SimpleDVConsumer i
   // we want codec to get necessary stuff from IW, but trading off against merge complexity.
 
   // nocommit should we pass SegmentWriteState...?
-  public abstract NumericDocValuesConsumer addNumericField(FieldInfo field, long minValue, long maxValue) throws IOException;
-  public abstract BinaryDocValuesConsumer addBinaryField(FieldInfo field, boolean fixedLength, int maxLength) throws IOException;
+  public abstract NumericDocValuesConsumer addNumericField(FieldInfo field, long minValue, long maxValue, int numDocs) throws IOException;
+  public abstract BinaryDocValuesConsumer addBinaryField(FieldInfo field, boolean fixedLength, int maxLength, int numDocs) throws IOException;
   // nocommit: figure out whats fair here.
-  public abstract SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, int maxLength) throws IOException;
+  public abstract SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, int maxLength, int numDocs) throws IOException;
   
   public void merge(MergeState mergeState) throws IOException {
     for (FieldInfo field : mergeState.fieldInfos) {
@@ -97,7 +97,7 @@ public abstract class SimpleDVConsumer i
       }
     }
     // now we can merge
-    NumericDocValuesConsumer field = addNumericField(mergeState.fieldInfo, minValue, maxValue);
+    NumericDocValuesConsumer field = addNumericField(mergeState.fieldInfo, minValue, maxValue, mergeState.segmentInfo.getDocCount());
     field.merge(mergeState);
   }
   
@@ -132,14 +132,14 @@ public abstract class SimpleDVConsumer i
     }
     // now we can merge
     assert maxLength >= 0; // could this happen (nothing to do?)
-    BinaryDocValuesConsumer field = addBinaryField(mergeState.fieldInfo, fixedLength, maxLength);
+    BinaryDocValuesConsumer field = addBinaryField(mergeState.fieldInfo, fixedLength, maxLength, mergeState.segmentInfo.getDocCount());
     field.merge(mergeState);
   }
 
   protected void mergeSortedField(MergeState mergeState) throws IOException {
     SortedDocValuesConsumer.Merger merger = new SortedDocValuesConsumer.Merger();
     merger.merge(mergeState);
-    SortedDocValuesConsumer consumer = addSortedField(mergeState.fieldInfo, merger.numMergedTerms, merger.fixedLength >= 0, merger.maxLength);
+    SortedDocValuesConsumer consumer = addSortedField(mergeState.fieldInfo, merger.numMergedTerms, merger.fixedLength >= 0, merger.maxLength, mergeState.segmentInfo.getDocCount());
     consumer.merge(mergeState, merger);
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SortedDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SortedDocValuesConsumer.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SortedDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/SortedDocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -39,6 +39,8 @@ public abstract class SortedDocValuesCon
   /** This is called once per document after all values are
    *  added. */
   public abstract void addDoc(int ord) throws IOException;
+  
+  public abstract void finish() throws IOException;
 
   public static class Merger {
 
@@ -194,5 +196,6 @@ public abstract class SortedDocValuesCon
   // nocommit why return int...?
   public void merge(MergeState mergeState, Merger merger) throws IOException {
     merger.finish(this);
+    this.finish();
   }
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Fri Nov 16 14:45:10 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.codecs.LiveDocs
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.SimpleDocValuesFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
@@ -34,6 +35,7 @@ import org.apache.lucene.codecs.lucene40
 import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
+import org.apache.lucene.codecs.lucene41.values.Lucene41SimpleDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
 /**
@@ -118,5 +120,12 @@ public class Lucene41Codec extends Codec
     return defaultFormat;
   }
   
+  private final SimpleDocValuesFormat simpleDocValuesFormat = new Lucene41SimpleDocValuesFormat();
+  
+  @Override
+  public SimpleDocValuesFormat simpleDocValuesFormat() {
+    return simpleDocValuesFormat;
+  }
+
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
 }

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesConsumer.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesConsumer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,85 @@
+package org.apache.lucene.codecs.lucene41.values;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.BinaryDocValuesConsumer;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingLongBuffer.Iterator;
+import org.apache.lucene.util.packed.PackedInts;
+
+/*
+ * 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.
+ */
+
+public class Lucene41BinaryDocValuesConsumer extends BinaryDocValuesConsumer {
+  
+  private final IndexOutput dataOut;
+  private final IndexOutput indexOut;
+  private final AppendingLongBuffer buffer;
+  private long offset = 0;
+  private long prevOffset = 0;
+  static final int VERSION_START = -1;
+  static final String CODEC_NAME = "Lucene41Binary";
+  static final int VALUE_SIZE_VAR = -1;
+
+  public Lucene41BinaryDocValuesConsumer(IndexOutput dataOut, IndexOutput indexOut, boolean fixedLength, int maxLength) throws IOException {
+    this.dataOut = dataOut;
+    this.indexOut = indexOut;
+    CodecUtil.writeHeader(dataOut, CODEC_NAME, VERSION_START);
+    dataOut.writeInt(fixedLength ? maxLength: VALUE_SIZE_VAR);
+    CodecUtil.writeHeader(indexOut, CODEC_NAME, VERSION_START);
+    buffer = fixedLength ? null : new AppendingLongBuffer();
+  }
+  
+  @Override
+  public void finish() throws IOException {
+    try {
+      indexOut.writeLong(offset);
+      if (buffer != null) {
+        Iterator iterator = buffer.iterator();
+        PackedInts.Writer writer = PackedInts.getWriter(indexOut,
+            buffer.size() + 1, PackedInts.bitsRequired(offset),
+            PackedInts.FASTEST);
+        long previous = 0;
+        while (iterator.hasNext()) {
+          long next = iterator.next() + previous;
+          previous = next;
+          writer.add(next);
+        }
+        writer.add(offset);
+        writer.finish();
+      }
+    } finally {
+      IOUtils.close(indexOut, dataOut);
+    }
+  }
+
+  @Override
+  public void add(BytesRef value) throws IOException {
+    dataOut.writeBytes(value.bytes, value.offset, value.length);
+    if (buffer != null) {
+      buffer.add(offset-prevOffset);
+      prevOffset = offset;
+    }
+    offset += value.length;
+  }
+  
+}

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesProducer.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesProducer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41BinaryDocValuesProducer.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,216 @@
+package org.apache.lucene.codecs.lucene41.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.codecs.CodecUtil;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.packed.PackedInts;
+
+public class Lucene41BinaryDocValuesProducer extends DocValues {
+  
+  private final IndexInput dataIn;
+  private final IndexInput indexIn;
+  private final int valueLength;
+  private final long dataFilePointer;
+  private long bytesToRead;
+  
+  public Lucene41BinaryDocValuesProducer(IndexInput dataIn, IndexInput indexIn)
+      throws IOException {
+    this.dataIn = dataIn;
+    CodecUtil.checkHeader(dataIn, Lucene41BinaryDocValuesConsumer.CODEC_NAME,
+        Lucene41BinaryDocValuesConsumer.VERSION_START,
+        Lucene41BinaryDocValuesConsumer.VERSION_START);
+    valueLength = dataIn.readInt();
+    dataFilePointer = dataIn.getFilePointer();
+    CodecUtil.checkHeader(indexIn, Lucene41BinaryDocValuesConsumer.CODEC_NAME,
+        Lucene41BinaryDocValuesConsumer.VERSION_START,
+        Lucene41BinaryDocValuesConsumer.VERSION_START);
+    bytesToRead = indexIn.readLong();
+    if (valueLength == Lucene41BinaryDocValuesConsumer.VALUE_SIZE_VAR) {
+      this.indexIn = indexIn;
+    } else {
+      indexIn.close();
+      this.indexIn = null;
+    }
+    
+  }
+  
+  @Override
+  protected Source loadSource() throws IOException {
+    if (valueLength == Lucene41BinaryDocValuesConsumer.VALUE_SIZE_VAR) {
+      assert indexIn != null;
+      return new VarStraightSource(dataIn.clone(), indexIn.clone(), bytesToRead);
+    } else {
+      assert indexIn == null;
+      return new FixedStraightSource(dataIn.clone(), valueLength, bytesToRead);
+    }
+  }
+  
+  @Override
+  protected Source loadDirectSource() throws IOException {
+    if (valueLength == Lucene41BinaryDocValuesConsumer.VALUE_SIZE_VAR) {
+      assert indexIn != null;
+      return new DirectVarStraightSource(dataIn.clone(), indexIn.clone(),
+          dataFilePointer);
+    } else {
+      assert indexIn == null;
+      return new DirectFixedStraightSource(dataIn.clone(), valueLength,
+          dataFilePointer);
+    }
+  }
+  
+  @Override
+  public Type getType() {
+    return valueLength == Lucene41BinaryDocValuesConsumer.VALUE_SIZE_VAR ? Type.BYTES_VAR_STRAIGHT
+        : Type.BYTES_FIXED_STRAIGHT;
+  }
+  
+  @Override
+  public void close() throws IOException {
+    super.close();
+    IOUtils.close(dataIn, indexIn);
+  }
+  
+  static abstract class BytesSourceBase extends Source {
+    private final PagedBytes pagedBytes;
+    protected final IndexInput datIn;
+    protected final IndexInput idxIn;
+    protected final static int PAGED_BYTES_BITS = 15;
+    protected final PagedBytes.Reader data;
+    protected final long totalLengthInBytes;
+    
+    protected BytesSourceBase(IndexInput datIn, IndexInput idxIn,
+        PagedBytes pagedBytes, long bytesToRead, Type type) throws IOException {
+      super(type);
+      assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
+          + (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
+      this.datIn = datIn;
+      this.totalLengthInBytes = bytesToRead;
+      this.pagedBytes = pagedBytes;
+      this.pagedBytes.copy(datIn, bytesToRead);
+      data = pagedBytes.freeze(true);
+      this.idxIn = idxIn;
+    }
+  }
+  
+  public final static class DirectVarStraightSource extends Source {
+    
+    private final PackedInts.Reader index;
+    private final IndexInput data;
+    private final long baseOffset;
+    
+    DirectVarStraightSource(IndexInput data, IndexInput index,
+        long dataFilePointer) throws IOException {
+      super(Type.BYTES_VAR_STRAIGHT);
+      this.data = data;
+      baseOffset = dataFilePointer;
+      this.index = PackedInts.getDirectReader(index); // nocommit read without
+                                                      // header
+    }
+    
+    private final int position(int docID) throws IOException {
+      final long offset = index.get(docID);
+      data.seek(baseOffset + offset);
+      // Safe to do 1+docID because we write sentinel at the end:
+      final long nextOffset = index.get(1 + docID);
+      return (int) (nextOffset - offset);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      try {
+        final int sizeToRead = position(docID);
+        ref.offset = 0;
+        ref.grow(sizeToRead);
+        data.readBytes(ref.bytes, 0, sizeToRead);
+        ref.length = sizeToRead;
+        return ref;
+      } catch (IOException ex) {
+        throw new IllegalStateException("failed to get value for docID: "
+            + docID, ex);
+      }
+    }
+  }
+  
+  private static final class VarStraightSource extends BytesSourceBase {
+    private final PackedInts.Reader addresses;
+    
+    public VarStraightSource(IndexInput datIn, IndexInput idxIn,
+        long bytesToRead) throws IOException {
+      super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), bytesToRead,
+          Type.BYTES_VAR_STRAIGHT);
+      addresses = PackedInts.getReader(idxIn); // nocommit read without header
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef bytesRef) {
+      final long address = addresses.get(docID);
+      return data.fillSlice(bytesRef, address,
+          (int) (addresses.get(docID + 1) - address));
+    }
+  }
+  
+  private final static class FixedStraightSource extends BytesSourceBase {
+    private final int size;
+    
+    public FixedStraightSource(IndexInput datIn, int size, long bytesToRead)
+        throws IOException {
+      super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), bytesToRead,
+          Type.BYTES_FIXED_STRAIGHT);
+      this.size = size;
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef bytesRef) {
+      return data.fillSlice(bytesRef, size * ((long) docID), size);
+    }
+  }
+  
+  public final static class DirectFixedStraightSource extends Source {
+    private final int size;
+    private IndexInput data;
+    private long baseOffset;
+    
+    DirectFixedStraightSource(IndexInput input, int size, long dataFilePointer) {
+      super(Type.BYTES_FIXED_STRAIGHT);
+      this.size = size;
+      this.data = input;
+      baseOffset = dataFilePointer;
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      try {
+        data.seek(baseOffset + size * ((long) docID));
+        ref.offset = 0;
+        ref.grow(size);
+        data.readBytes(ref.bytes, 0, size);
+        ref.length = size;
+        return ref;
+      } catch (IOException ex) {
+        throw new IllegalStateException("failed to get value for docID: "
+            + docID, ex);
+      }
+    }
+  }
+}

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41DocValuesConsumer.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41DocValuesConsumer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41DocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,111 @@
+package org.apache.lucene.codecs.lucene41.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.codecs.BinaryDocValuesConsumer;
+import org.apache.lucene.codecs.NumericDocValuesConsumer;
+import org.apache.lucene.codecs.SimpleDVConsumer;
+import org.apache.lucene.codecs.SortedDocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+public class Lucene41DocValuesConsumer extends SimpleDVConsumer {
+  /**
+   * Filename extension for index files
+   */
+  static final String INDEX_EXTENSION = "idx";
+  
+  /**
+   * Filename extension for data files.
+   */
+  static final String DATA_EXTENSION = "dat";
+  
+  static final String DV_SEGMENT_SUFFIX = "dv";
+  
+  private final SegmentInfo info;
+  private final Directory dir;
+  private final IOContext context;
+  
+  Lucene41DocValuesConsumer(Directory dir, SegmentInfo si, IOContext context)
+      throws IOException {
+    this.dir = dir;
+    this.info = si;
+    this.context = context;
+  }
+  
+  @Override
+  public void close() throws IOException {}
+  
+  @Override
+  public NumericDocValuesConsumer addNumericField(FieldInfo field,
+      long minValue, long maxValue, int numDocs) throws IOException {
+    String name = IndexFileNames.segmentFileName(this.info.name + "_"
+        + field.number, DV_SEGMENT_SUFFIX, DATA_EXTENSION);
+    IndexOutput dataOut = null;
+    boolean success = false;
+    try {
+      dataOut = dir.createOutput(name, context);
+      Lucene41NumericDocValuesConsumer consumer = new Lucene41NumericDocValuesConsumer(
+          dataOut, minValue, maxValue, numDocs);
+      success = true;
+      return consumer;
+    } finally {
+      if (!success) {
+        IOUtils.close(dataOut);
+      }
+    }
+  }
+  
+  @Override
+  public BinaryDocValuesConsumer addBinaryField(FieldInfo field,
+      boolean fixedLength, int maxLength, int numDocs) throws IOException {
+    String nameData = IndexFileNames.segmentFileName(this.info.name + "_"
+        + field.number, DV_SEGMENT_SUFFIX, DATA_EXTENSION);
+    String idxOut = IndexFileNames.segmentFileName(this.info.name + "_"
+        + field.number, DV_SEGMENT_SUFFIX, INDEX_EXTENSION);
+    boolean success = false;
+    IndexOutput dataOut = null;
+    IndexOutput indexOut = null;
+    try {
+      dataOut = dir.createOutput(nameData, context);
+      indexOut = dir.createOutput(idxOut, context);
+      Lucene41BinaryDocValuesConsumer consumer = new Lucene41BinaryDocValuesConsumer(
+          dataOut, indexOut, fixedLength, maxLength);
+      success = true;
+      return consumer;
+    } finally {
+      if (!success) {
+        IOUtils.close(dataOut, indexOut);
+      }
+    }
+  }
+  
+  @Override
+  public SortedDocValuesConsumer addSortedField(FieldInfo field,
+      int valueCount, boolean fixedLength, int maxLength, int numDocs)
+      throws IOException {
+    return null;
+  }
+  
+}

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesConsumer.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesConsumer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,46 @@
+package org.apache.lucene.codecs.lucene41.values;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NumericDocValuesConsumer;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedInts.Writer;
+
+public class Lucene41NumericDocValuesConsumer extends NumericDocValuesConsumer {
+  private final IndexOutput output;
+  private final Writer writer;
+  private final long minValue;
+  private int numDocsWritten = 0;
+  static final int VERSION_START = -1;
+  static final String CODEC_NAME = "Lucene41Numeric";
+
+  public Lucene41NumericDocValuesConsumer(IndexOutput output, long minValue, long maxValue, int valueCount) throws IOException {
+    this.output = output;
+    final long delta = maxValue - minValue;
+    final int bitsRequired = delta < 0 ? 64 : PackedInts.bitsRequired(delta);
+    CodecUtil.writeHeader(output, CODEC_NAME, VERSION_START);
+    output.writeLong(minValue);
+    this.minValue = minValue;
+    // nocommit write without header?
+    this.writer = PackedInts.getWriter(output, valueCount, bitsRequired, PackedInts.FASTEST);
+  }
+  
+  
+  @Override
+  public void add(long value) throws IOException {
+    writer.add(value-minValue);
+    numDocsWritten++;
+  }
+  
+  @Override
+  public void finish() throws IOException {
+    try {
+      writer.finish();
+    } finally {
+      output.close();
+    }
+  }
+  
+}

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesProducer.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesProducer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41NumericDocValuesProducer.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,115 @@
+package org.apache.lucene.codecs.lucene41.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.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesArraySource;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+
+
+public class Lucene41NumericDocValuesProducer extends DocValues {
+  
+  private final IndexInput datIn;
+  private final long minValue;
+  
+  public Lucene41NumericDocValuesProducer(IndexInput input, int numDocs)
+      throws IOException {
+    datIn = input;
+    boolean success = false;
+    try {
+      CodecUtil.checkHeader(datIn, Lucene41NumericDocValuesConsumer.CODEC_NAME,
+          Lucene41NumericDocValuesConsumer.VERSION_START,
+          Lucene41NumericDocValuesConsumer.VERSION_START);
+      minValue = datIn.readLong();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(datIn);
+      }
+    }
+  }
+  
+  /**
+   * Loads the actual values. You may call this more than once, eg if you
+   * already previously loaded but then discarded the Source.
+   */
+  @Override
+  protected Source loadSource() throws IOException {
+    boolean success = false;
+    final Source source;
+    IndexInput input = null;
+    try {
+      input = datIn.clone();
+      source = new PackedIntsSource(input, false, minValue);
+      success = true;
+      return source;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(input, datIn);
+      }
+    }
+  }
+  
+  @Override
+  public void close() throws IOException {
+    super.close();
+    datIn.close();
+  }
+  
+  @Override
+  public Type getType() {
+    return Type.VAR_INTS;
+  }
+  
+  @Override
+  protected Source loadDirectSource() throws IOException {
+    return new PackedIntsSource(datIn.clone(), true, minValue);
+  }
+  
+  static class PackedIntsSource extends Source {
+    private final PackedInts.Reader values;
+    private final long minValue;
+    
+    public PackedIntsSource(IndexInput dataIn, boolean direct, long minValue)
+        throws IOException {
+      super(Type.VAR_INTS);
+      this.minValue = minValue;
+      values = direct ? PackedInts.getDirectReader(dataIn) : PackedInts
+          .getReader(dataIn);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      ref.grow(8);
+      DocValuesArraySource.copyLong(ref, getInt(docID));
+      return ref;
+    }
+    
+    @Override
+    public long getInt(int docID) {
+      assert docID >= 0;
+      return values.get(docID) + minValue;
+    }
+  }
+  
+}

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SimpleDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SimpleDocValuesFormat.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SimpleDocValuesFormat.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SimpleDocValuesFormat.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,109 @@
+package org.apache.lucene.codecs.lucene41.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.Collection;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.codecs.PerDocProducerBase;
+import org.apache.lucene.codecs.SimpleDVConsumer;
+import org.apache.lucene.codecs.SimpleDocValuesFormat;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.IOUtils;
+
+
+public class Lucene41SimpleDocValuesFormat extends SimpleDocValuesFormat {
+  
+  @Override
+  public SimpleDVConsumer fieldsConsumer(SegmentWriteState state)
+      throws IOException {
+    return new Lucene41DocValuesConsumer(state.directory, state.segmentInfo, state.context);
+  }
+  
+  @Override
+  public PerDocProducer fieldsProducer(SegmentReadState state)
+      throws IOException {
+    return new Lucene41PerDocProducer(state);
+  }
+  //nocommit this is equivalent to sep - we should pack in CFS
+  private static final class Lucene41PerDocProducer extends PerDocProducerBase {
+    private final TreeMap<String, DocValues> docValues;
+
+    /**
+     * Creates a new {@link Lucene41PerDocProducer} instance and loads all
+     * {@link DocValues} instances for this segment and codec.
+     */
+    public Lucene41PerDocProducer(SegmentReadState state) throws IOException {
+      docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), state.dir, state.context);
+    }
+    
+    @Override
+    protected Map<String,DocValues> docValues() {
+      return docValues;
+    }
+    
+    @Override
+    protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
+      IOUtils.close(closeables);
+    }
+
+    @Override
+    protected DocValues loadDocValues(int docCount, Directory dir, String id,
+        Type type, IOContext context) throws IOException {
+        switch (type) {
+        case FIXED_INTS_16:
+        case FIXED_INTS_32:
+        case FIXED_INTS_64:
+        case FIXED_INTS_8:
+        case VAR_INTS:
+        case FLOAT_32:
+        case FLOAT_64:
+          return new Lucene41NumericDocValuesProducer(dir.openInput( IndexFileNames.segmentFileName(
+        id, Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX, Lucene41DocValuesConsumer.DATA_EXTENSION), context), docCount);
+        
+        case BYTES_FIXED_STRAIGHT:
+        case BYTES_FIXED_DEREF:
+        case BYTES_VAR_STRAIGHT:
+        case BYTES_VAR_DEREF:
+          //nocommit cose in case of an exception
+          IndexInput dataIn = dir.openInput(IndexFileNames.segmentFileName(id,
+              Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX,
+              Lucene41DocValuesConsumer.DATA_EXTENSION), context);
+          IndexInput indexIn = dir.openInput(IndexFileNames.segmentFileName(id,
+              Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX,
+              Lucene41DocValuesConsumer.INDEX_EXTENSION), context);
+          return new Lucene41BinaryDocValuesProducer(dataIn, indexIn);
+        case BYTES_VAR_SORTED:
+        case BYTES_FIXED_SORTED:
+        default:
+          throw new IllegalStateException("unrecognized index values mode " + type);
+        }
+      }
+  }
+  
+}

Added: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SortedDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SortedDocValuesConsumer.java?rev=1410376&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SortedDocValuesConsumer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/values/Lucene41SortedDocValuesConsumer.java Fri Nov 16 14:45:10 2012
@@ -0,0 +1,35 @@
+package org.apache.lucene.codecs.lucene41.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.codecs.SortedDocValuesConsumer;
+import org.apache.lucene.util.BytesRef;
+
+public class Lucene41SortedDocValuesConsumer extends SortedDocValuesConsumer {
+  
+  @Override
+  public void addValue(BytesRef value) throws IOException {}
+  
+  @Override
+  public void addDoc(int ord) throws IOException {}
+  
+  @Override
+  public void finish() throws IOException {}
+  
+}

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/BytesDVWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/BytesDVWriter.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/BytesDVWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/BytesDVWriter.java Fri Nov 16 14:45:10 2012
@@ -18,13 +18,11 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
 import org.apache.lucene.codecs.BinaryDocValuesConsumer;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefArray;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.RamUsageEstimator;
 
 
 /** Buffers up pending byte[] per doc, then flushes when
@@ -33,43 +31,38 @@ import org.apache.lucene.util.RamUsageEs
 // nocommit make this a consumer in the chain?
 class BytesDVWriter {
 
-  // nocommit more ram efficient?
-  private final ArrayList<byte[]> pending = new ArrayList<byte[]>();
-  private final Counter iwBytesUsed;
-  private int bytesUsed;
+  private final BytesRefArray bytesRefArray;
   private final FieldInfo fieldInfo;
+  private int addeValues = 0;
+  private final BytesRef emptyBytesRef = new BytesRef();
 
   // -2 means not set yet; -1 means length isn't fixed;
   // -otherwise it's the fixed length seen so far:
   int fixedLength = -2;
   int maxLength;
+  int totalSize;
 
-  public BytesDVWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+  public BytesDVWriter(FieldInfo fieldInfo, Counter counter) {
     this.fieldInfo = fieldInfo;
-    this.iwBytesUsed = iwBytesUsed;
+    this.bytesRefArray = new BytesRefArray(counter);
+    this.totalSize = 0;
   }
 
   public void addValue(int docID, BytesRef value) {
-    final int oldBytesUsed = bytesUsed;
     if (value == null) {
       // nocommit improve message
       throw new IllegalArgumentException("null binaryValue not allowed (field=" + fieldInfo.name + ")");
     }
     mergeLength(value.length);
+    
     // Fill in any holes:
-    while(pending.size() < docID) {
-      pending.add(BytesRef.EMPTY_BYTES);
-      bytesUsed += (int) (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 1.25);
+    while(addeValues < docID) {
+      addeValues++;
+      bytesRefArray.append(emptyBytesRef);
       mergeLength(0);
     }
-    byte[] bytes = new byte[value.length];
-    System.arraycopy(value.bytes, value.offset, bytes, 0, value.length);
-    pending.add(bytes);
-
-    // estimate 25% overhead for ArrayList:
-    bytesUsed += (int) (bytes.length + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 1.25));
-    iwBytesUsed.addAndGet(bytesUsed - oldBytesUsed);
-    //System.out.println("ADD: " + value);
+    addeValues++;
+    bytesRefArray.append(value);
   }
 
   private void mergeLength(int length) {
@@ -79,15 +72,14 @@ class BytesDVWriter {
       fixedLength = -1;
     }
     maxLength = Math.max(maxLength, length);
+    totalSize += length;
   }
 
   public void flush(FieldInfo fieldInfo, SegmentWriteState state, BinaryDocValuesConsumer consumer) throws IOException {
-    final int bufferedDocCount = pending.size();
+    final int bufferedDocCount = addeValues;
     BytesRef value = new BytesRef();
-
     for(int docID=0;docID<bufferedDocCount;docID++) {
-      value.bytes = pending.get(docID);
-      value.length = value.bytes.length;
+      bytesRefArray.get(value, docID);
       consumer.add(value);
     }
     final int maxDoc = state.segmentInfo.getDocCount();
@@ -95,6 +87,7 @@ class BytesDVWriter {
     for(int docID=bufferedDocCount;docID<maxDoc;docID++) {
       consumer.add(value);
     }
+    consumer.finish();
     reset();
     //System.out.println("FLUSH");
   }
@@ -104,10 +97,7 @@ class BytesDVWriter {
   }
 
   private void reset() {
-    pending.clear();
-    pending.trimToSize();
-    iwBytesUsed.addAndGet(-bytesUsed);
-    bytesUsed = 0;
+    bytesRefArray.clear();
     fixedLength = -2;
     maxLength = 0;
   }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Fri Nov 16 14:45:10 2012
@@ -110,7 +110,7 @@ final class DocFieldProcessor extends Do
             field.bytesDVWriter.flush(field.fieldInfo, state,
                                       dvConsumer.addBinaryField(field.fieldInfo,
                                                                 field.bytesDVWriter.fixedLength >= 0,
-                                                                field.bytesDVWriter.maxLength));
+                                                                field.bytesDVWriter.maxLength, state.segmentInfo.getDocCount()));
             // nocommit must null it out now else next seg
             // will flush even if no docs had DV...?
           }
@@ -119,7 +119,7 @@ final class DocFieldProcessor extends Do
                                             dvConsumer.addSortedField(field.fieldInfo,
                                                                       field.sortedBytesDVWriter.hash.size(),
                                                                       field.sortedBytesDVWriter.fixedLength >= 0,
-                                                                      field.sortedBytesDVWriter.maxLength));
+                                                                      field.sortedBytesDVWriter.maxLength, state.segmentInfo.getDocCount()));
             // nocommit must null it out now else next seg
             // will flush even if no docs had DV...?
           }
@@ -127,7 +127,7 @@ final class DocFieldProcessor extends Do
             field.numberDVWriter.flush(field.fieldInfo, state,
                                        dvConsumer.addNumericField(field.fieldInfo,
                                                                   field.numberDVWriter.minValue,
-                                                                  field.numberDVWriter.maxValue));
+                                                                  field.numberDVWriter.maxValue, state.segmentInfo.getDocCount()));
             // nocommit must null it out now else next seg
             // will flush even if no docs had DV...?
           }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/NumberDVWriter.java Fri Nov 16 14:45:10 2012
@@ -94,6 +94,7 @@ class NumberDVWriter {
     for(int docID=bufferedDocCount;docID<maxDoc;docID++) {
       consumer.add(0);
     }
+    consumer.finish();
     reset();
     //System.out.println("FLUSH");
   }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java Fri Nov 16 14:45:10 2012
@@ -21,12 +21,18 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter.DEFAULT;
+
 import org.apache.lucene.codecs.BinaryDocValuesConsumer;
 import org.apache.lucene.codecs.SortedDocValuesConsumer;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.BytesRefHash.BytesStartArray;
+import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 
 
 /** Buffers up pending byte[] per doc, deref and sorting via
@@ -34,16 +40,14 @@ import org.apache.lucene.util.RamUsageEs
 // nocommit name?
 // nocommit make this a consumer in the chain?
 class SortedBytesDVWriter {
-
-  // nocommit more ram efficient?
-  // nocommit pass allocator that counts RAM used!
-  final BytesRefHash hash = new BytesRefHash();
-  private final ArrayList<Integer> pending = new ArrayList<Integer>();
+  final BytesRefHash hash;
+  private int[] pending = new int[DEFAULT_PENDING_SIZE];
+  private int pendingIndex = 0;
   private final Counter iwBytesUsed;
-  private int bytesUsed;
   private final FieldInfo fieldInfo;
 
   private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
+  private static final int DEFAULT_PENDING_SIZE = 16;
 
   // -2 means not set yet; -1 means length isn't fixed;
   // -otherwise it's the fixed length seen so far:
@@ -53,37 +57,41 @@ class SortedBytesDVWriter {
   public SortedBytesDVWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
     this.iwBytesUsed = iwBytesUsed;
+    hash = new BytesRefHash(
+        new ByteBlockPool(
+            new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
+            BytesRefHash.DEFAULT_CAPACITY,
+            new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
+    iwBytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_INT * DEFAULT_PENDING_SIZE);
   }
 
   public void addValue(int docID, BytesRef value) {
-    final int oldBytesUsed = bytesUsed;
     if (value == null) {
       // nocommit improve message
       throw new IllegalArgumentException("null sortedValue not allowed (field=" + fieldInfo.name + ")");
     }
 
     // Fill in any holes:
-    while(pending.size() < docID) {
+    while(pendingIndex < docID) {
       addOneValue(EMPTY);
     }
 
     addOneValue(value);
-    iwBytesUsed.addAndGet(bytesUsed - oldBytesUsed);
   }
 
   private void addOneValue(BytesRef value) {
     mergeLength(value.length);
-
     int ord = hash.add(value);
     if (ord < 0) {
       ord = -ord-1;
-    } else {
-      // nocommit this is undercounting!
-      bytesUsed += value.length;
-    }
-    pending.add(ord);
-    // estimate 25% overhead for ArrayList:
-    bytesUsed += (int) (RamUsageEstimator.NUM_BYTES_OBJECT_REF * 1.25) + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT;
+    } 
+    
+    if (pendingIndex <= pending.length) {
+      int pendingLen = pending.length;
+      pending = ArrayUtil.grow(pending, pendingIndex+1);
+      iwBytesUsed.addAndGet((pending.length - pendingLen) * RamUsageEstimator.NUM_BYTES_INT);
+    }
+    pending[pendingIndex++] = ord;
   }
 
   private void mergeLength(int length) {
@@ -100,7 +108,7 @@ class SortedBytesDVWriter {
 
     final int maxDoc = state.segmentInfo.getDocCount();
     int emptyOrd = -1;
-    if (pending.size() < maxDoc) {
+    if (pendingIndex < maxDoc) {
       // Make sure we added EMPTY value before sorting:
       emptyOrd = hash.add(EMPTY);
       if (emptyOrd < 0) {
@@ -109,23 +117,24 @@ class SortedBytesDVWriter {
     }
 
     int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
-    // nocommit must budget this into RAM consumption up front!
-    int[] ordMap = new int[valueCount];
-
+    final int sortedValueRamUsage = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_INT*valueCount;
+    iwBytesUsed.addAndGet(sortedValueRamUsage);
+    final int[] ordMap = new int[valueCount];
     // Write values, in sorted order:
     BytesRef scratch = new BytesRef();
     for(int ord=0;ord<valueCount;ord++) {
       consumer.addValue(hash.get(sortedValues[ord], scratch));
       ordMap[sortedValues[ord]] = ord;
     }
-    final int bufferedDocCount = pending.size();
+    final int bufferedDocCount = pendingIndex;
 
     for(int docID=0;docID<bufferedDocCount;docID++) {
-      consumer.addDoc(ordMap[pending.get(docID)]);
+      consumer.addDoc(ordMap[pending[docID]]);
     }
     for(int docID=bufferedDocCount;docID<maxDoc;docID++) {
       consumer.addDoc(ordMap[emptyOrd]);
     }
+    iwBytesUsed.addAndGet(-sortedValueRamUsage);
     reset();
   }
 
@@ -134,11 +143,10 @@ class SortedBytesDVWriter {
   }
 
   private void reset() {
-    pending.clear();
-    pending.trimToSize();
+    iwBytesUsed.addAndGet((pending.length - DEFAULT_PENDING_SIZE) * RamUsageEstimator.NUM_BYTES_INT);
+    pending = ArrayUtil.shrink(pending, DEFAULT_PENDING_SIZE);
+    pendingIndex = 0;
     hash.clear();
-    iwBytesUsed.addAndGet(-bytesUsed);
-    bytesUsed = 0;
     fixedLength = -2;
     maxLength = 0;
   }

Copied: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java (from r1410243, lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java?p2=lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java&p1=lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java&r1=1410243&r2=1410376&rev=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java Fri Nov 16 14:45:10 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.suggest;
+package org.apache.lucene.util;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,13 +20,6 @@ package org.apache.lucene.search.suggest
 import java.util.Arrays;
 import java.util.Comparator;
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.ByteBlockPool;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
-import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.SorterTemplate;
 
 /**
  * A simple append only random-access {@link BytesRef} array that stores full
@@ -38,27 +31,34 @@ import org.apache.lucene.util.SorterTemp
  * @lucene.internal
  * @lucene.experimental
  */
-public final class BytesRefList {
-  // TODO rename to BytesRefArray
+public final class BytesRefArray {
   private final ByteBlockPool pool;
   private int[] offsets = new int[1];
   private int lastElement = 0;
   private int currentOffset = 0;
-  private final Counter bytesUsed = Counter.newCounter(false);
+  private final Counter bytesUsed;
   
   /**
-   * Creates a new {@link BytesRefList}
+   * Creates a new {@link BytesRefArray}
    */
-  public BytesRefList() {
+  public BytesRefArray() {
+    this(Counter.newCounter(false));
+  }  
+  
+  /**
+   * Creates a new {@link BytesRefArray} with a counter to track allocated bytes
+   */
+  public BytesRefArray(Counter bytesUsed) {
     this.pool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(
         bytesUsed));
     pool.nextBuffer();
     bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
         + RamUsageEstimator.NUM_BYTES_INT);
+    this.bytesUsed = bytesUsed;
   }
  
   /**
-   * Clears this {@link BytesRefList}
+   * Clears this {@link BytesRefArray}
    */
   public void clear() {
     lastElement = 0;
@@ -68,7 +68,7 @@ public final class BytesRefList {
   }
   
   /**
-   * Appends a copy of the given {@link BytesRef} to this {@link BytesRefList}.
+   * Appends a copy of the given {@link BytesRef} to this {@link BytesRefArray}.
    * @param bytes the bytes to append
    * @return the ordinal of the appended bytes
    */
@@ -86,18 +86,18 @@ public final class BytesRefList {
   }
   
   /**
-   * Returns the current size of this {@link BytesRefList}
-   * @return the current size of this {@link BytesRefList}
+   * Returns the current size of this {@link BytesRefArray}
+   * @return the current size of this {@link BytesRefArray}
    */
   public int size() {
     return lastElement;
   }
   
   /**
-   * Returns the <i>n'th</i> element of this {@link BytesRefList}
+   * Returns the <i>n'th</i> element of this {@link BytesRefArray}
    * @param spare a spare {@link BytesRef} instance
    * @param ord the elements ordinal to retrieve 
-   * @return the <i>n'th</i> element of this {@link BytesRefList}
+   * @return the <i>n'th</i> element of this {@link BytesRefArray}
    */
   public BytesRef get(BytesRef spare, int ord) {
     if (lastElement > ord) {

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Fri Nov 16 14:45:10 2012
@@ -69,7 +69,7 @@ public final class BytesRefHash {
   public BytesRefHash() { 
     this(new ByteBlockPool(new DirectAllocator()));
   }
-
+  
   /**
    * Creates a new {@link BytesRefHash}
    */
@@ -590,9 +590,13 @@ public final class BytesRefHash {
     private int[] bytesStart;
     private final Counter bytesUsed;
     
+    public DirectBytesStartArray(int initSize, Counter counter) {
+      this.bytesUsed = counter;
+      this.initSize = initSize;      
+    }
+    
     public DirectBytesStartArray(int initSize) {
-      this.bytesUsed = Counter.newCounter();
-      this.initSize = initSize;
+      this(initSize, Counter.newCounter());
     }
 
     @Override

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java Fri Nov 16 14:45:10 2012
@@ -239,6 +239,8 @@ public class TestDemoDocValue extends Lu
 
     ireader.close();
     directory.close();
+    
+    //nocommit fails with Lucene41 Codec since "dv" is created with var len but is in fact fixed len
   }
   
   public void testBytesTwoDocumentsMerged() throws IOException {

Copied: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java (from r1410243, lucene/dev/branches/lucene4547/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefList.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java?p2=lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java&p1=lucene/dev/branches/lucene4547/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefList.java&r1=1410243&r2=1410376&rev=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefList.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java Fri Nov 16 14:45:10 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.suggest;
+package org.apache.lucene.util;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,17 +20,17 @@ package org.apache.lucene.search.suggest
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.lucene.search.suggest.BytesRefList;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.BytesRefArray;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
-public class TestBytesRefList extends LuceneTestCase {
+public class TestBytesRefArray extends LuceneTestCase {
 
   public void testAppend() throws IOException {
     Random random = random();
-    BytesRefList list = new BytesRefList();
+    BytesRefArray list = new BytesRefArray();
     List<String> stringList = new ArrayList<String>();
     for (int j = 0; j < 2; j++) {
       if (j > 0 && random.nextBoolean()) {
@@ -71,7 +71,7 @@ public class TestBytesRefList extends Lu
 
   public void testSort() throws IOException {
     Random random = random();
-    BytesRefList list = new BytesRefList();
+    BytesRefArray list = new BytesRefArray();
     List<String> stringList = new ArrayList<String>();
 
     for (int j = 0; j < 2; j++) {

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java Fri Nov 16 14:45:10 2012
@@ -22,6 +22,7 @@ import java.util.Comparator;
 import org.apache.lucene.search.spell.TermFreqIterator;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefArray;
 
 /**
  * This wrapper buffers incoming elements.
@@ -30,7 +31,7 @@ import org.apache.lucene.util.BytesRef;
 public class BufferingTermFreqIteratorWrapper implements TermFreqIterator {
   // TODO keep this for now
   /** buffered term entries */
-  protected BytesRefList entries = new BytesRefList();
+  protected BytesRefArray entries = new BytesRefArray();
   /** current buffer position */
   protected int curPos = -1;
   /** buffered weights, parallel with {@link #entries} */

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/InMemorySorter.java Fri Nov 16 14:45:10 2012
@@ -19,9 +19,9 @@ package org.apache.lucene.search.suggest
 
 import java.util.Comparator;
 
-import org.apache.lucene.search.suggest.BytesRefList;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.BytesRefArray;
 
 /**
  * An {@link BytesRefSorter} that keeps all the entries in memory.
@@ -29,7 +29,7 @@ import org.apache.lucene.util.BytesRefIt
  * @lucene.internal
  */
 public final class InMemorySorter implements BytesRefSorter {
-  private final BytesRefList buffer = new BytesRefList();
+  private final BytesRefArray buffer = new BytesRefArray();
   private boolean closed = false;
   private final Comparator<BytesRef> comparator;
 

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java?rev=1410376&r1=1410375&r2=1410376&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/Sort.java Fri Nov 16 14:45:10 2012
@@ -20,7 +20,6 @@ package org.apache.lucene.search.suggest
 import java.io.*;
 import java.util.*;
 
-import org.apache.lucene.search.suggest.BytesRefList;
 import org.apache.lucene.util.*;
 import org.apache.lucene.util.PriorityQueue;
 
@@ -156,7 +155,7 @@ public final class Sort {
   private final BufferSize ramBufferSize;
   private final File tempDirectory;
   
-  private final BytesRefList buffer = new BytesRefList();
+  private final BytesRefArray buffer = new BytesRefArray();
   private SortInfo sortInfo;
   private int maxTempFiles;
   private final Comparator<BytesRef> comparator;
@@ -312,7 +311,7 @@ public final class Sort {
 
   /** Sort a single partition in-memory. */
   protected File sortPartition(int len) throws IOException {
-    BytesRefList data = this.buffer;
+    BytesRefArray data = this.buffer;
     File tempFile = File.createTempFile("sort", "partition", tempDirectory);
 
     long start = System.currentTimeMillis();