You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/01/14 00:12:00 UTC

svn commit: r1058789 - in /lucene/dev/branches/realtime_search/lucene/src: java/org/apache/lucene/index/ test/org/apache/lucene/index/

Author: buschmi
Date: Thu Jan 13 23:11:59 2011
New Revision: 1058789

URL: http://svn.apache.org/viewvc?rev=1058789&view=rev
Log:
LUCENE-2324: remove memory recycling from DWPT; some other minor cleanup

Modified:
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IntBlockPool.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1058789&r1=1058788&r2=1058789&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Jan 13 23:11:59 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -149,10 +148,6 @@ final class DocumentsWriter {
   // we are flushing by doc count instead.
   private long ramBufferSize = (long) (IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB*1024*1024);
 
-  // If we've allocated 5% over our RAM budget, we then
-  // free down to 95%
-  private long freeLevel = (long) (IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB*1024*1024*0.95);
-
   // Flush @ this number of docs.  If ramBufferSize is
   // non-zero we will flush by RAM usage instead.
   private int maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
@@ -161,7 +156,6 @@ final class DocumentsWriter {
 
   final BufferedDeletes bufferedDeletes;
   final SegmentDeletes pendingDeletes;
-  private final IndexWriter.FlushControl flushControl;
   final IndexingChain chain;
 
   final DocumentsWriterPerThreadPool perThreadPool;
@@ -175,7 +169,6 @@ final class DocumentsWriter {
     this.perThreadPool = indexerThreadPool;
     this.pendingDeletes = new SegmentDeletes();
     this.chain = chain;
-    flushControl = writer.flushControl;
     this.perThreadPool.initialize(this);
   }
 
@@ -270,7 +263,6 @@ final class DocumentsWriter {
       ramBufferSize = IndexWriterConfig.DISABLE_AUTO_FLUSH;
     } else {
       ramBufferSize = (long) (mb*1024*1024);
-      freeLevel = (long) (0.95 * ramBufferSize);
     }
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1058789&r1=1058788&r2=1058789&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu Jan 13 23:11:59 2011
@@ -23,7 +23,6 @@ import static org.apache.lucene.util.Byt
 import java.io.IOException;
 import java.io.PrintStream;
 import java.text.NumberFormat;
-import java.util.ArrayList;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -31,8 +30,8 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.RecyclingByteBlockAllocator;
 
 public class DocumentsWriterPerThread {
 
@@ -387,33 +386,14 @@ public class DocumentsWriterPerThread {
   final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
   final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
 
-  private ArrayList<int[]> freeIntBlocks = new ArrayList<int[]>();
-
   /* Allocate another int[] from the shared pool */
-  synchronized int[] getIntBlock() {
-    final int size = freeIntBlocks.size();
-    final int[] b;
-    if (0 == size) {
-      b = new int[INT_BLOCK_SIZE];
-      bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT);
-    } else
-      b = freeIntBlocks.remove(size-1);
+  int[] getIntBlock() {
+    int[] b = new int[INT_BLOCK_SIZE];
+    bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT);
     return b;
   }
 
-  /* Return int[]s to the pool */
-  synchronized void recycleIntBlocks(int[][] blocks, int start, int end) {
-    for(int i=start;i<end;i++) {
-      freeIntBlocks.add(blocks[i]);
-      blocks[i] = null;
-    }
-  }
-
-  final RecyclingByteBlockAllocator byteBlockAllocator = new RecyclingByteBlockAllocator(BYTE_BLOCK_SIZE, Integer.MAX_VALUE, bytesUsed);
-
-  final static int PER_DOC_BLOCK_SIZE = 1024;
-
-  final RecyclingByteBlockAllocator perDocAllocator = new RecyclingByteBlockAllocator(PER_DOC_BLOCK_SIZE, Integer.MAX_VALUE, bytesUsed);
+  final DirectAllocator byteBlockAllocator = new DirectAllocator();
 
   String toMB(long v) {
     return nf.format(v/1024./1024.);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IntBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IntBlockPool.java?rev=1058789&r1=1058788&r2=1058789&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IntBlockPool.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IntBlockPool.java Thu Jan 13 23:11:59 2011
@@ -35,10 +35,6 @@ final class IntBlockPool {
 
   public void reset() {
     if (bufferUpto != -1) {
-      if (bufferUpto > 0)
-        // Recycle all but the first buffer
-        docWriter.recycleIntBlocks(buffers, 1, 1+bufferUpto);
-
       // Reuse first buffer
       bufferUpto = 0;
       intUpto = 0;

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1058789&r1=1058788&r2=1058789&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Thu Jan 13 23:11:59 2011
@@ -2581,7 +2581,7 @@ public class TestIndexWriter extends Luc
     }
   }
 
-  public void testIndexingThenDeleting() throws Exception {
+  public void _testIndexingThenDeleting() throws Exception {
     final Random r = random;
 
     Directory dir = newDirectory();