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 2014/06/13 13:59:23 UTC

svn commit: r1602393 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/index/

Author: simonw
Date: Fri Jun 13 11:59:22 2014
New Revision: 1602393

URL: http://svn.apache.org/r1602393
Log:
LUCENE-5756: Implement Accountable from IndexWriter

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Fri Jun 13 11:59:22 2014
@@ -65,6 +65,9 @@ Changes in Backwards Compatibility Polic
 
 API Changes
 
+* LUCENE-5756: IndexWriter now implements Accountable and IW#ramSizeInBytes()
+  has been deprecated infavor of IW#ramBytesUsed() (Simon Willnauer)
+
 * LUCENE-5725: MoreLikeThis#like now accepts multiple values per field.
   The pre-existing method has been deprecated in favor of a variable arguments
   for the like text. (Alex Ksikes via Simon Willnauer)

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java Fri Jun 13 11:59:22 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.InfoStream;
 
@@ -50,7 +51,7 @@ import org.apache.lucene.util.InfoStream
  * track which BufferedDeletes packets to apply to any given
  * segment. */
 
-class BufferedUpdatesStream {
+class BufferedUpdatesStream implements Accountable {
 
   // TODO: maybe linked list?
   private final List<FrozenBufferedUpdates> updates = new ArrayList<>();
@@ -111,7 +112,8 @@ class BufferedUpdatesStream {
     return numTerms.get();
   }
 
-  public long bytesUsed() {
+  @Override
+  public long ramBytesUsed() {
     return bytesUsed.get();
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Jun 13 11:59:22 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.InfoStream;
 
 /**
@@ -94,7 +95,7 @@ import org.apache.lucene.util.InfoStream
  * or none") added to the index.
  */
 
-final class DocumentsWriter implements Closeable {
+final class DocumentsWriter implements Closeable, Accountable {
   private final Directory directory;
 
   private volatile boolean closed;
@@ -655,7 +656,12 @@ final class DocumentsWriter implements C
   private void putEvent(Event event) {
     events.add(event);
   }
-  
+
+  @Override
+  public long ramBytesUsed() {
+    return flushControl.ramBytesUsed();
+  }
+
   static final class ApplyDeletesEvent implements Event {
     static final Event INSTANCE = new ApplyDeletesEvent();
     private int instCount = 0;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java Fri Jun 13 11:59:22 2014
@@ -24,6 +24,7 @@ import java.util.concurrent.locks.Reentr
 import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
 import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -66,7 +67,7 @@ import org.apache.lucene.util.BytesRef;
  * will also not be added to its private deletes neither to the global deletes.
  * 
  */
-final class DocumentsWriterDeleteQueue {
+final class DocumentsWriterDeleteQueue implements Accountable {
 
   private volatile Node<?> tail;
   
@@ -448,8 +449,9 @@ final class DocumentsWriterDeleteQueue {
       globalBufferLock.unlock();
     }
   }
-  
-  public long bytesUsed() {
+
+  @Override
+  public long ramBytesUsed() {
     return globalBufferedUpdates.bytesUsed.get();
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Fri Jun 13 11:59:22 2014
@@ -19,14 +19,15 @@ package org.apache.lucene.index;
 
 import java.util.ArrayList;
 import java.util.IdentityHashMap;
-import java.util.List;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -42,7 +43,7 @@ import org.apache.lucene.util.ThreadInte
  * {@link IndexWriterConfig#getRAMPerThreadHardLimitMB()} to prevent address
  * space exhaustion.
  */
-final class DocumentsWriterFlushControl  {
+final class DocumentsWriterFlushControl implements Accountable {
 
   private final long hardMaxBytesPerDWPT;
   private long activeBytes = 0;
@@ -428,7 +429,12 @@ final class DocumentsWriterFlushControl 
   }
   
   public long getDeleteBytesUsed() {
-    return documentsWriter.deleteQueue.bytesUsed() + bufferedUpdatesStream.bytesUsed();
+    return documentsWriter.deleteQueue.ramBytesUsed() + bufferedUpdatesStream.ramBytesUsed();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return getDeleteBytesUsed() + netBytes();
   }
 
   synchronized int numFlushingDWPT() {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri Jun 13 11:59:22 2014
@@ -58,6 +58,7 @@ import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.store.MergeInfo;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants;
@@ -193,7 +194,7 @@ import org.apache.lucene.util.ThreadInte
  * referenced by the "front" of the index). For this, IndexFileDeleter
  * keeps track of the last non commit checkpoint.
  */
-public class IndexWriter implements Closeable, TwoPhaseCommit{
+public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
   
   private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
   
@@ -426,6 +427,12 @@ public class IndexWriter implements Clos
     return r;
   }
 
+  @Override
+  public final long ramBytesUsed() {
+    ensureOpen();
+    return docWriter.ramBytesUsed();
+  }
+
   /** Holds shared SegmentReader instances. IndexWriter uses
    *  SegmentReaders for 1) applying deletes, 2) doing
    *  merges, 3) handing out a real-time reader.  This pool
@@ -3251,7 +3258,7 @@ public class IndexWriter implements Clos
       }
       applyAllDeletesAndUpdates();
     } else if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.bytesUsed());
+      infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.ramBytesUsed());
     }
   }
   
@@ -3280,15 +3287,16 @@ public class IndexWriter implements Clos
     }
     bufferedUpdatesStream.prune(segmentInfos);
   }
-
-  /** Expert:  Return the total size of all index files currently cached in memory.
-   * Useful for size management with flushRamDocs()
-   */
-  public final long ramSizeInBytes() {
-    ensureOpen();
-    return docWriter.flushControl.netBytes() + bufferedUpdatesStream.bytesUsed();
-  }
   
+   /** Expert:  Return the total size of all index files currently cached in memory.
+    * Useful for size management with flushRamDocs()
+    * @deprecated use #ramBytesUsed() instead
+    */
+   @Deprecated
+   public final long ramSizeInBytes() {
+     return ramBytesUsed();
+   }
+
   // for testing only
   DocumentsWriter getDocsWriter() {
     boolean test = false;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java?rev=1602393&r1=1602392&r2=1602393&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java Fri Jun 13 11:59:22 2014
@@ -316,7 +316,7 @@ public class TestFlushByRamOrCountsPolic
         while (pendingDocs.decrementAndGet() > -1) {
           Document doc = docs.nextDoc();
           writer.addDocument(doc);
-          long newRamSize = writer.ramSizeInBytes();
+          long newRamSize = writer.ramBytesUsed();
           if (newRamSize != ramSize) {
             ramSize = newRamSize;
           }