You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2018/05/04 15:53:56 UTC

lucene-solr:branch_7x: SOLR-11277: Add auto hard commit setting based on tlog size (this closes #358)

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 27992e338 -> 6ec1198a5


SOLR-11277: Add auto hard commit setting based on tlog size (this closes #358)


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

Branch: refs/heads/branch_7x
Commit: 6ec1198a5144e73b47bc88cd79f534878cbcbef4
Parents: 27992e3
Author: Anshum Gupta <an...@apache.org>
Authored: Thu May 3 15:00:47 2018 -0700
Committer: Anshum Gupta <an...@apache.org>
Committed: Fri May 4 08:37:02 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../java/org/apache/solr/core/SolrConfig.java   |  45 ++-
 .../org/apache/solr/update/CommitTracker.java   |  66 ++-
 .../solr/update/DirectUpdateHandler2.java       |  45 ++-
 .../org/apache/solr/update/TransactionLog.java  |   7 +
 .../java/org/apache/solr/update/UpdateLog.java  |   7 +
 .../conf/bad-solrconfig-no-autocommit-tag.xml   |  52 +++
 .../solr/collection1/conf/solrconfig-tlog.xml   |   4 +
 .../test/org/apache/solr/core/TestConfig.java   |  36 ++
 .../solr/update/MaxSizeAutoCommitTest.java      | 398 +++++++++++++++++++
 10 files changed, 642 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6d64830..6eeb301 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -85,6 +85,8 @@ New Features
 
 * SOLR-11278: Add IgnoreLargeDocumentProcessFactory (Cao Manh Dat, David Smiley)
 
+* SOLR-11277: Add auto hard-commit settings based on tlog size (Rupa Shankar, Anshum Gupta)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index 6c67645..6bfa08d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -54,6 +54,7 @@ import java.util.regex.Pattern;
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.xpath.XPathConstants;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.util.Version;
@@ -448,6 +449,7 @@ public class SolrConfig extends Config implements MapSerializable {
     return new UpdateHandlerInfo(get("updateHandler/@class", null),
         getInt("updateHandler/autoCommit/maxDocs", -1),
         getInt("updateHandler/autoCommit/maxTime", -1),
+        convertHeapOptionStyleConfigStringToBytes(get("updateHandler/autoCommit/maxSize", "")),
         getBool("updateHandler/indexWriter/closeWaitsForMerges", true),
         getBool("updateHandler/autoCommit/openSearcher", true),
         getInt("updateHandler/autoSoftCommit/maxDocs", -1),
@@ -455,6 +457,44 @@ public class SolrConfig extends Config implements MapSerializable {
         getBool("updateHandler/commitWithin/softCommit", true));
   }
 
+  /**
+   * Converts a Java heap option-like config string to bytes. Valid suffixes are: 'k', 'm', 'g'
+   * (case insensitive). If there is no suffix, the default unit is bytes.
+   * For example, 50k = 50KB, 20m = 20MB, 4g = 4GB, 300 = 300 bytes
+   * @param configStr the config setting to parse
+   * @return the size, in bytes. -1 if the given config string is empty
+   */
+  protected static long convertHeapOptionStyleConfigStringToBytes(String configStr) {
+    if (configStr.isEmpty()) {
+      return -1;
+    }
+    long multiplier = 1;
+    String numericValueStr = configStr;
+    char suffix = Character.toLowerCase(configStr.charAt(configStr.length() - 1));
+    if (Character.isLetter(suffix)) {
+      if (suffix == 'k') {
+        multiplier = FileUtils.ONE_KB;
+      }
+      else if (suffix == 'm') {
+        multiplier = FileUtils.ONE_MB;
+      }
+      else if (suffix == 'g') {
+        multiplier = FileUtils.ONE_GB;
+      } else {
+        throw new RuntimeException("Invalid suffix. Valid suffixes are 'k' (KB), 'm' (MB), 'g' (G). "
+            + "No suffix means the amount is in bytes. ");
+      }
+      numericValueStr = configStr.substring(0, configStr.length() - 1);
+    }
+    try {
+      return Long.parseLong(numericValueStr) * multiplier;
+    } catch (NumberFormatException e) {
+      throw new RuntimeException("Invalid format. The config setting should be a long with an "
+          + "optional letter suffix. Valid suffixes are 'k' (KB), 'm' (MB), 'g' (G). "
+          + "No suffix means the amount is in bytes.");
+    }
+  }
+
   private void loadPluginInfo(SolrPluginInfo pluginInfo) {
     boolean requireName = pluginInfo.options.contains(REQUIRE_NAME);
     boolean requireClass = pluginInfo.options.contains(REQUIRE_CLASS);
@@ -631,6 +671,7 @@ public class SolrConfig extends Config implements MapSerializable {
     public final String className;
     public final int autoCommmitMaxDocs, autoCommmitMaxTime,
         autoSoftCommmitMaxDocs, autoSoftCommmitMaxTime;
+    public final long autoCommitMaxSizeBytes;
     public final boolean indexWriterCloseWaitsForMerges;
     public final boolean openSearcher;  // is opening a new searcher part of hard autocommit?
     public final boolean commitWithinSoftCommit;
@@ -638,12 +679,14 @@ public class SolrConfig extends Config implements MapSerializable {
     /**
      * @param autoCommmitMaxDocs       set -1 as default
      * @param autoCommmitMaxTime       set -1 as default
+     * @param autoCommitMaxSize        set -1 as default
      */
-    public UpdateHandlerInfo(String className, int autoCommmitMaxDocs, int autoCommmitMaxTime, boolean indexWriterCloseWaitsForMerges, boolean openSearcher,
+    public UpdateHandlerInfo(String className, int autoCommmitMaxDocs, int autoCommmitMaxTime, long autoCommitMaxSize, boolean indexWriterCloseWaitsForMerges, boolean openSearcher,
                              int autoSoftCommmitMaxDocs, int autoSoftCommmitMaxTime, boolean commitWithinSoftCommit) {
       this.className = className;
       this.autoCommmitMaxDocs = autoCommmitMaxDocs;
       this.autoCommmitMaxTime = autoCommmitMaxTime;
+      this.autoCommitMaxSizeBytes = autoCommitMaxSize;
       this.indexWriterCloseWaitsForMerges = indexWriterCloseWaitsForMerges;
       this.openSearcher = openSearcher;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/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 6cf7504..8f06d11 100644
--- a/solr/core/src/java/org/apache/solr/update/CommitTracker.java
+++ b/solr/core/src/java/org/apache/solr/update/CommitTracker.java
@@ -49,10 +49,13 @@ public final class CommitTracker implements Runnable {
   
   // scheduler delay for maxDoc-triggered autocommits
   public static final int DOC_COMMIT_DELAY_MS = 1;
+  // scheduler delay for maxSize-triggered autocommits
+  public static final int SIZE_COMMIT_DELAY_MS = 1;
   
   // settings, not final so we can change them in testing
   private int docsUpperBound;
   private long timeUpperBound;
+  private long tLogFileSizeUpperBound;
   
   private final ScheduledExecutorService scheduler = 
       Executors.newScheduledThreadPool(1, new DefaultSolrThreadFactory("commitScheduler"));
@@ -70,13 +73,15 @@ public final class CommitTracker implements Runnable {
 
   private String name;
   
-  public CommitTracker(String name, SolrCore core, int docsUpperBound, int timeUpperBound, boolean openSearcher, boolean softCommit) {
+  public CommitTracker(String name, SolrCore core, int docsUpperBound, int timeUpperBound, long tLogFileSizeUpperBound,
+                       boolean openSearcher, boolean softCommit) {
     this.core = core;
     this.name = name;
     pending = null;
     
     this.docsUpperBound = docsUpperBound;
     this.timeUpperBound = timeUpperBound;
+    this.tLogFileSizeUpperBound = tLogFileSizeUpperBound;
     
     this.softCommit = softCommit;
     this.openSearcher = openSearcher;
@@ -154,9 +159,34 @@ public final class CommitTracker implements Runnable {
   
   /**
    * Indicate that documents have been added
+   * @param commitWithin amount of time (in ms) within which a commit should be scheduled
    */
   public void addedDocument(int commitWithin) {
-    // maxDocs-triggered autoCommit.  Use == instead of > so we only trigger once on the way up
+    addedDocument(commitWithin, -1);
+  }
+
+  /**
+   * Indicate that documents have been added
+   * @param commitWithin amount of time (in ms) within which a commit should be scheduled
+   * @param currentTlogSize current tlog size (in bytes). Use -1 if we don't want to check for a max size triggered commit
+   */
+  public void addedDocument(int commitWithin, long currentTlogSize) {
+    // maxDocs-triggered autoCommit
+    _scheduleMaxDocsTriggeredCommitIfNeeded();
+
+    // maxTime-triggered autoCommit
+    _scheduleCommitWithinIfNeeded(commitWithin);
+
+    // maxSize-triggered autoCommit
+    _scheduleMaxSizeTriggeredCommitIfNeeded(currentTlogSize);
+  }
+
+  /**
+   * If a doc size upper bound is set, and the current number of documents has exceeded it, then
+   * schedule a commit and reset the counter
+   */
+  private void _scheduleMaxDocsTriggeredCommitIfNeeded() {
+    // Use == instead of > so we only trigger once on the way up
     if (docsUpperBound > 0) {
       long docs = docsSinceCommit.incrementAndGet();
       if (docs == docsUpperBound + 1) {
@@ -165,9 +195,6 @@ public final class CommitTracker implements Runnable {
         _scheduleCommitWithin(DOC_COMMIT_DELAY_MS);
       }
     }
-    
-    // maxTime-triggered autoCommit
-    _scheduleCommitWithinIfNeeded(commitWithin);
   }
   
   /** 
@@ -176,6 +203,26 @@ public final class CommitTracker implements Runnable {
   public void deletedDocument( int commitWithin ) {
     _scheduleCommitWithinIfNeeded(commitWithin);
   }
+
+  /**
+   * If the given current tlog size is greater than the file size upper bound, then schedule a commit
+   * @param currentTlogSize current tlog size (in bytes)
+   */
+  public void scheduleMaxSizeTriggeredCommitIfNeeded(long currentTlogSize) {
+    _scheduleMaxSizeTriggeredCommitIfNeeded(currentTlogSize);
+  }
+
+  /**
+   * If the given current tlog size is greater than the file size upper bound, then schedule a commit
+   * @param currentTlogSize current tlog size (in bytes)
+   */
+  private void _scheduleMaxSizeTriggeredCommitIfNeeded(long currentTlogSize) {
+    if (tLogFileSizeUpperBound > 0 && currentTlogSize > tLogFileSizeUpperBound) {
+      docsSinceCommit.set(0);
+      _scheduleCommitWithin(SIZE_COMMIT_DELAY_MS);
+    }
+  }
+
   
   /** Inform tracker that a commit has occurred */
   public void didCommit() {
@@ -254,6 +301,10 @@ public final class CommitTracker implements Runnable {
     return docsUpperBound;
   }
 
+  long getTLogFileSizeUpperBound() {
+    return tLogFileSizeUpperBound;
+  }
+
   void setDocsUpperBound(int docsUpperBound) {
     this.docsUpperBound = docsUpperBound;
   }
@@ -262,6 +313,11 @@ public final class CommitTracker implements Runnable {
   public void setTimeUpperBound(long timeUpperBound) {
     this.timeUpperBound = timeUpperBound;
   }
+
+  // only for testing - not thread safe
+  public void setTLogFileSizeUpperBound(int sizeUpperBound) {
+    this.tLogFileSizeUpperBound = sizeUpperBound;
+  }
   
   // only for testing - not thread safe
   public void setOpenSearcher(boolean openSearcher) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index 75453e1..0e9052f 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -73,6 +73,9 @@ import org.slf4j.LoggerFactory;
  * directly to the main Lucene index as opposed to adding to a separate smaller index.
  */
 public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState.IndexWriterCloser, SolrMetricProducer {
+
+  private static final int NO_FILE_SIZE_UPPER_BOUND_PLACEHOLDER = -1;
+
   protected final SolrCoreState solrCoreState;
 
   // stats
@@ -118,13 +121,14 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     
     UpdateHandlerInfo updateHandlerInfo = core.getSolrConfig()
         .getUpdateHandlerInfo();
-    int docsUpperBound = updateHandlerInfo.autoCommmitMaxDocs; // getInt("updateHandler/autoCommit/maxDocs", -1);
-    int timeUpperBound = updateHandlerInfo.autoCommmitMaxTime; // getInt("updateHandler/autoCommit/maxTime", -1);
-    commitTracker = new CommitTracker("Hard", core, docsUpperBound, timeUpperBound, updateHandlerInfo.openSearcher, false);
+    int docsUpperBound = updateHandlerInfo.autoCommmitMaxDocs;
+    int timeUpperBound = updateHandlerInfo.autoCommmitMaxTime;
+    long fileSizeUpperBound = updateHandlerInfo.autoCommitMaxSizeBytes;
+    commitTracker = new CommitTracker("Hard", core, docsUpperBound, timeUpperBound, fileSizeUpperBound, updateHandlerInfo.openSearcher, false);
     
-    int softCommitDocsUpperBound = updateHandlerInfo.autoSoftCommmitMaxDocs; // getInt("updateHandler/autoSoftCommit/maxDocs", -1);
-    int softCommitTimeUpperBound = updateHandlerInfo.autoSoftCommmitMaxTime; // getInt("updateHandler/autoSoftCommit/maxTime", -1);
-    softCommitTracker = new CommitTracker("Soft", core, softCommitDocsUpperBound, softCommitTimeUpperBound, true, true);
+    int softCommitDocsUpperBound = updateHandlerInfo.autoSoftCommmitMaxDocs;
+    int softCommitTimeUpperBound = updateHandlerInfo.autoSoftCommmitMaxTime;
+    softCommitTracker = new CommitTracker("Soft", core, softCommitDocsUpperBound, softCommitTimeUpperBound, NO_FILE_SIZE_UPPER_BOUND_PLACEHOLDER, true, true);
     
     commitWithinSoftCommit = updateHandlerInfo.commitWithinSoftCommit;
     indexWriterCloseWaitsForMerges = updateHandlerInfo.indexWriterCloseWaitsForMerges;
@@ -143,13 +147,14 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     
     UpdateHandlerInfo updateHandlerInfo = core.getSolrConfig()
         .getUpdateHandlerInfo();
-    int docsUpperBound = updateHandlerInfo.autoCommmitMaxDocs; // getInt("updateHandler/autoCommit/maxDocs", -1);
-    int timeUpperBound = updateHandlerInfo.autoCommmitMaxTime; // getInt("updateHandler/autoCommit/maxTime", -1);
-    commitTracker = new CommitTracker("Hard", core, docsUpperBound, timeUpperBound, updateHandlerInfo.openSearcher, false);
+    int docsUpperBound = updateHandlerInfo.autoCommmitMaxDocs;
+    int timeUpperBound = updateHandlerInfo.autoCommmitMaxTime;
+    long fileSizeUpperBound = updateHandlerInfo.autoCommitMaxSizeBytes;
+    commitTracker = new CommitTracker("Hard", core, docsUpperBound, timeUpperBound, fileSizeUpperBound, updateHandlerInfo.openSearcher, false);
     
-    int softCommitDocsUpperBound = updateHandlerInfo.autoSoftCommmitMaxDocs; // getInt("updateHandler/autoSoftCommit/maxDocs", -1);
-    int softCommitTimeUpperBound = updateHandlerInfo.autoSoftCommmitMaxTime; // getInt("updateHandler/autoSoftCommit/maxTime", -1);
-    softCommitTracker = new CommitTracker("Soft", core, softCommitDocsUpperBound, softCommitTimeUpperBound, updateHandlerInfo.openSearcher, true);
+    int softCommitDocsUpperBound = updateHandlerInfo.autoSoftCommmitMaxDocs;
+    int softCommitTimeUpperBound = updateHandlerInfo.autoSoftCommmitMaxTime;
+    softCommitTracker = new CommitTracker("Soft", core, softCommitDocsUpperBound, softCommitTimeUpperBound, NO_FILE_SIZE_UPPER_BOUND_PLACEHOLDER, updateHandlerInfo.openSearcher, true);
     
     commitWithinSoftCommit = updateHandlerInfo.commitWithinSoftCommit;
     indexWriterCloseWaitsForMerges = updateHandlerInfo.indexWriterCloseWaitsForMerges;
@@ -178,6 +183,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       manager.registerGauge(this, registryName, () -> "" + commitTracker.getTimeUpperBound() + "ms", tag, true, "autoCommitMaxTime",
           getCategory().toString(), scope);
     }
+    if (commitTracker.getTLogFileSizeUpperBound() > 0) {
+      manager.registerGauge(this, registryName, () -> commitTracker.getTLogFileSizeUpperBound(), tag, true, "autoCommitMaxSize",
+          getCategory().toString(), scope);
+    }
     if (softCommitTracker.getDocsUpperBound() > 0) {
       manager.registerGauge(this, registryName, () -> softCommitTracker.getDocsUpperBound(), tag, true, "softAutoCommitMaxDocs",
           getCategory().toString(), scope);
@@ -279,12 +288,13 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       }
 
       if ((cmd.getFlags() & UpdateCommand.IGNORE_AUTOCOMMIT) == 0) {
+        long currentTlogSize = getCurrentTLogSize();
         if (commitWithinSoftCommit) {
-          commitTracker.addedDocument(-1);
+          commitTracker.addedDocument(-1, currentTlogSize);
           softCommitTracker.addedDocument(cmd.commitWithin);
         } else {
           softCommitTracker.addedDocument(-1);
-          commitTracker.addedDocument(cmd.commitWithin);
+          commitTracker.addedDocument(cmd.commitWithin, currentTlogSize);
         }
       }
 
@@ -418,6 +428,9 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       if (commitTracker.getTimeUpperBound() > 0) {
         commitTracker.scheduleCommitWithin(commitTracker.getTimeUpperBound());
       }
+
+      long currentTlogSize = getCurrentTLogSize();
+      commitTracker.scheduleMaxSizeTriggeredCommitIfNeeded(currentTlogSize);
       
       if (softCommitTracker.getTimeUpperBound() > 0) {
         softCommitTracker.scheduleCommitWithin(softCommitTracker
@@ -990,6 +1003,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     return solrCoreState;
   }
 
+  private long getCurrentTLogSize() {
+    return ulog != null && ulog.hasUncommittedChanges() ? ulog.getCurrentLogSizeFromStream() : -1;
+  }
+
   // allow access for tests
   public CommitTracker getCommitTracker() {
     return commitTracker;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/java/org/apache/solr/update/TransactionLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/TransactionLog.java b/solr/core/src/java/org/apache/solr/update/TransactionLog.java
index be4dabc..96a928c 100644
--- a/solr/core/src/java/org/apache/solr/update/TransactionLog.java
+++ b/solr/core/src/java/org/apache/solr/update/TransactionLog.java
@@ -630,6 +630,13 @@ public class TransactionLog implements Closeable {
     return 0;
   }
 
+  /**
+   * @return the FastOutputStream size
+   */
+  public synchronized long getLogSizeFromStream() {
+    return fos.size();
+  }
+
   /** Returns a reader that can be used while a log is still in use.
    * Currently only *one* LogReader may be outstanding, and that log may only
    * be used from a single thread. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index fbdf616..09ff146 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -292,6 +292,13 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
     return size;
   }
 
+  /**
+   * @return the current transaction log's size (based on its output stream)
+   */
+  public long getCurrentLogSizeFromStream() {
+    return tlog.getLogSizeFromStream();
+  }
+
   public long getTotalLogsNumber() {
     synchronized (this) {
       return logs.size();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-no-autocommit-tag.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-no-autocommit-tag.xml b/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-no-autocommit-tag.xml
new file mode 100644
index 0000000..1040db6
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-no-autocommit-tag.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- Minimal solrconfig.xml with basic autoCommit settings, but without a valid "autoCommit" tag, to test
+ autoCommit-related defaults -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+
+    <!-- autocommit pending docs if certain criteria are met -->
+    <!-- <autoCommit> -->
+      <maxSize>5k</maxSize>
+    <!-- </autoCommit> -->
+
+    <updateLog class="${solr.ulog:solr.UpdateLog}"></updateLog>
+  </updateHandler>
+
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
index c8884c6..989023d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
@@ -54,6 +54,10 @@
   </peerSync>
 
   <updateHandler class="solr.DirectUpdateHandler2">
+    <!-- autocommit pending docs if certain criteria are met -->
+    <autoCommit>
+      <maxSize>${solr.autoCommit.maxSize:}</maxSize>
+    </autoCommit>
     <updateLog class="${solr.tests.ulog:solr.UpdateLog}">
       <str name="dir">${solr.ulog.dir:}</str>
       <str name="maxNumLogsToKeep">${solr.ulog.maxNumLogsToKeep:10}</str>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/solr/core/src/test/org/apache/solr/core/TestConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfig.java b/solr/core/src/test/org/apache/solr/core/TestConfig.java
index 5a7b706..aa43c25 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfig.java
@@ -30,6 +30,7 @@ import org.apache.solr.handler.admin.ShowFileRequestHandler;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.update.SolrIndexConfig;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.w3c.dom.Node;
@@ -181,6 +182,41 @@ public class TestConfig extends SolrTestCaseJ4 {
     assertEquals("numDefaultsTested vs. numDefaultsMapped+numNullDefaults ="+sic.toMap(new LinkedHashMap<>()).keySet(), numDefaultsTested, numDefaultsMapped+numNullDefaults);
   }
 
+  @Test
+  public void testConvertAutoCommitMaxSizeStringToBytes() {
+
+    // Valid values
+    Assert.assertEquals(300, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300"));
+    Assert.assertEquals(307200, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300k"));
+    Assert.assertEquals(307200, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300K"));
+    Assert.assertEquals(314572800, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300m"));
+    Assert.assertEquals(314572800, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300M"));
+    Assert.assertEquals(322122547200L, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300g"));
+    Assert.assertEquals(322122547200L, SolrConfig.convertHeapOptionStyleConfigStringToBytes("300G"));
+    Assert.assertEquals(-1, SolrConfig.convertHeapOptionStyleConfigStringToBytes(""));
+
+    // Invalid values
+    try {
+      SolrConfig.convertHeapOptionStyleConfigStringToBytes("3jbk32k"); // valid suffix but non-numeric prefix
+      Assert.fail();
+    } catch (RuntimeException e) {
+      Assert.assertTrue(e.getMessage().contains("Invalid"));
+    }
+    try {
+      SolrConfig.convertHeapOptionStyleConfigStringToBytes("300x"); // valid prefix but invalid suffix
+      Assert.fail();
+    } catch (RuntimeException e) {
+      Assert.assertTrue(e.getMessage().contains("Invalid"));
+    }
+  }
+
+  @Test
+  public void testMaxSizeSettingWithoutAutoCommit() throws Exception {
+    SolrConfig solrConfig = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "bad-solrconfig-no-autocommit-tag.xml", null);
+    Assert.assertEquals(-1, solrConfig.getUpdateHandlerInfo().autoCommitMaxSizeBytes);
+    Assert.assertEquals(-1, solrConfig.getUpdateHandlerInfo().autoCommmitMaxDocs);
+    Assert.assertEquals(-1, solrConfig.getUpdateHandlerInfo().autoCommmitMaxTime);
+  }
 
   // sanity check that sys propertis are working as expected
   public void testSanityCheckTestSysPropsAreUsed() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ec1198a/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
new file mode 100644
index 0000000..0db3616
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/MaxSizeAutoCommitTest.java
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
+
+  // 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));
+  // Given an ID, returns an XML string for a "delete document" request
+  private static final Function<Integer, String> DELETE_DOC_FN = (id) -> delI(Integer.toString(id));
+
+  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();
+    updateHandler = (DirectUpdateHandler2) core.getUpdateHandler();
+    hardCommitTracker = updateHandler.commitTracker;
+    // Only testing file-size based auto hard commits - disable other checks
+    hardCommitTracker.setTimeUpperBound(-1);
+    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 {
+    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;
+    SolrQueryResponse updateResp = new SolrQueryResponse();
+    int numTlogs = -1;
+    TreeMap<String, Long> tlogsInfo = null;
+
+    for (int batchCounter = 0; batchCounter < numBatches; batchCounter++) {
+      int docStartId = batchSize * batchCounter;
+
+      // Send batch update request
+      updateRequestHandler.handleRequest(constructBatchAddDocRequest(docStartId, batchSize), updateResp);
+
+      // 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);
+
+    // Verify that a commit happened. There should now be 2 tlogs, both of which are < maxFileSizeBound.
+    TreeMap<String, Long> tlogsInfoPostCommit = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer);
+    Assert.assertEquals(2, parseTotalNumTlogs(tlogsInfoPostCommit));
+
+    // And the current tlog's size should be less than the previous tlog's size
+    Assert.assertTrue(tlogsInfoPostCommit.lastEntry().getValue() < tlogsInfo.lastEntry().getValue());
+  }
+
+  @Test
+  public void testRedundantDeletes() throws Exception {
+    int maxFileSizeBound = 1000;
+    int maxFileSizeBoundWithBuffer = (int) (maxFileSizeBound * 1.25);
+
+    // Set max size bound
+    hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
+
+    // Add docs
+    int numDocsToAdd = 150;
+    SolrQueryResponse updateResp = new SolrQueryResponse();
+    updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
+    waitForCommit(200);
+
+    // Get the tlog file info
+    TreeMap<String, Long> tlogsInfoPreDeletes = getTlogFileSizes(tlogDirPath);
+
+    // Send a bunch of redundant deletes
+    int numDeletesToSend = 5000;
+    int docIdToDelete = 100;
+
+    SolrQueryRequestBase requestWithOneDelete = new SolrQueryRequestBase(core, new MapSolrParams(new HashMap<String, String>())) {};
+    List<String> 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);
+    }
+
+    // Verify that new tlogs have been created, and that their sizes are as expected
+    TreeMap<String, Long> tlogsInfoPostDeletes = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer);
+    Assert.assertTrue(parseTotalNumTlogs(tlogsInfoPreDeletes) < parseTotalNumTlogs(tlogsInfoPostDeletes));
+  }
+
+  @Test
+  public void deleteTest() throws Exception {
+    int maxFileSizeBound = 1000;
+    int maxFileSizeBoundWithBuffer = (int) (maxFileSizeBound * 1.25);
+
+    // Set max size bound
+    hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
+
+    // Add docs
+    int numDocsToAdd = 150;
+    SolrQueryResponse updateResp = new SolrQueryResponse();
+    updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
+    waitForCommit(200);
+
+    // Get the tlog file info
+    TreeMap<String, Long> tlogsInfoPreDeletes = getTlogFileSizes(tlogDirPath);
+
+    // 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<String, Long> tlogsInfoPostDeletes = getTlogFileSizes(tlogDirPath, maxFileSizeBoundWithBuffer);
+    Assert.assertTrue(parseTotalNumTlogs(tlogsInfoPreDeletes) < parseTotalNumTlogs(tlogsInfoPostDeletes));
+  }
+  
+  @Test
+  @Repeat(iterations = 5)
+  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<Integer> 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));
+    }
+
+    // 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));
+
+    // Evaluate the document counts
+    checkNumFoundDocuments(docCounts, numDocs);
+  }
+
+  /**
+   * 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
+   * @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);
+      if (!updateHandler.getUpdateLog().hasUncommittedChanges()) {
+        return;
+      }
+    }
+  }
+
+  /**
+   * 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<String, Long> 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
+   * @return a SolrQueryRequestBase
+   */
+  private SolrQueryRequestBase constructBatchAddDocRequest(int startId, int batchSize) {
+    return constructBatchRequestHelper(startId, batchSize, ADD_DOC_FN);
+  }
+
+  /**
+   * Construct a batch delete document request, with IDs incrementing from startId
+   * @param startId the document ID to begin with
+   * @param batchSize the number of documents to include in the batch
+   * @return a SolrQueryRequestBase
+   */
+  private SolrQueryRequestBase constructBatchDeleteDocRequest(int startId, int batchSize) {
+    return constructBatchRequestHelper(startId, batchSize, DELETE_DOC_FN);
+  }
+
+  /**
+   * Helper for constructing a batch update request
+   * @param startId the document ID to begin with
+   * @param batchSize the number of documents to include in the batch
+   * @param requestFn a function that takes an (int) ID and returns an XML string of the request to add to the batch request
+   * @return a SolrQueryRequestBase
+   */
+  private SolrQueryRequestBase constructBatchRequestHelper(int startId, int batchSize, Function<Integer, String> requestFn) {
+    SolrQueryRequestBase updateReq = new SolrQueryRequestBase(core, new MapSolrParams(new HashMap<>())) {};
+    List<String> docs = new ArrayList<>();
+    for (int i = startId; i < startId + batchSize; i++) {
+      docs.add(requestFn.apply(i));
+    }
+    updateReq.setContentStreams(toContentStreams(docs));
+    return updateReq;
+  }
+
+  /**
+   * 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<String, Object> root = (Map<String, Object>) objectMapper.readValue(responseStr, Object.class);
+      Map<String, Object> rootResponse = (Map<String, Object>) 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<Integer> 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<String, Long> 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<String, Long> getTlogFileSizes(String tlogDirPath, int maxSizeBound) {
+    File tlogDir = new File(tlogDirPath);
+    File[] tlogs = tlogDir.listFiles();
+    TreeMap<String, Long> 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
+   */
+  private List<ContentStream> toContentStreams(List<String> strs) {
+    ArrayList<ContentStream> streams = new ArrayList<>();
+    for (String str : strs) {
+      streams.addAll(ClientUtils.toContentStreams(str, "text/xml"));
+    }
+    return streams;
+  }
+}