You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2020/04/04 02:08:47 UTC

[lucene-solr] branch branch_8x updated: SOLR-14351: Fix/improve MDCLoggingContext usage * Some set/clear were not balanced. * Harden clear() in case of imbalance. * Sometimes coreContainger.getCore was called unnecessarily; just need a descriptor * SolrCore.open/close now calls MDCLoggerContext.setCore/clear * no need to clear MDC in HttpSolrCall

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new fd96a6c  SOLR-14351: Fix/improve MDCLoggingContext usage * Some set/clear were not balanced. * Harden clear() in case of imbalance. * Sometimes coreContainger.getCore was called unnecessarily; just need a descriptor * SolrCore.open/close now calls MDCLoggerContext.setCore/clear * no need to clear MDC in HttpSolrCall
fd96a6c is described below

commit fd96a6c8cd1380310d69d28e48e57954bfa8c41b
Author: David Smiley <ds...@apache.org>
AuthorDate: Fri Mar 20 16:47:42 2020 -0400

    SOLR-14351: Fix/improve MDCLoggingContext usage
    * Some set/clear were not balanced.
    * Harden clear() in case of imbalance.
    * Sometimes coreContainger.getCore was called unnecessarily; just need a descriptor
    * SolrCore.open/close now calls MDCLoggerContext.setCore/clear
    * no need to clear MDC in HttpSolrCall
---
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/api/V2HttpCall.java   |  20 ++--
 .../org/apache/solr/cloud/RecoveryStrategy.java    |   4 -
 .../java/org/apache/solr/cloud/SyncStrategy.java   |  34 +++---
 .../java/org/apache/solr/cloud/ZkController.java   | 120 ++++++++++-----------
 .../src/java/org/apache/solr/core/SolrCore.java    |   3 +-
 .../src/java/org/apache/solr/core/ZkContainer.java |  16 +--
 .../org/apache/solr/logging/MDCLoggingContext.java |  78 +++++++-------
 .../org/apache/solr/metrics/SolrMetricManager.java |   5 +-
 .../java/org/apache/solr/pkg/PackageListeners.java |   5 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   3 -
 .../src/java/org/apache/solr/update/PeerSync.java  |   3 -
 .../org/apache/solr/update/PeerSyncWithLeader.java |   3 -
 13 files changed, 134 insertions(+), 162 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7f8bab0..15c42bf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -81,6 +81,8 @@ Other Changes
 
 * SOLR-14367: Upgrade Tika to 1.24 (Mibo via Erick Erickson)
 
+* SOLR-14351: Hardened MDC logging; sometimes info was absent (David Smiley)
+
 ==================  8.5.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
index 64ef839..5dd89a3 100644
--- a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
+++ b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
@@ -17,12 +17,8 @@
 
 package org.apache.solr.api;
 
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.util.PathTrie.getPathSegments;
-import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
-import static org.apache.solr.servlet.SolrDispatchFilter.Action.PROCESS;
-import static org.apache.solr.servlet.SolrDispatchFilter.Action.REMOTEQUERY;
-
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -33,9 +29,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.function.Supplier;
 
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
+import com.google.common.collect.ImmutableSet;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.SolrThreadSafe;
@@ -49,7 +43,6 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.RequestHandlerUtils;
-import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
@@ -60,7 +53,11 @@ import org.apache.solr.servlet.SolrRequestParsers;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.ImmutableSet;
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.util.PathTrie.getPathSegments;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.PROCESS;
+import static org.apache.solr.servlet.SolrDispatchFilter.Action.REMOTEQUERY;
 
 // class that handle the '/v2' path
 @SolrThreadSafe
@@ -151,7 +148,6 @@ public class V2HttpCall extends HttpSolrCall {
       } else {
         api = apiInfo == null ? api : apiInfo;
       }
-      MDCLoggingContext.setCore(core);
       parseRequest();
 
       addCollectionParamIfNeeded(getCollectionsList());
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 038494a..8ce8886 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -54,7 +54,6 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ReplicationHandler;
-import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
@@ -310,7 +309,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
         SolrException.log(log, "SolrCore not found - cannot recover:" + coreName);
         return;
       }
-      MDCLoggingContext.setCore(core);
 
       log.info("Starting recovery process. recoveringAfterStartup=" + recoveringAfterStartup);
 
@@ -324,8 +322,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
         log.error("", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
       }
-    } finally {
-      MDCLoggingContext.clear();
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
index 2391414..cf840d9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
@@ -39,7 +39,6 @@ import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
-import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.update.PeerSync;
 import org.apache.solr.update.UpdateShardHandler;
 import org.slf4j.Logger;
@@ -89,27 +88,22 @@ public class SyncStrategy {
     if (SKIP_AUTO_RECOVERY) {
       return PeerSync.PeerSyncResult.success();
     }
-    
-    MDCLoggingContext.setCore(core);
-    try {
-      if (isClosed) {
-        log.warn("Closed, skipping sync up.");
-        return PeerSync.PeerSyncResult.failure();
-      }
-      
-      recoveryRequests.clear();
-      
-      log.info("Sync replicas to " + ZkCoreNodeProps.getCoreUrl(leaderProps));
-      
-      if (core.getUpdateHandler().getUpdateLog() == null) {
-        log.error("No UpdateLog found - cannot sync");
-        return PeerSync.PeerSyncResult.failure();
-      }
 
-      return syncReplicas(zkController, core, leaderProps, peerSyncOnlyWithActive);
-    } finally {
-      MDCLoggingContext.clear();
+    if (isClosed) {
+      log.warn("Closed, skipping sync up.");
+      return PeerSync.PeerSyncResult.failure();
     }
+
+    recoveryRequests.clear();
+
+    log.info("Sync replicas to " + ZkCoreNodeProps.getCoreUrl(leaderProps));
+
+    if (core.getUpdateHandler().getUpdateLog() == null) {
+      log.error("No UpdateLog found - cannot sync");
+      return PeerSync.PeerSyncResult.failure();
+    }
+
+    return syncReplicas(zkController, core, leaderProps, peerSyncOnlyWithActive);
   }
   
   private PeerSync.PeerSyncResult syncReplicas(ZkController zkController, SolrCore core,
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 515198c..5d4c770 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -298,8 +298,8 @@ public class ZkController implements Closeable {
     this.genericCoreNodeNames = cloudConfig.getGenericCoreNodeNames();
 
     // be forgiving and strip this off leading/trailing slashes
-    // this allows us to support users specifying hostContext="/" in 
-    // solr.xml to indicate the root context, instead of hostContext="" 
+    // this allows us to support users specifying hostContext="/" in
+    // solr.xml to indicate the root context, instead of hostContext=""
     // which means the default of "solr"
     String localHostContext = trimLeadingAndTrailingSlashes(cloudConfig.getSolrHostContext());
 
@@ -351,7 +351,7 @@ public class ZkController implements Closeable {
 
               // seems we dont need to do this again...
               // Overseer.createClientNodes(zkClient, getNodeName());
-              
+
               // start the overseer first as following code may need it's processing
               if (!zkRunOnly) {
                 ElectionContext context = new OverseerElectionContext(zkClient,
@@ -470,7 +470,7 @@ public class ZkController implements Closeable {
     });
 
     init(registerOnReconnect);
-    
+
     this.overseerJobQueue = overseer.getStateUpdateQueue();
     this.overseerCollectionQueue = overseer.getCollectionQueue(zkClient);
     this.overseerConfigSetQueue = overseer.getConfigSetQueue(zkClient);
@@ -496,7 +496,7 @@ public class ZkController implements Closeable {
     if (descriptors != null) {
       // before registering as live, make sure everyone is in a
       // down state
-      publishNodeAsDown(getNodeName()); 
+      publishNodeAsDown(getNodeName());
       for (CoreDescriptor descriptor : descriptors) {
         // if it looks like we are going to be the leader, we don't
         // want to wait for the following stuff
@@ -547,7 +547,7 @@ public class ZkController implements Closeable {
   }
 
   private void closeOutstandingElections(final CurrentCoreDescriptorProvider registerOnReconnect) {
-    
+
     List<CoreDescriptor> descriptors = registerOnReconnect.getCurrentDescriptors();
     if (descriptors != null) {
       for (CoreDescriptor descriptor : descriptors) {
@@ -555,20 +555,20 @@ public class ZkController implements Closeable {
       }
     }
   }
-  
+
   private ContextKey closeExistingElectionContext(CoreDescriptor cd) {
     // look for old context - if we find it, cancel it
     String collection = cd.getCloudDescriptor().getCollectionName();
     final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-    
+
     ContextKey contextKey = new ContextKey(collection, coreNodeName);
     ElectionContext prevContext = electionContexts.get(contextKey);
-    
+
     if (prevContext != null) {
       prevContext.close();
       electionContexts.remove(contextKey);
     }
-    
+
     return contextKey;
   }
 
@@ -1028,7 +1028,7 @@ public class ZkController implements Closeable {
   InterruptedException {
     publishAndWaitForDownStates(WAIT_DOWN_STATES_TIMEOUT_SECONDS);
   }
-  
+
   public void publishAndWaitForDownStates(int timeoutSeconds) throws KeeperException,
       InterruptedException {
 
@@ -1130,7 +1130,7 @@ public class ZkController implements Closeable {
     List<Op> ops = new ArrayList<>(2);
     ops.add(Op.delete(nodePath, -1));
     ops.add(Op.delete(nodeAddedPath, -1));
- 
+
     try {
       zkClient.multi(ops, true);
     } catch (NoNodeException e) {
@@ -1168,9 +1168,7 @@ public class ZkController implements Closeable {
    */
   public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores,
                          boolean afterExpiration, boolean skipRecovery) throws Exception {
-    try (SolrCore core = cc.getCore(desc.getName())) {
-      MDCLoggingContext.setCore(core);
-    }
+    MDCLoggingContext.setCoreDescriptor(cc, desc);
     try {
       // pre register has published our down state
       final String baseUrl = getBaseUrl();
@@ -1221,25 +1219,25 @@ public class ZkController implements Closeable {
       } catch (KeeperException | IOException e) {
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
       }
-      
+
       // in this case, we want to wait for the leader as long as the leader might
       // wait for a vote, at least - but also long enough that a large cluster has
       // time to get its act together
       String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
-      
+
       String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
       log.debug("We are " + ourUrl + " and leader is " + leaderUrl);
       boolean isLeader = leaderUrl.equals(ourUrl);
       assert !(isLeader && replica.getType() == Type.PULL) : "Pull replica became leader!";
 
       try (SolrCore core = cc.getCore(desc.getName())) {
-        
+
         // recover from local transaction log and wait for it to complete before
         // going active
         // TODO: should this be moved to another thread? To recoveryStrat?
         // TODO: should this actually be done earlier, before (or as part of)
         // leader election perhaps?
-        
+
         if (core == null) {
           throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "SolrCore is no longer available to register");
         }
@@ -1287,7 +1285,7 @@ public class ZkController implements Closeable {
         unregister(coreName, desc, false);
         throw e;
       }
-      
+
       // make sure we have an update cluster state right away
       zkStateReader.forceUpdateCollection(collection);
       // the watcher is added to a set so multiple calls of this method will left only one watcher
@@ -1377,7 +1375,7 @@ public class ZkController implements Closeable {
             .getCoreUrl();
       }
 
-    } catch (AlreadyClosedException e) { 
+    } catch (AlreadyClosedException e) {
       throw e;
     } catch (Exception e) {
       log.error("Error getting leader from zk", e);
@@ -1524,14 +1522,12 @@ public class ZkController implements Closeable {
         if (core == null || core.isClosed()) {
           return;
         }
-        MDCLoggingContext.setCore(core);
       }
-    } else {
-      MDCLoggingContext.setCoreDescriptor(cc, cd);
     }
+    MDCLoggingContext.setCoreDescriptor(cc, cd);
     try {
       String collection = cd.getCloudDescriptor().getCollectionName();
-      
+
       log.debug("publishing state={}", state.toString());
       // System.out.println(Thread.currentThread().getStackTrace()[3]);
       Integer numShards = cd.getCloudDescriptor().getNumShards();
@@ -1539,11 +1535,11 @@ public class ZkController implements Closeable {
         log.debug("numShards not found on descriptor - reading it from system property");
         numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP);
       }
-      
+
       assert collection != null && collection.length() > 0;
-      
+
       String shardId = cd.getCloudDescriptor().getShardId();
-      
+
       String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
 
       Map<String,Object> props = new HashMap<>();
@@ -1595,7 +1591,7 @@ public class ZkController implements Closeable {
       }
 
       ZkNodeProps m = new ZkNodeProps(props);
-      
+
       if (updateLastState) {
         cd.getCloudDescriptor().setLastPublished(state);
       }
@@ -1871,7 +1867,7 @@ public class ZkController implements Closeable {
       CoreDescriptor descriptor, final String coreZkNodeName) throws SessionExpiredException {
     // try not to wait too long here - if we are waiting too long, we should probably
     // move along and join the election
-    
+
     CloudDescriptor cloudDesc = descriptor.getCloudDescriptor();
     String collection = cloudDesc.getCollectionName();
     String shard = cloudDesc.getShardId();
@@ -2073,7 +2069,7 @@ public class ZkController implements Closeable {
    * has been reserved for the operation, meaning that no other thread/operation can claim
    * it. If for whatever reason, the operation is not scheduled, the asuncId needs to be
    * cleared using {@link #clearAsyncId(String)}.
-   * If this method returns false, no reservation has been made, and this asyncId can't 
+   * If this method returns false, no reservation has been made, and this asyncId can't
    * be used, since it's being used by another operation (currently or in the past)
    * @param asyncId A string representing the asyncId of an operation. Can't be null.
    * @return True if the reservation succeeds.
@@ -2088,7 +2084,7 @@ public class ZkController implements Closeable {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * Clears an asyncId previously claimed by calling {@link #claimAsyncId(String)}
    * @param asyncId A string representing the asyncId of an operation. Can't be null.
@@ -2189,8 +2185,8 @@ public class ZkController implements Closeable {
   }
 
   public void rejoinShardLeaderElection(SolrParams params) {
-    try {
-      
+
+
       String collectionName = params.get(COLLECTION_PROP);
       String shardId = params.get(SHARD_ID_PROP);
       String coreNodeName = params.get(CORE_NODE_NAME_PROP);
@@ -2198,32 +2194,32 @@ public class ZkController implements Closeable {
       String electionNode = params.get(ELECTION_NODE_PROP);
       String baseUrl = params.get(BASE_URL_PROP);
 
-      try (SolrCore core = cc.getCore(coreName)) {
-        MDCLoggingContext.setCore(core);
-        
-        log.info("Rejoin the shard leader election.");
-        
-        ContextKey contextKey = new ContextKey(collectionName, coreNodeName);
-        
-        ElectionContext prevContext = electionContexts.get(contextKey);
-        if (prevContext != null) prevContext.cancelElection();
-        
-        ZkNodeProps zkProps = new ZkNodeProps(BASE_URL_PROP, baseUrl, CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName);
-            
-        LeaderElector elect = ((ShardLeaderElectionContextBase) prevContext).getLeaderElector();
-        ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
-            coreNodeName, zkProps, this, getCoreContainer());
-            
-        context.leaderSeqPath = context.electionPath + LeaderElector.ELECTION_NODE + "/" + electionNode;
-        elect.setup(context);
-        electionContexts.put(contextKey, context);
-        
-        elect.retryElection(context, params.getBool(REJOIN_AT_HEAD_PROP, false));
-      }
+    try {
+      MDCLoggingContext.setCoreDescriptor(cc, cc.getCoreDescriptor(coreName));
+
+      log.info("Rejoin the shard leader election.");
+
+      ContextKey contextKey = new ContextKey(collectionName, coreNodeName);
+
+      ElectionContext prevContext = electionContexts.get(contextKey);
+      if (prevContext != null) prevContext.cancelElection();
+
+      ZkNodeProps zkProps = new ZkNodeProps(BASE_URL_PROP, baseUrl, CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName);
+
+      LeaderElector elect = ((ShardLeaderElectionContextBase) prevContext).getLeaderElector();
+      ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
+          coreNodeName, zkProps, this, getCoreContainer());
+
+      context.leaderSeqPath = context.electionPath + LeaderElector.ELECTION_NODE + "/" + electionNode;
+      elect.setup(context);
+      electionContexts.put(contextKey, context);
+
+      elect.retryElection(context, params.getBool(REJOIN_AT_HEAD_PROP, false));
     } catch (Exception e) {
       throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to rejoin election", e);
+    } finally {
+      MDCLoggingContext.clear();
     }
-
   }
 
   public void checkOverseerDesignate() {
@@ -2422,7 +2418,7 @@ public class ZkController implements Closeable {
         public void preClose(SolrCore core) {
           unregisterConfListener(confDir, listener);
         }
-        
+
         @Override
         public void postClose(SolrCore core) {
         }
@@ -2614,11 +2610,11 @@ public class ZkController implements Closeable {
     }
     return false;
   }
-  
-  
+
+
   /**
    * Best effort to set DOWN state for all replicas on node.
-   * 
+   *
    * @param nodeName to operate on
    */
   public void publishNodeAsDown(String nodeName) {
@@ -2634,7 +2630,7 @@ public class ZkController implements Closeable {
       log.debug("Publish node as down was interrupted.");
     } catch (KeeperException e) {
       log.warn("Could not publish node as down: " + e.getMessage());
-    } 
+    }
   }
 
   /**
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 630a8c4..5cd3619 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -931,7 +931,6 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
       coreContainer.solrCores.addCoreDescriptor(cd);
 
       setName(name);
-      MDCLoggingContext.setCore(this);
 
       this.solrConfig = configSet.getSolrConfig();
       this.resourceLoader = configSet.getSolrConfig().getResourceLoader();
@@ -1502,6 +1501,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
    */
   public void open() {
     refCount.incrementAndGet();
+    MDCLoggingContext.setCore(this);
   }
 
   /**
@@ -1531,6 +1531,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
    */
   @Override
   public void close() {
+    MDCLoggingContext.clear(); // balance out open with close
     int count = refCount.decrementAndGet();
     if (count > 0) return; // close is called often, and only actually closes if nothing is using it.
     if (count < 0) {
diff --git a/solr/core/src/java/org/apache/solr/core/ZkContainer.java b/solr/core/src/java/org/apache/solr/core/ZkContainer.java
index 3696df5..668ed63 100644
--- a/solr/core/src/java/org/apache/solr/core/ZkContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/ZkContainer.java
@@ -215,18 +215,12 @@ public class ZkContainer {
       }
     };
 
-    if (zkController != null) {
-      if (background) {
-        coreZkRegister.execute(r);
-      } else {
-        MDCLoggingContext.setCore(core);
-        try {
-          r.run();
-        } finally {
-          MDCLoggingContext.clear();
-        }
-      }
+    if (background) {
+      coreZkRegister.execute(r);
+    } else {
+      r.run();
     }
+
   }
   
   public ZkController getZkController() {
diff --git a/solr/core/src/java/org/apache/solr/logging/MDCLoggingContext.java b/solr/core/src/java/org/apache/solr/logging/MDCLoggingContext.java
index 53f751d..492735b 100644
--- a/solr/core/src/java/org/apache/solr/logging/MDCLoggingContext.java
+++ b/solr/core/src/java/org/apache/solr/logging/MDCLoggingContext.java
@@ -16,14 +16,6 @@
  */
 package org.apache.solr.logging;
 
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
-
-import java.util.function.Supplier;
-
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.StringUtils;
@@ -32,6 +24,12 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.slf4j.MDC;
 
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+
 /**
  * Set's per thread context info for logging. Nested calls will use the top level parent for all context. The first
  * caller always owns the context until it calls {@link #clear()}. Always call {@link #setCore(SolrCore)} or
@@ -40,13 +38,8 @@ import org.slf4j.MDC;
 public class MDCLoggingContext {
   public static final String TRACE_ID = "trace_id";
   // When a thread sets context and finds that the context is already set, we should noop and ignore the finally clear
-  private static ThreadLocal<Integer> CALL_DEPTH = ThreadLocal.withInitial(new Supplier<Integer>() {
-    @Override
-    public Integer get() {
-      return 0;
-    }
-  });
-  
+  private static ThreadLocal<Integer> CALL_DEPTH = ThreadLocal.withInitial(() -> 0);
+
   public static void setCollection(String collection) {
     if (collection != null) {
       MDC.put(COLLECTION_PROP, "c:" + collection);
@@ -112,28 +105,34 @@ public class MDCLoggingContext {
       MDC.remove(NODE_NAME_PROP);
     }
   }
-  
+
+  /**
+   * Sets multiple information from the params.
+   * REMEMBER TO CALL {@link #clear()} in a finally!
+   */
   public static void setCore(SolrCore core) {
-    if (core != null) {
-      setCoreDescriptor(core.getCoreContainer(), core.getCoreDescriptor());
-    }
+    CoreContainer coreContainer = core == null ? null : core.getCoreContainer();
+    CoreDescriptor coreDescriptor = core == null ? null : core.getCoreDescriptor();
+    setCoreDescriptor(coreContainer, coreDescriptor);
   }
-  
+
+  /**
+   * Sets multiple information from the params.
+   * REMEMBER TO CALL {@link #clear()} in a finally!
+   */
   public static void setCoreDescriptor(CoreContainer coreContainer, CoreDescriptor cd) {
+    setNode(coreContainer);
+
+    int callDepth = CALL_DEPTH.get();
+    CALL_DEPTH.set(callDepth + 1);
+    if (callDepth > 0) {
+      return;
+    }
+
     if (cd != null) {
-      int callDepth = CALL_DEPTH.get();
-      CALL_DEPTH.set(callDepth + 1);
-      if (callDepth > 0) {
-        return;
-      }
-      
+
+      assert cd.getName() != null;
       setCoreName(cd.getName());
-      if (coreContainer != null) {
-        ZkController zkController = coreContainer.getZkController();
-        if (zkController != null) {
-          setNodeName(zkController.getNodeName());
-        }
-      }
       
       CloudDescriptor ccd = cd.getCloudDescriptor();
       if (ccd != null) {
@@ -143,15 +142,21 @@ public class MDCLoggingContext {
       }
     }
   }
-  
+
+  /**
+   * Call this after {@link #setCore(SolrCore)} or {@link #setCoreDescriptor(CoreContainer, CoreDescriptor)} in a
+   * finally.
+   */
   public static void clear() {
     int used = CALL_DEPTH.get();
-    CALL_DEPTH.set(used - 1);
-    if (used == 0) {
+    if (used <= 1) {
+      CALL_DEPTH.set(0);
       MDC.remove(COLLECTION_PROP);
       MDC.remove(CORE_NAME_PROP);
       MDC.remove(REPLICA_PROP);
       MDC.remove(SHARD_ID_PROP);
+    } else {
+      CALL_DEPTH.set(used - 1);
     }
   }
   
@@ -163,7 +168,8 @@ public class MDCLoggingContext {
     MDC.remove(NODE_NAME_PROP);
     MDC.remove(TRACE_ID);
   }
-  
+
+  /** Resets to a cleared state.  Used in-between requests into Solr. */
   public static void reset() {
     CALL_DEPTH.set(0);
     removeAll();
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index e28bf32..9d6ab8e 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -967,10 +967,7 @@ public class SolrMetricManager {
         new Object[]{this, registry}
     );
     // prepare MDC for plugins that want to use its properties
-    MDCLoggingContext.setNode(coreContainer);
-    if (solrCore != null) {
-      MDCLoggingContext.setCore(solrCore);
-    }
+    MDCLoggingContext.setCoreDescriptor(coreContainer, solrCore == null ? null : solrCore.getCoreDescriptor());
     if (tag != null) {
       // add instance tag to MDC
       MDC.put("tag", "t:" + tag);
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java b/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
index 0287f5e..a483514 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
@@ -63,14 +63,13 @@ public class PackageListeners {
   }
 
   synchronized void packagesUpdated(List<PackageLoader.Package> pkgs) {
-    if(core != null) MDCLoggingContext.setCore(core);
+    MDCLoggingContext.setCore(core);
     try {
       for (PackageLoader.Package pkgInfo : pkgs) {
         invokeListeners(pkgInfo);
       }
     } finally {
-      if(core != null) MDCLoggingContext.clear();
-
+      MDCLoggingContext.clear();
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 44418e4..245baf2 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -307,7 +307,6 @@ public class HttpSolrCall {
 
     // With a valid core...
     if (core != null) {
-      MDCLoggingContext.setCore(core);
       config = core.getSolrConfig();
       // get or create/cache the parser for the core
       SolrRequestParsers parser = config.getRequestParsers();
@@ -614,8 +613,6 @@ public class HttpSolrCall {
         t = t.getCause();
       }
       return RETURN;
-    } finally {
-      MDCLoggingContext.clear();
     }
 
   }
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java
index 84cf2a1..e70c917 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -48,7 +48,6 @@ import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
-import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.LocalSolrQueryRequest;
@@ -166,7 +165,6 @@ public class PeerSync implements SolrMetricProducer {
       syncErrors.inc();
       return PeerSyncResult.failure();
     }
-    MDCLoggingContext.setCore(core);
     Timer.Context timerContext = null;
     try {
       log.info(msg() + "START replicas=" + replicas + " nUpdates=" + nUpdates);
@@ -250,7 +248,6 @@ public class PeerSync implements SolrMetricProducer {
       if (timerContext != null) {
         timerContext.close();
       }
-      MDCLoggingContext.clear();
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java b/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
index ae58662..4e9ae15 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
@@ -37,7 +37,6 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean;
-import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.slf4j.Logger;
@@ -124,7 +123,6 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
       return PeerSync.PeerSyncResult.failure();
     }
 
-    MDCLoggingContext.setCore(core);
     Timer.Context timerContext = null;
     try {
       log.info(msg() + "START leader=" + leaderUrl + " nUpdates=" + nUpdates);
@@ -187,7 +185,6 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
       } catch (IOException e) {
         log.warn("{} unable to close client to leader", msg(), e);
       }
-      MDCLoggingContext.clear();
     }
   }