You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/09/28 08:38:56 UTC

[GitHub] [lucene-solr] dweiss commented on a change in pull request #1925: Cleanup DWPT state handling

dweiss commented on a change in pull request #1925:
URL: https://github.com/apache/lucene-solr/pull/1925#discussion_r495776845



##########
File path: lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
##########
@@ -2933,7 +2933,7 @@ public void testFlushLargestWriter() throws IOException, InterruptedException {
     int numRamDocs = w.numRamDocs();
     int numDocsInDWPT = largestNonPendingWriter.getNumDocsInRAM();
     assertTrue(w.flushNextBuffer());
-    assertTrue(largestNonPendingWriter.hasFlushed());
+    assertEquals(DocumentsWriterPerThread.State.FLUSHED, largestNonPendingWriter.getState());

Review comment:
       Wouldn't this be more pleasant to read?
   assertTrue largestNonPendingWriter.inState(DocumentsWriterPerThread.State.FLUSHED)?

##########
File path: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
##########
@@ -593,9 +568,61 @@ void unlock() {
   }
 
   /**
-   * Returns <code>true</code> iff this DWPT has been flushed
+   * Returns the DWPTs current state.
    */
-  boolean hasFlushed() {
-    return hasFlushed.get() == Boolean.TRUE;
+  State getState() {
+    return state;
   }
+
+  /**
+   * Transitions the DWPT to the given state of fails if the transition is invalid.
+   * @throws IllegalStateException if the given state can not be transitioned to.
+   */
+  synchronized void transitionTo(State state) {
+    if (state.canTransitionFrom(this.state) == false) {
+      throw new IllegalStateException("Can't transition from " + this.state + " to " + state);
+    }
+    assert state.mustHoldLock == false || isHeldByCurrentThread() : "illegal state: " + state + " lock is held: " + isHeldByCurrentThread();
+    this.state = state;
+  }
+
+  /**
+   * Internal DWPT State.
+   */
+  enum State {
+    /**
+     * Default states when a DWPT is initialized and ready to index documents.
+     */
+    ACTIVE(null, true),
+    /**
+     * The DWPT can still index documents but should be moved to FLUSHING state as soon as possible.
+     * Transitions to this state can be done concurrently while another thread is actively indexing into this DWPT.
+     */
+    FLUSH_PENDING(ACTIVE, false),
+    /**
+     * The DWPT should not receive any further documents and is current flushing or queued up for flushing.
+     */
+    FLUSHING(FLUSH_PENDING, true),
+    /**
+     * The DWPT has been flushed and is ready to be garbage collected.
+     */
+    FLUSHED(FLUSHING, false);
+
+    private final State previousState;
+    final boolean mustHoldLock; // only for asserts
+
+    State(State previousState, boolean mustHoldLock) {

Review comment:
       Now you only need a formal correctness solver for this Petri net and you'll be good. 👍 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org