You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2010/05/11 16:58:03 UTC

svn commit: r943137 - in /lucene/dev/branches/branch_3x/lucene: backwards/src/test/org/apache/lucene/index/ src/java/org/apache/lucene/index/ src/test/org/apache/lucene/index/

Author: mikemccand
Date: Tue May 11 14:58:02 2010
New Revision: 943137

URL: http://svn.apache.org/viewvc?rev=943137&view=rev
Log:
LUCENE-2329: fix bug in mem tracking caused in merging back from trunk; clean up some dead code

Removed:
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/org/apache/lucene/index/TestByteSlices.java
Modified:
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ByteBlockPool.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IntBlockPool.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestByteSlices.java

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ByteBlockPool.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ByteBlockPool.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ByteBlockPool.java Tue May 11 14:58:02 2010
@@ -44,7 +44,7 @@ final class ByteBlockPool {
   abstract static class Allocator {
     abstract void recycleByteBlocks(byte[][] blocks, int start, int end);
     abstract void recycleByteBlocks(List<byte[]> blocks);
-    abstract byte[] getByteBlock(boolean trackAllocations);
+    abstract byte[] getByteBlock();
   }
 
   public byte[][] buffers = new byte[10][];
@@ -55,12 +55,10 @@ final class ByteBlockPool {
   public byte[] buffer;                              // Current head buffer
   public int byteOffset = -DocumentsWriter.BYTE_BLOCK_SIZE;          // Current head offset
 
-  private final boolean trackAllocations;
   private final Allocator allocator;
 
-  public ByteBlockPool(Allocator allocator, boolean trackAllocations) {
+  public ByteBlockPool(Allocator allocator) {
     this.allocator = allocator;
-    this.trackAllocations = trackAllocations;
   }
 
   public void reset() {
@@ -93,7 +91,7 @@ final class ByteBlockPool {
       System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
       buffers = newBuffers;
     }
-    buffer = buffers[1+bufferUpto] = allocator.getByteBlock(trackAllocations);
+    buffer = buffers[1+bufferUpto] = allocator.getByteBlock();
     bufferUpto++;
 
     byteUpto = 0;

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Tue May 11 14:58:02 2010
@@ -194,7 +194,7 @@ final class DocumentsWriter {
      */
     protected byte[] newBuffer(int size) {
       assert size == PER_DOC_BLOCK_SIZE;
-      return perDocAllocator.getByteBlock(false);
+      return perDocAllocator.getByteBlock();
     }
     
     /**
@@ -1255,18 +1255,12 @@ final class DocumentsWriter {
     
     /* Allocate another byte[] from the shared pool */
     @Override
-    byte[] getByteBlock(boolean trackAllocations) {
+    byte[] getByteBlock() {
       synchronized(DocumentsWriter.this) {
         final int size = freeByteBlocks.size();
         final byte[] b;
         if (0 == size) {
           b = new byte[blockSize];
-          // Always record a block allocated, even if
-          // trackAllocations is false.  This is necessary
-          // because this block will be shared between
-          // things that don't track allocations (term
-          // vectors) and things that do (freq/prox
-          // postings).
           numBytesUsed += blockSize;
         } else
           b = freeByteBlocks.remove(size-1);
@@ -1302,17 +1296,11 @@ final class DocumentsWriter {
   private ArrayList<int[]> freeIntBlocks = new ArrayList<int[]>();
 
   /* Allocate another int[] from the shared pool */
-  synchronized int[] getIntBlock(boolean trackAllocations) {
+  synchronized int[] getIntBlock() {
     final int size = freeIntBlocks.size();
     final int[] b;
     if (0 == size) {
       b = new int[INT_BLOCK_SIZE];
-      // Always record a block allocated, even if
-      // trackAllocations is false.  This is necessary
-      // because this block will be shared between
-      // things that don't track allocations (term
-      // vectors) and things that do (freq/prox
-      // postings).
       numBytesUsed += INT_BLOCK_SIZE*INT_NUM_BYTE;
     } else
       b = freeIntBlocks.remove(size-1);
@@ -1359,7 +1347,6 @@ final class DocumentsWriter {
     // because nothing that skips allocation tracking
     // (currently only term vectors) uses its own char
     // blocks.
-    numBytesUsed += CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
     return c;
   }
 

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IntBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IntBlockPool.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IntBlockPool.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/IntBlockPool.java Tue May 11 14:58:02 2010
@@ -28,11 +28,9 @@ final class IntBlockPool {
   public int intOffset = -DocumentsWriter.INT_BLOCK_SIZE;          // Current head offset
 
   final private DocumentsWriter docWriter;
-  final boolean trackAllocations;
 
-  public IntBlockPool(DocumentsWriter docWriter, boolean trackAllocations) {
+  public IntBlockPool(DocumentsWriter docWriter) {
     this.docWriter = docWriter;
-    this.trackAllocations = trackAllocations;
   }
 
   public void reset() {
@@ -55,7 +53,7 @@ final class IntBlockPool {
       System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
       buffers = newBuffers;
     }
-    buffer = buffers[1+bufferUpto] = docWriter.getIntBlock(trackAllocations);
+    buffer = buffers[1+bufferUpto] = docWriter.getIntBlock();
     bufferUpto++;
 
     intUpto = 0;

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java Tue May 11 14:58:02 2010
@@ -50,19 +50,6 @@ class ParallelPostingsArray {
     return newArray;
   }
 
-  final ParallelPostingsArray shrink(int targetSize, boolean doCopy) {
-    int shrinkSize = ArrayUtil.getShrinkSize(size, targetSize, bytesPerPosting());
-    if (shrinkSize != size) {
-      ParallelPostingsArray newArray = newInstance(targetSize);
-      if (doCopy) {
-        copyTo(newArray, targetSize);
-      }
-      return newArray;
-    } else {
-      return this;
-    }
-  }
-
   void copyTo(ParallelPostingsArray toArray, int numToCopy) {
     System.arraycopy(textStarts, 0, toArray.textStarts, 0, numToCopy);
     System.arraycopy(intStarts, 0, toArray.intStarts, 0, numToCopy);

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Tue May 11 14:58:02 2010
@@ -68,7 +68,6 @@ final class TermsHashPerField extends In
     fieldState = docInverterPerField.fieldState;
     this.consumer = perThread.consumer.addField(this, fieldInfo);
     initPostingsArray();
-    bytesUsed(postingsArray.size * postingsArray.bytesPerPosting());
 
     streamCount = consumer.getStreamCount();
     numPostingInt = 2*streamCount;
@@ -81,6 +80,7 @@ final class TermsHashPerField extends In
 
   private void initPostingsArray() {
     postingsArray = consumer.createPostingsArray(2);
+    bytesUsed(postingsArray.size * postingsArray.bytesPerPosting());
   }
 
   // sugar: just forwards to DW

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java Tue May 11 14:58:02 2010
@@ -46,8 +46,8 @@ final class TermsHashPerThread extends I
       primary = false;
     }
 
-    intPool = new IntBlockPool(termsHash.docWriter, termsHash.trackAllocations);
-    bytePool = new ByteBlockPool(termsHash.docWriter.byteBlockAllocator, termsHash.trackAllocations);
+    intPool = new IntBlockPool(termsHash.docWriter);
+    bytePool = new ByteBlockPool(termsHash.docWriter.byteBlockAllocator);
 
     if (nextTermsHash != null)
       nextPerThread = nextTermsHash.addThread(docInverterPerThread, this);

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestByteSlices.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestByteSlices.java?rev=943137&r1=943136&r2=943137&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestByteSlices.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestByteSlices.java Tue May 11 14:58:02 2010
@@ -27,7 +27,7 @@ public class TestByteSlices extends Luce
     
     /* Allocate another byte[] from the shared pool */
     @Override
-    synchronized byte[] getByteBlock(boolean trackAllocations) {
+    synchronized byte[] getByteBlock() {
       final int size = freeByteBlocks.size();
       final byte[] b;
       if (0 == size)
@@ -53,7 +53,7 @@ public class TestByteSlices extends Luce
   }
 
   public void testBasic() throws Throwable {
-    ByteBlockPool pool = new ByteBlockPool(new ByteBlockAllocator(), false);
+    ByteBlockPool pool = new ByteBlockPool(new ByteBlockAllocator());
 
     final int NUM_STREAM = 25;