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 2020/12/10 23:35:56 UTC

[lucene-solr] 01/02: @1238 Fix up SolrCore ref count stuff.

This is an automated email from the ASF dual-hosted git repository.

markrmiller pushed a commit to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 39eba03d6dfd7408014592af3aec9eba943d7d9b
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Mon Nov 16 14:56:01 2020 -0600

    @1238 Fix up SolrCore ref count stuff.
---
 .../java/org/apache/solr/core/CoreContainer.java   | 193 ++++++++++-----------
 .../src/java/org/apache/solr/core/SolrCore.java    |  68 ++++----
 .../src/java/org/apache/solr/core/SolrCores.java   |  14 +-
 .../org/apache/solr/core/TestCodecSupport.java     |   2 +-
 .../src/resources/logconf/log4j2-startup-debug.xml |   1 +
 5 files changed, 134 insertions(+), 144 deletions(-)

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 6983ab5..21444df 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -1150,7 +1150,7 @@ public class CoreContainer implements Closeable {
     return coresLocator;
   }
 
-  protected SolrCore registerCore(CoreDescriptor cd, SolrCore core, boolean registerInZk, boolean skipRecovery) {
+  protected SolrCore registerCore(CoreDescriptor cd, SolrCore core, boolean registerInZk, boolean skipRecovery, boolean closeOld) {
 
     log.info("registerCore name={}, registerInZk={}, skipRecovery={}", cd.getName(), registerInZk, skipRecovery);
 
@@ -1180,7 +1180,7 @@ public class CoreContainer implements Closeable {
       return null;
     } else {
       log.info("replacing core: " + cd.getName());
-      old.close();
+      if (closeOld) old.close();
       if (registerInZk) {
         zkSys.registerInZk(core, skipRecovery);
       }
@@ -1343,7 +1343,7 @@ public class CoreContainer implements Closeable {
 
       core.start();
 
-      registerCore(dcore, core, isZooKeeperAware(), false);
+      registerCore(dcore, core, isZooKeeperAware(), false, true);
       registered = true;
 
 
@@ -1606,101 +1606,103 @@ public class CoreContainer implements Closeable {
       throw new AlreadyClosedException();
     }
     SolrCore newCore = null;
-    try (SolrCore core = solrCores.getCoreFromAnyList(name, true)) {
-      if (core != null) {
+    SolrCore core = solrCores.getCoreFromAnyList(name);
+    if (core != null) {
 
-        // The underlying core properties files may have changed, we don't really know. So we have a (perhaps) stale
-        // CoreDescriptor and we need to reload it from the disk files
-        CoreDescriptor cd = core.getCoreDescriptor();
-        //        if (core.getDirectoryFactory().isPersistent()) {
-        //          cd = reloadCoreDescriptor(core.getCoreDescriptor());
-        //        } else {
-        //          cd = core.getCoreDescriptor();
-        //        }
-        //        solrCores.addCoreDescriptor(cd);
-        Closeable oldCore = null;
-        boolean success = false;
+      // The underlying core properties files may have changed, we don't really know. So we have a (perhaps) stale
+      // CoreDescriptor and we need to reload it from the disk files
+      CoreDescriptor cd = core.getCoreDescriptor();
+      //        if (core.getDirectoryFactory().isPersistent()) {
+      //          cd = reloadCoreDescriptor(core.getCoreDescriptor());
+      //        } else {
+      //          cd = core.getCoreDescriptor();
+      //        }
+      //        solrCores.addCoreDescriptor(cd);
+      Closeable oldCore = null;
+      boolean success = false;
+      try {
+        solrCores.waitForLoadingCoreToFinish(name, 15000);
+        ConfigSet coreConfig = coreConfigService.loadConfigSet(cd);
+        log.info("Reloading SolrCore '{}' using configuration from {}", name, coreConfig.getName());
+        DocCollection docCollection = null;
+        newCore = core.reload(coreConfig);
         try {
-          solrCores.waitForLoadingCoreToFinish(name, 15000);
-          ConfigSet coreConfig = coreConfigService.loadConfigSet(cd);
-          log.info("Reloading SolrCore '{}' using configuration from {}", name, coreConfig.getName());
-          DocCollection docCollection = null;
-          newCore = core.reload(coreConfig);
-         try {
-           if (getZkController() != null) {
-             core.old_reloaded = true;
-             docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
-             // turn off indexing now, before the new core is registered
-             if (docCollection.getBool(ZkStateReader.READ_ONLY, false)) {
-               newCore.readOnly = true;
-             }
-           }
-
-           registerCore(cd, newCore, false, false);
-         } catch (Exception e) {
-           log.error("Exception registering reloaded core", e);
-           newCore.close();
-           throw new SolrException(ErrorCode.SERVER_ERROR, e);
-         }
-
-          // force commit on old core if the new one is readOnly and prevent any new updates
-          if (newCore.readOnly) {
-            RefCounted<IndexWriter> iwRef = core.getSolrCoreState().getIndexWriter(null);
-            if (iwRef != null) {
-              IndexWriter iw = iwRef.get();
-              // switch old core to readOnly
-              core.readOnly = true;
-              try {
-                if (iw != null) {
-                  iw.commit();
-                }
-              } finally {
-                iwRef.decref();
-              }
+          if (getZkController() != null) {
+            core.old_reloaded = true;
+            docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
+            // turn off indexing now, before the new core is registered
+            if (docCollection.getBool(ZkStateReader.READ_ONLY, false)) {
+              newCore.readOnly = true;
             }
           }
 
-          if (docCollection != null) {
-            Replica replica = docCollection.getReplica(cd.getName());
-            assert replica != null;
-            if (replica.getType() == Replica.Type.TLOG) { // TODO: needed here?
-              getZkController().stopReplicationFromLeader(core.getName());
-              if (!cd.getCloudDescriptor().isLeader()) {
-                getZkController().startReplicationFromLeader(newCore.getName(), true);
-              }
+          registerCore(cd, newCore, false, false, false);
+        } catch (Exception e) {
+          log.error("Exception registering reloaded core", e);
+          newCore.close();
+          throw new SolrException(ErrorCode.SERVER_ERROR, e);
+        }
 
-            } else if (replica.getType() == Replica.Type.PULL) {
-              getZkController().startReplicationFromLeader(newCore.getName(), false);
+        // force commit on old core if the new one is readOnly and prevent any new updates
+        if (newCore.readOnly) {
+          RefCounted<IndexWriter> iwRef = core.getSolrCoreState().getIndexWriter(null);
+          if (iwRef != null) {
+            IndexWriter iw = iwRef.get();
+            // switch old core to readOnly
+            core.readOnly = true;
+            try {
+              if (iw != null) {
+                iw.commit();
+              }
+            } finally {
+              iwRef.decref();
             }
           }
-          success = true;
-        } catch (SolrCoreState.CoreIsClosedException e) {
-          throw e;
-        } catch (Exception e) {
-          ParWork.propagateInterrupt("Exception reloading SolrCore", e);
-          SolrException exp = new SolrException(ErrorCode.SERVER_ERROR, "Unable to reload core [" + cd.getName() + "]", e);
-          try {
-            coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, e));
+        }
 
-          } catch (Exception e1) {
-            ParWork.propagateInterrupt(e1);
-            exp.addSuppressed(e1);
-          }
-          throw exp;
-        } finally {
-          if (!success) {
-            ParWork.close(newCore);
+        if (docCollection != null) {
+          Replica replica = docCollection.getReplica(cd.getName());
+          assert replica != null;
+          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) {
+            getZkController().startReplicationFromLeader(newCore.getName(), false);
           }
         }
-      } else {
-        CoreLoadFailure clf = coreInitFailures.get(name);
-        if (clf != null) {
-          createFromDescriptor(clf.cd, false);
-        } else {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + name);
+        core.close();
+        core.closeAndWait(true);
+        success = true;
+      } catch (SolrCoreState.CoreIsClosedException e) {
+        throw e;
+      } catch (Exception e) {
+        ParWork.propagateInterrupt("Exception reloading SolrCore", e);
+        SolrException exp = new SolrException(ErrorCode.SERVER_ERROR, "Unable to reload core [" + cd.getName() + "]", e);
+        try {
+          coreInitFailures.put(cd.getName(), new CoreLoadFailure(cd, e));
+
+        } catch (Exception e1) {
+          ParWork.propagateInterrupt(e1);
+          exp.addSuppressed(e1);
+        }
+        throw exp;
+      } finally {
+        if (!success) {
+          ParWork.close(newCore);
         }
       }
+    } else {
+      CoreLoadFailure clf = coreInitFailures.get(name);
+      if (clf != null) {
+        createFromDescriptor(clf.cd, false);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + name);
+      }
     }
+
   }
 
   /**
@@ -1817,7 +1819,7 @@ public class CoreContainer implements Closeable {
 
     try {
       if (core != null) {
-        core.closeAndWait();
+        core.closeAndWait(true);
       }
 
       if (exception != null) {
@@ -1859,7 +1861,7 @@ public class CoreContainer implements Closeable {
         cd.setProperty("name", toName);
         solrCores.addCoreDescriptor(cd);
         core.setName(toName);
-        registerCore(cd, core, isZooKeeperAware(), false);
+        registerCore(cd, core, isZooKeeperAware(), false, true);
         SolrCore old = solrCores.remove(name);
 
         coresLocator.rename(this, old.getCoreDescriptor(), core.getCoreDescriptor());
@@ -1895,23 +1897,11 @@ public class CoreContainer implements Closeable {
    * @see SolrCore#close()
    */
   public SolrCore getCore(String name) {
-    return getCore(name, true);
-  }
-
-  /**
-   * Gets a core by name and increase its refcount.
-   *
-   * @param name the core name
-   * @return the core if found, null if a SolrCore by this name does not exist
-   * @throws SolrCoreInitializationException if a SolrCore with this name failed to be initialized
-   * @see SolrCore#close()
-   */
-  public SolrCore getCore(String name, boolean incRefCount) {
     SolrCore core = null;
     CoreDescriptor desc = null;
 
     // Do this in two phases since we don't want to lock access to the cores over a load.
-    core = solrCores.getCoreFromAnyList(name, incRefCount);
+    core = solrCores.getCoreFromAnyList(name);
 
     // If a core is loaded, we're done just return it.
     if (core != null) {
@@ -1943,11 +1933,6 @@ public class CoreContainer implements Closeable {
         core = createFromDescriptor(desc, false); // This should throw an error if it fails.
       }
     }
-
-    if (core != null) {
-      core.open();
-    }
-
     return core;
   }
 
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 995b6fd..6270d5f 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1418,30 +1418,6 @@ public final class SolrCore implements SolrInfoBean, Closeable {
     return updateLogDir;
   }
 
-  /**
-   * Close the core, if it is still in use waits until is no longer in use.
-   *
-   * @see #close()
-   * @see #isClosed()
-   */
-  public void closeAndWait() throws TimeoutException {
-    close();
-    int timeouts = 180;
-    TimeOut timeout = new TimeOut(timeouts, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-    synchronized (closeAndWait) {
-      while (!isClosed()) {
-        try {
-          closeAndWait.wait(500);
-        } catch (InterruptedException e) {
-          ParWork.propagateInterrupt(e);
-        }
-        if (timeout.hasTimedOut()) {
-          throw new TimeoutException("Timeout waiting for SolrCore close timeout=" + timeouts + "s");
-        }
-      }
-    }
-  }
-
   private Codec initCodec(SolrConfig solrConfig, final IndexSchema schema) {
     final PluginInfo info = solrConfig.getPluginInfo(CodecFactory.class.getName());
     final CodecFactory factory;
@@ -1588,6 +1564,41 @@ public final class SolrCore implements SolrInfoBean, Closeable {
    */
   @Override
   public void close() {
+    int count = refCount.decrementAndGet();
+    if (log.isDebugEnabled()) log.debug("close refcount {} {}", this, count);
+    if (count > 0) return;
+    if (count < 0) {
+      log.warn("Too many close [count:{}] on {}", count, this);
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Too many closes on SolrCore");
+    }
+  }
+
+
+  /**
+   * Close the core, if it is still in use waits until is no longer in use.
+   *
+   * @see #close()
+   * @see #isClosed()
+   */
+  public void closeAndWait(boolean decref) throws TimeoutException {
+    if (decref) {
+      close();
+    }
+    int timeouts = 180;
+    TimeOut timeout = new TimeOut(timeouts, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    synchronized (closeAndWait) {
+      while (!isClosed()) {
+        try {
+          closeAndWait.wait(500);
+          log.warn("close count is {}", refCount.get());
+        } catch (InterruptedException e) {
+          ParWork.propagateInterrupt(e);
+        }
+        if (timeout.hasTimedOut()) {
+          throw new TimeoutException("Timeout waiting for SolrCore close timeout=" + timeouts + "s");
+        }
+      }
+    }
     if (getUpdateHandler() != null && getUpdateHandler().getSolrCoreState() != null) {
       synchronized (getUpdateHandler().getSolrCoreState().getReloadLock()) {
         doClose();
@@ -1598,13 +1609,6 @@ public final class SolrCore implements SolrInfoBean, Closeable {
   }
 
   private void doClose() {
-    int count = refCount.decrementAndGet();
-    if (log.isTraceEnabled()) log.trace("close refcount {} {}", this, count);
-    if (count > 0) return;
-    if (count < 0) {
-      log.warn("Too many close [count:{}] on {}", count, this);
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Too many closes on SolrCore");
-    }
     log.info("{} CLOSING SolrCore {}", logid, this);
     this.closing = true;
     searcherReadyLatch.countDown();
@@ -1779,7 +1783,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
    * Whether this core is closed.
    */
   public boolean isClosed() {
-    return refCount.get() < 0;
+    return refCount.get() == 0;
   }
 
   public boolean isClosing() {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCores.java b/solr/core/src/java/org/apache/solr/core/SolrCores.java
index 1290637..9d3c845 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCores.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCores.java
@@ -118,7 +118,7 @@ class SolrCores implements Closeable {
         closer.collect("closeCore-" + core.getName(), () -> {
           MDCLoggingContext.setCore(core);
           try {
-            core.closeAndWait();
+            core.closeAndWait(true);
           } catch (Throwable e) {
             log.error("Error closing SolrCore", e);
             ParWork.propagateInterrupt("Error shutting down core", e);
@@ -133,7 +133,7 @@ class SolrCores implements Closeable {
         closer.collect("closeCore-" + core.getName(), () -> {
           MDCLoggingContext.setCore(core);
           try {
-            core.closeAndWait();
+            core.closeAndWait(true);
           } catch (Throwable e) {
             log.error("Error closing SolrCore", e);
             ParWork.propagateInterrupt("Error shutting down core", e);
@@ -244,13 +244,13 @@ class SolrCores implements Closeable {
       SolrCore c0 = cores.get(n0);
       SolrCore c1 = cores.get(n1);
       if (c0 == null) { // Might be an unloaded transient core
-        c0 = container.getCore(n0, false);
+        c0 = container.getCore(n0);
         if (c0 == null) {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n0);
         }
       }
       if (c1 == null) { // Might be an unloaded transient core
-        c1 = container.getCore(n1, false);
+        c1 = container.getCore(n1);
         if (c1 == null) {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n1);
         }
@@ -294,16 +294,16 @@ class SolrCores implements Closeable {
   }
 
   /* If you don't increment the reference count, someone could close the core before you use it. */
-  SolrCore getCoreFromAnyList(String name, boolean incRefCount) {
+  SolrCore getCoreFromAnyList(String name) {
     if (closed) {
       throw new AlreadyClosedException("SolrCores has been closed");
     }
     SolrCore core = cores.get(name);
-    if (core == null && getTransientCacheHandler() != null) {
+    if (core == null && residentDesciptors.get(name) != null && residentDesciptors.get(name).isTransient() &&  getTransientCacheHandler() != null) {
       core = getTransientCacheHandler().getCore(name);
     }
 
-    if (core != null && incRefCount) {
+    if (core != null) {
       core.open();
     }
 
diff --git a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
index 660f356..c4847a0 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
@@ -217,7 +217,7 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
       CoreDescriptor cd = new CoreDescriptor(newCoreName, testSolrHome.resolve(newCoreName), coreContainer);
       c = new SolrCore(coreContainer, cd,
           new ConfigSet("fakeConfigset", config, schema, null, true));
-      assertNull(coreContainer.registerCore(cd, c, false, false));
+      assertNull(coreContainer.registerCore(cd, c, false, false, true));
       h.coreName = newCoreName;
       assertEquals("We are not using the correct core", "solrconfig_codec2.xml", h.getCore().getConfigResource());
       assertU(add(doc("string_f", "foo")));
diff --git a/solr/test-framework/src/resources/logconf/log4j2-startup-debug.xml b/solr/test-framework/src/resources/logconf/log4j2-startup-debug.xml
index daf2060..556702e 100644
--- a/solr/test-framework/src/resources/logconf/log4j2-startup-debug.xml
+++ b/solr/test-framework/src/resources/logconf/log4j2-startup-debug.xml
@@ -63,6 +63,7 @@
         <AsyncLogger name="org.apache.solr.cloud.overseer.SliceMutator" level="DEBUG"/>-->
         <AsyncLogger name="org.apache.solr.client.solrj.impl.LBSolrClient" level="DEBUG"/>
         <AsyncLogger name="org.apache.solr.cloud.ZkController" level="DEBUG"/>
+        <AsyncLogger name="org.apache.solr.core.SolrCore" level="DEBUG"/>
         <AsyncLogger name="org.apache.solr.common.cloud.ZkMaintenanceUtils" level="DEBUG"/>
         <AsyncLogger name="org.apache.solr.update.processor.DistributedZkUpdateProcessor" level="WARN"/>
         <AsyncLogger name="com.google.inject.servlet" level="DEBUG"/>