You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2016/09/22 08:46:40 UTC

[2/2] lucene-solr:branch_6x: SOLR-5563: Quieten down SolrCloud logging

SOLR-5563: Quieten down SolrCloud logging


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

Branch: refs/heads/branch_6x
Commit: 082f8e3f92de67ba694c414e526e622a51d2c056
Parents: 06b0978
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Sep 22 09:41:42 2016 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Sep 22 09:46:20 2016 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +
 .../apache/solr/cloud/CreateCollectionCmd.java  | 11 ++--
 .../org/apache/solr/cloud/LeaderElector.java    | 10 ++--
 .../java/org/apache/solr/cloud/Overseer.java    | 19 +++---
 .../OverseerAutoReplicaFailoverThread.java      |  2 +-
 .../cloud/OverseerCollectionMessageHandler.java | 10 ++--
 .../solr/cloud/OverseerTaskProcessor.java       | 10 ++--
 .../org/apache/solr/cloud/ZkController.java     | 62 +++++++++-----------
 .../cloud/overseer/ClusterStateMutator.java     |  2 +-
 .../solr/cloud/overseer/ReplicaMutator.java     | 18 +++---
 .../solr/cloud/overseer/ZkStateWriter.java      |  6 +-
 .../org/apache/solr/core/CoreDescriptor.java    |  2 +-
 .../solr/common/cloud/ConnectionManager.java    | 12 ++--
 .../apache/solr/common/cloud/SolrZkClient.java  | 15 ++---
 .../apache/solr/common/cloud/ZkStateReader.java | 32 +++++-----
 15 files changed, 99 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4e53c57..2aeb2b0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -131,6 +131,9 @@ Other Changes
  "generate-website-quickstart" to convert the bundled version of the tutorial into one suitable
  for the website.
 
+* SOLR-5563: Move lots of SolrCloud logging from 'info' to 'debug' (janhoy, Alan
+  Woodward)
+
 ==================  6.2.1 ==================
 
 Bug Fixes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index 7f28600..01e3fdf 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -175,7 +175,7 @@ public class CreateCollectionCmd implements Cmd {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not fully create collection: " + collectionName);
 
       if (nodeList.isEmpty()) {
-        log.info("Finished create command for collection: {}", collectionName);
+        log.debug("Finished create command for collection: {}", collectionName);
         return;
       }
 
@@ -183,14 +183,14 @@ public class CreateCollectionCmd implements Cmd {
       Map<String, String> requestMap = new HashMap<>();
 
 
-      log.info(formatString("Creating SolrCores for new collection {0}, shardNames {1} , replicationFactor : {2}",
+      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , replicationFactor : {2}",
           collectionName, shardNames, repFactor));
       Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
       for (Map.Entry<ReplicaAssigner.Position, String> e : positionVsNodes.entrySet()) {
         ReplicaAssigner.Position position = e.getKey();
         String nodeName = e.getValue();
         String coreName = collectionName + "_" + position.shard + "_replica" + (position.index + 1);
-        log.info(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
+        log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
             , coreName, position.shard, collectionName, nodeName));
 
 
@@ -256,10 +256,9 @@ public class CreateCollectionCmd implements Cmd {
         // We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
         // element, which may be interpreted by the user as a positive ack
         ocmh.cleanupCollection(collectionName, new NamedList());
-        log.info("Cleaned up  artifacts for failed create collection for [" + collectionName + "]");
+        log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
       } else {
-        log.debug("Finished create command on all shards for collection: "
-            + collectionName);
+        log.debug("Finished create command on all shards for collection: {}", collectionName);
       }
     } catch (SolrException ex) {
       throw ex;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 719bc5f..71fdcfd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -150,7 +150,7 @@ public  class LeaderElector {
       try {
         String watchedNode = holdElectionPath + "/" + toWatch;
         zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath, watchedNode, getSeq(context.leaderSeqPath), context), null, true);
-        log.info("Watching path {} to know if I could be the leader", watchedNode);
+        log.debug("Watching path {} to know if I could be the leader", watchedNode);
       } catch (KeeperException.SessionExpiredException e) {
         throw e;
       } catch (KeeperException.NoNodeException e) {
@@ -238,14 +238,14 @@ public  class LeaderElector {
     while (cont) {
       try {
         if(joinAtHead){
-          log.info("Node {} trying to join election at the head", id);
+          log.debug("Node {} trying to join election at the head", id);
           List<String> nodes = OverseerTaskProcessor.getSortedElectionNodes(zkClient, shardsElectZkPath);
           if(nodes.size() <2){
             leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
                 CreateMode.EPHEMERAL_SEQUENTIAL, false);
           } else {
             String firstInLine = nodes.get(1);
-            log.info("The current head: {}", firstInLine);
+            log.debug("The current head: {}", firstInLine);
             Matcher m = LEADER_SEQ.matcher(firstInLine);
             if (!m.matches()) {
               throw new IllegalStateException("Could not find regex match in:"
@@ -259,7 +259,7 @@ public  class LeaderElector {
               CreateMode.EPHEMERAL_SEQUENTIAL, false);
         }
 
-        log.info("Joined leadership election with path: {}", leaderSeqPath);
+        log.debug("Joined leadership election with path: {}", leaderSeqPath);
         context.leaderSeqPath = leaderSeqPath;
         cont = false;
       } catch (ConnectionLossException e) {
@@ -333,7 +333,7 @@ public  class LeaderElector {
         return;
       }
       if (canceled) {
-        log.info("This watcher is not active anymore {}", myNode);
+        log.debug("This watcher is not active anymore {}", myNode);
         try {
           zkClient.delete(myNode, -1, true);
         } catch (KeeperException.NoNodeException nne) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 70ebd37..dc431fd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -122,7 +122,7 @@ public class Overseer implements Closeable {
         isLeader = amILeader();  // not a no, not a yes, try ask again
       }
 
-      log.info("Starting to work on the main queue");
+      log.debug("Starting to work on the main queue");
       try {
         ZkStateWriter zkStateWriter = null;
         ClusterState clusterState = null;
@@ -152,7 +152,7 @@ public class Overseer implements Closeable {
               boolean hadWorkItems = data != null;
               while (data != null)  {
                 final ZkNodeProps message = ZkNodeProps.load(data);
-                log.info("processMessage: workQueueSize: {}, message = {}", workQueue.getStats().getQueueLength(), message);
+                log.debug("processMessage: workQueueSize: {}, message = {}", workQueue.getStats().getQueueLength(), message);
                 // force flush to ZK after each message because there is no fallback if workQueue items
                 // are removed from workQueue but fail to be written to ZK
                 clusterState = processQueueItem(message, clusterState, zkStateWriter, false, null);
@@ -182,8 +182,7 @@ public class Overseer implements Closeable {
             head = stateUpdateQueue.peek(true);
           } catch (KeeperException e) {
             if (e.code() == KeeperException.Code.SESSIONEXPIRED) {
-              log.warn(
-                  "Solr cannot talk to ZK, exiting Overseer main queue loop", e);
+              log.warn("Solr cannot talk to ZK, exiting Overseer main queue loop", e);
               return;
             }
             log.error("Exception in Overseer main queue loop", e);
@@ -198,7 +197,7 @@ public class Overseer implements Closeable {
             while (head != null) {
               byte[] data = head;
               final ZkNodeProps message = ZkNodeProps.load(data);
-              log.info("processMessage: queueSize: {}, message = {} current state version: {}", stateUpdateQueue.getStats().getQueueLength(), message, clusterState.getZkClusterStateVersion());
+              log.debug("processMessage: queueSize: {}, message = {} current state version: {}", stateUpdateQueue.getStats().getQueueLength(), message, clusterState.getZkClusterStateVersion());
               // we can batch here because workQueue is our fallback in case a ZK write failed
               clusterState = processQueueItem(message, clusterState, zkStateWriter, true, new ZkStateWriter.ZkWriteCallback() {
                 @Override
@@ -297,7 +296,7 @@ public class Overseer implements Closeable {
         String id = (String) m.get("id");
         if(overseerCollectionConfigSetProcessor.getId().equals(id)){
           try {
-            log.info("I'm exiting , but I'm still the leader");
+            log.warn("I'm exiting, but I'm still the leader");
             zkClient.delete(path,stat.getVersion(),true);
           } catch (KeeperException.BadVersionException e) {
             //no problem ignore it some other Overseer has already taken over
@@ -306,7 +305,7 @@ public class Overseer implements Closeable {
           }
 
         } else{
-          log.info("somebody else has already taken up the overseer position");
+          log.debug("somebody else has already taken up the overseer position");
         }
       } finally {
         //if I am not shutting down, Then I need to rejoin election
@@ -406,9 +405,7 @@ public class Overseer implements Closeable {
         if (e.code() == KeeperException.Code.CONNECTIONLOSS) {
           log.error("", e);
           return LeaderStatus.DONT_KNOW;
-        } else if (e.code() == KeeperException.Code.SESSIONEXPIRED) {
-          log.info("", e);
-        } else {
+        } else if (e.code() != KeeperException.Code.SESSIONEXPIRED) {
           log.warn("", e);
         }
       } catch (InterruptedException e) {
@@ -546,7 +543,7 @@ public class Overseer implements Closeable {
   }
   
   public synchronized void close() {
-    if (closed) return;
+    if (closed || id == null) return;
     log.info("Overseer (id=" + id + ") closing");
     
     doClose();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index 093ec4d..99f0192 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -98,7 +98,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
     this.waitAfterExpiration = config.getAutoReplicaFailoverWaitAfterExpiration();
     int badNodeExpiration = config.getAutoReplicaFailoverBadNodeExpiration();
     
-    log.info(
+    log.debug(
         "Starting "
             + this.getClass().getSimpleName()
             + " autoReplicaFailoverWorkLoopDelay={} autoReplicaFailoverWaitAfterExpiration={} autoReplicaFailoverBadNodeExpiration={}",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 9f14e45..0520488 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -212,7 +212,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   @Override
   @SuppressWarnings("unchecked")
   public SolrResponse processMessage(ZkNodeProps message, String operation) {
-    log.info("OverseerCollectionMessageHandler.processMessage : "+ operation + " , "+ message.toString());
+    log.debug("OverseerCollectionMessageHandler.processMessage : "+ operation + " , "+ message.toString());
 
     NamedList results = new NamedList();
     try {
@@ -480,7 +480,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   }
 
   void commit(NamedList results, String slice, Replica parentShardLeader) {
-    log.info("Calling soft commit to make sub shard updates visible");
+    log.debug("Calling soft commit to make sub shard updates visible");
     String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
     // HttpShardHandler is hard coded to send a QueryRequest hence we go direct
     // and we force open a searcher so that we have documents to show upon switching states
@@ -537,7 +537,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   }
 
   void waitForNewShard(String collectionName, String sliceName) throws KeeperException, InterruptedException {
-    log.info("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
+    log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
     RTimer timer = new RTimer();
     int retryCount = 320;
     while (retryCount-- > 0) {
@@ -548,7 +548,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       }
       Slice slice = collection.getSlice(sliceName);
       if (slice != null) {
-        log.info("Waited for {}ms for slice {} of collection {} to be available",
+        log.debug("Waited for {}ms for slice {} of collection {} to be available",
             timer.getTime(), sliceName, collectionName);
         return;
       }
@@ -808,7 +808,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     
     if (configName != null) {
       String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
-      log.info("creating collections conf node {} ", collDir);
+      log.debug("creating collections conf node {} ", collDir);
       byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
       if (zkStateReader.getZkClient().exists(collDir, true)) {
         zkStateReader.getZkClient().setData(collDir, data, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index b736d1b..9a72fd5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -144,7 +144,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
   @Override
   public void run() {
-    log.info("Process current queue of overseer operations");
+    log.debug("Process current queue of overseer operations");
     LeaderStatus isLeader = amILeader();
     while (isLeader == LeaderStatus.DONT_KNOW) {
       log.debug("am_i_leader unclear {}", isLeader);
@@ -290,7 +290,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
               Thread.currentThread().interrupt();
               continue;
             }
-            log.info(messageHandler.getName() + ": Get the message id:" + head.getId() + " message:" + message.toString());
+            log.debug(messageHandler.getName() + ": Get the message id:" + head.getId() + " message:" + message.toString());
             Runner runner = new Runner(messageHandler, message,
                 operation, head, lock);
             tpe.execute(runner);
@@ -393,9 +393,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
       if (e.code() == KeeperException.Code.CONNECTIONLOSS) {
         log.error("", e);
         return LeaderStatus.DONT_KNOW;
-      } else if (e.code() == KeeperException.Code.SESSIONEXPIRED) {
-        log.info("", e);
-      } else {
+      } else if (e.code() != KeeperException.Code.SESSIONEXPIRED) {
         log.warn("", e);
       }
     } catch (InterruptedException e) {
@@ -486,7 +484,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
         log.debug("Marked task [{}] as completed.", head.getId());
         printTrackingMaps();
 
-        log.info(messageHandler.getName() + ": Message id:" + head.getId() +
+        log.debug(messageHandler.getName() + ": Message id:" + head.getId() +
             " complete, response:" + response.getResponse().toString());
         success = true;
       } catch (KeeperException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
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 f613141..1e9cc3d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -792,7 +792,7 @@ public class ZkController {
     if (!SolrZkClient.containsChroot(zkHost)) {
       return true;
     }
-    log.info("zkHost includes chroot");
+    log.trace("zkHost includes chroot");
     String chrootPath = zkHost.substring(zkHost.indexOf("/"), zkHost.length());
 
     SolrZkClient tmpClient = new SolrZkClient(zkHost.substring(0,
@@ -870,10 +870,8 @@ public class ZkController {
       props.put(ZkStateReader.CORE_NAME_PROP, coreName);
       props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
       
-      if (log.isInfoEnabled()) {
-        log.info("Register replica - core:" + coreName + " address:" + baseUrl + " collection:"
-            + cloudDesc.getCollectionName() + " shard:" + shardId);
-      }
+      log.debug("Register replica - core:{} address:{} collection:{} shard:{}",
+          coreName, baseUrl, cloudDesc.getCollectionName(), shardId);
       
       ZkNodeProps leaderProps = new ZkNodeProps(props);
       
@@ -900,7 +898,7 @@ public class ZkController {
       String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
       
       String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
-      log.info("We are " + ourUrl + " and leader is " + leaderUrl);
+      log.debug("We are " + ourUrl + " and leader is " + leaderUrl);
       boolean isLeader = leaderUrl.equals(ourUrl);
       
       try (SolrCore core = cc.getCore(desc.getName())) {
@@ -926,7 +924,7 @@ public class ZkController {
               // TODO: public as recovering in the mean time?
               // TODO: in the future we could do peersync in parallel with recoverFromLog
             } else {
-              log.info("No LogReplay needed for core=" + core.getName() + " baseURL=" + baseUrl);
+              log.debug("No LogReplay needed for core={} baseURL={}", core.getName(), baseUrl);
             }
           }
         }
@@ -1148,11 +1146,11 @@ public class ZkController {
     try {
       String collection = cd.getCloudDescriptor().getCollectionName();
       
-      log.info("publishing state={}", state.toString());
+      log.debug("publishing state={}", state.toString());
       // System.out.println(Thread.currentThread().getStackTrace()[3]);
       Integer numShards = cd.getCloudDescriptor().getNumShards();
       if (numShards == null) { // XXX sys prop hack
-        log.info("numShards not found on descriptor - reading it from system property");
+        log.debug("numShards not found on descriptor - reading it from system property");
         numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP);
       }
       
@@ -1278,12 +1276,12 @@ public class ZkController {
   public void createCollectionZkNode(CloudDescriptor cd) {
     String collection = cd.getCollectionName();
 
-    log.info("Check for collection zkNode:" + collection);
+    log.debug("Check for collection zkNode:" + collection);
     String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
 
     try {
       if (!zkClient.exists(collectionPath, true)) {
-        log.info("Creating collection in ZooKeeper:" + collection);
+        log.debug("Creating collection in ZooKeeper:" + collection);
 
         try {
           Map<String, Object> collectionProps = new HashMap<>();
@@ -1335,7 +1333,7 @@ public class ZkController {
           }
         }
       } else {
-        log.info("Collection zkNode exists");
+        log.debug("Collection zkNode exists");
       }
 
     } catch (KeeperException e) {
@@ -1356,7 +1354,7 @@ public class ZkController {
                            Map<String, Object> collectionProps) throws KeeperException,
       InterruptedException {
     // check for configName
-    log.info("Looking for collection configName");
+    log.debug("Looking for collection configName");
     List<String> configNames = null;
     int retry = 1;
     int retryLimt = 6;
@@ -1417,7 +1415,7 @@ public class ZkController {
 
   private void waitForCoreNodeName(CoreDescriptor descriptor) {
     int retryCount = 320;
-    log.info("look for our core node name");
+    log.debug("look for our core node name");
     while (retryCount-- > 0) {
       Map<String, Slice> slicesMap = zkStateReader.getClusterState()
           .getSlicesMap(descriptor.getCloudDescriptor().getCollectionName());
@@ -1450,7 +1448,7 @@ public class ZkController {
   }
 
   private void waitForShardId(CoreDescriptor cd) {
-    log.info("waiting to find shard id in clusterstate for " + cd.getName());
+    log.debug("waiting to find shard id in clusterstate for " + cd.getName());
     int retryCount = 320;
     while (retryCount-- > 0) {
       final String shardId = zkStateReader.getClusterState().getShardId(cd.getCollectionName(), getNodeName(), cd.getName());
@@ -1499,7 +1497,7 @@ public class ZkController {
       publish(cd, Replica.State.DOWN, false, true);
       String collectionName = cd.getCloudDescriptor().getCollectionName();
       DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
-      log.info(collection == null ?
+      log.debug(collection == null ?
               "Collection {} not visible yet, but flagging it so a watch is registered when it becomes visible" :
               "Registering watch for collection {}",
           collectionName);
@@ -1612,7 +1610,7 @@ public class ZkController {
       }
 
       if (lirState != null) {
-        log.info("Replica " + myCoreNodeName +
+        log.debug("Replica " + myCoreNodeName +
             " is already in leader-initiated recovery, so not waiting for leader to see down state.");
       } else {
 
@@ -1680,9 +1678,7 @@ public class ZkController {
 
   public static void linkConfSet(SolrZkClient zkClient, String collection, String confSetName) throws KeeperException, InterruptedException {
     String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
-    if (log.isInfoEnabled()) {
-      log.info("Load collection config from:" + path);
-    }
+    log.debug("Load collection config from:" + path);
     byte[] data;
     try {
       data = zkClient.getData(path, null, null, true);
@@ -2118,7 +2114,7 @@ public class ZkController {
           zkClient.makePath(znodePath, znodeData, retryOnConnLoss);
         }
       }
-      log.info("Wrote {} to {}", state.toString(), znodePath);
+      log.debug("Wrote {} to {}", state.toString(), znodePath);
     } catch (Exception exc) {
       if (exc instanceof SolrException) {
         throw (SolrException) exc;
@@ -2214,7 +2210,7 @@ public class ZkController {
     if (listener != null) {
       synchronized (reconnectListeners) {
         reconnectListeners.add(listener);
-        log.info("Added new OnReconnect listener "+listener);
+        log.debug("Added new OnReconnect listener "+listener);
       }
     }
   }
@@ -2229,7 +2225,7 @@ public class ZkController {
         wasRemoved = reconnectListeners.remove(listener);
       }
       if (wasRemoved) {
-        log.info("Removed OnReconnect listener "+listener);
+        log.debug("Removed OnReconnect listener "+listener);
       } else {
         log.warn("Was asked to remove OnReconnect listener "+listener+
             ", but remove operation did not find it in the list of registered listeners.");
@@ -2273,7 +2269,7 @@ public class ZkController {
           } catch (KeeperException.NodeExistsException nee) {
             try {
               Stat stat = zkClient.exists(resourceLocation, null, true);
-              log.info("failed to set data version in zk is {} and expected version is {} ", stat.getVersion(), znodeVersion);
+              log.debug("failed to set data version in zk is {} and expected version is {} ", stat.getVersion(), znodeVersion);
             } catch (Exception e1) {
               log.warn("could not get stat");
             }
@@ -2337,11 +2333,11 @@ public class ZkController {
         return;
       }
       if (listeners.remove(listener)) {
-        log.info("removed listener for config directory [{}]", confDir);
+        log.debug("removed listener for config directory [{}]", confDir);
       }
       if (listeners.isEmpty()) {
         // no more listeners for this confDir, remove it from the map
-        log.info("No more listeners for config directory [{}]", confDir);
+        log.debug("No more listeners for config directory [{}]", confDir);
         confDirectoryListeners.remove(confDir);
       }
     }
@@ -2378,7 +2374,7 @@ public class ZkController {
     assert Thread.holdsLock(confDirectoryListeners) : "confDirListeners lock not held by thread";
     Set<Runnable> confDirListeners = confDirectoryListeners.get(confDir);
     if (confDirListeners == null) {
-      log.info("watch zkdir {}" , confDir);
+      log.debug("watch zkdir {}" , confDir);
       confDirListeners = new HashSet<>();
       confDirectoryListeners.put(confDir, confDirListeners);
       setConfWatcher(confDir, new WatcherImpl(confDir), null);
@@ -2416,10 +2412,10 @@ public class ZkController {
         resetWatcher = fireEventListeners(zkDir);
       } finally {
         if (Event.EventType.None.equals(event.getType())) {
-          log.info("A node got unwatched for {}", zkDir);
+          log.debug("A node got unwatched for {}", zkDir);
         } else {
           if (resetWatcher) setConfWatcher(zkDir, this, stat);
-          else log.info("A node got unwatched for {}", zkDir);
+          else log.debug("A node got unwatched for {}", zkDir);
         }
       }
     }
@@ -2429,7 +2425,7 @@ public class ZkController {
     synchronized (confDirectoryListeners) {
       // if this is not among directories to be watched then don't set the watcher anymore
       if (!confDirectoryListeners.containsKey(zkDir)) {
-        log.info("Watcher on {} is removed ", zkDir);
+        log.debug("Watcher on {} is removed ", zkDir);
         return false;
       }
       final Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
@@ -2437,7 +2433,7 @@ public class ZkController {
         final Set<Runnable> listenersCopy = new HashSet<>(listeners);
         // run these in a separate thread because this can be long running
         new Thread(() -> {
-          log.info("Running listeners for {}", zkDir);
+          log.debug("Running listeners for {}", zkDir);
           for (final Runnable listener : listenersCopy) {
             try {
               listener.run();
@@ -2526,9 +2522,9 @@ public class ZkController {
       Overseer.getStateUpdateQueue(getZkClient()).offer(Utils.toJSON(m));
     } catch (InterruptedException e) {
       Thread.interrupted();
-      log.info("Publish node as down was interrupted.");
+      log.debug("Publish node as down was interrupted.");
     } catch (Exception e) {
-      log.info("Could not publish node as down: " + e.getMessage());
+      log.warn("Could not publish node as down: " + e.getMessage());
     } 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
index 9f1a29b..9002c19 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
@@ -50,7 +50,7 @@ public class ClusterStateMutator {
 
   public ZkWriteCommand createCollection(ClusterState clusterState, ZkNodeProps message) {
     String cName = message.getStr(NAME);
-    log.info("building a new cName: " + cName);
+    log.debug("building a new cName: " + cName);
     if (clusterState.hasCollection(cName)) {
       log.warn("Collection {} already exists. exit", cName);
       return ZkStateWriter.NO_OP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index 97aa555..f03eeeb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -135,8 +135,8 @@ public class ReplicaMutator {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection/slice/replica " +
           collectionName + "/" + sliceName + "/" + replicaName + " no action taken.");
     }
-    log.info("Setting property " + property + " with value: " + propVal +
-        " for collection: " + collectionName + ". Full message: " + message);
+    log.info("Setting property {} with value {} for collection {}", property, propVal, collectionName);
+    log.debug("Full message: {}", message);
     if (StringUtils.equalsIgnoreCase(replica.getStr(property), propVal)) return ZkStateWriter.NO_OP; // already the value we're going to set
 
     // OK, there's no way we won't change the cluster state now
@@ -184,13 +184,11 @@ public class ReplicaMutator {
           collectionName + "/" + sliceName + "/" + replicaName + " no action taken.");
     }
 
-    log.info("Deleting property " + property + " for collection: " + collectionName +
-        " slice " + sliceName + " replica " + replicaName + ". Full message: " + message);
+    log.info("Deleting property {} for collection: {} slice: {} replica: {}", property, collectionName, sliceName, replicaName);
+    log.debug("Full message: {}", message);
     String curProp = replica.getStr(property);
     if (curProp == null) return ZkStateWriter.NO_OP; // not there anyway, nothing to do.
 
-    log.info("Deleting property " + property + " for collection: " + collectionName +
-        " slice " + sliceName + " replica " + replicaName + ". Full message: " + message);
     Slice slice = collection.getSlice(sliceName);
     DocCollection newCollection = SliceMutator.updateReplica(collection,
         slice, replicaName, unsetProperty(replica, property));
@@ -209,7 +207,7 @@ public class ReplicaMutator {
     final String cName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     Integer numShards = message.getInt(ZkStateReader.NUM_SHARDS_PROP, null);
-    log.info("Update state numShards={} message={}", numShards, message);
+    log.debug("Update state numShards={} message={}", numShards, message);
 
     List<String> shardNames = new ArrayList<>();
 
@@ -239,7 +237,7 @@ public class ReplicaMutator {
       coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(collection,
           message.getStr(ZkStateReader.NODE_NAME_PROP), message.getStr(ZkStateReader.CORE_NAME_PROP));
       if (coreNodeName != null) {
-        log.info("node=" + coreNodeName + " is already registered");
+        log.debug("node=" + coreNodeName + " is already registered");
       } else {
         // if coreNodeName is null, auto assign one
         coreNodeName = Assign.assignNode(collection);
@@ -253,7 +251,7 @@ public class ReplicaMutator {
       //get shardId from ClusterState
       sliceName = ClusterStateMutator.getAssignedId(collection, coreNodeName);
       if (sliceName != null) {
-        log.info("shard=" + sliceName + " is already registered");
+        log.debug("shard=" + sliceName + " is already registered");
       }
     }
     if (sliceName == null) {
@@ -261,7 +259,7 @@ public class ReplicaMutator {
       if (collectionExists) {
         // use existing numShards
         numShards = collection.getSlices().size();
-        log.info("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards);
+        log.debug("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards);
       }
       sliceName = Assign.assignShard(collection, numShards);
       log.info("Assigning new node to shard shard=" + sliceName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
index e313e36..598b5b2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
@@ -221,17 +221,17 @@ public class ZkStateWriter {
 
           if (c == null) {
             // let's clean up the collections path for this collection
-            log.info("going to delete_collection {}", path);
+            log.debug("going to delete_collection {}", path);
             reader.getZkClient().clean("/collections/" + name);
           } else if (c.getStateFormat() > 1) {
             byte[] data = Utils.toJSON(singletonMap(c.getName(), c));
             if (reader.getZkClient().exists(path, true)) {
-              log.info("going to update_collection {} version: {}", path, c.getZNodeVersion());
+              log.debug("going to update_collection {} version: {}", path, c.getZNodeVersion());
               Stat stat = reader.getZkClient().setData(path, data, c.getZNodeVersion(), true);
               DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), stat.getVersion(), path);
               clusterState = clusterState.copyWith(name, newCollection);
             } else {
-              log.info("going to create_collection {}", path);
+              log.debug("going to create_collection {}", path);
               reader.getZkClient().create(path, data, CreateMode.PERSISTENT, true);
               DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), 0, path);
               clusterState = clusterState.copyWith(name, newCollection);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
index 9545549..a58de48 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
@@ -219,7 +219,7 @@ public class CoreDescriptor {
       cloudDesc = null;
     }
 
-    log.info("Created CoreDescriptor: " + coreProperties);
+    log.debug("Created CoreDescriptor: " + coreProperties);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
index c944dec..dfb580d 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
@@ -105,15 +105,13 @@ public class ConnectionManager implements Watcher {
   @Override
   public void process(WatchedEvent event) {
     if (event.getState() == AuthFailed || event.getState() == Disconnected || event.getState() == Expired) {
-      log.warn("Watcher " + this + " name:" + name + " got event " + event
-          + " path:" + event.getPath() + " type:" + event.getType());
-    } else if (log.isInfoEnabled()) {
-      log.info("Watcher " + this + " name:" + name + " got event " + event
-          + " path:" + event.getPath() + " type:" + event.getType());
+      log.warn("Watcher {} name: {} got event {} path: {} type: {}", this, name, event, event.getPath(), event.getType());
+    } else {
+      log.debug("Watcher {} name: {} got event {} path: {} type: {}", this, name, event, event.getPath(), event.getType());
     }
     
     if (isClosed) {
-      log.info("Client->ZooKeeper status change trigger but we are already closed");
+      log.debug("Client->ZooKeeper status change trigger but we are already closed");
       return;
     }
     
@@ -216,7 +214,7 @@ public class ConnectionManager implements Watcher {
 
   public synchronized void waitForConnected(long waitForConnection)
       throws TimeoutException {
-    log.info("Waiting for client to connect to ZooKeeper");
+    log.debug("Waiting for client to connect to ZooKeeper");
     long expire = System.nanoTime() + TimeUnit.NANOSECONDS.convert(waitForConnection, TimeUnit.MILLISECONDS);
     long left = 1;
     while (!connected && left > 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index f2b67ee..422d9e5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -209,7 +209,7 @@ public class SolrZkClient implements Closeable {
         log.warn("VM param zkCredentialsProvider does not point to a class implementing ZkCredentialsProvider and with a non-arg constructor", t);
       }
     }
-    log.info("Using default ZkCredentialsProvider");
+    log.debug("Using default ZkCredentialsProvider");
     return new DefaultZkCredentialsProvider();
   }
 
@@ -225,7 +225,7 @@ public class SolrZkClient implements Closeable {
         log.warn("VM param zkACLProvider does not point to a class implementing ZkACLProvider and with a non-arg constructor", t);
       }
     }
-    log.info("Using default ZkACLProvider");
+    log.debug("Using default ZkACLProvider");
     return new DefaultZkACLProvider();
   }
 
@@ -478,9 +478,7 @@ public class SolrZkClient implements Closeable {
    */
   public void makePath(String path, byte[] data, CreateMode createMode,
       Watcher watcher, boolean failOnExists, boolean retryOnConnLoss) throws KeeperException, InterruptedException {
-    if (log.isInfoEnabled()) {
-      log.info("makePath: " + path);
-    }
+    log.debug("makePath: {}", path);
     boolean retry = true;
 
     if (path.startsWith("/")) {
@@ -564,10 +562,7 @@ public class SolrZkClient implements Closeable {
    */
   public Stat setData(String path, File file, boolean retryOnConnLoss) throws IOException,
       KeeperException, InterruptedException {
-    if (log.isInfoEnabled()) {
-      log.info("Write to ZooKeepeer " + file.getAbsolutePath() + " to " + path);
-    }
-
+    log.debug("Write to ZooKeeper: {} to {}", file.getAbsolutePath(), path);
     byte[] data = FileUtils.readFileToByteArray(file);
     return setData(path, data, retryOnConnLoss);
   }
@@ -768,7 +763,7 @@ public class SolrZkClient implements Closeable {
     ZkMaintenanceUtils.traverseZkTree(this, root, ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, path -> {
       try {
         setACL(path, getZkACLProvider().getACLsToAdd(path), true);
-        log.info("Updated ACL on " + path);
+        log.debug("Updated ACL on {}", path);
       } catch (NoNodeException e) {
         // If a node was deleted, don't bother trying to set ACLs on it.
         return;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/082f8e3f/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 2f03286..7181d70 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -288,7 +288,7 @@ public class ZkStateReader implements Closeable {
 
     synchronized (getUpdateLock()) {
       if (clusterState == null) {
-        LOG.info("ClusterState watchers have not been initialized");
+        LOG.warn("ClusterState watchers have not been initialized");
         return;
       }
 
@@ -296,20 +296,20 @@ public class ZkStateReader implements Closeable {
       if (ref == null || legacyCollectionStates.containsKey(collection)) {
         // We either don't know anything about this collection (maybe it's new?) or it's legacy.
         // First update the legacy cluster state.
-        LOG.info("Checking legacy cluster state for collection {}", collection);
+        LOG.debug("Checking legacy cluster state for collection {}", collection);
         refreshLegacyClusterState(null);
         if (!legacyCollectionStates.containsKey(collection)) {
           // No dice, see if a new collection just got created.
           LazyCollectionRef tryLazyCollection = new LazyCollectionRef(collection);
           if (tryLazyCollection.get() != null) {
             // What do you know, it exists!
-            LOG.info("Adding lazily-loaded reference for collection {}", collection);
+            LOG.debug("Adding lazily-loaded reference for collection {}", collection);
             lazyCollectionStates.putIfAbsent(collection, tryLazyCollection);
             constructState(Collections.singleton(collection));
           }
         }
       } else if (ref.isLazilyLoaded()) {
-        LOG.info("Refreshing lazily-loaded state for collection {}", collection);
+        LOG.debug("Refreshing lazily-loaded state for collection {}", collection);
         if (ref.get() != null) {
           return;
         }
@@ -317,7 +317,7 @@ public class ZkStateReader implements Closeable {
         refreshLegacyClusterState(null);
       } else if (watchedCollectionStates.containsKey(collection)) {
         // Exists as a watched collection, force a refresh.
-        LOG.info("Forcing refresh of watched collection state for {}", collection);
+        LOG.debug("Forcing refresh of watched collection state for {}", collection);
         DocCollection newState = fetchCollectionState(collection, null);
         if (updateWatchedCollection(collection, newState)) {
           constructState(Collections.singleton(collection));
@@ -404,7 +404,7 @@ public class ZkStateReader implements Closeable {
                   final Stat stat = new Stat();
                   final byte[] data = zkClient.getData(ALIASES, thisWatch, stat, true);
                   ZkStateReader.this.aliases = ClusterState.load(data);
-                  LOG.info("New alias definition is: " + ZkStateReader.this.aliases.toString());
+                  LOG.debug("New alias definition is: " + ZkStateReader.this.aliases.toString());
                 }
               } catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
                 LOG.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
@@ -872,11 +872,11 @@ public class ZkStateReader implements Closeable {
         try {
           byte[] data = zkClient.getData(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, new Stat(), true);
           this.clusterProperties = (Map<String, Object>) Utils.fromJSON(data);
-          LOG.info("Loaded cluster properties: {}", this.clusterProperties);
+          LOG.debug("Loaded cluster properties: {}", this.clusterProperties);
           return;
         } catch (KeeperException.NoNodeException e) {
           this.clusterProperties = Collections.emptyMap();
-          LOG.info("Loaded empty cluster properties");
+          LOG.debug("Loaded empty cluster properties");
           // set an exists watch, and if the node has been created since the last call,
           // read the data again
           if (zkClient.exists(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, true) == null)
@@ -952,7 +952,7 @@ public class ZkStateReader implements Closeable {
 
       if (!collectionWatches.containsKey(coll)) {
         // This collection is no longer interesting, stop watching.
-        LOG.info("Uninteresting collection {}", coll);
+        LOG.debug("Uninteresting collection {}", coll);
         return;
       }
 
@@ -999,7 +999,7 @@ public class ZkStateReader implements Closeable {
         return;
       }
       int liveNodesSize = ZkStateReader.this.clusterState == null ? 0 : ZkStateReader.this.clusterState.getLiveNodes().size();
-      LOG.info("A cluster state change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodesSize);
+      LOG.debug("A cluster state change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodesSize);
       refreshAndWatch();
     }
 
@@ -1032,7 +1032,7 @@ public class ZkStateReader implements Closeable {
       if (EventType.None.equals(event.getType())) {
         return;
       }
-      LOG.info("A collections change: [{}], has occurred - updating...", event);
+      LOG.debug("A collections change: [{}], has occurred - updating...", event);
       refreshAndWatch();
       synchronized (getUpdateLock()) {
         constructState(Collections.emptySet());
@@ -1065,7 +1065,7 @@ public class ZkStateReader implements Closeable {
       if (EventType.None.equals(event.getType())) {
         return;
       }
-      LOG.info("A live node change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodes.size());
+      LOG.debug("A live node change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodes.size());
       refreshAndWatch();
     }
 
@@ -1286,7 +1286,7 @@ public class ZkStateReader implements Closeable {
   private boolean updateWatchedCollection(String coll, DocCollection newState) {
 
     if (newState == null) {
-      LOG.info("Deleting data for [{}]", coll);
+      LOG.debug("Removing cached collection state for [{}]", coll);
       watchedCollectionStates.remove(coll);
       return true;
     }
@@ -1300,7 +1300,7 @@ public class ZkStateReader implements Closeable {
       DocCollection oldState = watchedCollectionStates.get(coll);
       if (oldState == null) {
         if (watchedCollectionStates.putIfAbsent(coll, newState) == null) {
-          LOG.info("Add data for [{}] ver [{}]", coll, newState.getZNodeVersion());
+          LOG.debug("Add data for [{}] ver [{}]", coll, newState.getZNodeVersion());
           updated = true;
           break;
         }
@@ -1312,7 +1312,7 @@ public class ZkStateReader implements Closeable {
           break;
         }
         if (watchedCollectionStates.replace(coll, oldState, newState)) {
-          LOG.info("Updating data for [{}] from [{}] to [{}]", coll, oldState.getZNodeVersion(), newState.getZNodeVersion());
+          LOG.debug("Updating data for [{}] from [{}] to [{}]", coll, oldState.getZNodeVersion(), newState.getZNodeVersion());
           updated = true;
           break;
         }
@@ -1322,7 +1322,7 @@ public class ZkStateReader implements Closeable {
     // Resolve race with unregisterCore.
     if (!collectionWatches.containsKey(coll)) {
       watchedCollectionStates.remove(coll);
-      LOG.info("Removing uninteresting collection [{}]", coll);
+      LOG.debug("Removing uninteresting collection [{}]", coll);
     }
 
     return updated;