From commits-return-102939-archive-asf-public=cust-asf.ponee.io@lucene.apache.org Fri Aug 17 08:07:44 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id B24E1180627 for ; Fri, 17 Aug 2018 08:07:43 +0200 (CEST) Received: (qmail 21996 invoked by uid 500); 17 Aug 2018 06:07:42 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 21987 invoked by uid 99); 17 Aug 2018 06:07:42 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 17 Aug 2018 06:07:42 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8363FE0102; Fri, 17 Aug 2018 06:07:42 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: anshum@apache.org To: commits@lucene.apache.org Message-Id: <0da6251dd2a142bbaf1a99914acc8e91@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: lucene-solr:master: SOLR-12475: Fix failing MaxSizeAutoCommitTest Date: Fri, 17 Aug 2018 06:07:42 +0000 (UTC) Repository: lucene-solr Updated Branches: refs/heads/master 585ba164a -> 4309ae6f9 SOLR-12475: Fix failing MaxSizeAutoCommitTest Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/4309ae6f Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/4309ae6f Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/4309ae6f Branch: refs/heads/master Commit: 4309ae6f9fdb70d780ee08bfc32ec3a36e68389a Parents: 585ba16 Author: Anshum Gupta Authored: Thu Aug 16 23:03:27 2018 -0700 Committer: Anshum Gupta Committed: Thu Aug 16 23:05:21 2018 -0700 ---------------------------------------------------------------------- solr/CHANGES.txt | 2 + .../org/apache/solr/update/CommitTracker.java | 13 +- .../solr/update/MaxSizeAutoCommitTest.java | 268 ++++--------------- 3 files changed, 60 insertions(+), 223 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4309ae6f/solr/CHANGES.txt ---------------------------------------------------------------------- diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 7f851f4..29a5b52 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -243,6 +243,8 @@ Bug Fixes * SOLR-12668: Autoscaling trigger listeners should be executed in the order of their creation. (ab) +* SOLR-12475: Fix MaxSizeAutoCommitTest failures (Rupa Shankar, Anshum Gupta) + Optimizations ---------------------- http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4309ae6f/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 8f06d11..7da9651 100644 --- a/solr/core/src/java/org/apache/solr/update/CommitTracker.java +++ b/solr/core/src/java/org/apache/solr/update/CommitTracker.java @@ -18,6 +18,7 @@ package org.apache.solr.update; import java.lang.invoke.MethodHandles; +import java.util.Locale; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -282,12 +283,14 @@ public final class CommitTracker implements Runnable { @Override public String toString() { - if (timeUpperBound > 0 || docsUpperBound > 0) { - return (timeUpperBound > 0 ? ("if uncommited for " + timeUpperBound + "ms; ") + if (timeUpperBound > 0 || docsUpperBound > 0 || tLogFileSizeUpperBound > 0) { + return (timeUpperBound > 0 ? ("if uncommitted for " + timeUpperBound + "ms; ") : "") - + (docsUpperBound > 0 ? ("if " + docsUpperBound + " uncommited docs ") - : ""); - + + (docsUpperBound > 0 ? ("if " + docsUpperBound + " uncommitted docs; ") + : "") + + (tLogFileSizeUpperBound > 0 ? String.format(Locale.ROOT, "if tlog file size has exceeded %d bytes", + tLogFileSizeUpperBound) + : ""); } else { return "disabled"; } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4309ae6f/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 57390d9..33fd7de 100644 --- a/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java +++ b/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java @@ -17,25 +17,18 @@ package org.apache.solr.update; -import java.io.File; import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.function.Function; -import com.carrotsearch.randomizedtesting.annotations.Repeat; -import com.fasterxml.jackson.databind.ObjectMapper; 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.core.SolrCore; import org.apache.solr.handler.UpdateRequestHandler; -import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequestBase; import org.apache.solr.response.SolrQueryResponse; import org.junit.After; @@ -43,25 +36,22 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -//commented 2-Aug-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2018-06-18 public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 { // Given an ID, returns an XML string for an "add document" request private static final Function ADD_DOC_FN = (id) -> adoc("id", Integer.toString(id)); // Given an ID, returns an XML string for a "delete document" request private static final Function 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; - private ObjectMapper objectMapper; // for JSON parsing private SolrCore core; private DirectUpdateHandler2 updateHandler; private CommitTracker hardCommitTracker; private UpdateRequestHandler updateRequestHandler; - private String tlogDirPath; @Before public void setup() throws Exception { - objectMapper = new ObjectMapper(); - System.setProperty("solr.autoCommit.maxSize", "5k"); System.setProperty("solr.ulog", "solr.UpdateLog"); initCore("solrconfig-tlog.xml", "schema.xml"); core = h.getCore(); @@ -72,65 +62,45 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 { hardCommitTracker.setDocsUpperBound(-1); updateRequestHandler = new UpdateRequestHandler(); updateRequestHandler.init( null ); - tlogDirPath = core.getDataDir() + "/tlog"; } @After public void tearDown() throws Exception { super.tearDown(); - System.clearProperty("solr.autoCommit.maxSize"); System.clearProperty("solr.ulog"); deleteCore(); } @Test - public void simpleTest() throws Exception { + public void testAdds() throws Exception { int maxFileSizeBound = 1000; - int maxFileSizeBoundWithBuffer = (int) (maxFileSizeBound * 1.25); // Set max size bound hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound); - // Adding these docs will place the tlog size just under the threshold - int numDocs = 27; - int batchSize = 3; - int numBatches = numDocs / batchSize; + // Add a large number of docs - should trigger a commit + int numDocsToAdd = 500; SolrQueryResponse updateResp = new SolrQueryResponse(); - int numTlogs = -1; - TreeMap tlogsInfo = null; - for (int batchCounter = 0; batchCounter < numBatches; batchCounter++) { - int docStartId = batchSize * batchCounter; + Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0, + hardCommitTracker.getCommitCount()); - // Send batch update request - updateRequestHandler.handleRequest(constructBatchAddDocRequest(docStartId, batchSize), updateResp); + long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream(); + Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates); - // The sleep is to allow existing commits to finish (or at least mostly finish) before querying/submitting more documents - waitForCommit(200); - - // There should just be 1 tlog and its size should be within the (buffered) file size bound - tlogsInfo = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer); - numTlogs = parseTotalNumTlogs(tlogsInfo); - Assert.assertEquals(1, numTlogs); - } - - // Now that the core's tlog size is just under the threshold, one more update should induce a commit - int docStartId = batchSize * numBatches; - updateRequestHandler.handleRequest(constructBatchAddDocRequest(docStartId, batchSize), updateResp); - waitForCommit(200); + updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp); - // Verify that a commit happened. There should now be 2 tlogs, both of which are < maxFileSizeBound. - TreeMap tlogsInfoPostCommit = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer); - Assert.assertEquals(2, parseTotalNumTlogs(tlogsInfoPostCommit)); + // The long sleep is to allow for the triggered commit to finish + waitForCommit(1000); - // And the current tlog's size should be less than the previous tlog's size - Assert.assertTrue(tlogsInfoPostCommit.lastEntry().getValue() < tlogsInfo.lastEntry().getValue()); + // 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); } @Test public void testRedundantDeletes() throws Exception { int maxFileSizeBound = 1000; - int maxFileSizeBoundWithBuffer = (int) (maxFileSizeBound * 1.25); - // Set max size bound hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound); @@ -138,125 +108,73 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 { int numDocsToAdd = 150; SolrQueryResponse updateResp = new SolrQueryResponse(); updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp); - waitForCommit(200); + waitForCommit(1000); - // Get the tlog file info - TreeMap tlogsInfoPreDeletes = getTlogFileSizes(tlogDirPath); + // Get the current commit info + int commitCountPreDeletes = hardCommitTracker.getCommitCount(); // Send a bunch of redundant deletes - int numDeletesToSend = 5000; + int numDeletesToSend = 500; int docIdToDelete = 100; - SolrQueryRequestBase requestWithOneDelete = new SolrQueryRequestBase(core, new MapSolrParams(new HashMap())) {}; + SolrQueryRequestBase batchSingleDeleteRequest = new SolrQueryRequestBase(core, new MapSolrParams(new HashMap<>())) {}; List docs = new ArrayList<>(); - docs.add(delI(Integer.toString(docIdToDelete))); - - requestWithOneDelete.setContentStreams(toContentStreams(docs)); - for (int i = 0; i < numDeletesToSend; i++) { - if (i % 50 == 0) { - // Wait periodically to allow existing commits to finish before - // sending more delete requests - waitForCommit(200); - } - updateRequestHandler.handleRequest(requestWithOneDelete, updateResp); + docs.add(delI(Integer.toString(docIdToDelete))); } + batchSingleDeleteRequest.setContentStreams(toContentStreams(docs)); + + updateRequestHandler.handleRequest(batchSingleDeleteRequest, updateResp); - // Verify that new tlogs have been created, and that their sizes are as expected - TreeMap tlogsInfoPostDeletes = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer); - Assert.assertTrue(parseTotalNumTlogs(tlogsInfoPreDeletes) < parseTotalNumTlogs(tlogsInfoPostDeletes)); + // 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); } @Test - //commented 2-Aug-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 12-Jun-2018 - public void deleteTest() throws Exception { + public void testDeletes() throws Exception { int maxFileSizeBound = 1000; - int maxFileSizeBoundWithBuffer = (int) (maxFileSizeBound * 1.25); // Set max size bound hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound); // Add docs - int numDocsToAdd = 150; + int numDocsToAdd = 500; SolrQueryResponse updateResp = new SolrQueryResponse(); updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp); - waitForCommit(200); + waitForCommit(1000); - // Get the tlog file info - TreeMap tlogsInfoPreDeletes = getTlogFileSizes(tlogDirPath); + // Get the current commit info + int commitCountPreDeletes = hardCommitTracker.getCommitCount(); - // Delete documents (in batches, so we can allow commits to finish and new tlog files to be created) - int batchSize = 15; - int numBatches = numDocsToAdd / batchSize; - for (int batchCounter = 0; batchCounter < numBatches; batchCounter++) { - int docStartId = batchSize * batchCounter; - - // Send batch delete doc request - updateRequestHandler.handleRequest(constructBatchDeleteDocRequest(docStartId, batchSize), updateResp); - - // The sleep is to allow existing commits to finish before deleting more documents - waitForCommit(200); - } - - // Verify that the commit happened by seeing if a new tlog file was opened - TreeMap tlogsInfoPostDeletes = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer); - Assert.assertTrue(parseTotalNumTlogs(tlogsInfoPreDeletes) < parseTotalNumTlogs(tlogsInfoPostDeletes)); - } - - @Test - @Repeat(iterations = 5) - //commented 2-Aug-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 12-Jun-2018 - public void endToEndTest() throws Exception { - int maxFileSizeBound = 5000; - // Set max size bound - hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound); - - // Giving a 10% buffer for the max size bound - int maxFileSizeBoundWithBuffer = (int) (maxFileSizeBound * 1.1); - - SolrQueryRequest selectQuery = req("*:*"); - List docCounts = new ArrayList<>(); - - int numDocs = 1000; - int batchSize = 20; - int numBatches = numDocs / batchSize; - for (int batchCounter = 0; batchCounter < numBatches; batchCounter++) { - SolrQueryResponse updateResp = new SolrQueryResponse(); - int docStartId = batchSize * batchCounter; - - // Send batch add doc request - updateRequestHandler.handleRequest(constructBatchAddDocRequest(docStartId, batchSize), updateResp); - - // The sleep is to allow existing commits to finish before querying/submitting more documents - waitForCommit(200); - - // Check tlog file sizes - getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer); - - // See how many documents are currently visible. This should increase as more commits occur. - docCounts.add(queryCore(selectQuery)); - } + // Delete all documents - should trigger a commit + updateRequestHandler.handleRequest(constructBatchDeleteDocRequest(0, numDocsToAdd), updateResp); - // One final commit, after which all documents should be visible - CommitUpdateCommand commitUpdateCommand = new CommitUpdateCommand(req(), false); - updateHandler.commit(commitUpdateCommand); - waitForCommit(200); - docCounts.add(queryCore(selectQuery)); + // The long sleep is to allow for the expected triggered commit to finish + waitForCommit(1000); - // Evaluate the document counts - checkNumFoundDocuments(docCounts, numDocs); + // 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 50 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 + * 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(50); + Thread.sleep(COMMIT_CHECKING_SLEEP_TIME_MS); if (!updateHandler.getUpdateLog().hasUncommittedChanges()) { return; } @@ -264,32 +182,6 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 { } /** - * Returns the total number of tlogs that have been created for the core. - * - * The tlogs in a core's tlog directory are named: tlog.0000000000000000000, tlog.0000000000000000001, tlog.0000000000000000002, etc. - * Because old tlogs are periodically deleted, we can't just count the number of existing files. Instead, we take the - * highest ordering tlog file name (which would be the newest) and parse the extension. - * - * e.g if the most recently created tlog file is tlog.0000000000000000003, we know that this core has had 4 tlogs. - * - * @param tlogsInfo TreeMap of (tlog file name, tlog file size (in bytes)) pairs - * @return total number of tlogs created for this core - */ - private int parseTotalNumTlogs(TreeMap tlogsInfo) { - String mostRecentFileName = tlogsInfo.lastKey(); - int extensionDelimiterIndex = mostRecentFileName.lastIndexOf("."); - if (extensionDelimiterIndex == -1) { - throw new RuntimeException("Invalid tlog filename: " + mostRecentFileName); - } - String extension = mostRecentFileName.substring(extensionDelimiterIndex + 1); - try { - return Integer.parseInt(extension) + 1; - } catch (NumberFormatException e) { - throw new RuntimeException("Could not parse tlog filename: " + mostRecentFileName, e); - } - } - - /** * Construct a batch add document request with a series of very simple Solr docs with increasing IDs. * @param startId the document ID to begin with * @param batchSize the number of documents to include in the batch @@ -327,66 +219,6 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 { } /** - * Executes the given query - * @param query the query to execute - * @return the number of documents found - */ - public int queryCore(SolrQueryRequest query) throws Exception { - String responseStr = h.query(query); - try { - Map root = (Map) objectMapper.readValue(responseStr, Object.class); - Map rootResponse = (Map) root.get("response"); - return (int) rootResponse.get("numFound"); - } catch (Exception e) { - throw new RuntimeException("Unable to parse Solr query response", e); - } - } - - /** - * Checks the given list of document counts to make sure that they are increasing (as commits occur). - * @param numDocList list of the number of documents found in a given core. Ascending from oldest to newest - */ - private void checkNumFoundDocuments(List numDocList, int finalValue) { - long currentTotal = 0; - for (Integer count : numDocList) { - Assert.assertTrue(count >= currentTotal); - currentTotal = count; - } - Assert.assertEquals(finalValue, numDocList.get(numDocList.size() - 1).intValue()); - } - - - /** - * Goes through the given tlog directory and inspects each tlog. - * @param tlogDirPath tlog directory path - * @return a TreeMap of (tlog file name, tlog file size (in bytes)) pairs - */ - private TreeMap getTlogFileSizes(String tlogDirPath) { - return getTlogFileSizes(tlogDirPath, Integer.MAX_VALUE); - } - - /** - * Goes through the given tlog directory and inspects each tlog. Asserts that each tlog's size is <= the given max size bound. - * @param tlogDirPath tlog directory path - * @param maxSizeBound the max tlog size - * @return a TreeMap of (tlog file name, tlog file size (in bytes)) pairs - */ - private TreeMap getTlogFileSizes(String tlogDirPath, int maxSizeBound) { - File tlogDir = new File(tlogDirPath); - File[] tlogs = tlogDir.listFiles(); - TreeMap tlogInfo = new TreeMap<>(); - if (tlogs != null) { - for (File tlog : tlogs) { - String message = String.format(Locale.getDefault(), "Tlog size exceeds the max size bound. Tlog path: %s, tlog size: %d", - tlog.getPath(), tlog.length()); - Assert.assertTrue(message, tlog.length() <= maxSizeBound); - tlogInfo.put(tlog.getName(), tlog.length()); - } - } - return tlogInfo; - } - - /** * Convert the given list of strings into a list of streams, for Solr update requests * @param strs strings to convert into streams * @return list of streams