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 2015/07/14 19:14:05 UTC

svn commit: r1691035 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/

Author: mikemccand
Date: Tue Jul 14 17:14:04 2015
New Revision: 1691035

URL: http://svn.apache.org/r1691035
Log:
LUCENE-6676: remove isActive boolean

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1691035&r1=1691034&r2=1691035&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Tue Jul 14 17:14:04 2015
@@ -266,23 +266,18 @@ final class DocumentsWriter implements C
   /** Returns how many documents were aborted. */
   private int abortThreadState(final ThreadState perThread) {
     assert perThread.isHeldByCurrentThread();
-    if (perThread.isActive()) { // we might be closed
-      if (perThread.isInitialized()) { 
-        try {
-          int abortedDocCount = perThread.dwpt.getNumDocsInRAM();
-          subtractFlushedNumDocs(abortedDocCount);
-          perThread.dwpt.abort();
-          return abortedDocCount;
-        } finally {
-          flushControl.doOnAbort(perThread);
-        }
-      } else {
+    if (perThread.isInitialized()) { 
+      try {
+        int abortedDocCount = perThread.dwpt.getNumDocsInRAM();
+        subtractFlushedNumDocs(abortedDocCount);
+        perThread.dwpt.abort();
+        return abortedDocCount;
+      } finally {
         flushControl.doOnAbort(perThread);
-        // This DWPT was never initialized so it has no indexed documents:
-        return 0;
       }
     } else {
-      assert closed;
+      flushControl.doOnAbort(perThread);
+      // This DWPT was never initialized so it has no indexed documents:
       return 0;
     }
   }
@@ -391,9 +386,8 @@ final class DocumentsWriter implements C
   }
   
   private void ensureInitialized(ThreadState state) throws IOException {
-    if (state.isActive() && state.dwpt == null) {
-      final FieldInfos.Builder infos = new FieldInfos.Builder(
-          writer.globalFieldNumberMap);
+    if (state.dwpt == null) {
+      final FieldInfos.Builder infos = new FieldInfos.Builder(writer.globalFieldNumberMap);
       state.dwpt = new DocumentsWriterPerThread(writer, writer.newSegmentName(), directoryOrig,
                                                 directory, config, infoStream, deleteQueue, infos,
                                                 writer.pendingNumDocs, writer.enableTestPoints);
@@ -408,10 +402,9 @@ final class DocumentsWriter implements C
     final DocumentsWriterPerThread flushingDWPT;
     
     try {
-      if (!perThread.isActive()) {
-        ensureOpen();
-        assert false: "perThread is not active but we are still open";
-      }
+      // This must happen after we've pulled the ThreadState because IW.close
+      // waits for all ThreadStates to be released:
+      ensureOpen();
       ensureInitialized(perThread);
       assert perThread.isInitialized();
       final DocumentsWriterPerThread dwpt = perThread.dwpt;
@@ -446,10 +439,9 @@ final class DocumentsWriter implements C
 
     final DocumentsWriterPerThread flushingDWPT;
     try {
-      if (!perThread.isActive()) {
-        ensureOpen();
-        assert false: "perThread is not active but we are still open";
-      }
+      // This must happen after we've pulled the ThreadState because IW.close
+      // waits for all ThreadStates to be released:
+      ensureOpen();
       ensureInitialized(perThread);
       assert perThread.isInitialized();
       final DocumentsWriterPerThread dwpt = perThread.dwpt;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1691035&r1=1691034&r2=1691035&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Tue Jul 14 17:14:04 2015
@@ -290,7 +290,7 @@ final class DocumentsWriterFlushControl
       }
       assert assertMemory();
       // Take it out of the loop this DWPT is stale
-      perThreadPool.reset(state, closed);
+      perThreadPool.reset(state);
     } finally {
       updateStallState();
     }
@@ -308,7 +308,7 @@ final class DocumentsWriterFlushControl
       assert fullFlush : "can not block if fullFlush == false";
       final DocumentsWriterPerThread dwpt;
       final long bytes = perThread.bytesUsed;
-      dwpt = perThreadPool.reset(perThread, closed);
+      dwpt = perThreadPool.reset(perThread);
       numPending--;
       blockedFlushes.add(new BlockedFlush(dwpt, bytes));
     } finally {
@@ -316,8 +316,7 @@ final class DocumentsWriterFlushControl
     }
   }
 
-  private DocumentsWriterPerThread internalTryCheckOutForFlush(
-      ThreadState perThread) {
+  private DocumentsWriterPerThread internalTryCheckOutForFlush(ThreadState perThread) {
     assert Thread.holdsLock(this);
     assert perThread.flushPending;
     try {
@@ -329,7 +328,7 @@ final class DocumentsWriterFlushControl
             final DocumentsWriterPerThread dwpt;
             final long bytes = perThread.bytesUsed; // do that before
                                                          // replace!
-            dwpt = perThreadPool.reset(perThread, closed);
+            dwpt = perThreadPool.reset(perThread);
             assert !flushingWriters.containsKey(dwpt) : "DWPT is already flushing";
             // Record the flushing DWPT to reduce flushBytes in doAfterFlush
             flushingWriters.put(dwpt, Long.valueOf(bytes));
@@ -381,9 +380,7 @@ final class DocumentsWriterFlushControl
 
   synchronized void setClosed() {
     // set by DW to signal that we should not release new DWPT after close
-    if (!closed) {
-      this.closed = true;
-    }
+    this.closed = true;
   }
 
   /**
@@ -499,9 +496,6 @@ final class DocumentsWriterFlushControl
       next.lock();
       try {
         if (!next.isInitialized()) {
-          if (closed && next.isActive()) {
-            perThreadPool.deactivateThreadState(next);
-          }
           continue; 
         }
         assert next.dwpt.deleteQueue == flushingQueue
@@ -571,7 +565,7 @@ final class DocumentsWriterFlushControl
         fullFlushBuffer.add(flushingDWPT);
       }
     } else {
-        perThreadPool.reset(perThread, closed); // make this state inactive
+      perThreadPool.reset(perThread); // make this state inactive
     }
   }
   

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1691035&r1=1691034&r2=1691035&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Tue Jul 14 17:14:04 2015
@@ -60,20 +60,11 @@ final class DocumentsWriterPerThreadPool
     // TODO this should really be part of DocumentsWriterFlushControl
     // write access guarded by DocumentsWriterFlushControl
     long bytesUsed = 0;
-    // guarded by Reentrant lock
-    private boolean isActive = true;
 
     ThreadState(DocumentsWriterPerThread dpwt) {
       this.dwpt = dpwt;
     }
     
-    /** Mark this ThreadState as inactive, setting dwpt to null.
-     * @see #isActive() */
-    private void deactivate() {
-      isActive = false;
-      reset();
-    }
-    
     private void reset() {
       assert this.isHeldByCurrentThread();
       this.dwpt = null;
@@ -81,19 +72,9 @@ final class DocumentsWriterPerThreadPool
       this.flushPending = false;
     }
     
-    /**
-     * Returns <code>true</code> if this ThreadState is still open. This will
-     * only return <code>false</code> iff the DW has been closed and this
-     * ThreadState is already checked out for flush.
-     */
-    boolean isActive() {
-      assert this.isHeldByCurrentThread();
-      return isActive;
-    }
-    
     boolean isInitialized() {
       assert this.isHeldByCurrentThread();
-      return isActive() && dwpt != null;
+      return dwpt != null;
     }
     
     /**
@@ -170,14 +151,10 @@ final class DocumentsWriterPerThreadPool
     return threadState;
   }
 
-  DocumentsWriterPerThread reset(ThreadState threadState, boolean closed) {
+  DocumentsWriterPerThread reset(ThreadState threadState) {
     assert threadState.isHeldByCurrentThread();
     final DocumentsWriterPerThread dwpt = threadState.dwpt;
-    if (!closed) {
-      threadState.reset();
-    } else {
-      threadState.deactivate();
-    }
+    threadState.reset();
     return dwpt;
   }
   
@@ -267,16 +244,4 @@ final class DocumentsWriterPerThreadPool
     }
     return minThreadState;
   }
-
-  /**
-   * Deactivates an active {@link ThreadState}. Inactive {@link ThreadState} can
-   * not be used for indexing anymore once they are deactivated. This method should only be used
-   * if the parent {@link DocumentsWriter} is closed or aborted.
-   * 
-   * @param threadState the state to deactivate
-   */
-  void deactivateThreadState(ThreadState threadState) {
-    assert threadState.isActive();
-    threadState.deactivate();
-  }
 }