You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2018/12/04 18:04:57 UTC

[2/2] lucene-solr:branch_7x: SOLR-13032: harden MaxSizeAutoCommitTest to eliminate race conditions and eliminate use of arbitrary sleep

SOLR-13032: harden MaxSizeAutoCommitTest to eliminate race conditions and eliminate use of arbitrary sleep

(cherry picked from commit 868e005760e95b0e3632f9bc8cec57656aeddabc)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/b44ca8e8
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/b44ca8e8
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/b44ca8e8

Branch: refs/heads/branch_7x
Commit: b44ca8e8374b15d7c1fddd38adb6d8f58839a737
Parents: e8f5b1e
Author: Chris Hostetter <ho...@apache.org>
Authored: Tue Dec 4 10:44:05 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Tue Dec 4 10:44:31 2018 -0700

----------------------------------------------------------------------
 .../org/apache/solr/update/CommitTracker.java   |   5 +
 .../solr/update/MaxSizeAutoCommitTest.java      | 233 +++++++++++++------
 2 files changed, 165 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b44ca8e8/solr/core/src/java/org/apache/solr/update/CommitTracker.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/CommitTracker.java b/solr/core/src/java/org/apache/solr/update/CommitTracker.java
index d3929b2..55b2287 100644
--- a/solr/core/src/java/org/apache/solr/update/CommitTracker.java
+++ b/solr/core/src/java/org/apache/solr/update/CommitTracker.java
@@ -326,4 +326,9 @@ public final class CommitTracker implements Runnable {
   public void setOpenSearcher(boolean openSearcher) {
     this.openSearcher = openSearcher;
   }
+
+  // only for testing - not thread safe
+  public boolean hasPending() {
+    return (null != pending && !pending.isDone());
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b44ca8e8/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java b/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java
index 612e55b..36298b5 100644
--- a/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java
@@ -17,9 +17,13 @@
 
 package org.apache.solr.update;
 
+import java.lang.invoke.MethodHandles;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 
@@ -27,16 +31,24 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrEventListener;
+import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.handler.UpdateRequestHandler;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.response.SolrQueryResponse;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   // Given an ID, returns an XML string for an "add document" request
   private static final Function<Integer, String> ADD_DOC_FN = (id) -> adoc("id", Integer.toString(id));
@@ -44,11 +56,14 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
   private static final Function<Integer, String> DELETE_DOC_FN = (id) -> delI(Integer.toString(id));
   // How long to sleep while checking for commits
   private static final int COMMIT_CHECKING_SLEEP_TIME_MS = 50;
-
+  // max TLOG file size
+  private static final int MAX_FILE_SIZE = 1000;
+  
   private SolrCore core;
   private DirectUpdateHandler2 updateHandler;
   private CommitTracker hardCommitTracker;
   private UpdateRequestHandler updateRequestHandler;
+  private MockEventListener monitor;
 
   @Before
   public void setup() throws Exception {
@@ -56,63 +71,73 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
     initCore("solrconfig-tlog.xml", "schema.xml");
     core = h.getCore();
     updateHandler = (DirectUpdateHandler2) core.getUpdateHandler();
+
+    // we don't care about auto-commit's opening a new Searcher in this test, just skip it.
+    updateHandler.softCommitTracker.setOpenSearcher(false);
+    updateHandler.commitTracker.setOpenSearcher(false);
+
+    // we don't care about soft commit's at all
+    updateHandler.softCommitTracker.setTimeUpperBound(-1);
+    updateHandler.softCommitTracker.setDocsUpperBound(-1);
+    updateHandler.softCommitTracker.setTLogFileSizeUpperBound(-1);
+    
     hardCommitTracker = updateHandler.commitTracker;
     // Only testing file-size based auto hard commits - disable other checks
     hardCommitTracker.setTimeUpperBound(-1);
     hardCommitTracker.setDocsUpperBound(-1);
+    hardCommitTracker.setTLogFileSizeUpperBound(MAX_FILE_SIZE);
+
+    monitor = new MockEventListener();
+    updateHandler.registerCommitCallback(monitor);
+    
     updateRequestHandler = new UpdateRequestHandler();
     updateRequestHandler.init( null );
   }
 
   @After
   public void tearDown() throws Exception {
+    monitor.assertSaneOffers();
+    monitor.clear();
     super.tearDown();
     System.clearProperty("solr.ulog");
     deleteCore();
   }
 
   @Test
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13032")
   public void testAdds() throws Exception {
-    int maxFileSizeBound = 1000;
-    // Set max size bound
-    hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
-
-    // Add a large number of docs - should trigger a commit
-    int numDocsToAdd = 500;
-    SolrQueryResponse updateResp = new SolrQueryResponse();
 
     Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0,
-        hardCommitTracker.getCommitCount());
+                        hardCommitTracker.getCommitCount());
 
     long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
     Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates);
 
-    updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
-
-    // The long sleep is to allow for the triggered commit to finish
-    waitForCommit(1000);
-
-    // Verify commit information
-    Assert.assertTrue("At least one commit should have occurred", hardCommitTracker.getCommitCount() > 0);
-    long tlogSizePostUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
-    Assert.assertTrue("Current tlog size is larger than the max bound", tlogSizePostUpdates < maxFileSizeBound);
+    // Add a large number of docs - should trigger a commit
+    int numDocsToAdd = 500;
+    SolrQueryResponse updateResp = new SolrQueryResponse();
+    
+    monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
+        updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
+      });
   }
 
   @Test
   public void testRedundantDeletes() throws Exception {
-    int maxFileSizeBound = 1000;
-    // Set max size bound
-    hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
 
+    Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0,
+                        hardCommitTracker.getCommitCount());
+
+    long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
+    Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates);
+    
     // Add docs
     int numDocsToAdd = 150;
     SolrQueryResponse updateResp = new SolrQueryResponse();
-    updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
-    waitForCommit(1000);
 
-    // Get the current commit info
-    int commitCountPreDeletes = hardCommitTracker.getCommitCount();
+    monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
+        updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
+      });
+    
 
     // Send a bunch of redundant deletes
     int numDeletesToSend = 500;
@@ -124,63 +149,36 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
       docs.add(delI(Integer.toString(docIdToDelete)));
     }
     batchSingleDeleteRequest.setContentStreams(toContentStreams(docs));
-
-    updateRequestHandler.handleRequest(batchSingleDeleteRequest, updateResp);
-
-    // The long sleep is to allow for the expected triggered commit to finish
-    waitForCommit(1000);
-
-    // Verify commit information
-    Assert.assertTrue("At least one commit should have occurred",
-        hardCommitTracker.getCommitCount() > commitCountPreDeletes);
-    long tlogSizePostDeletes = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
-    Assert.assertTrue("Current tlog size is larger than the max bound", tlogSizePostDeletes < maxFileSizeBound);
+    
+    monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
+        updateRequestHandler.handleRequest(batchSingleDeleteRequest, updateResp);
+      });
+    
   }
 
   @Test
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13032")
   public void testDeletes() throws Exception {
-    int maxFileSizeBound = 1000;
 
-    // Set max size bound
-    hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
+    Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0,
+                        hardCommitTracker.getCommitCount());
 
+    long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
+    Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates);
+    
     // Add docs
     int numDocsToAdd = 500;
     SolrQueryResponse updateResp = new SolrQueryResponse();
-    updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
-    waitForCommit(1000);
-
-    // Get the current commit info
-    int commitCountPreDeletes = hardCommitTracker.getCommitCount();
-
+    
+    monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
+        updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
+      });
+    
     // Delete all documents - should trigger a commit
-    updateRequestHandler.handleRequest(constructBatchDeleteDocRequest(0, numDocsToAdd), updateResp);
-
-    // The long sleep is to allow for the expected triggered commit to finish
-    waitForCommit(1000);
-
-    // Verify commit information
-    Assert.assertTrue("At least one commit should have occurred",
-        hardCommitTracker.getCommitCount() > commitCountPreDeletes);
-    long tlogSizePostDeletes = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
-    Assert.assertTrue("Current tlog size is larger than the max bound", tlogSizePostDeletes < maxFileSizeBound);
-  }
-
-  /**
-   * Sleeps in increments of COMMIT_CHECKING_SLEEP_TIME_MS while checking to see if a commit completed. If it did,
-   * then return. If not, continue this cycle for at most the amount of time specified
-   * @param maxTotalWaitTimeMillis the max amount of time (in ms) to wait/check for a commit
-   */
-  private void waitForCommit(long maxTotalWaitTimeMillis) throws Exception {
-    long startTimeNanos = System.nanoTime();
-    long maxTotalWaitTimeNanos = TimeUnit.MILLISECONDS.toNanos(maxTotalWaitTimeMillis);
-    while (System.nanoTime() - startTimeNanos < maxTotalWaitTimeNanos) {
-      Thread.sleep(COMMIT_CHECKING_SLEEP_TIME_MS);
-      if (!updateHandler.getUpdateLog().hasUncommittedChanges()) {
-        return;
-      }
-    }
+    
+    monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
+        updateRequestHandler.handleRequest(constructBatchDeleteDocRequest(0, numDocsToAdd), updateResp);
+      });
+    
   }
 
   /**
@@ -232,4 +230,93 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
     }
     return streams;
   }
+
+  private static final class MockEventListener implements SolrEventListener {
+    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    
+    public MockEventListener() {
+      /* No-Op */
+    }
+    
+    // use capacity bound Queue just so we're sure we don't OOM 
+    public final BlockingQueue<Long> hard = new LinkedBlockingQueue<>(1000);
+    
+    // if non enpty, then at least one offer failed (queues full)
+    private StringBuffer fail = new StringBuffer();
+    
+    @Override
+    public void init(NamedList args) {}
+    
+    @Override
+    public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
+      // No-Op
+    }
+    
+    @Override
+    public void postCommit() {
+      Long now = System.nanoTime();
+      if (!hard.offer(now)) fail.append(", hardCommit @ " + now);
+    }
+    
+    @Override
+    public void postSoftCommit() {
+      // No-Op
+    }
+    
+    public void clear() {
+      hard.clear();
+      fail.setLength(0);
+    }
+
+    public void doStuffAndExpectAtLeastOneCommit(final CommitTracker commitTracker,
+                                                 final DirectUpdateHandler2 updateHandler,
+                                                 final Runnable stuff) throws InterruptedException {
+      assertSaneOffers();
+      
+      final int POLL_TIME = 5;
+      final TimeUnit POLL_UNIT = TimeUnit.SECONDS;
+      
+      final int preAutoCommitCount = commitTracker.getCommitCount();
+      log.info("Auto-Commit count prior to doing work: {}", preAutoCommitCount);
+      stuff.run();
+      log.info("Work Completed");
+      
+      int numIters = 0;
+      Long lastPostCommitTimeStampSeen = null;
+      final long startTimeNanos = System.nanoTime();
+      final long cutOffTime = startTimeNanos + TimeUnit.SECONDS.toNanos(300);
+      while (System.nanoTime() < cutOffTime) {
+        numIters++;
+        log.info("Polling at most {} {} for expected (post-)commit#{}", POLL_TIME, POLL_UNIT, numIters);
+        lastPostCommitTimeStampSeen = hard.poll(POLL_TIME, POLL_UNIT);
+        assertNotNull("(post-)commit#" + numIters + " didn't occur in allowed time frame",
+                      lastPostCommitTimeStampSeen);
+
+        synchronized (commitTracker) {
+          final int currentAutoCommitCount = commitTracker.getCommitCount() - preAutoCommitCount;
+          final long currentFileSize = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
+          if ((currentFileSize < MAX_FILE_SIZE) &&
+              (currentAutoCommitCount == numIters) &&
+              ( ! commitTracker.hasPending() )) {
+            // if all of these condiions are met, then we should be completely done
+            assertSaneOffers(); // last minute sanity check
+            return;
+          }
+          // else: log & loop...
+          log.info("(Auto-)commits triggered: {}; (post-)commits seen: {}; current tlog file size: {}",
+                   currentAutoCommitCount, numIters, currentFileSize);
+        }
+      }
+      
+      // if we didn't return already, then we ran out of time
+      fail("Exhausted cut off time polling for post-commit events (got " + numIters + ")");
+    }
+    
+    public void assertSaneOffers() {
+      assertEquals("Failure of MockEventListener" + fail.toString(), 
+                   0, fail.length());
+    }
+  }  
+  
 }
+