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

[01/34] lucene-solr:jira/http2: SOLR-12791: Add Metrics reporting for AuthenticationPlugin

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/http2 9d421c650 -> c0af83e45


SOLR-12791: Add Metrics reporting for AuthenticationPlugin


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

Branch: refs/heads/jira/http2
Commit: ef2f0cd88c6eb4b662aea06eaeb3b933288b23eb
Parents: 280f679
Author: Jan Høydahl <ja...@apache.org>
Authored: Wed Dec 12 11:33:32 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Wed Dec 12 11:33:32 2018 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/core/CoreContainer.java     |   6 +
 .../java/org/apache/solr/core/SolrInfoBean.java |   2 +-
 .../solr/security/AuthenticationPlugin.java     |  88 ++++++++++++++-
 .../apache/solr/security/BasicAuthPlugin.java   |  73 +++++++-----
 .../apache/solr/security/HadoopAuthPlugin.java  |  18 +++
 .../solr/security/PKIAuthenticationPlugin.java  |  10 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |   2 +-
 .../solr/security/BasicAuthIntegrationTest.java |  49 +++++++-
 .../solr/security/BasicAuthStandaloneTest.java  |   4 +-
 .../PKIAuthenticationIntegrationTest.java       |   6 +-
 .../security/TestPKIAuthenticationPlugin.java   |   8 +-
 .../TestSolrCloudWithHadoopAuthPlugin.java      |   8 +-
 .../solr/cloud/SolrCloudAuthTestCase.java       | 112 +++++++++++++++++++
 14 files changed, 336 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index bfa3666..a01c16e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -79,6 +79,8 @@ New Features
 
 * SOLR-12593: The default configSet now includes an "ignored_*" dynamic field.  (David Smiley)
 
+* SOLR-12791: Add Metrics reporting for AuthenticationPlugin (janhoy)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 1be03a8..3051dbf 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -804,6 +804,12 @@ public class CoreContainer {
     SecurityConfHandler.SecurityConfig securityConfig = securityConfHandler.getSecurityConfig(false);
     initializeAuthorizationPlugin((Map<String, Object>) securityConfig.getData().get("authorization"));
     initializeAuthenticationPlugin((Map<String, Object>) securityConfig.getData().get("authentication"));
+    if (authenticationPlugin != null) {
+      authenticationPlugin.plugin.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, "/authentication");
+    }
+    if (pkiAuthenticationPlugin != null && pkiAuthenticationPlugin.getMetricRegistry() == null) {
+      pkiAuthenticationPlugin.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, "/authentication/pki");
+    }
   }
 
   private static void checkForDuplicateCoreNames(List<CoreDescriptor> cds) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
index 38ffc99..bfb3428 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
@@ -34,7 +34,7 @@ public interface SolrInfoBean {
    * Category of Solr component.
    */
   enum Category { CONTAINER, ADMIN, CORE, QUERY, UPDATE, CACHE, HIGHLIGHTER, QUERYPARSER, SPELLCHECKER,
-    SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, OTHER }
+    SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, SECURITY, OTHER }
 
   /**
    * Top-level group of beans or metrics for a subsystem.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index a9d112a..48b073d 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -20,16 +20,42 @@ import javax.servlet.FilterChain;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
 import java.io.Closeable;
+import java.util.Arrays;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.core.SolrInfoBean;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 
 /**
  * 
  * @lucene.experimental
  */
-public abstract class AuthenticationPlugin implements Closeable {
+public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, SolrMetricProducer {
 
   final public static String AUTHENTICATION_PLUGIN_PROP = "authenticationPlugin";
 
+  // Metrics
+  private Set<String> metricNames = ConcurrentHashMap.newKeySet();
+  private MetricRegistry registry;
+
+  protected String registryName;
+  protected SolrMetricManager metricManager;
+  protected Meter numErrors = new Meter();
+  protected Counter requests = new Counter();
+  protected Timer requestTimes = new Timer();
+  protected Counter totalTime = new Counter();
+  protected Counter numAuthenticated = new Counter();
+  protected Counter numPassThrough = new Counter();
+  protected Counter numWrongCredentials = new Counter();
+  protected Counter numMissingCredentials = new Counter();
+
   /**
    * This is called upon loading up of a plugin, used for setting it up.
    * @param pluginConfig Config parameters, possibly from a ZK source
@@ -52,6 +78,23 @@ public abstract class AuthenticationPlugin implements Closeable {
   public abstract boolean doAuthenticate(ServletRequest request, ServletResponse response,
       FilterChain filterChain) throws Exception;
 
+  /**
+   * This method is called by SolrDispatchFilter in order to initiate authentication.
+   * It does some standard metrics counting.
+   */
+  public final boolean authenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
+    Timer.Context timer = requestTimes.time();
+    requests.inc();
+    try {
+      return doAuthenticate(request, response, filterChain);
+    } catch(Exception e) {
+      numErrors.mark();
+      throw e;
+    } finally {
+      long elapsed = timer.stop();
+      totalTime.inc(elapsed);
+    }
+  }
 
   /**
    * Cleanup any per request  data
@@ -59,4 +102,47 @@ public abstract class AuthenticationPlugin implements Closeable {
   public void closeRequest() {
   }
 
+  @Override
+  public void initializeMetrics(SolrMetricManager manager, String registryName, String tag, final String scope) {
+    this.metricManager = manager;
+    this.registryName = registryName;
+    // Metrics
+    registry = manager.registry(registryName);
+    numErrors = manager.meter(this, registryName, "errors", getCategory().toString(), scope);
+    requests = manager.counter(this, registryName, "requests", getCategory().toString(), scope);
+    numAuthenticated = manager.counter(this, registryName, "authenticated", getCategory().toString(), scope);
+    numPassThrough = manager.counter(this, registryName, "passThrough", getCategory().toString(), scope);
+    numWrongCredentials = manager.counter(this, registryName, "failWrongCredentials", getCategory().toString(), scope);
+    numMissingCredentials = manager.counter(this, registryName, "failMissingCredentials", getCategory().toString(), scope);
+    requestTimes = manager.timer(this, registryName, "requestTimes", getCategory().toString(), scope);
+    totalTime = manager.counter(this, registryName, "totalTime", getCategory().toString(), scope);
+    metricNames.addAll(Arrays.asList("errors", "requests", "authenticated", "passThrough",
+        "failWrongCredentials", "failMissingCredentials", "requestTimes", "totalTime"));
+  }
+  
+  @Override
+  public String getName() {
+    return this.getClass().getName();
+  }
+
+  @Override
+  public String getDescription() {
+    return "Authentication Plugin " + this.getClass().getName();
+  }
+
+  @Override
+  public Category getCategory() {
+    return Category.SECURITY;
+  }
+  
+  @Override
+  public Set<String> getMetricNames() {
+    return metricNames;
+  }
+
+  @Override
+  public MetricRegistry getMetricRegistry() {
+    return registry;
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index 1212452..7ac5a7c 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -127,44 +127,57 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       if (st.hasMoreTokens()) {
         String basic = st.nextToken();
         if (basic.equalsIgnoreCase("Basic")) {
-          try {
-            String credentials = new String(Base64.decodeBase64(st.nextToken()), "UTF-8");
-            int p = credentials.indexOf(":");
-            if (p != -1) {
-              final String username = credentials.substring(0, p).trim();
-              String pwd = credentials.substring(p + 1).trim();
-              if (!authenticate(username, pwd)) {
-                log.debug("Bad auth credentials supplied in Authorization header");
-                authenticationFailure(response, isAjaxRequest, "Bad credentials");
+          if (st.hasMoreTokens()) {
+            try {
+              String credentials = new String(Base64.decodeBase64(st.nextToken()), "UTF-8");
+              int p = credentials.indexOf(":");
+              if (p != -1) {
+                final String username = credentials.substring(0, p).trim();
+                String pwd = credentials.substring(p + 1).trim();
+                if (!authenticate(username, pwd)) {
+                  numWrongCredentials.inc();
+                  log.debug("Bad auth credentials supplied in Authorization header");
+                  authenticationFailure(response, isAjaxRequest, "Bad credentials");
+                  return false;
+                } else {
+                  HttpServletRequestWrapper wrapper = new HttpServletRequestWrapper(request) {
+                    @Override
+                    public Principal getUserPrincipal() {
+                      return new BasicUserPrincipal(username);
+                    }
+                  };
+                  numAuthenticated.inc();
+                  filterChain.doFilter(wrapper, response);
+                  return true;
+                }
               } else {
-                HttpServletRequestWrapper wrapper = new HttpServletRequestWrapper(request) {
-                  @Override
-                  public Principal getUserPrincipal() {
-                    return new BasicUserPrincipal(username);
-                  }
-                };
-                filterChain.doFilter(wrapper, response);
-                return true;
+                numErrors.mark();
+                authenticationFailure(response, isAjaxRequest, "Invalid authentication token");
+                return false;
               }
-
-            } else {
-              authenticationFailure(response, isAjaxRequest, "Invalid authentication token");
+            } catch (UnsupportedEncodingException e) {
+              throw new Error("Couldn't retrieve authentication", e);
             }
-          } catch (UnsupportedEncodingException e) {
-            throw new Error("Couldn't retrieve authentication", e);
+          } else {
+            numErrors.mark();
+            authenticationFailure(response, isAjaxRequest, "Malformed Basic Auth header");
+            return false;
           }
         }
       }
+    }
+    
+    // No auth header OR header empty OR Authorization header not of type Basic, i.e. "unknown" user
+    if (blockUnknown) {
+      numMissingCredentials.inc();
+      authenticationFailure(response, isAjaxRequest, "require authentication");
+      return false;
     } else {
-      if (blockUnknown) {
-        authenticationFailure(response, isAjaxRequest, "require authentication");
-      } else {
-        request.setAttribute(AuthenticationPlugin.class.getName(), authenticationProvider.getPromptHeaders());
-        filterChain.doFilter(request, response);
-        return true;
-      }
+      numPassThrough.inc();
+      request.setAttribute(AuthenticationPlugin.class.getName(), authenticationProvider.getPromptHeaders());
+      filterChain.doFilter(request, response);
+      return true;
     }
-    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
index 28352f3..f0e0fdc 100644
--- a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
@@ -244,6 +244,24 @@ public class HadoopAuthPlugin extends AuthenticationPlugin {
     };
     authFilter.doFilter(request, rspCloseShield, filterChain);
 
+    switch (frsp.getStatus()) {
+      case HttpServletResponse.SC_UNAUTHORIZED:
+        // Cannot tell whether the 401 is due to wrong or missing credentials
+        numWrongCredentials.inc();
+        break;
+
+      case HttpServletResponse.SC_FORBIDDEN:
+        // Are there other status codes which should also translate to error?
+        numErrors.mark();
+        break;
+      default:
+        if (frsp.getStatus() >= 200 && frsp.getStatus() <= 299) {
+          numAuthenticated.inc();
+        } else {
+          numErrors.mark();
+        }
+    }
+     
     if (TRACE_HTTP) {
       log.info("----------HTTP Response---------");
       log.info("Status : {}", frsp.getStatus());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index 54d09d8..fe1ae7b 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -90,6 +90,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
 
     String requestURI = ((HttpServletRequest) request).getRequestURI();
     if (requestURI.endsWith(PublicKeyHandler.PATH)) {
+      numPassThrough.inc();
       filterChain.doFilter(request, response);
       return true;
     }
@@ -98,6 +99,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     if (header == null) {
       //this must not happen
       log.error("No SolrAuth header present");
+      numMissingCredentials.inc();
       filterChain.doFilter(request, response);
       return true;
     }
@@ -105,6 +107,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     List<String> authInfo = StrUtils.splitWS(header, false);
     if (authInfo.size() < 2) {
       log.error("Invalid SolrAuth Header {}", header);
+      numErrors.mark();
       filterChain.doFilter(request, response);
       return true;
     }
@@ -115,19 +118,22 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     PKIHeaderData decipher = decipherHeader(nodeName, cipher);
     if (decipher == null) {
       log.error("Could not decipher a header {} . No principal set", header);
+      numMissingCredentials.inc();
       filterChain.doFilter(request, response);
       return true;
     }
     if ((receivedTime - decipher.timestamp) > MAX_VALIDITY) {
       log.error("Invalid key request timestamp: {} , received timestamp: {} , TTL: {}", decipher.timestamp, receivedTime, MAX_VALIDITY);
-        filterChain.doFilter(request, response);
-        return true;
+      numErrors.mark();
+      filterChain.doFilter(request, response);
+      return true;
     }
 
     final Principal principal = "$".equals(decipher.userName) ?
         SU :
         new BasicUserPrincipal(decipher.userName);
 
+    numAuthenticated.inc();
     filterChain.doFilter(getWrapper((HttpServletRequest) request, principal), response);
     return true;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 1a8b14e..956af17 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -478,7 +478,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       try {
         log.debug("Request to authenticate: {}, domain: {}, port: {}", request, request.getLocalName(), request.getLocalPort());
         // upon successful authentication, this should call the chain's next filter.
-        requestContinues = authenticationPlugin.doAuthenticate(request, response, (req, rsp) -> {
+        requestContinues = authenticationPlugin.authenticate(request, response, (req, rsp) -> {
           isAuthenticated.set(true);
           wrappedRequest.set((HttpServletRequest) req);
         });

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 6b6b4af..db47d37 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -30,6 +30,7 @@ import java.util.Objects;
 import java.util.Random;
 import java.util.function.Predicate;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
@@ -47,13 +48,15 @@ import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.request.RequestWriter.StringPayloadContentWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.request.V2Request;
-import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.SolrCloudAuthTestCase;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
@@ -67,7 +70,7 @@ import org.slf4j.LoggerFactory;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Collections.singletonMap;
 
-public class BasicAuthIntegrationTest extends SolrCloudTestCase {
+public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -116,7 +119,10 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
       
       baseUrl = randomJetty.getBaseUrl().toString();
       verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/class", "solr.BasicAuthPlugin", 20);
-
+      assertNumberOfMetrics(16); // Basic auth metrics available
+      assertAuthMetricsMinimums(1, 0, 1, 0, 0, 0);
+      assertPkiAuthMetricsMinimums(0, 0, 0, 0, 0, 0);
+      
       String command = "{\n" +
           "'set-user': {'harry':'HarryIsCool'}\n" +
           "}";
@@ -134,7 +140,9 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
         cluster.getSolrClient().request(genericReq);
       });
       assertEquals(401, exp.code());
-
+      assertAuthMetricsMinimums(2, 0, 2, 0, 0, 0);
+      assertPkiAuthMetricsMinimums(0, 0, 0, 0, 0, 0);
+      
       command = "{\n" +
           "'set-user': {'harry':'HarryIsUberCool'}\n" +
           "}";
@@ -148,6 +156,8 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
       int statusCode = r.getStatusLine().getStatusCode();
       Utils.consumeFully(r.getEntity());
       assertEquals("proper_cred sent, but access denied", 200, statusCode);
+      assertPkiAuthMetricsMinimums(0, 0, 0, 0, 0, 0);
+      assertAuthMetricsMinimums(4, 1, 3, 0, 0, 0);
 
       baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
 
@@ -157,6 +167,7 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
           "}";
 
       executeCommand(baseUrl + authzPrefix, cl,command, "solr", "SolrRocks");
+      assertAuthMetricsMinimums(6, 2, 4, 0, 0, 0);
 
       baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
       verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/user-role/harry", NOT_NULL_PREDICATE, 20);
@@ -167,10 +178,12 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
               "role", "dev"))), "harry", "HarryIsUberCool" );
 
       verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[1]/collection", "x", 20);
+      assertAuthMetricsMinimums(9, 3, 6, 0, 0, 0);
 
       executeCommand(baseUrl + authzPrefix, cl,Utils.toJSONString(singletonMap("set-permission", Utils.makeMap
           ("name", "collection-admin-edit", "role", "admin"))), "harry", "HarryIsUberCool"  );
       verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[2]/name", "collection-admin-edit", 20);
+      assertAuthMetricsMinimums(11, 4, 7, 0, 0, 0);
 
       CollectionAdminRequest.Reload reload = CollectionAdminRequest.reloadCollection(COLLECTION);
 
@@ -197,7 +210,7 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
             .setBasicAuthCredentials("harry", "Cool12345"));
         fail("This should not succeed");
       } catch (HttpSolrClient.RemoteSolrException e) {
-
+        assertAuthMetricsMinimums(15, 5, 9, 1, 0, 0);
       }
 
       executeCommand(baseUrl + authzPrefix, cl,"{set-permission : { name : update , role : admin}}", "harry", "HarryIsUberCool");
@@ -214,6 +227,7 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: true}}", "harry", "HarryIsUberCool");
       verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/blockUnknown", "true", 20, "harry", "HarryIsUberCool");
       verifySecurityStatus(cl, baseUrl + "/admin/info/key", "key", NOT_NULL_PREDICATE, 20);
+      assertAuthMetricsMinimums(18, 8, 9, 1, 0, 0);
 
       String[] toolArgs = new String[]{
           "status", "-solr", baseUrl};
@@ -232,6 +246,24 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
         log.error("RunExampleTool failed due to: " + e +
             "; stdout from tool prior to failure: " + baos.toString(StandardCharsets.UTF_8.name()));
       }
+
+      SolrParams params = new MapSolrParams(Collections.singletonMap("q", "*:*"));
+      // Query that fails due to missing credentials
+      exp = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
+        cluster.getSolrClient().query(COLLECTION, params);
+      });
+      assertEquals(401, exp.code());
+      assertAuthMetricsMinimums(20, 8, 9, 1, 2, 0);
+      assertPkiAuthMetricsMinimums(4, 4, 0, 0, 0, 0);
+
+      // Query that succeeds
+      GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/select", params);
+      req.setBasicAuthCredentials("harry", "HarryIsUberCool");
+      cluster.getSolrClient().request(req, COLLECTION);
+      
+      assertAuthMetricsMinimums(21, 9, 9, 1, 2, 0);
+      assertPkiAuthMetricsMinimums(7, 7, 0, 0, 0, 0);
+
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: false}}", "harry", "HarryIsUberCool");
     } finally {
       if (cl != null) {
@@ -240,6 +272,13 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
     }
   }
 
+  private void assertNumberOfMetrics(int num) {
+    MetricRegistry registry0 = cluster.getJettySolrRunner(0).getCoreContainer().getMetricManager().registry("solr.node");
+    assertNotNull(registry0);
+
+    assertEquals(num, registry0.getMetrics().entrySet().stream().filter(e -> e.getKey().startsWith("SECURITY")).count());
+  }
+
   public static void executeCommand(String url, HttpClient cl, String payload, String user, String pwd)
       throws IOException {
     HttpPost httpPost;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
index da77b22..1cfd681 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
@@ -29,6 +29,7 @@ import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.message.AbstractHttpMessage;
 import org.apache.http.message.BasicHeader;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
@@ -41,8 +42,6 @@ import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.LogLevel;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -84,7 +83,6 @@ public class BasicAuthStandaloneTest extends SolrTestCaseJ4 {
   }
 
   @Test
-  @LogLevel("org.apache.solr=DEBUG")
   public void testBasicAuth() throws Exception {
 
     String authcPrefix = "/admin/authentication";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
index 1f85f3a..aa70d05 100644
--- a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
@@ -25,7 +25,7 @@ import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.SolrCloudAuthTestCase;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
@@ -39,7 +39,7 @@ import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.Utils.makeMap;
 import static org.apache.solr.security.TestAuthorizationFramework.verifySecurityStatus;
 
-public class PKIAuthenticationIntegrationTest extends SolrCloudTestCase {
+public class PKIAuthenticationIntegrationTest extends SolrCloudAuthTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -96,7 +96,7 @@ public class PKIAuthenticationIntegrationTest extends SolrCloudTestCase {
     QueryRequest query = new QueryRequest(params);
     query.process(cluster.getSolrClient(), "collection");
     assertTrue("all nodes must get the user solr , no:of nodes got solr : " + count.get(), count.get() > 2);
-
+    assertPkiAuthMetricsMinimums(2, 2, 0, 0, 0, 0);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
index e6a04cf..e6dc790 100644
--- a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
@@ -99,7 +99,7 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     final AtomicReference<ServletRequest> wrappedRequestByFilter = new AtomicReference<>();
     HttpServletRequest mockReq = createMockRequest(header);
     FilterChain filterChain = (servletRequest, servletResponse) -> wrappedRequestByFilter.set(servletRequest);
-    mock.doAuthenticate(mockReq, null, filterChain);
+    mock.authenticate(mockReq, null, filterChain);
 
     assertNotNull(((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal());
     assertNotNull(wrappedRequestByFilter.get());
@@ -112,7 +112,7 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     request = new BasicHttpRequest("GET", "http://localhost:56565");
     mock.setHeader(request);
     assertNull(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
-    mock.doAuthenticate(mockReq, null, filterChain);
+    mock.authenticate(mockReq, null, filterChain);
     assertNotNull(wrappedRequestByFilter.get());
     assertNull(((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal());
 
@@ -129,7 +129,7 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     assertNotNull(header.get());
     assertTrue(header.get().getValue().startsWith(nodeName));
 
-    mock.doAuthenticate(mockReq, null, filterChain);
+    mock.authenticate(mockReq, null, filterChain);
     assertNotNull(wrappedRequestByFilter.get());
     assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
 
@@ -146,7 +146,7 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
       }
     };
 
-    mock1.doAuthenticate(mockReq, null,filterChain );
+    mock1.authenticate(mockReq, null,filterChain );
     assertNotNull(wrappedRequestByFilter.get());
     assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
     mock1.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
index 118dc47..1842da5 100644
--- a/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
@@ -27,14 +27,14 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.AbstractDistribZkTestBase;
 import org.apache.solr.cloud.KerberosTestServices;
-import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.SolrCloudAuthTestCase;
 import org.apache.solr.common.SolrInputDocument;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 // commented 20-July-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 12-Jun-2018
-public class TestSolrCloudWithHadoopAuthPlugin extends SolrCloudTestCase {
+public class TestSolrCloudWithHadoopAuthPlugin extends SolrCloudAuthTestCase {
   protected static final int NUM_SERVERS = 1;
   protected static final int NUM_SHARDS = 1;
   protected static final int REPLICATION_FACTOR = 1;
@@ -119,11 +119,14 @@ public class TestSolrCloudWithHadoopAuthPlugin extends SolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1",
         NUM_SHARDS, REPLICATION_FACTOR);
     create.process(solrClient);
+    // The metrics counter for wrong credentials here really just means  
+    assertAuthMetricsMinimums(6, 3, 0, 3, 0, 0);
 
     SolrInputDocument doc = new SolrInputDocument();
     doc.setField("id", "1");
     solrClient.add(collectionName, doc);
     solrClient.commit(collectionName);
+    assertAuthMetricsMinimums(10, 5, 0, 5, 0, 0);
 
     SolrQuery query = new SolrQuery();
     query.setQuery("*:*");
@@ -134,6 +137,7 @@ public class TestSolrCloudWithHadoopAuthPlugin extends SolrCloudTestCase {
     deleteReq.process(solrClient);
     AbstractDistribZkTestBase.waitForCollectionToDisappear(collectionName,
         solrClient.getZkStateReader(), true, true, 330);
+    assertAuthMetricsMinimums(16, 8, 0, 8, 0, 0);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef2f0cd8/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
new file mode 100644
index 0000000..3966426
--- /dev/null
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
@@ -0,0 +1,112 @@
+/*
+ * 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.cloud;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base test class for cloud tests wanting to track authentication metrics.
+ * The assertions provided by this base class require a *minimum* count, not exact count from metrics.
+ * Warning: Make sure that your test case does not break when beasting. 
+ */
+public class SolrCloudAuthTestCase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final List<String> AUTH_METRICS_KEYS = Arrays.asList("errors", "requests", "authenticated", 
+      "passThrough", "failWrongCredentials", "failMissingCredentials", "requestTimes", "totalTime");
+  private static final List<String> AUTH_METRICS_METER_KEYS = Arrays.asList("errors");
+  private static final List<String> AUTH_METRICS_TIMER_KEYS = Collections.singletonList("requestTimes");
+  private static final String METRICS_PREFIX_PKI = "SECURITY./authentication/pki.";
+  private static final String METRICS_PREFIX = "SECURITY./authentication.";
+  
+  /**
+   * Used to check metric counts for PKI auth
+   */
+  protected void assertPkiAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+    assertAuthMetricsMinimums(METRICS_PREFIX_PKI, requests, authenticated, passThrough, failWrongCredentials, failMissingCredentials, errors);
+  }
+  
+  /**
+   * Used to check metric counts for the AuthPlugin in use (except PKI)
+   */
+  protected void assertAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+    assertAuthMetricsMinimums(METRICS_PREFIX, requests, authenticated, passThrough, failWrongCredentials, failMissingCredentials, errors);
+  }  
+  
+  /**
+   * Common test method to be able to check security from any authentication plugin
+   * @param prefix the metrics key prefix, currently "SECURITY./authentication." for basic auth and "SECURITY./authentication/pki." for PKI 
+   */
+  private void assertAuthMetricsMinimums(String prefix, int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+    List<Map<String, Metric>> metrics = new ArrayList<>();
+    cluster.getJettySolrRunners().forEach(r -> {
+      MetricRegistry registry = r.getCoreContainer().getMetricManager().registry("solr.node");
+      assertNotNull(registry);
+      metrics.add(registry.getMetrics());
+    });
+
+    Map<String,Long> counts = new HashMap<>();
+    AUTH_METRICS_KEYS.forEach(k -> {
+      counts.put(k, sumCount(prefix, k, metrics));
+    });
+    
+    // check each counter
+    assertExpectedMetrics(requests, "requests", counts);
+    assertExpectedMetrics(authenticated, "authenticated", counts);
+    assertExpectedMetrics(passThrough, "passThrough", counts);
+    assertExpectedMetrics(failWrongCredentials, "failWrongCredentials", counts);
+    assertExpectedMetrics(failMissingCredentials, "failMissingCredentials", counts);
+    assertExpectedMetrics(errors, "errors", counts);
+    if (counts.get("requests") > 0) {
+      assertTrue("requestTimes count not > 1", counts.get("requestTimes") > 1);
+      assertTrue("totalTime not > 0", counts.get("totalTime") > 0);
+    }
+  }
+
+  // Check that the actual metric is equal to or greater than the expected value, never less
+  private void assertExpectedMetrics(int expected, String key, Map<String, Long> counts) {
+    long cnt = counts.get(key);
+    log.debug("Asserting that auth metrics count ({}) > expected ({})", cnt, expected);
+    assertTrue("Expected " + key + " metric count to be " + expected + " or higher, but got " + cnt, 
+        cnt >= expected);
+  }
+
+  // Have to sum the metrics from all three shards/nodes
+  private long sumCount(String prefix, String key, List<Map<String, Metric>> metrics) {
+    assertTrue("Metric " + prefix + key + " does not exist", metrics.get(0).containsKey(prefix + key)); 
+    if (AUTH_METRICS_METER_KEYS.contains(key))
+      return metrics.stream().mapToLong(l -> ((Meter)l.get(prefix + key)).getCount()).sum();
+    else if (AUTH_METRICS_TIMER_KEYS.contains(key))
+      return (long) ((long) 1000 * metrics.stream().mapToDouble(l -> ((Timer)l.get(prefix + key)).getMeanRate()).average().orElse(0.0d));
+    else
+      return metrics.stream().mapToLong(l -> ((Counter)l.get(prefix + key)).getCount()).sum();
+  }
+}


[33/34] lucene-solr:jira/http2: LUCENE-8609: Remove deprecated IW#numDocs() and IW#maxDoc() methdos

Posted by da...@apache.org.
LUCENE-8609: Remove deprecated IW#numDocs() and IW#maxDoc() methdos


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

Branch: refs/heads/jira/http2
Commit: 5c5c42cc37887236bf76b9e5f1cceaa25ee66886
Parents: e974311
Author: Simon Willnauer <si...@apache.org>
Authored: Fri Dec 14 17:33:20 2018 +0100
Committer: Simon Willnauer <si...@apache.org>
Committed: Fri Dec 14 19:36:25 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 ++
 lucene/MIGRATE.txt                              |  6 ++++
 .../org/apache/lucene/index/IndexWriter.java    | 35 ++------------------
 3 files changed, 11 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c5c42cc/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b4dd696..bed537b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -102,6 +102,9 @@ API Changes
   number of gaps between its component sub-intervals.  This can be used in a 
   new filter available via Intervals.maxgaps().  (Alan Woodward)
 
+* LUCENE-8609: Remove IndexWriter#numDocs() and IndexWriter#maxDoc() in favor
+  of IndexWriter#getDocStats(). (Simon Willnauer)
+
 Changes in Runtime Behavior
 
 * LUCENE-8333: Switch MoreLikeThis.setMaxDocFreqPct to use maxDoc instead of

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c5c42cc/lucene/MIGRATE.txt
----------------------------------------------------------------------
diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt
index 0b78d3c..045f00d 100644
--- a/lucene/MIGRATE.txt
+++ b/lucene/MIGRATE.txt
@@ -158,3 +158,9 @@ constant factor was removed from the numerator of the scoring formula.
 Ordering of results is preserved unless scores are computed from multiple
 fields using different similarities. The previous behaviour is now exposed
 by the LegacyBM25Similarity class which can be found in the lucene-misc jar.
+
+## IndexWriter#maxDoc()/#numDocs() removed in favor of IndexWriter#getDocStats() ##
+
+IndexWriter#getDocStats() should be used instead of #maxDoc() / #numDocs() which offers a consistent 
+view on document stats. Previously calling two methods in order ot get point in time stats was subject
+to concurrent changes.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c5c42cc/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 4771f3d..f9aaf34 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1131,18 +1131,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     return analyzer;
   }
 
-  /** Returns total number of docs in this index, including
-   *  docs not yet flushed (still in the RAM buffer),
-   *  not counting deletions.
-   *  @see #numDocs
-   *  @deprecated use {@link #getDocStats()} instead
-   *  */
-  @Deprecated
-  public synchronized int maxDoc() {
-    ensureOpen();
-    return docWriter.getNumDocs() + segmentInfos.totalMaxDoc();
-  }
-
   /** If {@link SegmentInfos#getVersion} is below {@code newVersion} then update it to this value.
    *
    * @lucene.internal */
@@ -1154,24 +1142,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     changed();
   }
 
-  /** Returns total number of docs in this index, including
-   *  docs not yet flushed (still in the RAM buffer), and
-   *  including deletions.  <b>NOTE:</b> buffered deletions
-   *  are not counted.  If you really need these to be
-   *  counted you should call {@link #commit()} first.
-   *  @see #maxDoc
-   *  @deprecated use {@link #getDocStats()} instead
-   *  */
-  @Deprecated
-  public synchronized int numDocs() {
-    ensureOpen();
-    int count = docWriter.getNumDocs();
-    for (final SegmentCommitInfo info : segmentInfos) {
-      count += info.info.maxDoc() - numDeletedDocs(info);
-    }
-    return count;
-  }
-
   /**
    * Returns true if this index has deletions (including
    * buffered deletions).  Note that this will return true
@@ -5297,9 +5267,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
   }
 
   /**
-   * Returns accurate {@link DocStats} form this writer. This is equivalent to calling {@link #numDocs()} and {@link #maxDoc()}
-   * but is not subject to race-conditions. The numDoc for instance can change after maxDoc is fetched that causes numDocs to be
-   * greater than maxDoc which makes it hard to get accurate document stats from IndexWriter.
+   * Returns accurate {@link DocStats} form this writer. The numDoc for instance can change after maxDoc is fetched
+   * that causes numDocs to be greater than maxDoc which makes it hard to get accurate document stats from IndexWriter.
    */
   public synchronized DocStats getDocStats() {
     ensureOpen();


[08/34] lucene-solr:jira/http2: SOLR-12791, SOLR-13067: Fix test failure for BasicAuthIntegrationTest Make PkiAuthenticationIntegrationTest beast-able

Posted by da...@apache.org.
SOLR-12791, SOLR-13067: Fix test failure for BasicAuthIntegrationTest
Make PkiAuthenticationIntegrationTest beast-able


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

Branch: refs/heads/jira/http2
Commit: 9728dbc1675bb7fd4ca84071d40ae3c0528e424c
Parents: 1e68726
Author: Jan Høydahl <ja...@apache.org>
Authored: Thu Dec 13 00:14:37 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Thu Dec 13 00:14:37 2018 +0100

----------------------------------------------------------------------
 .../apache/solr/security/BasicAuthIntegrationTest.java  |  4 ++--
 .../solr/security/PKIAuthenticationIntegrationTest.java | 12 +++++++-----
 2 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9728dbc1/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index db47d37..f704cdf 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -254,7 +254,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
       });
       assertEquals(401, exp.code());
       assertAuthMetricsMinimums(20, 8, 9, 1, 2, 0);
-      assertPkiAuthMetricsMinimums(4, 4, 0, 0, 0, 0);
+      assertPkiAuthMetricsMinimums(3, 3, 0, 0, 0, 0);
 
       // Query that succeeds
       GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/select", params);
@@ -262,7 +262,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
       cluster.getSolrClient().request(req, COLLECTION);
       
       assertAuthMetricsMinimums(21, 9, 9, 1, 2, 0);
-      assertPkiAuthMetricsMinimums(7, 7, 0, 0, 0, 0);
+      assertPkiAuthMetricsMinimums(6, 6, 0, 0, 0, 0);
 
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: false}}", "harry", "HarryIsUberCool");
     } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9728dbc1/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
index aa70d05..eb25b83 100644
--- a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
@@ -42,19 +42,21 @@ import static org.apache.solr.security.TestAuthorizationFramework.verifySecurity
 public class PKIAuthenticationIntegrationTest extends SolrCloudAuthTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
+  private static final String COLLECTION = "pkiCollection";
+  
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(2)
         .addConfig("conf", configset("cloud-minimal"))
         .configure();
+
+    CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient());
+
+    cluster.waitForActiveCollection(COLLECTION, 2, 2);
   }
 
   @Test
   public void testPkiAuth() throws Exception {
-
-    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
-
     // TODO make a SolrJ helper class for this
     byte[] bytes = Utils.toJSON(makeMap("authorization", singletonMap("class", MockAuthorizationPlugin.class.getName()),
         "authentication", singletonMap("class", MockAuthenticationPlugin.class.getName())));
@@ -94,7 +96,7 @@ public class PKIAuthenticationIntegrationTest extends SolrCloudAuthTestCase {
         return true;
     };
     QueryRequest query = new QueryRequest(params);
-    query.process(cluster.getSolrClient(), "collection");
+    query.process(cluster.getSolrClient(), COLLECTION);
     assertTrue("all nodes must get the user solr , no:of nodes got solr : " + count.get(), count.get() > 2);
     assertPkiAuthMetricsMinimums(2, 2, 0, 0, 0, 0);
   }


[29/34] lucene-solr:jira/http2: Fix active replica count reporting in SimClusterStateProvider

Posted by da...@apache.org.
Fix active replica count reporting in SimClusterStateProvider

One codepath assumed all replicas were active, instead of checking the
state of each individually.


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

Branch: refs/heads/jira/http2
Commit: 655e9ecf7cbd03c301620abfba04b172f96946c2
Parents: dd910d1
Author: Jason Gerlowski <ge...@apache.org>
Authored: Fri Dec 14 11:07:10 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Fri Dec 14 12:21:47 2018 -0500

----------------------------------------------------------------------
 .../solr/cloud/autoscaling/sim/SimClusterStateProvider.java   | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/655e9ecf/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index 6197c41..c411b21 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -1998,7 +1998,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           if (buffered != null) {
             bufferedDocs += buffered.get();
           }
-          activeReplicas += s.getReplicas().size();
+
+          for (Replica r : s.getReplicas()) {
+            if (r.getState() == Replica.State.ACTIVE) {
+              activeReplicas++;
+            }
+          }
           Replica leader = s.getLeader();
           if (leader == null) {
             noLeader++;


[20/34] lucene-solr:jira/http2: Merge remote-tracking branch 'refs/remotes/origin/master'

Posted by da...@apache.org.
Merge remote-tracking branch 'refs/remotes/origin/master'


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

Branch: refs/heads/jira/http2
Commit: 774e9aefbcd1016f1681f13409b37d9b1068af37
Parents: df07fb3 d3ee925
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Dec 13 20:58:12 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Dec 13 20:58:12 2018 -0700

----------------------------------------------------------------------
 .../lucene/index/TestBackwardsCompatibility.java   |   7 +++++--
 .../org/apache/lucene/index/index.7.6.0-cfs.zip    | Bin 0 -> 15655 bytes
 .../org/apache/lucene/index/index.7.6.0-nocfs.zip  | Bin 0 -> 15649 bytes
 .../test/org/apache/lucene/index/sorted.7.6.0.zip  | Bin 0 -> 74541 bytes
 4 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[15/34] lucene-solr:jira/http2: LUCENE-8608: Extract utility class to iterate over terms docs

Posted by da...@apache.org.
LUCENE-8608: Extract utility class to iterate over terms docs

Today we re-implement the same algorithm in various places
when we want to consume all docs for a set/list of terms. This
caused serious slowdowns for instance in the case of applying
updates fixed in LUCENE-8602. This change extracts the common
usage and shares the interation code including logic to reuse
Terms and PostingsEnum instances as much as possble and adds
tests for it.


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

Branch: refs/heads/jira/http2
Commit: 4272c631ca18ab4194fd443dfe2f64b3a81a9a04
Parents: d7ad2f4
Author: Simon Willnauer <si...@apache.org>
Authored: Fri Dec 7 22:17:26 2018 +0100
Committer: Simon Willnauer <si...@apache.org>
Committed: Thu Dec 13 15:30:34 2018 +0100

----------------------------------------------------------------------
 .../apache/lucene/index/BufferedUpdates.java    |   6 +-
 .../lucene/index/FreqProxTermsWriter.java       |  41 ++--
 .../lucene/index/FrozenBufferedUpdates.java     | 216 +++++++++++--------
 .../org/apache/lucene/util/BytesRefArray.java   |   3 +-
 .../apache/lucene/util/BytesRefIterator.java    |  10 +-
 .../lucene/index/TestFrozenBufferedUpdates.java | 104 +++++++++
 6 files changed, 250 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4272c631/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java
index 04b19b7..d8317f8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java
@@ -69,11 +69,11 @@ class BufferedUpdates implements Accountable {
   final AtomicInteger numTermDeletes = new AtomicInteger();
   final AtomicInteger numFieldUpdates = new AtomicInteger();
 
-  final Map<Term,Integer> deleteTerms = new HashMap<>();
+  final Map<Term,Integer> deleteTerms = new HashMap<>(); // TODO cut this over to FieldUpdatesBuffer
   final Map<Query,Integer> deleteQueries = new HashMap<>();
   final List<Integer> deleteDocIDs = new ArrayList<>();
 
-  final Map<String,FieldUpdatesBuffer> fieldUpdates = new HashMap<>();
+  final Map<String, FieldUpdatesBuffer> fieldUpdates = new HashMap<>();
   
 
   public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
@@ -109,7 +109,7 @@ class BufferedUpdates implements Accountable {
         s += " " + deleteDocIDs.size() + " deleted docIDs";
       }
       if (numFieldUpdates.get() != 0) {
-        s += " " + numFieldUpdates.get() + " field updates (unique count=" + fieldUpdates.size() + ")";
+        s += " " + numFieldUpdates.get() + " field updates";
       }
       if (bytesUsed.get() != 0) {
         s += " bytesUsed=" + bytesUsed.get();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4272c631/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index 2ebb13d..5180aaf 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
@@ -42,37 +43,21 @@ final class FreqProxTermsWriter extends TermsHash {
       Map<Term,Integer> segDeletes = state.segUpdates.deleteTerms;
       List<Term> deleteTerms = new ArrayList<>(segDeletes.keySet());
       Collections.sort(deleteTerms);
-      String lastField = null;
-      TermsEnum termsEnum = null;
-      PostingsEnum postingsEnum = null;
+      FrozenBufferedUpdates.TermDocsIterator iterator = new FrozenBufferedUpdates.TermDocsIterator(fields, true);
       for(Term deleteTerm : deleteTerms) {
-        if (deleteTerm.field().equals(lastField) == false) {
-          lastField = deleteTerm.field();
-          Terms terms = fields.terms(lastField);
-          if (terms != null) {
-            termsEnum = terms.iterator();
-          } else {
-            termsEnum = null;
-          }
-        }
-
-        if (termsEnum != null && termsEnum.seekExact(deleteTerm.bytes())) {
-          postingsEnum = termsEnum.postings(postingsEnum, 0);
+        DocIdSetIterator postings = iterator.nextTerm(deleteTerm.field(), deleteTerm.bytes());
+        if (postings != null ) {
           int delDocLimit = segDeletes.get(deleteTerm);
           assert delDocLimit < PostingsEnum.NO_MORE_DOCS;
-          while (true) {
-            int doc = postingsEnum.nextDoc();
-            if (doc < delDocLimit) {
-              if (state.liveDocs == null) {
-                state.liveDocs = new FixedBitSet(state.segmentInfo.maxDoc());
-                state.liveDocs.set(0, state.segmentInfo.maxDoc());
-              }
-              if (state.liveDocs.get(doc)) {
-                state.delCountOnFlush++;
-                state.liveDocs.clear(doc);
-              }
-            } else {
-              break;
+          int doc;
+          while ((doc = postings.nextDoc()) < delDocLimit) {
+            if (state.liveDocs == null) {
+              state.liveDocs = new FixedBitSet(state.segmentInfo.maxDoc());
+              state.liveDocs.set(0, state.segmentInfo.maxDoc());
+            }
+            if (state.liveDocs.get(doc)) {
+              state.delCountOnFlush++;
+              state.liveDocs.clear(doc);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4272c631/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
index 084bcc0..885e8e6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
@@ -86,7 +86,7 @@ final class FrozenBufferedUpdates {
                                    // only have Queries and doc values updates
   private final InfoStream infoStream;
 
-  public FrozenBufferedUpdates(InfoStream infoStream, BufferedUpdates updates, SegmentCommitInfo privateSegment) throws IOException {
+  public FrozenBufferedUpdates(InfoStream infoStream, BufferedUpdates updates, SegmentCommitInfo privateSegment) {
     this.infoStream = infoStream;
     this.privateSegment = privateSegment;
     assert updates.deleteDocIDs.isEmpty();
@@ -468,9 +468,6 @@ final class FrozenBufferedUpdates {
                                             long delGen,
                                             boolean segmentPrivateDeletes) throws IOException {
 
-    TermsEnum termsEnum = null;
-    PostingsEnum postingsEnum = null;
-
     // TODO: we can process the updates per DV field, from last to first so that
     // if multiple terms affect same document for the same field, we add an update
     // only once (that of the last term). To do that, we can keep a bitset which
@@ -492,26 +489,8 @@ final class FrozenBufferedUpdates {
       boolean isNumeric = value.isNumeric();
       FieldUpdatesBuffer.BufferedUpdateIterator iterator = value.iterator();
       FieldUpdatesBuffer.BufferedUpdate bufferedUpdate;
-      String previousField = null;
+      TermDocsIterator termDocsIterator = new TermDocsIterator(segState.reader, false);
       while ((bufferedUpdate = iterator.next()) != null) {
-        if (previousField == null || previousField.equals(bufferedUpdate.termField) == false) {
-          previousField = bufferedUpdate.termField;
-          Terms terms = segState.reader.terms(previousField);
-          termsEnum = terms == null ? null : terms.iterator();
-        }
-        if (termsEnum == null) {
-          // no terms in this segment for this field
-          continue;
-        }
-
-        final int limit;
-        if (delGen == segState.delGen) {
-          assert segmentPrivateDeletes;
-          limit = bufferedUpdate.docUpTo;
-        } else {
-          limit = Integer.MAX_VALUE;
-        }
-        
         // TODO: we traverse the terms in update order (not term order) so that we
         // apply the updates in the correct order, i.e. if two terms update the
         // same document, the last one that came in wins, irrespective of the
@@ -521,23 +500,26 @@ final class FrozenBufferedUpdates {
         // that we cannot rely only on docIDUpto because an app may send two updates
         // which will get same docIDUpto, yet will still need to respect the order
         // those updates arrived.
-
         // TODO: we could at least *collate* by field?
-
-
-        final BytesRef binaryValue;
-        final long longValue;
-        if (bufferedUpdate.hasValue == false) {
-          longValue = -1;
-          binaryValue = null;
-        } else {
-          longValue = bufferedUpdate.numericValue;
-          binaryValue = bufferedUpdate.binaryValue;
-        }
-
-        if (termsEnum.seekExact(bufferedUpdate.termValue)) {
-          // we don't need term frequencies for this
-          postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+        final DocIdSetIterator docIdSetIterator = termDocsIterator.nextTerm(bufferedUpdate.termField, bufferedUpdate.termValue);
+        if (docIdSetIterator != null) {
+          final int limit;
+          if (delGen == segState.delGen) {
+            assert segmentPrivateDeletes;
+            limit = bufferedUpdate.docUpTo;
+          } else {
+            limit = Integer.MAX_VALUE;
+          }
+          final BytesRef binaryValue;
+          final long longValue;
+          if (bufferedUpdate.hasValue == false) {
+            longValue = -1;
+            binaryValue = null;
+          } else {
+            longValue = bufferedUpdate.numericValue;
+            binaryValue = bufferedUpdate.binaryValue;
+          }
+           termDocsIterator.getDocs();
           if (dvUpdates == null) {
             if (isNumeric) {
               if (value.hasSingleValue()) {
@@ -566,7 +548,7 @@ final class FrozenBufferedUpdates {
           if (segState.rld.sortMap != null && segmentPrivateDeletes) {
             // This segment was sorted on flush; we must apply seg-private deletes carefully in this case:
             int doc;
-            while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+            while ((doc = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
               if (acceptDocs == null || acceptDocs.get(doc)) {
                 // The limit is in the pre-sorted doc space:
                 if (segState.rld.sortMap.newToOld(doc) < limit) {
@@ -577,7 +559,7 @@ final class FrozenBufferedUpdates {
             }
           } else {
             int doc;
-            while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+            while ((doc = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
               if (doc >= limit) {
                 break; // no more docs that can be updated for this term
               }
@@ -706,57 +688,13 @@ final class FrozenBufferedUpdates {
       }
 
       FieldTermIterator iter = deleteTerms.iterator();
-
       BytesRef delTerm;
-      String field = null;
-      TermsEnum termsEnum = null;
-      BytesRef readerTerm = null;
-      PostingsEnum postingsEnum = null;
+      TermDocsIterator termDocsIterator = new TermDocsIterator(segState.reader, true);
       while ((delTerm = iter.next()) != null) {
-
-        if (iter.field() != field) {
-          // field changed
-          field = iter.field();
-          Terms terms = segState.reader.terms(field);
-          if (terms != null) {
-            termsEnum = terms.iterator();
-            readerTerm = termsEnum.next();
-          } else {
-            termsEnum = null;
-          }
-        }
-
-        if (termsEnum != null) {
-          int cmp = delTerm.compareTo(readerTerm);
-          if (cmp < 0) {
-            // TODO: can we advance across del terms here?
-            // move to next del term
-            continue;
-          } else if (cmp == 0) {
-            // fall through
-          } else if (cmp > 0) {
-            TermsEnum.SeekStatus status = termsEnum.seekCeil(delTerm);
-            if (status == TermsEnum.SeekStatus.FOUND) {
-              // fall through
-            } else if (status == TermsEnum.SeekStatus.NOT_FOUND) {
-              readerTerm = termsEnum.term();
-              continue;
-            } else {
-              // TODO: can we advance to next field in deleted terms?
-              // no more terms in this segment
-              termsEnum = null;
-              continue;
-            }
-          }
-
-          // we don't need term frequencies for this
-          postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
-
-          assert postingsEnum != null;
-
+        final DocIdSetIterator iterator = termDocsIterator.nextTerm(iter.field(), delTerm);
+        if (iterator != null) {
           int docID;
-          while ((docID = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-
+          while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
             // NOTE: there is no limit check on the docID
             // when deleting by Term (unlike by Query)
             // because on flush we apply all Term deletes to
@@ -821,4 +759,104 @@ final class FrozenBufferedUpdates {
   boolean any() {
     return deleteTerms.size() > 0 || deleteQueries.length > 0 || fieldUpdatesCount > 0 ;
   }
+
+  /**
+   * This class helps iterating a term dictionary and consuming all the docs for each terms.
+   * It accepts a field, value tuple and returns a {@link DocIdSetIterator} if the field has an entry
+   * for the given value. It has an optimized way of iterating the term dictionary if the terms are
+   * passed in sorted order and makes sure terms and postings are reused as much as possible.
+   */
+  static final class TermDocsIterator {
+    private final TermsProvider provider;
+    private String field;
+    private TermsEnum termsEnum;
+    private PostingsEnum postingsEnum;
+    private final boolean sortedTerms;
+    private BytesRef readerTerm;
+    private BytesRef lastTerm; // only set with asserts
+
+    @FunctionalInterface
+    interface TermsProvider {
+      Terms terms(String field) throws IOException;
+    }
+
+    TermDocsIterator(Fields fields, boolean sortedTerms) {
+      this(fields::terms, sortedTerms);
+    }
+
+    TermDocsIterator(LeafReader reader, boolean sortedTerms) {
+      this(reader::terms, sortedTerms);
+    }
+
+    private TermDocsIterator(TermsProvider provider, boolean sortedTerms) {
+      this.sortedTerms = sortedTerms;
+      this.provider = provider;
+    }
+
+    private void setField(String field) throws IOException {
+      if (this.field == null || this.field.equals(field) == false) {
+        this.field = field;
+
+        Terms terms = provider.terms(field);
+        if (terms != null) {
+          termsEnum = terms.iterator();
+          if (sortedTerms) {
+            assert (lastTerm = null) == null; // need to reset otherwise we fail the assertSorted below since we sort per field
+            readerTerm = termsEnum.next();
+          }
+        } else {
+          termsEnum = null;
+        }
+      }
+    }
+
+    DocIdSetIterator nextTerm(String field, BytesRef term) throws IOException {
+      setField(field);
+      if (termsEnum != null) {
+        if (sortedTerms) {
+          assert assertSorted(term);
+          // in the sorted case we can take advantage of the "seeking forward" property
+          // this allows us depending on the term dict impl to reuse data-structures internally
+          // which speed up iteration over terms and docs significantly.
+          int cmp = term.compareTo(readerTerm);
+          if (cmp < 0) {
+            return null; // requested term does not exist in this segment
+          } else if (cmp == 0) {
+            return getDocs();
+          } else if (cmp > 0) {
+            TermsEnum.SeekStatus status = termsEnum.seekCeil(term);
+            switch (status) {
+              case FOUND:
+                return getDocs();
+              case NOT_FOUND:
+                readerTerm = termsEnum.term();
+                return null;
+              case END:
+                // no more terms in this segment
+                termsEnum = null;
+                return null;
+              default:
+                throw new AssertionError("unknown status");
+            }
+          }
+        } else if (termsEnum.seekExact(term)) {
+          return getDocs();
+        }
+      }
+      return null;
+    }
+
+    private boolean assertSorted(BytesRef term) {
+      assert sortedTerms;
+      assert lastTerm == null || term.compareTo(lastTerm) >= 0 : "boom: " + term.utf8ToString() + " last: " + lastTerm.utf8ToString();
+      lastTerm = BytesRef.deepCopyOf(term);
+      return true;
+    }
+
+    private DocIdSetIterator getDocs() throws IOException {
+      assert termsEnum != null;
+      return postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4272c631/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
index 00222f4..f169d1a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
@@ -35,7 +35,7 @@ public final class BytesRefArray implements SortableBytesRefArray {
   private int lastElement = 0;
   private int currentOffset = 0;
   private final Counter bytesUsed;
-  
+
   /**
    * Creates a new {@link BytesRefArray} with a counter to track allocated bytes
    */
@@ -190,7 +190,6 @@ public final class BytesRefArray implements SortableBytesRefArray {
     final int[] indices = comp == null ? null : sort(comp);
     return new BytesRefIterator() {
       int pos = 0;
-      
       @Override
       public BytesRef next() {
         if (pos < size) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4272c631/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java
index fa7bb45..b1bcefb 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java
@@ -35,14 +35,8 @@ public interface BytesRefIterator {
    *         the end of the iterator is reached.
    * @throws IOException If there is a low-level I/O error.
    */
-  public BytesRef next() throws IOException;
+  BytesRef next() throws IOException;
   
   /** Singleton BytesRefIterator that iterates over 0 BytesRefs. */
-  public static final BytesRefIterator EMPTY = new BytesRefIterator() {
-
-    @Override
-    public BytesRef next() {
-      return null;
-    }
-  };
+  BytesRefIterator EMPTY = () -> null;
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4272c631/lucene/core/src/test/org/apache/lucene/index/TestFrozenBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFrozenBufferedUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestFrozenBufferedUpdates.java
new file mode 100644
index 0000000..e8e50fa
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFrozenBufferedUpdates.java
@@ -0,0 +1,104 @@
+/*
+ * 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.lucene.index;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefArray;
+import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.index.FrozenBufferedUpdates.TermDocsIterator;
+import org.apache.lucene.util.TestUtil;
+
+public class TestFrozenBufferedUpdates extends LuceneTestCase {
+
+  public void testTermDocsIterator() throws IOException {
+    for (int j = 0; j < 5; j++) {
+      try (Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig())) {
+        boolean duplicates = random().nextBoolean();
+        boolean nonMatches = random().nextBoolean();
+        BytesRefArray array = new BytesRefArray(Counter.newCounter());
+        int numDocs = 10 + random().nextInt(1000);
+        Set<BytesRef> randomIds = new HashSet<>();
+        for (int i = 0; i < numDocs; i++) {
+          BytesRef id;
+          do {
+             id = new BytesRef(TestUtil.randomRealisticUnicodeString(random()));
+          } while (randomIds.add(id) == false);
+        }
+        List<BytesRef> asList = new ArrayList<>(randomIds);
+        for (BytesRef ref : randomIds) {
+          Document doc = new Document();
+          doc.add(new StringField("field", ref, Field.Store.NO));
+          array.append(ref);
+          if (duplicates && rarely()) {
+            array.append(RandomPicks.randomFrom(random(), asList));
+          }
+          if (nonMatches && rarely()) {
+            BytesRef id;
+            do {
+              id = new BytesRef(TestUtil.randomRealisticUnicodeString(random()));
+            } while (randomIds.contains(id));
+            array.append(id);
+          }
+          writer.addDocument(doc);
+        }
+        writer.forceMerge(1);
+        writer.commit();
+        try (DirectoryReader reader = DirectoryReader.open(dir)) {
+          boolean sorted = random().nextBoolean();
+          BytesRefIterator values = sorted ? array.iterator(Comparator.naturalOrder()) : array.iterator();
+          assertEquals(1, reader.leaves().size());
+          TermDocsIterator iterator = new TermDocsIterator(reader.leaves().get(0).reader(), sorted);
+          FixedBitSet bitSet = new FixedBitSet(reader.maxDoc());
+          BytesRef ref;
+          while ((ref = values.next()) != null) {
+            DocIdSetIterator docIdSetIterator = iterator.nextTerm("field", ref);
+            if (nonMatches == false) {
+              assertNotNull(docIdSetIterator);
+            }
+            if (docIdSetIterator != null) {
+              int doc;
+              while ((doc = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+                if (duplicates == false) {
+                  assertFalse(bitSet.get(doc));
+                }
+                bitSet.set(doc);
+              }
+            }
+          }
+          assertEquals(reader.maxDoc(), bitSet.cardinality());
+        }
+      }
+    }
+  }
+}


[02/34] lucene-solr:jira/http2: Revert "test"

Posted by da...@apache.org.
Revert "test"

This reverts commit 783f01bab3e096f4f843b9de62af25d6fbe07340.


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

Branch: refs/heads/jira/http2
Commit: 21dc2372486177d87be4c84742306328eccd0ed4
Parents: adf73ff
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Dec 11 10:11:27 2018 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 12 13:20:37 2018 +0100

----------------------------------------------------------------------
 .../index/BinaryDocValuesFieldUpdates.java      |  8 ---
 .../lucene/index/DocValuesFieldUpdates.java     | 56 ++++----------------
 .../index/NumericDocValuesFieldUpdates.java     |  6 ---
 3 files changed, 10 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21dc2372/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
index 67227ed..fa60929 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
@@ -108,14 +108,6 @@ final class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
   }
 
   @Override
-  protected void copy(int i, int j) {
-    super.copy(i, j);
-
-    offsets.set(j, offsets.get(i));
-    lengths.set(j, lengths.get(i));
-  }
-
-  @Override
   protected void grow(int size) {
     super.grow(size);
     offsets = offsets.grow(size);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21dc2372/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
index 71d9c7d..464a7f2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
@@ -21,10 +21,10 @@ import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.SparseFixedBitSet;
-import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PagedMutable;
 
@@ -32,11 +32,11 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 /**
  * Holds updates of a single DocValues field, for a set of documents within one segment.
- *
+ * 
  * @lucene.experimental
  */
 abstract class DocValuesFieldUpdates implements Accountable {
-
+  
   protected static final int PAGE_SIZE = 1024;
   private static final long HAS_VALUE_MASK = 1;
   private static final long HAS_NO_VALUE_MASK = 0;
@@ -260,7 +260,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
   final boolean getFinished() {
     return finished;
   }
-
+  
   abstract void add(int doc, long value);
 
   abstract void add(int doc, BytesRef value);
@@ -285,17 +285,11 @@ abstract class DocValuesFieldUpdates implements Accountable {
       throw new IllegalStateException("already finished");
     }
     finished = true;
-
-    // Swaps and comparisons are quite costly with all these packed ints,
-    // so we use TimSorter instead of InPlaceMergeSorter, which has the nice
-    // property of doing fewer swaps/copies at the expense of a higher memory
-    // usage
-    final int tempSlotsOffset = size;
-    // we grow the current structures to get temporary storage
-    resize(Math.addExact(size, size / 2)); // 50% extra memory for temp slots
-    final int numTempSlots = Math.toIntExact(docs.size() - size);
-    new TimSorter(numTempSlots) {
-
+    // shrink wrap
+    if (size < docs.size()) {
+      resize(size);
+    }
+    new InPlaceMergeSorter() {
       @Override
       protected void swap(int i, int j) {
         DocValuesFieldUpdates.this.swap(i, j);
@@ -308,33 +302,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
         // stable and preserving original order so the last update to that docID wins
         return Long.compare(docs.get(i)>>>1, docs.get(j)>>>1);
       }
-
-      @Override
-      protected void save(int i, int len) {
-        assert len <= numTempSlots;
-        for (int k = 0; k < len; ++k) {
-          copy(i + k, tempSlotsOffset + k);
-        }
-      }
-
-      @Override
-      protected void restore(int src, int dest) {
-        copy(tempSlotsOffset + src, dest);
-      }
-
-      @Override
-      protected void copy(int src, int dest) {
-        DocValuesFieldUpdates.this.copy(src, dest);
-      }
-
-      @Override
-      protected int compareSaved(int i, int j) {
-        return compare(tempSlotsOffset + i, j);
-      }
     }.sort(0, size);
-
-    // shrink wrap to save memory, this will also release temporary storage
-    resize(size);
   }
 
   /** Returns true if this instance contains any updates. */
@@ -382,10 +350,6 @@ abstract class DocValuesFieldUpdates implements Accountable {
     docs.set(i, tmpDoc);
   }
 
-  protected void copy(int i, int j) {
-    docs.set(j, docs.get(i));
-  }
-
   protected void grow(int size) {
     docs = docs.grow(size);
   }
@@ -513,7 +477,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
     }
 
     protected abstract BytesRef binaryValue();
-
+    
     protected abstract long longValue();
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21dc2372/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
index 05d5c14..550a86a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
@@ -100,12 +100,6 @@ final class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
   }
 
   @Override
-  protected void copy(int i, int j) {
-    super.copy(i, j);
-    values.set(j, values.get(i));
-  }
-
-  @Override
   protected void grow(int size) {
     super.grow(size);
     values = values.grow(size);


[23/34] lucene-solr:jira/http2: SOLR-11296: Spellcheck parameters not working in new UI

Posted by da...@apache.org.
SOLR-11296: Spellcheck parameters not working in new UI

(cherry picked from commit 62edbf2e7da7af6d96d6c912379ee63c80459a70)


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

Branch: refs/heads/jira/http2
Commit: bb2405aec870974b5799520e8fb61bf56cf4b586
Parents: 62edbf2
Author: Jan Høydahl <ja...@apache.org>
Authored: Fri Dec 14 15:27:18 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Fri Dec 14 15:27:18 2018 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                       | 2 ++
 solr/webapp/web/js/angular/services.js | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bb2405ae/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b4bdf37..34c9624 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -177,6 +177,8 @@ Bug Fixes
 
 * SOLR-11296: Spellcheck parameters not working in new UI (Matt Pearce via janhoy)
 
+* SOLR-10975: New Admin UI Query does not URL-encode the query produced in the URL box (janhoy)
+
 Improvements
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bb2405ae/solr/webapp/web/js/angular/services.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/services.js b/solr/webapp/web/js/angular/services.js
index 8eb148f..3391221 100644
--- a/solr/webapp/web/js/angular/services.js
+++ b/solr/webapp/web/js/angular/services.js
@@ -235,7 +235,7 @@ solrAdminServices.factory('System',
            for (key in params) {
                if (key != "core" && key != "handler") {
                    for (var i in params[key]) {
-                       qs.push(key + "=" + params[key][i]);
+                       qs.push(key + "=" + encodeURIComponent(params[key][i]));
                    }
                }
            }


[09/34] lucene-solr:jira/http2: SOLR-13066: A failure while reloading a SolrCore can result in the SolrCore not being closed.

Posted by da...@apache.org.
SOLR-13066: A failure while reloading a SolrCore can result in the SolrCore not being closed.


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

Branch: refs/heads/jira/http2
Commit: 7de72c9bc7069dd4f59c54924fe8435f524023bd
Parents: 9728dbc
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 12 17:02:40 2018 -0600
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 12 18:49:06 2018 -0600

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../org/apache/solr/core/CoreContainer.java     | 23 +++--
 .../src/java/org/apache/solr/core/SolrCore.java | 98 ++++++++++----------
 3 files changed, 69 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7de72c9b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a01c16e..9e3f351 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -167,7 +167,9 @@ Bug Fixes
   
 * SOLR-12933: Fix SolrCloud distributed commit. (Mark Miller)
 
-* SOLR-13014: URI Too Long with large streaming expressions in SolrJ (janhoy) 
+* SOLR-13014: URI Too Long with large streaming expressions in SolrJ (janhoy)
+
+* SOLR-13066: A failure while reloading a SolrCore can result in the SolrCore not being closed. (Mark Miller)
 
 Improvements
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7de72c9b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 3051dbf..3579490 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -32,6 +32,7 @@ import static org.apache.solr.common.params.CommonParams.ZK_STATUS_PATH;
 import static org.apache.solr.core.CorePropertiesLocator.PROPERTIES_FILENAME;
 import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
@@ -1432,6 +1433,7 @@ public class CoreContainer {
     if (isShutDown) {
       throw new AlreadyClosedException();
     }
+    SolrCore newCore = null;
     SolrCore core = solrCores.getCoreFromAnyList(name, false);
     if (core != null) {
 
@@ -1439,34 +1441,41 @@ public class CoreContainer {
       // CoreDescriptor and we need to reload it from the disk files
       CoreDescriptor cd = reloadCoreDescriptor(core.getCoreDescriptor());
       solrCores.addCoreDescriptor(cd);
+      Closeable oldCore = null;
+      boolean success = false;
       try {
         solrCores.waitAddPendingCoreOps(cd.getName());
         ConfigSet coreConfig = coreConfigService.getConfig(cd);
         log.info("Reloading SolrCore '{}' using configuration from {}", cd.getName(), coreConfig.getName());
-        SolrCore newCore = core.reload(coreConfig);
+        newCore = core.reload(coreConfig);
+
         registerCore(cd, newCore, false, false);
+
         if (getZkController() != null) {
           DocCollection docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
           Replica replica = docCollection.getReplica(cd.getCloudDescriptor().getCoreNodeName());
           assert replica != null;
-          if (replica.getType() == Replica.Type.TLOG) { //TODO: needed here?
+          if (replica.getType() == Replica.Type.TLOG) { // TODO: needed here?
             getZkController().stopReplicationFromLeader(core.getName());
             if (!cd.getCloudDescriptor().isLeader()) {
               getZkController().startReplicationFromLeader(newCore.getName(), true);
             }
 
-          } else if(replica.getType() == Replica.Type.PULL) {
+          } else if (replica.getType() == Replica.Type.PULL) {
             getZkController().stopReplicationFromLeader(core.getName());
             getZkController().startReplicationFromLeader(newCore.getName(), false);
           }
         }
+        success = true;
       } catch (SolrCoreState.CoreIsClosedException e) {
         throw e;
       } catch (Exception e) {
-        coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, e));
+        coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, (Exception) e));
         throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to reload core [" + cd.getName() + "]", e);
-      }
-      finally {
+      } finally {
+        if (!success && newCore != null && newCore.getOpenCount() > 0) {
+          IOUtils.closeQuietly(newCore);
+        }
         solrCores.removeFromPendingOps(cd.getName());
       }
     } else {
@@ -1479,7 +1488,7 @@ public class CoreContainer {
           solrCores.removeFromPendingOps(clf.cd.getName());
         }
       } else {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + name );
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + name);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7de72c9b/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index e66ca89..b64a6c6 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -674,7 +674,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
         return core;
       } finally {
         // close the new core on any errors that have occurred.
-        if (!success) {
+        if (!success && core != null && core.getOpenCount() > 0) {
           IOUtils.closeQuietly(core);
         }
       }
@@ -896,63 +896,63 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
   public SolrCore(CoreContainer coreContainer, String name, String dataDir, SolrConfig config,
                   IndexSchema schema, NamedList configSetProperties,
                   CoreDescriptor coreDescriptor, UpdateHandler updateHandler,
-                  IndexDeletionPolicyWrapper delPolicy, SolrCore prev, boolean reload) {
+      IndexDeletionPolicyWrapper delPolicy, SolrCore prev, boolean reload) {
 
-    this.coreContainer = coreContainer;
-    
     assert ObjectReleaseTracker.track(searcherExecutor); // ensure that in unclean shutdown tests we still close this
 
-    CoreDescriptor cd = Objects.requireNonNull(coreDescriptor, "coreDescriptor cannot be null");
-    coreContainer.solrCores.addCoreDescriptor(cd);
+    this.coreContainer = coreContainer;
 
-    setName(name);
-    MDCLoggingContext.setCore(this);
-    
-    resourceLoader = config.getResourceLoader();
-    this.solrConfig = config;
-    this.configSetProperties = configSetProperties;
-    // Initialize the metrics manager
-    this.coreMetricManager = initCoreMetricManager(config);
-    this.coreMetricManager.loadReporters();
+    final CountDownLatch latch = new CountDownLatch(1);
 
-    if (updateHandler == null) {
-      directoryFactory = initDirectoryFactory();
-      recoveryStrategyBuilder = initRecoveryStrategyBuilder();
-      solrCoreState = new DefaultSolrCoreState(directoryFactory, recoveryStrategyBuilder);
-    } else {
-      solrCoreState = updateHandler.getSolrCoreState();
-      directoryFactory = solrCoreState.getDirectoryFactory();
-      recoveryStrategyBuilder = solrCoreState.getRecoveryStrategyBuilder();
-      isReloaded = true;
-    }
+    try {
+
+      CoreDescriptor cd = Objects.requireNonNull(coreDescriptor, "coreDescriptor cannot be null");
+      coreContainer.solrCores.addCoreDescriptor(cd);
 
-    this.dataDir = initDataDir(dataDir, config, coreDescriptor);
-    this.ulogDir = initUpdateLogDir(coreDescriptor);
+      setName(name);
+      MDCLoggingContext.setCore(this);
 
-    log.info("[{}] Opening new SolrCore at [{}], dataDir=[{}]", logid, resourceLoader.getInstancePath(), this.dataDir);
+      resourceLoader = config.getResourceLoader();
+      this.solrConfig = config;
+      this.configSetProperties = configSetProperties;
+      // Initialize the metrics manager
+      this.coreMetricManager = initCoreMetricManager(config);
+      this.coreMetricManager.loadReporters();
 
-    checkVersionFieldExistsInSchema(schema, coreDescriptor);
+      if (updateHandler == null) {
+        directoryFactory = initDirectoryFactory();
+        recoveryStrategyBuilder = initRecoveryStrategyBuilder();
+        solrCoreState = new DefaultSolrCoreState(directoryFactory, recoveryStrategyBuilder);
+      } else {
+        solrCoreState = updateHandler.getSolrCoreState();
+        directoryFactory = solrCoreState.getDirectoryFactory();
+        recoveryStrategyBuilder = solrCoreState.getRecoveryStrategyBuilder();
+        isReloaded = true;
+      }
 
-    SolrMetricManager metricManager = coreContainer.getMetricManager();
+      this.dataDir = initDataDir(dataDir, config, coreDescriptor);
+      this.ulogDir = initUpdateLogDir(coreDescriptor);
 
-    // initialize searcher-related metrics
-    initializeMetrics(metricManager, coreMetricManager.getRegistryName(), metricTag, null);
+      log.info("[{}] Opening new SolrCore at [{}], dataDir=[{}]", logid, resourceLoader.getInstancePath(),
+          this.dataDir);
 
-    SolrFieldCacheBean solrFieldCacheBean = new SolrFieldCacheBean();
-    // this is registered at the CONTAINER level because it's not core-specific - for now we
-    // also register it here for back-compat
-    solrFieldCacheBean.initializeMetrics(metricManager, coreMetricManager.getRegistryName(), metricTag, "core");
-    infoRegistry.put("fieldCache", solrFieldCacheBean);
+      checkVersionFieldExistsInSchema(schema, coreDescriptor);
 
+      SolrMetricManager metricManager = coreContainer.getMetricManager();
 
-    initSchema(config, schema);
+      // initialize searcher-related metrics
+      initializeMetrics(metricManager, coreMetricManager.getRegistryName(), metricTag, null);
 
-    this.maxWarmingSearchers = config.maxWarmingSearchers;
-    this.slowQueryThresholdMillis = config.slowQueryThresholdMillis;
+      SolrFieldCacheBean solrFieldCacheBean = new SolrFieldCacheBean();
+      // this is registered at the CONTAINER level because it's not core-specific - for now we
+      // also register it here for back-compat
+      solrFieldCacheBean.initializeMetrics(metricManager, coreMetricManager.getRegistryName(), metricTag, "core");
+      infoRegistry.put("fieldCache", solrFieldCacheBean);
 
-    final CountDownLatch latch = new CountDownLatch(1);
+      initSchema(config, schema);
 
-    try {
+      this.maxWarmingSearchers = config.maxWarmingSearchers;
+      this.slowQueryThresholdMillis = config.slowQueryThresholdMillis;
 
       initListeners();
 
@@ -961,7 +961,9 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
       this.codec = initCodec(solrConfig, this.schema);
 
-      memClassLoader = new MemClassLoader(PluginBag.RuntimeLib.getLibObjects(this, solrConfig.getPluginInfos(PluginBag.RuntimeLib.class.getName())), getResourceLoader());
+      memClassLoader = new MemClassLoader(
+          PluginBag.RuntimeLib.getLibObjects(this, solrConfig.getPluginInfos(PluginBag.RuntimeLib.class.getName())),
+          getResourceLoader());
       initIndex(prev != null, reload);
 
       initWriters();
@@ -987,7 +989,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       });
 
       this.updateHandler = initUpdateHandler(updateHandler);
-      
+
       initSearcher(prev);
 
       // Initialize the RestManager
@@ -997,7 +999,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       resourceLoader.inform(resourceLoader);
       resourceLoader.inform(this); // last call before the latch is released.
       this.updateHandler.informEventListeners(this);
-   
+
       infoRegistry.put("core", this);
 
       // register any SolrInfoMBeans SolrResourceLoader initialized
@@ -1029,13 +1031,13 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       // should be fine, since counting down on a latch of 0 is still fine
       latch.countDown();
       if (e instanceof OutOfMemoryError) {
-        throw (OutOfMemoryError)e;
+        throw (OutOfMemoryError) e;
       }
 
       try {
         // close down the searcher and any other resources, if it exists, as this
         // is not recoverable
-       close();
+        close();
       } catch (Throwable t) {
         if (t instanceof OutOfMemoryError) {
           throw (OutOfMemoryError) t;
@@ -1048,7 +1050,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       // allow firstSearcher events to fire and make sure it is released
       latch.countDown();
     }
-    
+
     assert ObjectReleaseTracker.track(this);
   }
 


[07/34] lucene-solr:jira/http2: SOLR-12801: Suppress SSL and older codecs

Posted by da...@apache.org.
SOLR-12801: Suppress SSL and older codecs


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

Branch: refs/heads/jira/http2
Commit: 1e687268316369102306085f8c5410d62b5dafaf
Parents: 7e4555a
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Dec 12 15:38:40 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Dec 12 15:38:40 2018 -0500

----------------------------------------------------------------------
 solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1e687268/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
index eff3a6c..69ad469 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
@@ -25,6 +25,8 @@ import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -45,6 +47,9 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+@LuceneTestCase.Slow
+@SolrTestCaseJ4.SuppressSSL
+@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
 public class TestSQLHandler extends AbstractFullDistribZkTestBase {
 
   static {


[25/34] lucene-solr:jira/http2: SOLR-10975: New Admin UI Query does not URL-encode the query produced in the URL box

Posted by da...@apache.org.
SOLR-10975: New Admin UI Query does not URL-encode the query produced in the URL box


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

Branch: refs/heads/jira/http2
Commit: bfd28a8b3a1229978f68e4aa21848290fcfb3559
Parents: f8fcdbb2
Author: Jan Høydahl <ja...@apache.org>
Authored: Fri Dec 14 15:30:39 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Fri Dec 14 15:30:39 2018 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                       | 2 ++
 solr/webapp/web/js/angular/services.js | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfd28a8b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b4bdf37..34c9624 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -177,6 +177,8 @@ Bug Fixes
 
 * SOLR-11296: Spellcheck parameters not working in new UI (Matt Pearce via janhoy)
 
+* SOLR-10975: New Admin UI Query does not URL-encode the query produced in the URL box (janhoy)
+
 Improvements
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfd28a8b/solr/webapp/web/js/angular/services.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/services.js b/solr/webapp/web/js/angular/services.js
index 8eb148f..3391221 100644
--- a/solr/webapp/web/js/angular/services.js
+++ b/solr/webapp/web/js/angular/services.js
@@ -235,7 +235,7 @@ solrAdminServices.factory('System',
            for (key in params) {
                if (key != "core" && key != "handler") {
                    for (var i in params[key]) {
-                       qs.push(key + "=" + params[key][i]);
+                       qs.push(key + "=" + encodeURIComponent(params[key][i]));
                    }
                }
            }


[24/34] lucene-solr:jira/http2: Revert "SOLR-11296: Spellcheck parameters not working in new UI" due to wrong commit message

Posted by da...@apache.org.
Revert "SOLR-11296: Spellcheck parameters not working in new UI" due to wrong commit message

This reverts commit bb2405a


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

Branch: refs/heads/jira/http2
Commit: f8fcdbb20852c4980558e00dbb09cac400fc108d
Parents: bb2405a
Author: Jan Høydahl <ja...@apache.org>
Authored: Fri Dec 14 15:29:27 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Fri Dec 14 15:29:27 2018 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                       | 2 --
 solr/webapp/web/js/angular/services.js | 2 +-
 2 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8fcdbb2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 34c9624..b4bdf37 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -177,8 +177,6 @@ Bug Fixes
 
 * SOLR-11296: Spellcheck parameters not working in new UI (Matt Pearce via janhoy)
 
-* SOLR-10975: New Admin UI Query does not URL-encode the query produced in the URL box (janhoy)
-
 Improvements
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8fcdbb2/solr/webapp/web/js/angular/services.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/services.js b/solr/webapp/web/js/angular/services.js
index 3391221..8eb148f 100644
--- a/solr/webapp/web/js/angular/services.js
+++ b/solr/webapp/web/js/angular/services.js
@@ -235,7 +235,7 @@ solrAdminServices.factory('System',
            for (key in params) {
                if (key != "core" && key != "handler") {
                    for (var i in params[key]) {
-                       qs.push(key + "=" + encodeURIComponent(params[key][i]));
+                       qs.push(key + "=" + params[key][i]);
                    }
                }
            }


[11/34] lucene-solr:jira/http2: SOLR-13067: Harden BasicAuthIntegrationTest.

Posted by da...@apache.org.
SOLR-13067: Harden BasicAuthIntegrationTest.


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

Branch: refs/heads/jira/http2
Commit: 44b51cd041371051d0b73b54afebc99fc0fa4862
Parents: 7de72c9
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 12 16:00:08 2018 -0600
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 12 18:50:23 2018 -0600

----------------------------------------------------------------------
 .../solr/security/BasicAuthIntegrationTest.java | 46 +++++++++++++-------
 .../solr/cloud/SolrCloudAuthTestCase.java       |  2 +
 2 files changed, 32 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/44b51cd0/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index f704cdf..8c04933 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -16,6 +16,9 @@
  */
 package org.apache.solr.security;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Collections.singletonMap;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -28,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 
 import com.codahale.metrics.MetricRegistry;
@@ -62,22 +66,20 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.SolrCLI;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Collections.singletonMap;
-
 public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final String COLLECTION = "authCollection";
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  @Before
+  public void setupCluster() throws Exception {
     configureCluster(3)
         .addConfig("conf", configset("cloud-minimal"))
         .configure();
@@ -86,6 +88,11 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
     
     cluster.waitForActiveCollection(COLLECTION, 3, 3);
   }
+  
+  @After
+  public void tearDownCluster() throws Exception {
+    shutdownCluster();
+  }
 
   @Test
   //commented 9-Aug-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 21-May-2018
@@ -113,10 +120,12 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
       
       cluster.waitForJettyToStop(randomJetty);
       
-      randomJetty.start(false);
+      randomJetty.start();
       
       cluster.waitForAllNodes(30);
       
+      cluster.waitForActiveCollection(COLLECTION, 3, 3);
+      
       baseUrl = randomJetty.getBaseUrl().toString();
       verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/class", "solr.BasicAuthPlugin", 20);
       assertNumberOfMetrics(16); // Basic auth metrics available
@@ -135,7 +144,10 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
         ((GenericSolrRequest)genericReq).setContentWriter(new StringPayloadContentWriter(command, CommonParams.JSON_MIME));
       }
 
-
+      // avoid bad connection races due to shutdown
+      cluster.getSolrClient().getHttpClient().getConnectionManager().closeExpiredConnections();
+      cluster.getSolrClient().getHttpClient().getConnectionManager().closeIdleConnections(1, TimeUnit.MILLISECONDS);
+      
       HttpSolrClient.RemoteSolrException exp = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
         cluster.getSolrClient().request(genericReq);
       });
@@ -167,8 +179,10 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
           "}";
 
       executeCommand(baseUrl + authzPrefix, cl,command, "solr", "SolrRocks");
-      assertAuthMetricsMinimums(6, 2, 4, 0, 0, 0);
+      assertAuthMetricsMinimums(5, 2, 3, 0, 0, 0);
 
+      Thread.sleep(2000); // sad little wait to try and avoid other clients from hitting http noresponse after jetty restart
+      
       baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
       verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/user-role/harry", NOT_NULL_PREDICATE, 20);
 
@@ -178,12 +192,12 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
               "role", "dev"))), "harry", "HarryIsUberCool" );
 
       verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[1]/collection", "x", 20);
-      assertAuthMetricsMinimums(9, 3, 6, 0, 0, 0);
+      assertAuthMetricsMinimums(8, 3, 5, 0, 0, 0);
 
       executeCommand(baseUrl + authzPrefix, cl,Utils.toJSONString(singletonMap("set-permission", Utils.makeMap
           ("name", "collection-admin-edit", "role", "admin"))), "harry", "HarryIsUberCool"  );
       verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[2]/name", "collection-admin-edit", 20);
-      assertAuthMetricsMinimums(11, 4, 7, 0, 0, 0);
+      assertAuthMetricsMinimums(10, 4, 6, 0, 0, 0);
 
       CollectionAdminRequest.Reload reload = CollectionAdminRequest.reloadCollection(COLLECTION);
 
@@ -210,7 +224,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
             .setBasicAuthCredentials("harry", "Cool12345"));
         fail("This should not succeed");
       } catch (HttpSolrClient.RemoteSolrException e) {
-        assertAuthMetricsMinimums(15, 5, 9, 1, 0, 0);
+        assertAuthMetricsMinimums(14, 5, 8, 1, 0, 0);
       }
 
       executeCommand(baseUrl + authzPrefix, cl,"{set-permission : { name : update , role : admin}}", "harry", "HarryIsUberCool");
@@ -227,7 +241,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: true}}", "harry", "HarryIsUberCool");
       verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/blockUnknown", "true", 20, "harry", "HarryIsUberCool");
       verifySecurityStatus(cl, baseUrl + "/admin/info/key", "key", NOT_NULL_PREDICATE, 20);
-      assertAuthMetricsMinimums(18, 8, 9, 1, 0, 0);
+      assertAuthMetricsMinimums(17, 8, 8, 1, 0, 0);
 
       String[] toolArgs = new String[]{
           "status", "-solr", baseUrl};
@@ -253,7 +267,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
         cluster.getSolrClient().query(COLLECTION, params);
       });
       assertEquals(401, exp.code());
-      assertAuthMetricsMinimums(20, 8, 9, 1, 2, 0);
+      assertAuthMetricsMinimums(19, 8, 8, 1, 2, 0);
       assertPkiAuthMetricsMinimums(3, 3, 0, 0, 0, 0);
 
       // Query that succeeds
@@ -261,8 +275,8 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
       req.setBasicAuthCredentials("harry", "HarryIsUberCool");
       cluster.getSolrClient().request(req, COLLECTION);
       
-      assertAuthMetricsMinimums(21, 9, 9, 1, 2, 0);
-      assertPkiAuthMetricsMinimums(6, 6, 0, 0, 0, 0);
+      assertAuthMetricsMinimums(20, 8, 8, 1, 2, 0);
+      assertPkiAuthMetricsMinimums(5, 5, 0, 0, 0, 0);
 
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: false}}", "harry", "HarryIsUberCool");
     } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/44b51cd0/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
index 3966426..c68526e 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
@@ -56,6 +56,8 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
   
   /**
    * Used to check metric counts for the AuthPlugin in use (except PKI)
+   * 
+   * TODO: many of these params have to be under specified - this should wait a bit to see the desired params and timeout
    */
   protected void assertAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
     assertAuthMetricsMinimums(METRICS_PREFIX, requests, authenticated, passThrough, failWrongCredentials, failMissingCredentials, errors);


[14/34] lucene-solr:jira/http2: SOLR-13037: Harden TestSimGenericDistributedQueue

Posted by da...@apache.org.
SOLR-13037: Harden TestSimGenericDistributedQueue


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

Branch: refs/heads/jira/http2
Commit: d7ad2f46c366e466b45d129c737ffd3f125cff5a
Parents: 81dbad5
Author: Jason Gerlowski <ge...@apache.org>
Authored: Wed Dec 12 22:01:07 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Thu Dec 13 06:27:34 2018 -0500

----------------------------------------------------------------------
 .../autoscaling/sim/SimDistribStateManager.java | 76 ++++++++++++--------
 .../sim/TestSimGenericDistributedQueue.java     |  3 -
 2 files changed, 46 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d7ad2f46/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
index 7b73038..afccdaa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
@@ -311,17 +311,7 @@ public class SimDistribStateManager implements DistribStateManager {
       n = parentNode.children != null ? parentNode.children.get(currentName) : null;
       if (n == null) {
         if (create) {
-          if ((parentNode.mode == CreateMode.EPHEMERAL || parentNode.mode == CreateMode.EPHEMERAL_SEQUENTIAL) &&
-              (mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL)) {
-            throw new IOException("NoChildrenEphemerals for " + parentNode.path);
-          }
-          if (CreateMode.PERSISTENT_SEQUENTIAL == mode || CreateMode.EPHEMERAL_SEQUENTIAL == mode) {
-            currentName = currentName + String.format(Locale.ROOT, "%010d", parentNode.seq);
-            parentNode.seq++;
-          }
-          currentPath.append(currentName);
-          n = new Node(parentNode, currentName, currentPath.toString(), mode, id);
-          parentNode.setChild(currentName, n);
+          n = createNode(parentNode, mode, currentPath, currentName,true);
         } else {
           break;
         }
@@ -333,6 +323,26 @@ public class SimDistribStateManager implements DistribStateManager {
     return n;
   }
 
+  private Node createNode(Node parentNode, CreateMode mode, StringBuilder fullChildPath, String baseChildName, boolean attachToParent) throws IOException {
+    String nodeName = baseChildName;
+    if ((parentNode.mode == CreateMode.EPHEMERAL || parentNode.mode == CreateMode.EPHEMERAL_SEQUENTIAL) &&
+        (mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL)) {
+      throw new IOException("NoChildrenEphemerals for " + parentNode.path);
+    }
+    if (CreateMode.PERSISTENT_SEQUENTIAL == mode || CreateMode.EPHEMERAL_SEQUENTIAL == mode) {
+      nodeName = nodeName + String.format(Locale.ROOT, "%010d", parentNode.seq);
+      parentNode.seq++;
+    }
+
+    fullChildPath.append(nodeName);
+    Node child = new Node(parentNode, nodeName, fullChildPath.toString(), mode, id);
+
+    if (attachToParent) {
+      parentNode.setChild(nodeName, child);
+    }
+    return child;
+  }
+
   @Override
   public void close() throws IOException {
     multiLock.lock();
@@ -444,33 +454,39 @@ public class SimDistribStateManager implements DistribStateManager {
     if ((CreateMode.EPHEMERAL == mode || CreateMode.PERSISTENT == mode) && hasData(path)) {
       throw new AlreadyExistsException(path);
     }
-    // check if parent exists
+
     String relPath = path.charAt(0) == '/' ? path.substring(1) : path;
-    if (relPath.length() > 0) { // non-root path - check if parent exists
-      String[] elements = relPath.split("/");
-      StringBuilder sb = new StringBuilder();
-      for (int i = 0; i < elements.length - 1; i++) {
-        sb.append('/');
-        sb.append(elements[i]);
-      }
-      if (!hasData(sb.toString())) {
-        throw new NoSuchElementException(sb.toString());
-      }
+    if (relPath.length() == 0) {
+      // TODO should trying to create a root node throw an exception since its always init'd in the ctor?
+      return null;
     }
-    Node n = null;
-    multiLock.lock();
-    try {
-      n = traverse(path, true, mode);
-    } finally {
-      multiLock.unlock();
+
+    // non-root-node.  Make sure parent exists.
+    String[] elements = relPath.split("/");
+    StringBuilder parentStringBuilder = new StringBuilder();
+    for (int i = 0; i < elements.length - 1; i++) {
+      parentStringBuilder.append('/');
+      parentStringBuilder.append(elements[i]);
+    }
+    if (!hasData(parentStringBuilder.toString())) {
+      throw new NoSuchElementException(parentStringBuilder.toString());
     }
+
+    multiLock.lock();
     try {
-      n.setData(data, -1);
-      return n.path;
+      String nodeName = elements[elements.length-1];
+      Node parentNode = traverse(parentStringBuilder.toString(), false, mode);
+      Node childNode = createNode(parentNode, mode, parentStringBuilder.append("/"), nodeName, false);
+      childNode.setData(data, -1);
+      parentNode.setChild(childNode.name, childNode);
+      return childNode.path;
     } catch (BadVersionException e) {
       // not happening
       return null;
+    } finally {
+      multiLock.unlock();
     }
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d7ad2f46/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimGenericDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimGenericDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimGenericDistributedQueue.java
index 68c37a4..1ff6d12 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimGenericDistributedQueue.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimGenericDistributedQueue.java
@@ -16,15 +16,12 @@
  */
 package org.apache.solr.cloud.autoscaling.sim;
 
-import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
-
 import org.apache.solr.client.solrj.cloud.DistributedQueue;
 import org.apache.solr.client.solrj.cloud.DistribStateManager;
 
 /**
  *
  */
-@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13037")
 public class TestSimGenericDistributedQueue extends TestSimDistributedQueue {
   DistribStateManager stateManager = new SimDistribStateManager();
 


[12/34] lucene-solr:jira/http2: LUCENE-8607: MatchAllDocsQuery can skip counting hits

Posted by da...@apache.org.
LUCENE-8607: MatchAllDocsQuery can skip counting hits


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

Branch: refs/heads/jira/http2
Commit: fa025e1f78786c57130399acb61fdd81092835f6
Parents: 42f1373
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Dec 13 09:01:49 2018 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 13 09:01:49 2018 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +++
 .../apache/lucene/search/MatchAllDocsQuery.java |  4 +++-
 .../lucene/search/TestMatchAllDocsQuery.java    | 23 ++++++++++++++++++++
 3 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa025e1f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b4c7ca5..d6276aa 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -220,6 +220,9 @@ Optimizations
   to early terminate the iterator if the minimum score is greater than the constant
   score. (Christophe Bismuth via Jim Ferenczi)
 
+* LUCENE-8607: MatchAllDocsQuery can shortcut when total hit count is not
+  required (Alan Woodward, Adrien Grand)
+
 ======================= Lucene 7.7.0 =======================
 
 Build

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa025e1f/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
index 826494e..80e8d32 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
@@ -24,7 +24,6 @@ import org.apache.lucene.util.Bits;
 
 /**
  * A query that matches all documents.
- *
  */
 public final class MatchAllDocsQuery extends Query {
 
@@ -47,6 +46,9 @@ public final class MatchAllDocsQuery extends Query {
 
       @Override
       public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        if (scoreMode == ScoreMode.TOP_SCORES) {
+          return super.bulkScorer(context);
+        }
         final float score = score();
         final int maxDoc = context.reader().maxDoc();
         return new BulkScorer() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa025e1f/lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java
index ee02eed..a2e794e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java
@@ -99,4 +99,27 @@ public class TestMatchAllDocsQuery extends LuceneTestCase {
     iw.addDocument(doc);
   }
 
+  public void testEarlyTermination() throws IOException {
+
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(analyzer).setMaxBufferedDocs(2).setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < 500; i++) {
+      addDoc("doc" + i, iw);
+    }
+    IndexReader ir = DirectoryReader.open(iw);
+
+    IndexSearcher is = newSearcher(ir);
+
+    final int totalHitsThreshold = 200;
+    TopScoreDocCollector c = TopScoreDocCollector.create(10, null, totalHitsThreshold);
+
+    is.search(new MatchAllDocsQuery(), c);
+    assertEquals(totalHitsThreshold, c.totalHits);
+
+    iw.close();
+    ir.close();
+    dir.close();
+
+  }
+
 }


[13/34] lucene-solr:jira/http2: SOLR-12799: Allow Authentication Plugins to intercept internode requests on a per-request basis Add 'forwardCredentials' parameter to BasicAuth which will then skip using PKI on sub requests

Posted by da...@apache.org.
SOLR-12799: Allow Authentication Plugins to intercept internode requests on a per-request basis
Add 'forwardCredentials' parameter to BasicAuth which will then skip using PKI on sub requests


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

Branch: refs/heads/jira/http2
Commit: 81dbad54e0af82cfe830f4926f06b57bb8b02f9e
Parents: fa025e1
Author: Jan Høydahl <ja...@apache.org>
Authored: Thu Dec 13 10:51:56 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Thu Dec 13 10:51:56 2018 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +
 .../org/apache/solr/core/CoreContainer.java     | 19 ++--
 .../handler/component/HttpShardHandler.java     |  3 +
 .../solr/security/AuthenticationPlugin.java     | 22 +++++
 .../apache/solr/security/BasicAuthPlugin.java   | 93 ++++++++++++++++++--
 .../solr/security/PKIAuthenticationPlugin.java  | 22 ++---
 .../apache/solr/servlet/SolrDispatchFilter.java |  2 -
 .../apache/solr/update/SolrCmdDistributor.java  |  9 +-
 .../solr/core/TestQuerySenderListener.java      |  8 +-
 .../solr/security/BasicAuthIntegrationTest.java | 63 ++++++++++---
 .../security/TestPKIAuthenticationPlugin.java   |  5 --
 ...uthentication-and-authorization-plugins.adoc |  8 +-
 .../src/basic-authentication-plugin.adoc        | 14 +--
 .../apache/solr/client/solrj/SolrRequest.java   | 11 +++
 .../solr/client/solrj/impl/HttpSolrClient.java  | 13 ++-
 .../solr/cloud/SolrCloudAuthTestCase.java       | 42 ++++++---
 16 files changed, 263 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9e3f351..28baf4d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -79,6 +79,10 @@ New Features
 
 * SOLR-12593: The default configSet now includes an "ignored_*" dynamic field.  (David Smiley)
 
+* SOLR-12799: Allow Authentication Plugins to intercept internode requests on a per-request basis.
+  The BasicAuth plugin now supports a new parameter 'forwardCredentials', and when set to 'true',
+  user's BasicAuth credentials will be used instead of PKI for client initiated internode requests. (janhoy, noble) 
+
 * SOLR-12791: Add Metrics reporting for AuthenticationPlugin (janhoy)
 
 Bug Fixes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 3579490..dfd190b 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -110,6 +110,7 @@ import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.search.SolrFieldCacheBean;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationPlugin;
@@ -432,15 +433,11 @@ public class CoreContainer {
       }
 
       HttpClientUtil.setHttpClientRequestContextBuilder(httpClientBuilder);
-
-    } else {
-      if (pkiAuthenticationPlugin != null) {
-        //this happened due to an authc plugin reload. no need to register the pkiAuthc plugin again
-        if(pkiAuthenticationPlugin.isInterceptorRegistered()) return;
-        log.info("PKIAuthenticationPlugin is managing internode requests");
-        setupHttpClientForAuthPlugin(pkiAuthenticationPlugin);
-        pkiAuthenticationPlugin.setInterceptorRegistered();
-      }
+    }
+    // Always register PKI auth interceptor, which will then delegate the decision of who should secure
+    // each request to the configured authentication plugin.
+    if (pkiAuthenticationPlugin != null && !pkiAuthenticationPlugin.isInterceptorRegistered()) {
+      pkiAuthenticationPlugin.getHttpClientBuilder(HttpClientUtil.getHttpClientBuilder());
     }
   }
 
@@ -1888,6 +1885,10 @@ public class CoreContainer {
     return tragicException != null;
   }
 
+  static {
+    ExecutorUtil.addThreadLocalProvider(SolrRequestInfo.getInheritableThreadLocalProvider());
+  }
+
 }
 
 class CloserThread extends Thread {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
index a548031..e12e753 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
@@ -56,6 +56,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -154,6 +155,8 @@ public class HttpShardHandler extends ShardHandler {
 
         QueryRequest req = makeQueryRequest(sreq, params, shard);
         req.setMethod(SolrRequest.METHOD.POST);
+        SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
+        if (requestInfo != null) req.setUserPrincipal(requestInfo.getReq().getUserPrincipal());
 
         // no need to set the response parser as binary is the default
         // req.setResponseParser(new BinaryResponseParser());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index 48b073d..a8caf73 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -33,6 +33,9 @@ import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 
+import org.apache.http.HttpRequest;
+import org.apache.http.protocol.HttpContext;
+
 /**
  * 
  * @lucene.experimental
@@ -97,6 +100,25 @@ public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, S
   }
 
   /**
+   * Override this method to intercept internode requests. This allows your authentication
+   * plugin to decide on per-request basis whether it should handle inter-node requests or
+   * delegate to {@link PKIAuthenticationPlugin}. Return true to indicate that your plugin
+   * did handle the request, or false to signal that PKI plugin should handle it. This method
+   * will be called by {@link PKIAuthenticationPlugin}'s interceptor.
+   * 
+   * <p>
+   *   If not overridden, this method will return true for plugins implementing {@link HttpClientBuilderPlugin}.
+   *   This method can be overridden by subclasses e.g. to set HTTP headers, even if you don't use a clientBuilder. 
+   * </p>
+   * @param httpRequest the httpRequest that is about to be sent to another internal Solr node
+   * @param httpContext the context of that request.
+   * @return true if this plugin handled authentication for the request, else false
+   */
+  protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
+    return this instanceof HttpClientBuilderPlugin;
+  }
+  
+  /**
    * Cleanup any per request  data
    */
   public void closeRequest() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index 7ac5a7c..3db21cd 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.security;
 
+import javax.security.auth.Subject;
 import javax.servlet.FilterChain;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
@@ -23,21 +24,30 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpServletResponse;
 import java.io.IOException;
+import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
 import java.security.Principal;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.StringTokenizer;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.http.Header;
 import org.apache.http.HttpHeaders;
-import org.apache.http.auth.BasicUserPrincipal;
+import org.apache.http.HttpRequest;
+import org.apache.http.annotation.Contract;
+import org.apache.http.annotation.ThreadingBehavior;
+import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.message.BasicHeader;
+import org.apache.http.protocol.HttpContext;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SpecProvider;
 import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.ValidatingJsonMap;
@@ -50,6 +60,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
   private final static ThreadLocal<Header> authHeader = new ThreadLocal<>();
   private static final String X_REQUESTED_WITH_HEADER = "X-Requested-With";
   private boolean blockUnknown = false;
+  private boolean forwardCredentials = false;
 
   public boolean authenticate(String username, String pwd) {
     return authenticationProvider.authenticate(username, pwd);
@@ -62,7 +73,15 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       try {
         blockUnknown = Boolean.parseBoolean(o.toString());
       } catch (Exception e) {
-        log.error(e.getMessage());
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid value for parameter " + PROPERTY_BLOCK_UNKNOWN);
+      }
+    }
+    o = pluginConfig.get(FORWARD_CREDENTIALS);
+    if (o != null) {
+      try {
+        forwardCredentials = Boolean.parseBoolean(o.toString());
+      } catch (Exception e) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid value for parameter " + FORWARD_CREDENTIALS);
       }
     }
     authenticationProvider = getAuthenticationProvider(pluginConfig);
@@ -87,7 +106,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       ConfigEditablePlugin editablePlugin = (ConfigEditablePlugin) authenticationProvider;
       return editablePlugin.edit(latestConf, commands);
     }
-    throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "This cannot be edited");
+    throw new SolrException(ErrorCode.BAD_REQUEST, "This cannot be edited");
   }
 
   protected AuthenticationProvider getAuthenticationProvider(Map<String, Object> pluginConfig) {
@@ -143,7 +162,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
                   HttpServletRequestWrapper wrapper = new HttpServletRequestWrapper(request) {
                     @Override
                     public Principal getUserPrincipal() {
-                      return new BasicUserPrincipal(username);
+                      return new BasicAuthUserPrincipal(username, pwd);
                     }
                   };
                   numAuthenticated.inc();
@@ -199,6 +218,22 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
   }
 
   @Override
+  protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
+    if (forwardCredentials) {
+      if (httpContext instanceof HttpClientContext) {
+        HttpClientContext httpClientContext = (HttpClientContext) httpContext;
+        if (httpClientContext.getUserToken() instanceof BasicAuthUserPrincipal) {
+          BasicAuthUserPrincipal principal = (BasicAuthUserPrincipal) httpClientContext.getUserToken();
+          String userPassBase64 = Base64.encodeBase64String((principal.getName() + ":" + principal.getPassword()).getBytes(StandardCharsets.UTF_8));
+          httpRequest.setHeader(HttpHeaders.AUTHORIZATION, "Basic " + userPassBase64);
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  @Override
   public ValidatingJsonMap getSpec() {
     return authenticationProvider.getSpec();
   }
@@ -208,7 +243,8 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
 
   public static final String PROPERTY_BLOCK_UNKNOWN = "blockUnknown";
   public static final String PROPERTY_REALM = "realm";
-  private static final Set<String> PROPS = ImmutableSet.of(PROPERTY_BLOCK_UNKNOWN, PROPERTY_REALM);
+  public static final String FORWARD_CREDENTIALS = "forwardCredentials";
+  private static final Set<String> PROPS = ImmutableSet.of(PROPERTY_BLOCK_UNKNOWN, PROPERTY_REALM, FORWARD_CREDENTIALS);
 
   /**
    * Check if the request is an AJAX request, i.e. from the Admin UI or other SPA front 
@@ -218,4 +254,51 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
   private boolean isAjaxRequest(HttpServletRequest request) {
     return "XMLHttpRequest".equalsIgnoreCase(request.getHeader(X_REQUESTED_WITH_HEADER));
   }
+  
+  @Contract(threading = ThreadingBehavior.IMMUTABLE)
+  private class BasicAuthUserPrincipal implements Principal, Serializable {
+    private String username;
+    private final String password;
+
+    public BasicAuthUserPrincipal(String username, String pwd) {
+      this.username = username;
+      this.password = pwd;
+    }
+
+    @Override
+    public String getName() {
+        return this.username;
+    }
+
+    public String getPassword() {
+      return password;
+    }
+
+    @Override
+    public boolean implies(Subject subject) {
+      return false;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      BasicAuthUserPrincipal that = (BasicAuthUserPrincipal) o;
+      return Objects.equals(username, that.username) &&
+          Objects.equals(password, that.password);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(username, password);
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+          .append("username", username)
+          .append("pwd", "*****")
+          .toString();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index fe1ae7b..6bb96d5 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -65,10 +65,6 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   private final HttpHeaderClientInterceptor interceptor = new HttpHeaderClientInterceptor();
   private boolean interceptorRegistered = false;
 
-  public void setInterceptorRegistered(){
-    this.interceptorRegistered = true;
-  }
-
   public boolean isInterceptorRegistered(){
     return interceptorRegistered;
   }
@@ -230,6 +226,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   @Override
   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
     HttpClientUtil.addRequestInterceptor(interceptor);
+    interceptorRegistered = true;
     return builder;
   }
 
@@ -244,8 +241,15 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
 
     @Override
     public void process(HttpRequest httpRequest, HttpContext httpContext) throws HttpException, IOException {
-      if (disabled()) return;
-      setHeader(httpRequest);
+      if (cores.getAuthenticationPlugin() == null) {
+        return;
+      }
+      if (!cores.getAuthenticationPlugin().interceptInternodeRequest(httpRequest, httpContext)) {
+        log.debug("{} secures this internode request", this.getClass().getSimpleName());
+        setHeader(httpRequest);
+      } else {
+        log.debug("{} secures this internode request", cores.getAuthenticationPlugin().getClass().getSimpleName());
+      }
     }
   }
 
@@ -288,14 +292,10 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     return SolrRequestInfo.getRequestInfo();
   }
 
-  boolean disabled() {
-    return cores.getAuthenticationPlugin() == null ||
-        cores.getAuthenticationPlugin() instanceof HttpClientBuilderPlugin;
-  }
-
   @Override
   public void close() throws IOException {
     HttpClientUtil.removeRequestInterceptor(interceptor);
+    interceptorRegistered = false;
   }
 
   public String getPublicKey() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 956af17..2165fe3 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -72,7 +72,6 @@ import org.apache.solr.metrics.AltBufferPoolMetricSet;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.OperatingSystemMetricSet;
 import org.apache.solr.metrics.SolrMetricManager;
-import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
 import org.apache.solr.security.PublicKeyHandler;
@@ -168,7 +167,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
         extraProperties = new Properties();
 
       String solrHome = (String) config.getServletContext().getAttribute(SOLRHOME_ATTRIBUTE);
-      ExecutorUtil.addThreadLocalProvider(SolrRequestInfo.getInheritableThreadLocalProvider());
 
       coresInit = createCoreContainer(solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome),
                                        extraProperties);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
index 9536f9d..39b46d1 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
@@ -32,12 +32,13 @@ import java.util.Set;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
+
 import org.apache.http.HttpResponse;
 import org.apache.http.NoHttpResponseException;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
-import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient; // jdoc
+import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -47,6 +48,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.Diagnostics;
+import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.solr.update.processor.DistributedUpdateProcessor.LeaderRequestReplicationTracker;
 import org.apache.solr.update.processor.DistributedUpdateProcessor.RollupRequestReplicationTracker;
@@ -282,6 +284,11 @@ public class SolrCmdDistributor implements Closeable {
   }
 
   private void submit(final Req req, boolean isCommit) {
+    // Copy user principal from the original request to the new update request, for later authentication interceptor use
+    if (SolrRequestInfo.getRequestInfo() != null) {
+      req.uReq.setUserPrincipal(SolrRequestInfo.getRequestInfo().getReq().getUserPrincipal());
+    }
+
     if (req.synchronous) {
       blockAndDoRetries();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java b/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
index 1a330e3..ad564c2 100644
--- a/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
+++ b/solr/core/src/test/org/apache/solr/core/TestQuerySenderListener.java
@@ -18,8 +18,6 @@ package org.apache.solr.core;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.EventParams;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.request.SolrRequestInfo;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -38,12 +36,8 @@ public class TestQuerySenderListener extends SolrTestCaseJ4 {
     // in the same VM
     preInitMockListenerCount = MockEventListener.getCreateCount();
 
-    if (usually()) {
-      // This is set by the SolrDispatchFilter, used in Http calls but not Embedded
-      ExecutorUtil.addThreadLocalProvider(SolrRequestInfo.getInheritableThreadLocalProvider());
-    }
     initCore("solrconfig-querysender.xml","schema.xml");
-    
+
   }
 
   public void testListenerCreationCounts() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 8c04933..a7c5aa9 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -34,6 +34,7 @@ import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 
+import org.apache.commons.io.IOUtils;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
@@ -44,14 +45,17 @@ import org.apache.http.message.AbstractHttpMessage;
 import org.apache.http.message.BasicHeader;
 import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.RequestWriter.StringPayloadContentWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.SolrCloudAuthTestCase;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
@@ -229,14 +233,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
 
       executeCommand(baseUrl + authzPrefix, cl,"{set-permission : { name : update , role : admin}}", "harry", "HarryIsUberCool");
 
-      SolrInputDocument doc = new SolrInputDocument();
-      doc.setField("id","4");
-      UpdateRequest update = new UpdateRequest();
-      update.setBasicAuthCredentials("harry","HarryIsUberCool");
-      update.add(doc);
-      update.setCommitWithin(100);
-      cluster.getSolrClient().request(update, COLLECTION);
-
+      addDocument("harry","HarryIsUberCool","id", "4");
 
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: true}}", "harry", "HarryIsUberCool");
       verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/blockUnknown", "true", 20, "harry", "HarryIsUberCool");
@@ -276,8 +273,22 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
       cluster.getSolrClient().request(req, COLLECTION);
       
       assertAuthMetricsMinimums(20, 8, 8, 1, 2, 0);
-      assertPkiAuthMetricsMinimums(5, 5, 0, 0, 0, 0);
-
+      assertPkiAuthMetricsMinimums(10, 10, 0, 0, 0, 0);
+
+      addDocument("harry","HarryIsUberCool","id", "5");
+      assertAuthMetricsMinimums(23, 11, 9, 1, 2, 0);
+      assertPkiAuthMetricsMinimums(14, 14, 0, 0, 0, 0);
+
+      // Validate forwardCredentials
+      assertEquals(1, executeQuery(params("q", "id:5"), "harry", "HarryIsUberCool").getResults().getNumFound());
+      assertAuthMetricsMinimums(24, 12, 9, 1, 2, 0);
+      assertPkiAuthMetricsMinimums(18, 18, 0, 0, 0, 0);
+      executeCommand(baseUrl + authcPrefix, cl, "{set-property : { forwardCredentials: true}}", "harry", "HarryIsUberCool");
+      verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/forwardCredentials", "true", 20, "harry", "HarryIsUberCool");
+      assertEquals(1, executeQuery(params("q", "id:5"), "harry", "HarryIsUberCool").getResults().getNumFound());
+      assertAuthMetricsMinimums(31, 19, 9, 1, 2, 0);
+      assertPkiAuthMetricsMinimums(18, 18, 0, 0, 0, 0);
+      
       executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: false}}", "harry", "HarryIsUberCool");
     } finally {
       if (cl != null) {
@@ -293,6 +304,34 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
     assertEquals(num, registry0.getMetrics().entrySet().stream().filter(e -> e.getKey().startsWith("SECURITY")).count());
   }
 
+  private QueryResponse executeQuery(ModifiableSolrParams params, String user, String pass) throws IOException, SolrServerException {
+    SolrRequest req = new QueryRequest(params);
+    req.setBasicAuthCredentials(user, pass);
+    QueryResponse resp = (QueryResponse) req.process(cluster.getSolrClient(), COLLECTION);
+    assertNull(resp.getException());
+    assertEquals(0, resp.getStatus());
+    return resp;
+  }
+
+  private void addDocument(String user, String pass, String... fields) throws IOException, SolrServerException {
+    SolrInputDocument doc = new SolrInputDocument();
+    boolean isKey = true;
+    String key = null;
+    for (String field : fields) {
+      if (isKey) {
+        key = field;
+        isKey = false;
+      } else {
+        doc.setField(key, field);
+      }
+    }
+    UpdateRequest update = new UpdateRequest();
+    update.setBasicAuthCredentials(user, pass);
+    update.add(doc);
+    cluster.getSolrClient().request(update, COLLECTION);
+    update.commit(cluster.getSolrClient(), COLLECTION);
+  }
+
   public static void executeCommand(String url, HttpClient cl, String payload, String user, String pwd)
       throws IOException {
     HttpPost httpPost;
@@ -302,7 +341,9 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
     httpPost.setEntity(new ByteArrayEntity(payload.getBytes(UTF_8)));
     httpPost.addHeader("Content-Type", "application/json; charset=UTF-8");
     r = cl.execute(httpPost);
-    assertEquals(200, r.getStatusLine().getStatusCode());
+    String response = IOUtils.toString(r.getEntity().getContent(), StandardCharsets.UTF_8);
+    assertEquals("Non-200 response code. Response was " + response, 200, r.getStatusLine().getStatusCode());
+    assertFalse("Response contained errors: " + response, response.contains("errorMessages"));
     Utils.consumeFully(r.getEntity());
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
index e6dc790..9cb9bb1 100644
--- a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
@@ -52,11 +52,6 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     }
 
     @Override
-    boolean disabled() {
-      return false;
-    }
-
-    @Override
     SolrRequestInfo getRequestInfo() {
       return solrRequestInfo;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc b/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
index dabd869..89b027f 100644
--- a/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
+++ b/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
@@ -169,12 +169,14 @@ If your plugin of choice is not supported, you will have to interact with Solr s
 
 == Securing Inter-Node Requests
 
-There are a lot of requests that originate from the Solr nodes itself. For example, requests from overseer to nodes, recovery threads, etc. Each Authentication plugin declares whether it is capable of securing inter-node requests or not. If not, Solr will fall back to using a special internode authentication mechanism where each Solr node is a super user and is fully trusted by other Solr nodes, described below.
+There are a lot of requests that originate from the Solr nodes itself. For example, requests from overseer to nodes, recovery threads, etc. We call these 'inter-node' request. Solr has a special built-in `PKIAuthenticationPlugin` (see below) that will always be available to secure inter-node traffic. 
+
+Each Authentication plugin may also decide to secure inter-node requests on its own. They may do this through the so-called `HttpClientBuilder` mechanism, or they may alternatively choose on a per-request basis whether to delegate to PKI or not by overriding a `interceptInternodeRequest()` method from the base class, where any HTTP headers can be set. 
 
 === PKIAuthenticationPlugin
 
-The PKIAuthenticationPlugin is used when there is any request going on between two Solr nodes, and the configured Authentication plugin does not wish to handle inter-node security.
+The `PKIAuthenticationPlugin` provides a built-in authentication mechanism where each Solr node is a super user and is fully trusted by other Solr nodes through the use of Public Key Infrastructure (PKI). Each Authentication plugn may choose to delegate all or some inter-node traffic to the PKI plugin.
 
-For each outgoing request `PKIAuthenticationPlugin` adds a special header `'SolrAuth'` which carries the timestamp and principal encrypted using the private key of that node. The public key is exposed through an API so that any node can read it whenever it needs it. Any node who gets the request with that header, would get the public key from the sender and decrypt the information. If it is able to decrypt the data, the request trusted. It is invalid if the timestamp is more than 5 secs old. This assumes that the clocks of different nodes in the cluster are synchronized.
+For each outgoing request `PKIAuthenticationPlugin` adds a special header `'SolrAuth'` which carries the timestamp and principal encrypted using the private key of that node. The public key is exposed through an API so that any node can read it whenever it needs it. Any node who gets the request with that header, would get the public key from the sender and decrypt the information. If it is able to decrypt the data, the request trusted. It is invalid if the timestamp is more than 5 secs old. This assumes that the clocks of different nodes in the cluster are synchronized. Only traffic from other Solr nodes registered with Zookeeper is trusted.
 
 The timeout is configurable through a system property called `pkiauth.ttl`. For example, if you wish to bump up the time-to-live to 10 seconds (10000 milliseconds), start each node with a property `'-Dpkiauth.ttl=10000'`.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/basic-authentication-plugin.adoc b/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
index 308c5a2..f6c872e 100644
--- a/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
+++ b/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
@@ -37,13 +37,14 @@ An example `security.json` showing both sections is shown below to show how thes
    "blockUnknown": true, <2>
    "class":"solr.BasicAuthPlugin",
    "credentials":{"solr":"IV0EHq1OnNrj6gvRCwvFwTrZ1+z1oBbnQdiVC3otuq0= Ndd7LKvVBAaZIF0QAVi1ekCfAJXr1GGfLtRUXhgrF8c="}, <3>
-   "realm":"My Solr users" <4>
+   "realm":"My Solr users", <4>
+   "forwardCredentials": false <5>
 },
 "authorization":{
    "class":"solr.RuleBasedAuthorizationPlugin",
    "permissions":[{"name":"security-edit",
-      "role":"admin"}], <5>
-   "user-role":{"solr":"admin"} <6>
+      "role":"admin"}], <6>
+   "user-role":{"solr":"admin"} <7>
 }}
 ----
 
@@ -53,8 +54,9 @@ There are several things defined in this file:
 <2> The parameter `"blockUnknown":true` means that unauthenticated requests are not allowed to pass through.
 <3> A user called 'solr', with a password `'SolrRocks'` has been defined.
 <4> We override the `realm` property to display another text on the login prompt.
-<5> The 'admin' role has been defined, and it has permission to edit security settings.
-<6> The 'solr' user has been defined to the 'admin' role.
+<5> The parameter `"forwardCredentials":false` means we let Solr's PKI authenticaion handle distributed request instead of forwarding the Basic Auth header. 
+<6> The 'admin' role has been defined, and it has permission to edit security settings.
+<7> The 'solr' user has been defined to the 'admin' role.
 
 Save your settings to a file called `security.json` locally. If you are using Solr in standalone mode, you should put this file in `$SOLR_HOME`.
 
@@ -143,7 +145,7 @@ curl --user solr:SolrRocks http://localhost:8983/api/cluster/security/authentica
 
 === Set a Property
 
-Set properties for the authentication plugin. The currently supported properties for the Basic Authentication plugin are `blockUnknown` and `realm`.
+Set properties for the authentication plugin. The currently supported properties for the Basic Authentication plugin are `blockUnknown`, `realm` and `forwardCredentials`.
 
 [.dynamic-tabs]
 --

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
index 7dbaab9..04b94f7 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
@@ -18,6 +18,7 @@ package org.apache.solr.client.solrj;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.security.Principal;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -36,6 +37,16 @@ import static java.util.Collections.unmodifiableSet;
  * @since solr 1.3
  */
 public abstract class SolrRequest<T extends SolrResponse> implements Serializable {
+  // This user principal is typically used by Auth plugins during distributed/sharded search
+  private Principal userPrincipal;
+
+  public void setUserPrincipal(Principal userPrincipal) {
+    this.userPrincipal = userPrincipal;
+  }
+
+  public Principal getUserPrincipal() {
+    return userPrincipal;
+  }
 
   public enum METHOD {
     GET,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index ad845e8..8831448 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -24,6 +24,7 @@ import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
 import java.net.SocketTimeoutException;
 import java.nio.charset.StandardCharsets;
+import java.security.Principal;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -252,7 +253,7 @@ public class HttpSolrClient extends SolrClient {
       throws SolrServerException, IOException {
     HttpRequestBase method = createMethod(request, collection);
     setBasicAuthHeader(request, method);
-    return executeMethod(method, processor, isV2ApiRequest(request));
+    return executeMethod(method, request.getUserPrincipal(), processor, isV2ApiRequest(request));
   }
 
   private boolean isV2ApiRequest(final SolrRequest request) {
@@ -296,7 +297,7 @@ public class HttpSolrClient extends SolrClient {
     ExecutorService pool = ExecutorUtil.newMDCAwareFixedThreadPool(1, new SolrjNamedThreadFactory("httpUriRequest"));
     try {
       MDC.put("HttpSolrClient.url", baseUrl);
-      mrr.future = pool.submit(() -> executeMethod(method, processor, isV2ApiRequest(request)));
+      mrr.future = pool.submit(() -> executeMethod(method, request.getUserPrincipal(), processor, isV2ApiRequest(request)));
  
     } finally {
       pool.shutdown();
@@ -517,7 +518,7 @@ public class HttpSolrClient extends SolrClient {
 
   private static final List<String> errPath = Arrays.asList("metadata", "error-class");//Utils.getObjectByPath(err, false,"metadata/error-class")
 
-  protected NamedList<Object> executeMethod(HttpRequestBase method, final ResponseParser processor, final boolean isV2Api) throws SolrServerException {
+  protected NamedList<Object> executeMethod(HttpRequestBase method, Principal userPrincipal, final ResponseParser processor, final boolean isV2Api) throws SolrServerException {
     method.addHeader("User-Agent", AGENT);
  
     org.apache.http.client.config.RequestConfig.Builder requestConfigBuilder = HttpClientUtil.createDefaultRequestConfigBuilder();
@@ -539,6 +540,12 @@ public class HttpSolrClient extends SolrClient {
     try {
       // Execute the method.
       HttpClientContext httpClientRequestContext = HttpClientUtil.createNewHttpClientRequestContext();
+      if (userPrincipal != null) {
+        // Normally the context contains a static userToken to enable reuse resources.
+        // However, if a personal Principal object exists, we use that instead, also as a means
+        // to transfer authentication information to Auth plugins that wish to intercept the request later
+        httpClientRequestContext.setUserToken(userPrincipal);
+      }
       final HttpResponse response = httpClient.execute(method, httpClientRequestContext);
 
       int httpStatus = response.getStatusLine().getStatusCode();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81dbad54/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
index c68526e..4bcf8b9 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
@@ -61,13 +61,13 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
    */
   protected void assertAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
     assertAuthMetricsMinimums(METRICS_PREFIX, requests, authenticated, passThrough, failWrongCredentials, failMissingCredentials, errors);
-  }  
-  
+  }
+
   /**
    * Common test method to be able to check security from any authentication plugin
    * @param prefix the metrics key prefix, currently "SECURITY./authentication." for basic auth and "SECURITY./authentication/pki." for PKI 
    */
-  private void assertAuthMetricsMinimums(String prefix, int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+  Map<String,Long> countAuthMetrics(String prefix) {
     List<Map<String, Metric>> metrics = new ArrayList<>();
     cluster.getJettySolrRunners().forEach(r -> {
       MetricRegistry registry = r.getCoreContainer().getMetricManager().registry("solr.node");
@@ -79,14 +79,33 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
     AUTH_METRICS_KEYS.forEach(k -> {
       counts.put(k, sumCount(prefix, k, metrics));
     });
+    return counts;
+  } 
+  
+  /**
+   * Common test method to be able to check security from any authentication plugin
+   * @param prefix the metrics key prefix, currently "SECURITY./authentication." for basic auth and "SECURITY./authentication/pki." for PKI 
+   */
+  private void assertAuthMetricsMinimums(String prefix, int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+    Map<String, Long> counts = countAuthMetrics(prefix);
     
     // check each counter
-    assertExpectedMetrics(requests, "requests", counts);
-    assertExpectedMetrics(authenticated, "authenticated", counts);
-    assertExpectedMetrics(passThrough, "passThrough", counts);
-    assertExpectedMetrics(failWrongCredentials, "failWrongCredentials", counts);
-    assertExpectedMetrics(failMissingCredentials, "failMissingCredentials", counts);
-    assertExpectedMetrics(errors, "errors", counts);
+    boolean success = isMetricEuqalOrLarger(requests, "requests", counts)
+        & isMetricEuqalOrLarger(authenticated, "authenticated", counts)
+        & isMetricEuqalOrLarger(passThrough, "passThrough", counts)
+        & isMetricEuqalOrLarger(failWrongCredentials, "failWrongCredentials", counts)
+        & isMetricEuqalOrLarger(failMissingCredentials, "failMissingCredentials", counts)
+        & isMetricEuqalOrLarger(errors, "errors", counts);
+    
+    Map<String, Long> expectedCounts = new HashMap<>();
+    expectedCounts.put("requests", (long) requests);
+    expectedCounts.put("authenticated", (long) authenticated);
+    expectedCounts.put("passThrough", (long) passThrough);
+    expectedCounts.put("failWrongCredentials", (long) failWrongCredentials);
+    expectedCounts.put("failMissingCredentials", (long) failMissingCredentials);
+    expectedCounts.put("errors", (long) errors);
+    assertTrue("Expected metric minimums for prefix " + prefix + ": " + expectedCounts + ", but got: " + counts, success);
+    
     if (counts.get("requests") > 0) {
       assertTrue("requestTimes count not > 1", counts.get("requestTimes") > 1);
       assertTrue("totalTime not > 0", counts.get("totalTime") > 0);
@@ -94,11 +113,10 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
   }
 
   // Check that the actual metric is equal to or greater than the expected value, never less
-  private void assertExpectedMetrics(int expected, String key, Map<String, Long> counts) {
+  private boolean isMetricEuqalOrLarger(int expected, String key, Map<String, Long> counts) {
     long cnt = counts.get(key);
     log.debug("Asserting that auth metrics count ({}) > expected ({})", cnt, expected);
-    assertTrue("Expected " + key + " metric count to be " + expected + " or higher, but got " + cnt, 
-        cnt >= expected);
+    return(cnt >= expected);
   }
 
   // Have to sum the metrics from all three shards/nodes


[16/34] lucene-solr:jira/http2: SOLR-13068: Update many cloud/autoscaling tests to use TimeSource.NANO_TIME (instead of CURRENT_TIME) for comparisons

Posted by da...@apache.org.
SOLR-13068: Update many cloud/autoscaling tests to use TimeSource.NANO_TIME (instead of CURRENT_TIME) for comparisons

as much as possible, the tests actual use the time source directly from the cluster, so that even if the cluster is changed to use an alternate time source, the tests will be consistent


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

Branch: refs/heads/jira/http2
Commit: 46516b7077a52b4df9fbf640c17e30d8d8d012fe
Parents: 4272c63
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Dec 13 11:02:24 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Dec 13 11:02:24 2018 -0700

----------------------------------------------------------------------
 .../org/apache/solr/cloud/HttpPartitionTest.java  |  2 +-
 .../apache/solr/cloud/TestCloudConsistency.java   |  4 ++--
 .../cloud/autoscaling/ExecutePlanActionTest.java  |  9 +++++----
 .../autoscaling/MetricTriggerIntegrationTest.java | 10 ++++++++--
 .../NodeAddedTriggerIntegrationTest.java          |  2 +-
 .../cloud/autoscaling/NodeAddedTriggerTest.java   | 18 +++++++++---------
 .../NodeLostTriggerIntegrationTest.java           |  2 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java    | 18 +++++++++---------
 .../autoscaling/RestoreTriggerStateTest.java      |  3 +--
 .../SearchRateTriggerIntegrationTest.java         |  3 ++-
 .../TriggerCooldownIntegrationTest.java           |  9 ++++++---
 .../cloud/autoscaling/TriggerIntegrationTest.java | 10 +++++-----
 .../TriggerSetPropertiesIntegrationTest.java      |  3 +--
 .../autoscaling/sim/TestSimExecutePlanAction.java |  5 +++--
 14 files changed, 54 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index 012bc23..b8a6048 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -319,7 +319,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
   }
 
   protected void waitForState(String collection, String replicaName, Replica.State state, long ms) throws KeeperException, InterruptedException {
-    TimeOut timeOut = new TimeOut(ms, TimeUnit.MILLISECONDS, TimeSource.CURRENT_TIME);
+    TimeOut timeOut = new TimeOut(ms, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     Replica.State replicaState = Replica.State.ACTIVE;
     while (!timeOut.hasTimedOut()) {
       ZkStateReader zkr = cloudClient.getZkStateReader();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
index 9b27522..66ebb2c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
@@ -171,7 +171,7 @@ public class TestCloudConsistency extends SolrCloudTestCase {
     cluster.waitForNode(j1, 30);
     cluster.waitForNode(j2, 30);
     
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.CURRENT_TIME);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica newLeader = getCollectionState(collection).getSlice("shard1").getLeader();
       if (newLeader != null && !newLeader.getName().equals(leader.getName()) && newLeader.getState() == Replica.State.ACTIVE) {
@@ -213,7 +213,7 @@ public class TestCloudConsistency extends SolrCloudTestCase {
     waitForState("Timeout waiting for leader goes DOWN", collection, (liveNodes, collectionState)
         -> collectionState.getReplica(leader.getName()).getState() == Replica.State.DOWN);
 
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.CURRENT_TIME);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica newLeader = getCollectionState(collection).getLeader("shard1");
       if (newLeader != null && !newLeader.getName().equals(leader.getName()) && newLeader.getState() == Replica.State.ACTIVE) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
index cbd0bac..9ca38fd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
@@ -43,7 +43,6 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
-import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.Before;
@@ -151,9 +150,11 @@ public class ExecutePlanActionTest extends SolrCloudTestCase {
         }
       };
       List<CollectionAdminRequest.AsyncCollectionAdminRequest> operations = Lists.asList(moveReplica, new CollectionAdminRequest.AsyncCollectionAdminRequest[]{mockRequest});
-      NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST,
-          "mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTimeNs()),
-          Collections.singletonList(sourceNodeName), CollectionParams.CollectionAction.MOVEREPLICA.toLower());
+      NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent
+        (TriggerEventType.NODELOST, "mock_trigger_name",
+         Collections.singletonList(cloudManager.getTimeSource().getTimeNs()),
+         Collections.singletonList(sourceNodeName),
+         CollectionParams.CollectionAction.MOVEREPLICA.toLower());
       ActionContext actionContext = new ActionContext(survivor.getCoreContainer().getZkController().getSolrCloudManager(), null,
           new HashMap<>(Collections.singletonMap("operations", operations)));
       action.process(nodeLostEvent, actionContext);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
index a562633..418a626 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.metrics.SolrCoreMetricManager;
@@ -50,7 +51,6 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.WAIT_FOR_DELTA_NANOS;
-import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.timeSource;
 
 /**
  * Integration test for {@link MetricTrigger}
@@ -58,6 +58,9 @@ import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.timeSourc
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG")
 public class MetricTriggerIntegrationTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final TimeSource timeSource = TimeSource.NANO_TIME;
+  
   static Map<String, List<CapturedEvent>> listenerEvents = new HashMap<>();
   static CountDownLatch listenerCreated = new CountDownLatch(1);
   private static CountDownLatch triggerFiredLatch;
@@ -212,7 +215,7 @@ public class MetricTriggerIntegrationTest extends SolrCloudTestCase {
     public void process(TriggerEvent event, ActionContext context) throws Exception {
       try {
         events.add(event);
-        long currentTimeNanos = timeSource.getTimeNs();
+        long currentTimeNanos = context.getCloudManager().getTimeSource().getTimeNs();
         long eventTimeNanos = event.getEventTime();
         long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
         if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -227,10 +230,12 @@ public class MetricTriggerIntegrationTest extends SolrCloudTestCase {
   }
 
   public static class TestTriggerListener extends TriggerListenerBase {
+    private TimeSource timeSource;
     @Override
     public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
       super.configure(loader, cloudManager, config);
       listenerCreated.countDown();
+      timeSource = cloudManager.getTimeSource();
     }
 
     @Override
@@ -238,6 +243,7 @@ public class MetricTriggerIntegrationTest extends SolrCloudTestCase {
                                      ActionContext context, Throwable error, String message) {
       List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
       lst.add(new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message));
+                                
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
index 795c530..bce3ea1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
@@ -258,7 +258,7 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
       try {
         if (triggerFired.compareAndSet(false, true)) {
           events.add(event);
-          long currentTimeNanos = TriggerIntegrationTest.timeSource.getTimeNs();
+          long currentTimeNanos = actionContext.getCloudManager().getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index 8a78520..de4af4b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@ -31,7 +31,6 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrResourceLoader;
 import org.junit.After;
 import org.junit.Before;
@@ -51,8 +50,6 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     return true;
   };
 
-  private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
-  // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
   private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(2);
 
   @BeforeClass
@@ -82,7 +79,8 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     Map<String, Object> props = createTriggerProps(waitForSeconds);
 
     try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger")) {
-      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      final SolrCloudManager cloudManager = container.getZkController().getSolrCloudManager();
+      trigger.configure(container.getResourceLoader(), cloudManager, props);
       trigger.init();
       trigger.setProcessor(noFirstRunProcessor);
       trigger.run();
@@ -97,7 +95,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
       trigger.setProcessor(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = cloudManager.getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -127,7 +125,8 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     // add a new node but remove it before the waitFor period expires
     // and assert that the trigger doesn't fire at all
     try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger")) {
-      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      final SolrCloudManager cloudManager = container.getZkController().getSolrCloudManager();
+      trigger.configure(container.getResourceLoader(), cloudManager, props);
       trigger.init();
       final long waitTime = 2;
       props.put("waitFor", waitTime);
@@ -138,7 +137,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
       AtomicBoolean fired = new AtomicBoolean(false);
       trigger.setProcessor(event -> {
         if (fired.compareAndSet(false, true)) {
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = cloudManager.getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -281,13 +280,14 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     }
 
     try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger"))  {
-      newTrigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      final SolrCloudManager cloudManager = container.getZkController().getSolrCloudManager();
+      newTrigger.configure(container.getResourceLoader(), cloudManager, props);
       newTrigger.init();
       AtomicBoolean stop = new AtomicBoolean(false);
       AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       newTrigger.setProcessor(event -> {
         //the processor may get called 2 times, for newly added node and initial nodes
-        long currentTimeNanos = timeSource.getTimeNs();
+        long currentTimeNanos = cloudManager.getTimeSource().getTimeNs();
         long eventTimeNanos = event.getEventTime();
         long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
         if (currentTimeNanos - eventTimeNanos <= waitForNanos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
index 744f1da..1e75b89 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
@@ -283,7 +283,7 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
       try {
         if (triggerFired.compareAndSet(false, true)) {
           events.add(event);
-          long currentTimeNanos = TriggerIntegrationTest.timeSource.getTimeNs();
+          long currentTimeNanos = actionContext.getCloudManager().getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index ebe5081..c974de0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -51,9 +51,6 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
     return true;
   };
 
-  // use the same time source as the trigger
-  private final TimeSource timeSource = TimeSource.CURRENT_TIME;
-  // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
   private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
 
   @After
@@ -82,7 +79,8 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
     Map<String, Object> props = createTriggerProps(waitForSeconds);
 
     try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger")) {
-      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      final SolrCloudManager cloudManager = container.getZkController().getSolrCloudManager();
+      trigger.configure(container.getResourceLoader(), cloudManager, props);
       trigger.init();
       trigger.setProcessor(noFirstRunProcessor);
       trigger.run();
@@ -99,7 +97,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
       trigger.setProcessor(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = cloudManager.getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -130,7 +128,8 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
     // remove a node but add it back before the waitFor period expires
     // and assert that the trigger doesn't fire at all
     try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger")) {
-      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      final SolrCloudManager cloudManager = container.getZkController().getSolrCloudManager();
+      trigger.configure(container.getResourceLoader(), cloudManager, props);
       final long waitTime = 2;
       props.put("waitFor", waitTime);
       trigger.init();
@@ -143,7 +142,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
       AtomicBoolean fired = new AtomicBoolean(false);
       trigger.setProcessor(event -> {
         if (fired.compareAndSet(false, true)) {
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = cloudManager.getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitTime, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -329,14 +328,15 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
     }
 
     try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger")) {
-      newTrigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      final SolrCloudManager cloudManager = container.getZkController().getSolrCloudManager();
+      newTrigger.configure(container.getResourceLoader(), cloudManager, props);
       newTrigger.init();
       AtomicBoolean fired = new AtomicBoolean(false);
       AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       newTrigger.setProcessor(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = cloudManager.getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/RestoreTriggerStateTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/RestoreTriggerStateTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/RestoreTriggerStateTest.java
index 3088424..602b08d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/RestoreTriggerStateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/RestoreTriggerStateTest.java
@@ -40,7 +40,6 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.WAIT_FOR_DELTA_NANOS;
-import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.timeSource;
 
 /**
  * Integration test to ensure that triggers can restore state from ZooKeeper after overseer restart
@@ -144,7 +143,7 @@ public class RestoreTriggerStateTest extends SolrCloudTestCase {
       try {
         if (triggerFired.compareAndSet(false, true))  {
           events.add(event);
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = actionContext.getCloudManager().getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
index 76e4b83..ea8e915 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
@@ -47,6 +47,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
@@ -59,7 +60,6 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.WAIT_FOR_DELTA_NANOS;
-import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.timeSource;
 import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
 
 /**
@@ -71,6 +71,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_P
 public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private static final TimeSource timeSource = TimeSource.NANO_TIME;
   private static CountDownLatch listenerCreated = new CountDownLatch(1);
   private static Map<String, List<CapturedEvent>> listenerEvents = new HashMap<>();
   private static CountDownLatch finished = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerCooldownIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerCooldownIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerCooldownIntegrationTest.java
index 5c9ae90..5974f77 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerCooldownIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerCooldownIntegrationTest.java
@@ -38,6 +38,7 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.junit.BeforeClass;
@@ -47,7 +48,6 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.WAIT_FOR_DELTA_NANOS;
-import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.timeSource;
 
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG")
 public class TriggerCooldownIntegrationTest extends SolrCloudTestCase {
@@ -199,7 +199,7 @@ public class TriggerCooldownIntegrationTest extends SolrCloudTestCase {
       try {
         if (triggerFired.compareAndSet(false, true)) {
           events.add(event);
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = actionContext.getCloudManager().getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -224,17 +224,20 @@ public class TriggerCooldownIntegrationTest extends SolrCloudTestCase {
   }
 
   public static class TestTriggerListener extends TriggerListenerBase {
+    private TimeSource timeSource;
     @Override
     public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
       super.configure(loader, cloudManager, config);
       listenerCreated.countDown();
+      timeSource = cloudManager.getTimeSource();
     }
 
     @Override
     public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
                                      ActionContext context, Throwable error, String message) {
       List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
-      lst.add(new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message));
+      lst.add(new CapturedEvent(timeSource.getTimeNs(),
+                                context, config, stage, actionName, event, message));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index 2fe3b95..c0b03ee 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -78,9 +78,6 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   public static volatile long eventQueueActionWait = 5000;
   private static SolrCloudManager cloudManager;
 
-  // use the same time source as triggers use
-  static final TimeSource timeSource = TimeSource.CURRENT_TIME;
-
   static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
 
   @BeforeClass
@@ -292,7 +289,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         return;
       }
       try {
-        long currentTime = timeSource.getTimeNs();
+        long currentTime = actionContext.getCloudManager().getTimeSource().getTimeNs();
         if (lastActionExecutedAt.get() != 0)  {
           long minDiff = TimeUnit.MILLISECONDS.toNanos(throttlingDelayMs.get() - DELTA_MS);
           log.info("last action at " + lastActionExecutedAt.get() + " current time = " + currentTime +
@@ -378,7 +375,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       try {
         if (triggerFired.compareAndSet(false, true))  {
           events.add(event);
-          long currentTimeNanos = timeSource.getTimeNs();
+          long currentTimeNanos = actionContext.getCloudManager().getTimeSource().getTimeNs();
           long eventTimeNanos = event.getEventTime();
           long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
           if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
@@ -501,10 +498,12 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   static boolean failDummyAction = false;
 
   public static class TestTriggerListener extends TriggerListenerBase {
+    private TimeSource timeSource;
     @Override
     public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
       super.configure(loader, cloudManager, config);
       listenerCreated.countDown();
+      timeSource = cloudManager.getTimeSource();
     }
 
     @Override
@@ -512,6 +511,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
                                      ActionContext context, Throwable error, String message) {
       List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
       CapturedEvent ev = new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message);
+                                           
       lst.add(ev);
       allListenerEvents.add(ev);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerSetPropertiesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerSetPropertiesIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerSetPropertiesIntegrationTest.java
index c59e60b..a86bb39 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerSetPropertiesIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerSetPropertiesIntegrationTest.java
@@ -48,7 +48,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
-import static org.apache.solr.cloud.autoscaling.TriggerIntegrationTest.timeSource;
 
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG")
 public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
@@ -89,7 +88,7 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
         @Override
         public void run() {
           log.info("Running {} in {}", this.getName(), Thread.currentThread().getName());
-          timestamps.offer(timeSource.getTimeNs());
+          timestamps.offer(solrCloudManager.getTimeSource().getTimeNs());
         }
       };
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/46516b70/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
index 07b02c9..1d36f14 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
@@ -61,11 +61,12 @@ import com.google.common.collect.Lists;
 public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private static final TimeSource SIM_TIME_SOURCE = TimeSource.get("simTime:50");
   private static final int NODE_COUNT = 2;
 
   @Before
   public void setupCluster() throws Exception {
-    configureCluster(NODE_COUNT, TimeSource.get("simTime:50"));
+    configureCluster(NODE_COUNT, SIM_TIME_SOURCE);
   }
 
   @After
@@ -138,7 +139,7 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
       };
       List<CollectionAdminRequest.AsyncCollectionAdminRequest> operations = Lists.asList(moveReplica, new CollectionAdminRequest.AsyncCollectionAdminRequest[]{mockRequest});
       NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST,
-          "mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTimeNs()),
+          "mock_trigger_name", Collections.singletonList(SIM_TIME_SOURCE.getTimeNs()),
           Collections.singletonList(sourceNodeName), CollectionParams.CollectionAction.MOVEREPLICA.toLower());
       ActionContext actionContext = new ActionContext(cluster, null,
           new HashMap<>(Collections.singletonMap("operations", operations)));


[32/34] lucene-solr:jira/http2: LUCENE-8609: Allow getting consistent docstats from IndexWriter

Posted by da...@apache.org.
LUCENE-8609: Allow getting consistent docstats from IndexWriter

Today we have #numDocs() and #maxDoc() on IndexWriter. This is enough
to get all stats for the current index but it's subject to concurrency
and might return numbers that are not consistent ie. some cases can
return maxDoc < numDocs which is undesirable. This change adds a getDocStats()
method to index writer to allow fetching consistent numbers for these stats.

This change also deprecates IndexWriter#numDocs() and IndexWriter#maxDoc()
and replaces all their usages wiht IndexWriter#getDocStats()


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

Branch: refs/heads/jira/http2
Commit: e974311d91fd3f2f3aa015c9e56cf2d689290f41
Parents: eb4fda6
Author: Simon Willnauer <si...@apache.org>
Authored: Thu Dec 13 16:05:47 2018 +0100
Committer: Simon Willnauer <si...@apache.org>
Committed: Fri Dec 14 19:36:25 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 .../miscellaneous/TestEmptyTokenStream.java     |  2 +-
 .../index/TestBackwardsCompatibility.java       |  8 +--
 .../org/apache/lucene/index/IndexWriter.java    | 52 +++++++++++++++-
 .../perfield/TestPerFieldPostingsFormat2.java   | 12 ++--
 .../org/apache/lucene/index/TestAddIndexes.java | 62 ++++++++++----------
 .../index/TestConcurrentMergeScheduler.java     |  2 +-
 .../test/org/apache/lucene/index/TestCrash.java |  2 +-
 .../apache/lucene/index/TestDeletionPolicy.java |  8 +--
 .../index/TestFlushByRamOrCountsPolicy.java     | 16 ++---
 .../lucene/index/TestIndexManyDocuments.java    |  2 +-
 .../apache/lucene/index/TestIndexWriter.java    | 45 ++++++++++----
 .../lucene/index/TestIndexWriterCommit.java     |  2 +-
 .../lucene/index/TestIndexWriterDelete.java     |  2 +-
 .../lucene/index/TestIndexWriterFromReader.java | 20 +++----
 .../lucene/index/TestIndexWriterMaxDocs.java    |  6 +-
 .../index/TestIndexWriterMergePolicy.java       |  2 +-
 .../lucene/index/TestIndexWriterMerging.java    |  8 +--
 .../lucene/index/TestIndexWriterReader.java     |  4 +-
 .../org/apache/lucene/index/TestIsCurrent.java  |  8 +--
 .../apache/lucene/index/TestRollingUpdates.java |  2 +-
 .../TestSoftDeletesRetentionMergePolicy.java    | 20 +++----
 .../lucene/index/TestThreadedForceMerge.java    |  4 +-
 .../lucene/index/TestTieredMergePolicy.java     | 28 ++++-----
 .../org/apache/lucene/search/TestBoolean2.java  |  2 +-
 .../lucene/search/TestSearcherManager.java      |  2 +-
 .../lucene/store/TestByteBuffersDirectory.java  |  2 +-
 .../apache/lucene/store/TestRAMDirectory.java   |  2 +-
 .../directory/DirectoryTaxonomyWriter.java      |  4 +-
 .../lucene/queries/mlt/TestMoreLikeThis.java    |  2 +-
 .../apache/lucene/index/RandomIndexWriter.java  |  8 +--
 .../ThreadedIndexingAndSearchingTestCase.java   |  2 +-
 .../ClassificationUpdateProcessorTest.java      |  2 +-
 33 files changed, 209 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d6276aa..b4dd696 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -282,6 +282,10 @@ Other
 * LUCENE-8605: Separate bounding box spatial logic from query logic on LatLonShapeBoundingBoxQuery.
   (Ignacio Vera)
 
+* LUCENE-8609: Deprecated IndexWriter#numDocs() and IndexWriter#maxDoc() in favor of IndexWriter#getDocStats()
+  that allows to get consistent numDocs and maxDoc stats that are not subject to concurrent changes.
+  (Simon Willnauer, Nhat Nguyen)
+
 ======================= Lucene 7.6.0 =======================
 
 Build

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestEmptyTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestEmptyTokenStream.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestEmptyTokenStream.java
index 556aff0..92d4049 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestEmptyTokenStream.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestEmptyTokenStream.java
@@ -62,7 +62,7 @@ public class TestEmptyTokenStream extends BaseTokenStreamTestCase {
     // this should not fail because we have no TermToBytesRefAttribute
     writer.addDocument(doc);
     
-    assertEquals(1, writer.numDocs());
+    assertEquals(1, writer.getDocStats().numDocs);
 
     writer.close();
     directory.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 23fac47..b2ffc1f 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -1071,7 +1071,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
 
     // make sure writer sees right total -- writer seems not to know about deletes in .del?
     final int expected = 45;
-    assertEquals("wrong doc count", expected, writer.numDocs());
+    assertEquals("wrong doc count", expected, writer.getDocStats().numDocs);
     writer.close();
 
     // make sure searching sees right # hits
@@ -1139,7 +1139,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     for(int i=0;i<35;i++) {
       addDoc(writer, i);
     }
-    assertEquals("wrong doc count", 35, writer.maxDoc());
+    assertEquals("wrong doc count", 35, writer.getDocStats().maxDoc);
     if (fullyMerged) {
       writer.forceMerge(1);
     }
@@ -1599,7 +1599,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     verifyUsesDefaultCodec(dir, dvUpdatesIndex);
     IndexWriterConfig conf = new IndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("__soft_delete");
     IndexWriter writer = new IndexWriter(dir, conf);
-    int maxDoc = writer.maxDoc();
+    int maxDoc = writer.getDocStats().maxDoc;
     writer.updateDocValues(new Term("id", "1"),new NumericDocValuesField("__soft_delete", 1));
 
     if (random().nextBoolean()) {
@@ -1607,7 +1607,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     }
     writer.forceMerge(1);
     writer.commit();
-    assertEquals(maxDoc-1, writer.maxDoc());
+    assertEquals(maxDoc-1, writer.getDocStats().maxDoc);
     writer.close();
     dir.close();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index f841582..4771f3d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1134,7 +1134,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
   /** Returns total number of docs in this index, including
    *  docs not yet flushed (still in the RAM buffer),
    *  not counting deletions.
-   *  @see #numDocs */
+   *  @see #numDocs
+   *  @deprecated use {@link #getDocStats()} instead
+   *  */
+  @Deprecated
   public synchronized int maxDoc() {
     ensureOpen();
     return docWriter.getNumDocs() + segmentInfos.totalMaxDoc();
@@ -1156,7 +1159,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
    *  including deletions.  <b>NOTE:</b> buffered deletions
    *  are not counted.  If you really need these to be
    *  counted you should call {@link #commit()} first.
-   *  @see #numDocs */
+   *  @see #maxDoc
+   *  @deprecated use {@link #getDocStats()} instead
+   *  */
+  @Deprecated
   public synchronized int numDocs() {
     ensureOpen();
     int count = docWriter.getNumDocs();
@@ -5289,4 +5295,46 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
   final synchronized SegmentInfos cloneSegmentInfos() {
     return segmentInfos.clone();
   }
+
+  /**
+   * Returns accurate {@link DocStats} form this writer. This is equivalent to calling {@link #numDocs()} and {@link #maxDoc()}
+   * but is not subject to race-conditions. The numDoc for instance can change after maxDoc is fetched that causes numDocs to be
+   * greater than maxDoc which makes it hard to get accurate document stats from IndexWriter.
+   */
+  public synchronized DocStats getDocStats() {
+    ensureOpen();
+    int numDocs = docWriter.getNumDocs();
+    int maxDoc = numDocs;
+    for (final SegmentCommitInfo info : segmentInfos) {
+      maxDoc += info.info.maxDoc();
+      numDocs += info.info.maxDoc() - numDeletedDocs(info);
+    }
+    assert maxDoc >= numDocs : "maxDoc is less than numDocs: " + maxDoc + " < " + numDocs;
+    return new DocStats(maxDoc, numDocs);
+  }
+
+  /**
+   * DocStats for this index
+   */
+  public static final class DocStats {
+    /**
+     * The total number of docs in this index, including
+     * docs not yet flushed (still in the RAM buffer),
+     * not counting deletions.
+     */
+    public final int maxDoc;
+    /**
+     * The total number of docs in this index, including
+     * docs not yet flushed (still in the RAM buffer), and
+     * including deletions.  <b>NOTE:</b> buffered deletions
+     * are not counted.  If you really need these to be
+     * counted you should call {@link IndexWriter#commit()} first.
+     */
+    public final int numDocs;
+
+    private DocStats(int maxDoc, int numDocs) {
+      this.maxDoc = maxDoc;
+      this.numDocs = numDocs;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
index b7cc2bf..df18a74 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
@@ -120,10 +120,10 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
     writer.commit();
     addDocs2(writer, 10);
     writer.commit();
-    assertEquals(30, writer.maxDoc());
+    assertEquals(30, writer.getDocStats().maxDoc);
     TestUtil.checkIndex(dir);
     writer.forceMerge(1);
-    assertEquals(30, writer.maxDoc());
+    assertEquals(30, writer.getDocStats().maxDoc);
     writer.close();
     dir.close();
   }
@@ -173,7 +173,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
     addDocs2(writer, 10);
     writer.commit();
     codec = iwconf.getCodec();
-    assertEquals(30, writer.maxDoc());
+    assertEquals(30, writer.getDocStats().maxDoc);
     assertQuery(new Term("content", "bbb"), dir, 10);
     assertQuery(new Term("content", "ccc"), dir, 10);   ////
     assertQuery(new Term("content", "aaa"), dir, 10);
@@ -186,13 +186,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
     assertQuery(new Term("content", "ccc"), dir, 10);
     assertQuery(new Term("content", "bbb"), dir, 20);
     assertQuery(new Term("content", "aaa"), dir, 10);
-    assertEquals(40, writer.maxDoc());
+    assertEquals(40, writer.getDocStats().maxDoc);
 
     if (VERBOSE) {
       System.out.println("TEST: now optimize");
     }
     writer.forceMerge(1);
-    assertEquals(40, writer.maxDoc());
+    assertEquals(40, writer.getDocStats().maxDoc);
     writer.close();
     assertQuery(new Term("content", "ccc"), dir, 10);
     assertQuery(new Term("content", "bbb"), dir, 20);
@@ -258,7 +258,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
         writer.forceMerge(1);
       }
       writer.commit();
-      assertEquals((i + 1) * docsPerRound, writer.maxDoc());
+      assertEquals((i + 1) * docsPerRound, writer.getDocStats().maxDoc);
       writer.close();
     }
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
index 1d0a319..7e5f008 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
@@ -66,7 +66,7 @@ public class TestAddIndexes extends LuceneTestCase {
         .setOpenMode(OpenMode.CREATE));
     // add 100 documents
     addDocs(writer, 100);
-    assertEquals(100, writer.maxDoc());
+    assertEquals(100, writer.getDocStats().maxDoc);
     writer.close();
     TestUtil.checkIndex(dir);
 
@@ -78,20 +78,20 @@ public class TestAddIndexes extends LuceneTestCase {
     );
     // add 40 documents in separate files
     addDocs(writer, 40);
-    assertEquals(40, writer.maxDoc());
+    assertEquals(40, writer.getDocStats().maxDoc);
     writer.close();
 
     writer = newWriter(aux2, newIndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE));
     // add 50 documents in compound files
     addDocs2(writer, 50);
-    assertEquals(50, writer.maxDoc());
+    assertEquals(50, writer.getDocStats().maxDoc);
     writer.close();
 
     // test doc count before segments are merged
     writer = newWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
-    assertEquals(100, writer.maxDoc());
+    assertEquals(100, writer.getDocStats().maxDoc);
     writer.addIndexes(aux, aux2);
-    assertEquals(190, writer.maxDoc());
+    assertEquals(190, writer.getDocStats().maxDoc);
     writer.close();
     TestUtil.checkIndex(dir);
 
@@ -106,14 +106,14 @@ public class TestAddIndexes extends LuceneTestCase {
     writer = newWriter(aux3, newIndexWriterConfig(new MockAnalyzer(random())));
     // add 40 documents
     addDocs(writer, 40);
-    assertEquals(40, writer.maxDoc());
+    assertEquals(40, writer.getDocStats().maxDoc);
     writer.close();
 
     // test doc count before segments are merged
     writer = newWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
-    assertEquals(190, writer.maxDoc());
+    assertEquals(190, writer.getDocStats().maxDoc);
     writer.addIndexes(aux3);
-    assertEquals(230, writer.maxDoc());
+    assertEquals(230, writer.getDocStats().maxDoc);
     writer.close();
 
     // make sure the new index is correct
@@ -142,9 +142,9 @@ public class TestAddIndexes extends LuceneTestCase {
     writer.close();
 
     writer = newWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
-    assertEquals(230, writer.maxDoc());
+    assertEquals(230, writer.getDocStats().maxDoc);
     writer.addIndexes(aux4);
-    assertEquals(231, writer.maxDoc());
+    assertEquals(231, writer.getDocStats().maxDoc);
     writer.close();
 
     verifyNumDocs(dir, 231);
@@ -284,7 +284,7 @@ public class TestAddIndexes extends LuceneTestCase {
     writer = newWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
     // add 100 documents
     addDocs(writer, 100);
-    assertEquals(100, writer.maxDoc());
+    assertEquals(100, writer.getDocStats().maxDoc);
     writer.close();
 
     writer = newWriter(
@@ -312,7 +312,7 @@ public class TestAddIndexes extends LuceneTestCase {
     expectThrows(IllegalArgumentException.class, () -> {
       writer2.addIndexes(aux, dir);
     });
-    assertEquals(100, writer2.maxDoc());
+    assertEquals(100, writer2.getDocStats().maxDoc);
     writer2.close();
 
     // make sure the index is correct
@@ -342,7 +342,7 @@ public class TestAddIndexes extends LuceneTestCase {
     addDocs(writer, 10);
 
     writer.addIndexes(aux);
-    assertEquals(1040, writer.maxDoc());
+    assertEquals(1040, writer.getDocStats().maxDoc);
     assertEquals(1000, writer.maxDoc(0));
     writer.close();
 
@@ -371,7 +371,7 @@ public class TestAddIndexes extends LuceneTestCase {
     addDocs(writer, 2);
 
     writer.addIndexes(aux);
-    assertEquals(1032, writer.maxDoc());
+    assertEquals(1032, writer.getDocStats().maxDoc);
     assertEquals(1000, writer.maxDoc(0));
     writer.close();
 
@@ -399,7 +399,7 @@ public class TestAddIndexes extends LuceneTestCase {
     );
 
     writer.addIndexes(aux, new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(aux)));
-    assertEquals(1060, writer.maxDoc());
+    assertEquals(1060, writer.getDocStats().maxDoc);
     assertEquals(1000, writer.maxDoc(0));
     writer.close();
 
@@ -441,7 +441,7 @@ public class TestAddIndexes extends LuceneTestCase {
       System.out.println("\nTEST: now addIndexes");
     }
     writer.addIndexes(aux, new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(aux)));
-    assertEquals(1020, writer.maxDoc());
+    assertEquals(1020, writer.getDocStats().maxDoc);
     assertEquals(1000, writer.maxDoc(0));
     writer.close();
     dir.close();
@@ -466,7 +466,7 @@ public class TestAddIndexes extends LuceneTestCase {
             setMergePolicy(newLogMergePolicy(10))
     );
     writer.addIndexes(aux);
-    assertEquals(30, writer.maxDoc());
+    assertEquals(30, writer.getDocStats().maxDoc);
     assertEquals(3, writer.getSegmentCount());
     writer.close();
 
@@ -501,7 +501,7 @@ public class TestAddIndexes extends LuceneTestCase {
     );
 
     writer.addIndexes(aux, aux2);
-    assertEquals(1040, writer.maxDoc());
+    assertEquals(1040, writer.getDocStats().maxDoc);
     assertEquals(1000, writer.maxDoc(0));
     writer.close();
     dir.close();
@@ -570,7 +570,7 @@ public class TestAddIndexes extends LuceneTestCase {
     } else {
       addDocs(writer, 1000);
     }
-    assertEquals(1000, writer.maxDoc());
+    assertEquals(1000, writer.getDocStats().maxDoc);
     assertEquals(1, writer.getSegmentCount());
     writer.close();
 
@@ -597,7 +597,7 @@ public class TestAddIndexes extends LuceneTestCase {
               setMergePolicy(newLogMergePolicy(false, 10))
       );
     }
-    assertEquals(30, writer.maxDoc());
+    assertEquals(30, writer.getDocStats().maxDoc);
     assertEquals(3, writer.getSegmentCount());
     writer.close();
   }
@@ -815,7 +815,7 @@ public class TestAddIndexes extends LuceneTestCase {
     c.joinThreads();
 
     int expectedNumDocs = 100+NUM_COPY*(4*NUM_ITER/5)*RunAddIndexesThreads.NUM_THREADS*RunAddIndexesThreads.NUM_INIT_DOCS;
-    assertEquals("expected num docs don't match - failures: " + c.failures, expectedNumDocs, c.writer2.numDocs());
+    assertEquals("expected num docs don't match - failures: " + c.failures, expectedNumDocs, c.writer2.getDocStats().numDocs);
 
     c.close(true);
 
@@ -1002,7 +1002,7 @@ public class TestAddIndexes extends LuceneTestCase {
       TestUtil.addIndexesSlowly(writer, r);
     }
     writer.commit();
-    assertEquals("Documents from the incoming index should not have been deleted", 1, writer.numDocs());
+    assertEquals("Documents from the incoming index should not have been deleted", 1, writer.getDocStats().numDocs);
     writer.close();
 
     for (Directory dir : dirs) {
@@ -1037,7 +1037,7 @@ public class TestAddIndexes extends LuceneTestCase {
                               .setOpenMode(OpenMode.CREATE).setCodec(codec));
     // add 100 documents
     addDocsWithID(writer, 100, 0);
-    assertEquals(100, writer.maxDoc());
+    assertEquals(100, writer.getDocStats().maxDoc);
     writer.commit();
     writer.close();
     TestUtil.checkIndex(dir);
@@ -1052,7 +1052,7 @@ public class TestAddIndexes extends LuceneTestCase {
     );
     // add 40 documents in separate files
     addDocs(writer, 40);
-    assertEquals(40, writer.maxDoc());
+    assertEquals(40, writer.getDocStats().maxDoc);
     writer.commit();
     writer.close();
 
@@ -1064,7 +1064,7 @@ public class TestAddIndexes extends LuceneTestCase {
     );
     // add 40 documents in compound files
     addDocs2(writer, 50);
-    assertEquals(50, writer.maxDoc());
+    assertEquals(50, writer.getDocStats().maxDoc);
     writer.commit();
     writer.close();
 
@@ -1075,9 +1075,9 @@ public class TestAddIndexes extends LuceneTestCase {
             setOpenMode(OpenMode.APPEND).
             setCodec(codec)
     );
-    assertEquals(100, writer.maxDoc());
+    assertEquals(100, writer.getDocStats().maxDoc);
     writer.addIndexes(aux, aux2);
-    assertEquals(190, writer.maxDoc());
+    assertEquals(190, writer.getDocStats().maxDoc);
     writer.close();
 
     dir.close();
@@ -1439,8 +1439,8 @@ public class TestAddIndexes extends LuceneTestCase {
       readers[i] = (CodecReader)reader.leaves().get(i).reader();
     }
     writer.addIndexes(readers);
-    assertEquals(wrappedReader.numDocs(), writer.numDocs());
-    assertEquals(maxDoc, writer.maxDoc());
+    assertEquals(wrappedReader.numDocs(), writer.getDocStats().numDocs);
+    assertEquals(maxDoc, writer.getDocStats().maxDoc);
     writer.commit();
     SegmentCommitInfo commitInfo = writer.listOfSegmentCommitInfos().get(0);
     assertEquals(maxDoc-wrappedReader.numDocs(), commitInfo.getSoftDelCount());
@@ -1454,8 +1454,8 @@ public class TestAddIndexes extends LuceneTestCase {
       readers[i] = (CodecReader)wrappedReader.leaves().get(i).reader();
     }
     writer.addIndexes(readers);
-    assertEquals(wrappedReader.numDocs(), writer.numDocs());
-    assertEquals(wrappedReader.numDocs(), writer.maxDoc());
+    assertEquals(wrappedReader.numDocs(), writer.getDocStats().numDocs);
+    assertEquals(wrappedReader.numDocs(), writer.getDocStats().maxDoc);
     IOUtils.close(reader, writer, dir3, dir2, dir1);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java b/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
index 9d389e2..fad831f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
@@ -552,7 +552,7 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
       }
     }.start();
 
-    while (w.numDocs() != 8) {
+    while (w.getDocStats().numDocs != 8) {
       Thread.sleep(10);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestCrash.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCrash.java b/lucene/core/src/test/org/apache/lucene/index/TestCrash.java
index 227b808..f6d15c9 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCrash.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCrash.java
@@ -135,7 +135,7 @@ public class TestCrash extends LuceneTestCase {
 
     writer.close();
     writer = initIndex(random(), dir, false, true);
-    assertEquals(314, writer.maxDoc());
+    assertEquals(314, writer.getDocStats().maxDoc);
     crash(writer);
 
     /*

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
index 869ef8c..5999624 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
@@ -427,7 +427,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                     .setIndexDeletionPolicy(policy));
     addDoc(writer);
-    assertEquals(11, writer.numDocs());
+    assertEquals(11, writer.getDocStats().numDocs);
     writer.forceMerge(1);
     writer.close();
 
@@ -437,7 +437,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                     .setIndexDeletionPolicy(policy)
                                     .setIndexCommit(lastCommit));
-    assertEquals(10, writer.numDocs());
+    assertEquals(10, writer.getDocStats().numDocs);
 
     // Should undo our rollback:
     writer.rollback();
@@ -451,7 +451,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                     .setIndexDeletionPolicy(policy)
                                     .setIndexCommit(lastCommit));
-    assertEquals(10, writer.numDocs());
+    assertEquals(10, writer.getDocStats().numDocs);
     // Commits the rollback:
     writer.close();
 
@@ -480,7 +480,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
     // but this time keeping only the last commit:
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                     .setIndexCommit(lastCommit));
-    assertEquals(10, writer.numDocs());
+    assertEquals(10, writer.getDocStats().numDocs);
     
     // Reader still sees fully merged index, because writer
     // opened on the prior commit has not yet committed:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
index fa6cfa9..1184a88 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
@@ -96,8 +96,8 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
     final long maxRAMBytes = (long) (iwc.getRAMBufferSizeMB() * 1024. * 1024.);
     assertEquals(" all flushes must be due numThreads=" + numThreads, 0,
         writer.getFlushingBytes());
-    assertEquals(numDocumentsToIndex, writer.numDocs());
-    assertEquals(numDocumentsToIndex, writer.maxDoc());
+    assertEquals(numDocumentsToIndex, writer.getDocStats().numDocs);
+    assertEquals(numDocumentsToIndex, writer.getDocStats().maxDoc);
     assertTrue("peak bytes without flush exceeded watermark",
         flushPolicy.peakBytesWithoutFlush <= maxRAMBytes);
     assertActiveBytesAfter(flushControl);
@@ -151,8 +151,8 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
 
       assertEquals(" all flushes must be due numThreads=" + numThreads[i], 0,
           writer.getFlushingBytes());
-      assertEquals(numDocumentsToIndex, writer.numDocs());
-      assertEquals(numDocumentsToIndex, writer.maxDoc());
+      assertEquals(numDocumentsToIndex, writer.getDocStats().numDocs);
+      assertEquals(numDocumentsToIndex, writer.getDocStats().maxDoc);
       assertTrue("peak bytes without flush exceeded watermark",
           flushPolicy.peakDocCountWithoutFlush <= iwc.getMaxBufferedDocs());
       assertActiveBytesAfter(flushControl);
@@ -195,8 +195,8 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
       threads[x].join();
     }
     assertEquals(" all flushes must be due", 0, writer.getFlushingBytes());
-    assertEquals(numDocumentsToIndex, writer.numDocs());
-    assertEquals(numDocumentsToIndex, writer.maxDoc());
+    assertEquals(numDocumentsToIndex, writer.getDocStats().numDocs);
+    assertEquals(numDocumentsToIndex, writer.getDocStats().maxDoc);
     if (flushPolicy.flushOnRAM() && !flushPolicy.flushOnDocCount()) {
       final long maxRAMBytes = (long) (iwc.getRAMBufferSizeMB() * 1024. * 1024.);
       assertTrue("peak bytes without flush exceeded watermark",
@@ -256,8 +256,8 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
       assertNotNull(docsWriter);
       DocumentsWriterFlushControl flushControl = docsWriter.flushControl;
       assertEquals(" all flushes must be due", 0, writer.getFlushingBytes());
-      assertEquals(numDocumentsToIndex, writer.numDocs());
-      assertEquals(numDocumentsToIndex, writer.maxDoc());
+      assertEquals(numDocumentsToIndex, writer.getDocStats().numDocs);
+      assertEquals(numDocumentsToIndex, writer.getDocStats().maxDoc);
       if (numThreads[i] == 1) {
         assertFalse(
             "single thread must not block numThreads: " + numThreads[i],

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java
index eb31e73..889dc4e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java
@@ -61,7 +61,7 @@ public class TestIndexManyDocuments extends LuceneTestCase {
       thread.join();
     }
 
-    assertEquals("lost " + (numDocs - w.maxDoc()) + " documents; maxBufferedDocs=" + iwc.getMaxBufferedDocs(), numDocs, w.maxDoc());
+    assertEquals("lost " + (numDocs - w.getDocStats().maxDoc) + " documents; maxBufferedDocs=" + iwc.getMaxBufferedDocs(), numDocs, w.getDocStats().maxDoc);
     w.close();
              
     IndexReader r = DirectoryReader.open(dir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index a5b45f7..66a9460 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -127,16 +127,35 @@ public class TestIndexWriter extends LuceneTestCase {
     // add 100 documents
     for (i = 0; i < 100; i++) {
       addDocWithIndex(writer,i);
+      if (random().nextBoolean()) {
+        writer.commit();
+      }
     }
-    assertEquals(100, writer.maxDoc());
+    IndexWriter.DocStats docStats = writer.getDocStats();
+    assertEquals(100, docStats.maxDoc);
+    assertEquals(100, docStats.numDocs);
     writer.close();
 
     // delete 40 documents
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                             .setMergePolicy(NoMergePolicy.INSTANCE));
+                             .setMergePolicy(new FilterMergePolicy(NoMergePolicy.INSTANCE) {
+                               @Override
+                               public boolean keepFullyDeletedSegment(IOSupplier<CodecReader>
+                                                                          readerIOSupplier) {
+                                 return true;
+                               }
+                             }));
+
     for (i = 0; i < 40; i++) {
       writer.deleteDocuments(new Term("id", ""+i));
+      if (random().nextBoolean()) {
+        writer.commit();
+      }
     }
+    writer.flush();
+    docStats = writer.getDocStats();
+    assertEquals(100, docStats.maxDoc);
+    assertEquals(60, docStats.numDocs);
     writer.close();
 
     reader = DirectoryReader.open(dir);
@@ -145,10 +164,11 @@ public class TestIndexWriter extends LuceneTestCase {
 
     // merge the index down and check that the new doc count is correct
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    assertEquals(60, writer.numDocs());
+    assertEquals(60, writer.getDocStats().numDocs);
     writer.forceMerge(1);
-    assertEquals(60, writer.maxDoc());
-    assertEquals(60, writer.numDocs());
+    docStats = writer.getDocStats();
+    assertEquals(60, docStats.maxDoc);
+    assertEquals(60, docStats.numDocs);
     writer.close();
 
     // check that the index reader gives the same numbers.
@@ -161,8 +181,9 @@ public class TestIndexWriter extends LuceneTestCase {
     // this existing one works correctly:
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                              .setOpenMode(OpenMode.CREATE));
-    assertEquals(0, writer.maxDoc());
-    assertEquals(0, writer.numDocs());
+    docStats = writer.getDocStats();
+    assertEquals(0, docStats.maxDoc);
+    assertEquals(0, docStats.numDocs);
     writer.close();
     dir.close();
   }
@@ -226,7 +247,7 @@ public class TestIndexWriter extends LuceneTestCase {
     // now open index for create:
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                              .setOpenMode(OpenMode.CREATE));
-    assertEquals("should be zero documents", writer.maxDoc(), 0);
+    assertEquals("should be zero documents", writer.getDocStats().maxDoc, 0);
     addDoc(writer);
     writer.close();
 
@@ -2751,7 +2772,7 @@ public class TestIndexWriter extends LuceneTestCase {
       try (IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setIndexCommit(indexCommit))) {
         writer.addDocument(new Document());
         writer.commit();
-        assertEquals(1, writer.maxDoc());
+        assertEquals(1, writer.getDocStats().maxDoc);
         // now check that we moved to 3
         dir.openInput("segments_3", IOContext.READ).close();;
       }
@@ -3147,7 +3168,8 @@ public class TestIndexWriter extends LuceneTestCase {
     for (SegmentCommitInfo info : writer.cloneSegmentInfos()) {
      numSoftDeleted += info.getSoftDelCount();
     }
-    assertEquals(writer.maxDoc() - writer.numDocs(), numSoftDeleted);
+    IndexWriter.DocStats docStats = writer.getDocStats();
+    assertEquals(docStats.maxDoc - docStats.numDocs, numSoftDeleted);
     for (LeafReaderContext context : reader.leaves()) {
       LeafReader leaf = context.reader();
       assertNull(((SegmentReader) leaf).getHardLiveDocs());
@@ -3300,7 +3322,8 @@ public class TestIndexWriter extends LuceneTestCase {
     for (SegmentCommitInfo info : writer.cloneSegmentInfos()) {
       numSoftDeleted += info.getSoftDelCount() + info.getDelCount();
     }
-    assertEquals(writer.maxDoc() - writer.numDocs(), numSoftDeleted);
+    IndexWriter.DocStats docStats = writer.getDocStats();
+    assertEquals(docStats.maxDoc - docStats.numDocs, numSoftDeleted);
     writer.commit();
     try (DirectoryReader dirReader = DirectoryReader.open(dir)) {
       int delCount = 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
index a7c2c9b..e9673b9 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
@@ -445,7 +445,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
                                .setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE)
                                .setIndexCommit(commit));
 
-    assertEquals(1, w.numDocs());
+    assertEquals(1, w.getDocStats().numDocs);
 
     // commit IndexWriter to "third"
     w.addDocument(doc);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
index d01749f..4c9a4e0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
@@ -947,7 +947,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
 
     modifier.deleteDocuments(new TermQuery(new Term("nada", "nada")));
     modifier.commit();
-    assertEquals(5, modifier.numDocs());
+    assertEquals(5, modifier.getDocStats().numDocs);
     modifier.close();
     dir.close();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterFromReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterFromReader.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterFromReader.java
index 2891c25..c196913 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterFromReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterFromReader.java
@@ -47,9 +47,9 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
     IndexWriter w2 = new IndexWriter(dir, iwc);
     r.close();
 
-    assertEquals(1, w2.maxDoc());
+    assertEquals(1, w2.getDocStats().maxDoc);
     w2.addDocument(new Document());
-    assertEquals(2, w2.maxDoc());
+    assertEquals(2, w2.getDocStats().maxDoc);
     w2.close();
     
     IndexReader r2 = DirectoryReader.open(dir);
@@ -74,9 +74,9 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
     assertEquals(1, r.maxDoc());
     r.close();
 
-    assertEquals(1, w2.maxDoc());
+    assertEquals(1, w2.getDocStats().maxDoc);
     w2.addDocument(new Document());
-    assertEquals(2, w2.maxDoc());
+    assertEquals(2, w2.getDocStats().maxDoc);
     w2.close();
     
     IndexReader r2 = DirectoryReader.open(dir);
@@ -141,7 +141,7 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
 
     // Add another doc
     w.addDocument(new Document());
-    assertEquals(2, w.maxDoc());
+    assertEquals(2, w.getDocStats().maxDoc);
     w.close();
 
     IndexWriterConfig iwc = newIndexWriterConfig();
@@ -249,7 +249,7 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
           // rollback writer to last nrt reader
           if (random().nextBoolean()) {
             if (VERBOSE) {
-              System.out.println("  close writer and open new writer from non-NRT reader numDocs=" + w.numDocs());
+              System.out.println("  close writer and open new writer from non-NRT reader numDocs=" + w.getDocStats().numDocs);
             }
             w.close();
             r.close();
@@ -259,7 +259,7 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
             nrtLiveIDs = new HashSet<>(liveIDs);
           } else {
             if (VERBOSE) {
-              System.out.println("  rollback writer and open new writer from NRT reader numDocs=" + w.numDocs());
+              System.out.println("  rollback writer and open new writer from NRT reader numDocs=" + w.getDocStats().numDocs);
             }
             w.rollback();
           }
@@ -383,7 +383,7 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setIndexCommit(r.getIndexCommit());
     w = new IndexWriter(dir, iwc);
-    assertEquals(1, w.numDocs());
+    assertEquals(1, w.getDocStats().numDocs);
 
     r.close();
     DirectoryReader r3 = DirectoryReader.open(w);
@@ -413,7 +413,7 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setIndexCommit(r.getIndexCommit());
     w = new IndexWriter(dir, iwc);
-    assertEquals(2, w.numDocs());
+    assertEquals(2, w.getDocStats().numDocs);
 
     r.close();
     w.close();
@@ -456,7 +456,7 @@ public class TestIndexWriterFromReader extends LuceneTestCase {
     iwc = newIndexWriterConfig();
     iwc.setIndexCommit(r.getIndexCommit());
     IndexWriter w2 = new IndexWriter(dir, iwc);
-    assertEquals(2, w2.maxDoc());
+    assertEquals(2, w2.getDocStats().maxDoc);
     IOUtils.close(r, w2, dir);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
index 29f885a..cff028f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
@@ -189,7 +189,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
 
       w.forceMerge(1);
 
-      assertEquals(5, w.maxDoc());
+      assertEquals(5, w.getDocStats().maxDoc);
 
       // Add 5 more docs
       for(int i=0;i<5;i++) {
@@ -233,7 +233,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
 
       w.forceMerge(1);
 
-      assertEquals(5, w.maxDoc());
+      assertEquals(5, w.getDocStats().maxDoc);
 
       // Add 5 more docs
       for(int i=0;i<5;i++) {
@@ -269,7 +269,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
         w2.addIndexes(new Directory[] {dir});
       });
 
-      assertEquals(1, w2.maxDoc());
+      assertEquals(1, w2.getDocStats().maxDoc);
       DirectoryReader ir = DirectoryReader.open(dir);
       expectThrows(IllegalArgumentException.class, () -> {
         TestUtil.addIndexesSlowly(w2, ir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
index 2d5680c..b05c8f3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
@@ -221,7 +221,7 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
     writer.waitForMerges();
     writer.commit();
     checkInvariants(writer);
-    assertEquals(10, writer.maxDoc());
+    assertEquals(10, writer.getDocStats().maxDoc);
 
     writer.close();
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
index 6931efa..4f7280a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
@@ -164,10 +164,10 @@ public class TestIndexWriterMerging extends LuceneTestCase {
 
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                     .setMergePolicy(newLogMergePolicy()));
-    assertEquals(8, writer.numDocs());
-    assertEquals(10, writer.maxDoc());
+    assertEquals(8, writer.getDocStats().numDocs);
+    assertEquals(10, writer.getDocStats().maxDoc);
     writer.forceMergeDeletes();
-    assertEquals(8, writer.numDocs());
+    assertEquals(8, writer.getDocStats().numDocs);
     writer.close();
     ir = DirectoryReader.open(dir);
     assertEquals(8, ir.maxDoc());
@@ -232,7 +232,7 @@ public class TestIndexWriterMerging extends LuceneTestCase {
         newIndexWriterConfig(new MockAnalyzer(random()))
           .setMergePolicy(newLogMergePolicy(3))
     );
-    assertEquals(49, writer.numDocs());
+    assertEquals(49, writer.getDocStats().numDocs);
     writer.forceMergeDeletes();
     writer.close();
     ir = DirectoryReader.open(dir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
index 2085a77..1e01712 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
@@ -385,8 +385,8 @@ public class TestIndexWriterReader extends LuceneTestCase {
     addDirThreads.joinThreads();
     
     //assertEquals(100 + numDirs * (3 * numIter / 4) * addDirThreads.numThreads
-    //    * addDirThreads.NUM_INIT_DOCS, addDirThreads.mainWriter.numDocs());
-    assertEquals(addDirThreads.count.intValue(), addDirThreads.mainWriter.numDocs());
+    //    * addDirThreads.NUM_INIT_DOCS, addDirThreads.mainwriter.getDocStats().numDocs);
+    assertEquals(addDirThreads.count.intValue(), addDirThreads.mainWriter.getDocStats().numDocs);
 
     addDirThreads.close(true);
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestIsCurrent.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIsCurrent.java b/lucene/core/src/test/org/apache/lucene/index/TestIsCurrent.java
index 7d6e31d..7344ec8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIsCurrent.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIsCurrent.java
@@ -64,7 +64,7 @@ public class TestIsCurrent extends LuceneTestCase {
     DirectoryReader reader = writer.getReader();
 
     // assert index has a document and reader is up2date 
-    assertEquals("One document should be in the index", 1, writer.numDocs());
+    assertEquals("One document should be in the index", 1, writer.getDocStats().numDocs);
     assertTrue("One document added, reader should be current", reader.isCurrent());
 
     // remove document
@@ -73,7 +73,7 @@ public class TestIsCurrent extends LuceneTestCase {
     writer.commit();
 
     // assert document has been deleted (index changed), reader is stale
-    assertEquals("Document should be removed", 0, writer.numDocs());
+    assertEquals("Document should be removed", 0, writer.getDocStats().numDocs);
     assertFalse("Reader should be stale", reader.isCurrent());
 
     reader.close();
@@ -89,7 +89,7 @@ public class TestIsCurrent extends LuceneTestCase {
     DirectoryReader reader = writer.getReader();
 
     // assert index has a document and reader is up2date 
-    assertEquals("One document should be in the index", 1, writer.numDocs());
+    assertEquals("One document should be in the index", 1, writer.getDocStats().numDocs);
     assertTrue("Document added, reader should be stale ", reader.isCurrent());
 
     // remove all documents
@@ -97,7 +97,7 @@ public class TestIsCurrent extends LuceneTestCase {
     writer.commit();
 
     // assert document has been deleted (index changed), reader is stale
-    assertEquals("Document should be removed", 0, writer.numDocs());
+    assertEquals("Document should be removed", 0, writer.getDocStats().numDocs);
     assertFalse("Reader should be stale", reader.isCurrent());
 
     reader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
index fbafc9b..a9db59e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
@@ -136,7 +136,7 @@ public class TestRollingUpdates extends LuceneTestCase {
     }
 
     w.commit();
-    assertEquals(SIZE, w.numDocs());
+    assertEquals(SIZE, w.getDocStats().numDocs);
 
     w.close();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
index fed2d19..8ff7330 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
@@ -241,16 +241,16 @@ public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase {
     DirectoryReader reader = writer.getReader();
     assertEquals(0, reader.numDocs());
     assertEquals(3, reader.maxDoc());
-    assertEquals(0, writer.numDocs());
-    assertEquals(3, writer.maxDoc());
+    assertEquals(0, writer.getDocStats().numDocs);
+    assertEquals(3, writer.getDocStats().maxDoc);
     assertEquals(3, reader.leaves().size());
     reader.close();
     writer.forceMerge(1);
     reader = writer.getReader();
     assertEquals(0, reader.numDocs());
     assertEquals(3, reader.maxDoc());
-    assertEquals(0, writer.numDocs());
-    assertEquals(3, writer.maxDoc());
+    assertEquals(0, writer.getDocStats().numDocs);
+    assertEquals(3, writer.getDocStats().maxDoc);
     assertEquals(1, reader.leaves().size());
     IOUtils.close(reader, writer, dir);
   }
@@ -395,8 +395,8 @@ public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase {
     // when calling forceMergeDeletes.
     writer.forceMergeDeletes(true);
     assertEquals(1, writer.listOfSegmentCommitInfos().size());
-    assertEquals(1, writer.numDocs());
-    assertEquals(1, writer.maxDoc());
+    assertEquals(1, writer.getDocStats().numDocs);
+    assertEquals(1, writer.getDocStats().maxDoc);
     writer.close();
     dir.close();
   }
@@ -538,8 +538,8 @@ public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase {
       }
     });
     writer.forceMerge(1);
-    assertEquals(2, writer.numDocs());
-    assertEquals(2, writer.maxDoc());
+    assertEquals(2, writer.getDocStats().numDocs);
+    assertEquals(2, writer.getDocStats().maxDoc);
     assertFalse(delete.get());
     IOUtils.close(reader, writer, dir);
   }
@@ -700,8 +700,8 @@ public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase {
     writer.softUpdateDocument(new Term("id", "bar-1"), d, new NumericDocValuesField("soft_deletes", 1));
 
     writer.forceMerge(1);
-    assertEquals(2, writer.numDocs()); // foo-2, bar-2
-    assertEquals(3, writer.maxDoc());  // foo-1, foo-2, bar-2
+    assertEquals(2, writer.getDocStats().numDocs); // foo-2, bar-2
+    assertEquals(3, writer.getDocStats().maxDoc);  // foo-1, foo-2, bar-2
     IOUtils.close(writer, dir);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java b/lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java
index dae20b5..ad8c131 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java
@@ -121,8 +121,8 @@ public class TestThreadedForceMerge extends LuceneTestCase {
 
       final int expectedDocCount = (int) ((1+iter)*(200+8*NUM_ITER2*(NUM_THREADS/2.0)*(1+NUM_THREADS)));
 
-      assertEquals("index=" + writer.segString() + " numDocs=" + writer.numDocs() + " maxDoc=" + writer.maxDoc() + " config=" + writer.getConfig(), expectedDocCount, writer.numDocs());
-      assertEquals("index=" + writer.segString() + " numDocs=" + writer.numDocs() + " maxDoc=" + writer.maxDoc() + " config=" + writer.getConfig(), expectedDocCount, writer.maxDoc());
+      assertEquals("index=" + writer.segString() + " numDocs=" + writer.getDocStats().numDocs + " maxDoc=" + writer.getDocStats().maxDoc + " config=" + writer.getConfig(), expectedDocCount, writer.getDocStats().numDocs);
+      assertEquals("index=" + writer.segString() + " numDocs=" + writer.getDocStats().numDocs + " maxDoc=" + writer.getDocStats().maxDoc + " config=" + writer.getConfig(), expectedDocCount, writer.getDocStats().maxDoc);
 
       writer.close();
       writer = new IndexWriter(directory, newIndexWriterConfig(ANALYZER)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
index bbc8051..bd11a58 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
@@ -110,8 +110,8 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
       doc.add(newTextField("content", "aaa " + (i%4), Field.Store.NO));
       w.addDocument(doc);
     }
-    assertEquals(80, w.maxDoc());
-    assertEquals(80, w.numDocs());
+    assertEquals(80, w.getDocStats().maxDoc);
+    assertEquals(80, w.getDocStats().numDocs);
 
     if (VERBOSE) {
       System.out.println("\nTEST: delete docs");
@@ -119,16 +119,16 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
     w.deleteDocuments(new Term("content", "0"));
     w.forceMergeDeletes();
 
-    assertEquals(80, w.maxDoc());
-    assertEquals(60, w.numDocs());
+    assertEquals(80, w.getDocStats().maxDoc);
+    assertEquals(60, w.getDocStats().numDocs);
 
     if (VERBOSE) {
       System.out.println("\nTEST: forceMergeDeletes2");
     }
     ((TieredMergePolicy) w.getConfig().getMergePolicy()).setForceMergeDeletesPctAllowed(10.0);
     w.forceMergeDeletes();
-    assertEquals(60, w.maxDoc());
-    assertEquals(60, w.numDocs());
+    assertEquals(60, w.getDocStats().maxDoc);
+    assertEquals(60, w.getDocStats().numDocs);
     w.close();
     dir.close();
   }
@@ -296,8 +296,8 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
     w.forceMergeDeletes();
     remainingDocs -= deletedThisPass;
     checkSegmentsInExpectations(w, segNamesBefore, false); // There should have been no merges
-    assertEquals("NumDocs should reflect removed documents ", remainingDocs, w.numDocs());
-    assertTrue("Should still be deleted docs in the index", w.numDocs() < w.maxDoc());
+    assertEquals("NumDocs should reflect removed documents ", remainingDocs, w.getDocStats().numDocs);
+    assertTrue("Should still be deleted docs in the index", w.getDocStats().numDocs < w.getDocStats().maxDoc);
 
     // This time, forceMerge. By default this should respect max segment size.
     // Will change for LUCENE-8236
@@ -307,8 +307,8 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
     // Now forceMerge down to one segment, there should be exactly remainingDocs in exactly one segment.
     w.forceMerge(1);
     assertEquals("There should be exaclty one segment now", 1, w.getSegmentCount());
-    assertEquals("maxDoc and numDocs should be identical", w.numDocs(), w.maxDoc());
-    assertEquals("There should be an exact number of documents in that one segment", remainingDocs, w.numDocs());
+    assertEquals("maxDoc and numDocs should be identical", w.getDocStats().numDocs, w.getDocStats().maxDoc);
+    assertEquals("There should be an exact number of documents in that one segment", remainingDocs, w.getDocStats().numDocs);
 
     // Delete 5% and expunge, should be no change.
     segNamesBefore = getSegmentNames(w);
@@ -316,7 +316,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
     w.forceMergeDeletes();
     checkSegmentsInExpectations(w, segNamesBefore, false);
     assertEquals("There should still be only one segment. ", 1, w.getSegmentCount());
-    assertTrue("The segment should have deleted documents", w.numDocs() < w.maxDoc());
+    assertTrue("The segment should have deleted documents", w.getDocStats().numDocs < w.getDocStats().maxDoc);
 
     w.forceMerge(1); // back to one segment so deletePctDocsFromEachSeg still works
 
@@ -325,17 +325,17 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
     w.forceMergeDeletes();
 
     assertEquals("There should still be only one segment. ", 1, w.getSegmentCount());
-    assertEquals("The segment should have no deleted documents", w.numDocs(), w.maxDoc());
+    assertEquals("The segment should have no deleted documents", w.getDocStats().numDocs, w.getDocStats().maxDoc);
 
 
     // sanity check, at this point we should have an over`-large segment, we know we have exactly one.
-    assertTrue("Our single segment should have quite a few docs", w.numDocs() > 1_000);
+    assertTrue("Our single segment should have quite a few docs", w.getDocStats().numDocs > 1_000);
 
     // Delete 60% of the documents and then add a few more docs and commit. This should "singleton merge" the large segment
     // created above. 60% leaves some wriggle room, LUCENE-8263 will change this assumption and should be tested
     // when we deal with that JIRA.
 
-    deletedThisPass = deletePctDocsFromEachSeg(w, (w.numDocs() * 60) / 100, true);
+    deletedThisPass = deletePctDocsFromEachSeg(w, (w.getDocStats().numDocs * 60) / 100, true);
     remainingDocs -= deletedThisPass;
 
     for (int i = 0; i < 50; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
index 19fa917..13323a4 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
@@ -171,7 +171,7 @@ public class TestBoolean2 extends LuceneTestCase {
       RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
       w.addIndexes(copy);
       copy.close();
-      docCount = w.maxDoc();
+      docCount = w.getDocStats().maxDoc;
       w.close();
       mulFactor *= 2;
     } while(docCount < 3000 * NUM_FILLER_DOCS);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
index b70784c..5f4134c 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
@@ -583,7 +583,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
             }
             docs.close();
             if (VERBOSE) {
-              System.out.println("TEST: index count=" + writerRef.get().maxDoc());
+              System.out.println("TEST: index count=" + writerRef.get().getDocStats().maxDoc);
             }
           } catch (IOException ioe) {
             throw new RuntimeException(ioe);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDirectory.java
index 5f2d447..11647c6 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDirectory.java
@@ -58,7 +58,7 @@ public class TestByteBuffersDirectory extends BaseDirectoryTestCase {
         writer.addDocument(doc);
       }
       writer.commit();
-      assertEquals(docs, writer.numDocs());
+      assertEquals(docs, writer.getDocStats().numDocs);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
index f0f2d46..02ea9d7 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
@@ -64,7 +64,7 @@ public class TestRAMDirectory extends BaseDirectoryTestCase {
       doc.add(newStringField("content", English.intToEnglish(i).trim(), Field.Store.YES));
       writer.addDocument(doc);
     }
-    assertEquals(DOCS_TO_ADD, writer.maxDoc());
+    assertEquals(DOCS_TO_ADD, writer.getDocStats().maxDoc);
     writer.close();
     dir.close();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
index 5efeb3e..b36bf39 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
@@ -195,7 +195,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
     parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream, ft);
     fullPathField = new StringField(Consts.FULL, "", Field.Store.YES);
 
-    nextID = indexWriter.maxDoc();
+    nextID = indexWriter.getDocStats().maxDoc;
 
     if (cache == null) {
       cache = defaultTaxonomyWriterCache();
@@ -968,7 +968,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
     shouldRefreshReaderManager = true;
     initReaderManager(); // ensure that it's initialized
     refreshReaderManager();
-    nextID = indexWriter.maxDoc();
+    nextID = indexWriter.getDocStats().maxDoc;
     taxoArrays = null; // must nullify so that it's re-computed next time it's needed
     
     // need to clear the cache, so that addCategory won't accidentally return

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java b/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
index 32a610b..5f18840 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
@@ -262,7 +262,7 @@ public class TestMoreLikeThis extends LuceneTestCase {
       doc.add(newTextField(NOT_FOR_SALE, item, Field.Store.YES));
     }
     writer.addDocument(doc);
-    return writer.numDocs() - 1;
+    return writer.getDocStats().numDocs - 1;
   }
 
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-7161")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
index 33c4421..eb208c6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
@@ -315,12 +315,8 @@ public class RandomIndexWriter implements Closeable {
     return w.commit();
   }
   
-  public int numDocs() {
-    return w.numDocs();
-  }
-
-  public int maxDoc() {
-    return w.maxDoc();
+  public IndexWriter.DocStats getDocStats() {
+    return w.getDocStats();
   }
 
   public long deleteAll() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
index 575c6c9..b6b15a1 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
@@ -635,7 +635,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
 
     writer.commit();
 
-    assertEquals("index=" + writer.segString() + " addCount=" + addCount + " delCount=" + delCount, addCount.get() - delCount.get(), writer.numDocs());
+    assertEquals("index=" + writer.segString() + " addCount=" + addCount + " delCount=" + delCount, addCount.get() - delCount.get(), writer.getDocStats().numDocs);
 
     doClose();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e974311d/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
index 2a9055a..2c5dddc 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/ClassificationUpdateProcessorTest.java
@@ -502,6 +502,6 @@ public class ClassificationUpdateProcessorTest extends SolrTestCaseJ4 {
 
   private int addDoc(RandomIndexWriter writer, Document doc) throws IOException {
     writer.addDocument(doc);
-    return writer.numDocs() - 1;
+    return writer.getDocStats().numDocs - 1;
   }
 }


[27/34] lucene-solr:jira/http2: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr

Posted by da...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr


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

Branch: refs/heads/jira/http2
Commit: 3ca7e65388a124696f4d9c50a82dab795f1ff986
Parents: 15b3629 bfd28a8
Author: Erick Erickson <Er...@gmail.com>
Authored: Fri Dec 14 07:43:06 2018 -0800
Committer: Erick Erickson <Er...@gmail.com>
Committed: Fri Dec 14 07:43:06 2018 -0800

----------------------------------------------------------------------
 .../index/TestBackwardsCompatibility.java       |   7 +++++--
 .../org/apache/lucene/index/index.7.6.0-cfs.zip | Bin 0 -> 15655 bytes
 .../apache/lucene/index/index.7.6.0-nocfs.zip   | Bin 0 -> 15649 bytes
 .../org/apache/lucene/index/sorted.7.6.0.zip    | Bin 0 -> 74541 bytes
 solr/CHANGES.txt                                |   4 ++++
 .../apache/solr/cloud/MiniSolrCloudCluster.java |  19 ++++---------------
 solr/webapp/web/js/angular/app.js               |   2 +-
 solr/webapp/web/js/angular/controllers/query.js |   3 ++-
 solr/webapp/web/js/angular/services.js          |   2 +-
 9 files changed, 17 insertions(+), 20 deletions(-)
----------------------------------------------------------------------



[19/34] lucene-solr:jira/http2: Add 7.6.0 back compat test indexes

Posted by da...@apache.org.
Add 7.6.0 back compat test indexes


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

Branch: refs/heads/jira/http2
Commit: d3ee925bdd75e4c1be97b73410dd49df58e3593b
Parents: f844461
Author: Nicholas Knize <nk...@gmail.com>
Authored: Thu Dec 13 21:50:53 2018 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Thu Dec 13 21:50:53 2018 -0600

----------------------------------------------------------------------
 .../lucene/index/TestBackwardsCompatibility.java   |   7 +++++--
 .../org/apache/lucene/index/index.7.6.0-cfs.zip    | Bin 0 -> 15655 bytes
 .../org/apache/lucene/index/index.7.6.0-nocfs.zip  | Bin 0 -> 15649 bytes
 .../test/org/apache/lucene/index/sorted.7.6.0.zip  | Bin 0 -> 74541 bytes
 4 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3ee925b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 44911db..23fac47 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -304,7 +304,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     "7.4.0-cfs",
     "7.4.0-nocfs",
     "7.5.0-cfs",
-    "7.5.0-nocfs"
+    "7.5.0-nocfs",
+    "7.6.0-cfs",
+    "7.6.0-nocfs"
   };
 
   public static String[] getOldNames() {
@@ -320,7 +322,8 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     "sorted.7.3.0",
     "sorted.7.3.1",
     "sorted.7.4.0",
-    "sorted.7.5.0"
+    "sorted.7.5.0",
+    "sorted.7.6.0"
   };
 
   public static String[] getOldSortedNames() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3ee925b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-cfs.zip
new file mode 100644
index 0000000..79fb654
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3ee925b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-nocfs.zip
new file mode 100644
index 0000000..daaee82
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.6.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3ee925b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.6.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.6.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.6.0.zip
new file mode 100644
index 0000000..f510fce
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.6.0.zip differ


[26/34] lucene-solr:jira/http2: SOLR-12727: Fix tests to work with 'interesting' entries in the hosts file like 'fe80::1%lo0 localhost'

Posted by da...@apache.org.
SOLR-12727: Fix tests to work with 'interesting' entries in the hosts file like 'fe80::1%lo0 localhost'


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

Branch: refs/heads/jira/http2
Commit: 15b36299c49ab0fb933ffada6eed8b40d47844c1
Parents: f844461
Author: Erick Erickson <Er...@gmail.com>
Authored: Fri Dec 14 07:42:57 2018 -0800
Committer: Erick Erickson <Er...@gmail.com>
Committed: Fri Dec 14 07:42:57 2018 -0800

----------------------------------------------------------------------
 .../src/java/org/apache/solr/cloud/ZkTestServer.java             | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15b36299/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
index 8aa1e7d..f52e231 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
@@ -456,7 +456,7 @@ public class ZkTestServer {
   }
 
   public String getZkHost() {
-    return "localhost:" + zkServer.getLocalPort();
+    return "127.0.0.1:" + zkServer.getLocalPort();
   }
 
   public String getZkAddress() {
@@ -471,7 +471,7 @@ public class ZkTestServer {
   public String getZkAddress(String chroot) {
     if (!chroot.startsWith("/"))
       chroot = "/" + chroot;
-    return "localhost:" + zkServer.getLocalPort() + chroot;
+    return "127.0.0.1:" + zkServer.getLocalPort() + chroot;
   }
 
   /**


[03/34] lucene-solr:jira/http2: test

Posted by da...@apache.org.
test


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

Branch: refs/heads/jira/http2
Commit: adf73ff6bd49bd09701e8a960cdcfb492cd3c6e0
Parents: ef2f0cd
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Dec 11 10:11:04 2018 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 12 13:20:37 2018 +0100

----------------------------------------------------------------------
 .../index/BinaryDocValuesFieldUpdates.java      |  8 +++
 .../lucene/index/DocValuesFieldUpdates.java     | 56 ++++++++++++++++----
 .../index/NumericDocValuesFieldUpdates.java     |  6 +++
 3 files changed, 60 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adf73ff6/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
index fa60929..67227ed 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java
@@ -108,6 +108,14 @@ final class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
   }
 
   @Override
+  protected void copy(int i, int j) {
+    super.copy(i, j);
+
+    offsets.set(j, offsets.get(i));
+    lengths.set(j, lengths.get(i));
+  }
+
+  @Override
   protected void grow(int size) {
     super.grow(size);
     offsets = offsets.grow(size);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adf73ff6/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
index 464a7f2..71d9c7d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
@@ -21,10 +21,10 @@ import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.SparseFixedBitSet;
+import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PagedMutable;
 
@@ -32,11 +32,11 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 /**
  * Holds updates of a single DocValues field, for a set of documents within one segment.
- * 
+ *
  * @lucene.experimental
  */
 abstract class DocValuesFieldUpdates implements Accountable {
-  
+
   protected static final int PAGE_SIZE = 1024;
   private static final long HAS_VALUE_MASK = 1;
   private static final long HAS_NO_VALUE_MASK = 0;
@@ -260,7 +260,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
   final boolean getFinished() {
     return finished;
   }
-  
+
   abstract void add(int doc, long value);
 
   abstract void add(int doc, BytesRef value);
@@ -285,11 +285,17 @@ abstract class DocValuesFieldUpdates implements Accountable {
       throw new IllegalStateException("already finished");
     }
     finished = true;
-    // shrink wrap
-    if (size < docs.size()) {
-      resize(size);
-    }
-    new InPlaceMergeSorter() {
+
+    // Swaps and comparisons are quite costly with all these packed ints,
+    // so we use TimSorter instead of InPlaceMergeSorter, which has the nice
+    // property of doing fewer swaps/copies at the expense of a higher memory
+    // usage
+    final int tempSlotsOffset = size;
+    // we grow the current structures to get temporary storage
+    resize(Math.addExact(size, size / 2)); // 50% extra memory for temp slots
+    final int numTempSlots = Math.toIntExact(docs.size() - size);
+    new TimSorter(numTempSlots) {
+
       @Override
       protected void swap(int i, int j) {
         DocValuesFieldUpdates.this.swap(i, j);
@@ -302,7 +308,33 @@ abstract class DocValuesFieldUpdates implements Accountable {
         // stable and preserving original order so the last update to that docID wins
         return Long.compare(docs.get(i)>>>1, docs.get(j)>>>1);
       }
+
+      @Override
+      protected void save(int i, int len) {
+        assert len <= numTempSlots;
+        for (int k = 0; k < len; ++k) {
+          copy(i + k, tempSlotsOffset + k);
+        }
+      }
+
+      @Override
+      protected void restore(int src, int dest) {
+        copy(tempSlotsOffset + src, dest);
+      }
+
+      @Override
+      protected void copy(int src, int dest) {
+        DocValuesFieldUpdates.this.copy(src, dest);
+      }
+
+      @Override
+      protected int compareSaved(int i, int j) {
+        return compare(tempSlotsOffset + i, j);
+      }
     }.sort(0, size);
+
+    // shrink wrap to save memory, this will also release temporary storage
+    resize(size);
   }
 
   /** Returns true if this instance contains any updates. */
@@ -350,6 +382,10 @@ abstract class DocValuesFieldUpdates implements Accountable {
     docs.set(i, tmpDoc);
   }
 
+  protected void copy(int i, int j) {
+    docs.set(j, docs.get(i));
+  }
+
   protected void grow(int size) {
     docs = docs.grow(size);
   }
@@ -477,7 +513,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
     }
 
     protected abstract BytesRef binaryValue();
-    
+
     protected abstract long longValue();
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adf73ff6/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
index 550a86a..05d5c14 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
@@ -100,6 +100,12 @@ final class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
   }
 
   @Override
+  protected void copy(int i, int j) {
+    super.copy(i, j);
+    values.set(j, values.get(i));
+  }
+
+  @Override
   protected void grow(int size) {
     super.grow(size);
     values = values.grow(size);


[05/34] lucene-solr:jira/http2: LUCENE-8605: Separate bounding box spatial logic from query logic on LatLonShapeBoundingBoxQuery

Posted by da...@apache.org.
LUCENE-8605: Separate bounding box spatial logic from query logic on LatLonShapeBoundingBoxQuery


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

Branch: refs/heads/jira/http2
Commit: ce9a8012c080dbf2a96a6755a0b7048ab5739419
Parents: 55993ec
Author: iverase <iv...@apache.org>
Authored: Wed Dec 12 13:46:35 2018 +0100
Committer: iverase <iv...@apache.org>
Committed: Wed Dec 12 13:46:35 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../document/LatLonShapeBoundingBoxQuery.java   | 269 +---------------
 .../java/org/apache/lucene/geo/Rectangle2D.java | 315 +++++++++++++++++++
 .../org/apache/lucene/geo/TestRectangle2D.java  | 100 ++++++
 4 files changed, 428 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce9a8012/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 212607f..b4c7ca5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -276,6 +276,9 @@ Other
 * LUCENE-8573: BKDWriter now uses FutureArrays#mismatch to compute shared prefixes.
   (Christoph Büscher via Adrien Grand)
 
+* LUCENE-8605: Separate bounding box spatial logic from query logic on LatLonShapeBoundingBoxQuery.
+  (Ignacio Vera)
+
 ======================= Lucene 7.6.0 =======================
 
 Build

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce9a8012/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
index ebbdeed..43fe28e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
@@ -16,25 +16,11 @@
  */
 package org.apache.lucene.document;
 
-import java.util.Arrays;
-
 import org.apache.lucene.geo.Rectangle;
-import org.apache.lucene.geo.Tessellator;
+import org.apache.lucene.geo.Rectangle2D;
 import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.util.FutureArrays;
 import org.apache.lucene.util.NumericUtils;
 
-import static org.apache.lucene.document.LatLonShape.BYTES;
-import static org.apache.lucene.geo.GeoEncodingUtils.MAX_LON_ENCODED;
-import static org.apache.lucene.geo.GeoEncodingUtils.MIN_LON_ENCODED;
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitudeCeil;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
-import static org.apache.lucene.geo.GeoUtils.orient;
-
 /**
  * Finds all previously indexed shapes that intersect the specified bounding box.
  *
@@ -44,88 +30,18 @@ import static org.apache.lucene.geo.GeoUtils.orient;
  *  @lucene.experimental
  **/
 final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
-  final byte[] bbox;
-  final byte[] west;
-  final int minX;
-  final int maxX;
-  final int minY;
-  final int maxY;
+  final Rectangle2D rectangle2D;
 
   public LatLonShapeBoundingBoxQuery(String field, LatLonShape.QueryRelation queryRelation, double minLat, double maxLat, double minLon, double maxLon) {
     super(field, queryRelation);
-
-    this.bbox = new byte[4 * LatLonShape.BYTES];
-    int minXenc = encodeLongitudeCeil(minLon);
-    int maxXenc = encodeLongitude(maxLon);
-    int minYenc = encodeLatitudeCeil(minLat);
-    int maxYenc = encodeLatitude(maxLat);
-    if (minYenc > maxYenc) {
-      minYenc = maxYenc;
-    }
-    this.minY = minYenc;
-    this.maxY = maxYenc;
-
-    if (minLon > maxLon == true) {
-      // crossing dateline is split into east/west boxes
-      this.west = new byte[4 * LatLonShape.BYTES];
-      this.minX = minXenc;
-      this.maxX = maxXenc;
-      encode(MIN_LON_ENCODED, this.maxX, this.minY, this.maxY, this.west);
-      encode(this.minX, MAX_LON_ENCODED, this.minY, this.maxY, this.bbox);
-    } else {
-      // encodeLongitudeCeil may cause minX to be > maxX iff
-      // the delta between the longtude < the encoding resolution
-      if (minXenc > maxXenc) {
-        minXenc = maxXenc;
-      }
-      this.west = null;
-      this.minX = minXenc;
-      this.maxX = maxXenc;
-      encode(this.minX, this.maxX, this.minY, this.maxY, bbox);
-    }
-  }
-
-  /** encodes a bounding box into the provided byte array */
-  private static void encode(final int minX, final int maxX, final int minY, final int maxY, byte[] b) {
-    if (b == null) {
-      b = new byte[4 * LatLonShape.BYTES];
-    }
-    LatLonShape.encodeTriangleBoxVal(minY, b, 0);
-    LatLonShape.encodeTriangleBoxVal(minX, b, BYTES);
-    LatLonShape.encodeTriangleBoxVal(maxY, b, 2 * BYTES);
-    LatLonShape.encodeTriangleBoxVal(maxX, b, 3 * BYTES);
+    Rectangle rectangle = new Rectangle(minLat, maxLat, minLon, maxLon);
+    this.rectangle2D = Rectangle2D.create(rectangle);
   }
 
   @Override
   protected Relation relateRangeBBoxToQuery(int minXOffset, int minYOffset, byte[] minTriangle,
                                             int maxXOffset, int maxYOffset, byte[] maxTriangle) {
-    Relation eastRelation = compareBBoxToRangeBBox(this.bbox, minXOffset, minYOffset, minTriangle, maxXOffset, maxYOffset, maxTriangle);
-    if (this.crossesDateline() && eastRelation == Relation.CELL_OUTSIDE_QUERY) {
-      return compareBBoxToRangeBBox(this.west, minXOffset, minYOffset, minTriangle, maxXOffset, maxYOffset, maxTriangle);
-    }
-
-    return eastRelation;
-  }
-
-  /** static utility method to compare a bbox with a range of triangles (just the bbox of the triangle collection) */
-  protected static Relation compareBBoxToRangeBBox(final byte[] bbox,
-                                                   int minXOffset, int minYOffset, byte[] minTriangle,
-                                                   int maxXOffset, int maxYOffset, byte[] maxTriangle) {
-    // check bounding box (DISJOINT)
-    if (FutureArrays.compareUnsigned(minTriangle, minXOffset, minXOffset + BYTES, bbox, 3 * BYTES, 4 * BYTES) > 0 ||
-        FutureArrays.compareUnsigned(maxTriangle, maxXOffset, maxXOffset + BYTES, bbox, BYTES, 2 * BYTES) < 0 ||
-        FutureArrays.compareUnsigned(minTriangle, minYOffset, minYOffset + BYTES, bbox, 2 * BYTES, 3 * BYTES) > 0 ||
-        FutureArrays.compareUnsigned(maxTriangle, maxYOffset, maxYOffset + BYTES, bbox, 0, BYTES) < 0) {
-      return Relation.CELL_OUTSIDE_QUERY;
-    }
-
-    if (FutureArrays.compareUnsigned(minTriangle, minXOffset, minXOffset + BYTES, bbox, BYTES, 2 * BYTES) >= 0 &&
-        FutureArrays.compareUnsigned(maxTriangle, maxXOffset, maxXOffset + BYTES, bbox, 3 * BYTES, 4 * BYTES) <= 0 &&
-        FutureArrays.compareUnsigned(minTriangle, minYOffset, minYOffset + BYTES, bbox, 0, BYTES) >= 0 &&
-        FutureArrays.compareUnsigned(maxTriangle, maxYOffset, maxYOffset + BYTES, bbox, 2 * BYTES, 3 * BYTES) <= 0) {
-      return Relation.CELL_INSIDE_QUERY;
-    }
-    return Relation.CELL_CROSSES_QUERY;
+    return rectangle2D.relateRangeBBox(minXOffset, minYOffset, minTriangle, maxXOffset, maxYOffset, maxTriangle);
   }
 
   /** returns true if the query matches the encoded triangle */
@@ -144,160 +60,9 @@ final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
     int cY = (int)(c & 0x00000000FFFFFFFFL);
 
     if (queryRelation == LatLonShape.QueryRelation.WITHIN) {
-      return queryContainsTriangle(aX, aY, bX, bY, cX, cY);
-    }
-    return queryMatches(aX, aY, bX, bY, cX, cY);
-  }
-
-  private boolean queryContainsTriangle(int ax, int ay, int bx, int by, int cx, int cy) {
-    if (this.crossesDateline() == true) {
-      return bboxContainsTriangle(ax, ay, bx, by, cx, cy, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
-          || bboxContainsTriangle(ax, ay, bx, by, cx, cy, this.minX, MAX_LON_ENCODED, this.minY, this.maxY);
-    }
-    return bboxContainsTriangle(ax, ay, bx, by, cx, cy, minX, maxX, minY, maxY);
-  }
-
-  /** static utility method to check if a bounding box contains a point */
-  private static boolean bboxContainsPoint(int x, int y, int minX, int maxX, int minY, int maxY) {
-    return (x < minX || x > maxX || y < minY || y > maxY) == false;
-  }
-
-  /** static utility method to check if a bounding box contains a triangle */
-  private static boolean bboxContainsTriangle(int ax, int ay, int bx, int by, int cx, int cy,
-                                              int minX, int maxX, int minY, int maxY) {
-    return bboxContainsPoint(ax, ay, minX, maxX, minY, maxY)
-        && bboxContainsPoint(bx, by, minX, maxX, minY, maxY)
-        && bboxContainsPoint(cx, cy, minX, maxX, minY, maxY);
-  }
-
-  /** instance method to check if query box contains point */
-  private boolean queryContainsPoint(int x, int y) {
-    if (this.crossesDateline() == true) {
-      return bboxContainsPoint(x, y, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
-          || bboxContainsPoint(x, y, this.minX, MAX_LON_ENCODED, this.minY, this.maxY);
-    }
-    return bboxContainsPoint(x, y, this.minX, this.maxX, this.minY, this.maxY);
-  }
-
-  protected boolean queryMatches(int aX, int aY, int bX, int bY, int cX, int cY) {
-    // 1. query contains any triangle points
-    if (queryContainsPoint(aX, aY) || queryContainsPoint(bX, bY) || queryContainsPoint(cX, cY)) {
-      return true;
-    }
-
-    // compute bounding box of triangle
-    int tMinX = StrictMath.min(StrictMath.min(aX, bX), cX);
-    int tMaxX = StrictMath.max(StrictMath.max(aX, bX), cX);
-    int tMinY = StrictMath.min(StrictMath.min(aY, bY), cY);
-    int tMaxY = StrictMath.max(StrictMath.max(aY, bY), cY);
-
-    // 2. check bounding boxes are disjoint
-    if (this.crossesDateline() == true) {
-      if (boxesAreDisjoint(tMinX, tMaxX, tMinY, tMaxY, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
-          && boxesAreDisjoint(tMinX, tMaxX, tMinY, tMaxY, this.minX, MAX_LON_ENCODED, this.minY, this.maxY)) {
-        return false;
-      }
-    } else if (tMaxX < minX || tMinX > maxX || tMinY > maxY || tMaxY < minY) {
-      return false;
+      return rectangle2D.containsTriangle(aX, aY, bX, bY, cX, cY);
     }
-
-    // 3. check triangle contains any query points
-    if (Tessellator.pointInTriangle(minX, minY, aX, aY, bX, bY, cX, cY)) {
-      return true;
-    } else if (Tessellator.pointInTriangle(maxX, minY, aX, aY, bX, bY, cX, cY)) {
-      return true;
-    } else if (Tessellator.pointInTriangle(maxX, maxY, aX, aY, bX, bY, cX, cY)) {
-      return true;
-    } else if (Tessellator.pointInTriangle(minX, maxY, aX, aY, bX, bY, cX, cY)) {
-      return true;
-    }
-
-    // 4. last ditch effort: check crossings
-    if (queryIntersects(aX, aY, bX, bY, cX, cY)) {
-      return true;
-    }
-    return false;
-  }
-
-  /** returns true if the edge (defined by (ax, ay) (bx, by)) intersects the query */
-  private static boolean edgeIntersectsBox(int ax, int ay, int bx, int by,
-                                           int minX, int maxX, int minY, int maxY) {
-    // shortcut: if edge is a point (occurs w/ Line shapes); simply check bbox w/ point
-    if (ax == bx && ay == by) {
-      return Rectangle.containsPoint(ay, ax, minY, maxY, minX, maxX);
-    }
-
-    // shortcut: check if either of the end points fall inside the box
-    if (bboxContainsPoint(ax, ay, minX, maxX, minY, maxY)
-        || bboxContainsPoint(bx, by, minX, maxX, minY, maxY)) {
-      return true;
-    }
-
-    // shortcut: check bboxes of edges are disjoint
-    if (boxesAreDisjoint(Math.min(ax, bx), Math.max(ax, bx), Math.min(ay, by), Math.max(ay, by),
-        minX, maxX, minY, maxY)) {
-      return false;
-    }
-
-    // shortcut: edge is a point
-    if (ax == bx && ay == by) {
-      return false;
-    }
-
-    // top
-    if (orient(ax, ay, bx, by, minX, maxY) * orient(ax, ay, bx, by, maxX, maxY) <= 0 &&
-        orient(minX, maxY, maxX, maxY, ax, ay) * orient(minX, maxY, maxX, maxY, bx, by) <= 0) {
-      return true;
-    }
-
-    // right
-    if (orient(ax, ay, bx, by, maxX, maxY) * orient(ax, ay, bx, by, maxX, minY) <= 0 &&
-        orient(maxX, maxY, maxX, minY, ax, ay) * orient(maxX, maxY, maxX, minY, bx, by) <= 0) {
-      return true;
-    }
-
-    // bottom
-    if (orient(ax, ay, bx, by, maxX, minY) * orient(ax, ay, bx, by, minX, minY) <= 0 &&
-        orient(maxX, minY, minX, minY, ax, ay) * orient(maxX, minY, minX, minY, bx, by) <= 0) {
-      return true;
-    }
-
-    // left
-    if (orient(ax, ay, bx, by, minX, minY) * orient(ax, ay, bx, by, minX, maxY) <= 0 &&
-        orient(minX, minY, minX, maxY, ax, ay) * orient(minX, minY, minX, maxY, bx, by) <= 0) {
-      return true;
-    }
-    return false;
-  }
-
-  /** returns true if the edge (defined by (ax, ay) (bx, by)) intersects the query */
-  private boolean edgeIntersectsQuery(int ax, int ay, int bx, int by) {
-    if (this.crossesDateline() == true) {
-      return edgeIntersectsBox(ax, ay, bx, by, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
-          || edgeIntersectsBox(ax, ay, bx, by, this.minX, MAX_LON_ENCODED, this.minY, this.maxY);
-    }
-    return edgeIntersectsBox(ax, ay, bx, by, this.minX, this.maxX, this.minY, this.maxY);
-  }
-
-  /** returns true if the query intersects the provided triangle (in encoded space) */
-  private boolean queryIntersects(int ax, int ay, int bx, int by, int cx, int cy) {
-    // check each edge of the triangle against the query
-    if (edgeIntersectsQuery(ax, ay, bx, by) ||
-        edgeIntersectsQuery(bx, by, cx, cy) ||
-        edgeIntersectsQuery(cx, cy, ax, ay)) {
-      return true;
-    }
-    return false;
-  }
-
-  /** utility method to check if two boxes are disjoint */
-  public static boolean boxesAreDisjoint(final int aMinX, final int aMaxX, final int aMinY, final int aMaxY,
-                                          final int bMinX, final int bMaxX, final int bMinY, final int bMaxY) {
-    return (aMaxX < bMinX || aMinX > bMaxX || aMaxY < bMinY || aMinY > bMaxY);
-  }
-
-  public boolean crossesDateline() {
-    return minX > maxX;
+    return rectangle2D.intersectsTriangle(aX, aY, bX, bY, cX, cY);
   }
 
   @Override
@@ -307,16 +72,13 @@ final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
 
   @Override
   protected boolean equalsTo(Object o) {
-    return super.equalsTo(o)
-        && Arrays.equals(bbox, ((LatLonShapeBoundingBoxQuery)o).bbox)
-        && Arrays.equals(west, ((LatLonShapeBoundingBoxQuery)o).west);
+    return super.equalsTo(o) && rectangle2D.equals(((LatLonShapeBoundingBoxQuery)o).rectangle2D);
   }
 
   @Override
   public int hashCode() {
     int hash = super.hashCode();
-    hash = 31 * hash + Arrays.hashCode(bbox);
-    hash = 31 * hash + Arrays.hashCode(west);
+    hash = 31 * hash + rectangle2D.hashCode();
     return hash;
   }
 
@@ -330,18 +92,7 @@ final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
       sb.append(this.field);
       sb.append(':');
     }
-    sb.append("Rectangle(lat=");
-    sb.append(decodeLatitude(minY));
-    sb.append(" TO ");
-    sb.append(decodeLatitude(maxY));
-    sb.append(" lon=");
-    sb.append(decodeLongitude(minX));
-    sb.append(" TO ");
-    sb.append(decodeLongitude(maxX));
-    if (maxX < minX) {
-      sb.append(" [crosses dateline!]");
-    }
-    sb.append(")");
+    sb.append(rectangle2D.toString());
     return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce9a8012/lucene/sandbox/src/java/org/apache/lucene/geo/Rectangle2D.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/geo/Rectangle2D.java b/lucene/sandbox/src/java/org/apache/lucene/geo/Rectangle2D.java
new file mode 100644
index 0000000..c3acaa5
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/geo/Rectangle2D.java
@@ -0,0 +1,315 @@
+/*
+ * 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.lucene.geo;
+
+import java.util.Arrays;
+
+import org.apache.lucene.document.LatLonShape;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.FutureArrays;
+
+import static org.apache.lucene.document.LatLonShape.BYTES;
+import static org.apache.lucene.geo.GeoEncodingUtils.MAX_LON_ENCODED;
+import static org.apache.lucene.geo.GeoEncodingUtils.MIN_LON_ENCODED;
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitudeCeil;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
+import static org.apache.lucene.geo.GeoUtils.orient;
+
+/**
+ * 2D rectangle implementation containing spatial logic.
+ *
+ * @lucene.internal
+ */
+public class Rectangle2D {
+  final byte[] bbox;
+  final byte[] west;
+  final int minX;
+  final int maxX;
+  final int minY;
+  final int maxY;
+
+  private Rectangle2D(double minLat, double maxLat, double minLon, double maxLon) {
+    this.bbox = new byte[4 * BYTES];
+    int minXenc = encodeLongitudeCeil(minLon);
+    int maxXenc = encodeLongitude(maxLon);
+    int minYenc = encodeLatitudeCeil(minLat);
+    int maxYenc = encodeLatitude(maxLat);
+    if (minYenc > maxYenc) {
+      minYenc = maxYenc;
+    }
+    this.minY = minYenc;
+    this.maxY = maxYenc;
+
+    if (minLon > maxLon == true) {
+      // crossing dateline is split into east/west boxes
+      this.west = new byte[4 * BYTES];
+      this.minX = minXenc;
+      this.maxX = maxXenc;
+      encode(MIN_LON_ENCODED, this.maxX, this.minY, this.maxY, this.west);
+      encode(this.minX, MAX_LON_ENCODED, this.minY, this.maxY, this.bbox);
+    } else {
+      // encodeLongitudeCeil may cause minX to be > maxX iff
+      // the delta between the longitude < the encoding resolution
+      if (minXenc > maxXenc) {
+        minXenc = maxXenc;
+      }
+      this.west = null;
+      this.minX = minXenc;
+      this.maxX = maxXenc;
+      encode(this.minX, this.maxX, this.minY, this.maxY, bbox);
+    }
+  }
+
+  /** Builds a Rectangle2D from rectangle */
+  public static Rectangle2D create(Rectangle rectangle) {
+    return new Rectangle2D(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon);
+  }
+
+  public boolean crossesDateline() {
+    return minX > maxX;
+  }
+
+  /** Checks if the rectangle contains the provided point **/
+  public boolean queryContainsPoint(int x, int y) {
+    if (this.crossesDateline() == true) {
+      return bboxContainsPoint(x, y, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
+          || bboxContainsPoint(x, y, this.minX, MAX_LON_ENCODED, this.minY, this.maxY);
+    }
+    return bboxContainsPoint(x, y, this.minX, this.maxX, this.minY, this.maxY);
+  }
+
+  /** compare this to a provided rangle bounding box **/
+  public PointValues.Relation relateRangeBBox(int minXOffset, int minYOffset, byte[] minTriangle,
+                                              int maxXOffset, int maxYOffset, byte[] maxTriangle) {
+    PointValues.Relation eastRelation = compareBBoxToRangeBBox(this.bbox, minXOffset, minYOffset, minTriangle, maxXOffset, maxYOffset, maxTriangle);
+    if (this.crossesDateline() && eastRelation == PointValues.Relation.CELL_OUTSIDE_QUERY) {
+      return compareBBoxToRangeBBox(this.west, minXOffset, minYOffset, minTriangle, maxXOffset, maxYOffset, maxTriangle);
+    }
+    return eastRelation;
+  }
+
+  /** Checks if the rectangle intersects the provided triangle **/
+  public boolean intersectsTriangle(int aX, int aY, int bX, int bY, int cX, int cY) {
+    // 1. query contains any triangle points
+    if (queryContainsPoint(aX, aY) || queryContainsPoint(bX, bY) || queryContainsPoint(cX, cY)) {
+      return true;
+    }
+
+    // compute bounding box of triangle
+    int tMinX = StrictMath.min(StrictMath.min(aX, bX), cX);
+    int tMaxX = StrictMath.max(StrictMath.max(aX, bX), cX);
+    int tMinY = StrictMath.min(StrictMath.min(aY, bY), cY);
+    int tMaxY = StrictMath.max(StrictMath.max(aY, bY), cY);
+
+    // 2. check bounding boxes are disjoint
+    if (this.crossesDateline() == true) {
+      if (boxesAreDisjoint(tMinX, tMaxX, tMinY, tMaxY, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
+          && boxesAreDisjoint(tMinX, tMaxX, tMinY, tMaxY, this.minX, MAX_LON_ENCODED, this.minY, this.maxY)) {
+        return false;
+      }
+    } else if (tMaxX < minX || tMinX > maxX || tMinY > maxY || tMaxY < minY) {
+      return false;
+    }
+
+    // 3. check triangle contains any query points
+    if (Tessellator.pointInTriangle(minX, minY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    } else if (Tessellator.pointInTriangle(maxX, minY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    } else if (Tessellator.pointInTriangle(maxX, maxY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    } else if (Tessellator.pointInTriangle(minX, maxY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    }
+
+    // 4. last ditch effort: check crossings
+    if (queryIntersects(aX, aY, bX, bY, cX, cY)) {
+      return true;
+    }
+    return false;
+  }
+
+  /** Checks if the rectangle contains the provided triangle **/
+  public boolean containsTriangle(int ax, int ay, int bx, int by, int cx, int cy) {
+    if (this.crossesDateline() == true) {
+      return bboxContainsTriangle(ax, ay, bx, by, cx, cy, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
+          || bboxContainsTriangle(ax, ay, bx, by, cx, cy, this.minX, MAX_LON_ENCODED, this.minY, this.maxY);
+    }
+    return bboxContainsTriangle(ax, ay, bx, by, cx, cy, minX, maxX, minY, maxY);
+  }
+
+  /** static utility method to compare a bbox with a range of triangles (just the bbox of the triangle collection) */
+  private static PointValues.Relation compareBBoxToRangeBBox(final byte[] bbox,
+                                                            int minXOffset, int minYOffset, byte[] minTriangle,
+                                                            int maxXOffset, int maxYOffset, byte[] maxTriangle) {
+    // check bounding box (DISJOINT)
+    if (FutureArrays.compareUnsigned(minTriangle, minXOffset, minXOffset + BYTES, bbox, 3 * BYTES, 4 * BYTES) > 0 ||
+        FutureArrays.compareUnsigned(maxTriangle, maxXOffset, maxXOffset + BYTES, bbox, BYTES, 2 * BYTES) < 0 ||
+        FutureArrays.compareUnsigned(minTriangle, minYOffset, minYOffset + BYTES, bbox, 2 * BYTES, 3 * BYTES) > 0 ||
+        FutureArrays.compareUnsigned(maxTriangle, maxYOffset, maxYOffset + BYTES, bbox, 0, BYTES) < 0) {
+      return PointValues.Relation.CELL_OUTSIDE_QUERY;
+    }
+
+    if (FutureArrays.compareUnsigned(minTriangle, minXOffset, minXOffset + BYTES, bbox, BYTES, 2 * BYTES) >= 0 &&
+        FutureArrays.compareUnsigned(maxTriangle, maxXOffset, maxXOffset + BYTES, bbox, 3 * BYTES, 4 * BYTES) <= 0 &&
+        FutureArrays.compareUnsigned(minTriangle, minYOffset, minYOffset + BYTES, bbox, 0, BYTES) >= 0 &&
+        FutureArrays.compareUnsigned(maxTriangle, maxYOffset, maxYOffset + BYTES, bbox, 2 * BYTES, 3 * BYTES) <= 0) {
+      return PointValues.Relation.CELL_INSIDE_QUERY;
+    }
+    return PointValues.Relation.CELL_CROSSES_QUERY;
+  }
+
+  /**
+   * encodes a bounding box into the provided byte array
+   */
+  private static void encode(final int minX, final int maxX, final int minY, final int maxY, byte[] b) {
+    if (b == null) {
+      b = new byte[4 * LatLonShape.BYTES];
+    }
+    LatLonShape.encodeTriangleBoxVal(minY, b, 0);
+    LatLonShape.encodeTriangleBoxVal(minX, b, BYTES);
+    LatLonShape.encodeTriangleBoxVal(maxY, b, 2 * BYTES);
+    LatLonShape.encodeTriangleBoxVal(maxX, b, 3 * BYTES);
+  }
+
+  /** returns true if the query intersects the provided triangle (in encoded space) */
+  private boolean queryIntersects(int ax, int ay, int bx, int by, int cx, int cy) {
+    // check each edge of the triangle against the query
+    if (edgeIntersectsQuery(ax, ay, bx, by) ||
+        edgeIntersectsQuery(bx, by, cx, cy) ||
+        edgeIntersectsQuery(cx, cy, ax, ay)) {
+      return true;
+    }
+    return false;
+  }
+
+  /** returns true if the edge (defined by (ax, ay) (bx, by)) intersects the query */
+  private boolean edgeIntersectsQuery(int ax, int ay, int bx, int by) {
+    if (this.crossesDateline() == true) {
+      return edgeIntersectsBox(ax, ay, bx, by, MIN_LON_ENCODED, this.maxX, this.minY, this.maxY)
+          || edgeIntersectsBox(ax, ay, bx, by, this.minX, MAX_LON_ENCODED, this.minY, this.maxY);
+    }
+    return edgeIntersectsBox(ax, ay, bx, by, this.minX, this.maxX, this.minY, this.maxY);
+  }
+
+  /** static utility method to check if a bounding box contains a point */
+  private static boolean bboxContainsPoint(int x, int y, int minX, int maxX, int minY, int maxY) {
+    return (x < minX || x > maxX || y < minY || y > maxY) == false;
+  }
+
+  /** static utility method to check if a bounding box contains a triangle */
+  private static boolean bboxContainsTriangle(int ax, int ay, int bx, int by, int cx, int cy,
+                                             int minX, int maxX, int minY, int maxY) {
+    return bboxContainsPoint(ax, ay, minX, maxX, minY, maxY)
+        && bboxContainsPoint(bx, by, minX, maxX, minY, maxY)
+        && bboxContainsPoint(cx, cy, minX, maxX, minY, maxY);
+  }
+
+  /** returns true if the edge (defined by (ax, ay) (bx, by)) intersects the query */
+  private static boolean edgeIntersectsBox(int ax, int ay, int bx, int by,
+                                           int minX, int maxX, int minY, int maxY) {
+    // shortcut: if edge is a point (occurs w/ Line shapes); simply check bbox w/ point
+    if (ax == bx && ay == by) {
+      return Rectangle.containsPoint(ay, ax, minY, maxY, minX, maxX);
+    }
+
+    // shortcut: check if either of the end points fall inside the box
+    if (bboxContainsPoint(ax, ay, minX, maxX, minY, maxY)
+        || bboxContainsPoint(bx, by, minX, maxX, minY, maxY)) {
+      return true;
+    }
+
+    // shortcut: check bboxes of edges are disjoint
+    if (boxesAreDisjoint(Math.min(ax, bx), Math.max(ax, bx), Math.min(ay, by), Math.max(ay, by),
+        minX, maxX, minY, maxY)) {
+      return false;
+    }
+
+    // shortcut: edge is a point
+    if (ax == bx && ay == by) {
+      return false;
+    }
+
+    // top
+    if (orient(ax, ay, bx, by, minX, maxY) * orient(ax, ay, bx, by, maxX, maxY) <= 0 &&
+        orient(minX, maxY, maxX, maxY, ax, ay) * orient(minX, maxY, maxX, maxY, bx, by) <= 0) {
+      return true;
+    }
+
+    // right
+    if (orient(ax, ay, bx, by, maxX, maxY) * orient(ax, ay, bx, by, maxX, minY) <= 0 &&
+        orient(maxX, maxY, maxX, minY, ax, ay) * orient(maxX, maxY, maxX, minY, bx, by) <= 0) {
+      return true;
+    }
+
+    // bottom
+    if (orient(ax, ay, bx, by, maxX, minY) * orient(ax, ay, bx, by, minX, minY) <= 0 &&
+        orient(maxX, minY, minX, minY, ax, ay) * orient(maxX, minY, minX, minY, bx, by) <= 0) {
+      return true;
+    }
+
+    // left
+    if (orient(ax, ay, bx, by, minX, minY) * orient(ax, ay, bx, by, minX, maxY) <= 0 &&
+        orient(minX, minY, minX, maxY, ax, ay) * orient(minX, minY, minX, maxY, bx, by) <= 0) {
+      return true;
+    }
+    return false;
+  }
+
+  /** utility method to check if two boxes are disjoint */
+  private static boolean boxesAreDisjoint(final int aMinX, final int aMaxX, final int aMinY, final int aMaxY,
+                                         final int bMinX, final int bMaxX, final int bMinY, final int bMaxY) {
+    return (aMaxX < bMinX || aMinX > bMaxX || aMaxY < bMinY || aMinY > bMaxY);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return Arrays.equals(bbox, ((Rectangle2D)o).bbox)
+        && Arrays.equals(west, ((Rectangle2D)o).west);
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = super.hashCode();
+    hash = 31 * hash + Arrays.hashCode(bbox);
+    hash = 31 * hash + Arrays.hashCode(west);
+    return hash;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder();
+    sb.append("Rectangle(lat=");
+    sb.append(decodeLatitude(minY));
+    sb.append(" TO ");
+    sb.append(decodeLatitude(maxY));
+    sb.append(" lon=");
+    sb.append(decodeLongitude(minX));
+    sb.append(" TO ");
+    sb.append(decodeLongitude(maxX));
+    if (maxX < minX) {
+      sb.append(" [crosses dateline!]");
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce9a8012/lucene/sandbox/src/test/org/apache/lucene/geo/TestRectangle2D.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/geo/TestRectangle2D.java b/lucene/sandbox/src/test/org/apache/lucene/geo/TestRectangle2D.java
new file mode 100644
index 0000000..2714936
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/geo/TestRectangle2D.java
@@ -0,0 +1,100 @@
+/*
+ * 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.lucene.geo;
+
+import org.apache.lucene.document.LatLonShape;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.LuceneTestCase;
+
+import static org.apache.lucene.document.LatLonShape.BYTES;
+
+public class TestRectangle2D extends LuceneTestCase {
+
+  public void testTriangleDisjoint() {
+    Rectangle rectangle = new Rectangle(0, 1, 0, 1);
+    Rectangle2D rectangle2D = Rectangle2D.create(rectangle);
+    int ax = GeoEncodingUtils.encodeLongitude(4);
+    int ay = GeoEncodingUtils.encodeLatitude(4);
+    int bx = GeoEncodingUtils.encodeLongitude(5);
+    int by = GeoEncodingUtils.encodeLatitude(5);
+    int cx = GeoEncodingUtils.encodeLongitude(5);
+    int cy = GeoEncodingUtils.encodeLatitude(4);
+    assertFalse(rectangle2D.intersectsTriangle(ax, ay, bx, by , cx, cy));
+    assertFalse(rectangle2D.containsTriangle(ax, ay, bx, by , cx, cy));
+  }
+
+  public void testTriangleIntersects() {
+    Rectangle rectangle = new Rectangle(0, 1, 0, 1);
+    Rectangle2D rectangle2D =  Rectangle2D.create(rectangle);
+    int ax = GeoEncodingUtils.encodeLongitude(0.5);
+    int ay = GeoEncodingUtils.encodeLatitude(0.5);
+    int bx = GeoEncodingUtils.encodeLongitude(2);
+    int by = GeoEncodingUtils.encodeLatitude(2);
+    int cx = GeoEncodingUtils.encodeLongitude(0.5);
+    int cy = GeoEncodingUtils.encodeLatitude(2);
+    assertTrue(rectangle2D.intersectsTriangle(ax, ay, bx, by , cx, cy));
+    assertFalse(rectangle2D.containsTriangle(ax, ay, bx, by , cx, cy));
+  }
+
+  public void testTriangleContains() {
+    Rectangle rectangle = new Rectangle(0, 1, 0, 1);
+    Rectangle2D rectangle2D =  Rectangle2D.create(rectangle);
+    int ax = GeoEncodingUtils.encodeLongitude(0.25);
+    int ay = GeoEncodingUtils.encodeLatitude(0.25);
+    int bx = GeoEncodingUtils.encodeLongitude(0.5);
+    int by = GeoEncodingUtils.encodeLatitude(0.5);
+    int cx = GeoEncodingUtils.encodeLongitude(0.5);
+    int cy = GeoEncodingUtils.encodeLatitude(0.25);
+    assertTrue(rectangle2D.intersectsTriangle(ax, ay, bx, by , cx, cy));
+    assertTrue(rectangle2D.containsTriangle(ax, ay, bx, by , cx, cy));
+  }
+
+  public void testRandomTriangles() {
+    Rectangle rectangle = GeoTestUtil.nextBox();
+    Rectangle2D rectangle2D = Rectangle2D.create(rectangle);
+
+    for (int i =0; i < 100; i++) {
+      int ax = GeoEncodingUtils.encodeLongitude(GeoTestUtil.nextLongitude());
+      int ay = GeoEncodingUtils.encodeLatitude(GeoTestUtil.nextLatitude());
+      int bx = GeoEncodingUtils.encodeLongitude(GeoTestUtil.nextLongitude());
+      int by = GeoEncodingUtils.encodeLatitude(GeoTestUtil.nextLatitude());
+      int cx = GeoEncodingUtils.encodeLongitude(GeoTestUtil.nextLongitude());
+      int cy = GeoEncodingUtils.encodeLatitude(GeoTestUtil.nextLatitude());
+
+      int tMinX = StrictMath.min(StrictMath.min(ax, bx), cx);
+      int tMaxX = StrictMath.max(StrictMath.max(ax, bx), cx);
+      int tMinY = StrictMath.min(StrictMath.min(ay, by), cy);
+      int tMaxY = StrictMath.max(StrictMath.max(ay, by), cy);
+
+      byte[] triangle = new byte[4 * LatLonShape.BYTES];
+      LatLonShape.encodeTriangleBoxVal(tMinY, triangle, 0);
+      LatLonShape.encodeTriangleBoxVal(tMinX, triangle, BYTES);
+      LatLonShape.encodeTriangleBoxVal(tMaxY, triangle, 2 * BYTES);
+      LatLonShape.encodeTriangleBoxVal(tMaxX, triangle, 3 * BYTES);
+
+      PointValues.Relation r = rectangle2D.relateRangeBBox(LatLonShape.BYTES, 0, triangle, 3 * LatLonShape.BYTES, 2 * LatLonShape.BYTES, triangle);
+      if (r == PointValues.Relation.CELL_OUTSIDE_QUERY) {
+        assertFalse(rectangle2D.intersectsTriangle(ax, ay, bx, by , cx, cy));
+        assertFalse(rectangle2D.containsTriangle(ax, ay, bx, by , cx, cy));
+      }
+      else if (rectangle2D.containsTriangle(ax, ay, bx, by , cx, cy)) {
+        assertTrue(rectangle2D.intersectsTriangle(ax, ay, bx, by , cx, cy));
+      }
+    }
+  }
+}


[06/34] lucene-solr:jira/http2: SOLR-13057: Allow search, facet and timeseries Streaming Expressions to accept a comma delimited list of collections

Posted by da...@apache.org.
SOLR-13057: Allow search, facet and timeseries Streaming Expressions to accept a comma delimited list of collections


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

Branch: refs/heads/jira/http2
Commit: 7e4555a2fdb863d6aac2f785116f8f13e51bf16b
Parents: ce9a801
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Dec 12 09:15:41 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Dec 12 09:16:08 2018 -0500

----------------------------------------------------------------------
 .../client/solrj/io/stream/CloudSolrStream.java |  21 +++-
 .../client/solrj/io/stream/FacetStream.java     |  11 +-
 .../solrj/io/stream/SearchFacadeStream.java     |   6 ++
 .../client/solrj/io/stream/SearchStream.java    |   6 +-
 .../solrj/io/stream/TimeSeriesStream.java       |  11 +-
 .../solrj/io/stream/StreamExpressionTest.java   | 108 +++++++++++++++++++
 6 files changed, 155 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e4555a2/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
index 2cff0a7..f871473 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
@@ -170,7 +170,11 @@ public class CloudSolrStream extends TupleStream implements Expressible {
     StreamExpression expression = new StreamExpression("search");
     
     // collection
-    expression.addParameter(collection);
+    if(collection.indexOf(',') > -1) {
+      expression.addParameter("\""+collection+"\"");
+    } else {
+      expression.addParameter(collection);
+    }
     
     for (Entry<String, String[]> param : params.getMap().entrySet()) {
       for (String val : param.getValue()) {
@@ -334,11 +338,18 @@ public class CloudSolrStream extends TupleStream implements Expressible {
     //  which is something already supported in other parts of Solr
 
     // check for alias or collection
-    List<String> collections = checkAlias
-        ? zkStateReader.getAliases().resolveAliases(collectionName)  // if not an alias, returns collectionName
-        : Collections.singletonList(collectionName);
+
+    List<String> allCollections = new ArrayList();
+    String[] collectionNames = collectionName.split(",");
+    for(String col : collectionNames) {
+      List<String> collections = checkAlias
+          ? zkStateReader.getAliases().resolveAliases(col)  // if not an alias, returns collectionName
+          : Collections.singletonList(collectionName);
+      allCollections.addAll(collections);
+    }
+
     // Lookup all actives slices for these collections
-    List<Slice> slices = collections.stream()
+    List<Slice> slices = allCollections.stream()
         .map(collectionsMap::get)
         .filter(Objects::nonNull)
         .flatMap(docCol -> Arrays.stream(docCol.getActiveSlicesArr()))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e4555a2/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
index b84967c..38d0904 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
@@ -100,6 +100,11 @@ public class FacetStream extends TupleStream implements Expressible  {
   public FacetStream(StreamExpression expression, StreamFactory factory) throws IOException{   
     // grab all parameters out
     String collectionName = factory.getValueOperand(expression, 0);
+
+    if(collectionName.indexOf('"') > -1) {
+      collectionName = collectionName.replaceAll("\"", "").replaceAll(" ", "");
+    }
+
     List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
     StreamExpressionNamedParameter bucketExpression = factory.getNamedOperand(expression, "buckets");
     StreamExpressionNamedParameter bucketSortExpression = factory.getNamedOperand(expression, "bucketSorts");
@@ -378,7 +383,11 @@ public class FacetStream extends TupleStream implements Expressible  {
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
     
     // collection
-    expression.addParameter(collection);
+    if(collection.indexOf(',') > -1) {
+      expression.addParameter("\""+collection+"\"");
+    } else {
+      expression.addParameter(collection);
+    }
     
     // parameters
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e4555a2/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchFacadeStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchFacadeStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchFacadeStream.java
index 5e8b549..de2b20e 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchFacadeStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchFacadeStream.java
@@ -48,6 +48,12 @@ public class SearchFacadeStream extends TupleStream implements Expressible {
   public SearchFacadeStream(StreamExpression expression, StreamFactory factory) throws IOException{
     // grab all parameters out
     String collectionName = factory.getValueOperand(expression, 0);
+
+    //Handle comma delimited list of collections.
+    if(collectionName.indexOf('"') > -1) {
+      collectionName = collectionName.replaceAll("\"", "").replaceAll(" ", "");
+    }
+
     List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
     StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e4555a2/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchStream.java
index a4ed996..3643969 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SearchStream.java
@@ -128,7 +128,11 @@ public class SearchStream extends TupleStream implements Expressible  {
     StreamExpression expression = new StreamExpression("search");
 
     // collection
-    expression.addParameter(collection);
+    if(collection.indexOf(',') > -1) {
+      expression.addParameter("\""+collection+"\"");
+    } else {
+      expression.addParameter(collection);
+    }
 
     for (Entry<String, String[]> param : params.getMap().entrySet()) {
       for (String val : param.getValue()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e4555a2/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
index 13e72fa..764c70b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
@@ -89,6 +89,11 @@ public class TimeSeriesStream extends TupleStream implements Expressible  {
   public TimeSeriesStream(StreamExpression expression, StreamFactory factory) throws IOException{
     // grab all parameters out
     String collectionName = factory.getValueOperand(expression, 0);
+
+    if(collectionName.indexOf('"') > -1) {
+      collectionName = collectionName.replaceAll("\"", "").replaceAll(" ", "");
+    }
+
     List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
     StreamExpressionNamedParameter startExpression = factory.getNamedOperand(expression, "start");
     StreamExpressionNamedParameter endExpression = factory.getNamedOperand(expression, "end");
@@ -212,7 +217,11 @@ public class TimeSeriesStream extends TupleStream implements Expressible  {
     // function name
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
     // collection
-    expression.addParameter(collection);
+    if(collection.indexOf(',') > -1) {
+      expression.addParameter("\""+collection+"\"");
+    } else {
+      expression.addParameter(collection);
+    }
 
     // parameters
     ModifiableSolrParams tmpParams = new ModifiableSolrParams(params);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e4555a2/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 2725903..e271401 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -1400,6 +1400,114 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testMultiCollection() throws Exception {
+
+    CollectionAdminRequest.createCollection("collection2", "conf", 2, 1).process(cluster.getSolrClient());
+    cluster.waitForActiveCollection("collection2", 2, 2);
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "test_dt", getDateString("2016", "5", "1"), "i_multi", "4", "i_multi", "7")
+        .add(id, "2", "a_s", "hello", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "test_dt", getDateString("2016", "5", "1"), "i_multi", "44", "i_multi", "77")
+        .add(id, "3", "a_s", "hello", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "test_dt", getDateString("2016", "5", "1"), "i_multi", "444", "i_multi", "777")
+        .add(id, "4", "a_s", "hello", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "test_dt", getDateString("2016", "5", "1"), "i_multi", "4444", "i_multi", "7777")
+        .add(id, "1", "a_s", "hello", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "test_dt", getDateString("2016", "5", "1"), "i_multi", "44444", "i_multi", "77777")
+        .commit(cluster.getSolrClient(), "collection1");
+
+    new UpdateRequest()
+        .add(id, "10", "a_s", "hello", "a_i", "10", "a_f", "0", "s_multi", "aaaa", "test_dt", getDateString("2016", "5", "1"), "i_multi", "4", "i_multi", "7")
+        .add(id, "12", "a_s", "hello", "a_i", "12", "a_f", "0", "s_multi", "aaaa1", "test_dt", getDateString("2016", "5", "1"), "i_multi", "44", "i_multi", "77")
+        .add(id, "13", "a_s", "hello", "a_i", "13", "a_f", "3", "s_multi", "aaaa2", "test_dt", getDateString("2016", "5", "1"),  "i_multi", "444", "i_multi", "777")
+        .add(id, "14", "a_s", "hello", "a_i", "14", "a_f", "4", "s_multi", "aaaa3", "test_dt", getDateString("2016", "5", "1"), "i_multi", "4444", "i_multi", "7777")
+        .add(id, "11", "a_s", "hello", "a_i", "11", "a_f", "1", "s_multi", "aaaa4", "test_dt", getDateString("2016", "5", "1"), "i_multi", "44444", "i_multi", "77777")
+        .commit(cluster.getSolrClient(), "collection2");
+
+
+    List<Tuple> tuples;
+    StreamContext streamContext = new StreamContext();
+    SolrClientCache solrClientCache = new SolrClientCache();
+    streamContext.setSolrClientCache(solrClientCache);
+    List<String> shardUrls = TupleStream.getShards(cluster.getZkServer().getZkAddress(), COLLECTIONORALIAS, streamContext);
+
+    try {
+      StringBuilder buf = new StringBuilder();
+      for (String shardUrl : shardUrls) {
+        if (buf.length() > 0) {
+          buf.append(",");
+        }
+        buf.append(shardUrl);
+      }
+
+      ModifiableSolrParams solrParams = new ModifiableSolrParams();
+      solrParams.add("qt", "/stream");
+      solrParams.add("expr", "search(\"collection1, collection2\", q=\"*:*\", fl=\"id, a_i\", rows=50, sort=\"a_i asc\")");
+      SolrStream solrStream = new SolrStream(shardUrls.get(0), solrParams);
+      solrStream.setStreamContext(streamContext);
+      tuples = getTuples(solrStream);
+      assert (tuples.size() == 10);
+      assertOrder(tuples, 0, 1, 2, 3, 4,10,11,12,13,14);
+
+      //Test with export handler, different code path.
+
+      solrParams = new ModifiableSolrParams();
+      solrParams.add("qt", "/stream");
+      solrParams.add("expr", "search(\"collection1, collection2\", q=\"*:*\", fl=\"id, a_i\", sort=\"a_i asc\", qt=\"/export\")");
+      solrStream = new SolrStream(shardUrls.get(0), solrParams);
+      solrStream.setStreamContext(streamContext);
+      tuples = getTuples(solrStream);
+      assert (tuples.size() == 10);
+      assertOrder(tuples, 0, 1, 2, 3, 4,10,11,12,13,14);
+
+
+      solrParams = new ModifiableSolrParams();
+      solrParams.add("qt", "/stream");
+      solrParams.add("expr", "facet(\"collection1, collection2\", q=\"*:*\", buckets=\"a_s\", bucketSorts=\"count(*) asc\", count(*))");
+      solrStream = new SolrStream(shardUrls.get(0), solrParams);
+      solrStream.setStreamContext(streamContext);
+      tuples = getTuples(solrStream);
+      assert (tuples.size() == 1);
+      Tuple tuple = tuples.get(0);
+      assertEquals(tuple.getString("a_s"), "hello");
+      assertEquals(tuple.getLong("count(*)").longValue(), 10);
+
+      String expr = "timeseries(\"collection1, collection2\", q=\"*:*\", " +
+              "start=\"2016-01-01T01:00:00.000Z\", " +
+              "end=\"2016-12-01T01:00:00.000Z\", " +
+              "gap=\"+1YEAR\", " +
+              "field=\"test_dt\", " +
+              "format=\"yyyy\","+
+              "count(*))";
+
+      solrParams = new ModifiableSolrParams();
+      solrParams.add("qt", "/stream");
+      solrParams.add("expr", expr);
+      solrStream = new SolrStream(shardUrls.get(0), solrParams);
+      solrStream.setStreamContext(streamContext);
+      tuples = getTuples(solrStream);
+      assert (tuples.size() == 1);
+      tuple = tuples.get(0);
+      assertEquals(tuple.getString("test_dt"), "2016");
+      assertEquals(tuple.getLong("count(*)").longValue(), 10);
+
+      //Test parallel
+
+      solrParams = new ModifiableSolrParams();
+      solrParams.add("qt", "/stream");
+      solrParams.add("expr", "parallel(collection1, sort=\"a_i asc\", workers=2, search(\"collection1, collection2\", q=\"*:*\", fl=\"id, a_i\", sort=\"a_i asc\", qt=\"/export\", partitionKeys=\"a_s\"))");
+      solrStream = new SolrStream(shardUrls.get(0), solrParams);
+      solrStream.setStreamContext(streamContext);
+      tuples = getTuples(solrStream);
+      assert (tuples.size() == 10);
+      assertOrder(tuples, 0, 1, 2, 3, 4,10,11,12,13,14);
+
+    } finally {
+      CollectionAdminRequest.deleteCollection("collection2").process(cluster.getSolrClient());
+      solrClientCache.close();
+    }
+
+
+  }
+
+  @Test
   public void testSubFacetStream() throws Exception {
 
     new UpdateRequest()


[21/34] lucene-solr:jira/http2: SOLR-7896: Followup fix to non-working core dropdown

Posted by da...@apache.org.
SOLR-7896: Followup fix to non-working core dropdown


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

Branch: refs/heads/jira/http2
Commit: 04e05782a37004a501a2c84477d1121b75cace03
Parents: 774e9ae
Author: Jan Høydahl <ja...@apache.org>
Authored: Fri Dec 14 14:40:02 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Fri Dec 14 14:40:02 2018 +0100

----------------------------------------------------------------------
 solr/webapp/web/js/angular/app.js | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/04e05782/solr/webapp/web/js/angular/app.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/app.js b/solr/webapp/web/js/angular/app.js
index cb04ba3..eb442d8 100644
--- a/solr/webapp/web/js/angular/app.js
+++ b/solr/webapp/web/js/angular/app.js
@@ -76,7 +76,7 @@ solrAdminApp.config([
         templateUrl: 'partials/cluster_suggestions.html',
         controller: 'ClusterSuggestionsController'
       }).
-      when('/:core', {
+      when('/:core/core-overview', {
         templateUrl: 'partials/core_overview.html',
         controller: 'CoreOverviewController'
       }).


[30/34] lucene-solr:jira/http2: SOLR-13065: Harden TestSimExecutePlanAction

Posted by da...@apache.org.
SOLR-13065: Harden TestSimExecutePlanAction


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

Branch: refs/heads/jira/http2
Commit: 1d601ed6460c9041faabeec7f5491ef21a052a00
Parents: 655e9ec
Author: Jason Gerlowski <ge...@apache.org>
Authored: Fri Dec 14 11:09:43 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Fri Dec 14 12:22:09 2018 -0500

----------------------------------------------------------------------
 .../sim/SimClusterStateProvider.java            | 35 ++++++++++++++------
 .../sim/TestSimExecutePlanAction.java           |  2 --
 2 files changed, 25 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d601ed6/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index c411b21..fd6c955 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -411,16 +411,16 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     lock.lockInterruptibly();
     try {
       setReplicaStates(nodeId, Replica.State.ACTIVE, collections);
+      if (!collections.isEmpty()) {
+        collectionsStatesRef.set(null);
+        simRunLeaderElection(collections, true);
+        return true;
+      } else {
+        return false;
+      }
     } finally {
       lock.unlock();
     }
-    if (!collections.isEmpty()) {
-      collectionsStatesRef.set(null);
-      simRunLeaderElection(collections, true);
-      return true;
-    } else {
-      return false;
-    }
   }
 
   /**
@@ -657,7 +657,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   private void simRunLeaderElection(Collection<String> collections, boolean saveClusterState) throws Exception {
     ensureNotClosed();
     if (saveClusterState) {
-      collectionsStatesRef.set(null);
+      lock.lockInterruptibly();
+      try {
+        collectionsStatesRef.set(null);
+      } finally {
+        lock.unlock();
+      }
     }
     ClusterState state = getClusterState();
     state.forEachCollection(dc -> {
@@ -798,7 +803,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     CreateCollectionCmd.checkReplicaTypes(props);
 
     // always force getting fresh state
-    collectionsStatesRef.set(null);
+    lock.lockInterruptibly();
+    try {
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
     final ClusterState clusterState = getClusterState();
 
     String withCollection = props.getStr(CollectionAdminParams.WITH_COLLECTION);
@@ -928,7 +938,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     });
 
     // force recreation of collection states
-    collectionsStatesRef.set(null);
+    lock.lockInterruptibly();
+    try {
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
     //simRunLeaderElection(Collections.singleton(collectionName), true);
     if (waitForFinalState) {
       boolean finished = finalStateLatch.await(cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, 60, TimeUnit.MILLISECONDS),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d601ed6/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
index 1d36f14..8f95288 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
@@ -83,7 +83,6 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
 
   @Test
   @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 28-June-2018
-  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   public void testExecute() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
     String collectionName = "testExecute";
@@ -159,7 +158,6 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
   }
 
   @Test
-  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // this test can fail to elect a leader, seems to be common among sim tests
   public void testIntegration() throws Exception  {
     SolrClient solrClient = cluster.simGetSolrClient();
 


[31/34] lucene-solr:jira/http2: update project DOAP files to the latest 7.6.0 release

Posted by da...@apache.org.
update project DOAP files to the latest 7.6.0 release


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

Branch: refs/heads/jira/http2
Commit: eb4fda65e55898350e09ac5fd5fcdb9146074d96
Parents: 1d601ed
Author: Nicholas Knize <nk...@gmail.com>
Authored: Fri Dec 14 11:43:39 2018 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Fri Dec 14 11:44:22 2018 -0600

----------------------------------------------------------------------
 dev-tools/doap/lucene.rdf | 7 +++++++
 dev-tools/doap/solr.rdf   | 7 +++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb4fda65/dev-tools/doap/lucene.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/lucene.rdf b/dev-tools/doap/lucene.rdf
index 6b377b0..ac98f9d 100644
--- a/dev-tools/doap/lucene.rdf
+++ b/dev-tools/doap/lucene.rdf
@@ -69,6 +69,13 @@
     <!-- NOTE: please insert releases in numeric order, NOT chronologically. -->
     <release>
       <Version>
+        <name>lucene-7.6.0</name>
+        <created>2018-12-14</created>
+        <revision>7.6.0</revision>
+      </Version>
+    </release>
+    <release>
+      <Version>
         <name>lucene-7.5.0</name>
         <created>2018-09-24</created>
         <revision>7.5.0</revision>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb4fda65/dev-tools/doap/solr.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/solr.rdf b/dev-tools/doap/solr.rdf
index 92484bd..cc6be54 100644
--- a/dev-tools/doap/solr.rdf
+++ b/dev-tools/doap/solr.rdf
@@ -69,6 +69,13 @@
     <!-- NOTE: please insert releases in numeric order, NOT chronologically. -->
     <release>
       <Version>
+        <name>solr-7.6.0</name>
+        <created>2018-12-14</created>
+        <revision>7.6.0</revision>
+      </Version>
+    </release>
+    <release>
+      <Version>
         <name>solr-7.5.0</name>
         <created>2018-09-24</created>
         <revision>7.5.0</revision>


[28/34] lucene-solr:jira/http2: Indicate collection in `clusterShape` log messages

Posted by da...@apache.org.
Indicate collection in `clusterShape` log messages

Many of Solr's tests use CloudTestUtils' `waitForState` and
`clusterShape` methods to wait until a SolrCloud cluster matches a
particular expected shape.  The code periodically checks on the cluster
state, and logs a message if a collection doesn't match the state we
expect of it.  Prior to this commit, these log messages omitted the
collection name though, which makes things a little confusing when a
test is checking on the state of multiple collections simultaneously (as
can happen when Triggers fire in the background).


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

Branch: refs/heads/jira/http2
Commit: dd910d1a06455032a9a18e5f0de3ca45e065ceef
Parents: 3ca7e65
Author: Jason Gerlowski <ge...@apache.org>
Authored: Fri Dec 14 10:57:20 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Fri Dec 14 12:21:19 2018 -0500

----------------------------------------------------------------------
 solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dd910d1a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
index a1fccd2..bb74967 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
@@ -141,7 +141,7 @@ public class CloudTestUtils {
       }
       Collection<Slice> slices = withInactive ? collectionState.getSlices() : collectionState.getActiveSlices();
       if (slices.size() != expectedShards) {
-        log.info("-- wrong number of slices, expected={}, found={}: {}", expectedShards, collectionState.getSlices().size(), collectionState.getSlices());
+        log.info("-- wrong number of slices for collection {}, expected={}, found={}: {}", collectionState.getName(), expectedShards, collectionState.getSlices().size(), collectionState.getSlices());
         return false;
       }
       Set<String> leaderless = new HashSet<>();
@@ -160,7 +160,7 @@ public class CloudTestUtils {
             activeReplicas++;
         }
         if (activeReplicas != expectedReplicas) {
-          log.info("-- wrong number of active replicas in slice {}, expected={}, found={}", slice.getName(), expectedReplicas, activeReplicas);
+          log.info("-- wrong number of active replicas for collection {} in slice {}, expected={}, found={}", collectionState.getName(), slice.getName(), expectedReplicas, activeReplicas);
           return false;
         }
       }


[04/34] lucene-solr:jira/http2: Fix test failure when floorSegmentMB > maxMergedSegmentMB.

Posted by da...@apache.org.
Fix test failure when floorSegmentMB > maxMergedSegmentMB.


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

Branch: refs/heads/jira/http2
Commit: 55993ecb9b06d9c0d2bd54a752a08f19698a13bc
Parents: 21dc237
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Dec 12 13:21:06 2018 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 12 13:21:06 2018 +0100

----------------------------------------------------------------------
 .../src/test/org/apache/lucene/index/TestTieredMergePolicy.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/55993ecb/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
index b4a7ff5..bbc8051 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
@@ -169,7 +169,8 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
       }
       w.forceMerge(targetCount);
 
-      final long max125Pct = (long) ((tmp.getMaxMergedSegmentMB() * 1024.0 * 1024.0) * 1.25);
+      final double maxSegmentSize = Math.max(tmp.getMaxMergedSegmentMB(), tmp.getFloorSegmentMB());
+      final long max125Pct = (long) ((maxSegmentSize * 1024.0 * 1024.0) * 1.25);
       // Other than in the case where the target count is 1 we can't say much except no segment should be > 125% of max seg size.
       if (targetCount == 1) {
         assertEquals("Should have merged down to one segment", targetCount, w.getSegmentCount());


[22/34] lucene-solr:jira/http2: SOLR-11296: Spellcheck parameters not working in new UI

Posted by da...@apache.org.
SOLR-11296: Spellcheck parameters not working in new UI


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

Branch: refs/heads/jira/http2
Commit: 62edbf2e7da7af6d96d6c912379ee63c80459a70
Parents: 04e0578
Author: Jan Høydahl <ja...@apache.org>
Authored: Fri Dec 14 15:07:28 2018 +0100
Committer: Jan Høydahl <ja...@apache.org>
Committed: Fri Dec 14 15:07:28 2018 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                | 2 ++
 solr/webapp/web/js/angular/controllers/query.js | 3 ++-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62edbf2e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 28baf4d..b4bdf37 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -175,6 +175,8 @@ Bug Fixes
 
 * SOLR-13066: A failure while reloading a SolrCore can result in the SolrCore not being closed. (Mark Miller)
 
+* SOLR-11296: Spellcheck parameters not working in new UI (Matt Pearce via janhoy)
+
 Improvements
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62edbf2e/solr/webapp/web/js/angular/controllers/query.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/controllers/query.js b/solr/webapp/web/js/angular/controllers/query.js
index 63f0830..7267b00 100644
--- a/solr/webapp/web/js/angular/controllers/query.js
+++ b/solr/webapp/web/js/angular/controllers/query.js
@@ -43,7 +43,8 @@ solrAdminApp.controller('QueryController',
       var copy = function(params, query) {
         for (var key in query) {
           terms = query[key];
-          if (terms.length > 0 && key[0]!="$") {
+          // Booleans have no length property - only set them if true
+          if (((typeof(terms) == typeof(true) && terms) || terms.length > 0) && key[0]!="$") {
             set(key, terms);
           }
         }


[17/34] lucene-solr:jira/http2: SOLR-13072: disable flawed test of flawed functionality

Posted by da...@apache.org.
SOLR-13072: disable flawed test of flawed functionality


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

Branch: refs/heads/jira/http2
Commit: f844461357d43838da51697295a1dcbb69699d9c
Parents: 46516b7
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Dec 13 15:32:09 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Dec 13 15:32:09 2018 -0700

----------------------------------------------------------------------
 .../solr/cloud/autoscaling/NodeMarkersRegistrationTest.java       | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8444613/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
index 7a8fa53..fd7a1fa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
@@ -30,6 +30,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
+
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
@@ -51,6 +53,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG")
 public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());


[18/34] lucene-solr:jira/http2: SOLR-13043: cleanup suspicious ExecutorService lifecycles in MiniSolrCloudCluster

Posted by da...@apache.org.
SOLR-13043: cleanup suspicious ExecutorService lifecycles in MiniSolrCloudCluster


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

Branch: refs/heads/jira/http2
Commit: df07fb33bb2ea0ece6ad9229a50c9661d710fdbe
Parents: f844461
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Dec 13 19:59:35 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Dec 13 19:59:35 2018 -0700

----------------------------------------------------------------------
 .../apache/solr/cloud/MiniSolrCloudCluster.java  | 19 ++++---------------
 1 file changed, 4 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/df07fb33/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index 9b52b80..fd719ed 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -122,9 +122,6 @@ public class MiniSolrCloudCluster {
   private final CloudSolrClient solrClient;
   private final JettyConfig jettyConfig;
 
-  private final ExecutorService executorLauncher = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-launcher"));
-  private final ExecutorService executorCloser = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-closer"));
-  
   private final AtomicInteger nodeIds = new AtomicInteger();
 
   /**
@@ -272,7 +269,9 @@ public class MiniSolrCloudCluster {
       startups.add(() -> startJettySolrRunner(newNodeName(), jettyConfig.context, jettyConfig));
     }
 
+    final ExecutorService executorLauncher = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-launcher"));
     Collection<Future<JettySolrRunner>> futures = executorLauncher.invokeAll(startups);
+    ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
     Exception startupError = checkForExceptions("Error starting up MiniSolrCloudCluster", futures);
     if (startupError != null) {
       try {
@@ -294,10 +293,6 @@ public class MiniSolrCloudCluster {
 
   private void waitForAllNodes(int numServers, int timeoutSeconds) throws IOException, InterruptedException, TimeoutException {
     
-    executorLauncher.shutdown();
-    
-    ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
-    
     int numRunning = 0;
     TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     
@@ -327,10 +322,6 @@ public class MiniSolrCloudCluster {
   public void waitForNode(JettySolrRunner jetty, int timeoutSeconds)
       throws IOException, InterruptedException, TimeoutException {
 
-    executorLauncher.shutdown();
-
-    ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
-
     ZkStateReader reader = getSolrClient().getZkStateReader();
 
     reader.waitForLiveNodes(30, TimeUnit.SECONDS, (o, n) -> n.contains(jetty.getNodeName()));
@@ -577,21 +568,19 @@ public class MiniSolrCloudCluster {
     try {
     
       IOUtils.closeQuietly(solrClient);
-      // accept no new tasks
-      executorLauncher.shutdown();
       List<Callable<JettySolrRunner>> shutdowns = new ArrayList<>(jettys.size());
       for (final JettySolrRunner jetty : jettys) {
         shutdowns.add(() -> stopJettySolrRunner(jetty));
       }
       jettys.clear();
+      final ExecutorService executorCloser = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-closer"));
       Collection<Future<JettySolrRunner>> futures = executorCloser.invokeAll(shutdowns);
+      ExecutorUtil.shutdownAndAwaitTermination(executorCloser);
       Exception shutdownError = checkForExceptions("Error shutting down MiniSolrCloudCluster", futures);
       if (shutdownError != null) {
         throw shutdownError;
       }
     } finally {
-      ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
-      ExecutorUtil.shutdownAndAwaitTermination(executorCloser);
       try {
         if (!externalZkServer) {
           zkServer.shutdown();


[10/34] lucene-solr:jira/http2: SOLR-12801: Harden SimSolrCloudTests.

Posted by da...@apache.org.
SOLR-12801: Harden SimSolrCloudTests.


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

Branch: refs/heads/jira/http2
Commit: 42f13731b3a037ee9682df49bb946ca0b4ca8544
Parents: 44b51cd
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 12 18:21:06 2018 -0600
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 12 18:50:23 2018 -0600

----------------------------------------------------------------------
 .../autoscaling/sim/SimSolrCloudTestCase.java   | 39 --------------------
 .../sim/TestSimComputePlanAction.java           | 10 +----
 .../sim/TestSimExecutePlanAction.java           | 13 ++++---
 .../autoscaling/sim/TestSimExtremeIndexing.java | 18 ++++++---
 .../autoscaling/sim/TestSimLargeCluster.java    | 23 ++++++------
 .../sim/TestSimNodeAddedTrigger.java            | 16 ++++----
 .../autoscaling/sim/TestSimNodeLostTrigger.java | 15 ++++----
 .../autoscaling/sim/TestSimPolicyCloud.java     | 16 +++++---
 .../sim/TestSimTriggerIntegration.java          | 24 +++---------
 9 files changed, 66 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
index 69954cd..611a46f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
@@ -16,13 +16,10 @@
  */
 package org.apache.solr.cloud.autoscaling.sim;
 
-import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
-
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.TimeUnit;
@@ -34,14 +31,10 @@ import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -88,38 +81,6 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
     super.setUp();
   }
 
-  @Before
-  public void checkClusterConfiguration() throws Exception {
-    if (cluster == null)
-      throw new RuntimeException("SimCloudManager not configured - have you called configureCluster()?");
-    // clear any persisted configuration
-    cluster.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), -1);
-    cluster.getDistribStateManager().setData(ZkStateReader.ROLES, Utils.toJSON(new HashMap<>()), -1);
-    cluster.getSimNodeStateProvider().simRemoveDeadNodes();
-    cluster.getSimClusterStateProvider().simRemoveDeadNodes();
-    // restore the expected number of nodes
-    int currentSize = cluster.getLiveNodesSet().size();
-    if (currentSize < clusterNodeCount) {
-      int addCnt = clusterNodeCount - currentSize;
-      while (addCnt-- > 0) {
-        cluster.simAddNode();
-      }
-    } else if (currentSize > clusterNodeCount) {
-      cluster.simRemoveRandomNodes(currentSize - clusterNodeCount, true, random());
-    }
-    // clean any persisted trigger state or events
-    removeChildren(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
-    removeChildren(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
-    removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
-    removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
-    cluster.getSimClusterStateProvider().simResetLeaderThrottles();
-    cluster.simRestartOverseer(null);
-    cluster.getSimClusterStateProvider().simDeleteAllCollections();
-    cluster.simClearSystemCollection();
-    cluster.getTimeSource().sleep(10000);
-    cluster.simResetOpCounts();
-  }
-
   protected void removeChildren(String path) throws Exception {
     
     TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
index 255f800..efaa558 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
@@ -52,7 +52,6 @@ import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,14 +72,9 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
   private static final AtomicReference<Map> actionContextPropsRef = new AtomicReference<>();
   private static final AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(1, TimeSource.get("simTime:50"));
-  }
-
   @Before
   public void init() throws Exception {
-
+    configureCluster(1, TimeSource.get("simTime:50"));
     fired.set(false);
     triggerFiredLatch = new CountDownLatch(1);
     actionContextPropsRef.set(null);
@@ -118,7 +112,7 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
     for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
       log.info("* Collection " + coll + " state: " + state.getCollection(coll));
     }
-
+    shutdownCluster();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
index a9c84be..07b02c9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
@@ -26,7 +26,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
-import com.google.common.collect.Lists;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -44,15 +43,17 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
-import org.apache.solr.common.util.TimeSource;
 import org.junit.After;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+
 /**
  * Test for {@link ExecutePlanAction}
  */
@@ -62,8 +63,8 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
 
   private static final int NODE_COUNT = 2;
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  @Before
+  public void setupCluster() throws Exception {
     configureCluster(NODE_COUNT, TimeSource.get("simTime:50"));
   }
 
@@ -76,7 +77,7 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
     for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
       log.info("* Collection " + coll + " state: " + state.getCollection(coll));
     }
-
+    shutdownCluster();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
index 15d676b..c648e54 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
@@ -16,13 +16,13 @@
  */
 package org.apache.solr.cloud.autoscaling.sim;
 
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
 import java.lang.invoke.MethodHandles;
 import java.util.Iterator;
 import java.util.Locale;
 import java.util.concurrent.TimeUnit;
 
-import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
-
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -37,13 +37,14 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.LogLevel;
+import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
 /**
  *
@@ -75,8 +76,8 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
   private static TimeSource timeSource;
   private static SolrClient solrClient;
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  @Before
+  public void setupCluster() throws Exception {
     configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
     timeSource = cluster.getTimeSource();
     solrClient = cluster.simGetSolrClient();
@@ -87,6 +88,11 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
   public static void tearDownCluster() throws Exception {
     solrClient = null;
   }
+  
+  @After
+  public void afterTest() throws Exception {
+    shutdownCluster();
+  }
 
   @Test
   public void testScaleUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
index c8527e1..8298831 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
@@ -62,7 +62,6 @@ import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -74,7 +73,7 @@ import org.slf4j.LoggerFactory;
 public class TestSimLargeCluster extends SimSolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final int SPEED = 50;
+  public static final int SPEED = 100;
 
   public static final int NUM_NODES = 100;
  
@@ -85,19 +84,15 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
   static CountDownLatch triggerFinishedLatch;
   static int waitForSeconds;
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
-  }
-
   @After
   public void tearDownTest() throws Exception {
     shutdownCluster();
-    configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
   }
   
   @Before
   public void setupTest() throws Exception {
+    configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
+    
     waitForSeconds = 5;
     triggerStartedCount.set(0);
     triggerFinishedCount.set(0);
@@ -351,11 +346,16 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
         CloudTestUtils.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
 
-    int count = 50;
+    int count = 1000;
     SolrInputDocument finishedEvent = null;
     long lastNumOps = cluster.simGetOpCount("MOVEREPLICA");
     while (count-- > 0) {
       cluster.getTimeSource().sleep(10000);
+      
+      if (cluster.simGetOpCount("MOVEREPLICA") < 2) {
+        continue;
+      }
+      
       long currentNumOps = cluster.simGetOpCount("MOVEREPLICA");
       if (currentNumOps == lastNumOps) {
         int size = systemColl.size() - 1;
@@ -432,7 +432,6 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
           if (cluster != null) {
             cluster.close();
           }
-          setupCluster();
           setUp();
           setupTest();
           long total = doTestNodeLost(wait, delay * 1000, 0);
@@ -699,9 +698,9 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     cluster.getTimeSource().sleep(25000);
     
-    assertNotNull(listenerEvents.entrySet().toString(), listenerEvents.get("srt"));
+    assertNotNull(listenerEvents.entrySet().iterator().toString(), listenerEvents.get("srt"));
 
-    assertTrue(listenerEvents.entrySet().toString(), listenerEvents.get("srt").size() >= 1);
+    assertTrue(listenerEvents.entrySet().iterator().toString(), listenerEvents.get("srt").size() >= 1);
 
     CapturedEvent ev = listenerEvents.get("srt").get(0);
     assertEquals(TriggerEventType.SEARCHRATE, ev.event.getEventType());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeAddedTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeAddedTrigger.java
index c9e506c..928046a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeAddedTrigger.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeAddedTrigger.java
@@ -36,8 +36,8 @@ import org.apache.solr.cloud.autoscaling.TriggerEvent;
 import org.apache.solr.cloud.autoscaling.TriggerValidationException;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrResourceLoader;
+import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -60,18 +60,20 @@ public class TestSimNodeAddedTrigger extends SimSolrCloudTestCase {
 
   private static TimeSource timeSource;
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(1, TimeSource.get("simTime:" + SPEED));
-    timeSource = cluster.getTimeSource();
-  }
-
   @Before
   public void beforeTest() throws Exception {
+    configureCluster(1, TimeSource.get("simTime:" + SPEED));
+    timeSource = cluster.getTimeSource();
+    
     actionConstructorCalled = new AtomicBoolean(false);
     actionInitCalled = new AtomicBoolean(false);
     actionCloseCalled = new AtomicBoolean(false);
   }
+  
+  @After
+  public void afterTest() throws Exception {
+    shutdownCluster();
+  }
 
   @Test
   public void testTrigger() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
index 0a9bd19..8eb6156 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
@@ -37,8 +37,8 @@ import org.apache.solr.cloud.autoscaling.TriggerEvent;
 import org.apache.solr.cloud.autoscaling.TriggerValidationException;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrResourceLoader;
+import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -60,18 +60,19 @@ public class TestSimNodeLostTrigger extends SimSolrCloudTestCase {
   // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
   private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(5, TimeSource.get("simTime:" + SPEED));
-    timeSource = cluster.getTimeSource();
-  }
-
   @Before
   public void beforeTest() throws Exception {
+    configureCluster(5, TimeSource.get("simTime:" + SPEED));
+    timeSource = cluster.getTimeSource();
     actionConstructorCalled = new AtomicBoolean(false);
     actionInitCalled = new AtomicBoolean(false);
     actionCloseCalled = new AtomicBoolean(false);
   }
+  
+  @After
+  public void afterTest() throws Exception {
+    shutdownCluster();
+  }
 
   @Test
   public void testTrigger() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
index eb251010..c8be232 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.cloud.autoscaling.sim;
 
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -47,13 +49,12 @@ import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
 import org.apache.zookeeper.KeeperException;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.rules.ExpectedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
-
 @LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
 public class TestSimPolicyCloud extends SimSolrCloudTestCase {
 
@@ -61,10 +62,15 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
   @org.junit.Rule
   public ExpectedException expectedException = ExpectedException.none();
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  @Before
+  public void setupCluster() throws Exception {
     configureCluster(5, TimeSource.get("simTime:50"));
   }
+  
+  @After
+  public void afterTest() throws Exception {
+    shutdownCluster();
+  }
 
   public void testDataProviderPerReplicaDetails() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42f13731/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
index c8b6528..7e24f4f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
@@ -68,9 +68,8 @@ import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -103,14 +102,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
   private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
 
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(2, TimeSource.get("simTime:" + SPEED));
-  }
-  
-  @AfterClass
-  public static void teardownCluster() {
-    cluster.simClearSystemCollection();
+  @After
+  public void afterTest() throws Exception {
+    shutdownCluster();
   }
 
   private static CountDownLatch getTriggerFiredLatch() {
@@ -131,6 +125,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
 
   @Before
   public void setupTest() throws Exception {
+    configureCluster(2, TimeSource.get("simTime:" + SPEED));
+    
     // disable .scheduled_maintenance
     String suspendTriggerCommand = "{" +
         "'suspend-trigger' : {'name' : '.scheduled_maintenance'}" +
@@ -157,14 +153,6 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     triggerFinishedCount = new AtomicInteger();
     events.clear();
     listenerEvents.clear();
-    cluster.getLiveNodesSet().removeAllLiveNodesListeners();
-    while (cluster.getClusterStateProvider().getLiveNodes().size() < 2) {
-      // perhaps a test stopped a node but didn't start it back
-      // lets start a node
-      cluster.simAddNode();
-      cluster.getTimeSource().sleep(1000);
-    }
-    cluster.getTimeSource().sleep(10000);
   }
 
   @Test


[34/34] lucene-solr:jira/http2: Merge with master

Posted by da...@apache.org.
Merge with master


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

Branch: refs/heads/jira/http2
Commit: c0af83e45f7fd1a81de9aca059f924b0503f44c5
Parents: 9d421c6 5c5c42c
Author: Cao Manh Dat <da...@apache.org>
Authored: Sat Dec 15 10:03:31 2018 +0000
Committer: Cao Manh Dat <da...@apache.org>
Committed: Sat Dec 15 10:03:31 2018 +0000

----------------------------------------------------------------------
 dev-tools/doap/lucene.rdf                       |   7 +
 dev-tools/doap/solr.rdf                         |   7 +
 lucene/CHANGES.txt                              |  13 +
 lucene/MIGRATE.txt                              |   6 +
 .../miscellaneous/TestEmptyTokenStream.java     |   2 +-
 .../index/TestBackwardsCompatibility.java       |  15 +-
 .../org/apache/lucene/index/index.7.6.0-cfs.zip | Bin 0 -> 15655 bytes
 .../apache/lucene/index/index.7.6.0-nocfs.zip   | Bin 0 -> 15649 bytes
 .../org/apache/lucene/index/sorted.7.6.0.zip    | Bin 0 -> 74541 bytes
 .../apache/lucene/index/BufferedUpdates.java    |   6 +-
 .../lucene/index/FreqProxTermsWriter.java       |  41 +--
 .../lucene/index/FrozenBufferedUpdates.java     | 216 +++++++------
 .../org/apache/lucene/index/IndexWriter.java    |  65 ++--
 .../apache/lucene/search/MatchAllDocsQuery.java |   4 +-
 .../org/apache/lucene/util/BytesRefArray.java   |   3 +-
 .../apache/lucene/util/BytesRefIterator.java    |  10 +-
 .../perfield/TestPerFieldPostingsFormat2.java   |  12 +-
 .../org/apache/lucene/index/TestAddIndexes.java |  62 ++--
 .../index/TestConcurrentMergeScheduler.java     |   2 +-
 .../test/org/apache/lucene/index/TestCrash.java |   2 +-
 .../apache/lucene/index/TestDeletionPolicy.java |   8 +-
 .../index/TestFlushByRamOrCountsPolicy.java     |  16 +-
 .../lucene/index/TestFrozenBufferedUpdates.java | 104 ++++++
 .../lucene/index/TestIndexManyDocuments.java    |   2 +-
 .../apache/lucene/index/TestIndexWriter.java    |  45 ++-
 .../lucene/index/TestIndexWriterCommit.java     |   2 +-
 .../lucene/index/TestIndexWriterDelete.java     |   2 +-
 .../lucene/index/TestIndexWriterFromReader.java |  20 +-
 .../lucene/index/TestIndexWriterMaxDocs.java    |   6 +-
 .../index/TestIndexWriterMergePolicy.java       |   2 +-
 .../lucene/index/TestIndexWriterMerging.java    |   8 +-
 .../lucene/index/TestIndexWriterReader.java     |   4 +-
 .../org/apache/lucene/index/TestIsCurrent.java  |   8 +-
 .../apache/lucene/index/TestRollingUpdates.java |   2 +-
 .../TestSoftDeletesRetentionMergePolicy.java    |  20 +-
 .../lucene/index/TestThreadedForceMerge.java    |   4 +-
 .../lucene/index/TestTieredMergePolicy.java     |  31 +-
 .../org/apache/lucene/search/TestBoolean2.java  |   2 +-
 .../lucene/search/TestMatchAllDocsQuery.java    |  23 ++
 .../lucene/search/TestSearcherManager.java      |   2 +-
 .../lucene/store/TestByteBuffersDirectory.java  |   2 +-
 .../apache/lucene/store/TestRAMDirectory.java   |   2 +-
 .../directory/DirectoryTaxonomyWriter.java      |   4 +-
 .../lucene/queries/mlt/TestMoreLikeThis.java    |   2 +-
 .../document/LatLonShapeBoundingBoxQuery.java   | 269 +---------------
 .../java/org/apache/lucene/geo/Rectangle2D.java | 315 +++++++++++++++++++
 .../org/apache/lucene/geo/TestRectangle2D.java  | 100 ++++++
 .../apache/lucene/index/RandomIndexWriter.java  |   8 +-
 .../ThreadedIndexingAndSearchingTestCase.java   |   2 +-
 solr/CHANGES.txt                                |  14 +-
 .../org/apache/solr/core/CoreContainer.java     |  50 ++-
 .../src/java/org/apache/solr/core/SolrCore.java |  98 +++---
 .../java/org/apache/solr/core/SolrInfoBean.java |   2 +-
 .../handler/component/HttpShardHandler.java     |   3 +
 .../solr/security/AuthenticationPlugin.java     | 129 +++++++-
 .../apache/solr/security/BasicAuthPlugin.java   | 180 +++++++++--
 .../apache/solr/security/HadoopAuthPlugin.java  |  18 ++
 .../solr/security/PKIAuthenticationPlugin.java  |  71 +++--
 .../apache/solr/servlet/SolrDispatchFilter.java |   4 +-
 .../apache/solr/update/SolrCmdDistributor.java  |   9 +-
 .../org/apache/solr/cloud/CloudTestUtils.java   |   4 +-
 .../apache/solr/cloud/HttpPartitionTest.java    |   2 +-
 .../apache/solr/cloud/TestCloudConsistency.java |   4 +-
 .../autoscaling/ExecutePlanActionTest.java      |   9 +-
 .../MetricTriggerIntegrationTest.java           |  10 +-
 .../NodeAddedTriggerIntegrationTest.java        |   2 +-
 .../cloud/autoscaling/NodeAddedTriggerTest.java |  18 +-
 .../NodeLostTriggerIntegrationTest.java         |   2 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |  18 +-
 .../NodeMarkersRegistrationTest.java            |   3 +
 .../autoscaling/RestoreTriggerStateTest.java    |   3 +-
 .../SearchRateTriggerIntegrationTest.java       |   3 +-
 .../TriggerCooldownIntegrationTest.java         |   9 +-
 .../autoscaling/TriggerIntegrationTest.java     |  10 +-
 .../TriggerSetPropertiesIntegrationTest.java    |   3 +-
 .../sim/SimClusterStateProvider.java            |  42 ++-
 .../autoscaling/sim/SimDistribStateManager.java |  76 +++--
 .../autoscaling/sim/SimSolrCloudTestCase.java   |  39 ---
 .../sim/TestSimComputePlanAction.java           |  10 +-
 .../sim/TestSimExecutePlanAction.java           |  20 +-
 .../autoscaling/sim/TestSimExtremeIndexing.java |  18 +-
 .../sim/TestSimGenericDistributedQueue.java     |   3 -
 .../autoscaling/sim/TestSimLargeCluster.java    |  23 +-
 .../sim/TestSimNodeAddedTrigger.java            |  16 +-
 .../autoscaling/sim/TestSimNodeLostTrigger.java |  15 +-
 .../autoscaling/sim/TestSimPolicyCloud.java     |  16 +-
 .../sim/TestSimTriggerIntegration.java          |  24 +-
 .../solr/core/TestQuerySenderListener.java      |   8 +-
 .../org/apache/solr/handler/TestSQLHandler.java |   5 +
 .../solr/security/BasicAuthIntegrationTest.java | 138 ++++++--
 .../solr/security/BasicAuthStandaloneTest.java  |   4 +-
 .../PKIAuthenticationIntegrationTest.java       |  18 +-
 .../security/TestPKIAuthenticationPlugin.java   |  13 +-
 .../TestSolrCloudWithHadoopAuthPlugin.java      |   8 +-
 .../ClassificationUpdateProcessorTest.java      |   2 +-
 ...uthentication-and-authorization-plugins.adoc |   8 +-
 .../src/basic-authentication-plugin.adoc        |  14 +-
 .../apache/solr/client/solrj/SolrRequest.java   |  11 +
 .../solr/client/solrj/impl/Http2SolrClient.java |   5 +
 .../solr/client/solrj/impl/HttpSolrClient.java  |  13 +-
 .../client/solrj/io/stream/CloudSolrStream.java |  21 +-
 .../client/solrj/io/stream/FacetStream.java     |  11 +-
 .../solrj/io/stream/SearchFacadeStream.java     |   6 +
 .../client/solrj/io/stream/SearchStream.java    |   6 +-
 .../solrj/io/stream/TimeSeriesStream.java       |  11 +-
 .../solrj/io/stream/StreamExpressionTest.java   | 108 +++++++
 .../apache/solr/cloud/MiniSolrCloudCluster.java |  19 +-
 .../solr/cloud/SolrCloudAuthTestCase.java       | 132 ++++++++
 solr/webapp/web/js/angular/app.js               |   2 +-
 solr/webapp/web/js/angular/controllers/query.js |   3 +-
 solr/webapp/web/js/angular/services.js          |   2 +-
 111 files changed, 2060 insertions(+), 976 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 2b1ad2a,dfd190b..5ce3c06
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@@ -434,15 -433,11 +436,13 @@@ public class CoreContainer 
        }
  
        HttpClientUtil.setHttpClientRequestContextBuilder(httpClientBuilder);
- 
-     } else {
-       if (pkiAuthenticationPlugin != null) {
-         //this happened due to an authc plugin reload. no need to register the pkiAuthc plugin again
-         if(pkiAuthenticationPlugin.isInterceptorRegistered()) return;
-         log.info("PKIAuthenticationPlugin is managing internode requests");
-         setupHttpClientForAuthPlugin(pkiAuthenticationPlugin);
-         pkiAuthenticationPlugin.setInterceptorRegistered();
-       }
+     }
+     // Always register PKI auth interceptor, which will then delegate the decision of who should secure
+     // each request to the configured authentication plugin.
+     if (pkiAuthenticationPlugin != null && !pkiAuthenticationPlugin.isInterceptorRegistered()) {
+       pkiAuthenticationPlugin.getHttpClientBuilder(HttpClientUtil.getHttpClientBuilder());
++      shardHandlerFactory.setSecurityBuilder(pkiAuthenticationPlugin);
++      updateShardHandler.setSecurityBuilder(pkiAuthenticationPlugin);
      }
    }
  

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index a9d112a,a8caf73..21fd5c1
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@@ -20,7 -20,21 +20,22 @@@ import javax.servlet.FilterChain
  import javax.servlet.ServletRequest;
  import javax.servlet.ServletResponse;
  import java.io.Closeable;
+ import java.util.Arrays;
  import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ 
+ import com.codahale.metrics.Counter;
+ import com.codahale.metrics.Meter;
+ import com.codahale.metrics.MetricRegistry;
+ import com.codahale.metrics.Timer;
+ import org.apache.solr.core.SolrInfoBean;
+ import org.apache.solr.metrics.SolrMetricManager;
+ import org.apache.solr.metrics.SolrMetricProducer;
+ 
+ import org.apache.http.HttpRequest;
+ import org.apache.http.protocol.HttpContext;
++import org.eclipse.jetty.client.api.Request;
  
  /**
   * 
@@@ -52,8 -81,44 +82,62 @@@ public abstract class AuthenticationPlu
    public abstract boolean doAuthenticate(ServletRequest request, ServletResponse response,
        FilterChain filterChain) throws Exception;
  
+   /**
+    * This method is called by SolrDispatchFilter in order to initiate authentication.
+    * It does some standard metrics counting.
+    */
+   public final boolean authenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
+     Timer.Context timer = requestTimes.time();
+     requests.inc();
+     try {
+       return doAuthenticate(request, response, filterChain);
+     } catch(Exception e) {
+       numErrors.mark();
+       throw e;
+     } finally {
+       long elapsed = timer.stop();
+       totalTime.inc(elapsed);
+     }
+   }
+ 
+   /**
+    * Override this method to intercept internode requests. This allows your authentication
+    * plugin to decide on per-request basis whether it should handle inter-node requests or
+    * delegate to {@link PKIAuthenticationPlugin}. Return true to indicate that your plugin
+    * did handle the request, or false to signal that PKI plugin should handle it. This method
+    * will be called by {@link PKIAuthenticationPlugin}'s interceptor.
+    * 
+    * <p>
+    *   If not overridden, this method will return true for plugins implementing {@link HttpClientBuilderPlugin}.
+    *   This method can be overridden by subclasses e.g. to set HTTP headers, even if you don't use a clientBuilder. 
+    * </p>
+    * @param httpRequest the httpRequest that is about to be sent to another internal Solr node
+    * @param httpContext the context of that request.
+    * @return true if this plugin handled authentication for the request, else false
+    */
+   protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
+     return this instanceof HttpClientBuilderPlugin;
+   }
 +
 +  /**
++   * Override this method to intercept internode requests. This allows your authentication
++   * plugin to decide on per-request basis whether it should handle inter-node requests or
++   * delegate to {@link PKIAuthenticationPlugin}. Return true to indicate that your plugin
++   * did handle the request, or false to signal that PKI plugin should handle it. This method
++   * will be called by {@link PKIAuthenticationPlugin}'s interceptor.
++   *
++   * <p>
++   *   If not overridden, this method will return true for plugins implementing {@link HttpClientBuilderPlugin}.
++   *   This method can be overridden by subclasses e.g. to set HTTP headers, even if you don't use a clientBuilder.
++   * </p>
++   * @param request the httpRequest that is about to be sent to another internal Solr node
++   * @return true if this plugin handled authentication for the request, else false
++   */
++  protected boolean interceptInternodeRequest(Request request) {
++    return this instanceof HttpClientBuilderPlugin;
++  }
+   
+   /**
     * Cleanup any per request  data
     */
    public void closeRequest() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index 1212452,3db21cd..72afb95
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@@ -33,14 -37,20 +37,22 @@@ import java.util.StringTokenizer
  
  import com.google.common.collect.ImmutableSet;
  import org.apache.commons.codec.binary.Base64;
+ import org.apache.commons.lang3.builder.ToStringBuilder;
  import org.apache.http.Header;
  import org.apache.http.HttpHeaders;
- import org.apache.http.auth.BasicUserPrincipal;
+ import org.apache.http.HttpRequest;
+ import org.apache.http.annotation.Contract;
+ import org.apache.http.annotation.ThreadingBehavior;
+ import org.apache.http.client.protocol.HttpClientContext;
  import org.apache.http.message.BasicHeader;
+ import org.apache.http.protocol.HttpContext;
++import org.apache.solr.client.solrj.impl.Http2SolrClient;
  import org.apache.solr.common.SolrException;
+ import org.apache.solr.common.SolrException.ErrorCode;
  import org.apache.solr.common.SpecProvider;
  import org.apache.solr.common.util.CommandOperation;
  import org.apache.solr.common.util.ValidatingJsonMap;
++import org.eclipse.jetty.client.api.Request;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -186,6 -218,22 +220,36 @@@ public class BasicAuthPlugin extends Au
    }
  
    @Override
+   protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
+     if (forwardCredentials) {
+       if (httpContext instanceof HttpClientContext) {
+         HttpClientContext httpClientContext = (HttpClientContext) httpContext;
+         if (httpClientContext.getUserToken() instanceof BasicAuthUserPrincipal) {
+           BasicAuthUserPrincipal principal = (BasicAuthUserPrincipal) httpClientContext.getUserToken();
+           String userPassBase64 = Base64.encodeBase64String((principal.getName() + ":" + principal.getPassword()).getBytes(StandardCharsets.UTF_8));
+           httpRequest.setHeader(HttpHeaders.AUTHORIZATION, "Basic " + userPassBase64);
+           return true;
+         }
++      }
++    }
++    return false;
++  }
++
++  @Override
++  protected boolean interceptInternodeRequest(Request request) {
++    if (forwardCredentials) {
++      Object userToken = request.getAttributes().get(Http2SolrClient.REQ_PRINCIPAL_KEY);
++      if (userToken instanceof BasicAuthUserPrincipal) {
++        BasicAuthUserPrincipal principal = (BasicAuthUserPrincipal) userToken;
++        String userPassBase64 = Base64.encodeBase64String((principal.getName() + ":" + principal.getPassword()).getBytes(StandardCharsets.UTF_8));
++        request.header(HttpHeaders.AUTHORIZATION, "Basic " + userPassBase64);
++        return true;
+       }
+     }
+     return false;
+   }
+ 
+   @Override
    public ValidatingJsonMap getSpec() {
      return authenticationProvider.getSpec();
    }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index b69cb8b,6bb96d5..a60e700
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@@ -225,26 -224,37 +227,56 @@@ public class PKIAuthenticationPlugin ex
    }
  
    @Override
 +  public void setup(Http2SolrClient client) {
 +    final HttpListenerFactory.RequestResponseListener listener = new HttpListenerFactory.RequestResponseListener() {
 +      @Override
 +      public void onQueued(Request request) {
-         generateToken().ifPresent(s -> request.header(HEADER, myNodeName + " " + s));
++        if (cores.getAuthenticationPlugin() == null) {
++          return;
++        }
++        if (!cores.getAuthenticationPlugin().interceptInternodeRequest(request)) {
++          log.debug("{} secures this internode request", this.getClass().getSimpleName());
++          generateToken().ifPresent(s -> request.header(HEADER, myNodeName + " " + s));
++        } else {
++          log.debug("{} secures this internode request", cores.getAuthenticationPlugin().getClass().getSimpleName());
++        }
 +      }
 +    };
 +    client.addListenerFactory(() -> listener);
 +  }
 +
 +  @Override
    public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
      HttpClientUtil.addRequestInterceptor(interceptor);
+     interceptorRegistered = true;
      return builder;
    }
  
+   public boolean needsAuthorization(HttpServletRequest req) {
+     return req.getUserPrincipal() != SU;
+   }
+ 
+   private class HttpHeaderClientInterceptor implements HttpRequestInterceptor {
+ 
+     public HttpHeaderClientInterceptor() {
+     }
+ 
+     @Override
+     public void process(HttpRequest httpRequest, HttpContext httpContext) throws HttpException, IOException {
+       if (cores.getAuthenticationPlugin() == null) {
+         return;
+       }
+       if (!cores.getAuthenticationPlugin().interceptInternodeRequest(httpRequest, httpContext)) {
+         log.debug("{} secures this internode request", this.getClass().getSimpleName());
+         setHeader(httpRequest);
+       } else {
+         log.debug("{} secures this internode request", cores.getAuthenticationPlugin().getClass().getSimpleName());
+       }
+     }
+   }
+ 
    @SuppressForbidden(reason = "Needs currentTimeMillis to set current time in header")
 -  void setHeader(HttpRequest httpRequest) {
 +  private Optional<String> generateToken() {
-     if (disabled()) return Optional.empty();
- 
      SolrRequestInfo reqInfo = getRequestInfo();
      String usr;
      if (reqInfo != null) {
@@@ -272,26 -281,7 +304,11 @@@
      byte[] payload = s.getBytes(UTF_8);
      byte[] payloadCipher = publicKeyHandler.keyPair.encrypt(ByteBuffer.wrap(payload));
      String base64Cipher = Base64.byteArrayToBase64(payloadCipher);
 -    httpRequest.setHeader(HEADER, myNodeName + " " + base64Cipher);
 +    return Optional.of(base64Cipher);
 +  }
 +
-   public boolean needsAuthorization(HttpServletRequest req) {
-     return req.getUserPrincipal() != SU;
-   }
- 
-   private class HttpHeaderClientInterceptor implements HttpRequestInterceptor {
- 
-     public HttpHeaderClientInterceptor() {
-     }
- 
-     @Override
-     public void process(HttpRequest httpRequest, HttpContext httpContext) throws HttpException, IOException {
-       generateToken().ifPresent(s -> httpRequest.setHeader(HEADER, myNodeName + " " + s));
-     }
-   }
-   
 +  void setHeader(HttpRequest httpRequest) {
 +    generateToken().ifPresent(s -> httpRequest.setHeader(HEADER, myNodeName + " " + s));
    }
  
    boolean isSolrThread() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
index 384566a,39b46d1..f73d1c5
--- a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
@@@ -32,11 -32,14 +32,12 @@@ import java.util.Set
  import java.util.concurrent.CompletionService;
  import java.util.concurrent.ExecutorCompletionService;
  import java.util.concurrent.Future;
+ 
 -import org.apache.http.HttpResponse;
  import org.apache.http.NoHttpResponseException;
  import org.apache.solr.client.solrj.SolrClient;
  import org.apache.solr.client.solrj.SolrServerException;
  import org.apache.solr.client.solrj.impl.BinaryResponseParser;
- import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient; // jdoc
+ import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
 -import org.apache.solr.client.solrj.impl.HttpSolrClient;
  import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
  import org.apache.solr.client.solrj.request.UpdateRequest;
  import org.apache.solr.common.SolrException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --cc solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 4113d6c,a7c5aa9..7bf38ce
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@@ -202,19 -233,7 +233,12 @@@ public class BasicAuthIntegrationTest e
  
        executeCommand(baseUrl + authzPrefix, cl,"{set-permission : { name : update , role : admin}}", "harry", "HarryIsUberCool");
  
 +      UpdateRequest del = new UpdateRequest().deleteByQuery("*:*");
 +      del.setBasicAuthCredentials("harry","HarryIsUberCool");
 +      del.setCommitWithin(10);
 +      del.process(cluster.getSolrClient(), COLLECTION);
 +
-       SolrInputDocument doc = new SolrInputDocument();
-       doc.setField("id","4");
-       UpdateRequest update = new UpdateRequest();
-       update.setBasicAuthCredentials("harry","HarryIsUberCool");
-       update.add(doc);
-       update.setCommitWithin(100);
-       cluster.getSolrClient().request(update, COLLECTION);
- 
+       addDocument("harry","HarryIsUberCool","id", "4");
  
        executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: true}}", "harry", "HarryIsUberCool");
        verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/blockUnknown", "true", 20, "harry", "HarryIsUberCool");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c0af83e4/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
----------------------------------------------------------------------
diff --cc solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index 2f7e29f,0000000..a50f20a
mode 100644,000000..100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@@ -1,974 -1,0 +1,979 @@@
 +/*
 + * 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.client.solrj.impl;
 +
 +import java.io.ByteArrayOutputStream;
 +import java.io.Closeable;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.UnsupportedEncodingException;
 +import java.lang.invoke.MethodHandles;
 +import java.net.ConnectException;
 +import java.nio.charset.StandardCharsets;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Iterator;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Locale;
 +import java.util.Set;
 +import java.util.concurrent.ExecutionException;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Phaser;
 +import java.util.concurrent.Semaphore;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.TimeoutException;
 +
 +import org.apache.commons.io.IOUtils;
 +import org.apache.http.HttpStatus;
 +import org.apache.http.entity.ContentType;
 +import org.apache.solr.client.solrj.ResponseParser;
 +import org.apache.solr.client.solrj.SolrClient;
 +import org.apache.solr.client.solrj.SolrRequest;
 +import org.apache.solr.client.solrj.SolrServerException;
 +import org.apache.solr.client.solrj.V2RequestSupport;
 +import org.apache.solr.client.solrj.embedded.SSLConfig;
 +import org.apache.solr.client.solrj.request.RequestWriter;
 +import org.apache.solr.client.solrj.request.UpdateRequest;
 +import org.apache.solr.client.solrj.request.V2Request;
 +import org.apache.solr.client.solrj.util.ClientUtils;
 +import org.apache.solr.client.solrj.util.Constants;
 +import org.apache.solr.common.SolrException;
 +import org.apache.solr.common.StringUtils;
 +import org.apache.solr.common.params.CommonParams;
 +import org.apache.solr.common.params.ModifiableSolrParams;
 +import org.apache.solr.common.params.SolrParams;
 +import org.apache.solr.common.params.UpdateParams;
 +import org.apache.solr.common.util.Base64;
 +import org.apache.solr.common.util.ContentStream;
 +import org.apache.solr.common.util.ExecutorUtil;
 +import org.apache.solr.common.util.NamedList;
 +import org.apache.solr.common.util.ObjectReleaseTracker;
 +import org.apache.solr.common.util.SolrjNamedThreadFactory;
 +import org.eclipse.jetty.client.HttpClient;
 +import org.eclipse.jetty.client.HttpClientTransport;
 +import org.eclipse.jetty.client.ProtocolHandlers;
 +import org.eclipse.jetty.client.api.Request;
 +import org.eclipse.jetty.client.api.Response;
 +import org.eclipse.jetty.client.api.Result;
 +import org.eclipse.jetty.client.http.HttpClientTransportOverHTTP;
 +import org.eclipse.jetty.client.util.BufferingResponseListener;
 +import org.eclipse.jetty.client.util.BytesContentProvider;
 +import org.eclipse.jetty.client.util.FormContentProvider;
 +import org.eclipse.jetty.client.util.InputStreamContentProvider;
 +import org.eclipse.jetty.client.util.InputStreamResponseListener;
 +import org.eclipse.jetty.client.util.MultiPartContentProvider;
 +import org.eclipse.jetty.client.util.OutputStreamContentProvider;
 +import org.eclipse.jetty.client.util.StringContentProvider;
 +import org.eclipse.jetty.http.HttpField;
 +import org.eclipse.jetty.http.HttpFields;
 +import org.eclipse.jetty.http.HttpHeader;
 +import org.eclipse.jetty.http.HttpMethod;
 +import org.eclipse.jetty.http2.client.HTTP2Client;
 +import org.eclipse.jetty.http2.client.http.HttpClientTransportOverHTTP2;
 +import org.eclipse.jetty.util.BlockingArrayQueue;
 +import org.eclipse.jetty.util.Fields;
 +import org.eclipse.jetty.util.ssl.SslContextFactory;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import static org.apache.solr.common.util.Utils.getObjectByPath;
 +
 +// TODO: error handling, small Http2SolrClient features, security, ssl
 +/**
 + * @lucene.experimental
 + */
 +public class Http2SolrClient extends SolrClient {
++  public static final String REQ_PRINCIPAL_KEY = "solr-req-principal";
++
 +  private static volatile SSLConfig defaultSSLConfig;
 +
 +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 +  private static final String AGENT = "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
 +  private static final String UTF_8 = StandardCharsets.UTF_8.name();
 +  private static final String DEFAULT_PATH = "/select";
 +  private static final List<String> errPath = Arrays.asList("metadata", "error-class");
 +
 +  private HttpClient httpClient;
 +  private volatile Set<String> queryParams = Collections.emptySet();
 +  private int idleTimeout;
 +
 +  private ResponseParser parser = new BinaryResponseParser();
 +  private volatile RequestWriter requestWriter = new BinaryRequestWriter();
 +  private List<HttpListenerFactory> listenerFactory = new LinkedList<>();
 +  private AsyncTracker asyncTracker = new AsyncTracker();
 +  /**
 +   * The URL of the Solr server.
 +   */
 +  private String serverBaseUrl;
 +  private boolean closeClient;
 +
 +  protected Http2SolrClient(String serverBaseUrl, Builder builder) {
 +    if (serverBaseUrl != null)  {
 +      if (!serverBaseUrl.equals("/") && serverBaseUrl.endsWith("/")) {
 +        serverBaseUrl = serverBaseUrl.substring(0, serverBaseUrl.length() - 1);
 +      }
 +
 +      if (serverBaseUrl.startsWith("//")) {
 +        serverBaseUrl = serverBaseUrl.substring(1, serverBaseUrl.length());
 +      }
 +      this.serverBaseUrl = serverBaseUrl;
 +    }
 +
 +    if (builder.idleTimeout != null) idleTimeout = builder.idleTimeout;
 +    else idleTimeout = HttpClientUtil.DEFAULT_SO_TIMEOUT;
 +
 +    if (builder.httpClient == null) {
 +      httpClient = createHttpClient(builder);
 +      closeClient = true;
 +    } else {
 +      httpClient = builder.httpClient;
 +    }
 +    if (!httpClient.isStarted()) {
 +      try {
 +        httpClient.start();
 +      } catch (Exception e) {
 +        throw new RuntimeException(e);
 +      }
 +    }
 +
 +    assert ObjectReleaseTracker.track(this);
 +  }
 +
 +  public void addListenerFactory(HttpListenerFactory factory) {
 +    this.listenerFactory.add(factory);
 +  }
 +
 +  HttpClient getHttpClient() {
 +    return httpClient;
 +  }
 +
 +  ProtocolHandlers getProtocolHandlers() {
 +    return httpClient.getProtocolHandlers();
 +  }
 +
 +  private HttpClient createHttpClient(Builder builder) {
 +    HttpClient httpClient;
 +
 +    BlockingArrayQueue<Runnable> queue = new BlockingArrayQueue<>(256, 256);
 +    ThreadPoolExecutor httpClientExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(32,
 +        256, 60, TimeUnit.SECONDS, queue, new SolrjNamedThreadFactory("h2sc"));
 +
 +    SslContextFactory sslContextFactory;
 +    boolean ssl;
 +    if (builder.sslConfig == null) {
 +      sslContextFactory = getDefaultSslContextFactory();
 +      ssl = sslContextFactory.getTrustStore() != null || sslContextFactory.getTrustStorePath() != null;
 +    } else {
 +      sslContextFactory = builder.sslConfig.createContextFactory();
 +      ssl = true;
 +    }
 +
 +    boolean sslOnJava8OrLower = ssl && !Constants.JRE_IS_MINIMUM_JAVA9;
 +    HttpClientTransport transport;
 +    if (builder.useHttp1_1 || sslOnJava8OrLower) {
 +      if (sslOnJava8OrLower && !builder.useHttp1_1) {
 +        log.warn("Create Http2SolrClient with HTTP/1.1 transport since Java 8 or lower versions does not support SSL + HTTP/2");
 +      } else {
 +        log.debug("Create Http2SolrClient with HTTP/1.1 transport");
 +      }
 +      transport = new HttpClientTransportOverHTTP(2);
 +      httpClient = new HttpClient(transport, sslContextFactory);
 +      if (builder.maxConnectionsPerHost != null) httpClient.setMaxConnectionsPerDestination(builder.maxConnectionsPerHost);
 +    } else {
 +      log.debug("Create Http2SolrClient with HTTP/2 transport");
 +      HTTP2Client http2client = new HTTP2Client();
 +      transport = new HttpClientTransportOverHTTP2(http2client);
 +      httpClient = new HttpClient(transport, sslContextFactory);
 +      httpClient.setMaxConnectionsPerDestination(4);
 +    }
 +
 +    httpClient.setExecutor(httpClientExecutor);
 +    httpClient.setStrictEventOrdering(false);
 +    httpClient.setConnectBlocking(true);
 +    httpClient.setFollowRedirects(false);
 +    httpClient.setMaxRequestsQueuedPerDestination(asyncTracker.getMaxRequestsQueuedPerDestination());
 +    httpClient.setUserAgentField(new HttpField(HttpHeader.USER_AGENT, AGENT));
 +
 +    if (builder.idleTimeout != null) httpClient.setIdleTimeout(builder.idleTimeout);
 +    if (builder.connectionTimeout != null) httpClient.setConnectTimeout(builder.connectionTimeout);
 +    return httpClient;
 +  }
 +
 +  public void close() {
 +    // we wait for async requests, so far devs don't want to give sugar for this
 +    asyncTracker.waitForComplete();
 +    if (closeClient) {
 +      try {
 +        ExecutorService executor = (ExecutorService) httpClient.getExecutor();
 +        httpClient.setStopTimeout(1000);
 +        httpClient.stop();
 +        ExecutorUtil.shutdownAndAwaitTermination(executor);
 +      } catch (Exception e) {
 +        throw new RuntimeException("Exception on closing client", e);
 +      }
 +    }
 +
 +    assert ObjectReleaseTracker.release(this);
 +    System.out.println("Done close " + httpClient.getExecutor().toString());
 +  }
 +
 +  public boolean isV2ApiRequest(final SolrRequest request) {
 +    return request instanceof V2Request || request.getPath().contains("/____v2");
 +  }
 +
 +  public long getIdleTimeout() {
 +    return idleTimeout;
 +  }
 +
 +  public static class OutStream implements Closeable{
 +    private final String origCollection;
 +    private final ModifiableSolrParams origParams;
 +    private final OutputStreamContentProvider outProvider;
 +    private final InputStreamResponseListener responseListener;
 +    private final boolean isXml;
 +
 +    public OutStream(String origCollection, ModifiableSolrParams origParams,
 +                     OutputStreamContentProvider outProvider, InputStreamResponseListener responseListener, boolean isXml) {
 +      this.origCollection = origCollection;
 +      this.origParams = origParams;
 +      this.outProvider = outProvider;
 +      this.responseListener = responseListener;
 +      this.isXml = isXml;
 +    }
 +
 +    boolean belongToThisStream(SolrRequest solrRequest, String collection) {
 +      ModifiableSolrParams solrParams = new ModifiableSolrParams(solrRequest.getParams());
 +      if (!origParams.toNamedList().equals(solrParams.toNamedList()) || !StringUtils.equals(origCollection, collection)) {
 +        return false;
 +      }
 +      return true;
 +    }
 +
 +    public void write(byte b[]) throws IOException {
 +      this.outProvider.getOutputStream().write(b);
 +    }
 +
 +    public void flush() throws IOException {
 +      this.outProvider.getOutputStream().flush();
 +    }
 +
 +    @Override
 +    public void close() throws IOException {
 +      if (isXml) {
 +        write("</stream>".getBytes(StandardCharsets.UTF_8));
 +      }
 +      this.outProvider.getOutputStream().close();
 +    }
 +
 +    //TODO this class should be hidden
 +    public InputStreamResponseListener getResponseListener() {
 +      return responseListener;
 +    }
 +  }
 +
 +  public OutStream initOutStream(String baseUrl,
 +                                 UpdateRequest updateRequest,
 +                                 String collection) throws IOException {
 +    String contentType = requestWriter.getUpdateContentType();
 +    final ModifiableSolrParams origParams = new ModifiableSolrParams(updateRequest.getParams());
 +
 +    // The parser 'wt=' and 'version=' params are used instead of the
 +    // original params
 +    ModifiableSolrParams requestParams = new ModifiableSolrParams(origParams);
 +    requestParams.set(CommonParams.WT, parser.getWriterType());
 +    requestParams.set(CommonParams.VERSION, parser.getVersion());
 +
 +    String basePath = baseUrl;
 +    if (collection != null)
 +      basePath += "/" + collection;
 +    if (!basePath.endsWith("/"))
 +      basePath += "/";
 +
 +    OutputStreamContentProvider provider = new OutputStreamContentProvider();
 +    Request postRequest = httpClient
 +        .newRequest(basePath + "update"
 +            + requestParams.toQueryString())
 +        .method(HttpMethod.POST)
 +        .header("User-Agent", HttpSolrClient.AGENT)
 +        .header("Content-Type", contentType)
 +        .content(provider);
 +    setListeners(updateRequest, postRequest);
 +    InputStreamResponseListener responseListener = new InputStreamResponseListener();
 +    postRequest.send(responseListener);
 +
 +    boolean isXml = ClientUtils.TEXT_XML.equals(requestWriter.getUpdateContentType());
 +    OutStream outStream = new OutStream(collection, origParams, provider, responseListener,
 +        isXml);
 +    if (isXml) {
 +      outStream.write("<stream>".getBytes(StandardCharsets.UTF_8));
 +    }
 +    return outStream;
 +  }
 +
 +  public void send(OutStream outStream, SolrRequest req, String collection) throws IOException {
 +    assert outStream.belongToThisStream(req, collection);
 +    this.requestWriter.write(req, outStream.outProvider.getOutputStream());
 +    if (outStream.isXml) {
 +      // check for commit or optimize
 +      SolrParams params = req.getParams();
 +      if (params != null) {
 +        String fmt = null;
 +        if (params.getBool(UpdateParams.OPTIMIZE, false)) {
 +          fmt = "<optimize waitSearcher=\"%s\" />";
 +        } else if (params.getBool(UpdateParams.COMMIT, false)) {
 +          fmt = "<commit waitSearcher=\"%s\" />";
 +        }
 +        if (fmt != null) {
 +          byte[] content = String.format(Locale.ROOT,
 +              fmt, params.getBool(UpdateParams.WAIT_SEARCHER, false)
 +                  + "")
 +              .getBytes(StandardCharsets.UTF_8);
 +          outStream.write(content);
 +        }
 +      }
 +    }
 +    outStream.flush();
 +  }
 +
 +  public NamedList<Object> request(SolrRequest solrRequest,
 +                                      String collection,
 +                                      OnComplete onComplete) throws IOException, SolrServerException {
 +    Request req = makeRequest(solrRequest, collection);
 +    final ResponseParser parser = solrRequest.getResponseParser() == null
 +        ? this.parser: solrRequest.getResponseParser();
 +
 +    if (onComplete != null) {
 +      // This async call only suitable for indexing since the response size is limited by 5MB
 +      req.onRequestQueued(asyncTracker.queuedListener)
 +          .onComplete(asyncTracker.completeListener).send(new BufferingResponseListener(5 * 1024 * 1024) {
 +
 +        @Override
 +        public void onComplete(Result result) {
 +          if (result.isFailed()) {
 +            onComplete.onFailure(result.getFailure());
 +            return;
 +          }
 +
 +          NamedList<Object> rsp;
 +          try {
 +            InputStream is = getContentAsInputStream();
 +            assert ObjectReleaseTracker.track(is);
 +            rsp = processErrorsAndResponse(result.getResponse(),
 +                parser, is, getEncoding(), isV2ApiRequest(solrRequest));
 +            onComplete.onSuccess(rsp);
 +          } catch (Exception e) {
 +            onComplete.onFailure(e);
 +          }
 +        }
 +      });
 +      return null;
 +    } else {
 +      try {
 +        InputStreamResponseListener listener = new InputStreamResponseListener();
 +        req.send(listener);
 +        Response response = listener.get(idleTimeout, TimeUnit.MILLISECONDS);
 +        InputStream is = listener.getInputStream();
 +        assert ObjectReleaseTracker.track(is);
 +        return processErrorsAndResponse(response, parser, is, getEncoding(response), isV2ApiRequest(solrRequest));
 +      } catch (InterruptedException e) {
 +        Thread.currentThread().interrupt();
 +        throw new RuntimeException(e);
 +      } catch (TimeoutException e) {
 +        throw new SolrServerException(
 +            "Timeout occured while waiting response from server at: " + req.getURI(), e);
 +      } catch (ExecutionException e) {
 +        Throwable cause = e.getCause();
 +        if (cause instanceof ConnectException) {
 +          throw new SolrServerException("Server refused connection at: " + req.getURI(), cause);
 +        }
 +        if (cause instanceof SolrServerException) {
 +          throw (SolrServerException) cause;
 +        } else if (cause instanceof IOException) {
 +          throw new SolrServerException(
 +              "IOException occured when talking to server at: " + getBaseURL(), cause);
 +        }
 +        throw new SolrServerException(cause.getMessage(), cause);
 +      }
 +    }
 +  }
 +
 +  private String getEncoding(Response response) {
 +    String contentType = response.getHeaders().get(HttpHeader.CONTENT_TYPE);
 +    if (contentType != null) {
 +      String charset = "charset=";
 +      int index = contentType.toLowerCase(Locale.ENGLISH).indexOf(charset);
 +      if (index > 0) {
 +        String encoding = contentType.substring(index + charset.length());
 +        // Sometimes charsets arrive with an ending semicolon.
 +        int semicolon = encoding.indexOf(';');
 +        if (semicolon > 0)
 +          encoding = encoding.substring(0, semicolon).trim();
 +        // Sometimes charsets are quoted.
 +        int lastIndex = encoding.length() - 1;
 +        if (encoding.charAt(0) == '"' && encoding.charAt(lastIndex) == '"')
 +          encoding = encoding.substring(1, lastIndex).trim();
 +        return encoding;
 +      }
 +    }
 +    return null;
 +  }
 +
 +  private void setBasicAuthHeader(SolrRequest solrRequest, Request req) {
 +    if (solrRequest.getBasicAuthUser() != null && solrRequest.getBasicAuthPassword() != null) {
 +      String userPass = solrRequest.getBasicAuthUser() + ":" + solrRequest.getBasicAuthPassword();
 +      String encoded = Base64.byteArrayToBase64(userPass.getBytes(StandardCharsets.UTF_8));
 +      req.header("Authorization", "Basic " + encoded);
 +    }
 +  }
 +
 +  private Request makeRequest(SolrRequest solrRequest, String collection)
 +      throws SolrServerException, IOException {
 +    Request req = createRequest(solrRequest, collection);
 +    setListeners(solrRequest, req);
++    if (solrRequest.getUserPrincipal() != null) {
++      req.attribute(REQ_PRINCIPAL_KEY, solrRequest.getUserPrincipal());
++    }
 +
 +    return req;
 +  }
 +
 +  private void setListeners(SolrRequest solrRequest, Request req) {
 +    setBasicAuthHeader(solrRequest, req);
 +    for (HttpListenerFactory factory : listenerFactory) {
 +      HttpListenerFactory.RequestResponseListener listener = factory.get();
 +      req.onRequestQueued(listener);
 +      req.onRequestBegin(listener);
 +      req.onComplete(listener);
 +    }
 +  }
 +
 +  private Request createRequest(SolrRequest solrRequest, String collection) throws IOException, SolrServerException {
 +    if (solrRequest.getBasePath() == null && serverBaseUrl == null)
 +      throw new IllegalArgumentException("Destination node is not provided!");
 +
 +    if (solrRequest instanceof V2RequestSupport) {
 +      solrRequest = ((V2RequestSupport) solrRequest).getV2Request();
 +    }
 +    SolrParams params = solrRequest.getParams();
 +    RequestWriter.ContentWriter contentWriter = requestWriter.getContentWriter(solrRequest);
 +    Collection<ContentStream> streams = contentWriter == null ? requestWriter.getContentStreams(solrRequest) : null;
 +    String path = requestWriter.getPath(solrRequest);
 +    if (path == null || !path.startsWith("/")) {
 +      path = DEFAULT_PATH;
 +    }
 +
 +    ResponseParser parser = solrRequest.getResponseParser();
 +    if (parser == null) {
 +      parser = this.parser;
 +    }
 +
 +    // The parser 'wt=' and 'version=' params are used instead of the original
 +    // params
 +    ModifiableSolrParams wparams = new ModifiableSolrParams(params);
 +    if (parser != null) {
 +      wparams.set(CommonParams.WT, parser.getWriterType());
 +      wparams.set(CommonParams.VERSION, parser.getVersion());
 +    }
 +
 +    //TODO add invariantParams support
 +
 +    String basePath = solrRequest.getBasePath() == null ? serverBaseUrl : solrRequest.getBasePath();
 +    if (collection != null)
 +      basePath += "/" + collection;
 +
 +    if (solrRequest instanceof V2Request) {
 +      if (System.getProperty("solr.v2RealPath") == null) {
 +        basePath = serverBaseUrl.replace("/solr", "/api");
 +      } else {
 +        basePath = serverBaseUrl + "/____v2";
 +      }
 +    }
 +
 +    if (SolrRequest.METHOD.GET == solrRequest.getMethod()) {
 +      if (streams != null || contentWriter != null) {
 +        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "GET can't send streams!");
 +      }
 +
 +      return httpClient.newRequest(basePath + path + wparams.toQueryString()).method(HttpMethod.GET);
 +    }
 +
 +    if (SolrRequest.METHOD.DELETE == solrRequest.getMethod()) {
 +      return httpClient.newRequest(basePath + path + wparams.toQueryString()).method(HttpMethod.DELETE);
 +    }
 +
 +    if (SolrRequest.METHOD.POST == solrRequest.getMethod() || SolrRequest.METHOD.PUT == solrRequest.getMethod()) {
 +
 +      String url = basePath + path;
 +      boolean hasNullStreamName = false;
 +      if (streams != null) {
 +        hasNullStreamName = streams.stream().anyMatch(cs -> cs.getName() == null);
 +      }
 +
 +      boolean isMultipart = streams != null && streams.size() > 1 && !hasNullStreamName;
 +
 +      HttpMethod method = SolrRequest.METHOD.POST == solrRequest.getMethod() ? HttpMethod.POST : HttpMethod.PUT;
 +
 +      if (contentWriter != null) {
 +        Request req = httpClient
 +            .newRequest(url + wparams.toQueryString())
 +            .method(method);
 +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +        contentWriter.write(baos);
 +
 +        //TODO reduce memory usage
 +        return req.content(new BytesContentProvider(contentWriter.getContentType(), baos.toByteArray()));
 +      } else if (streams == null || isMultipart) {
 +        // send server list and request list as query string params
 +        ModifiableSolrParams queryParams = calculateQueryParams(this.queryParams, wparams);
 +        queryParams.add(calculateQueryParams(solrRequest.getQueryParams(), wparams));
 +        Request req = httpClient
 +            .newRequest(url + queryParams.toQueryString())
 +            .method(method);
 +        return fillContentStream(req, streams, wparams, isMultipart);
 +      } else {
 +        // It is has one stream, it is the post body, put the params in the URL
 +        ContentStream contentStream = streams.iterator().next();
 +        return httpClient
 +            .newRequest(url + wparams.toQueryString())
 +            .method(method)
 +            .content(new InputStreamContentProvider(contentStream.getStream()), contentStream.getContentType());
 +      }
 +    }
 +
 +    throw new SolrServerException("Unsupported method: " + solrRequest.getMethod());
 +  }
 +
 +  private Request fillContentStream(Request req, Collection<ContentStream> streams,
 +                                    ModifiableSolrParams wparams,
 +                                    boolean isMultipart) throws IOException {
 +    if (isMultipart) {
 +      // multipart/form-data
 +      MultiPartContentProvider content = new MultiPartContentProvider();
 +      Iterator<String> iter = wparams.getParameterNamesIterator();
 +      while (iter.hasNext()) {
 +        String key = iter.next();
 +        String[] vals = wparams.getParams(key);
 +        if (vals != null) {
 +          for (String val : vals) {
 +            content.addFieldPart(key, new StringContentProvider(val), null);
 +          }
 +        }
 +      }
 +      if (streams != null) {
 +        for (ContentStream contentStream : streams) {
 +          String contentType = contentStream.getContentType();
 +          if (contentType == null) {
 +            contentType = BinaryResponseParser.BINARY_CONTENT_TYPE; // default
 +          }
 +          String name = contentStream.getName();
 +          if (name == null) {
 +            name = "";
 +          }
 +          HttpFields fields = new HttpFields();
 +          fields.add(HttpHeader.CONTENT_TYPE, contentType);
 +          content.addFilePart(name, contentStream.getName(), new InputStreamContentProvider(contentStream.getStream()), fields);
 +        }
 +      }
 +      req.content(content);
 +    } else {
 +      // application/x-www-form-urlencoded
 +      Fields fields = new Fields();
 +      Iterator<String> iter = wparams.getParameterNamesIterator();
 +      while (iter.hasNext()) {
 +        String key = iter.next();
 +        String[] vals = wparams.getParams(key);
 +        if (vals != null) {
 +          for (String val : vals) {
 +            fields.add(key, val);
 +          }
 +        }
 +      }
 +      req.content(new FormContentProvider(fields, StandardCharsets.UTF_8));
 +    }
 +
 +    return req;
 +  }
 +
 +  private boolean wantStream(final ResponseParser processor) {
 +    return processor == null || processor instanceof InputStreamResponseParser;
 +  }
 +
 +  private NamedList<Object> processErrorsAndResponse(Response response,
 +                                                     final ResponseParser processor,
 +                                                     InputStream is,
 +                                                     String encoding,
 +                                                     final boolean isV2Api)
 +      throws SolrServerException {
 +    boolean shouldClose = true;
 +    try {
 +      // handle some http level checks before trying to parse the response
 +      int httpStatus = response.getStatus();
 +
 +      String contentType;
 +      contentType = response.getHeaders().get("content-type");
 +      if (contentType == null) contentType = "";
 +
 +      switch (httpStatus) {
 +        case HttpStatus.SC_OK:
 +        case HttpStatus.SC_BAD_REQUEST:
 +        case HttpStatus.SC_CONFLICT: // 409
 +          break;
 +        case HttpStatus.SC_MOVED_PERMANENTLY:
 +        case HttpStatus.SC_MOVED_TEMPORARILY:
 +          if (!httpClient.isFollowRedirects()) {
 +            throw new SolrServerException("Server at " + getBaseURL()
 +                + " sent back a redirect (" + httpStatus + ").");
 +          }
 +          break;
 +        default:
 +          if (processor == null || "".equals(contentType)) {
 +            throw new RemoteSolrException(serverBaseUrl, httpStatus, "non ok status: " + httpStatus
 +                + ", message:" + response.getReason(),
 +                null);
 +          }
 +      }
 +
 +      if (wantStream(parser)) {
 +        // no processor specified, return raw stream
 +        NamedList<Object> rsp = new NamedList<>();
 +        rsp.add("stream", is);
 +        // Only case where stream should not be closed
 +        shouldClose = false;
 +        return rsp;
 +      }
 +
 +      String procCt = processor.getContentType();
 +      if (procCt != null) {
 +        String procMimeType = ContentType.parse(procCt).getMimeType().trim().toLowerCase(Locale.ROOT);
 +        String mimeType = ContentType.parse(contentType).getMimeType().trim().toLowerCase(Locale.ROOT);
 +        if (!procMimeType.equals(mimeType)) {
 +          // unexpected mime type
 +          String msg = "Expected mime type " + procMimeType + " but got " + mimeType + ".";
 +          try {
 +            msg = msg + " " + IOUtils.toString(is, encoding);
 +          } catch (IOException e) {
 +            throw new RemoteSolrException(serverBaseUrl, httpStatus, "Could not parse response with encoding " + encoding, e);
 +          }
 +          throw new RemoteSolrException(serverBaseUrl, httpStatus, msg, null);
 +        }
 +      }
 +
 +      NamedList<Object> rsp;
 +      try {
 +        rsp = processor.processResponse(is, encoding);
 +      } catch (Exception e) {
 +        throw new RemoteSolrException(serverBaseUrl, httpStatus, e.getMessage(), e);
 +      }
 +
 +      Object error = rsp == null ? null : rsp.get("error");
 +      if (error != null && (String.valueOf(getObjectByPath(error, true, errPath)).endsWith("ExceptionWithErrObject"))) {
 +        throw RemoteExecutionException.create(serverBaseUrl, rsp);
 +      }
 +      if (httpStatus != HttpStatus.SC_OK && !isV2Api) {
 +        NamedList<String> metadata = null;
 +        String reason = null;
 +        try {
 +          NamedList err = (NamedList) rsp.get("error");
 +          if (err != null) {
 +            reason = (String) err.get("msg");
 +            if (reason == null) {
 +              reason = (String) err.get("trace");
 +            }
 +            metadata = (NamedList<String>) err.get("metadata");
 +          }
 +        } catch (Exception ex) {}
 +        if (reason == null) {
 +          StringBuilder msg = new StringBuilder();
 +          msg.append(response.getReason())
 +              .append("\n\n")
 +              .append("request: ")
 +              .append(response.getRequest().getMethod());
 +          try {
 +            reason = java.net.URLDecoder.decode(msg.toString(), UTF_8);
 +          } catch (UnsupportedEncodingException e) {
 +          }
 +        }
 +        RemoteSolrException rss = new RemoteSolrException(serverBaseUrl, httpStatus, reason, null);
 +        if (metadata != null) rss.setMetadata(metadata);
 +        throw rss;
 +      }
 +      return rsp;
 +    } finally {
 +      if (shouldClose) {
 +        try {
 +          is.close();
 +          assert ObjectReleaseTracker.release(is);
 +        } catch (IOException e) {
 +          // quitely
 +        }
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
 +    return request(request, collection, null);
 +  }
 +
 +  public void setRequestWriter(RequestWriter requestWriter) {
 +    this.requestWriter = requestWriter;
 +  }
 +
 +  public interface OnComplete {
 +    void onSuccess(NamedList<Object> result);
 +
 +    void onFailure(Throwable e);
 +  }
 +
 +  public void setFollowRedirects(boolean follow) {
 +    httpClient.setFollowRedirects(follow);
 +  }
 +
 +  public String getBaseURL() {
 +    return serverBaseUrl;
 +  }
 +
 +  private static class AsyncTracker {
 +    private static final int MAX_OUTSTANDING_REQUESTS = 1000;
 +
 +    // wait for async requests
 +    private final Phaser phaser;
 +    // maximum outstanding requests left
 +    private final Semaphore available;
 +    private final Request.QueuedListener queuedListener;
 +    private final Response.CompleteListener completeListener;
 +
 +    AsyncTracker() {
 +      // TODO: what about shared instances?
 +      phaser = new Phaser(1);
 +      available = new Semaphore(MAX_OUTSTANDING_REQUESTS, false);
 +      queuedListener = request -> {
 +        phaser.register();
 +        try {
 +          available.acquire();
 +        } catch (InterruptedException ignored) {
 +
 +        }
 +      };
 +      completeListener = result -> {
 +        phaser.arriveAndDeregister();
 +        available.release();
 +      };
 +    }
 +
 +    int getMaxRequestsQueuedPerDestination() {
 +      // comfortably above max outstanding requests
 +      return MAX_OUTSTANDING_REQUESTS * 3;
 +    }
 +
 +    public void waitForComplete() {
 +      phaser.arriveAndAwaitAdvance();
 +      phaser.arriveAndDeregister();
 +    }
 +  }
 +
 +  public static class Builder {
 +
 +    private HttpClient httpClient;
 +    private SSLConfig sslConfig = defaultSSLConfig;
 +    private Integer idleTimeout;
 +    private Integer connectionTimeout;
 +    private Integer maxConnectionsPerHost;
 +    private boolean useHttp1_1 = Boolean.getBoolean("solr.http1");
 +    protected String baseSolrUrl;
 +
 +    public Builder() {
 +
 +    }
 +
 +    public Builder(String baseSolrUrl) {
 +      this.baseSolrUrl = baseSolrUrl;
 +    }
 +
 +    public Http2SolrClient build() {
 +      return new Http2SolrClient(baseSolrUrl, this);
 +    }
 +
 +    public Builder withHttpClient(HttpClient httpClient) {
 +      this.httpClient = httpClient;
 +      return this;
 +    }
 +
 +    public Builder withSSLConfig(SSLConfig sslConfig) {
 +      this.sslConfig = sslConfig;
 +      return this;
 +    }
 +
 +    /**
 +     * Set maxConnectionsPerHost for http1 connections, maximum number http2 connections is limited by 4
 +     */
 +    public Builder maxConnectionsPerHost(int max) {
 +      this.maxConnectionsPerHost = max;
 +      return this;
 +    }
 +
 +    public Builder idleTimeout(int idleConnectionTimeout) {
 +      this.idleTimeout = idleConnectionTimeout;
 +      return this;
 +    }
 +
 +    public Builder useHttp1_1(boolean useHttp1_1) {
 +      this.useHttp1_1 = useHttp1_1;
 +      return this;
 +    }
 +
 +    public Builder connectionTimeout(int connectionTimeOut) {
 +      this.connectionTimeout = connectionTimeOut;
 +      return this;
 +    }
 +
 +  }
 +
 +  /**
 +   * Subclass of SolrException that allows us to capture an arbitrary HTTP status code that may have been returned by
 +   * the remote server or a proxy along the way.
 +   */
 +  public static class RemoteSolrException extends SolrException {
 +    /**
 +     * @param remoteHost the host the error was received from
 +     * @param code       Arbitrary HTTP status code
 +     * @param msg        Exception Message
 +     * @param th         Throwable to wrap with this Exception
 +     */
 +    public RemoteSolrException(String remoteHost, int code, String msg, Throwable th) {
 +      super(code, "Error from server at " + remoteHost + ": " + msg, th);
 +    }
 +  }
 +
 +  /**
 +   * This should be thrown when a server has an error in executing the request and it sends a proper payload back to the
 +   * client
 +   */
 +  public static class RemoteExecutionException extends RemoteSolrException {
 +    private NamedList meta;
 +
 +    public RemoteExecutionException(String remoteHost, int code, String msg, NamedList meta) {
 +      super(remoteHost, code, msg, null);
 +      this.meta = meta;
 +    }
 +
 +    public static RemoteExecutionException create(String host, NamedList errResponse) {
 +      Object errObj = errResponse.get("error");
 +      if (errObj != null) {
 +        Number code = (Number) getObjectByPath(errObj, true, Collections.singletonList("code"));
 +        String msg = (String) getObjectByPath(errObj, true, Collections.singletonList("msg"));
 +        return new RemoteExecutionException(host, code == null ? ErrorCode.UNKNOWN.code : code.intValue(),
 +            msg == null ? "Unknown Error" : msg, errResponse);
 +
 +      } else {
 +        throw new RuntimeException("No error");
 +      }
 +    }
 +
 +    public NamedList getMetaData() {
 +      return meta;
 +    }
 +  }
 +
 +  public Set<String> getQueryParams() {
 +    return queryParams;
 +  }
 +
 +  /**
 +   * Expert Method
 +   *
 +   * @param queryParams set of param keys to only send via the query string
 +   *                    Note that the param will be sent as a query string if the key is part
 +   *                    of this Set or the SolrRequest's query params.
 +   * @see org.apache.solr.client.solrj.SolrRequest#getQueryParams
 +   */
 +  public void setQueryParams(Set<String> queryParams) {
 +    this.queryParams = queryParams;
 +  }
 +
 +  private ModifiableSolrParams calculateQueryParams(Set<String> queryParamNames,
 +                                                    ModifiableSolrParams wparams) {
 +    ModifiableSolrParams queryModParams = new ModifiableSolrParams();
 +    if (queryParamNames != null) {
 +      for (String param : queryParamNames) {
 +        String[] value = wparams.getParams(param);
 +        if (value != null) {
 +          for (String v : value) {
 +            queryModParams.add(param, v);
 +          }
 +          wparams.remove(param);
 +        }
 +      }
 +    }
 +    return queryModParams;
 +  }
 +
 +  public ResponseParser getParser() {
 +    return parser;
 +  }
 +
 +  public void setParser(ResponseParser processor) {
 +    parser = processor;
 +  }
 +
 +  public static void setDefaultSSLConfig(SSLConfig sslConfig) {
 +    Http2SolrClient.defaultSSLConfig = sslConfig;
 +  }
 +
 +  // public for testing, only used by tests
 +  public static void resetSslContextFactory() {
 +    Http2SolrClient.defaultSSLConfig = null;
 +  }
 +
 +  private static SslContextFactory getDefaultSslContextFactory() {
 +    SslContextFactory sslContextFactory = new SslContextFactory(false);
 +
 +    if (null != System.getProperty("javax.net.ssl.keyStore")) {
 +      sslContextFactory.setKeyStorePath
 +          (System.getProperty("javax.net.ssl.keyStore"));
 +    }
 +    if (null != System.getProperty("javax.net.ssl.keyStorePassword")) {
 +      sslContextFactory.setKeyStorePassword
 +          (System.getProperty("javax.net.ssl.keyStorePassword"));
 +    }
 +    if (null != System.getProperty("javax.net.ssl.trustStore")) {
 +      sslContextFactory.setTrustStorePath
 +          (System.getProperty("javax.net.ssl.trustStore"));
 +    }
 +    if (null != System.getProperty("javax.net.ssl.trustStorePassword")) {
 +      sslContextFactory.setTrustStorePassword
 +          (System.getProperty("javax.net.ssl.trustStorePassword"));
 +    }
 +
 +    String checkPeerNameStr = System.getProperty(HttpClientUtil.SYS_PROP_CHECK_PEER_NAME);
 +    boolean sslCheckPeerName = true;
 +    if (checkPeerNameStr == null || "false".equalsIgnoreCase(checkPeerNameStr)) {
 +      sslCheckPeerName = false;
 +    }
 +
 +    if (System.getProperty("tests.jettySsl.clientAuth") != null) {
 +      sslCheckPeerName = sslCheckPeerName || Boolean.getBoolean("tests.jettySsl.clientAuth");
 +    }
 +
 +    sslContextFactory.setNeedClientAuth(sslCheckPeerName);
 +    return sslContextFactory;
 +  }
 +
 +}