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

[1/3] lucene-solr:branch_7x: SOLR-13066: A failure while reloading a SolrCore can result in the SolrCore not being closed.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 7ac559df9 -> 3d6a09e9d


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/4bcad180
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/4bcad180
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/4bcad180

Branch: refs/heads/branch_7x
Commit: 4bcad18084c8b09486cc071b14e031062c6f927e
Parents: 7ac559d
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 12 17:02:40 2018 -0600
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 12 19:23:21 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/4bcad180/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9ab5a7a..424f4ae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -51,7 +51,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)
 
 * SOLR-13058: Fix block that was synchronizing on the wrong collection in OverseerTaskProcessor (Gus Heck)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4bcad180/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 706951d..be879b8 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -31,6 +31,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;
@@ -1421,6 +1422,7 @@ public class CoreContainer {
     if (isShutDown) {
       throw new AlreadyClosedException();
     }
+    SolrCore newCore = null;
     SolrCore core = solrCores.getCoreFromAnyList(name, false);
     if (core != null) {
 
@@ -1428,34 +1430,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 {
@@ -1468,7 +1477,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/4bcad180/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);
   }
 


[3/3] lucene-solr:branch_7x: SOLR-12801: Harden SimSolrCloudTests.

Posted by ma...@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/3d6a09e9
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/3d6a09e9
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/3d6a09e9

Branch: refs/heads/branch_7x
Commit: 3d6a09e9d96a57637293ccde795bf170ec410621
Parents: ca0ded6
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 12 18:21:06 2018 -0600
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 12 19:28:37 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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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/3d6a09e9/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 81f8831..7834fe2 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


[2/3] lucene-solr:branch_7x: SOLR-13067: Harden BasicAuthIntegrationTest.

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

# Conflicts:
#	solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
#	solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java


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

Branch: refs/heads/branch_7x
Commit: ca0ded6f878fcd57e0640ed056e0b63b92ed78c2
Parents: 4bcad18
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 12 16:00:08 2018 -0600
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 12 19:28:24 2018 -0600

----------------------------------------------------------------------
 .../solr/security/BasicAuthIntegrationTest.java | 27 ++++++++++++++++----
 1 file changed, 22 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca0ded6f/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..34c9bd2 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 org.apache.http.HttpResponse;
@@ -59,7 +63,8 @@ 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;
@@ -73,8 +78,8 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
 
   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();
@@ -83,6 +88,11 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
     
     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
@@ -110,10 +120,12 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
       
       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);
 
@@ -129,7 +141,10 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
         ((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);
       });
@@ -158,6 +173,8 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
 
       executeCommand(baseUrl + authzPrefix, cl,command, "solr", "SolrRocks");
 
+      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);