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 2011/05/19 16:22:55 UTC

svn commit: r1124825 - in /lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene: document/ index/ index/codecs/ index/values/ util/

Author: simonw
Date: Thu May 19 14:22:54 2011
New Revision: 1124825

URL: http://svn.apache.org/viewvc?rev=1124825&view=rev
Log:
LUCENE-3108: Enable memory tracking for ByteBlockPool allocations in DocValues writer impls.

Modified:
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRefHash.java

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java Thu May 19 14:22:54 2011
@@ -303,7 +303,7 @@ public abstract class AbstractField impl
   }
   
   public boolean hasDocValues() {
-    return docValues != null;
+    return docValues != null && docValues.type() != null;
   }
   
   public ValueType docValuesType() {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu May 19 14:22:54 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.search.Similari
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.RamUsageEstimator;
 
 public class DocumentsWriterPerThread {
@@ -169,6 +170,7 @@ public class DocumentsWriterPerThread {
   DocumentsWriterDeleteQueue deleteQueue;
   DeleteSlice deleteSlice;
   private final NumberFormat nf = NumberFormat.getInstance();
+  final Allocator byteBlockAllocator;
 
   
   public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
@@ -181,9 +183,9 @@ public class DocumentsWriterPerThread {
     this.docState = new DocState(this);
     this.docState.similarityProvider = parent.indexWriter.getConfig()
         .getSimilarityProvider();
-
-    consumer = indexingChain.getChain(this);
     bytesUsed = new AtomicLong(0);
+    byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
+    consumer = indexingChain.getChain(this);
     pendingDeletes = new BufferedDeletes(false);
     initialize();
   }
@@ -462,32 +464,6 @@ public class DocumentsWriterPerThread {
     bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT)));
   }
 
-  final Allocator byteBlockAllocator = new DirectTrackingAllocator();
-    
-    
- private class DirectTrackingAllocator extends Allocator {
-    public DirectTrackingAllocator() {
-      this(BYTE_BLOCK_SIZE);
-    }
-
-    public DirectTrackingAllocator(int blockSize) {
-      super(blockSize);
-    }
-
-    public byte[] getByteBlock() {
-      bytesUsed.addAndGet(blockSize);
-      return new byte[blockSize];
-    }
-    @Override
-    public void recycleByteBlocks(byte[][] blocks, int start, int end) {
-      bytesUsed.addAndGet(-((end-start)* blockSize));
-      for (int i = start; i < end; i++) {
-        blocks[i] = null;
-      }
-    }
-    
-  };
-
   PerDocWriteState newPerDocWriteState(int codecId) {
     assert segment != null;
     return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, codecId);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java Thu May 19 14:22:54 2011
@@ -151,9 +151,19 @@ public class MultiPerDocValues extends P
   }
 
   public void close() throws IOException {
-    PerDocValues[] perDocValues = this.subs;
+    final PerDocValues[] perDocValues = this.subs;
+    IOException ex = null;
     for (PerDocValues values : perDocValues) {
-      values.close();
+      try {
+        values.close();
+      } catch (IOException e) {
+        if (ex == null) {
+          ex = e;
+        }
+      }
+    }
+    if (ex != null) {
+      throw ex;
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java Thu May 19 14:22:54 2011
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.lucene.index.codecs.Codec;
@@ -243,8 +244,7 @@ final class PerFieldCodecWrapper extends
   }
   
   private final class PerDocProducers extends PerDocValues {
-    private final Set<String> fields = new TreeSet<String>();
-    private final Map<String, PerDocValues> codecs = new HashMap<String, PerDocValues>();
+    private final TreeMap<String, PerDocValues> codecs = new TreeMap<String, PerDocValues>();
 
     public PerDocProducers(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
         int readBufferSize, int indexDivisor) throws IOException {
@@ -253,7 +253,6 @@ final class PerFieldCodecWrapper extends
       try {
         for (FieldInfo fi : fieldInfos) {
           if (fi.hasDocValues()) { 
-            fields.add(fi.name);
             assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
             Codec codec = segmentCodecs.codecs[fi.getCodecId()];
             if (!producers.containsKey(codec)) {
@@ -280,9 +279,10 @@ final class PerFieldCodecWrapper extends
         }
       }
     }
+    
     @Override
     public Collection<String> fields() {
-      return fields;
+      return codecs.keySet();
     }
     @Override
     public DocValues docValues(String field) throws IOException {
@@ -302,11 +302,11 @@ final class PerFieldCodecWrapper extends
           if (next != null) {
             next.close();
           }
-        } catch (IOException ioe) {
+        } catch (Exception ioe) {
           // keep first IOException we hit but keep
           // closing the rest
           if (err == null) {
-            err = ioe;
+            err = new IOException(ioe);
           }
         }
       }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Thu May 19 14:22:54 2011
@@ -575,13 +575,15 @@ final class SegmentMerger {
       mergeState.multiDeletedDocs = new MultiBits(perDocBits, perDocBitsStarts);
       final PerDocConsumer docsConsumer = codec
           .docsConsumer(new PerDocWriteState(segmentWriteState));
+      MultiPerDocValues multiPerDocValues = null; 
       try {
-        docsConsumer.merge(
-            mergeState,
-            new MultiPerDocValues(perDocProducers
-                .toArray(PerDocValues.EMPTY_ARRAY), perDocSlices
-                .toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
+        multiPerDocValues = new MultiPerDocValues(perDocProducers
+            .toArray(PerDocValues.EMPTY_ARRAY), perDocSlices
+            .toArray(ReaderUtil.Slice.EMPTY_ARRAY));
+        docsConsumer.merge(mergeState, multiPerDocValues);
       } finally {
+        if (multiPerDocValues != null)
+          multiPerDocValues.close();
         docsConsumer.close();
       }
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java Thu May 19 14:22:54 2011
@@ -131,7 +131,7 @@ public class DefaultDocValuesProducer ex
   }
 
   public void close() throws IOException {
-    Collection<DocValues> values = docValues.values();
+    final Collection<DocValues> values = docValues.values();
     IOException ex = null;
     for (DocValues docValues : values) {
       try {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java Thu May 19 14:22:54 2011
@@ -341,45 +341,31 @@ public final class Bytes {
 
   // TODO: open up this API?!
   static abstract class BytesWriterBase extends Writer {
-    private final Directory dir;
     private final String id;
     protected IndexOutput idxOut;
     protected IndexOutput datOut;
     protected BytesRef bytesRef;
-    private final String codecName;
-    private final int version;
     protected final ByteBlockPool pool;
 
     protected BytesWriterBase(Directory dir, String id, String codecName,
-        int version, boolean initIndex, boolean initData, ByteBlockPool pool,
+        int version, boolean initIndex, ByteBlockPool pool,
         AtomicLong bytesUsed) throws IOException {
       super(bytesUsed);
-      this.dir = dir;
       this.id = id;
-      this.codecName = codecName;
-      this.version = version;
       this.pool = pool;
-      if (initData) {
-        initDataOut();
-      }
+        datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+            DATA_EXTENSION));
+        CodecUtil.writeHeader(datOut, codecName, version);
 
       if (initIndex) {
-        initIndexOut();
+        idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+            INDEX_EXTENSION));
+        CodecUtil.writeHeader(idxOut, codecName, version);
+      } else {
+        idxOut = null;
       }
     }
 
-    private void initDataOut() throws IOException {
-      datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
-          DATA_EXTENSION));
-      CodecUtil.writeHeader(datOut, codecName, version);
-    }
-
-    private void initIndexOut() throws IOException {
-      idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
-          INDEX_EXTENSION));
-      CodecUtil.writeHeader(idxOut, codecName, version);
-    }
-
     /**
      * Must be called only with increasing docIDs. It's OK for some docIDs to be
      * skipped; they will be filled with 0 bytes.
@@ -390,7 +376,6 @@ public final class Bytes {
     @Override
     public void finish(int docCount) throws IOException {
       try {
-        if (datOut != null)
           datOut.close();
       } finally {
         try {
@@ -483,9 +468,7 @@ public final class Bytes {
         super.close();
       } finally {
         try {
-          if (datIn != null) {
             datIn.close();
-          }
         } finally {
           if (idxIn != null) {
             idxIn.close();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Thu May 19 14:22:54 2011
@@ -33,7 +33,7 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -54,16 +54,15 @@ class FixedDerefBytesImpl {
     private final BytesRefHash hash = new BytesRefHash(pool,
         BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
             BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
-
     public Writer(Directory dir, String id, AtomicLong bytesUsed)
         throws IOException {
-      this(dir, id, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+      this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed);
     }
 
     public Writer(Directory dir, String id, Allocator allocator,
         AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
           new ByteBlockPool(allocator), bytesUsed);
       docToID = new int[1];
       bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
@@ -249,8 +248,11 @@ class FixedDerefBytesImpl {
       }
 
       public void close() throws IOException {
-        datIn.close();
-        idx.close();
+        try {
+          datIn.close();
+        } finally {
+          idx.close();
+        }
       }
 
       protected void fill(long address, BytesRef ref) throws IOException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Thu May 19 14:22:54 2011
@@ -36,7 +36,7 @@ import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -63,13 +63,13 @@ class FixedSortedBytesImpl {
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         AtomicLong bytesUsed) throws IOException {
-      this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+      this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed);
     }
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Allocator allocator, AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
           new ByteBlockPool(allocator), bytesUsed);
       docToEntry = new int[1];
       // docToEntry[0] = -1;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Thu May 19 14:22:54 2011
@@ -46,7 +46,7 @@ class FixedStraightBytesImpl {
     private byte[] oneRecord;
 
     protected Writer(Directory dir, String id) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, true, null, null);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null);
     }
 
     // TODO - impl bulk copy here!
@@ -87,7 +87,13 @@ class FixedStraightBytesImpl {
         }
         fill(state.docBase);
         // TODO should we add a transfer to API to each reader?
-        datOut.copyBytes(reader.cloneData(), size * maxDocs);
+        final IndexInput cloneData = reader.cloneData();
+        try {
+          datOut.copyBytes(cloneData, size * maxDocs);
+        } finally {
+          cloneData.close();  
+        }
+        
         lastDocID += maxDocs - 1;
       } else
         super.merge(state);
@@ -116,7 +122,7 @@ class FixedStraightBytesImpl {
     }
 
     public long ramBytesUsed() {
-      return 0;
+      return oneRecord == null ? 0 : oneRecord.length;
     }
 
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Thu May 19 14:22:54 2011
@@ -36,7 +36,7 @@ import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -117,13 +117,13 @@ class VarDerefBytesImpl {
 
     public Writer(Directory dir, String id, AtomicLong bytesUsed)
         throws IOException {
-      this(dir, id, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+      this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed);
     }
 
     public Writer(Directory dir, String id, Allocator allocator,
         AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
           new ByteBlockPool(allocator), bytesUsed);
       docToAddress = new int[1];
       bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Thu May 19 14:22:54 2011
@@ -35,7 +35,7 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -62,13 +62,13 @@ class VarSortedBytesImpl {
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         AtomicLong bytesUsed) throws IOException {
-      this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
+      this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed);
     }
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Allocator allocator, AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
           new ByteBlockPool(allocator), bytesUsed);
       this.comp = comp;
       docToEntry = new int[1];

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1124825&r1=1124824&r2=1124825&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Thu May 19 14:22:54 2011
@@ -51,7 +51,7 @@ class VarStraightBytesImpl {
 
     public Writer(Directory dir, String id, AtomicLong bytesUsed)
         throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true, null, bytesUsed);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, null, bytesUsed);
       docToAddress = new long[1];
       bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
     }

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=1124825&r1=1124824&r2=1124825&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 Thu May 19 14:22:54 2011
@@ -18,6 +18,8 @@ package org.apache.lucene.util;
  */
 import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
 import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
 
 /** 
@@ -78,6 +80,33 @@ public final class ByteBlockPool {
     }
     
   }
+  
+  public static class DirectTrackingAllocator extends Allocator {
+    private final AtomicLong bytesUsed;
+    
+    public DirectTrackingAllocator(AtomicLong bytesUsed) {
+      this(BYTE_BLOCK_SIZE, bytesUsed);
+    }
+
+    public DirectTrackingAllocator(int blockSize, AtomicLong bytesUsed) {
+      super(blockSize);
+      this.bytesUsed = bytesUsed;
+    }
+
+    public byte[] getByteBlock() {
+      bytesUsed.addAndGet(blockSize);
+      return new byte[blockSize];
+    }
+    @Override
+    public void recycleByteBlocks(byte[][] blocks, int start, int end) {
+      bytesUsed.addAndGet(-((end-start)* blockSize));
+      for (int i = start; i < end; i++) {
+        blocks[i] = null;
+      }
+    }
+    
+  };
+
 
   public byte[][] buffers = new byte[10][];
 
@@ -92,6 +121,20 @@ public final class ByteBlockPool {
   public ByteBlockPool(Allocator allocator) {
     this.allocator = allocator;
   }
+  
+  public void dropBuffersAndReset() {
+    if (bufferUpto != -1) {
+      // Recycle all but the first buffer
+      allocator.recycleByteBlocks(buffers, 0, 1+bufferUpto);
+
+      // Re-use the first buffer
+      bufferUpto = -1;
+      byteUpto = BYTE_BLOCK_SIZE;
+      byteOffset = -BYTE_BLOCK_SIZE;
+      buffers = new byte[10][];
+      buffer = null;
+    }
+  }
 
   public void reset() {
     if (bufferUpto != -1) {
@@ -115,7 +158,7 @@ public final class ByteBlockPool {
       buffer = buffers[0];
     }
   }
-
+  
   public void nextBuffer() {
     if (1+bufferUpto == buffers.length) {
       byte[][] newBuffers = new byte[ArrayUtil.oversize(buffers.length+1,

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=1124825&r1=1124824&r2=1124825&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 Thu May 19 14:22:54 2011
@@ -228,7 +228,7 @@ public final class BytesRefHash {
     lastCount = count;
     count = 0;
     if (resetPool) {
-      pool.reset();
+      pool.dropBuffersAndReset();
     }
     bytesStart = bytesStartArray.clear();
     if (lastCount != -1 && shrink(lastCount)) {