Browse Source

ZOOKEEPER-3509: Revisit log format

1. Always wrap `.trace` with `isTraceEnabled`. It is because that some of trace calls are a bit complex.
2. Never wrap other levels with `isXXXEnabled`. Logger already checks log level and we don't make side effect or expensive operation in log statement.
3. Always use `{}` instead of string concatenation or `String.format`. Just for unification and it is what is recommended.
4. Always log throwable in warn or error level or debug if omit
5. Always log throwable as the last arg and never log `throwable.getMessage` since Logger already log stack trace.

Author: tison <wander4096@gmail.com>

Reviewers: enixon@apache.org, andor@apache.org

Closes #1100 from TisonKun/ZOOKEEPER-3509 and squashes the following commits:

0c3a4ded8 [tison] Address comment
dcb68eb61 [tison] ZOOKEEPER-3509: Revisit log format
tison 5 năm trước cách đây
mục cha
commit
11c07921c1
100 tập tin đã thay đổi với 1070 bổ sung984 xóa
  1. 3 3
      zookeeper-metrics-providers/zookeeper-prometheus-metrics/src/main/java/org/apache/zookeeper/metrics/prometheus/PrometheusMetricsProvider.java
  2. 4 4
      zookeeper-recipes/zookeeper-recipes-election/src/main/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java
  3. 2 4
      zookeeper-recipes/zookeeper-recipes-election/src/test/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java
  4. 6 6
      zookeeper-recipes/zookeeper-recipes-lock/src/main/java/org/apache/zookeeper/recipes/lock/ProtocolSupport.java
  5. 7 7
      zookeeper-recipes/zookeeper-recipes-lock/src/main/java/org/apache/zookeeper/recipes/lock/WriteLock.java
  6. 3 3
      zookeeper-recipes/zookeeper-recipes-lock/src/main/java/org/apache/zookeeper/recipes/lock/ZNodeName.java
  7. 5 5
      zookeeper-recipes/zookeeper-recipes-queue/src/main/java/org/apache/zookeeper/recipes/queue/DistributedQueue.java
  8. 0 1
      zookeeper-recipes/zookeeper-recipes-queue/src/test/java/org/apache/zookeeper/recipes/queue/DistributedQueueTest.java
  9. 54 82
      zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
  10. 5 2
      zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocket.java
  11. 10 4
      zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNIO.java
  12. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java
  13. 38 25
      zookeeper-server/src/main/java/org/apache/zookeeper/Login.java
  14. 22 14
      zookeeper-server/src/main/java/org/apache/zookeeper/SaslClientCallbackHandler.java
  15. 5 4
      zookeeper-server/src/main/java/org/apache/zookeeper/SaslServerPrincipal.java
  16. 24 17
      zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
  17. 7 13
      zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java
  18. 5 4
      zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java
  19. 4 3
      zookeeper-server/src/main/java/org/apache/zookeeper/common/AtomicFileOutputStream.java
  20. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java
  21. 3 9
      zookeeper-server/src/main/java/org/apache/zookeeper/common/NettyUtils.java
  22. 15 13
      zookeeper-server/src/main/java/org/apache/zookeeper/common/SSLContextAndOptions.java
  23. 13 18
      zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java
  24. 2 4
      zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java
  25. 1 3
      zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKHostnameVerifier.java
  26. 4 1
      zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKTrustManager.java
  27. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/jmx/MBeanRegistry.java
  28. 8 4
      zookeeper-server/src/main/java/org/apache/zookeeper/server/BlueThrottle.java
  29. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ConnectionBean.java
  30. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ContainerManager.java
  31. 33 24
      zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java
  32. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/DatadirCleanupManager.java
  33. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
  34. 18 18
      zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java
  35. 19 15
      zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxnFactory.java
  36. 12 18
      zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
  37. 10 7
      zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java
  38. 11 16
      zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
  39. 6 4
      zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java
  40. 3 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ReferenceCountedACLCache.java
  41. 13 20
      zookeeper-server/src/main/java/org/apache/zookeeper/server/SessionTrackerImpl.java
  42. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java
  43. 23 13
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java
  44. 82 68
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
  45. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooTrace.java
  46. 3 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java
  47. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/ProviderRegistry.java
  48. 3 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/SaslServerCallbackHandler.java
  49. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FilePadding.java
  50. 3 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java
  51. 13 13
      zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java
  52. 9 8
      zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java
  53. 7 7
      zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapStream.java
  54. 10 11
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
  55. 8 5
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java
  56. 69 69
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java
  57. 10 6
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java
  58. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java
  59. 29 24
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
  60. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderRequestProcessor.java
  61. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java
  62. 41 23
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
  63. 72 65
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java
  64. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandlerBean.java
  65. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerMaster.java
  66. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java
  67. 11 9
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java
  68. 16 14
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java
  69. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverZooKeeperServer.java
  70. 55 46
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
  71. 28 20
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
  72. 6 5
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
  73. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
  74. 2 4
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java
  75. 0 6
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java
  76. 7 6
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/UnifiedServerSocket.java
  77. 1 1
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/UpgradeableSessionTracker.java
  78. 8 6
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java
  79. 12 7
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthServer.java
  80. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java
  81. 2 6
      zookeeper-server/src/main/java/org/apache/zookeeper/server/util/MessageTracker.java
  82. 2 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RequestPathMetricsCollector.java
  83. 12 11
      zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherCleaner.java
  84. 11 6
      zookeeper-server/src/main/java/org/apache/zookeeper/util/SecurityUtils.java
  85. 5 2
      zookeeper-server/src/test/java/org/apache/zookeeper/JUnit4ZKTestRunner.java
  86. 5 2
      zookeeper-server/src/test/java/org/apache/zookeeper/PortAssignment.java
  87. 53 53
      zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java
  88. 4 4
      zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java
  89. 6 6
      zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java
  90. 5 3
      zookeeper-server/src/test/java/org/apache/zookeeper/server/CRCTest.java
  91. 8 12
      zookeeper-server/src/test/java/org/apache/zookeeper/server/DeserializationPerfTest.java
  92. 1 1
      zookeeper-server/src/test/java/org/apache/zookeeper/server/InvalidSnapshotTest.java
  93. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java
  94. 8 6
      zookeeper-server/src/test/java/org/apache/zookeeper/server/SerializationPerfTest.java
  95. 1 1
      zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerStartupTest.java
  96. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java
  97. 4 4
      zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnLogTest.java
  98. 5 5
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java
  99. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java
  100. 1 3
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java

+ 3 - 3
zookeeper-metrics-providers/zookeeper-prometheus-metrics/src/main/java/org/apache/zookeeper/metrics/prometheus/PrometheusMetricsProvider.java

@@ -282,7 +282,7 @@ public class PrometheusMetricsProvider implements MetricsProvider {
             try {
                 inner.inc(delta);
             } catch (IllegalArgumentException err) {
-                LOG.error("invalid delta " + delta + " for metric " + name, err);
+                LOG.error("invalid delta {} for metric {}", delta, name, err);
             }
         }
 
@@ -354,7 +354,7 @@ public class PrometheusMetricsProvider implements MetricsProvider {
             try {
                 inner.observe(delta);
             } catch (IllegalArgumentException err) {
-                LOG.error("invalid delta " + delta + " for metric " + name, err);
+                LOG.error("invalid delta {} for metric {}", delta, name, err);
             }
         }
 
@@ -389,7 +389,7 @@ public class PrometheusMetricsProvider implements MetricsProvider {
             try {
                 inner.labels(key).observe(value);
             } catch (IllegalArgumentException err) {
-                LOG.error("invalid value " + value + " for metric " + name + " with key " + key, err);
+                LOG.error("invalid value {} for metric {} with key {}", value, name, key, err);
             }
         }
 

+ 4 - 4
zookeeper-recipes/zookeeper-recipes-election/src/main/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupport.java

@@ -228,7 +228,7 @@ public class LeaderElectionSupport implements Watcher {
         throws KeeperException, InterruptedException {
 
         LOG.info(
-            "{} not elected leader. Watching node:{}",
+            "{} not elected leader. Watching node: {}",
             getLeaderOffer().getNodePath(),
             neighborLeaderOffer.getNodePath());
 
@@ -262,13 +262,13 @@ public class LeaderElectionSupport implements Watcher {
         state = State.ELECTED;
         dispatchEvent(EventType.ELECTED_START);
 
-        LOG.info("Becoming leader with node:{}", getLeaderOffer().getNodePath());
+        LOG.info("Becoming leader with node: {}", getLeaderOffer().getNodePath());
 
         dispatchEvent(EventType.ELECTED_COMPLETE);
     }
 
     private void becomeFailed(Exception e) {
-        LOG.error("Failed in state {} - Exception:{}", state, e);
+        LOG.error("Failed in state {}", state, e);
 
         state = State.FAILED;
         dispatchEvent(EventType.FAILED);
@@ -341,7 +341,7 @@ public class LeaderElectionSupport implements Watcher {
     }
 
     private void dispatchEvent(EventType eventType) {
-        LOG.debug("Dispatching event:{}", eventType);
+        LOG.debug("Dispatching event: {}", eventType);
 
         synchronized (listeners) {
             if (listeners.size() > 0) {

+ 2 - 4
zookeeper-recipes/zookeeper-recipes-election/src/test/java/org/apache/zookeeper/recipes/leader/LeaderElectionSupportTest.java

@@ -196,7 +196,7 @@ public class LeaderElectionSupportTest extends ClientBase {
                         // stopping the ELECTED node, so re-election will happen.
                         electionSupport1.stop();
                     } catch (Exception e) {
-                        LOGGER.error("Unexpected error", e);
+                        LOGGER.error("Unexpected exception", e);
                     }
                 }
                 if (eventType == EventType.ELECTED_COMPLETE) {
@@ -255,9 +255,7 @@ public class LeaderElectionSupportTest extends ClientBase {
 
                 latch.countDown();
             } catch (Exception e) {
-                LOGGER.warn(
-                    "Failed to run leader election due to: {}",
-                    e.getMessage());
+                LOGGER.warn("Failed to run leader election.", e);
                 failureCounter.incrementAndGet();
             }
         });

+ 6 - 6
zookeeper-recipes/zookeeper-recipes-lock/src/main/java/org/apache/zookeeper/recipes/lock/ProtocolSupport.java

@@ -38,11 +38,11 @@ import org.slf4j.LoggerFactory;
 class ProtocolSupport {
 
     private static final Logger LOG = LoggerFactory.getLogger(ProtocolSupport.class);
+    private static final int RETRY_COUNT = 10;
 
     protected final ZooKeeper zookeeper;
     private AtomicBoolean closed = new AtomicBoolean(false);
     private long retryDelay = 500L;
-    private int retryCount = 10;
     private List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
 
     public ProtocolSupport(ZooKeeper zookeeper) {
@@ -121,17 +121,17 @@ class ProtocolSupport {
     protected Object retryOperation(ZooKeeperOperation operation)
         throws KeeperException, InterruptedException {
         KeeperException exception = null;
-        for (int i = 0; i < retryCount; i++) {
+        for (int i = 0; i < RETRY_COUNT; i++) {
             try {
                 return operation.execute();
             } catch (KeeperException.SessionExpiredException e) {
-                LOG.warn("Session expired for: " + zookeeper + " so reconnecting due to: " + e, e);
+                LOG.warn("Session expired {}. Reconnecting...", zookeeper, e);
                 throw e;
             } catch (KeeperException.ConnectionLossException e) {
                 if (exception == null) {
                     exception = e;
                 }
-                LOG.debug("Attempt {} failed with connection loss so attempting to reconnect", i, e);
+                LOG.debug("Attempt {} failed with connection loss. Reconnecting...", i);
                 retryDelay(i);
             }
         }
@@ -171,7 +171,7 @@ class ProtocolSupport {
                 return true;
             });
         } catch (KeeperException | InterruptedException e) {
-            LOG.warn("Caught: " + e, e);
+            LOG.warn("Unexpected exception", e);
         }
     }
 
@@ -194,7 +194,7 @@ class ProtocolSupport {
             try {
                 Thread.sleep(attemptCount * retryDelay);
             } catch (InterruptedException e) {
-                LOG.debug("Failed to sleep", e);
+                LOG.warn("Failed to sleep.", e);
             }
         }
     }

+ 7 - 7
zookeeper-recipes/zookeeper-recipes-lock/src/main/java/org/apache/zookeeper/recipes/lock/WriteLock.java

@@ -130,13 +130,13 @@ public class WriteLock extends ProtocolSupport {
                 };
                 zopdel.execute();
             } catch (InterruptedException e) {
-                LOG.warn("Caught: " + e, e);
-                //set that we have been interrupted.
+                LOG.warn("Unexpected exception", e);
+                // set that we have been interrupted.
                 Thread.currentThread().interrupt();
             } catch (KeeperException.NoNodeException e) {
                 // do nothing
             } catch (KeeperException e) {
-                LOG.warn("Caught: " + e, e);
+                LOG.warn("Unexpected exception", e);
                 throw new RuntimeException(e.getMessage(), e);
             } finally {
                 LockListener lockListener = getLockListener();
@@ -161,7 +161,7 @@ public class WriteLock extends ProtocolSupport {
             try {
                 lock();
             } catch (Exception e) {
-                LOG.warn("Failed to acquire lock: " + e, e);
+                LOG.warn("Failed to acquire lock", e);
             }
         }
 
@@ -221,7 +221,7 @@ public class WriteLock extends ProtocolSupport {
                 }
                 List<String> names = zookeeper.getChildren(dir, false);
                 if (names.isEmpty()) {
-                    LOG.warn("No children in: " + dir + " when we've just created one! Lets recreate it...");
+                    LOG.warn("No children in: {} when we've just created one! Lets recreate it...", dir);
                     // lets force the recreation of the id
                     id = null;
                 } else {
@@ -235,12 +235,12 @@ public class WriteLock extends ProtocolSupport {
                     if (!lessThanMe.isEmpty()) {
                         ZNodeName lastChildName = lessThanMe.last();
                         lastChildId = lastChildName.getName();
-                        LOG.debug("watching less than me node: {}", lastChildId);
+                        LOG.debug("Watching less than me node: {}", lastChildId);
                         Stat stat = zookeeper.exists(lastChildId, new LockWatcher());
                         if (stat != null) {
                             return Boolean.FALSE;
                         } else {
-                            LOG.warn("Could not find the stats for less than me: " + lastChildName.getName());
+                            LOG.warn("Could not find the stats for less than me: {}", lastChildName.getName());
                         }
                     } else {
                         if (isOwner()) {

+ 3 - 3
zookeeper-recipes/zookeeper-recipes-lock/src/main/java/org/apache/zookeeper/recipes/lock/ZNodeName.java

@@ -44,12 +44,12 @@ class ZNodeName implements Comparable<ZNodeName> {
             this.prefix = name.substring(0, idx);
             try {
                 this.sequence = Integer.parseInt(name.substring(idx + 1));
-                // If an exception occurred we misdetected a sequence suffix,
+                // If an exception occurred we mis-detected a sequence suffix,
                 // so return -1.
             } catch (NumberFormatException e) {
-                LOG.info("Number format exception for " + idx, e);
+                LOG.warn("Number format exception for {}.", idx, e);
             } catch (ArrayIndexOutOfBoundsException e) {
-                LOG.info("Array out of bounds for " + idx, e);
+                LOG.warn("Array out of bounds for {}.", idx, e);
             }
         }
     }

+ 5 - 5
zookeeper-recipes/zookeeper-recipes-queue/src/main/java/org/apache/zookeeper/recipes/queue/DistributedQueue.java

@@ -72,14 +72,14 @@ public class DistributedQueue {
             try {
                 //Check format
                 if (!childName.regionMatches(0, prefix, 0, prefix.length())) {
-                    LOG.warn("Found child node with improper name: " + childName);
+                    LOG.warn("Found child node with improper name: {}", childName);
                     continue;
                 }
                 String suffix = childName.substring(prefix.length());
                 Long childId = Long.parseLong(suffix);
                 orderedChildren.put(childId, childName);
             } catch (NumberFormatException e) {
-                LOG.warn("Found child node with improper format : " + childName + " " + e, e);
+                LOG.warn("Found child node with improper format : {}", childName, e);
             }
         }
 
@@ -99,7 +99,7 @@ public class DistributedQueue {
         try {
             childNames = zookeeper.getChildren(dir, false);
         } catch (KeeperException.NoNodeException e) {
-            LOG.warn("Caught: " + e, e);
+            LOG.warn("Unexpected exception", e);
             return null;
         }
 
@@ -107,7 +107,7 @@ public class DistributedQueue {
             try {
                 //Check format
                 if (!childName.regionMatches(0, prefix, 0, prefix.length())) {
-                    LOG.warn("Found child node with improper name: " + childName);
+                    LOG.warn("Found child node with improper name: {}", childName);
                     continue;
                 }
                 String suffix = childName.substring(prefix.length());
@@ -117,7 +117,7 @@ public class DistributedQueue {
                     minName = childName;
                 }
             } catch (NumberFormatException e) {
-                LOG.warn("Found child node with improper format : " + childName + " " + e, e);
+                LOG.warn("Found child node with improper format : {}", childName, e);
             }
         }
 

+ 0 - 1
zookeeper-recipes/zookeeper-recipes-queue/src/test/java/org/apache/zookeeper/recipes/queue/DistributedQueueTest.java

@@ -34,7 +34,6 @@ public class DistributedQueueTest extends ClientBase {
     @After
     public void tearDown() throws Exception {
         super.tearDown();
-        LOG.info("FINISHED " + getTestName());
     }
 
     @Test

+ 54 - 82
zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java

@@ -27,7 +27,6 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketAddress;
-import java.net.SocketException;
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
@@ -324,7 +323,7 @@ public class ClientCnxn {
                 this.bb.putInt(this.bb.capacity() - 4);
                 this.bb.rewind();
             } catch (IOException e) {
-                LOG.warn("Ignoring unexpected exception", e);
+                LOG.warn("Unexpected exception", e);
             }
         }
 
@@ -713,7 +712,7 @@ public class ClientCnxn {
                     }
                 }
             } catch (Throwable t) {
-                LOG.error("Caught unexpected throwable", t);
+                LOG.error("Unexpected throwable", t);
             }
         }
 
@@ -859,13 +858,10 @@ public class ClientCnxn {
             replyHdr.deserialize(bbia, "header");
             if (replyHdr.getXid() == -2) {
                 // -2 is the xid for pings
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Got ping response for sessionid: 0x"
-                              + Long.toHexString(sessionId)
-                              + " after "
-                              + ((System.nanoTime() - lastPingSentNs) / 1000000)
-                              + "ms");
-                }
+                LOG.debug(
+                    "Got ping response for session id: 0x{} after {}ms.",
+                    Long.toHexString(sessionId),
+                    ((System.nanoTime() - lastPingSentNs) / 1000000));
                 return;
             }
             if (replyHdr.getXid() == -4) {
@@ -875,16 +871,12 @@ public class ClientCnxn {
                     eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.AuthFailed, null));
                     eventThread.queueEventOfDeath();
                 }
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Got auth sessionid:0x" + Long.toHexString(sessionId));
-                }
+                LOG.debug("Got auth session id: 0x{}", Long.toHexString(sessionId));
                 return;
             }
             if (replyHdr.getXid() == -1) {
                 // -1 means notification
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Got notification sessionid:0x" + Long.toHexString(sessionId));
-                }
+                LOG.debug("Got notification session id: 0x{}", Long.toHexString(sessionId));
                 WatcherEvent event = new WatcherEvent();
                 event.deserialize(bbia, "response");
 
@@ -896,18 +888,15 @@ public class ClientCnxn {
                     } else if (serverPath.length() > chrootPath.length()) {
                         event.setPath(serverPath.substring(chrootPath.length()));
                     } else {
-                        LOG.warn("Got server path "
-                                 + event.getPath()
-                                 + " which is too short for chroot path "
-                                 + chrootPath);
+                        LOG.warn(
+                            "Got server path {} which is too short for chroot path {}.",
+                            event.getPath(),
+                            chrootPath);
                     }
                 }
 
                 WatchedEvent we = new WatchedEvent(event);
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Got " + we + " for sessionid 0x" + Long.toHexString(sessionId));
-                }
-
+                LOG.debug("Got {} for session id 0x{}", we, Long.toHexString(sessionId));
                 eventThread.queueEvent(we);
                 return;
             }
@@ -952,9 +941,7 @@ public class ClientCnxn {
                     packet.response.deserialize(bbia, "response");
                 }
 
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Reading reply sessionid:0x" + Long.toHexString(sessionId) + ", packet:: " + packet);
-                }
+                LOG.debug("Reading reply session id: 0x{}, packet:: {}", Long.toHexString(sessionId), packet);
             } finally {
                 finishPacket(packet);
             }
@@ -1117,10 +1104,10 @@ public class ClientCnxn {
                     // for Kerberos this means that the client failed to authenticate with the KDC.
                     // This is different from an authentication error that occurs during communication
                     // with the Zookeeper server, which is handled below.
-                    LOG.warn("SASL configuration failed: "
-                             + e
-                             + " Will continue connection to Zookeeper server without "
-                             + "SASL authentication, if Zookeeper server allows it.");
+                    LOG.warn(
+                        "SASL configuration failed. "
+                            + "Will continue connection to Zookeeper server without "
+                            + "SASL authentication, if Zookeeper server allows it.", e);
                     eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.AuthFailed, null));
                     saslLoginFailed = true;
                 }
@@ -1131,14 +1118,12 @@ public class ClientCnxn {
         }
 
         private void logStartConnect(InetSocketAddress addr) {
-            String msg = "Opening socket connection to server " + addr;
+            LOG.info("Opening socket connection to server {}.", addr);
             if (zooKeeperSaslClient != null) {
-                msg += ". " + zooKeeperSaslClient.getConfigStatus();
+                LOG.info("SASL config status: {}", zooKeeperSaslClient.getConfigStatus());
             }
-            LOG.info(msg);
         }
 
-        private static final String RETRY_CONN_MSG = ", closing socket connection and attempting reconnect";
         @Override
         public void run() {
             clientCnxnSocket.introduce(this, sessionId, outgoingQueue);
@@ -1173,7 +1158,7 @@ public class ClientCnxn {
                                 try {
                                     zooKeeperSaslClient.initialize(ClientCnxn.this);
                                 } catch (SaslException e) {
-                                    LOG.error("SASL authentication with Zookeeper Quorum member failed: " + e);
+                                    LOG.error("SASL authentication with Zookeeper Quorum member failed.", e);
                                     state = States.AUTH_FAILED;
                                     sendAuthEvent = true;
                                 }
@@ -1204,10 +1189,10 @@ public class ClientCnxn {
                     }
 
                     if (to <= 0) {
-                        String warnInfo;
-                        warnInfo = "Client session timed out, have not heard from server in "
-                                   + clientCnxnSocket.getIdleRecv() + "ms"
-                                   + " for sessionid 0x" + Long.toHexString(sessionId);
+                        String warnInfo = String.format(
+                            "Client session timed out, have not heard from server in %dms for session id 0x%s",
+                            clientCnxnSocket.getIdleRecv(),
+                            Long.toHexString(sessionId));
                         LOG.warn(warnInfo);
                         throw new SessionTimeoutException(warnInfo);
                     }
@@ -1244,39 +1229,27 @@ public class ClientCnxn {
                     clientCnxnSocket.doTransport(to, pendingQueue, ClientCnxn.this);
                 } catch (Throwable e) {
                     if (closing) {
-                        if (LOG.isDebugEnabled()) {
-                            // closing so this is expected
-                            LOG.debug("An exception was thrown while closing send thread for session 0x"
-                                      + Long.toHexString(getSessionId())
-                                      + " : "
-                                      + e.getMessage());
-                        }
+                        // closing so this is expected
+                        LOG.warn(
+                            "An exception was thrown while closing send thread for session 0x{}.",
+                            Long.toHexString(getSessionId()),
+                            e);
                         break;
                     } else {
-                        // this is ugly, you have a better way speak up
-                        if (e instanceof SessionExpiredException) {
-                            LOG.info(e.getMessage() + ", closing socket connection");
-                        } else if (e instanceof SessionTimeoutException) {
-                            LOG.info(e.getMessage() + RETRY_CONN_MSG);
-                        } else if (e instanceof EndOfStreamException) {
-                            LOG.info(e.getMessage() + RETRY_CONN_MSG);
-                        } else if (e instanceof RWServerFoundException) {
-                            LOG.info(e.getMessage());
-                        } else if (e instanceof SocketException) {
-                            LOG.info("Socket error occurred: {}: {}", serverAddress, e.getMessage());
-                        } else {
-                            LOG.warn("Session 0x{} for server {}, unexpected error{}",
-                                     Long.toHexString(getSessionId()),
-                                     serverAddress,
-                                     RETRY_CONN_MSG,
-                                     e);
-                        }
+                        LOG.warn(
+                            "Session 0x{} for sever {}, Closing socket connection. "
+                                + "Attempting reconnect except it is a SessionExpiredException.",
+                            Long.toHexString(getSessionId()),
+                            serverAddress,
+                            e);
+
                         // At this point, there might still be new packets appended to outgoingQueue.
                         // they will be handled in next connection or cleared up if closed.
                         cleanAndNotifyState();
                     }
                 }
             }
+
             synchronized (state) {
                 // When it comes to this point, it guarantees that later queued
                 // packet to outgoingQueue will be notified of death.
@@ -1305,7 +1278,8 @@ public class ClientCnxn {
         private void pingRwServer() throws RWServerFoundException {
             String result = null;
             InetSocketAddress addr = hostProvider.next(0);
-            LOG.info("Checking server " + addr + " for being r/w." + " Timeout " + pingRwTimeout);
+
+            LOG.info("Checking server {} for being r/w. Timeout {}", addr, pingRwTimeout);
 
             Socket sock = null;
             BufferedReader br = null;
@@ -1323,7 +1297,7 @@ public class ClientCnxn {
                 // ignore, this just means server is not up
             } catch (IOException e) {
                 // some unexpected error, warn about it
-                LOG.warn("Exception while seeking for r/w server " + e.getMessage(), e);
+                LOG.warn("Exception while seeking for r/w server.", e);
             } finally {
                 if (sock != null) {
                     try {
@@ -1392,16 +1366,17 @@ public class ClientCnxn {
                 eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.Expired, null));
                 eventThread.queueEventOfDeath();
 
-                String warnInfo;
-                warnInfo = "Unable to reconnect to ZooKeeper service, session 0x"
-                           + Long.toHexString(sessionId)
-                           + " has expired";
+                String warnInfo = String.format(
+                    "Unable to reconnect to ZooKeeper service, session 0x%s has expired",
+                    Long.toHexString(sessionId));
                 LOG.warn(warnInfo);
                 throw new SessionExpiredException(warnInfo);
             }
+
             if (!readOnly && isRO) {
                 LOG.error("Read/write client got connected to read-only server");
             }
+
             readTimeout = negotiatedSessionTimeout * 2 / 3;
             connectTimeout = negotiatedSessionTimeout / hostProvider.size();
             hostProvider.onConnected();
@@ -1409,11 +1384,12 @@ public class ClientCnxn {
             sessionPasswd = _sessionPasswd;
             state = (isRO) ? States.CONNECTEDREADONLY : States.CONNECTED;
             seenRwServerBefore |= !isRO;
-            LOG.info("Session establishment complete on server "
-                     + clientCnxnSocket.getRemoteSocketAddress()
-                     + ", sessionid = 0x" + Long.toHexString(sessionId)
-                     + ", negotiated timeout = " + negotiatedSessionTimeout
-                     + (isRO ? " (READ-ONLY mode)" : ""));
+            LOG.info(
+                "Session establishment complete on server {}, session id = 0x{}, negotiated timeout = {}{}",
+                clientCnxnSocket.getRemoteSocketAddress(),
+                Long.toHexString(sessionId),
+                negotiatedSessionTimeout,
+                (isRO ? " (READ-ONLY mode)" : ""));
             KeeperState eventState = (isRO) ? KeeperState.ConnectedReadOnly : KeeperState.SyncConnected;
             eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, eventState, null));
         }
@@ -1461,9 +1437,7 @@ public class ClientCnxn {
      * behavior.
      */
     public void disconnect() {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Disconnecting client for session: 0x" + Long.toHexString(getSessionId()));
-        }
+        LOG.debug("Disconnecting client for session: 0x{}", Long.toHexString(getSessionId()));
 
         sendThread.close();
         try {
@@ -1484,9 +1458,7 @@ public class ClientCnxn {
      * @throws IOException
      */
     public void close() throws IOException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Closing client for session: 0x" + Long.toHexString(getSessionId()));
-        }
+        LOG.debug("Closing client for session: 0x{}", Long.toHexString(getSessionId()));
 
         try {
             RequestHeader h = new RequestHeader();

+ 5 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocket.java

@@ -127,11 +127,14 @@ abstract class ClientCnxnSocket {
         if (LOG.isTraceEnabled()) {
             StringBuilder buf = new StringBuilder("0x[");
             for (byte b : incomingBuffer.array()) {
-                buf.append(Integer.toHexString(b) + ",");
+                buf.append(Integer.toHexString(b)).append(",");
             }
             buf.append("]");
-            LOG.trace("readConnectResult " + incomingBuffer.remaining() + " " + buf.toString());
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("readConnectResult {} {}", incomingBuffer.remaining(), buf.toString());
+            }
         }
+
         ByteBufferInputStream bbis = new ByteBufferInputStream(incomingBuffer);
         BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
         ConnectResponse conRsp = new ConnectResponse();

+ 10 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNIO.java

@@ -176,7 +176,7 @@ public class ClientCnxnSocketNIO extends ClientCnxnSocket {
             } else {
                 // Non-priming packet: defer it until later, leaving it in the queue
                 // until authentication completes.
-                LOG.debug("deferring non-priming packet {} until SASL authentation completes.", p);
+                LOG.debug("Deferring non-priming packet {} until SASL authentication completes.", p);
             }
         }
         return null;
@@ -219,9 +219,15 @@ public class ClientCnxnSocketNIO extends ClientCnxnSocket {
     @Override
     void close() {
         try {
-            LOG.trace("Doing client selector close");
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Doing client selector close");
+            }
+
             selector.close();
-            LOG.trace("Closed client selector");
+
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Closed client selector");
+            }
         } catch (IOException e) {
             LOG.warn("Ignoring exception during selector close", e);
         }
@@ -261,7 +267,7 @@ public class ClientCnxnSocketNIO extends ClientCnxnSocket {
         try {
             registerAndConnect(sock, addr);
         } catch (IOException e) {
-            LOG.error("Unable to open socket to " + addr);
+            LOG.error("Unable to open socket to {}", addr);
             sock.close();
             throw e;
         }

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java

@@ -147,7 +147,7 @@ public class ClientCnxnSocketNetty extends ClientCnxnSocket {
                     connectLock.lock();
                     try {
                         if (!channelFuture.isSuccess()) {
-                            LOG.info("future isn't success, cause:", channelFuture.cause());
+                            LOG.warn("future isn't success.", channelFuture.cause());
                             return;
                         } else if (connectFuture == null) {
                             LOG.info("connect attempt cancelled");
@@ -519,7 +519,7 @@ public class ClientCnxnSocketNetty extends ClientCnxnSocket {
 
         @Override
         public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-            LOG.warn("Exception caught", cause);
+            LOG.error("Unexpected throwable", cause);
             cleanup();
         }
 

+ 38 - 25
zookeeper-server/src/main/java/org/apache/zookeeper/Login.java

@@ -144,14 +144,17 @@ public class Login {
                         long expiry = tgt.getEndTime().getTime();
                         Date expiryDate = new Date(expiry);
                         if ((isUsingTicketCache) && (tgt.getEndTime().equals(tgt.getRenewTill()))) {
-                            Object[] logPayload = {expiryDate, principal, principal};
-                            LOG.error("The TGT cannot be renewed beyond the next expiry date: {}."
-                                      + "This process will not be able to authenticate new SASL connections after that "
-                                      + "time (for example, it will not be authenticate a new connection with a Zookeeper "
-                                      + "Quorum member).  Ask your system administrator to either increase the "
-                                      + "'renew until' time by doing : 'modprinc -maxrenewlife {}' within "
-                                      + "kadmin, or instead, to generate a keytab for {}. Because the TGT's "
-                                      + "expiry cannot be further extended by refreshing, exiting refresh thread now.", logPayload);
+                            LOG.error(
+                                "The TGT cannot be renewed beyond the next expiry date: {}."
+                                    + "This process will not be able to authenticate new SASL connections after that "
+                                    + "time (for example, it will not be authenticate a new connection with a Zookeeper "
+                                    + "Quorum member).  Ask your system administrator to either increase the "
+                                    + "'renew until' time by doing : 'modprinc -maxrenewlife {}' within "
+                                    + "kadmin, or instead, to generate a keytab for {}. Because the TGT's "
+                                    + "expiry cannot be further extended by refreshing, exiting refresh thread now.",
+                                expiryDate,
+                                principal,
+                                principal);
                             return;
                         }
                         // determine how long to sleep from looking at ticket's expiry.
@@ -166,19 +169,25 @@ public class Login {
                                 // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN).
                                 Date until = new Date(nextRefresh);
                                 Date newuntil = new Date(now + MIN_TIME_BEFORE_RELOGIN);
-                                Object[] logPayload = {until, newuntil, (MIN_TIME_BEFORE_RELOGIN / 1000)};
-                                LOG.warn("TGT refresh thread time adjusted from : {} to : {} since "
-                                         + "the former is sooner than the minimum refresh interval ("
-                                         + "{} seconds) from now.", logPayload);
+                                LOG.warn(
+                                    "TGT refresh thread time adjusted from : {} to : {} since "
+                                        + "the former is sooner than the minimum refresh interval ("
+                                        + "{} seconds) from now.",
+                                    until,
+                                    newuntil,
+                                    (MIN_TIME_BEFORE_RELOGIN / 1000));
                             }
                             nextRefresh = Math.max(nextRefresh, now + MIN_TIME_BEFORE_RELOGIN);
                         }
                         nextRefreshDate = new Date(nextRefresh);
                         if (nextRefresh > expiry) {
-                            Object[] logPayload = {nextRefreshDate, expiryDate};
-                            LOG.error("next refresh: {} is later than expiry {}."
-                                      + " This may indicate a clock skew problem. Check that this host and the KDC's "
-                                      + "hosts' clocks are in sync. Exiting refresh thread.", logPayload);
+                            LOG.error(
+                                "next refresh: {} is later than expiry {}."
+                                    + " This may indicate a clock skew problem."
+                                    + " Check that this host and the KDC's "
+                                    + "hosts' clocks are in sync. Exiting refresh thread.",
+                                nextRefreshDate,
+                                expiryDate);
                             return;
                         }
                     }
@@ -194,10 +203,12 @@ public class Login {
                             break;
                         }
                     } else {
-                        LOG.error("nextRefresh:{} is in the past: exiting refresh thread. Check"
-                                  + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)."
-                                  + " Manual intervention will be required for this client to successfully authenticate."
-                                  + " Exiting refresh thread.", nextRefreshDate);
+                        LOG.error(
+                            "nextRefresh:{} is in the past: exiting refresh thread. Check"
+                                + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)."
+                                + " Manual intervention will be required for this client to successfully authenticate."
+                                + " Exiting refresh thread.",
+                            nextRefreshDate);
                         break;
                     }
                     if (isUsingTicketCache) {
@@ -220,10 +231,12 @@ public class Login {
                                         return;
                                     }
                                 } else {
-                                    Object[] logPayload = {cmd, kinitArgs, e.toString(), e};
-                                    LOG.warn("Could not renew TGT due to problem running shell command: '{}"
-                                             + " {}'; exception was:{}. Exiting refresh thread.",
-                                             logPayload);
+                                    LOG.warn(
+                                        "Could not renew TGT due to problem running shell command: '{} {}'."
+                                            + " Exiting refresh thread.",
+                                        cmd,
+                                        kinitArgs,
+                                        e);
                                     return;
                                 }
                             }
@@ -273,7 +286,7 @@ public class Login {
             try {
                 t.join();
             } catch (InterruptedException e) {
-                LOG.warn("error while waiting for Login thread to shutdown: ", e);
+                LOG.warn("error while waiting for Login thread to shutdown.", e);
             }
         }
     }

+ 22 - 14
zookeeper-server/src/main/java/org/apache/zookeeper/SaslClientCallbackHandler.java

@@ -57,20 +57,28 @@ public class SaslClientCallbackHandler implements CallbackHandler {
                     if (password != null) {
                         pc.setPassword(this.password.toCharArray());
                     } else {
-                        LOG.warn("Could not login: the {} is being asked for a password, but the ZooKeeper {}"
-                                 + " code does not currently support obtaining a password from the user."
-                                 + " Make sure that the {} is configured to use a ticket cache (using"
-                                 + " the JAAS configuration setting 'useTicketCache=true)' and restart the {}. If"
-                                 + " you still get this message after that, the TGT in the ticket cache has expired and must"
-                                 + " be manually refreshed. To do so, first determine if you are using a password or a"
-                                 + " keytab. If the former, run kinit in a Unix shell in the environment of the user who"
-                                 + " is running this Zookeeper {} using the command"
-                                 + " 'kinit <princ>' (where <princ> is the name of the {}'s Kerberos principal)."
-                                 + " If the latter, do"
-                                 + " 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the Kerberos principal, and"
-                                 + " <keytab> is the location of the keytab file). After manually refreshing your cache,"
-                                 + " restart this {}. If you continue to see this message after manually refreshing"
-                                 + " your cache, ensure that your KDC host's clock is in sync with this host's clock.", entity, entity, entity, entity, entity, entity, entity);
+                        LOG.warn(
+                            "Could not login: the {} is being asked for a password, but the ZooKeeper {}"
+                                + " code does not currently support obtaining a password from the user."
+                                + " Make sure that the {} is configured to use a ticket cache (using"
+                                + " the JAAS configuration setting 'useTicketCache=true)' and restart the {}. If"
+                                + " you still get this message after that, the TGT in the ticket cache has expired and must"
+                                + " be manually refreshed. To do so, first determine if you are using a password or a"
+                                + " keytab. If the former, run kinit in a Unix shell in the environment of the user who"
+                                + " is running this Zookeeper {} using the command"
+                                + " 'kinit <princ>' (where <princ> is the name of the {}'s Kerberos principal)."
+                                + " If the latter, do"
+                                + " 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the Kerberos principal, and"
+                                + " <keytab> is the location of the keytab file). After manually refreshing your cache,"
+                                + " restart this {}. If you continue to see this message after manually refreshing"
+                                + " your cache, ensure that your KDC host's clock is in sync with this host's clock.",
+                            entity,
+                            entity,
+                            entity,
+                            entity,
+                            entity,
+                            entity,
+                            entity);
                     }
                 } else {
                     if (callback instanceof RealmCallback) {

+ 5 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/SaslServerPrincipal.java

@@ -65,10 +65,11 @@ public class SaslServerPrincipal {
         try {
             canonicalize = Boolean.parseBoolean(canonicalizeText);
         } catch (IllegalArgumentException ea) {
-            LOG.warn("Could not parse config {} \"{}\" into a boolean using default {}",
-                     ZKClientConfig.ZK_SASL_CLIENT_CANONICALIZE_HOSTNAME,
-                     canonicalizeText,
-                     canonicalize);
+            LOG.warn(
+                "Could not parse config {} \"{}\" into a boolean using default {}",
+                ZKClientConfig.ZK_SASL_CLIENT_CANONICALIZE_HOSTNAME,
+                canonicalizeText,
+                canonicalize);
         }
 
         if (canonicalize) {

+ 24 - 17
zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java

@@ -522,9 +522,13 @@ public class ZooKeeper implements AutoCloseable {
                 }
                 break;
             default:
-                String msg = "Unhandled watch event type " + type + " with state " + state + " on path " + clientPath;
-                LOG.error(msg);
-                throw new RuntimeException(msg);
+                String errorMsg = String.format(
+                    "Unhandled watch event type %s with state %s on path %s",
+                    type,
+                    state,
+                    clientPath);
+                LOG.error(errorMsg);
+                throw new RuntimeException(errorMsg);
             }
 
             return result;
@@ -879,9 +883,11 @@ public class ZooKeeper implements AutoCloseable {
         boolean canBeReadOnly,
         HostProvider aHostProvider,
         ZKClientConfig clientConfig) throws IOException {
-        LOG.info("Initiating client connection, connectString=" + connectString
-                 + " sessionTimeout=" + sessionTimeout
-                 + " watcher=" + watcher);
+        LOG.info(
+            "Initiating client connection, connectString={} sessionTimeout={} watcher={}",
+            connectString,
+            sessionTimeout,
+            watcher);
 
         if (clientConfig == null) {
             clientConfig = new ZKClientConfig();
@@ -1273,11 +1279,14 @@ public class ZooKeeper implements AutoCloseable {
         boolean canBeReadOnly,
         HostProvider aHostProvider,
         ZKClientConfig clientConfig) throws IOException {
-        LOG.info("Initiating client connection, connectString=" + connectString
-                 + " sessionTimeout=" + sessionTimeout
-                 + " watcher=" + watcher
-                 + " sessionId=" + Long.toHexString(sessionId)
-                 + " sessionPasswd=" + (sessionPasswd == null ? "<null>" : "<hidden>"));
+        LOG.info(
+            "Initiating client connection, connectString={} "
+                + "sessionTimeout={} watcher={} sessionId=0x{} sessionPasswd={}",
+            connectString,
+            sessionTimeout,
+            watcher,
+            Long.toHexString(sessionId),
+            (sessionPasswd == null ? "<null>" : "<hidden>"));
 
         if (clientConfig == null) {
             clientConfig = new ZKClientConfig();
@@ -1479,9 +1488,7 @@ public class ZooKeeper implements AutoCloseable {
             return;
         }
 
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Closing session: 0x" + Long.toHexString(getSessionId()));
-        }
+        LOG.debug("Closing session: 0x" + Long.toHexString(getSessionId()));
 
         try {
             cnxn.close();
@@ -1489,7 +1496,7 @@ public class ZooKeeper implements AutoCloseable {
             LOG.debug("Ignoring unexpected exception during close", e);
         }
 
-        LOG.info("Session: 0x" + Long.toHexString(getSessionId()) + " closed");
+        LOG.info("Session: 0x{} closed", Long.toHexString(getSessionId()));
     }
 
     /**
@@ -1931,13 +1938,13 @@ public class ZooKeeper implements AutoCloseable {
             try {
                 op.validate();
             } catch (IllegalArgumentException iae) {
-                LOG.error("IllegalArgumentException: " + iae.getMessage());
+                LOG.error("Unexpected exception", iae);
                 ErrorResult err = new ErrorResult(KeeperException.Code.BADARGUMENTS.intValue());
                 results.add(err);
                 error = true;
                 continue;
             } catch (KeeperException ke) {
-                LOG.error("KeeperException: " + ke.getMessage());
+                LOG.error("Unexpected exception", ke);
                 ErrorResult err = new ErrorResult(ke.code().intValue());
                 results.add(err);
                 error = true;

+ 7 - 13
zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java

@@ -328,19 +328,12 @@ public class ZooKeeperMain {
                 while ((line = (String) readLine.invoke(console, getPrompt())) != null) {
                     executeLine(line);
                 }
-            } catch (ClassNotFoundException e) {
-                LOG.debug("Unable to start jline", e);
-                jlinemissing = true;
-            } catch (NoSuchMethodException e) {
-                LOG.debug("Unable to start jline", e);
-                jlinemissing = true;
-            } catch (InvocationTargetException e) {
-                LOG.debug("Unable to start jline", e);
-                jlinemissing = true;
-            } catch (IllegalAccessException e) {
-                LOG.debug("Unable to start jline", e);
-                jlinemissing = true;
-            } catch (InstantiationException e) {
+            } catch (ClassNotFoundException
+                | NoSuchMethodException
+                | InvocationTargetException
+                | IllegalAccessException
+                | InstantiationException e
+            ) {
                 LOG.debug("Unable to start jline", e);
                 jlinemissing = true;
             }
@@ -396,6 +389,7 @@ public class ZooKeeperMain {
         }
 
         boolean watch = false;
+
         LOG.debug("Processing {}", cmd);
 
         if (cmd.equals("quit")) {

+ 5 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java

@@ -254,7 +254,7 @@ public class ZooKeeperSaslClient {
             throw e;
         } catch (Exception e) {
             // ..but consume (with a log message) all other types of exceptions.
-            LOG.error("Exception while trying to create SASL client: " + e);
+            LOG.error("Exception while trying to create SASL client.", e);
             return null;
         }
     }
@@ -272,9 +272,10 @@ public class ZooKeeperSaslClient {
                     sendSaslPacket(saslToken, cnxn);
                 }
             } catch (SaslException e) {
-                LOG.error("SASL authentication failed using login context '"
-                          + this.getLoginContext()
-                          + "' with exception: {}", e);
+                LOG.error(
+                    "SASL authentication failed using login context '{}'.",
+                    this.getLoginContext(),
+                    e);
                 saslState = SaslState.FAILED;
                 gotLastPacket = true;
             }

+ 4 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/common/AtomicFileOutputStream.java

@@ -99,7 +99,7 @@ public class AtomicFileOutputStream extends FilterOutputStream {
                 }
                 // close wasn't successful, try to delete the tmp file
                 if (!tmpFile.delete()) {
-                    LOG.warn("Unable to delete tmp file " + tmpFile);
+                    LOG.warn("Unable to delete tmp file {}", tmpFile);
                 }
             }
         }
@@ -113,10 +113,11 @@ public class AtomicFileOutputStream extends FilterOutputStream {
         try {
             super.close();
         } catch (IOException ioe) {
-            LOG.warn("Unable to abort file " + tmpFile, ioe);
+            LOG.warn("Unable to abort file {}", tmpFile, ioe);
         }
+
         if (!tmpFile.delete()) {
-            LOG.warn("Unable to delete tmp file during abort " + tmpFile);
+            LOG.warn("Unable to delete tmp file during abort {}", tmpFile);
         }
     }
 

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java

@@ -182,7 +182,7 @@ public final class FileChangeWatcher {
         @Override
         public void run() {
             try {
-                LOG.info(getName() + " thread started");
+                LOG.info("{} thread started", getName());
                 if (!compareAndSetState(FileChangeWatcher.State.STARTING, FileChangeWatcher.State.RUNNING)) {
                     // stop() called shortly after start(), before
                     // this thread started running.
@@ -202,7 +202,7 @@ public final class FileChangeWatcher {
                 } catch (IOException e) {
                     LOG.warn("Error closing watch service", e);
                 }
-                LOG.info(getName() + " thread finished");
+                LOG.info("{} thread finished", getName());
                 FileChangeWatcher.this.setState(FileChangeWatcher.State.STOPPED);
             }
         }

+ 3 - 9
zookeeper-server/src/main/java/org/apache/zookeeper/common/NettyUtils.java

@@ -130,21 +130,15 @@ public class NettyUtils {
             for (NetworkInterface networkInterface : Collections.list(allNetworkInterfaces)) {
                 for (InetAddress inetAddress : Collections.list(networkInterface.getInetAddresses())) {
                     if (inetAddress.isLinkLocalAddress()) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Ignoring link-local InetAddress {}", inetAddress);
-                        }
+                        LOG.debug("Ignoring link-local InetAddress {}", inetAddress);
                         continue;
                     }
                     if (inetAddress.isMulticastAddress()) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Ignoring multicast InetAddress {}", inetAddress);
-                        }
+                        LOG.debug("Ignoring multicast InetAddress {}", inetAddress);
                         continue;
                     }
                     if (inetAddress.isLoopbackAddress()) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Ignoring loopback InetAddress {}", inetAddress);
-                        }
+                        LOG.debug("Ignoring loopback InetAddress {}", inetAddress);
                         continue;
                     }
                     validInetAddresses.add(inetAddress);

+ 15 - 13
zookeeper-server/src/main/java/org/apache/zookeeper/common/SSLContextAndOptions.java

@@ -137,23 +137,21 @@ public class SSLContextAndOptions {
 
     private void configureSslParameters(SSLParameters sslParameters, boolean isClientSocket) {
         if (cipherSuites != null) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug(
-                    "Setup cipher suites for {} socket: {}",
-                    isClientSocket ? "client" : "server",
-                    Arrays.toString(cipherSuites));
-            }
+            LOG.debug(
+                "Setup cipher suites for {} socket: {}",
+                isClientSocket ? "client" : "server",
+                Arrays.toString(cipherSuites));
             sslParameters.setCipherSuites(cipherSuites);
         }
+
         if (enabledProtocols != null) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug(
-                    "Setup enabled protocols for {} socket: {}",
-                    isClientSocket ? "client" : "server",
-                    Arrays.toString(enabledProtocols));
-            }
+            LOG.debug(
+                "Setup enabled protocols for {} socket: {}",
+                isClientSocket ? "client" : "server",
+                Arrays.toString(enabledProtocols));
             sslParameters.setProtocols(enabledProtocols);
         }
+
         if (!isClientSocket) {
             switch (clientAuth) {
             case NEED:
@@ -200,7 +198,11 @@ public class SSLContextAndOptions {
             if (result < 1) {
                 // Timeout of 0 is not allowed, since an infinite timeout can permanently lock up an
                 // accept() thread.
-                LOG.warn("Invalid value for {}: {}, using the default value of {}", x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), result, X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS);
+                LOG.warn(
+                    "Invalid value for {}: {}, using the default value of {}",
+                    x509Util.getSslHandshakeDetectionTimeoutMillisProperty(),
+                    result,
+                    X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS);
                 result = X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS;
             }
         }

+ 13 - 18
zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java

@@ -297,7 +297,7 @@ public abstract class X509Util implements Closeable, AutoCloseable {
             LOG.error("Error creating SSL context and options", e);
             return DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS;
         } catch (Exception e) {
-            LOG.error("Error parsing config property " + getSslHandshakeDetectionTimeoutMillisProperty(), e);
+            LOG.error("Error parsing config property {}", getSslHandshakeDetectionTimeoutMillisProperty(), e);
             return DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS;
         }
     }
@@ -306,9 +306,8 @@ public abstract class X509Util implements Closeable, AutoCloseable {
     public SSLContextAndOptions createSSLContextAndOptions(ZKConfig config) throws SSLContextException {
         final String supplierContextClassName = config.getProperty(sslContextSupplierClassProperty);
         if (supplierContextClassName != null) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Loading SSLContext supplier from property '{}'", sslContextSupplierClassProperty);
-            }
+            LOG.debug("Loading SSLContext supplier from property '{}'", sslContextSupplierClassProperty);
+
             try {
                 Class<?> sslContextClass = Class.forName(supplierContextClassName);
                 Supplier<SSLContext> sslContextSupplier = (Supplier<SSLContext>) sslContextClass.getConstructor().newInstance();
@@ -343,7 +342,7 @@ public abstract class X509Util implements Closeable, AutoCloseable {
         // specified by the user.
 
         if (keyStoreLocationProp.isEmpty()) {
-            LOG.warn(getSslKeystoreLocationProperty() + " not specified");
+            LOG.warn("{} not specified", getSslKeystoreLocationProperty());
         } else {
             try {
                 keyManagers = new KeyManager[]{createKeyManager(keyStoreLocationProp, keyStorePasswordProp, keyStoreTypeProp)};
@@ -364,7 +363,7 @@ public abstract class X509Util implements Closeable, AutoCloseable {
         boolean sslClientHostnameVerificationEnabled = sslServerHostnameVerificationEnabled && shouldVerifyClientHostname();
 
         if (trustStoreLocationProp.isEmpty()) {
-            LOG.warn(getSslTruststoreLocationProperty() + " not specified");
+            LOG.warn("{} not specified", getSslTruststoreLocationProperty());
         } else {
             try {
                 trustManagers = new TrustManager[]{createTrustManager(trustStoreLocationProp, trustStorePasswordProp, trustStoreTypeProp, sslCrlEnabled, sslOcspEnabled, sslServerHostnameVerificationEnabled, sslClientHostnameVerificationEnabled)};
@@ -629,24 +628,20 @@ public abstract class X509Util implements Closeable, AutoCloseable {
         }
         // Note: we don't care about delete events
         if (shouldResetContext) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Attempting to reset default SSL context after receiving watch event: "
-                          + event.kind()
-                          + " with context: "
-                          + event.context());
-            }
+            LOG.debug(
+                "Attempting to reset default SSL context after receiving watch event: {} with context: {}",
+                event.kind(),
+                event.context());
             try {
                 this.resetDefaultSSLContextAndOptions();
             } catch (SSLContextException e) {
                 throw new RuntimeException(e);
             }
         } else {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Ignoring watch event and keeping previous default SSL context. Event kind: "
-                          + event.kind()
-                          + " with context: "
-                          + event.context());
-            }
+            LOG.debug(
+                "Ignoring watch event and keeping previous default SSL context. Event kind: {} with context: {}",
+                event.kind(),
+                event.context());
         }
     }
 

+ 2 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java

@@ -176,10 +176,8 @@ public class ZKConfig {
             throw new IllegalArgumentException("property key is null.");
         }
         String oldValue = properties.put(key, value);
-        if (LOG.isDebugEnabled()) {
-            if (null != oldValue && !oldValue.equals(value)) {
-                LOG.debug("key {}'s value {} is replaced with new value {}", key, oldValue, value);
-            }
+        if (null != oldValue && !oldValue.equals(value)) {
+            LOG.debug("key {}'s value {} is replaced with new value {}", key, oldValue, value);
         }
     }
 

+ 1 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKHostnameVerifier.java

@@ -151,9 +151,7 @@ final class ZKHostnameVerifier implements HostnameVerifier {
             verify(host, x509);
             return true;
         } catch (final SSLException ex) {
-            if (log.isDebugEnabled()) {
-                log.debug(ex.getMessage(), ex);
-            }
+            log.debug("Unexpected exception", ex);
             return false;
         }
     }

+ 4 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKTrustManager.java

@@ -151,7 +151,10 @@ public class ZKTrustManager extends X509ExtendedTrustManager {
             hostnameVerifier.verify(hostAddress, certificate);
         } catch (SSLException addressVerificationException) {
             try {
-                LOG.debug("Failed to verify host address: {} attempting to verify host name with reverse dns lookup", hostAddress, addressVerificationException);
+                LOG.debug(
+                    "Failed to verify host address: {} attempting to verify host name with reverse dns lookup",
+                    hostAddress,
+                    addressVerificationException);
                 hostName = inetAddress.getHostName();
                 hostnameVerifier.verify(hostName, certificate);
             } catch (SSLException hostnameVerificationException) {

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/jmx/MBeanRegistry.java

@@ -107,7 +107,7 @@ public class MBeanRegistry {
                 mapBean2Path.put(bean, path);
             }
         } catch (JMException e) {
-            LOG.warn("Failed to register MBean " + bean.getName());
+            LOG.warn("Failed to register MBean {}", bean.getName());
             throw e;
         }
     }
@@ -215,7 +215,7 @@ public class MBeanRegistry {
         try {
             return new ObjectName(beanName.toString());
         } catch (MalformedObjectNameException e) {
-            LOG.warn("Invalid name \"" + beanName.toString() + "\" for class " + bean.getClass().toString());
+            LOG.warn("Invalid name \"{}\" for class {}", beanName, bean.getClass());
             throw e;
         }
     }

+ 8 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/server/BlueThrottle.java

@@ -161,8 +161,10 @@ public class BlueThrottle {
             LOG.warn("Invalid global session weight {}. It should be larger than 0", globalWeight);
             DEFAULT_GLOBAL_SESSION_WEIGHT = 3;
         } else if (globalWeight < localWeight) {
-            LOG.warn("The global session weight {} is less than the local session weight {}. Use the local session weight.",
-                    globalWeight, localWeight);
+            LOG.warn(
+                "The global session weight {} is less than the local session weight {}. Use the local session weight.",
+                globalWeight,
+                localWeight);
             DEFAULT_GLOBAL_SESSION_WEIGHT = localWeight;
         } else {
             DEFAULT_GLOBAL_SESSION_WEIGHT = globalWeight;
@@ -179,8 +181,10 @@ public class BlueThrottle {
             LOG.warn("Invalid renew session weight {}. It should be larger than 0", renewWeight);
             DEFAULT_RENEW_SESSION_WEIGHT = 2;
         } else if (renewWeight < localWeight) {
-            LOG.warn("The renew session weight {} is less than the local session weight {}. Use the local session weight.",
-                    renewWeight, localWeight);
+            LOG.warn(
+                "The renew session weight {} is less than the local session weight {}. Use the local session weight.",
+                renewWeight,
+                localWeight);
             DEFAULT_RENEW_SESSION_WEIGHT = localWeight;
         } else {
             DEFAULT_RENEW_SESSION_WEIGHT = renewWeight;

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/ConnectionBean.java

@@ -100,7 +100,7 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
         try {
             zk.closeSession(sessionId);
         } catch (Exception e) {
-            LOG.warn("Unable to closeSession() for session: 0x" + getSessionId(), e);
+            LOG.warn("Unable to closeSession() for session: 0x{}", getSessionId(), e);
         }
     }
 

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/ContainerManager.java

@@ -64,7 +64,7 @@ public class ContainerManager {
         this.maxPerMinute = maxPerMinute;
         timer = new Timer("ContainerManagerTask", true);
 
-        LOG.info(String.format("Using checkIntervalMs=%d maxPerMinute=%d", checkIntervalMs, maxPerMinute));
+        LOG.info("Using checkIntervalMs={} maxPerMinute={}", checkIntervalMs, maxPerMinute);
     }
 
     /**

+ 33 - 24
zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java

@@ -296,7 +296,7 @@ public class DataTree {
             dataWatches = WatchManagerFactory.createWatchManager();
             childWatches = WatchManagerFactory.createWatchManager();
         } catch (Exception e) {
-            LOG.error("Unexpected exception when creating WatchManager, " + "exiting abnormally", e);
+            LOG.error("Unexpected exception when creating WatchManager, exiting abnormally", e);
             System.exit(ExitCode.UNEXPECTED_ERROR.getValue());
         }
     }
@@ -379,7 +379,7 @@ public class DataTree {
         StatsTrack updatedStat = null;
         if (node == null) {
             // should not happen
-            LOG.error("Missing count node for stat " + statNode);
+            LOG.error("Missing count node for stat {}", statNode);
             return;
         }
         synchronized (node) {
@@ -394,21 +394,25 @@ public class DataTree {
         StatsTrack thisStats = null;
         if (node == null) {
             // should not happen
-            LOG.error("Missing count node for quota " + quotaNode);
+            LOG.error("Missing count node for quota {}", quotaNode);
             return;
         }
         synchronized (node) {
             thisStats = new StatsTrack(new String(node.data));
         }
         if (thisStats.getCount() > -1 && (thisStats.getCount() < updatedStat.getCount())) {
-            LOG.warn("Quota exceeded: " + lastPrefix
-                     + " count=" + updatedStat.getCount()
-                     + " limit=" + thisStats.getCount());
+            LOG.warn(
+                "Quota exceeded: {} count={} limit={}",
+                lastPrefix,
+                updatedStat.getCount(),
+                thisStats.getCount());
         }
         if (thisStats.getBytes() > -1 && (thisStats.getBytes() < updatedStat.getBytes())) {
-            LOG.warn("Quota exceeded: " + lastPrefix
-                     + " bytes=" + updatedStat.getBytes()
-                     + " limit=" + thisStats.getBytes());
+            LOG.warn(
+                "Quota exceeded: {} bytes={} limit={}",
+                lastPrefix,
+                updatedStat.getBytes(),
+                thisStats.getBytes());
         }
     }
 
@@ -642,6 +646,7 @@ public class DataTree {
                 ZooTrace.EVENT_DELIVERY_TRACE_MASK,
                 "childWatches.triggerWatch " + parentName);
         }
+
         WatcherOrBitSet processed = dataWatches.triggerWatch(path, EventType.NodeDeleted);
         childWatches.triggerWatch(path, EventType.NodeDeleted, processed);
         childWatches.triggerWatch("".equals(parentName) ? "/" : parentName, EventType.NodeChildrenChanged);
@@ -1068,7 +1073,7 @@ public class DataTree {
             try {
                 setCversionPzxid(parentName, cTxn.getParentCVersion(), header.getZxid());
             } catch (KeeperException.NoNodeException e) {
-                LOG.error("Failed to set parent cversion for: " + parentName, e);
+                LOG.error("Failed to set parent cversion for: {}", parentName, e);
                 rc.err = e.code().intValue();
             }
         } else if (rc.err != Code.OK.intValue()) {
@@ -1147,9 +1152,10 @@ public class DataTree {
                 paths2DeleteLocal.remove(path);
             }
             if (!paths2DeleteLocal.isEmpty()) {
-                LOG.warn("Unexpected extra paths under session {} which "
-                        + "are not in txn 0x{}", paths2DeleteLocal,
-                        Long.toHexString(zxid));
+                LOG.warn(
+                    "Unexpected extra paths under session {} which are not in txn 0x{}",
+                    paths2DeleteLocal,
+                    Long.toHexString(zxid));
             }
         }
 
@@ -1160,12 +1166,12 @@ public class DataTree {
         for (String path : paths2Delete) {
             try {
                 deleteNode(path, zxid);
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Deleting ephemeral node {} for session 0x{}", path, Long.toHexString(session));
-                }
+                LOG.debug("Deleting ephemeral node {} for session 0x{}", path, Long.toHexString(session));
             } catch (NoNodeException e) {
-                LOG.warn("Ignoring NoNodeException for path {} while removing ephemeral for dead session 0x{}",
-                        path, Long.toHexString(session));
+                LOG.warn(
+                    "Ignoring NoNodeException for path {} while removing ephemeral for dead session 0x{}",
+                    path,
+                    Long.toHexString(session));
             }
         }
     }
@@ -1224,7 +1230,7 @@ public class DataTree {
         DataNode node = getNode(statPath);
         // it should exist
         if (node == null) {
-            LOG.warn("Missing quota stat node " + statPath);
+            LOG.warn("Missing quota stat node {}", statPath);
             return;
         }
         synchronized (node) {
@@ -1689,7 +1695,7 @@ public class DataTree {
             }
             return true;
         } catch (EOFException e) {
-            LOG.warn("Got EOF exception while reading the digest, " + "likely due to the reading an older snapshot.");
+            LOG.warn("Got EOF exception while reading the digest, likely due to the reading an older snapshot.");
             return false;
         }
     }
@@ -1703,8 +1709,10 @@ public class DataTree {
     public void compareSnapshotDigests(long zxid) {
         if (zxid == digestFromLoadedSnapshot.zxid) {
             if (digestCalculator.getDigestVersion() != digestFromLoadedSnapshot.digestVersion) {
-                LOG.info("Digest version changed, local: {}, new: {}, "
-                         + "skip comparing digest now.", digestFromLoadedSnapshot.digestVersion, digestCalculator.getDigestVersion());
+                LOG.info(
+                    "Digest version changed, local: {}, new: {}, skip comparing digest now.",
+                    digestFromLoadedSnapshot.digestVersion,
+                    digestCalculator.getDigestVersion());
                 digestFromLoadedSnapshot = null;
                 return;
             }
@@ -1713,8 +1721,9 @@ public class DataTree {
             }
             digestFromLoadedSnapshot = null;
         } else if (digestFromLoadedSnapshot.zxid != 0 && zxid > digestFromLoadedSnapshot.zxid) {
-            LOG.error("Watching for zxid 0x{} during snapshot recovery, but it wasn't found.",
-                      Long.toHexString(digestFromLoadedSnapshot.zxid));
+            LOG.error(
+                "Watching for zxid 0x{} during snapshot recovery, but it wasn't found.",
+                Long.toHexString(digestFromLoadedSnapshot.zxid));
         }
     }
 

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/DatadirCleanupManager.java

@@ -75,8 +75,8 @@ public class DatadirCleanupManager {
         this.dataLogDir = dataLogDir;
         this.snapRetainCount = snapRetainCount;
         this.purgeInterval = purgeInterval;
-        LOG.info("autopurge.snapRetainCount set to " + snapRetainCount);
-        LOG.info("autopurge.purgeInterval set to " + purgeInterval);
+        LOG.info("autopurge.snapRetainCount set to {}", snapRetainCount);
+        LOG.info("autopurge.purgeInterval set to {}", purgeInterval);
     }
 
     /**

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java

@@ -536,14 +536,14 @@ public class FinalRequestProcessor implements RequestProcessor {
         } catch (Exception e) {
             // log at error level as we are returning a marshalling
             // error to the user
-            LOG.error("Failed to process " + request, e);
+            LOG.error("Failed to process {}", request, e);
             StringBuilder sb = new StringBuilder();
             ByteBuffer bb = request.request;
             bb.rewind();
             while (bb.hasRemaining()) {
                 sb.append(Integer.toHexString(bb.get() & 0xff));
             }
-            LOG.error("Dumping request buffer: 0x" + sb.toString());
+            LOG.error("Dumping request buffer: 0x{}", sb.toString());
             err = Code.MARSHALLINGERROR;
         }
 

+ 18 - 18
zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java

@@ -131,7 +131,9 @@ public class NIOServerCnxn extends ServerCnxn {
      * asynchronous writes.
      */
     public void sendBuffer(ByteBuffer... buffers) {
-        LOG.trace("Add a buffer to outgoingBuffers, sk {} is valid: {}", sk, sk.isValid());
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Add a buffer to outgoingBuffers, sk {} is valid: {}", sk, sk.isValid());
+        }
 
         synchronized (outgoingBuffers) {
             for (ByteBuffer buffer : buffers) {
@@ -312,7 +314,7 @@ public class NIOServerCnxn extends ServerCnxn {
     void doIO(SelectionKey k) throws InterruptedException {
         try {
             if (!isSocketOpen()) {
-                LOG.warn("trying to do i/o on a null socket for session:0x" + Long.toHexString(sessionId));
+                LOG.warn("trying to do i/o on a null socket for session: 0x{}", Long.toHexString(sessionId));
 
                 return;
             }
@@ -348,7 +350,7 @@ public class NIOServerCnxn extends ServerCnxn {
                 }
             }
         } catch (CancelledKeyException e) {
-            LOG.warn("CancelledKeyException causing close of session 0x" + Long.toHexString(sessionId));
+            LOG.warn("CancelledKeyException causing close of session: 0x{}", Long.toHexString(sessionId));
 
             LOG.debug("CancelledKeyException stack trace", e);
 
@@ -357,21 +359,16 @@ public class NIOServerCnxn extends ServerCnxn {
             // expecting close to log session closure
             close();
         } catch (EndOfStreamException e) {
-            LOG.warn(e.getMessage());
+            LOG.warn("Unexpected exception", e);
             // expecting close to log session closure
             close(e.getReason());
         } catch (ClientCnxnLimitException e) {
             // Common case exception, print at debug level
             ServerMetrics.getMetrics().CONNECTION_REJECTED.add(1);
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Exception causing close of session 0x" + Long.toHexString(sessionId)
-                          + ": " + e.getMessage());
-            }
+            LOG.warn("Closing session 0x{}", Long.toHexString(sessionId), e);
             close(DisconnectReason.CLIENT_CNX_LIMIT);
         } catch (IOException e) {
-            LOG.warn("Exception causing close of session 0x" + Long.toHexString(sessionId) + ": " + e.getMessage());
-
-            LOG.debug("IOException stack trace", e);
+            LOG.warn("Close of session 0x{}", Long.toHexString(sessionId), e);
             close(DisconnectReason.IO_EXCEPTION);
         }
     }
@@ -488,7 +485,7 @@ public class NIOServerCnxn extends ServerCnxn {
             try {
                 k.cancel();
             } catch (Exception e) {
-                LOG.error("Error cancelling command selection key ", e);
+                LOG.error("Error cancelling command selection key", e);
             }
         }
 
@@ -505,7 +502,7 @@ public class NIOServerCnxn extends ServerCnxn {
             return true;
         }
 
-        LOG.info("Processing " + cmd + " command from " + sock.socket().getRemoteSocketAddress());
+        LOG.info("Processing {} command from {}", cmd, sock.socket().getRemoteSocketAddress());
 
         if (len == FourLetterCommands.setTraceMaskCmd) {
             incomingBuffer = ByteBuffer.allocate(8);
@@ -614,11 +611,14 @@ public class NIOServerCnxn extends ServerCnxn {
             return;
         }
 
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Closed socket connection for client "
-                      + sock.socket().getRemoteSocketAddress()
-                      + (sessionId != 0 ? " which had sessionid 0x" + Long.toHexString(sessionId) : " (no session established for client)"));
-        }
+        String logMsg = String.format(
+            "Closed socket connection for client %s %s",
+            sock.socket().getRemoteSocketAddress(),
+            sessionId != 0
+                ? "which had sessionid 0x" + Long.toHexString(sessionId)
+                : "(no session established for client)"
+            );
+        LOG.debug(logMsg);
 
         closeSock(sock);
     }

+ 19 - 15
zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxnFactory.java

@@ -84,7 +84,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
     static {
         Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
             public void uncaughtException(Thread t, Throwable e) {
-                LOG.error("Thread " + t + " died", e);
+                LOG.error("Thread {} died", t, e);
             }
         });
         /**
@@ -136,7 +136,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
             try {
                 selector.close();
             } catch (IOException e) {
-                LOG.warn("ignored exception during selector close " + e.getMessage());
+                LOG.warn("ignored exception during selector close.", e);
             }
         }
 
@@ -156,7 +156,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
                     // Hard close immediately, discarding buffers
                     sc.socket().setSoLinger(true, 0);
                 } catch (SocketException e) {
-                    LOG.warn("Unable to set socket linger to 0, socket close" + " may stall in CLOSE_WAIT", e);
+                    LOG.warn("Unable to set socket linger to 0, socket close may stall in CLOSE_WAIT", e);
                 }
                 NIOServerCnxn.closeSock(sc);
             }
@@ -235,7 +235,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
                             pauseAccept(10);
                         }
                     } else {
-                        LOG.warn("Unexpected ops in accept select " + key.readyOps());
+                        LOG.warn("Unexpected ops in accept select {}", key.readyOps());
                     }
                 }
             } catch (IOException e) {
@@ -428,7 +428,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
                     if (key.isReadable() || key.isWritable()) {
                         handleIO(key);
                     } else {
-                        LOG.warn("Unexpected ops in select " + key.readyOps());
+                        LOG.warn("Unexpected ops in select {}", key.readyOps());
                     }
                 }
             } catch (IOException e) {
@@ -654,11 +654,12 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
         numWorkerThreads = Integer.getInteger(ZOOKEEPER_NIO_NUM_WORKER_THREADS, 2 * numCores);
         workerShutdownTimeoutMS = Long.getLong(ZOOKEEPER_NIO_SHUTDOWN_TIMEOUT, 5000);
 
-        LOG.info("Configuring NIO connection handler with "
-                 + (sessionlessCnxnTimeout / 1000) + "s sessionless connection timeout, "
-                 + numSelectorThreads + " selector thread(s), "
-                 + (numWorkerThreads > 0 ? numWorkerThreads : "no") + " worker threads, and "
-                 + (directBufferBytes == 0 ? "gathered writes." : ("" + (directBufferBytes / 1024) + " kB direct buffers.")));
+        String logMsg = "Configuring NIO connection handler with "
+            + (sessionlessCnxnTimeout / 1000) + "s sessionless connection timeout, "
+            + numSelectorThreads + " selector thread(s), "
+            + (numWorkerThreads > 0 ? numWorkerThreads : "no") + " worker threads, and "
+            + (directBufferBytes == 0 ? "gathered writes." : ("" + (directBufferBytes / 1024) + " kB direct buffers."));
+        LOG.info(logMsg);
         for (int i = 0; i < numSelectorThreads; ++i) {
             selectorThreads.add(new SelectorThread(i));
         }
@@ -666,7 +667,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
         listenBacklog = backlog;
         this.ss = ServerSocketChannel.open();
         ss.socket().setReuseAddress(true);
-        LOG.info("binding to port " + addr);
+        LOG.info("binding to port {}", addr);
         if (listenBacklog == -1) {
             ss.socket().bind(addr);
         } else {
@@ -694,18 +695,18 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
             try {
                 acceptThread.join();
             } catch (InterruptedException e) {
-                LOG.error("Error joining old acceptThread when reconfiguring client port {}", e.getMessage());
+                LOG.error("Error joining old acceptThread when reconfiguring client port.", e);
                 Thread.currentThread().interrupt();
             }
             this.ss = ServerSocketChannel.open();
             ss.socket().setReuseAddress(true);
-            LOG.info("binding to port " + addr);
+            LOG.info("binding to port {}", addr);
             ss.socket().bind(addr);
             ss.configureBlocking(false);
             acceptThread = new AcceptThread(ss, addr, selectorThreads);
             acceptThread.start();
         } catch (IOException e) {
-            LOG.error("Error reconfiguring client port to {} {}", addr, e.getMessage());
+            LOG.error("Error reconfiguring client port to {}", addr, e);
             tryClose(oldSS);
         }
     }
@@ -853,7 +854,10 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory {
                 // This will remove the cnxn from cnxns
                 cnxn.close(reason);
             } catch (Exception e) {
-                LOG.warn("Ignoring exception closing cnxn sessionid 0x" + Long.toHexString(cnxn.getSessionId()), e);
+                LOG.warn(
+                    "Ignoring exception closing cnxn session id 0x{}",
+                    Long.toHexString(cnxn.getSessionId()),
+                    e);
             }
         }
     }

+ 12 - 18
zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java

@@ -94,9 +94,8 @@ public class NettyServerCnxn extends ServerCnxn {
     public void close() {
         closingChannel = true;
 
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("close called for sessionid:0x{}", Long.toHexString(sessionId));
-        }
+        LOG.debug("close called for session id: 0x{}", Long.toHexString(sessionId));
+
         setStale();
 
         // ZOOKEEPER-2743:
@@ -106,14 +105,11 @@ public class NettyServerCnxn extends ServerCnxn {
 
         // if this is not in cnxns then it's already closed
         if (!factory.cnxns.remove(this)) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("cnxns size:{}", factory.cnxns.size());
-            }
+            LOG.debug("cnxns size:{}", factory.cnxns.size());
             return;
         }
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("close in progress for sessionid:0x{}", Long.toHexString(sessionId));
-        }
+
+        LOG.debug("close in progress for session id: 0x{}", Long.toHexString(sessionId));
 
         factory.removeCnxnFromSessionMap(this);
 
@@ -165,9 +161,7 @@ public class NettyServerCnxn extends ServerCnxn {
         try {
             sendResponse(h, e, "notification");
         } catch (IOException e1) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Problem sending to " + getRemoteSocketAddress(), e1);
-            }
+            LOG.debug("Problem sending to {}", getRemoteSocketAddress(), e1);
             close();
         }
     }
@@ -335,9 +329,7 @@ public class NettyServerCnxn extends ServerCnxn {
      */
     void processMessage(ByteBuf buf) {
         checkIsInEventLoop("processMessage");
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("0x{} queuedBuffer: {}", Long.toHexString(sessionId), queuedBuffer);
-        }
+        LOG.debug("0x{} queuedBuffer: {}", Long.toHexString(sessionId), queuedBuffer);
 
         if (LOG.isTraceEnabled()) {
             LOG.trace("0x{} buf {}", Long.toHexString(sessionId), ByteBufUtil.hexDump(buf));
@@ -364,7 +356,9 @@ public class NettyServerCnxn extends ServerCnxn {
                 // Have to check !closingChannel, because an error in
                 // receiveMessage() could have led to close() being called.
                 if (!closingChannel && buf.isReadable()) {
-                    LOG.trace("Before copy {}", buf);
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("Before copy {}", buf);
+                    }
 
                     if (queuedBuffer == null) {
                         queuedBuffer = channel.alloc().compositeBuffer();
@@ -514,13 +508,13 @@ public class NettyServerCnxn extends ServerCnxn {
                 }
             }
         } catch (IOException e) {
-            LOG.warn("Closing connection to " + getRemoteSocketAddress(), e);
+            LOG.warn("Closing connection to {}", getRemoteSocketAddress(), e);
             close(DisconnectReason.IO_EXCEPTION);
         } catch (ClientCnxnLimitException e) {
             // Common case exception, print at debug level
             ServerMetrics.getMetrics().CONNECTION_REJECTED.add(1);
 
-            LOG.debug("Closing connection to " + getRemoteSocketAddress(), e);
+            LOG.debug("Closing connection to {}", getRemoteSocketAddress(), e);
             close(DisconnectReason.CLIENT_RATE_LIMIT);
         }
     }

+ 10 - 7
zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java

@@ -212,6 +212,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
             if (LOG.isTraceEnabled()) {
                 LOG.trace("Channel inactive {}", ctx.channel());
             }
+
             allChannels.remove(ctx.channel());
             NettyServerCnxn cnxn = ctx.channel().attr(CONNECTION_ATTRIBUTE).getAndSet(null);
             if (cnxn != null) {
@@ -267,7 +268,9 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
         @Override
         public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
             try {
-                LOG.trace("message received called {}", msg);
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("message received called {}", msg);
+                }
                 try {
                     LOG.debug("New message {} from {}", msg, ctx.channel());
                     NettyServerCnxn cnxn = ctx.channel().attr(CONNECTION_ATTRIBUTE).get();
@@ -291,7 +294,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                 NettyServerCnxn cnxn = ctx.channel().attr(CONNECTION_ATTRIBUTE).get();
                 if (cnxn != null && cnxn.getQueuedReadableBytes() == 0 && cnxn.readIssuedAfterReadComplete == 0) {
                     ctx.read();
-                    LOG.debug("Issued a read since we do not have " + "anything to consume after channelReadComplete");
+                    LOG.debug("Issued a read since we do not have anything to consume after channelReadComplete");
                 }
             }
 
@@ -302,7 +305,9 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
         // Note: this listener is only added when LOG.isTraceEnabled() is true,
         // so it should not do any work other than trace logging.
         private final GenericFutureListener<Future<Void>> onWriteCompletedTracer = (f) -> {
-            LOG.trace("write success: {}", f.isSuccess());
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("write success: {}", f.isSuccess());
+            }
         };
 
         @Override
@@ -330,9 +335,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
          */
         public void operationComplete(Future<Channel> future) {
             if (future.isSuccess()) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Successful handshake with session 0x{}", Long.toHexString(cnxn.getSessionId()));
-                }
+                LOG.debug("Successful handshake with session 0x{}", Long.toHexString(cnxn.getSessionId()));
                 SSLEngine eng = sslHandler.engine();
                 // Don't try to verify certificate if we didn't ask client to present one
                 if (eng.getNeedClientAuth() || eng.getWantClientAuth()) {
@@ -511,7 +514,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                 // This will remove the cnxn from cnxns
                 cnxn.close(reason);
             } catch (Exception e) {
-                LOG.warn("Ignoring exception closing cnxn sessionid 0x" + Long.toHexString(cnxn.getSessionId()), e);
+                LOG.warn("Ignoring exception closing cnxn sessionid 0x{}", Long.toHexString(cnxn.getSessionId()), e);
             }
         }
 

+ 11 - 16
zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java

@@ -58,7 +58,6 @@ import org.apache.zookeeper.proto.SetDataRequest;
 import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord;
 import org.apache.zookeeper.server.auth.ProviderRegistry;
 import org.apache.zookeeper.server.auth.ServerAuthenticationProvider;
-import org.apache.zookeeper.server.quorum.Leader.XidRolloverException;
 import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
@@ -142,11 +141,6 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
                 request.prepStartTime = Time.currentElapsedTime();
                 pRequest(request);
             }
-        } catch (RequestProcessorException e) {
-            if (e.getCause() instanceof XidRolloverException) {
-                LOG.info(e.getCause().getMessage());
-            }
-            handleException(this.getName(), e);
         } catch (Exception e) {
             handleException(this.getName(), e);
         }
@@ -287,7 +281,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
     private String validatePathForCreate(String path, long sessionId) throws BadArgumentsException {
         int lastSlash = path.lastIndexOf('/');
         if (lastSlash == -1 || path.indexOf('\0') != -1 || failCreate) {
-            LOG.info("Invalid path %s with session 0x%s", path, Long.toHexString(sessionId));
+            LOG.info("Invalid path {} with session 0x{}", path, Long.toHexString(sessionId));
             throw new KeeperException.BadArgumentsException(path);
         }
         return path.substring(0, lastSlash);
@@ -572,7 +566,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
                 checkAndIncVersion(nodeRecord.stat.getVersion(), checkVersionRequest.getVersion(), path)));
             break;
         default:
-            LOG.warn("unknown type " + type);
+            LOG.warn("unknown type {}", type);
             break;
         }
     }
@@ -810,7 +804,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
                 zks.sessionTracker.checkSession(request.sessionId, request.getOwner());
                 break;
             default:
-                LOG.warn("unknown type " + request.type);
+                LOG.warn("unknown type {}", request.type);
                 break;
             }
         } catch (KeeperException e) {
@@ -820,16 +814,17 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
             }
 
             if (e.code().intValue() > Code.APIERROR.intValue()) {
-                LOG.info("Got user-level KeeperException when processing {} Error Path:{} Error:{}",
-                         request.toString(),
-                         e.getPath(),
-                         e.getMessage());
+                LOG.info(
+                    "Got user-level KeeperException when processing {} Error Path:{} Error:{}",
+                    request.toString(),
+                    e.getPath(),
+                    e.getMessage());
             }
             request.setException(e);
         } catch (Exception e) {
             // log at error level as we are returning a marshalling
             // error to the user
-            LOG.error("Failed to process " + request, e);
+            LOG.error("Failed to process {}", request, e);
 
             StringBuilder sb = new StringBuilder();
             ByteBuffer bb = request.request;
@@ -842,7 +837,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
                 sb.append("request buffer is null");
             }
 
-            LOG.error("Dumping request buffer: 0x" + sb.toString());
+            LOG.error("Dumping request buffer: 0x{}", sb.toString());
             if (request.getHdr() != null) {
                 request.getHdr().setType(OpCode.error);
                 request.setTxn(new ErrorTxn(Code.MARSHALLINGERROR.intValue()));
@@ -926,7 +921,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
                 for (Id cid : authInfo) {
                     ServerAuthenticationProvider ap = ProviderRegistry.getServerProvider(cid.getScheme());
                     if (ap == null) {
-                        LOG.error("Missing AuthenticationProvider for " + cid.getScheme());
+                        LOG.error("Missing AuthenticationProvider for {}", cid.getScheme());
                     } else if (ap.isAuthenticated()) {
                         authIdValid = true;
                         rv.add(new ACL(a.getPerms(), cid));

+ 6 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java

@@ -148,12 +148,14 @@ public class PurgeTxnLog {
 
         // remove the old files
         for (File f : files) {
-            final String msg = "Removing file: "
-                               + DateFormat.getDateTimeInstance().format(f.lastModified())
-                               + "\t"
-                               + f.getPath();
+            final String msg = String.format(
+                "Removing file: %s\t%s",
+                DateFormat.getDateTimeInstance().format(f.lastModified()),
+                f.getPath());
+
             LOG.info(msg);
             System.out.println(msg);
+
             if (!f.delete()) {
                 System.err.println("Failed to remove " + f.getPath());
             }

+ 3 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/ReferenceCountedACLCache.java

@@ -89,7 +89,7 @@ public class ReferenceCountedACLCache {
         }
         List<ACL> acls = longKeyMap.get(longVal);
         if (acls == null) {
-            LOG.error("ERROR: ACL not available for long " + longVal);
+            LOG.error("ERROR: ACL not available for long {}", longVal);
             throw new RuntimeException("Failed to fetch acls for " + longVal);
         }
         return acls;
@@ -171,7 +171,7 @@ public class ReferenceCountedACLCache {
         }
 
         if (!longKeyMap.containsKey(acl)) {
-            LOG.info("Ignoring acl " + acl + " as it does not exist in the cache");
+            LOG.info("Ignoring acl {} as it does not exist in the cache", acl);
             return;
         }
 
@@ -189,7 +189,7 @@ public class ReferenceCountedACLCache {
         }
 
         if (!longKeyMap.containsKey(acl)) {
-            LOG.info("Ignoring acl " + acl + " as it does not exist in the cache");
+            LOG.info("Ignoring acl {} as it does not exist in the cache", acl);
             return;
         }
 

+ 13 - 20
zookeeper-server/src/main/java/org/apache/zookeeper/server/SessionTrackerImpl.java

@@ -193,17 +193,15 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
     }
 
     private void logTraceTouchSession(long sessionId, int timeout, String sessionStatus) {
-        if (!LOG.isTraceEnabled()) {
-            return;
-        }
-
-        String msg = MessageFormat.format(
-            "SessionTrackerImpl --- Touch {0}session: 0x{1} with timeout {2}",
-            sessionStatus,
-            Long.toHexString(sessionId),
-            Integer.toString(timeout));
+        if (LOG.isTraceEnabled()) {
+            String msg = MessageFormat.format(
+                "SessionTrackerImpl --- Touch {0}session: 0x{1} with timeout {2}",
+                sessionStatus,
+                Long.toHexString(sessionId),
+                Integer.toString(timeout));
 
-        ZooTrace.logTraceMessage(LOG, ZooTrace.CLIENT_PING_TRACE_MASK, msg);
+            ZooTrace.logTraceMessage(LOG, ZooTrace.CLIENT_PING_TRACE_MASK, msg);
+        }
     }
 
     private void logTraceTouchInvalidSession(long sessionId, int timeout) {
@@ -220,8 +218,9 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
 
     public synchronized void setSessionClosing(long sessionId) {
         if (LOG.isTraceEnabled()) {
-            LOG.trace("Session closing: 0x" + Long.toHexString(sessionId));
+            LOG.trace("Session closing: 0x{}", Long.toHexString(sessionId));
         }
+
         SessionImpl s = sessionsById.get(sessionId);
         if (s == null) {
             return;
@@ -230,9 +229,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
     }
 
     public synchronized void removeSession(long sessionId) {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Removing session 0x{}", Long.toHexString(sessionId));
-        }
+        LOG.debug("Removing session 0x{}", Long.toHexString(sessionId));
         SessionImpl s = sessionsById.remove(sessionId);
         sessionsWithTimeout.remove(sessionId);
         if (LOG.isTraceEnabled()) {
@@ -278,9 +275,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
             session = existedSession;
         } else {
             added = true;
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Adding session 0x{}", Long.toHexString(id));
-            }
+            LOG.debug("Adding session 0x{}", Long.toHexString(id));
         }
 
         if (LOG.isTraceEnabled()) {
@@ -305,9 +300,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
     }
 
     public synchronized void checkSession(long sessionId, Object owner) throws KeeperException.SessionExpiredException, KeeperException.SessionMovedException, KeeperException.UnknownSessionException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Checking session 0x" + Long.toHexString(sessionId));
-        }
+        LOG.debug("Checking session 0x{}", Long.toHexString(sessionId));
         SessionImpl session = sessionsById.get(sessionId);
 
         if (session == null) {

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java

@@ -259,7 +259,7 @@ public class SyncRequestProcessor extends ZooKeeperCriticalThread implements Req
             this.join();
             this.flush();
         } catch (InterruptedException e) {
-            LOG.warn("Interrupted while wating for " + this + " to finish");
+            LOG.warn("Interrupted while wating for {} to finish", this);
             Thread.currentThread().interrupt();
         } catch (IOException e) {
             LOG.warn("Got IO exception during shutdown");

+ 23 - 13
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java

@@ -115,14 +115,19 @@ public class ZKDatabase {
                             Double.toString(DEFAULT_SNAPSHOT_SIZE_FACTOR)));
             if (snapshotSizeFactor > 1) {
                 snapshotSizeFactor = DEFAULT_SNAPSHOT_SIZE_FACTOR;
-                LOG.warn("The configured {} is invalid, going to use the default {}",
-                         SNAPSHOT_SIZE_FACTOR,
-                         DEFAULT_SNAPSHOT_SIZE_FACTOR);
+                LOG.warn(
+                    "The configured {} is invalid, going to use the default {}",
+                    SNAPSHOT_SIZE_FACTOR,
+                    DEFAULT_SNAPSHOT_SIZE_FACTOR);
             }
         } catch (NumberFormatException e) {
-            LOG.error("Error parsing {}, using default value {}", SNAPSHOT_SIZE_FACTOR, DEFAULT_SNAPSHOT_SIZE_FACTOR);
+            LOG.error(
+                "Error parsing {}, using default value {}",
+                SNAPSHOT_SIZE_FACTOR,
+                DEFAULT_SNAPSHOT_SIZE_FACTOR);
             snapshotSizeFactor = DEFAULT_SNAPSHOT_SIZE_FACTOR;
         }
+
         LOG.info("{} = {}", SNAPSHOT_SIZE_FACTOR, snapshotSizeFactor);
 
         try {
@@ -131,13 +136,16 @@ public class ZKDatabase {
                             Integer.toString(DEFAULT_COMMIT_LOG_COUNT)));
             if (commitLogCount < DEFAULT_COMMIT_LOG_COUNT) {
                 commitLogCount = DEFAULT_COMMIT_LOG_COUNT;
-                LOG.warn("The configured commitLogCount {} is less than the recommended {}"
-                         + ", going to use the recommended one",
-                         COMMIT_LOG_COUNT, DEFAULT_COMMIT_LOG_COUNT);
+                LOG.warn(
+                    "The configured commitLogCount {} is less than the recommended {}, going to use the recommended one",
+                    COMMIT_LOG_COUNT,
+                    DEFAULT_COMMIT_LOG_COUNT);
             }
         } catch (NumberFormatException e) {
-            LOG.error("Error parsing {} - use default value {}",
-                    COMMIT_LOG_COUNT, DEFAULT_COMMIT_LOG_COUNT);
+            LOG.error(
+                "Error parsing {} - use default value {}",
+                COMMIT_LOG_COUNT,
+                DEFAULT_COMMIT_LOG_COUNT);
             commitLogCount = DEFAULT_COMMIT_LOG_COUNT;
         }
         LOG.info("{}={}", COMMIT_LOG_COUNT, commitLogCount);
@@ -277,7 +285,7 @@ public class ZKDatabase {
         initialized = true;
         long loadTime = Time.currentElapsedTime() - startTime;
         ServerMetrics.getMetrics().DB_INIT_TIME.add(loadTime);
-        LOG.info("Snapshot loaded in " + loadTime + " ms");
+        LOG.info("Snapshot loaded in {} ms", loadTime);
         return zxid;
     }
 
@@ -331,7 +339,7 @@ public class ZKDatabase {
     public boolean isTxnLogSyncEnabled() {
         boolean enabled = snapshotSizeFactor >= 0;
         if (enabled) {
-            LOG.info("On disk txn sync enabled with snapshotSizeFactor " + snapshotSizeFactor);
+            LOG.info("On disk txn sync enabled with snapshotSizeFactor {}", snapshotSizeFactor);
         } else {
             LOG.info("On disk txn sync disabled");
         }
@@ -373,7 +381,9 @@ public class ZKDatabase {
             // If we cannot guarantee that this is strictly the starting txn
             // after a given zxid, we should fail.
             if ((itr.getHeader() != null) && (itr.getHeader().getZxid() > startZxid)) {
-                LOG.warn("Unable to find proposals from txnlog for zxid: " + startZxid);
+                LOG.warn(
+                    "Unable to find proposals from txnlog for zxid: 0x{}",
+                    Long.toHexString(startZxid));
                 itr.close();
                 return TxnLogProposalIterator.EMPTY_ITERATOR;
             }
@@ -381,7 +391,7 @@ public class ZKDatabase {
             if (sizeLimit > 0) {
                 long txnSize = itr.getStorageSize();
                 if (txnSize > sizeLimit) {
-                    LOG.info("Txnlog size: " + txnSize + " exceeds sizeLimit: " + sizeLimit);
+                    LOG.info("Txnlog size: {} exceeds sizeLimit: {}", txnSize, sizeLimit);
                     itr.close();
                     return TxnLogProposalIterator.EMPTY_ITERATOR;
                 }

+ 82 - 68
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java

@@ -123,11 +123,11 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         Environment.logEnv("Server environment:", LOG);
 
         enableEagerACLCheck = Boolean.getBoolean(ENABLE_EAGER_ACL_CHECK);
-        LOG.info(ENABLE_EAGER_ACL_CHECK + " = {}", enableEagerACLCheck);
+        LOG.info("{} = {}", ENABLE_EAGER_ACL_CHECK, enableEagerACLCheck);
 
         skipACL = System.getProperty(SKIP_ACL, "no").equals("yes");
         if (skipACL) {
-            LOG.info(SKIP_ACL + "==\"yes\", ACL checks will be skipped");
+            LOG.info("{}==\"yes\", ACL checks will be skipped", SKIP_ACL);
         }
 
         digestEnabled = Boolean.parseBoolean(System.getProperty(ZOOKEEPER_DIGEST_ENABLED, "true"));
@@ -232,7 +232,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
             throw new IllegalArgumentException(msg);
         }
 
-        LOG.info(INT_BUFFER_STARTING_SIZE_BYTES + " = " + intBufferStartingSizeBytes);
+        LOG.info("{} = {}", INT_BUFFER_STARTING_SIZE_BYTES, intBufferStartingSizeBytes);
     }
 
     // Connection throttling
@@ -316,13 +316,20 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
 
         this.initLargeRequestThrottlingSettings();
 
-        LOG.info("Created server with tickTime " + tickTime
-                 + " minSessionTimeout " + getMinSessionTimeout()
-                 + " maxSessionTimeout " + getMaxSessionTimeout()
-                 + " clientPortListenBacklog " + getClientPortListenBacklog()
-                 + " datadir " + txnLogFactory.getDataDir()
-                 + " snapdir " + txnLogFactory.getSnapDir());
-
+        LOG.info(
+            "Created server with"
+                + " tickTime {}"
+                + " minSessionTimeout {}"
+                + " maxSessionTimeout {}"
+                + " clientPortListenBacklog {}"
+                + " datadir {}"
+                + " snapdir {}",
+            tickTime,
+            getMinSessionTimeout(),
+            getMaxSessionTimeout(),
+            getClientPortListenBacklog(),
+            txnLogFactory.getDataDir(),
+            txnLogFactory.getSnapDir());
     }
 
     public String getInitialConfig() {
@@ -499,7 +506,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
             System.exit(ExitCode.TXNLOG_ERROR_TAKING_SNAPSHOT.getValue());
         }
         long elapsed = Time.currentElapsedTime() - start;
-        LOG.info("Snapshot taken in " + elapsed + " ms");
+        LOG.info("Snapshot taken in {} ms", elapsed);
         ServerMetrics.getMetrics().SNAPSHOT_TIME.add(elapsed);
     }
 
@@ -558,7 +565,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     }
 
     public void closeSession(long sessionId) {
-        LOG.info("Closing session 0x" + Long.toHexString(sessionId));
+        LOG.info("Closing session 0x{}", Long.toHexString(sessionId));
 
         // we do not want to wait for a session close. send it as soon as we
         // detect it!
@@ -580,9 +587,10 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
 
     public void expire(Session session) {
         long sessionId = session.getSessionId();
-        LOG.info("Expiring session 0x" + Long.toHexString(sessionId)
-                 + ", timeout of " + session.getTimeout()
-                 + "ms exceeded");
+        LOG.info(
+            "Expiring session 0x{}, timeout of {}ms exceeded",
+            Long.toHexString(sessionId),
+            session.getTimeout());
         close(sessionId);
     }
 
@@ -726,8 +734,9 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         if (zkShutdownHandler != null) {
             zkShutdownHandler.handle(state);
         } else {
-            LOG.debug("ZKShutdownHandler is not registered, so ZooKeeper server "
-                      + "won't take any action on ERROR or SHUTDOWN server state changes");
+            LOG.debug(
+                "ZKShutdownHandler is not registered, so ZooKeeper server"
+                    + " won't take any action on ERROR or SHUTDOWN server state changes");
         }
     }
 
@@ -943,8 +952,10 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         if (checkPasswd(sessionId, passwd)) {
             revalidateSession(cnxn, sessionId, sessionTimeout);
         } else {
-            LOG.warn("Incorrect password from " + cnxn.getRemoteSocketAddress()
-                     + " for session 0x" + Long.toHexString(sessionId));
+            LOG.warn(
+                "Incorrect password from {} for session 0x{}",
+                cnxn.getRemoteSocketAddress(),
+                Long.toHexString(sessionId));
             finishSessionInit(cnxn, false);
         }
     }
@@ -983,17 +994,18 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
             cnxn.sendBuffer(bb);
 
             if (valid) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Established session 0x" + Long.toHexString(cnxn.getSessionId())
-                              + " with negotiated timeout " + cnxn.getSessionTimeout()
-                              + " for client " + cnxn.getRemoteSocketAddress());
-                }
+                LOG.debug(
+                    "Established session 0x{} with negotiated timeout {} for client {}",
+                    Long.toHexString(cnxn.getSessionId()),
+                    cnxn.getSessionTimeout(),
+                    cnxn.getRemoteSocketAddress());
                 cnxn.enableRecv();
             } else {
 
-                LOG.info("Invalid session 0x" + Long.toHexString(cnxn.getSessionId())
-                         + " for client " + cnxn.getRemoteSocketAddress()
-                         + ", probably expired");
+                LOG.info(
+                    "Invalid session 0x{} for client {}, probably expired",
+                    Long.toHexString(cnxn.getSessionId()),
+                    cnxn.getRemoteSocketAddress());
                 cnxn.sendBuffer(ServerCnxnFactory.closeConn);
             }
 
@@ -1076,19 +1088,17 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                     incInProcess();
                 }
             } else {
-                LOG.warn("Received packet at server of unknown type " + si.type);
+                LOG.warn("Received packet at server of unknown type {}", si.type);
                 // Update request accounting/throttling limits
                 requestFinished(si);
                 new UnimplementedRequestProcessor().processRequest(si);
             }
         } catch (MissingSessionException e) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Dropping request: " + e.getMessage());
-            }
+            LOG.debug("Dropping request.", e);
             // Update request accounting/throttling limits
             requestFinished(si);
         } catch (RequestProcessorException e) {
-            LOG.error("Unable to process request:" + e.getMessage(), e);
+            LOG.error("Unable to process request", e);
             // Update request accounting/throttling limits
             requestFinished(si);
         }
@@ -1196,7 +1206,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     }
 
     public void setTickTime(int tickTime) {
-        LOG.info("tickTime set to " + tickTime);
+        LOG.info("tickTime set to {}", tickTime);
         this.tickTime = tickTime;
     }
 
@@ -1224,7 +1234,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
 
     public void setClientPortListenBacklog(int backlog) {
         this.listenBacklog = backlog;
-        LOG.info("clientPortListenBacklog set to " + backlog);
+        LOG.info("clientPortListenBacklog set to {}", backlog);
     }
 
     public int getClientPort() {
@@ -1284,10 +1294,11 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         BinaryInputArchive bia = BinaryInputArchive.getArchive(new ByteBufferInputStream(incomingBuffer));
         ConnectRequest connReq = new ConnectRequest();
         connReq.deserialize(bia, "connect");
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Session establishment request from client " + cnxn.getRemoteSocketAddress()
-                      + " client's lastZxid is 0x" + Long.toHexString(connReq.getLastZxidSeen()));
-        }
+        LOG.debug(
+            "Session establishment request from client {} client's lastZxid is 0x{}",
+            cnxn.getRemoteSocketAddress(),
+            Long.toHexString(connReq.getLastZxidSeen()));
+
         long sessionId = connReq.getSessionId();
         int tokensNeeded = 1;
         if (connThrottle.isConnectionWeightEnabled()) {
@@ -1316,9 +1327,9 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         } catch (IOException e) {
             // this is ok -- just a packet from an old client which
             // doesn't contain readOnly field
-            LOG.warn("Connection request from old client "
-                     + cnxn.getRemoteSocketAddress()
-                     + "; will be dropped if server is in r-o mode");
+            LOG.warn(
+                "Connection request from old client {}; will be dropped if server is in r-o mode",
+                cnxn.getRemoteSocketAddress());
         }
         if (!readOnly && this instanceof ReadOnlyZooKeeperServer) {
             String msg = "Refusing session request for not-read-only client " + cnxn.getRemoteSocketAddress();
@@ -1353,22 +1364,20 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         cnxn.disableRecv();
         if (sessionId == 0) {
             long id = createSession(cnxn, passwd, sessionTimeout);
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Client attempting to establish new session: session = 0x{}, zxid = 0x{}, timeout = {}, address = {}",
-                          Long.toHexString(id),
-                          Long.toHexString(connReq.getLastZxidSeen()),
-                          connReq.getTimeOut(),
-                          cnxn.getRemoteSocketAddress());
-            }
+            LOG.debug(
+                "Client attempting to establish new session: session = 0x{}, zxid = 0x{}, timeout = {}, address = {}",
+                Long.toHexString(id),
+                Long.toHexString(connReq.getLastZxidSeen()),
+                connReq.getTimeOut(),
+                cnxn.getRemoteSocketAddress());
         } else {
             long clientSessionId = connReq.getSessionId();
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Client attempting to renew session: session = 0x{}, zxid = 0x{}, timeout = {}, address = {}",
-                          Long.toHexString(clientSessionId),
-                          Long.toHexString(connReq.getLastZxidSeen()),
-                          connReq.getTimeOut(),
-                          cnxn.getRemoteSocketAddress());
-            }
+                LOG.debug(
+                    "Client attempting to renew session: session = 0x{}, zxid = 0x{}, timeout = {}, address = {}",
+                    Long.toHexString(clientSessionId),
+                    Long.toHexString(connReq.getLastZxidSeen()),
+                    connReq.getTimeOut(),
+                    cnxn.getRemoteSocketAddress());
             if (serverCnxnFactory != null) {
                 serverCnxnFactory.closeSession(sessionId, ServerCnxn.DisconnectReason.CLIENT_RECONNECT);
             }
@@ -1518,7 +1527,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         // to the start of the txn
         incomingBuffer = incomingBuffer.slice();
         if (h.getType() == OpCode.auth) {
-            LOG.info("got auth packet " + cnxn.getRemoteSocketAddress());
+            LOG.info("got auth packet {}", cnxn.getRemoteSocketAddress());
             AuthPacket authPacket = new AuthPacket();
             ByteBufferInputStream.byteBuffer2Record(incomingBuffer, authPacket);
             String scheme = authPacket.getScheme();
@@ -1532,21 +1541,23 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                         new ServerAuthenticationProvider.ServerObjs(this, cnxn),
                         authPacket.getAuth());
                 } catch (RuntimeException e) {
-                    LOG.warn("Caught runtime exception from AuthenticationProvider: " + scheme + " due to " + e);
+                    LOG.warn("Caught runtime exception from AuthenticationProvider: {}", scheme, e);
                     authReturn = KeeperException.Code.AUTHFAILED;
                 }
             }
             if (authReturn == KeeperException.Code.OK) {
                 LOG.debug("Authentication succeeded for scheme: {}", scheme);
-                LOG.info("auth success " + cnxn.getRemoteSocketAddress());
+                LOG.info("auth success {}", cnxn.getRemoteSocketAddress());
                 ReplyHeader rh = new ReplyHeader(h.getXid(), 0, KeeperException.Code.OK.intValue());
                 cnxn.sendResponse(rh, null, null);
             } else {
                 if (ap == null) {
-                    LOG.warn("No authentication provider for scheme: " + scheme
-                             + " has " + ProviderRegistry.listProviders());
+                    LOG.warn(
+                        "No authentication provider for scheme: {} has {}",
+                        scheme,
+                        ProviderRegistry.listProviders());
                 } else {
-                    LOG.warn("Authentication failed for scheme: " + scheme);
+                    LOG.warn("Authentication failed for scheme: {}", scheme);
                 }
                 // send a response...
                 ReplyHeader rh = new ReplyHeader(h.getXid(), 0, KeeperException.Code.AUTHFAILED.intValue());
@@ -1611,7 +1622,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                 responseToken = saslServer.evaluateResponse(clientToken);
                 if (saslServer.isComplete()) {
                     String authorizationID = saslServer.getAuthorizationID();
-                    LOG.info("adding SASL authorization for authorizationID: " + authorizationID);
+                    LOG.info("adding SASL authorization for authorizationID: {}", authorizationID);
                     cnxn.addAuthInfo(new Id("sasl", authorizationID));
                     if (System.getProperty("zookeeper.superUser") != null
                         && authorizationID.equals(System.getProperty("zookeeper.superUser"))) {
@@ -1625,9 +1636,10 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                 } else {
                     int error;
                     if (shouldRequireClientSaslAuth()) {
-                        LOG.warn("Closing client connection due to server requires client SASL authenticaiton,"
-                                 + "but client SASL authentication has failed, or client is not configured with SASL "
-                                 + "authentication.");
+                        LOG.warn(
+                            "Closing client connection due to server requires client SASL authenticaiton,"
+                                + "but client SASL authentication has failed, or client is not configured with SASL "
+                                + "authentication.");
                         error = Code.SESSIONCLOSEDREQUIRESASLAUTH.intValue();
                     } else {
                         LOG.warn("Closing client connection due to SASL authentication failure.");
@@ -1683,8 +1695,10 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                     ChangeRecord cr = outstandingChanges.remove();
                     ServerMetrics.getMetrics().OUTSTANDING_CHANGES_REMOVED.add(1);
                     if (cr.zxid < zxid) {
-                        LOG.warn("Zxid outstanding " + cr.zxid
-                                + " is less than current " + zxid);
+                        LOG.warn(
+                            "Zxid outstanding 0x{} is less than current 0x{}",
+                            Long.toHexString(cr.zxid),
+                            Long.toHexString(zxid));
                     }
                     if (outstandingChangesForPath.get(cr.path) == cr) {
                         outstandingChangesForPath.remove(cr.path);
@@ -1709,7 +1723,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                 CreateSessionTxn cst = (CreateSessionTxn) txn;
                 sessionTracker.commitSession(sessionId, cst.getTimeOut());
             } else if (request == null || !request.isLocalSession()) {
-                LOG.warn("*****>>>>> Got " + txn.getClass() + " " + txn.toString());
+                LOG.warn("*****>>>>> Got {} {}",  txn.getClass(), txn.toString());
             }
         } else if (opCode == OpCode.closeSession) {
             sessionTracker.removeSession(sessionId);

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooTrace.java

@@ -58,7 +58,7 @@ public class ZooTrace {
     public static synchronized void setTextTraceLevel(long mask) {
         traceMask = mask;
         final Logger LOG = LoggerFactory.getLogger(ZooTrace.class);
-        LOG.info("Set text trace mask to 0x" + Long.toHexString(mask));
+        LOG.info("Set text trace mask to 0x{}", Long.toHexString(mask));
     }
 
     public static synchronized boolean isTraceEnabled(Logger log, long mask) {

+ 3 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java

@@ -125,10 +125,10 @@ public class JettyAdminServer implements AdminServer {
                 try {
                     keyStore = X509Util.loadKeyStore(privateKeyPath, privateKeyPassword, privateKeyType);
                     trustStore = X509Util.loadTrustStore(certAuthPath, certAuthPassword, certAuthType);
-                    LOG.info("Successfully loaded private key from " + privateKeyPath);
-                    LOG.info("Successfully loaded certificate authority from " + certAuthPath);
+                    LOG.info("Successfully loaded private key from {}", privateKeyPath);
+                    LOG.info("Successfully loaded certificate authority from {}", certAuthPath);
                 } catch (Exception e) {
-                    LOG.error("Failed to load authentication certificates for admin server: " + e);
+                    LOG.error("Failed to load authentication certificates for admin server.", e);
                     throw e;
                 }
 

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/ProviderRegistry.java

@@ -56,7 +56,7 @@ public class ProviderRegistry {
                         AuthenticationProvider ap = (AuthenticationProvider) c.getDeclaredConstructor().newInstance();
                         authenticationProviders.put(ap.getScheme(), ap);
                     } catch (Exception e) {
-                        LOG.warn("Problems loading " + className, e);
+                        LOG.warn("Problems loading {}", className, e);
                     }
                 }
             }

+ 3 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/SaslServerCallbackHandler.java

@@ -89,7 +89,7 @@ public class SaslServerCallbackHandler implements CallbackHandler {
     private void handleNameCallback(NameCallback nc) {
         // check to see if this user is in the user password database.
         if (credentials.get(nc.getDefaultName()) == null) {
-            LOG.warn("User '" + nc.getDefaultName() + "' not found in list of DIGEST-MD5 authenticateable users.");
+            LOG.warn("User '{}' not found in list of DIGEST-MD5 authenticateable users.", nc.getDefaultName());
             return;
         }
         nc.setName(nc.getDefaultName());
@@ -103,7 +103,7 @@ public class SaslServerCallbackHandler implements CallbackHandler {
         } else if (credentials.containsKey(userName)) {
             pc.setPassword(credentials.get(userName).toCharArray());
         } else {
-            LOG.warn("No password found for user: " + userName);
+            LOG.warn("No password found for user: {}", userName);
         }
     }
 
@@ -132,7 +132,7 @@ public class SaslServerCallbackHandler implements CallbackHandler {
             if (shouldAppendRealm(kerberosName)) {
                 userNameBuilder.append("@").append(kerberosName.getRealm());
             }
-            LOG.info("Setting authorizedID: " + userNameBuilder);
+            LOG.info("Setting authorizedID: {}", userNameBuilder);
             ac.setAuthorizedID(userNameBuilder.toString());
         } catch (IOException e) {
             LOG.error("Failed to set name based on Kerberos authentication rules.", e);

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FilePadding.java

@@ -38,7 +38,7 @@ public class FilePadding {
             try {
                 preAllocSize = Long.parseLong(size) * 1024;
             } catch (NumberFormatException e) {
-                LOG.warn(size + " is not a valid value for preAllocSize");
+                LOG.warn("{} is not a valid value for preAllocSize", size);
             }
         }
     }

+ 3 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java

@@ -81,7 +81,7 @@ public class FileSnap implements SnapShot {
         boolean foundValid = false;
         for (int i = 0, snapListSize = snapList.size(); i < snapListSize; i++) {
             snap = snapList.get(i);
-            LOG.info("Reading snapshot " + snap);
+            LOG.info("Reading snapshot {}", snap);
             try (CheckedInputStream snapIS = SnapStream.getInputStream(snap)) {
                 InputArchive ia = BinaryInputArchive.getArchive(snapIS);
                 deserialize(dt, sessions, ia);
@@ -93,7 +93,7 @@ public class FileSnap implements SnapShot {
                 foundValid = true;
                 break;
             } catch (IOException e) {
-                LOG.warn("problem reading snap file " + snap, e);
+                LOG.warn("problem reading snap file {}", snap, e);
             }
         }
         if (!foundValid) {
@@ -167,7 +167,7 @@ public class FileSnap implements SnapShot {
                     }
                 }
             } catch (IOException e) {
-                LOG.info("invalid snapshot " + f, e);
+                LOG.warn("invalid snapshot {}", f, e);
             }
         }
         return list;

+ 13 - 13
zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java

@@ -264,14 +264,16 @@ public class FileTxnLog implements TxnLog {
             return false;
         }
         if (hdr.getZxid() <= lastZxidSeen) {
-            LOG.warn("Current zxid " + hdr.getZxid() + " is <= " + lastZxidSeen + " for " + hdr.getType());
+            LOG.warn(
+                "Current zxid {} is <= {} for {}",
+                hdr.getZxid(),
+                lastZxidSeen,
+                hdr.getType());
         } else {
             lastZxidSeen = hdr.getZxid();
         }
         if (logStream == null) {
-            if (LOG.isInfoEnabled()) {
-                LOG.info("Creating new log file: " + Util.makeLogName(hdr.getZxid()));
-            }
+            LOG.info("Creating new log file: {}", Util.makeLogName(hdr.getZxid()));
 
             logFileWrite = new File(logDir, Util.makeLogName(hdr.getZxid()));
             fos = new FileOutputStream(logFileWrite);
@@ -394,15 +396,13 @@ public class FileTxnLog implements TxnLog {
                     if (serverStats != null) {
                         serverStats.incrementFsyncThresholdExceedCount();
                     }
-                    LOG.warn("fsync-ing the write ahead log in "
-                             + Thread.currentThread().getName()
-                             + " took "
-                             + syncElapsedMS
-                             + "ms which will adversely effect operation latency. "
-                             + "File size is "
-                             + channel.size()
-                             + " bytes. "
-                             + "See the ZooKeeper troubleshooting guide");
+
+                    LOG.warn(
+                        "fsync-ing the write ahead log in {} took {}ms which will adversely effect operation latency."
+                            + "File size is {} bytes. See the ZooKeeper troubleshooting guide",
+                        Thread.currentThread().getName(),
+                        syncElapsedMS,
+                        channel.size());
                 }
 
                 ServerMetrics.getMetrics().FSYNC_TIME.add(syncElapsedMS);

+ 9 - 8
zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java

@@ -108,7 +108,7 @@ public class FileTxnSnapLog {
             System.getProperty(ZOOKEEPER_DATADIR_AUTOCREATE, ZOOKEEPER_DATADIR_AUTOCREATE_DEFAULT));
 
         trustEmptySnapshot = Boolean.getBoolean(ZOOKEEPER_SNAPSHOT_TRUST_EMPTY);
-        LOG.info(ZOOKEEPER_SNAPSHOT_TRUST_EMPTY + " : " + trustEmptySnapshot);
+        LOG.info("{} : {}", ZOOKEEPER_SNAPSHOT_TRUST_EMPTY, trustEmptySnapshot);
 
         if (!this.dataDir.exists()) {
             if (!enableAutocreate) {
@@ -250,7 +250,7 @@ public class FileTxnSnapLog {
                 if (!trustEmptySnapshot) {
                     throw new IOException(EMPTY_SNAPSHOT_WARNING + "Something is broken!");
                 } else {
-                    LOG.warn(EMPTY_SNAPSHOT_WARNING + "This should only be allowed during upgrading.");
+                    LOG.warn("{}This should only be allowed during upgrading.", EMPTY_SNAPSHOT_WARNING);
                 }
             }
 
@@ -276,10 +276,11 @@ public class FileTxnSnapLog {
         // zxid or that txn is missing
         DataTree.ZxidDigest snapshotZxidDigest = dt.getDigestFromLoadedSnapshot();
         if (snapshotZxidDigest != null) {
-            LOG.warn("Highest txn zxid 0x{} is not covering the snapshot "
-                     + "digest zxid 0x{}, which might lead to inconsistent state",
-                     Long.toHexString(highestZxid),
-                     Long.toHexString(snapshotZxidDigest.getZxid()));
+            LOG.warn(
+                "Highest txn zxid 0x{} is not covering the snapshot digest zxid 0x{}, "
+                    + "which might lead to inconsistent state",
+                Long.toHexString(highestZxid),
+                Long.toHexString(snapshotZxidDigest.getZxid()));
         }
         return highestZxid;
     }
@@ -458,9 +459,9 @@ public class FileTxnSnapLog {
                  * Doing so will eventually result in valid snapshots being
                  * removed during cleanup. */
                 if (snapshotFile.delete()) {
-                    LOG.info("Deleted empty snapshot file: " + snapshotFile.getAbsolutePath());
+                    LOG.info("Deleted empty snapshot file: {}", snapshotFile.getAbsolutePath());
                 } else {
-                    LOG.warn("Could not delete empty snapshot file: " + snapshotFile.getAbsolutePath());
+                    LOG.warn("Could not delete empty snapshot file: {}", snapshotFile.getAbsolutePath());
                 }
             } else {
                 /* Something else went wrong when writing the snapshot out to

+ 7 - 7
zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/SnapStream.java

@@ -57,7 +57,7 @@ public class SnapStream {
                            StreamMode.DEFAULT_MODE.getName()));
 
     static {
-        LOG.info(ZOOKEEPER_SHAPSHOT_STREAM_MODE + "=" + streamMode);
+        LOG.info("{} = {}", ZOOKEEPER_SHAPSHOT_STREAM_MODE, streamMode);
     }
 
     public enum StreamMode {
@@ -234,7 +234,7 @@ public class SnapStream {
         byte[] byteArray = new byte[2];
         try (FileInputStream fis = new FileInputStream(f)) {
             if (2 != fis.read(byteArray, 0, 2)) {
-                LOG.error("Read incorrect number of bytes from " + f.getName());
+                LOG.error("Read incorrect number of bytes from {}", f.getName());
                 return false;
             }
             ByteBuffer bb = ByteBuffer.wrap(byteArray);
@@ -243,7 +243,7 @@ public class SnapStream {
             int magic = magicHeader[0] & 0xff | ((magicHeader[1] << 8) & 0xff00);
             return magic == GZIPInputStream.GZIP_MAGIC;
         } catch (FileNotFoundException e) {
-            LOG.error("Unable to open file " + f.getName() + " : ", e);
+            LOG.error("Unable to open file {}", f.getName(), e);
             return false;
         }
     }
@@ -260,7 +260,7 @@ public class SnapStream {
         byte[] byteArray = new byte[SnappyCodec.MAGIC_LEN];
         try (FileInputStream fis = new FileInputStream(f)) {
             if (SnappyCodec.MAGIC_LEN != fis.read(byteArray, 0, SnappyCodec.MAGIC_LEN)) {
-                LOG.error("Read incorrect number of bytes from " + f.getName());
+                LOG.error("Read incorrect number of bytes from {}", f.getName());
                 return false;
             }
             ByteBuffer bb = ByteBuffer.wrap(byteArray);
@@ -268,7 +268,7 @@ public class SnapStream {
             bb.get(magicHeader, 0, SnappyCodec.MAGIC_LEN);
             return Arrays.equals(magicHeader, SnappyCodec.getMagicHeader());
         } catch (FileNotFoundException e) {
-            LOG.error("Unable to open file " + f.getName() + " : ", e);
+            LOG.error("Unable to open file {}", f.getName(), e);
             return false;
         }
     }
@@ -297,14 +297,14 @@ public class SnapStream {
                 readlen += l;
             }
             if (readlen != bytes.length) {
-                LOG.info("Invalid snapshot " + f.getName() + ". too short, len = " + readlen + " bytes");
+                LOG.info("Invalid snapshot {}. too short, len = {} bytes", f.getName(), readlen);
                 return false;
             }
             ByteBuffer bb = ByteBuffer.wrap(bytes);
             int len = bb.getInt();
             byte b = bb.get();
             if (len != 1 || b != '/') {
-                LOG.info("Invalid snapshot " + f.getName() + ". len = " + len + ", byte = " + (b & 0xff));
+                LOG.info("Invalid snapshot {}. len = {}, byte = {}", f.getName(), len, (b & 0xff));
                 return false;
             }
         }

+ 10 - 11
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java

@@ -243,14 +243,13 @@ public class AuthFastLeaderElection implements Election {
                     }
                     // Receive new message
                     if (responsePacket.getLength() != responseBytes.length) {
-                        LOG.warn("Got a short response: " + responsePacket.getLength()
-                                 + " " + responsePacket.toString());
+                        LOG.warn("Got a short response: {} {}", responsePacket.getLength(), responsePacket.toString());
                         continue;
                     }
                     responseBuffer.clear();
                     int type = responseBuffer.getInt();
                     if ((type > 3) || (type < 0)) {
-                        LOG.warn("Got bad Msg type: " + type);
+                        LOG.warn("Got bad Msg type: {}", type);
                         continue;
                     }
                     long tag = responseBuffer.getLong();
@@ -267,7 +266,7 @@ public class AuthFastLeaderElection implements Election {
                         ackstate = QuorumPeer.ServerState.FOLLOWING;
                         break;
                     default:
-                        LOG.warn("unknown type " + responseBuffer.getInt());
+                        LOG.warn("unknown type {}", responseBuffer.getInt());
                         break;
                     }
 
@@ -331,10 +330,10 @@ public class AuthFastLeaderElection implements Election {
 
                                         sendqueue.offer(a);
                                     } else {
-                                        LOG.warn("Incorrect challenge: " + recChallenge + ", " + addrChallengeMap.toString());
+                                        LOG.warn("Incorrect challenge: {}, {}", recChallenge, addrChallengeMap.toString());
                                     }
                                 } else {
-                                    LOG.warn("No challenge for host: " + addr + " " + tag);
+                                    LOG.warn("No challenge for host: {} {}", addr, tag);
                                 }
                             }
                         } else {
@@ -371,7 +370,7 @@ public class AuthFastLeaderElection implements Election {
                             if (tmpMap != null) {
                                 tmpMap.remove(tag);
                             } else {
-                                LOG.warn("No such address in the ensemble configuration " + responsePacket.getSocketAddress());
+                                LOG.warn("No such address in the ensemble configuration {}", responsePacket.getSocketAddress());
                             }
                         }
 
@@ -389,7 +388,7 @@ public class AuthFastLeaderElection implements Election {
                         break;
                     // Default case
                     default:
-                        LOG.warn("Received message of incorrect type " + type);
+                        LOG.warn("Received message of incorrect type {}", type);
                         break;
                     }
                 }
@@ -535,7 +534,7 @@ public class AuthFastLeaderElection implements Election {
                             LOG.warn("Exception while sending challenge: ", e);
                         }
                     } else {
-                        LOG.error("Address is not in the configuration: " + m.addr);
+                        LOG.error("Address is not in the configuration: {}", m.addr);
                     }
 
                     break;
@@ -617,7 +616,7 @@ public class AuthFastLeaderElection implements Election {
                                 if (tmpLong != null) {
                                     requestBuffer.putLong(tmpLong);
                                 } else {
-                                    LOG.warn("No challenge with tag: " + m.tag);
+                                    LOG.warn("No challenge with tag: {}", m.tag);
                                 }
                             }
                             mySocket.send(requestPacket);
@@ -688,7 +687,7 @@ public class AuthFastLeaderElection implements Election {
                     }
                     break;
                 default:
-                    LOG.warn("unknown type " + m.type);
+                    LOG.warn("unknown type {}", m.type);
                     break;
                 }
             }

+ 8 - 5
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java

@@ -434,9 +434,9 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements RequestP
 
         initBatchSizes();
 
-        LOG.info("Configuring CommitProcessor with "
-                 + (numWorkerThreads > 0 ? numWorkerThreads : "no")
-                 + " worker threads.");
+        LOG.info(
+            "Configuring CommitProcessor with {} worker threads.",
+            numWorkerThreads > 0 ? numWorkerThreads : "no");
         if (workerPool == null) {
             workerPool = new WorkerService("CommitProcWork", numWorkerThreads, true);
         }
@@ -471,7 +471,10 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements RequestP
             throw new IllegalArgumentException(errorMsg);
         }
 
-        LOG.info("Configuring CommitProcessor with readBatchSize {} commitBatchSize {}", maxReadBatchSize, maxCommitBatchSize);
+        LOG.info
+            ("Configuring CommitProcessor with readBatchSize {} commitBatchSize {}",
+             maxReadBatchSize,
+             maxCommitBatchSize);
     }
 
     private static void processCommitMetrics(Request request, boolean isWrite) {
@@ -529,7 +532,7 @@ public class CommitProcessor extends ZooKeeperCriticalThread implements RequestP
         @Override
         public void cleanup() {
             if (!stopped) {
-                LOG.error("Exception thrown by downstream processor," + " unable to continue.");
+                LOG.error("Exception thrown by downstream processor, unable to continue.");
                 CommitProcessor.this.halt();
             }
         }

+ 69 - 69
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java

@@ -239,7 +239,7 @@ public class FastLeaderElection implements Election {
 
                         // The current protocol and two previous generations all send at least 28 bytes
                         if (response.buffer.capacity() < 28) {
-                            LOG.error("Got a short response: " + response.buffer.capacity());
+                            LOG.error("Got a short response: {}", response.buffer.capacity());
                             continue;
                         }
 
@@ -373,9 +373,18 @@ public class FastLeaderElection implements Election {
                             /*
                              * Print notification info
                              */
-                            if (LOG.isInfoEnabled()) {
-                                printNotification(n);
-                            }
+                            LOG.info(
+                                "Notification: my state:{}; n.sid:{}, n.state:{}, n.leader:{}, n.round:0x{}, "
+                                    + "n.peerEpoch:0x{}, n.zxid:0x{}, message format version:0x{}, n.config version:0x{}",
+                                self.getPeerState(),
+                                n.sid,
+                                n.state,
+                                n.leader,
+                                Long.toHexString(n.electionEpoch),
+                                Long.toHexString(n.peerEpoch),
+                                Long.toHexString(n.zxid),
+                                Long.toHexString(n.version),
+                                (n.qv != null ? (Long.toHexString(n.qv.getVersion())) : "0"));
 
                             /*
                              * If this server is looking, then send proposed leader
@@ -418,14 +427,15 @@ public class FastLeaderElection implements Election {
                                         }
                                         self.leader.reportLookingSid(response.sid);
                                     }
-                                    if (LOG.isDebugEnabled()) {
-                                        LOG.debug("Sending new notification. My id ={} recipient={} zxid=0x{} leader={} config version = {}",
-                                                  self.getId(),
-                                                  response.sid,
-                                                  Long.toHexString(current.getZxid()),
-                                                  current.getId(),
-                                                  Long.toHexString(self.getQuorumVerifier().getVersion()));
-                                    }
+
+
+                                    LOG.debug(
+                                        "Sending new notification. My id ={} recipient={} zxid=0x{} leader={} config version = {}",
+                                        self.getId(),
+                                        response.sid,
+                                        Long.toHexString(current.getZxid()),
+                                        current.getId(),
+                                        Long.toHexString(self.getQuorumVerifier().getVersion()));
 
                                     QuorumVerifier qv = self.getQuorumVerifier();
                                     ToSend notmsg = new ToSend(
@@ -442,7 +452,7 @@ public class FastLeaderElection implements Election {
                             }
                         }
                     } catch (InterruptedException e) {
-                        LOG.warn("Interrupted Exception while waiting for new message" + e.toString());
+                        LOG.warn("Interrupted Exception while waiting for new message", e);
                     }
                 }
                 LOG.info("WorkerReceiver is down");
@@ -634,12 +644,12 @@ public class FastLeaderElection implements Election {
     }
 
     private void leaveInstance(Vote v) {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("About to leave FLE instance: leader={}, zxid=0x{}, my id={}, my state={}",
-                      v.getId(),
-                      Long.toHexString(v.getZxid()),
-                      self.getId(), self.getPeerState());
-        }
+        LOG.debug(
+            "About to leave FLE instance: leader={}, zxid=0x{}, my id={}, my state={}",
+            v.getId(),
+            Long.toHexString(v.getZxid()),
+            self.getId(),
+            self.getPeerState());
         recvqueue.clear();
     }
 
@@ -675,45 +685,33 @@ public class FastLeaderElection implements Election {
                 sid,
                 proposedEpoch,
                 qv.toString().getBytes());
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Sending Notification: " + proposedLeader
-                          + " (n.leader), 0x" + Long.toHexString(proposedZxid)
-                          + " (n.zxid), 0x" + Long.toHexString(logicalclock.get())
-                          + " (n.round), " + sid
-                          + " (recipient), " + self.getId()
-                          + " (myid), 0x" + Long.toHexString(proposedEpoch)
-                          + " (n.peerEpoch)");
-            }
+
+            LOG.debug(
+                "Sending Notification: {} (n.leader), 0x{} (n.zxid), 0x{} (n.round), {} (recipient),"
+                    + " {} (myid), 0x{} (n.peerEpoch) ",
+                proposedLeader,
+                Long.toHexString(proposedZxid),
+                Long.toHexString(logicalclock.get()),
+                sid,
+                self.getId(),
+                Long.toHexString(proposedEpoch));
+
             sendqueue.offer(notmsg);
         }
     }
 
-    private void printNotification(Notification n) {
-        LOG.info("Notification: my state:{}; n.sid:{}, n.state:{}, n.leader:{}, n.round:0x{}, "
-                 + "n.peerEpoch:0x{}, n.zxid:0x{}, message format version:0x{}, n.config version:0x{}",
-                 self.getPeerState(),
-                 n.sid,
-                 n.state,
-                 n.leader,
-                 Long.toHexString(n.electionEpoch),
-                 Long.toHexString(n.peerEpoch),
-                 Long.toHexString(n.zxid),
-                 Long.toHexString(n.version),
-                 (n.qv != null ? (Long.toHexString(n.qv.getVersion())) : "0"));
-    }
-
     /**
      * Check if a pair (server id, zxid) succeeds our
      * current vote.
      *
      */
     protected boolean totalOrderPredicate(long newId, long newZxid, long newEpoch, long curId, long curZxid, long curEpoch) {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("id: " + newId
-                      + ", proposed id: " + curId
-                      + ", zxid: 0x" + Long.toHexString(newZxid)
-                      + ", proposed zxid: 0x" + Long.toHexString(curZxid));
-        }
+        LOG.debug(
+            "id: {}, proposed id: {}, zxid: 0x{}, proposed zxid: 0x{}",
+            newId,
+            curId,
+            Long.toHexString(newZxid),
+            Long.toHexString(curZxid));
 
         if (self.getQuorumVerifier().getWeight(newId) == 0) {
             return false;
@@ -801,13 +799,13 @@ public class FastLeaderElection implements Election {
     }
 
     synchronized void updateProposal(long leader, long zxid, long epoch) {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Updating proposal: " + leader
-                      + " (newleader), 0x" + Long.toHexString(zxid)
-                      + " (newzxid), " + proposedLeader
-                      + " (oldleader), 0x" + Long.toHexString(proposedZxid)
-                      + " (oldzxid)");
-        }
+        LOG.debug(
+            "Updating proposal: {} (newleader), 0x{} (newzxid), {} (oldleader), 0x{} (oldzxid)",
+            leader,
+            Long.toHexString(zxid),
+            proposedLeader,
+            Long.toHexString(proposedZxid));
+
         proposedLeader = leader;
         proposedZxid = zxid;
         proposedEpoch = epoch;
@@ -930,7 +928,10 @@ public class FastLeaderElection implements Election {
                 updateProposal(getInitId(), getInitLastLoggedZxid(), getPeerEpoch());
             }
 
-            LOG.info("New election. My id =  " + self.getId() + ", proposed zxid=0x" + Long.toHexString(proposedZxid));
+            LOG.info(
+                "New election. My id = {}, proposed zxid=0x{}",
+                self.getId(),
+                Long.toHexString(proposedZxid));
             sendNotifications();
 
             SyncedLearnerTracker voteSet;
@@ -961,8 +962,8 @@ public class FastLeaderElection implements Election {
                      * Exponential backoff
                      */
                     int tmpTimeOut = notTimeout * 2;
-                    notTimeout = (tmpTimeOut < maxNotificationInterval ? tmpTimeOut : maxNotificationInterval);
-                    LOG.info("Notification time out: " + notTimeout);
+                    notTimeout = Math.min(tmpTimeOut, maxNotificationInterval);
+                    LOG.info("Notification time out: {}", notTimeout);
                 } else if (validVoter(n.sid) && validVoter(n.leader)) {
                     /*
                      * Only proceed if the vote comes from a replica in the current or next
@@ -989,23 +990,22 @@ public class FastLeaderElection implements Election {
                             }
                             sendNotifications();
                         } else if (n.electionEpoch < logicalclock.get()) {
-                            if (LOG.isDebugEnabled()) {
                                 LOG.debug(
-                                    "Notification election epoch is smaller than logicalclock. n.electionEpoch = 0x" + Long.toHexString(n.electionEpoch)
-                                    + ", logicalclock=0x" + Long.toHexString(logicalclock.get()));
-                            }
+                                    "Notification election epoch is smaller than logicalclock. n.electionEpoch = 0x{}, logicalclock=0x{}",
+                                    Long.toHexString(n.electionEpoch),
+                                    Long.toHexString(logicalclock.get()));
                             break;
                         } else if (totalOrderPredicate(n.leader, n.zxid, n.peerEpoch, proposedLeader, proposedZxid, proposedEpoch)) {
                             updateProposal(n.leader, n.zxid, n.peerEpoch);
                             sendNotifications();
                         }
 
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Adding vote: from=" + n.sid
-                                      + ", proposed leader=" + n.leader
-                                      + ", proposed zxid=0x" + Long.toHexString(n.zxid)
-                                      + ", proposed election epoch=0x" + Long.toHexString(n.electionEpoch));
-                        }
+                        LOG.debug(
+                            "Adding vote: from={}, proposed leader={}, proposed zxid=0x{}, proposed election epoch=0x{}",
+                            n.sid,
+                            n.leader,
+                            Long.toHexString(n.zxid),
+                            Long.toHexString(n.electionEpoch));
 
                         // don't care about the version if it's in LOOKING state
                         recvset.put(n.sid, new Vote(n.leader, n.zxid, n.electionEpoch, n.peerEpoch));
@@ -1075,7 +1075,7 @@ public class FastLeaderElection implements Election {
                         }
                         break;
                     default:
-                        LOG.warn("Notification state unrecoginized: " + n.state + " (n.state), " + n.sid + " (n.sid)");
+                        LOG.warn("Notification state unrecoginized: {} (n.state), {}(n.sid)", n.state, n.sid);
                         break;
                     }
                 } else {

+ 10 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java

@@ -138,9 +138,11 @@ public class Follower extends Learner {
 
             if (connectionTime != 0) {
                 long connectionDuration = System.currentTimeMillis() - connectionTime;
-                LOG.info("Disconnected from leader (with address: {}). "
-                        + "Was connected for {}ms. Sync state: {}",
-                    leaderAddr, connectionDuration, completedSync);
+                LOG.info(
+                    "Disconnected from leader (with address: {}). Was connected for {}ms. Sync state: {}",
+                    leaderAddr,
+                    connectionDuration,
+                    completedSync);
                 messageTracker.dumpToLog(leaderAddr.toString());
             }
         }
@@ -161,8 +163,10 @@ public class Follower extends Learner {
             TxnHeader hdr = new TxnHeader();
             Record txn = SerializeUtils.deserializeTxn(qp.getData(), hdr);
             if (hdr.getZxid() != lastQueued + 1) {
-                LOG.warn("Got zxid 0x" + Long.toHexString(hdr.getZxid())
-                         + " expected 0x" + Long.toHexString(lastQueued + 1));
+                LOG.warn(
+                    "Got zxid 0x{} expected 0x{}",
+                    Long.toHexString(hdr.getZxid()),
+                    Long.toHexString(lastQueued + 1));
             }
             lastQueued = hdr.getZxid();
 
@@ -281,7 +285,7 @@ public class Follower extends Learner {
 
     @Override
     public void shutdown() {
-        LOG.info("shutdown called", new Exception("shutdown Follower"));
+        LOG.info("shutdown Follower");
         super.shutdown();
     }
 

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java

@@ -131,7 +131,7 @@ public class FollowerRequestProcessor extends ZooKeeperCriticalThread implements
                         request.setTxn(new ErrorTxn(ke.code().intValue()));
                     }
                     request.setException(ke);
-                    LOG.info("Error creating upgrade request", ke);
+                    LOG.warn("Error creating upgrade request", ke);
                 } catch (IOException ie) {
                     LOG.error("Unexpected error in upgrade", ie);
                 }

+ 29 - 24
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java

@@ -72,7 +72,7 @@ public class Leader extends LearnerMaster {
     private static final boolean nodelay = System.getProperty("leader.nodelay", "true").equals("true");
 
     static {
-        LOG.info("TCP NoDelay set to: " + nodelay);
+        LOG.info("TCP NoDelay set to: {}", nodelay);
     }
 
     public static class Proposal extends SyncedLearnerTracker {
@@ -93,7 +93,7 @@ public class Leader extends LearnerMaster {
 
     static {
         ackLoggingFrequency = Integer.getInteger(ACK_LOGGING_FREQUENCY, 1000);
-        LOG.info(ACK_LOGGING_FREQUENCY + " = " + ackLoggingFrequency);
+        LOG.info("{} = {}", ACK_LOGGING_FREQUENCY, ackLoggingFrequency);
     }
 
     public static void setAckLoggingFrequency(int frequency) {
@@ -297,9 +297,9 @@ public class Leader extends LearnerMaster {
             }
         } catch (BindException e) {
             if (self.getQuorumListenOnAllIPs()) {
-                LOG.error("Couldn't bind to port " + self.getQuorumAddress().getPort(), e);
+                LOG.error("Couldn't bind to port {}", self.getQuorumAddress().getPort(), e);
             } else {
-                LOG.error("Couldn't bind to " + self.getQuorumAddress(), e);
+                LOG.error("Couldn't bind to {}", self.getQuorumAddress(), e);
             }
             throw e;
         }
@@ -444,7 +444,7 @@ public class Leader extends LearnerMaster {
                     } catch (SocketException e) {
                         error = true;
                         if (stop) {
-                            LOG.info("exception while shutting down acceptor: " + e);
+                            LOG.warn("exception while shutting down acceptor.", e);
 
                             // When Leader.shutdown() calls ss.close(),
                             // the call to accept throws an exception.
@@ -471,7 +471,7 @@ public class Leader extends LearnerMaster {
                     }
                 }
             } catch (Exception e) {
-                LOG.warn("Exception while accepting follower", e.getMessage());
+                LOG.warn("Exception while accepting follower", e);
                 handleException(this.getName(), e);
             }
         }
@@ -543,7 +543,7 @@ public class Leader extends LearnerMaster {
             newLeaderProposal.packet = new QuorumPacket(NEWLEADER, zk.getZxid(), null, null);
 
             if ((newLeaderProposal.packet.getZxid() & 0xffffffffL) != 0) {
-                LOG.info("NEWLEADER proposal has Zxid of " + Long.toHexString(newLeaderProposal.packet.getZxid()));
+                LOG.info("NEWLEADER proposal has Zxid of {}", Long.toHexString(newLeaderProposal.packet.getZxid()));
             }
 
             QuorumVerifier lastSeenQV = self.getLastSeenQuorumVerifier();
@@ -612,7 +612,7 @@ public class Leader extends LearnerMaster {
                     }
                 }
                 if (initTicksShouldBeIncreased) {
-                    LOG.warn("Enough followers present. " + "Perhaps the initTicks need to be increased.");
+                    LOG.warn("Enough followers present. Perhaps the initTicks need to be increased.");
                 }
                 return;
             }
@@ -731,7 +731,7 @@ public class Leader extends LearnerMaster {
             return;
         }
 
-        LOG.info("Shutdown called", new Exception("shutdown Leader! reason: " + reason));
+        LOG.info("Shutdown called. For the reason {}", reason);
 
         if (cnxAcceptor != null) {
             cnxAcceptor.halt();
@@ -838,8 +838,11 @@ public class Leader extends LearnerMaster {
 
         // commit proposals in order
         if (zxid != lastCommitted + 1) {
-            LOG.warn("Commiting zxid 0x" + Long.toHexString(zxid) + " from " + followerAddr + " not first!");
-            LOG.warn("First is " + (lastCommitted + 1));
+            LOG.warn(
+                "Commiting zxid 0x{} from {} noy first!",
+                Long.toHexString(zxid),
+                followerAddr);
+            LOG.warn("First is {}", (lastCommitted + 1));
         }
 
         outstandingProposals.remove(zxid);
@@ -849,7 +852,7 @@ public class Leader extends LearnerMaster {
         }
 
         if (p.request == null) {
-            LOG.warn("Going to commmit null: " + p);
+            LOG.warn("Going to commit null: {}", p);
         } else if (p.request.getHdr().getType() == OpCode.reconfig) {
             LOG.debug("Committing a reconfiguration! {}", outstandingProposals.size());
 
@@ -926,11 +929,10 @@ public class Leader extends LearnerMaster {
             return;
         }
         if (lastCommitted >= zxid) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("proposal has already been committed, pzxid: 0x{} zxid: 0x{}",
-                          Long.toHexString(lastCommitted),
-                          Long.toHexString(zxid));
-            }
+            LOG.debug(
+                "proposal has already been committed, pzxid: 0x{} zxid: 0x{}",
+                Long.toHexString(lastCommitted),
+                Long.toHexString(zxid));
             // The proposal has already been committed
             return;
         }
@@ -1018,7 +1020,7 @@ public class Leader extends LearnerMaster {
                         return;
                     }
                 }
-                LOG.error("Committed request not found on toBeApplied: " + request);
+                LOG.error("Committed request not found on toBeApplied: {}", request);
             }
         }
 
@@ -1475,9 +1477,11 @@ public class Leader extends LearnerMaster {
 
             long currentZxid = newLeaderProposal.packet.getZxid();
             if (zxid != currentZxid) {
-                LOG.error("NEWLEADER ACK from sid: " + sid
-                          + " is from a different epoch - current 0x" + Long.toHexString(currentZxid)
-                          + " receieved 0x" + Long.toHexString(zxid));
+                LOG.error(
+                    "NEWLEADER ACK from sid: {} is from a different epoch - current 0x{} received 0x{}",
+                    sid,
+                    Long.toHexString(currentZxid),
+                    Long.toHexString(zxid));
                 return;
             }
 
@@ -1634,9 +1638,10 @@ public class Leader extends LearnerMaster {
                 // set the session owner as the follower that owns the session
                 zk.setOwner(id, learnerHandler);
             } catch (KeeperException.SessionExpiredException e) {
-                LOG.error("Somehow session "
-                          + Long.toHexString(id)
-                          + " expired right after being renewed! (impossible)", e);
+                LOG.error(
+                    "Somehow session 0x{} expired right after being renewed! (impossible)",
+                    Long.toHexString(id),
+                    e);
             }
         }
         if (LOG.isTraceEnabled()) {

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderRequestProcessor.java

@@ -63,7 +63,7 @@ public class LeaderRequestProcessor implements RequestProcessor {
                 request.setTxn(new ErrorTxn(ke.code().intValue()));
             }
             request.setException(ke);
-            LOG.info("Error creating upgrade request " + ke.getMessage());
+            LOG.warn("Error creating upgrade request", ke);
         } catch (IOException ie) {
             LOG.error("Unexpected error in upgrade", ie);
         }

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderSessionTracker.java

@@ -86,7 +86,7 @@ public class LeaderSessionTracker extends UpgradeableSessionTracker {
         if (localSessionsEnabled && tracked) {
             // Only do extra logging so we know what kind of session this is
             // if we're supporting both kinds of sessions
-            LOG.info("Tracking global session 0x" + Long.toHexString(sessionId));
+            LOG.info("Tracking global session 0x{}", Long.toHexString(sessionId));
         }
         return tracked;
     }
@@ -101,7 +101,7 @@ public class LeaderSessionTracker extends UpgradeableSessionTracker {
         boolean added = globalSessionTracker.commitSession(sessionId, sessionTimeout);
 
         if (added) {
-            LOG.info("Committing global session 0x" + Long.toHexString(sessionId));
+            LOG.info("Committing global session 0x{}", Long.toHexString(sessionId));
         }
 
         // If the session moved before the session upgrade finished, it's

+ 41 - 23
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java

@@ -124,7 +124,7 @@ public class Learner {
      * @throws IOException
      */
     void validateSession(ServerCnxn cnxn, long clientId, int timeout) throws IOException {
-        LOG.info("Revalidating client: 0x" + Long.toHexString(clientId));
+        LOG.info("Revalidating client: 0x{}", Long.toHexString(clientId));
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream(baos);
         dos.writeLong(clientId);
@@ -224,7 +224,7 @@ public class Learner {
             }
         }
         if (leaderServer == null) {
-            LOG.warn("Couldn't find the leader with id = " + current.getId());
+            LOG.warn("Couldn't find the leader with id = {}", current.getId());
         }
         return leaderServer;
     }
@@ -290,19 +290,28 @@ public class Learner {
                 remainingTimeout = connectTimeout - (int) ((nanoTime() - startNanoTime) / 1000000);
 
                 if (remainingTimeout <= 1000) {
-                    LOG.error("Unexpected exception, connectToLeader exceeded. tries=" + tries
-                              + ", remaining init limit=" + remainingTimeout
-                              + ", connecting to " + addr, e);
+                    LOG.error(
+                        "Unexpected exception, connectToLeader exceeded. tries={}, remaining init limit={}, connecting to {}",
+                        tries,
+                        remainingTimeout,
+                        addr,
+                        e);
                     throw e;
                 } else if (tries >= 4) {
-                    LOG.error("Unexpected exception, retries exceeded. tries=" + tries
-                              + ", remaining init limit=" + remainingTimeout
-                              + ", connecting to " + addr, e);
+                    LOG.error(
+                        "Unexpected exception, retries exceeded. tries={}, remaining init limit={}, connecting to {}",
+                        tries,
+                        remainingTimeout,
+                        addr,
+                        e);
                     throw e;
                 } else {
-                    LOG.warn("Unexpected exception, tries=" + tries
-                             + ", remaining init limit=" + remainingTimeout
-                             + ", connecting to " + addr, e);
+                    LOG.warn(
+                        "Unexpected exception, tries={}, remaining init limit={}, connecting to {}",
+                        tries,
+                        remainingTimeout,
+                        addr,
+                        e);
                     this.sock = createSocket();
                 }
             }
@@ -418,7 +427,7 @@ public class Learner {
                 snapshotNeeded = false;
             } else if (qp.getType() == Leader.SNAP) {
                 self.setSyncMode(QuorumPeer.SyncMode.SNAP);
-                LOG.info("Getting a snapshot from leader 0x" + Long.toHexString(qp.getZxid()));
+                LOG.info("Getting a snapshot from leader 0x{}", Long.toHexString(qp.getZxid()));
                 // The leader is going to dump the database
                 // db is clear as part of deserializeSnapshot()
                 zk.getZKDatabase().deserializeSnapshot(leaderIs);
@@ -431,7 +440,7 @@ public class Learner {
                 }
                 String signature = leaderIs.readString("signature");
                 if (!signature.equals("BenWasHere")) {
-                    LOG.error("Missing signature. Got " + signature);
+                    LOG.error("Missing signature. Got {}", signature);
                     throw new IOException("Missing signature");
                 }
                 zk.getZKDatabase().setlastProcessedZxid(qp.getZxid());
@@ -441,11 +450,11 @@ public class Learner {
             } else if (qp.getType() == Leader.TRUNC) {
                 //we need to truncate the log to the lastzxid of the leader
                 self.setSyncMode(QuorumPeer.SyncMode.TRUNC);
-                LOG.warn("Truncating log to get in sync with the leader 0x" + Long.toHexString(qp.getZxid()));
+                LOG.warn("Truncating log to get in sync with the leader 0x{}", Long.toHexString(qp.getZxid()));
                 boolean truncated = zk.getZKDatabase().truncateLog(qp.getZxid());
                 if (!truncated) {
                     // not able to truncate the log
-                    LOG.error("Not able to truncate the log " + Long.toHexString(qp.getZxid()));
+                    LOG.error("Not able to truncate the log 0x{}", Long.toHexString(qp.getZxid()));
                     System.exit(ExitCode.QUORUM_PACKET_ERROR.getValue());
                 }
                 zk.getZKDatabase().setlastProcessedZxid(qp.getZxid());
@@ -477,8 +486,10 @@ public class Learner {
                     pif.hdr = new TxnHeader();
                     pif.rec = SerializeUtils.deserializeTxn(qp.getData(), pif.hdr);
                     if (pif.hdr.getZxid() != lastQueued + 1) {
-                        LOG.warn("Got zxid 0x" + Long.toHexString(pif.hdr.getZxid())
-                                 + " expected 0x" + Long.toHexString(lastQueued + 1));
+                        LOG.warn(
+                            "Got zxid 0x{} expected 0x{}",
+                            Long.toHexString(pif.hdr.getZxid()),
+                            Long.toHexString(lastQueued + 1));
                     }
                     lastQueued = pif.hdr.getZxid();
 
@@ -505,7 +516,10 @@ public class Learner {
                     }
                     if (!writeToTxnLog) {
                         if (pif.hdr.getZxid() != qp.getZxid()) {
-                            LOG.warn("Committing " + qp.getZxid() + ", but next proposal is " + pif.hdr.getZxid());
+                            LOG.warn(
+                                "Committing 0x{}, but next proposal is 0x{}",
+                                Long.toHexString(qp.getZxid()),
+                                Long.toHexString(pif.hdr.getZxid()));
                         } else {
                             zk.processTxn(pif.hdr, pif.rec);
                             packetsNotCommitted.remove();
@@ -534,8 +548,10 @@ public class Learner {
                         packet.rec = SerializeUtils.deserializeTxn(qp.getData(), packet.hdr);
                         // Log warning message if txn comes out-of-order
                         if (packet.hdr.getZxid() != lastQueued + 1) {
-                            LOG.warn("Got zxid 0x" + Long.toHexString(packet.hdr.getZxid())
-                                     + " expected 0x" + Long.toHexString(lastQueued + 1));
+                            LOG.warn(
+                                "Got zxid 0x{} expected 0x{}",
+                                Long.toHexString(packet.hdr.getZxid()),
+                                Long.toHexString(lastQueued + 1));
                         }
                         lastQueued = packet.hdr.getZxid();
                     }
@@ -620,8 +636,10 @@ public class Learner {
                 if (p.hdr.getZxid() != zxid) {
                     // log warning message if there is no matching commit
                     // old leader send outstanding proposal to observer
-                    LOG.warn("Committing " + Long.toHexString(zxid)
-                             + ", but next proposal is " + Long.toHexString(p.hdr.getZxid()));
+                    LOG.warn(
+                        "Committing 0x{}, but next proposal is 0x{}",
+                        Long.toHexString(zxid),
+                        Long.toHexString(p.hdr.getZxid()));
                     continue;
                 }
                 packetsCommitted.remove();
@@ -643,7 +661,7 @@ public class Learner {
         boolean valid = dis.readBoolean();
         ServerCnxn cnxn = pendingRevalidations.remove(sessionId);
         if (cnxn == null) {
-            LOG.warn("Missing session 0x" + Long.toHexString(sessionId) + " for validation");
+            LOG.warn("Missing session 0x{} for validation", Long.toHexString(sessionId));
         } else {
             zk.finishSessionInit(cnxn, valid);
         }

+ 72 - 65
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java

@@ -169,7 +169,10 @@ public class LearnerHandler extends ZooKeeperThread {
                 nextTime = 0;
                 nextZxid = 0;
             } else if (nextZxid == zxid) {
-                LOG.warn("ACK for " + zxid + " received before ACK for " + currentZxid + "!!!!");
+                LOG.warn(
+                    "ACK for 0x{} received before ACK for 0x{}",
+                    Long.toHexString(zxid),
+                    Long.toHexString(currentZxid));
                 nextTime = 0;
                 nextZxid = 0;
             }
@@ -357,14 +360,14 @@ public class LearnerHandler extends ZooKeeperThread {
                 messageTracker.trackSent(p.getType());
             } catch (IOException e) {
                 if (!sock.isClosed()) {
-                    LOG.warn("Unexpected exception at " + this, e);
+                    LOG.warn("Unexpected exception at {}", this, e);
                     try {
                         // this will cause everything to shutdown on
                         // this learner handler and will help notify
                         // the learner/observer instantaneously
                         sock.close();
                     } catch (IOException ie) {
-                        LOG.warn("Error closing socket for handler " + this, ie);
+                        LOG.warn("Error closing socket for handler {}", this, ie);
                     }
                 }
                 break;
@@ -473,7 +476,7 @@ public class LearnerHandler extends ZooKeeperThread {
 
             messageTracker.trackReceived(qp.getType());
             if (qp.getType() != Leader.FOLLOWERINFO && qp.getType() != Leader.OBSERVERINFO) {
-                LOG.error("First packet " + qp.toString() + " is not FOLLOWERINFO or OBSERVERINFO!");
+                LOG.error("First packet {} is not FOLLOWERINFO or OBSERVERINFO!", qp.toString());
 
                 return;
             }
@@ -502,12 +505,12 @@ public class LearnerHandler extends ZooKeeperThread {
 
             String followerInfo = learnerMaster.getPeerInfo(this.sid);
             if (followerInfo.isEmpty()) {
-                LOG.info("Follower sid: "
-                         + this.sid
-                         + " not in the current config "
-                         + Long.toHexString(learnerMaster.getQuorumVerifierVersion()));
+                LOG.info(
+                    "Follower sid: {} not in the current config {}",
+                    this.sid,
+                    Long.toHexString(learnerMaster.getQuorumVerifierVersion()));
             } else {
-                LOG.info("Follower sid: " + this.sid + " : info : " + followerInfo);
+                LOG.info("Follower sid: {} : info : {}", this.sid, followerInfo);
             }
 
             if (qp.getType() == Leader.OBSERVERINFO) {
@@ -541,7 +544,7 @@ public class LearnerHandler extends ZooKeeperThread {
                 ia.readRecord(ackEpochPacket, "packet");
                 messageTracker.trackReceived(ackEpochPacket.getType());
                 if (ackEpochPacket.getType() != Leader.ACKEPOCH) {
-                    LOG.error(ackEpochPacket.toString() + " is not ACKEPOCH");
+                    LOG.error("{} is not ACKEPOCH", ackEpochPacket.toString());
                     return;
                 }
                 ByteBuffer bbepoch = ByteBuffer.wrap(ackEpochPacket.getData());
@@ -568,14 +571,15 @@ public class LearnerHandler extends ZooKeeperThread {
                     messageTracker.trackSent(Leader.SNAP);
                     bufferedOutput.flush();
 
-                    LOG.info("Sending snapshot last zxid of peer is 0x{}, zxid of leader is 0x{}, "
-                             + "send zxid of db as 0x{}, {} concurrent snapshot sync, "
-                             + "snapshot sync was {} from throttle",
-                             Long.toHexString(peerLastZxid),
-                             Long.toHexString(leaderLastZxid),
-                             Long.toHexString(zxidToSend),
-                             syncThrottler.getSyncInProgress(),
-                             exemptFromThrottle ? "exempt" : "not exempt");
+                    LOG.info(
+                        "Sending snapshot last zxid of peer is 0x{}, zxid of leader is 0x{}, "
+                            + "send zxid of db as 0x{}, {} concurrent snapshot sync, "
+                            + "snapshot sync was {} from throttle",
+                        Long.toHexString(peerLastZxid),
+                        Long.toHexString(leaderLastZxid),
+                        Long.toHexString(zxidToSend),
+                        syncThrottler.getSyncInProgress(),
+                        exemptFromThrottle ? "exempt" : "not exempt");
                     // Dump data to peer
                     learnerMaster.getZKDatabase().serializeSnapshot(oa);
                     oa.writeString("BenWasHere", "signature");
@@ -615,7 +619,7 @@ public class LearnerHandler extends ZooKeeperThread {
 
             messageTracker.trackReceived(qp.getType());
             if (qp.getType() != Leader.ACK) {
-                LOG.error("Next packet was supposed to be an ACK," + " but received packet: {}", packetToString(qp));
+                LOG.error("Next packet was supposed to be an ACK, but received packet: {}", packetToString(qp));
                 return;
             }
 
@@ -709,7 +713,7 @@ public class LearnerHandler extends ZooKeeperThread {
             }
         } catch (IOException e) {
             if (sock != null && !sock.isClosed()) {
-                LOG.error("Unexpected exception causing shutdown while sock " + "still open", e);
+                LOG.error("Unexpected exception causing shutdown while sock still open", e);
                 //close the socket to make sure the
                 //other side can see it being close
                 try {
@@ -719,12 +723,12 @@ public class LearnerHandler extends ZooKeeperThread {
                 }
             }
         } catch (InterruptedException e) {
-            LOG.error("Unexpected exception in LearnerHandler: ", e);
+            LOG.error("Unexpected exception in LearnerHandler.", e);
         } catch (SyncThrottleException e) {
-            LOG.error("too many concurrent syncs: " + e);
+            LOG.error("too many concurrent sync.", e);
             syncThrottler = null;
         } catch (Exception e) {
-            LOG.error("Unexpected exception in LearnerHandler: ", e);
+            LOG.error("Unexpected exception in LearnerHandler.", e);
             throw e;
         } finally {
             if (syncThrottler != null) {
@@ -750,7 +754,7 @@ public class LearnerHandler extends ZooKeeperThread {
                     try {
                         sendPackets();
                     } catch (InterruptedException e) {
-                        LOG.warn("Unexpected interruption " + e.getMessage());
+                        LOG.warn("Unexpected interruption", e);
                     }
                 }
             }.start();
@@ -847,25 +851,26 @@ public class LearnerHandler extends ZooKeeperThread {
                 LOG.warn("Forcing snapshot sync - should not see this in production");
             } else if (lastProcessedZxid == peerLastZxid) {
                 // Follower is already sync with us, send empty diff
-                LOG.info("Sending DIFF zxid=0x" + Long.toHexString(peerLastZxid) + " for peer sid: " + getSid());
+                LOG.info(
+                    "Sending DIFF zxid=0x{} for peer sid: {}",
+                    Long.toHexString(peerLastZxid),
+                    getSid());
                 queueOpPacket(Leader.DIFF, peerLastZxid);
                 needOpPacket = false;
                 needSnap = false;
             } else if (peerLastZxid > maxCommittedLog && !isPeerNewEpochZxid) {
                 // Newer than committedLog, send trunc and done
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Sending TRUNC to follower zxidToSend=0x"
-                              + Long.toHexString(maxCommittedLog)
-                              + " for peer sid:"
-                              + getSid());
-                }
+                LOG.debug(
+                    "Sending TRUNC to follower zxidToSend=0x{} for peer sid:{}",
+                    Long.toHexString(maxCommittedLog),
+                    getSid());
                 queueOpPacket(Leader.TRUNC, maxCommittedLog);
                 currentZxid = maxCommittedLog;
                 needOpPacket = false;
                 needSnap = false;
             } else if ((maxCommittedLog >= peerLastZxid) && (minCommittedLog <= peerLastZxid)) {
                 // Follower is within commitLog range
-                LOG.info("Using committedLog for peer sid: " + getSid());
+                LOG.info("Using committedLog for peer sid: {}", getSid());
                 Iterator<Proposal> itr = db.getCommittedLog().iterator();
                 currentZxid = queueCommittedProposals(itr, peerLastZxid, null, maxCommittedLog);
                 needSnap = false;
@@ -878,22 +883,21 @@ public class LearnerHandler extends ZooKeeperThread {
                 // is older than on-disk txnlog
                 Iterator<Proposal> txnLogItr = db.getProposalsFromTxnLog(peerLastZxid, sizeLimit);
                 if (txnLogItr.hasNext()) {
-                    LOG.info("Use txnlog and committedLog for peer sid: " + getSid());
+                    LOG.info("Use txnlog and committedLog for peer sid: {}", getSid());
                     currentZxid = queueCommittedProposals(txnLogItr, peerLastZxid, minCommittedLog, maxCommittedLog);
 
                     if (currentZxid < minCommittedLog) {
-                        LOG.info("Detected gap between end of txnlog: 0x{} and start of committedLog: 0x{}",
-                                 Long.toHexString(currentZxid),
-                                 Long.toHexString(minCommittedLog));
+                        LOG.info(
+                            "Detected gap between end of txnlog: 0x{} and start of committedLog: 0x{}",
+                            Long.toHexString(currentZxid),
+                            Long.toHexString(minCommittedLog));
                         currentZxid = peerLastZxid;
                         // Clear out currently queued requests and revert
                         // to sending a snapshot.
                         queuedPackets.clear();
                         needOpPacket = true;
                     } else {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Queueing committedLog 0x{}", Long.toHexString(currentZxid));
-                        }
+                        LOG.debug("Queueing committedLog 0x{}", Long.toHexString(currentZxid));
                         Iterator<Proposal> committedLogItr = db.getCommittedLog().iterator();
                         currentZxid = queueCommittedProposals(committedLogItr, currentZxid, null, maxCommittedLog);
                         needSnap = false;
@@ -905,22 +909,22 @@ public class LearnerHandler extends ZooKeeperThread {
                     txnProposalItr.close();
                 }
             } else {
-                LOG.warn("Unhandled scenario for peer sid: {} maxCommittedLog=0x{}"
-                         + " minCommittedLog=0x{} lastProcessedZxid=0x{}"
-                         + " peerLastZxid=0x{} txnLogSyncEnabled={}",
-                         getSid(),
-                         Long.toHexString(maxCommittedLog),
-                         Long.toHexString(minCommittedLog),
-                         Long.toHexString(lastProcessedZxid),
-                         Long.toHexString(peerLastZxid),
-                         txnLogSyncEnabled);
+                LOG.warn(
+                    "Unhandled scenario for peer sid: {} maxCommittedLog=0x{}"
+                        + " minCommittedLog=0x{} lastProcessedZxid=0x{}"
+                        + " peerLastZxid=0x{} txnLogSyncEnabled={}",
+                    getSid(),
+                    Long.toHexString(maxCommittedLog),
+                    Long.toHexString(minCommittedLog),
+                    Long.toHexString(lastProcessedZxid),
+                    Long.toHexString(peerLastZxid),
+                    txnLogSyncEnabled);
             }
             if (needSnap) {
                 currentZxid = db.getDataTreeLastProcessedZxid();
             }
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Start forwarding 0x" + Long.toHexString(currentZxid) + " for peer sid: " + getSid());
-            }
+
+            LOG.debug("Start forwarding 0x{} for peer sid: {}", Long.toHexString(currentZxid), getSid());
             leaderLastZxid = learnerMaster.startForwarding(this, currentZxid);
         } finally {
             rl.unlock();
@@ -929,7 +933,7 @@ public class LearnerHandler extends ZooKeeperThread {
         if (needOpPacket && !needSnap) {
             // This should never happen, but we should fall back to sending
             // snapshot just in case.
-            LOG.error("Unhandled scenario for peer sid: " + getSid() + " fall back to use snapshot");
+            LOG.error("Unhandled scenario for peer sid: {} fall back to use snapshot",  getSid());
             needSnap = true;
         }
 
@@ -974,10 +978,10 @@ public class LearnerHandler extends ZooKeeperThread {
 
                 // Send diff when we see the follower's zxid in our history
                 if (packetZxid == peerLastZxid) {
-                    LOG.info("Sending DIFF zxid=0x"
-                             + Long.toHexString(lastCommittedZxid)
-                             + " for peer sid: "
-                             + getSid());
+                    LOG.info(
+                        "Sending DIFF zxid=0x{}  for peer sid: {}",
+                        Long.toHexString(lastCommittedZxid),
+                        getSid());
                     queueOpPacket(Leader.DIFF, lastCommittedZxid);
                     needOpPacket = false;
                     continue;
@@ -985,10 +989,10 @@ public class LearnerHandler extends ZooKeeperThread {
 
                 if (isPeerNewEpochZxid) {
                     // Send diff and fall through if zxid is of a new-epoch
-                    LOG.info("Sending DIFF zxid=0x"
-                             + Long.toHexString(lastCommittedZxid)
-                             + " for peer sid: "
-                             + getSid());
+                    LOG.info(
+                        "Sending DIFF zxid=0x{}  for peer sid: {}",
+                        Long.toHexString(lastCommittedZxid),
+                        getSid());
                     queueOpPacket(Leader.DIFF, lastCommittedZxid);
                     needOpPacket = false;
                 } else if (packetZxid > peerLastZxid) {
@@ -1002,10 +1006,10 @@ public class LearnerHandler extends ZooKeeperThread {
                         return queuedZxid;
                     }
 
-                    LOG.info("Sending TRUNC zxid=0x"
-                             + Long.toHexString(prevProposalZxid)
-                             + " for peer sid: "
-                             + getSid());
+                    LOG.info(
+                        "Sending TRUNC zxid=0x{}  for peer sid: {}",
+                        Long.toHexString(prevProposalZxid),
+                        getSid());
                     queueOpPacket(Leader.TRUNC, prevProposalZxid);
                     needOpPacket = false;
                 }
@@ -1029,7 +1033,10 @@ public class LearnerHandler extends ZooKeeperThread {
             // We will send DIFF for this kind of zxid in any case. This if-block
             // is the catch when our history older than learner and there is
             // no new txn since then. So we need an empty diff
-            LOG.info("Sending DIFF zxid=0x" + Long.toHexString(lastCommittedZxid) + " for peer sid: " + getSid());
+            LOG.info(
+                "Sending TRUNC zxid=0x{}  for peer sid: {}",
+                Long.toHexString(lastCommittedZxid),
+                getSid());
             queueOpPacket(Leader.DIFF, lastCommittedZxid);
             needOpPacket = false;
         }

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandlerBean.java

@@ -59,7 +59,7 @@ public class LearnerHandlerBean implements LearnerHandlerMXBean, ZKMBeanInfo {
 
     @Override
     public void terminateConnection() {
-        LOG.info("terminating learner handler connection on demand " + toString());
+        LOG.info("terminating learner handler connection on demand {}", toString());
         learnerHandler.shutdown();
     }
 

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerMaster.java

@@ -44,10 +44,10 @@ public abstract class LearnerMaster {
 
     static {
         DEFAULT_CONCURRENT_SNAPSYNCS = Integer.getInteger(MAX_CONCURRENT_SNAPSYNCS, 10);
-        LOG.info(MAX_CONCURRENT_SNAPSYNCS + " = " + DEFAULT_CONCURRENT_SNAPSYNCS);
+        LOG.info("{} = {}", MAX_CONCURRENT_SNAPSYNCS, DEFAULT_CONCURRENT_SNAPSYNCS);
 
         DEFAULT_CONCURRENT_DIFF_SYNCS = Integer.getInteger(MAX_CONCURRENT_DIFF_SYNCS, 100);
-        LOG.info(MAX_CONCURRENT_DIFF_SYNCS + " = " + DEFAULT_CONCURRENT_DIFF_SYNCS);
+        LOG.info("{} = {}", MAX_CONCURRENT_DIFF_SYNCS, DEFAULT_CONCURRENT_DIFF_SYNCS);
     }
 
     private volatile int maxConcurrentSnapSyncs = DEFAULT_CONCURRENT_SNAPSYNCS;

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerSessionTracker.java

@@ -113,7 +113,7 @@ public class LearnerSessionTracker extends UpgradeableSessionTracker {
         if (added) {
             // Only do extra logging so we know what kind of session this is
             // if we're supporting both kinds of sessions
-            LOG.info("Committing global session 0x" + Long.toHexString(sessionId));
+            LOG.info("Committing global session 0x{}", Long.toHexString(sessionId));
         }
 
         // If the session moved before the session upgrade finished, it's

+ 11 - 9
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java

@@ -66,9 +66,9 @@ public class Observer extends Learner {
 
     static {
         reconnectDelayMs = Long.getLong(OBSERVER_RECONNECT_DELAY_MS, 0);
-        LOG.info(OBSERVER_RECONNECT_DELAY_MS + " = " + reconnectDelayMs);
+        LOG.info("{} = {}", OBSERVER_RECONNECT_DELAY_MS, reconnectDelayMs);
         observerElectionDelayMs = Long.getLong(OBSERVER_ELECTION_DELAY_MS, 200);
-        LOG.info(OBSERVER_ELECTION_DELAY_MS + " = " + observerElectionDelayMs);
+        LOG.info("{} = {}", OBSERVER_ELECTION_DELAY_MS , observerElectionDelayMs);
     }
 
     /**
@@ -133,9 +133,11 @@ public class Observer extends Learner {
             if (connectTime != 0) {
                 long connectionDuration = System.currentTimeMillis() - connectTime;
 
-                LOG.info("Disconnected from leader (with address: {}). "
-                        + "Was connected for {}ms. Sync state: {}",
-                    leaderAddr, connectionDuration, completedSync);
+                LOG.info(
+                    "Disconnected from leader (with address: {}). Was connected for {}ms. Sync state: {}",
+                    leaderAddr,
+                    connectionDuration,
+                    completedSync);
                 messageTracker.dumpToLog(leaderAddr.toString());
             }
         }
@@ -227,7 +229,7 @@ public class Observer extends Learner {
      * Shutdown the Observer.
      */
     public void shutdown() {
-        LOG.info("shutdown called", new Exception("shutdown Observer"));
+        LOG.info("shutdown Observer");
         super.shutdown();
     }
 
@@ -242,11 +244,11 @@ public class Observer extends Learner {
     private static void waitForReconnectDelayHelper(long delayValueMs) {
         if (delayValueMs > 0) {
             long randomDelay = (long) (delayValueMs * Math.random());
-            LOG.info("Waiting for " + randomDelay + " ms before reconnecting with the leader");
+            LOG.info("Waiting for {} ms before reconnecting with the leader", randomDelay);
             try {
                 Thread.sleep(randomDelay);
             } catch (InterruptedException e) {
-                LOG.warn("Interrupted while waiting" + e.getMessage());
+                LOG.warn("Interrupted while waiting", e);
             }
         }
     }
@@ -285,7 +287,7 @@ public class Observer extends Learner {
 
     public static void setObserverElectionDelayMs(long electionDelayMs) {
         observerElectionDelayMs = electionDelayMs;
-        LOG.info(OBSERVER_ELECTION_DELAY_MS + " = " + observerElectionDelayMs);
+        LOG.info("{} = {}", OBSERVER_ELECTION_DELAY_MS, observerElectionDelayMs);
     }
 
 }

+ 16 - 14
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java

@@ -293,11 +293,12 @@ public class ObserverMaster extends LearnerMaster implements Runnable {
         if (itr.hasNext()) {
             QuorumPacket packet = itr.next();
             if (packet.getZxid() > lastSeenZxid + 1) {
-                LOG.error("LearnerHandler is too far behind ({} < {}), disconnecting {} at {}",
-                          Long.toHexString(lastSeenZxid + 1),
-                          Long.toHexString(packet.getZxid()),
-                          learnerHandler.getSid(),
-                          learnerHandler.getRemoteAddress());
+                LOG.error(
+                    "LearnerHandler is too far behind (0x{} < 0x{}), disconnecting {} at {}",
+                    Long.toHexString(lastSeenZxid + 1),
+                    Long.toHexString(packet.getZxid()),
+                    learnerHandler.getSid(),
+                    learnerHandler.getRemoteAddress());
                 learnerHandler.shutdown();
                 return -1;
             } else if (packet.getZxid() == lastSeenZxid + 1) {
@@ -313,15 +314,16 @@ public class ObserverMaster extends LearnerMaster implements Runnable {
                 learnerHandler.queuePacket(packet);
                 queueBytesUsed += LearnerHandler.packetSize(packet);
             }
-            LOG.info("finished syncing observer from retained commit queue: sid {}, "
-                     + "queue head 0x{}, queue tail 0x{}, sync position 0x{}, num packets used {}, "
-                     + "num bytes used {}",
-                     learnerHandler.getSid(),
-                     Long.toHexString(queueHeadZxid),
-                     Long.toHexString(packet.getZxid()),
-                     Long.toHexString(lastSeenZxid),
-                     packet.getZxid() - lastSeenZxid,
-                     queueBytesUsed);
+            LOG.info(
+                "finished syncing observer from retained commit queue: sid {}, "
+                    + "queue head 0x{}, queue tail 0x{}, sync position 0x{}, num packets used {}, "
+                    + "num bytes used {}",
+                learnerHandler.getSid(),
+                Long.toHexString(queueHeadZxid),
+                Long.toHexString(packet.getZxid()),
+                Long.toHexString(lastSeenZxid),
+                packet.getZxid() - lastSeenZxid,
+                queueBytesUsed);
         }
         activeObservers.add(learnerHandler);
         return lastProposedZxid;

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverZooKeeperServer.java

@@ -52,7 +52,7 @@ public class ObserverZooKeeperServer extends LearnerZooKeeperServer {
 
     ObserverZooKeeperServer(FileTxnSnapLog logFactory, QuorumPeer self, ZKDatabase zkDb) throws IOException {
         super(logFactory, self.tickTime, self.minSessionTimeout, self.maxSessionTimeout, self.clientPortListenBacklog, zkDb, self);
-        LOG.info("syncEnabled =" + syncRequestProcessorEnabled);
+        LOG.info("syncEnabled ={}", syncRequestProcessorEnabled);
     }
 
     public Observer getObserver() {

+ 55 - 46
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

@@ -326,9 +326,12 @@ public class QuorumCnxManager {
         try {
             startConnection(sock, sid);
         } catch (IOException e) {
-            LOG.error("Exception while connecting, id: {}, addr: {}, closing learner connection", new Object[]{sid, sock.getRemoteSocketAddress()}, e);
+            LOG.error(
+                "Exception while connecting, id: {}, addr: {}, closing learner connection",
+                sid,
+                sock.getRemoteSocketAddress(),
+                e);
             closeSocket(sock);
-            return;
         }
     }
 
@@ -417,8 +420,10 @@ public class QuorumCnxManager {
 
         // If lost the challenge, then drop the new connection
         if (sid > self.getId()) {
-            LOG.info("Have smaller server identifier, so dropping the "
-                     + "connection: (" + sid + ", " + self.getId() + ")");
+            LOG.info(
+                "Have smaller server identifier, so dropping the connection: ({}, {})",
+                sid,
+                self.getId());
             closeSocket(sock);
             // Otherwise proceed with the connection
         } else {
@@ -521,10 +526,10 @@ public class QuorumCnxManager {
                  * the connection.
                  */
                 sid = observerCounter.getAndDecrement();
-                LOG.info("Setting arbitrary identifier to observer: " + sid);
+                LOG.info("Setting arbitrary identifier to observer: {}", sid);
             }
         } catch (IOException e) {
-            LOG.warn("Exception reading or writing challenge: {}", e);
+            LOG.warn("Exception reading or writing challenge", e);
             closeSocket(sock);
             return;
         }
@@ -649,15 +654,15 @@ public class QuorumCnxManager {
             // exception to be thrown, also UAE cannot be wrapped cleanly
             // so we log the exception in order to capture this critical
             // detail.
-            LOG.warn("Cannot open channel to " + sid + " at election address " + electionAddr, e);
+            LOG.warn("Cannot open channel to {} at election address {}", sid, electionAddr, e);
             closeSocket(sock);
             throw e;
         } catch (X509Exception e) {
-            LOG.warn("Cannot open secure channel to " + sid + " at election address " + electionAddr, e);
+            LOG.warn("Cannot open secure channel to {} at election address {}", sid, electionAddr, e);
             closeSocket(sock);
             return false;
         } catch (IOException e) {
-            LOG.warn("Cannot open channel to " + sid + " at election address " + electionAddr, e);
+            LOG.warn("Cannot open channel to {} at election address {}", sid, electionAddr, e);
             closeSocket(sock);
             return false;
         }
@@ -697,7 +702,7 @@ public class QuorumCnxManager {
                 }
             }
             if (!knownId) {
-                LOG.warn("Invalid server id: " + sid);
+                LOG.warn("Invalid server id: {} ", sid);
             }
         }
     }
@@ -838,11 +843,11 @@ public class QuorumCnxManager {
                 LOG.info("Election port bind maximum retries is {}", maxRetry == 0 ? "infinite" : maxRetry);
                 portBindMaxRetry = maxRetry;
             } else {
-                LOG.info("'{}' contains invalid value: {}(must be >= 0). "
-                         + "Use default value of {} instead.",
-                         ELECTION_PORT_BIND_RETRY,
-                         maxRetry,
-                         DEFAULT_PORT_BIND_MAX_RETRY);
+                LOG.info(
+                    "'{}' contains invalid value: {}(must be >= 0). Use default value of {} instead.",
+                    ELECTION_PORT_BIND_RETRY,
+                    maxRetry,
+                    DEFAULT_PORT_BIND_MAX_RETRY);
                 portBindMaxRetry = DEFAULT_PORT_BIND_MAX_RETRY;
             }
         }
@@ -886,14 +891,14 @@ public class QuorumCnxManager {
                         self.recreateSocketAddresses(self.getId());
                         addr = self.getElectionAddress();
                     }
-                    LOG.info("My election bind port: " + addr.toString());
+                    LOG.info("My election bind port: {}", addr.toString());
                     setName(addr.toString());
                     ss.bind(addr);
                     while (!shutdown) {
                         try {
                             client = ss.accept();
                             setSockOpts(client);
-                            LOG.info("Received connection request " + formatInetAddr((InetSocketAddress) client.getRemoteSocketAddress()));
+                            LOG.info("Received connection request {}", formatInetAddr((InetSocketAddress) client.getRemoteSocketAddress()));
                             // Receive and handle the connection request
                             // asynchronously if the quorum sasl authentication is
                             // enabled. This is required because sasl server
@@ -906,9 +911,10 @@ public class QuorumCnxManager {
                             }
                             numRetries = 0;
                         } catch (SocketTimeoutException e) {
-                            LOG.warn("The socket is listening for the election accepted "
-                                     + "and it timed out unexpectedly, but will retry."
-                                     + "see ZOOKEEPER-2836");
+                            LOG.warn(
+                                "The socket is listening for the election accepted "
+                                    + "and it timed out unexpectedly, but will retry."
+                                    + "see ZOOKEEPER-2836");
                         }
                     }
                 } catch (IOException e) {
@@ -924,23 +930,21 @@ public class QuorumCnxManager {
                     } catch (IOException ie) {
                         LOG.error("Error closing server socket", ie);
                     } catch (InterruptedException ie) {
-                        LOG.error("Interrupted while sleeping. " + "Ignoring exception", ie);
+                        LOG.error("Interrupted while sleeping. Ignoring exception", ie);
                     }
                     closeSocket(client);
                 }
             }
             LOG.info("Leaving listener");
             if (!shutdown) {
-                LOG.error("As I'm leaving the listener thread after "
-                          + numRetries
-                          + " errors. "
-                          + "I won't be able to participate in leader "
-                          + "election any longer: "
-                          + formatInetAddr(self.getElectionAddress())
-                          + ". Use "
-                          + ELECTION_PORT_BIND_RETRY
-                          + " property to "
-                          + "increase retry count.");
+                LOG.error(
+                    "As I'm leaving the listener thread after {} errors. "
+                        + "I won't be able to participate in leader election any longer: {}."
+                        + "Use {} property to increase retry count.",
+                          numRetries,
+                          formatInetAddr(self.getElectionAddress()),
+                          ELECTION_PORT_BIND_RETRY);
+
                 if (exitException instanceof SocketException) {
                     // After leaving listener thread, the host cannot join the
                     // quorum anymore, this is a severe error that we cannot
@@ -965,11 +969,11 @@ public class QuorumCnxManager {
             try {
                 LOG.debug("Trying to close listener: {}", ss);
                 if (ss != null) {
-                    LOG.debug("Closing listener: {}", +QuorumCnxManager.this.mySid);
+                    LOG.debug("Closing listener: {}", QuorumCnxManager.this.mySid);
                     ss.close();
                 }
             } catch (IOException e) {
-                LOG.warn("Exception when shutting down listener: " + e);
+                LOG.warn("Exception when shutting down listener", e);
             }
         }
 
@@ -1103,7 +1107,7 @@ public class QuorumCnxManager {
                         if (bq != null) {
                             b = pollSendQueue(bq, 1000, TimeUnit.MILLISECONDS);
                         } else {
-                            LOG.error("No queue of incoming messages for " + "server " + sid);
+                            LOG.error("No queue of incoming messages for server {}", sid);
                             break;
                         }
 
@@ -1116,12 +1120,15 @@ public class QuorumCnxManager {
                     }
                 }
             } catch (Exception e) {
-                LOG.warn("Exception when using channel: for id " + sid
-                         + " my id = " + QuorumCnxManager.this.mySid
-                         + " error = " + e);
+                LOG.warn(
+                    "Exception when using channel: for id {} my id = {}",
+                    sid ,
+                    QuorumCnxManager.this.mySid,
+                    e);
             }
             this.finish();
-            LOG.warn("Send worker leaving thread " + " id " + sid + " my id = " + self.getId());
+
+            LOG.warn("Send worker leaving thread id {} my id = {}", sid, self.getId());
         }
 
     }
@@ -1148,7 +1155,7 @@ public class QuorumCnxManager {
                 // OK to wait until socket disconnects while reading.
                 sock.setSoTimeout(0);
             } catch (IOException e) {
-                LOG.error("Error while accessing socket for " + sid, e);
+                LOG.error("Error while accessing socket for {}", sid, e);
                 closeSocket(sock);
                 running = false;
             }
@@ -1194,9 +1201,11 @@ public class QuorumCnxManager {
                     addToRecvQueue(new Message(ByteBuffer.wrap(msgArray), sid));
                 }
             } catch (Exception e) {
-                LOG.warn("Connection broken for id " + sid
-                         + ", my id = " + QuorumCnxManager.this.mySid
-                         + ", error = ", e);
+                LOG.warn(
+                    "Connection broken for id {}, my id = {}",
+                    sid,
+                    QuorumCnxManager.this.mySid,
+                    e);
             } finally {
                 LOG.warn("Interrupting SendWorker");
                 sw.finish();
@@ -1230,14 +1239,14 @@ public class QuorumCnxManager {
                 queue.remove();
             } catch (NoSuchElementException ne) {
                 // element could be removed by poll()
-                LOG.debug("Trying to remove from an empty " + "Queue. Ignoring exception.", ne);
+                LOG.debug("Trying to remove from an empty Queue. Ignoring exception.", ne);
             }
         }
         try {
             queue.add(buffer);
         } catch (IllegalStateException ie) {
             // This should never happen
-            LOG.error("Unable to insert an element in the queue " + ie);
+            LOG.error("Unable to insert an element in the queue ", ie);
         }
     }
 
@@ -1290,14 +1299,14 @@ public class QuorumCnxManager {
                     recvQueue.remove();
                 } catch (NoSuchElementException ne) {
                     // element could be removed by poll()
-                    LOG.debug("Trying to remove from an empty " + "recvQueue. Ignoring exception.", ne);
+                    LOG.debug("Trying to remove from an empty recvQueue. Ignoring exception.", ne);
                 }
             }
             try {
                 recvQueue.add(msg);
             } catch (IllegalStateException ie) {
                 // This should never happen
-                LOG.error("Unable to insert element in the recvQueue " + ie);
+                LOG.error("Unable to insert element in the recvQueue ", ie);
             }
         }
     }

+ 28 - 20
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -723,7 +723,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                 while (running) {
                     udpSocket.receive(packet);
                     if (packet.getLength() != 4) {
-                        LOG.warn("Got more than just an xid! Len = " + packet.getLength());
+                        LOG.warn("Got more than just an xid! Len = {}", packet.getLength());
                     } else {
                         responseBuffer.clear();
                         responseBuffer.getInt(); // Skip the xid
@@ -1037,7 +1037,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                 // this should only happen once when moving to a
                 // new code version
                 currentEpoch = epochOfZxid;
-                LOG.info(CURRENT_EPOCH_FILENAME + " not found! Creating with a reasonable default of {}. This should only happen when you are upgrading your installation", currentEpoch);
+                LOG.info(
+                    "{} not found! Creating with a reasonable default of {}. "
+                        + "This should only happen when you are upgrading your installation",
+                    CURRENT_EPOCH_FILENAME,
+                    currentEpoch);
                 writeLongToFile(CURRENT_EPOCH_FILENAME, currentEpoch);
             }
             if (epochOfZxid > currentEpoch) {
@@ -1053,7 +1057,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                 // this should only happen once when moving to a
                 // new code version
                 acceptedEpoch = epochOfZxid;
-                LOG.info(ACCEPTED_EPOCH_FILENAME + " not found! Creating with a reasonable default of {}. This should only happen when you are upgrading your installation", acceptedEpoch);
+                LOG.info(
+                    "{} not found! Creating with a reasonable default of {}. "
+                        + "This should only happen when you are upgrading your installation",
+                    ACCEPTED_EPOCH_FILENAME,
+                    acceptedEpoch);
                 writeLongToFile(ACCEPTED_EPOCH_FILENAME, acceptedEpoch);
             }
             if (acceptedEpoch < currentEpoch) {
@@ -1587,7 +1595,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
      * Set the number of milliseconds of each tick
      */
     public void setTickTime(int tickTime) {
-        LOG.info("tickTime set to " + tickTime);
+        LOG.info("tickTime set to {}", tickTime);
         this.tickTime = tickTime;
     }
 
@@ -1609,7 +1617,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     /** Whether to enable local sessions */
     public void enableLocalSessions(boolean flag) {
-        LOG.info("Local sessions " + (flag ? "enabled" : "disabled"));
+        LOG.info("Local sessions {}", (flag ? "enabled" : "disabled"));
         localSessionsEnabled = flag;
     }
 
@@ -1620,7 +1628,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     /** Whether to allow local sessions to upgrade to global sessions */
     public void enableLocalSessionsUpgrading(boolean flag) {
-        LOG.info("Local session upgrading " + (flag ? "enabled" : "disabled"));
+        LOG.info("Local session upgrading {}", (flag ? "enabled" : "disabled"));
         localSessionsUpgradingEnabled = flag;
     }
 
@@ -1631,7 +1639,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     /** minimum session timeout in milliseconds */
     public void setMinSessionTimeout(int min) {
-        LOG.info("minSessionTimeout set to " + min);
+        LOG.info("minSessionTimeout set to {}", min);
         this.minSessionTimeout = min;
     }
 
@@ -1642,7 +1650,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     /** maximum session timeout in milliseconds */
     public void setMaxSessionTimeout(int max) {
-        LOG.info("maxSessionTimeout set to " + max);
+        LOG.info("maxSessionTimeout set to {}", max);
         this.maxSessionTimeout = max;
     }
 
@@ -1667,7 +1675,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
      * Set the number of ticks that the initial synchronization phase can take
      */
     public void setInitLimit(int initLimit) {
-        LOG.info("initLimit set to " + initLimit);
+        LOG.info("initLimit set to {}", initLimit);
         this.initLimit = initLimit;
     }
 
@@ -1770,7 +1778,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                             QuorumPeerConfig.writeDynamicConfig(fileName, qv, true);
                         }
                     } catch (IOException e) {
-                        LOG.error("Error writing next dynamic config file to disk: ", e.getMessage());
+                        LOG.error("Error writing next dynamic config file to disk", e);
                     }
                 }
             }
@@ -1782,11 +1790,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
             if ((quorumVerifier != null) && (quorumVerifier.getVersion() >= qv.getVersion())) {
                 // this is normal. For example - server found out about new config through FastLeaderElection gossiping
                 // and then got the same config in UPTODATE message so its already known
-                LOG.debug("{} setQuorumVerifier called with known or old config {}."
-                          + " Current version: {}",
-                          getId(),
-                          qv.getVersion(),
-                          quorumVerifier.getVersion());
+                LOG.debug(
+                    "{} setQuorumVerifier called with known or old config {}. Current version: {}",
+                    getId(),
+                    qv.getVersion(),
+                    quorumVerifier.getVersion());
                 return quorumVerifier;
             }
             QuorumVerifier prevQV = quorumVerifier;
@@ -1803,7 +1811,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                         QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename, qv, false);
                         QuorumPeerConfig.editStaticConfig(configFilename, dynamicConfigFilename, needEraseClientInfoFromStaticConfig());
                     } catch (IOException e) {
-                        LOG.error("Error closing file: ", e.getMessage());
+                        LOG.error("Error closing file", e);
                     }
                 } else {
                     LOG.info("writeToDisk == true but configFilename == null");
@@ -1849,7 +1857,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
      * Set the synclimit
      */
     public void setSyncLimit(int syncLimit) {
-        LOG.info("syncLimit set to " + syncLimit);
+        LOG.info("syncLimit set to {}", syncLimit);
         this.syncLimit = syncLimit;
     }
 
@@ -1864,7 +1872,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
      * Set the connectToLearnerMasterLimit
      */
     public void setConnectToLearnerMasterLimit(int connectToLearnerMasterLimit) {
-        LOG.info("connectToLearnerMasterLimit set to " + connectToLearnerMasterLimit);
+        LOG.info("connectToLearnerMasterLimit set to {}", connectToLearnerMasterLimit);
         this.connectToLearnerMasterLimit = connectToLearnerMasterLimit;
     }
 
@@ -1880,7 +1888,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
      */
     public boolean getSyncEnabled() {
         if (System.getProperty(SYNC_ENABLED) != null) {
-            LOG.info(SYNC_ENABLED + "=" + Boolean.getBoolean(SYNC_ENABLED));
+            LOG.info("{}={}", SYNC_ENABLED, Boolean.getBoolean(SYNC_ENABLED));
             return Boolean.getBoolean(SYNC_ENABLED);
         } else {
             return syncEnabled;
@@ -2296,7 +2304,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         if (currentVote != null && designatedLeader != currentVote.getId()) {
             setCurrentVote(new Vote(designatedLeader, zxid));
             reconfigFlagSet();
-            LOG.warn("Suggested leader: " + designatedLeader);
+            LOG.warn("Suggested leader: {}", designatedLeader);
             return true;
         }
         return false;

+ 6 - 5
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

@@ -641,7 +641,7 @@ public class QuorumPeerConfig {
             try {
                 f.delete();
             } catch (Exception e) {
-                LOG.warn("deleting " + filename + " failed");
+                LOG.warn("deleting {} failed", filename);
             }
         }
     }
@@ -709,7 +709,7 @@ public class QuorumPeerConfig {
         } else {
             if (warnings) {
                 if (numParticipators <= 2) {
-                    LOG.warn("No server failure will be tolerated. " + "You need at least 3 servers.");
+                    LOG.warn("No server failure will be tolerated. You need at least 3 servers.");
                 } else if (numParticipators % 2 == 0) {
                     LOG.warn("Non-optimial configuration, consider an odd number of servers.");
                 }
@@ -774,9 +774,10 @@ public class QuorumPeerConfig {
             ? LearnerType.OBSERVER
             : LearnerType.PARTICIPANT;
         if (roleByServersList != peerType) {
-            LOG.warn("Peer type from servers list (" + roleByServersList
-                     + ") doesn't match peerType (" + peerType
-                     + "). Defaulting to servers list.");
+            LOG.warn(
+                "Peer type from servers list ({}) doesn't match peerType ({}). Defaulting to servers list.",
+                roleByServersList,
+                peerType);
 
             peerType = roleByServersList;
         }

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

@@ -128,7 +128,7 @@ public class QuorumPeerMain {
         if (args.length == 1 && config.isDistributed()) {
             runFromConfig(config);
         } else {
-            LOG.warn("Either no config or no quorum defined in config, running " + " in standalone mode");
+            LOG.warn("Either no config or no quorum defined in config, running in standalone mode");
             // there is only server in the quorum -- run as standalone
             ZooKeeperServerMain.main(args);
         }

+ 2 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java

@@ -129,7 +129,7 @@ public abstract class QuorumZooKeeperServer extends ZooKeeperServer {
     public void upgrade(long sessionId) {
         Request request = makeUpgradeRequest(sessionId);
         if (request != null) {
-            LOG.info("Upgrading session 0x" + Long.toHexString(sessionId));
+            LOG.info("Upgrading session 0x{}", Long.toHexString(sessionId));
             // This must be a global request
             submitRequest(request);
         }
@@ -152,9 +152,7 @@ public abstract class QuorumZooKeeperServer extends ZooKeeperServer {
                 si.setLocalSession(true);
                 reqType = "local";
             }
-            LOG.info("Submitting " + reqType
-                     + " closeSession request"
-                     + " for session 0x" + Long.toHexString(si.sessionId));
+            LOG.info("Submitting {} closeSession request for session 0x{}", reqType, Long.toHexString(si.sessionId));
             break;
         default:
             break;

+ 0 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyRequestProcessor.java

@@ -28,7 +28,6 @@ import org.apache.zookeeper.server.RequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperCriticalThread;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooTrace;
-import org.apache.zookeeper.server.quorum.Leader.XidRolloverException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -104,11 +103,6 @@ public class ReadOnlyRequestProcessor extends ZooKeeperCriticalThread implements
                     nextProcessor.processRequest(request);
                 }
             }
-        } catch (RequestProcessorException e) {
-            if (e.getCause() instanceof XidRolloverException) {
-                LOG.info(e.getCause().getMessage());
-            }
-            handleException(this.getName(), e);
         } catch (Exception e) {
             handleException(this.getName(), e);
         }

+ 7 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/UnifiedServerSocket.java

@@ -240,7 +240,7 @@ public class UnifiedServerSocket extends ServerSocket {
                 bytesRead = prependableSocket.getInputStream().read(litmus, 0, litmus.length);
             } catch (SocketTimeoutException e) {
                 // Didn't read anything within the timeout, fallthrough and assume the connection is plaintext.
-                LOG.warn("Socket mode detection timed out after " + newTimeout + " ms, assuming PLAINTEXT");
+                LOG.warn("Socket mode detection timed out after {} ms, assuming PLAINTEXT", newTimeout);
             } finally {
                 // restore socket timeout to the old value
                 try {
@@ -248,7 +248,7 @@ public class UnifiedServerSocket extends ServerSocket {
                         prependableSocket.setSoTimeout(oldTimeout);
                     }
                 } catch (Exception e) {
-                    LOG.warn("Failed to restore old socket timeout value of " + oldTimeout + " ms", e);
+                    LOG.warn("Failed to restore old socket timeout value of {} ms", oldTimeout, e);
                 }
             }
             if (bytesRead < 0) { // Got a EOF right away, definitely not using TLS. Fallthrough.
@@ -263,10 +263,11 @@ public class UnifiedServerSocket extends ServerSocket {
                 }
                 prependableSocket = null;
                 mode = Mode.TLS;
-                LOG.info("Accepted TLS connection from {} - {} - {}",
-                         sslSocket.getRemoteSocketAddress(),
-                         sslSocket.getSession().getProtocol(),
-                         sslSocket.getSession().getCipherSuite());
+                LOG.info(
+                    "Accepted TLS connection from {} - {} - {}",
+                    sslSocket.getRemoteSocketAddress(),
+                    sslSocket.getSession().getProtocol(),
+                    sslSocket.getSession().getCipherSuite());
             } else if (allowInsecureConnection) {
                 prependableSocket.prependToInputStream(litmus, 0, bytesRead);
                 mode = Mode.PLAINTEXT;

+ 1 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/UpgradeableSessionTracker.java

@@ -89,7 +89,7 @@ public abstract class UpgradeableSessionTracker implements SessionTracker {
         // will get the timeout from the map
         Integer timeout = localSessionsWithTimeouts.remove(sessionId);
         if (timeout != null) {
-            LOG.info("Upgrading session 0x" + Long.toHexString(sessionId));
+            LOG.info("Upgrading session 0x{}", Long.toHexString(sessionId));
             // Track global session, which will add to global session tracker
             // on leader and do nothing on learner. Need to start track global
             // session in leader now to update the session expire between

+ 8 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java

@@ -75,9 +75,10 @@ public class SaslQuorumAuthLearner implements QuorumAuthLearner {
     @Override
     public void authenticate(Socket sock, String hostName) throws IOException {
         if (!quorumRequireSasl) { // let it through, we don't require auth
-            LOG.info("Skipping SASL authentication as {}={}",
-                     QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED,
-                     quorumRequireSasl);
+            LOG.info(
+                "Skipping SASL authentication as {}={}",
+                QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED,
+                quorumRequireSasl);
             return;
         }
         SaslClient sc = null;
@@ -138,9 +139,10 @@ public class SaslQuorumAuthLearner implements QuorumAuthLearner {
 
     private void checkAuthStatus(Socket sock, QuorumAuth.Status qpStatus) throws SaslException {
         if (qpStatus == QuorumAuth.Status.SUCCESS) {
-            LOG.info("Successfully completed the authentication using SASL. server addr: {}, status: {}",
-                     sock.getRemoteSocketAddress(),
-                     qpStatus);
+            LOG.info(
+                "Successfully completed the authentication using SASL. server addr: {}, status: {}",
+                sock.getRemoteSocketAddress(),
+                qpStatus);
         } else {
             throw new SaslException("Authentication failed against server addr: " + sock.getRemoteSocketAddress()
                                     + ", qpStatus: " + qpStatus);

+ 12 - 7
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthServer.java

@@ -94,9 +94,10 @@ public class SaslQuorumAuthServer implements QuorumAuthServer {
                     // limited number of retries.
                     if (++tries > MAX_RETRIES) {
                         send(dout, challenge, QuorumAuth.Status.ERROR);
-                        LOG.warn("Failed to authenticate using SASL, server addr: {}, retries={} exceeded.",
-                                 sock.getRemoteSocketAddress(),
-                                 tries);
+                        LOG.warn(
+                            "Failed to authenticate using SASL, server addr: {}, retries={} exceeded.",
+                            sock.getRemoteSocketAddress(),
+                            tries);
                         break;
                     }
                     send(dout, challenge, QuorumAuth.Status.IN_PROGRESS);
@@ -106,8 +107,9 @@ public class SaslQuorumAuthServer implements QuorumAuthServer {
             // Authentication exchange has completed
             if (ss.isComplete()) {
                 send(dout, challenge, QuorumAuth.Status.SUCCESS);
-                LOG.info("Successfully completed the authentication using SASL. learner addr: {}",
-                         sock.getRemoteSocketAddress());
+                LOG.info(
+                    "Successfully completed the authentication using SASL. learner addr: {}",
+                    sock.getRemoteSocketAddress());
             }
         } catch (Exception e) {
             try {
@@ -127,8 +129,11 @@ public class SaslQuorumAuthServer implements QuorumAuthServer {
                 throw new SaslException("Failed to authenticate using SASL: " + e.getMessage());
             } else {
                 LOG.warn("Failed to authenticate using SASL", e);
-                LOG.warn("Maintaining learner connection despite SASL authentication failure. server addr: {}, {}: {}",
-                         sock.getRemoteSocketAddress(), QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED, quorumRequireSasl);
+                LOG.warn(
+                    "Maintaining learner connection despite SASL authentication failure. server addr: {}, {}: {}",
+                    sock.getRemoteSocketAddress(),
+                    QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED,
+                    quorumRequireSasl);
                 // let it through, we don't require auth
             }
         } finally {

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java

@@ -136,7 +136,7 @@ public class QuorumHierarchical implements QuorumVerifier {
 
     public QuorumHierarchical(Properties qp) throws ConfigException {
         parse(qp);
-        LOG.info(serverWeight.size() + ", " + serverGroup.size() + ", " + groupWeight.size());
+        LOG.info("{}, {}, {}", serverWeight.size(), serverGroup.size(), groupWeight.size());
     }
 
     /**
@@ -155,7 +155,7 @@ public class QuorumHierarchical implements QuorumVerifier {
     private void readConfigFile(String filename) throws ConfigException {
         File configFile = new File(filename);
 
-        LOG.info("Reading configuration from: " + configFile);
+        LOG.info("Reading configuration from: {}", configFile);
 
         try {
             if (!configFile.exists()) {

+ 2 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/server/util/MessageTracker.java

@@ -109,13 +109,9 @@ public class MessageTracker {
         if (messages.isEmpty()) {
             LOG.info("No buffered timestamps for messages {} {}", sentOrReceivedText, serverAddr);
         } else {
-            LOG.warn("Last {} timestamps for messages {} {}:",
-                messages.size(), sentOrReceivedText, serverAddr);
+            LOG.warn("Last {} timestamps for messages {} {}:", messages.size(), sentOrReceivedText, serverAddr);
             while (!messages.isEmpty()) {
-                LOG.warn("{} {}  {}",
-                    sentOrReceivedText,
-                    serverAddr,
-                    messages.take().toString());
+                LOG.warn("{} {}  {}", sentOrReceivedText, serverAddr, messages.take().toString());
             }
         }
     }

+ 2 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RequestPathMetricsCollector.java

@@ -186,10 +186,10 @@ public class RequestPathMetricsCollector {
         scheduledExecutor.scheduleWithFixedDelay(() -> {
             LOG.info("%nHere are the top Read paths:");
             logTopPaths(aggregatePaths(4, queue -> !queue.isWriteOperation()),
-                        entry -> LOG.info(entry.getKey() + " : " + entry.getValue()));
+                        entry -> LOG.info("{} : {}", entry.getKey(), entry.getValue()));
             LOG.info("%nHere are the top Write paths:");
             logTopPaths(aggregatePaths(4, queue -> queue.isWriteOperation()),
-                        entry -> LOG.info(entry.getKey() + " : " + entry.getValue()));
+                        entry -> LOG.info("{} : {}", entry.getKey(), entry.getValue()));
         }, COLLECTOR_INITIAL_DELAY, COLLECTOR_DELAY, TimeUnit.MINUTES);
     }
 

+ 12 - 11
zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherCleaner.java

@@ -76,20 +76,21 @@ public class WatcherCleaner extends Thread {
         int suggestedMaxInProcessingThreshold = watcherCleanThreshold * watcherCleanThreadsNum;
         if (maxInProcessingDeadWatchers > 0 && maxInProcessingDeadWatchers < suggestedMaxInProcessingThreshold) {
             maxInProcessingDeadWatchers = suggestedMaxInProcessingThreshold;
-            LOG.info("The maxInProcessingDeadWatchers config is smaller "
-                     + "than the suggested one, change it to use {}",
-                     maxInProcessingDeadWatchers);
+            LOG.info(
+                "The maxInProcessingDeadWatchers config is smaller than the suggested one, change it to use {}",
+                maxInProcessingDeadWatchers);
         }
         this.maxInProcessingDeadWatchers = maxInProcessingDeadWatchers;
         this.deadWatchers = new HashSet<Integer>();
         this.cleaners = new WorkerService("DeadWatcherCleanner", watcherCleanThreadsNum, false);
 
-        LOG.info("watcherCleanThreshold={}, watcherCleanIntervalInSeconds={}"
-                 + ", watcherCleanThreadsNum={}, maxInProcessingDeadWatchers={}",
-                 watcherCleanThreshold,
-                 watcherCleanIntervalInSeconds,
-                 watcherCleanThreadsNum,
-                 maxInProcessingDeadWatchers);
+        LOG.info(
+            "watcherCleanThreshold={}, watcherCleanIntervalInSeconds={}"
+                + ", watcherCleanThreadsNum={}, maxInProcessingDeadWatchers={}",
+            watcherCleanThreshold,
+            watcherCleanIntervalInSeconds,
+            watcherCleanThreadsNum,
+            maxInProcessingDeadWatchers);
     }
 
     public void addDeadWatcher(int watcherBit) {
@@ -106,7 +107,7 @@ public class WatcherCleaner extends Thread {
                 long latency = Time.currentElapsedTime() - startTime;
                 ServerMetrics.getMetrics().ADD_DEAD_WATCHER_STALL_TIME.add(latency);
             } catch (InterruptedException e) {
-                LOG.info("Got interrupted while waiting for dead watches " + "queue size");
+                LOG.info("Got interrupted while waiting for dead watches queue size");
                 break;
             }
         }
@@ -136,7 +137,7 @@ public class WatcherCleaner extends Thread {
                         cleanEvent.wait(maxWaitMs);
                     }
                 } catch (InterruptedException e) {
-                    LOG.info("Received InterruptedException while " + "waiting for cleanEvent");
+                    LOG.info("Received InterruptedException while waiting for cleanEvent");
                     break;
                 }
             }

+ 11 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/util/SecurityUtils.java

@@ -120,7 +120,10 @@ public final class SecurityUtils {
                         String[] mechs = {"GSSAPI"};
                         LOG.debug(
                             "creating sasl client: {}={};service={};serviceHostname={}",
-                            entity, clientPrincipalName, serviceName, serviceHostname);
+                            entity,
+                            clientPrincipalName,
+                            serviceName,
+                            serviceHostname);
                         SaslClient saslClient = Sasl.createSaslClient(
                             mechs,
                             clientPrincipalName,
@@ -208,8 +211,10 @@ public final class SecurityUtils {
                                 GSSName.NT_HOSTBASED_SERVICE);
                             GSSCredential cred = manager.createCredential(gssName, GSSContext.DEFAULT_LIFETIME, krb5Mechanism, GSSCredential.ACCEPT_ONLY);
                             subject.getPrivateCredentials().add(cred);
-                            LOG.debug("Added private credential to service principal name: '{}',"
-                                      + " GSSCredential name: {}", servicePrincipalName, cred.getName());
+                            LOG.debug(
+                                "Added private credential to service principal name: '{}', GSSCredential name: {}",
+                                servicePrincipalName,
+                                cred.getName());
                         } catch (GSSException ex) {
                             LOG.warn("Cannot add private credential to subject; clients authentication may fail", ex);
                         }
@@ -222,17 +227,17 @@ public final class SecurityUtils {
                                     saslServer = Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, callbackHandler);
                                     return saslServer;
                                 } catch (SaslException e) {
-                                    LOG.error("Zookeeper Server failed to create a SaslServer to interact with a client during session initiation: ", e);
+                                    LOG.error("Zookeeper Server failed to create a SaslServer to interact with a client during session initiation", e);
                                     return null;
                                 }
                             }
                         });
                     } catch (PrivilegedActionException e) {
                         // TODO: exit server at this point(?)
-                        LOG.error("Zookeeper Quorum member experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context:", e);
+                        LOG.error("Zookeeper Quorum member experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context", e);
                     }
                 } catch (IndexOutOfBoundsException e) {
-                    LOG.error("server principal name/hostname determination error: ", e);
+                    LOG.error("server principal name/hostname determination error", e);
                 }
             } else {
                 // JAAS non-GSSAPI authentication: assuming and supporting only

+ 5 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/JUnit4ZKTestRunner.java

@@ -54,7 +54,10 @@ public class JUnit4ZKTestRunner extends BlockJUnit4ClassRunner {
             try {
                 list = Arrays.asList(new FrameworkMethod(klass.getMethod(methodName)));
             } catch (NoSuchMethodException nsme) {
-                LOG.warn("{} does not have test.method={}. failing to default methods.", klass.getName(), methodName);
+                LOG.warn(
+                    "{} does not have test.method={}. failing to default methods.",
+                    klass.getName(),
+                    methodName);
             }
         }
         return list;
@@ -99,7 +102,7 @@ public class JUnit4ZKTestRunner extends BlockJUnit4ClassRunner {
                     && annotation.expected().isAssignableFrom(t.getClass())) {
                     LOG.info("TEST METHOD {} THREW EXPECTED EXCEPTION {}", name, annotation.expected());
                 } else {
-                    LOG.info("TEST METHOD FAILED {}", name, t);
+                    LOG.warn("TEST METHOD FAILED {}", name, t);
                 }
                 throw t;
             }

+ 5 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/PortAssignment.java

@@ -87,8 +87,11 @@ public final class PortAssignment {
                 LOG.info("Assigned port {} from range {}.", nextPort, portRange);
                 return nextPort;
             } catch (IOException e) {
-                LOG.debug("Could not bind to port {} from range {}.  Attempting next port.",
-                          candidatePort, portRange, e);
+                LOG.debug(
+                    "Could not bind to port {} from range {}.  Attempting next port.",
+                    candidatePort,
+                    portRange,
+                    e);
             }
         }
     }

+ 53 - 53
zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java

@@ -140,10 +140,10 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         zk1.create("/node2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         MyWatcher w1 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
         MyWatcher w2 = new MyWatcher("/node2", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node2", w2));
         removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK);
         assertEquals("Didn't find data watcher", 1, zk2.getDataWatches().size());
@@ -170,10 +170,10 @@ public class RemoveWatchesTest extends ClientBase {
     public void testMultipleDataWatchers() throws IOException, InterruptedException, KeeperException {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         MyWatcher w1 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
         removeWatches(zk2, "/node1", w2, WatcherType.Data, false, Code.OK);
         assertEquals("Didn't find data watcher", 1, zk2.getDataWatches().size());
@@ -199,10 +199,10 @@ public class RemoveWatchesTest extends ClientBase {
     public void testMultipleChildWatchers() throws IOException, InterruptedException, KeeperException {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w1);
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w2);
         removeWatches(zk2, "/node1", w2, WatcherType.Children, false, Code.OK);
         assertTrue("Didn't remove child watcher", w2.matches());
@@ -234,13 +234,13 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 2);
         MyWatcher w2 = new MyWatcher("/node1", 2);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w2);
         removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK);
         removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK);
@@ -258,13 +258,13 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 1);
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w2);
         removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK);
         removeWatches(zk2, "/node1", w2, WatcherType.Data, false, Code.OK);
@@ -299,13 +299,13 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 1);
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w2);
         removeWatches(zk2, "/node1", w1, WatcherType.Children, false, Code.OK);
         removeWatches(zk2, "/node1", w2, WatcherType.Children, false, Code.OK);
@@ -339,13 +339,13 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 2);
         MyWatcher w2 = new MyWatcher("/node1", 2);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNull("Didn't set data watches", zk2.exists("/node2", w2));
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w2);
 
         // New Watcher which will be used for removal
@@ -387,12 +387,12 @@ public class RemoveWatchesTest extends ClientBase {
         MyWatcher w1 = new MyWatcher("/node1", 1);
         MyWatcher w2 = new MyWatcher("/node1", 2);
         // Add multiple data watches
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
         // Add child watch
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w2);
         removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK);
         assertTrue("Didn't remove data watcher", w1.matches());
@@ -411,12 +411,12 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 2);
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
         // Add multiple child watches
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w2);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w1);
         removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK);
         assertTrue("Didn't remove child watcher", w2.matches());
@@ -435,12 +435,12 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 2);
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
         // Add multiple child watches
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w1);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w2);
         stopServer();
         removeWatches(zk2, "/node1", w2, WatcherType.Any, true, Code.OK);
@@ -475,7 +475,7 @@ public class RemoveWatchesTest extends ClientBase {
             final String nodePath = path + i;
             w = new MyWatcher(nodePath, 1);
             wList.add(w);
-            LOG.info("Adding pre node watcher {} on path {}", new Object[]{w, nodePath});
+            LOG.info("Adding pre node watcher {} on path {}", w, nodePath);
             zk1.exists(nodePath, w);
         }
         assertEquals("Failed to add watchers!", count, zk1.getExistWatches().size());
@@ -508,7 +508,7 @@ public class RemoveWatchesTest extends ClientBase {
             String nodePath = path + i;
             w = new MyWatcher(path + i, 1);
             wList.add(w);
-            LOG.info("Adding child watcher {} on path {}", new Object[]{w, nodePath});
+            LOG.info("Adding child watcher {} on path {}", w, nodePath);
             zk1.getChildren(nodePath, w);
             nodePath += "/";
         }
@@ -538,7 +538,7 @@ public class RemoveWatchesTest extends ClientBase {
             w = new MyWatcher(path + i, 1);
             wList.add(w);
             zk1.create(nodePath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            LOG.info("Adding data watcher {} on path {}", new Object[]{w, nodePath});
+            LOG.info("Adding data watcher {} on path {}", w, nodePath);
             zk1.getData(nodePath, w, null);
             nodePath += "/";
         }
@@ -572,7 +572,7 @@ public class RemoveWatchesTest extends ClientBase {
             String nodePath = path + i;
             w = new MyWatcher(path + i, 2);
             wList.add(w);
-            LOG.info("Adding child watcher {} on path {}", new Object[]{w, nodePath});
+            LOG.info("Adding child watcher {} on path {}", w, nodePath);
             zk1.getChildren(nodePath, w);
             nodePath += "/";
         }
@@ -582,7 +582,7 @@ public class RemoveWatchesTest extends ClientBase {
         for (int i = 0; i < count; i++) {
             String nodePath = path + i;
             w = wList.get(i);
-            LOG.info("Adding data watcher {} on path {}", new Object[]{w, nodePath});
+            LOG.info("Adding data watcher {} on path {}", w, nodePath);
             zk1.getData(nodePath, w, null);
             nodePath += "/";
         }
@@ -619,12 +619,12 @@ public class RemoveWatchesTest extends ClientBase {
         zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         MyWatcher w1 = new MyWatcher("/node1", 2);
         MyWatcher w2 = new MyWatcher("/node1", 1);
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
         // Add multiple child watches
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         zk2.getChildren("/node1", w2);
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         zk2.getChildren("/node1", w1);
         removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK);
         assertTrue("Didn't remove child watcher", w1.matches());
@@ -726,9 +726,9 @@ public class RemoveWatchesTest extends ClientBase {
             }
         };
         // Add multiple data watches
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
 
         removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK);
@@ -759,9 +759,9 @@ public class RemoveWatchesTest extends ClientBase {
             }
         };
         // Add multiple child watches
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w1).size());
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w2).size());
 
         removeWatches(zk2, "/node1", w1, WatcherType.Children, false, Code.OK);
@@ -806,9 +806,9 @@ public class RemoveWatchesTest extends ClientBase {
             }
         };
         // Add multiple data watches
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
 
         assertTrue("Server session is not a watcher", isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Data));
@@ -852,9 +852,9 @@ public class RemoveWatchesTest extends ClientBase {
             }
         };
         // Add multiple child watches
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w1).size());
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w2).size());
 
         assertTrue("Server session is not a watcher", isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Children));
@@ -902,15 +902,15 @@ public class RemoveWatchesTest extends ClientBase {
             }
         };
         // Add multiple child watches
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
         assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w1).size());
-        LOG.info("Adding child watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
         assertEquals("Didn't set child watches", 0, zk2.getChildren("/node1", w2).size());
 
         // Add multiple data watches
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w1, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w1));
-        LOG.info("Adding data watcher {} on path {}", new Object[]{w2, "/node1"});
+        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
         assertNotNull("Didn't set data watches", zk2.exists("/node1", w2));
 
         assertTrue("Server session is not a watcher", isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Data));
@@ -979,7 +979,7 @@ public class RemoveWatchesTest extends ClientBase {
         }
 
         public void process(WatchedEvent event) {
-            LOG.debug("Event path : {}, eventPath : {}" + new Object[]{path, event.getPath()});
+            LOG.debug("Event path : {}, eventPath : {}", path, event.getPath());
             this.eventPath = event.getPath();
             // notifies watcher removal
             if (latch.getCount() == 0) {
@@ -1005,7 +1005,7 @@ public class RemoveWatchesTest extends ClientBase {
                 LOG.error("Failed waiting to remove the watches");
                 return false;
             }
-            LOG.debug("Client path : {} eventPath : {}", new Object[]{path, eventPath});
+            LOG.debug("Client path : {} eventPath : {}", path, eventPath);
             return path.equals(eventPath);
         }
 

+ 4 - 4
zookeeper-server/src/test/java/org/apache/zookeeper/ZKTestCase.java

@@ -58,22 +58,22 @@ public class ZKTestCase {
             // Here we enable the 4lw which ZooKeeper tests depends.
             System.setProperty("zookeeper.4lw.commands.whitelist", "*");
             testName = method.getMethodName();
-            LOG.info("STARTING " + testName);
+            LOG.info("STARTING {}", testName);
         }
 
         @Override
         public void finished(Description method) {
-            LOG.info("FINISHED " + testName);
+            LOG.info("FINISHED {}", testName);
         }
 
         @Override
         public void succeeded(Description method) {
-            LOG.info("SUCCEEDED " + testName);
+            LOG.info("SUCCEEDED {}", testName);
         }
 
         @Override
         public void failed(Throwable e, Description method) {
-            LOG.info("FAILED " + testName, e);
+            LOG.error("FAILED {}", testName, e);
         }
 
     };

+ 6 - 6
zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java

@@ -67,7 +67,7 @@ public class FileChangeWatcherTest extends ZKTestCase {
         try {
             final List<WatchEvent<?>> events = new ArrayList<>();
             watcher = new FileChangeWatcher(tempDir.toPath(), event -> {
-                LOG.info("Got an update: " + event.kind() + " " + event.context());
+                LOG.info("Got an update: {} {}", event.kind(), event.context());
                 // Filter out the extra ENTRY_CREATE events that are
                 // sometimes seen at the start. Even though we create the watcher
                 // after the file exists, sometimes we still get a create event.
@@ -83,7 +83,7 @@ public class FileChangeWatcherTest extends ZKTestCase {
             watcher.waitForState(FileChangeWatcher.State.RUNNING);
             Thread.sleep(1000L); // TODO hack
             for (int i = 0; i < 3; i++) {
-                LOG.info("Modifying file, attempt " + (i + 1));
+                LOG.info("Modifying file, attempt {}", (i + 1));
                 FileUtils.writeStringToFile(tempFile, "Hello world " + i + "\n", StandardCharsets.UTF_8, true);
                 synchronized (events) {
                     if (events.size() < i + 1) {
@@ -109,7 +109,7 @@ public class FileChangeWatcherTest extends ZKTestCase {
         try {
             final List<WatchEvent<?>> events = new ArrayList<>();
             watcher = new FileChangeWatcher(tempDir.toPath(), event -> {
-                LOG.info("Got an update: " + event.kind() + " " + event.context());
+                LOG.info("Got an update: {} {}", event.kind(), event.context());
                 // Filter out the extra ENTRY_CREATE events that are
                 // sometimes seen at the start. Even though we create the watcher
                 // after the file exists, sometimes we still get a create event.
@@ -149,7 +149,7 @@ public class FileChangeWatcherTest extends ZKTestCase {
         try {
             final List<WatchEvent<?>> events = new ArrayList<>();
             watcher = new FileChangeWatcher(tempDir.toPath(), event -> {
-                LOG.info("Got an update: " + event.kind() + " " + event.context());
+                LOG.info("Got an update: {} {}", event.kind(), event.context());
                 synchronized (events) {
                     events.add(event);
                     events.notifyAll();
@@ -183,7 +183,7 @@ public class FileChangeWatcherTest extends ZKTestCase {
         try {
             final List<WatchEvent<?>> events = new ArrayList<>();
             watcher = new FileChangeWatcher(tempDir.toPath(), event -> {
-                LOG.info("Got an update: " + event.kind() + " " + event.context());
+                LOG.info("Got an update: {} {}", event.kind(), event.context());
                 // Filter out the extra ENTRY_CREATE events that are
                 // sometimes seen at the start. Even though we create the watcher
                 // after the file exists, sometimes we still get a create event.
@@ -222,7 +222,7 @@ public class FileChangeWatcherTest extends ZKTestCase {
         try {
             final AtomicInteger callCount = new AtomicInteger(0);
             watcher = new FileChangeWatcher(tempDir.toPath(), event -> {
-                LOG.info("Got an update: " + event.kind() + " " + event.context());
+                LOG.info("Got an update: {} {}", event.kind(), event.context());
                 int oldValue;
                 synchronized (callCount) {
                     oldValue = callCount.getAndIncrement();

+ 5 - 3
zookeeper-server/src/test/java/org/apache/zookeeper/server/CRCTest.java

@@ -20,6 +20,7 @@ package org.apache.zookeeper.server;
 
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import java.io.BufferedInputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -130,7 +131,7 @@ public class CRCTest extends ZKTestCase {
         File snapFile = null;
         File logFile = null;
         for (File file : list) {
-            LOG.info("file is " + file);
+            LOG.info("file is {}", file);
             if (file.getName().startsWith("log")) {
                 logFile = file;
                 corruptFile(logFile);
@@ -141,10 +142,11 @@ public class CRCTest extends ZKTestCase {
         //we will get a checksum failure
         try {
             while (itr.next()) {
+                // no op
             }
-            assertTrue(false);
+            fail();
         } catch (IOException ie) {
-            LOG.info("crc corruption", ie);
+            LOG.warn("crc corruption", ie);
         }
         itr.close();
         // find the last snapshot

+ 8 - 12
zookeeper-server/src/test/java/org/apache/zookeeper/server/DeserializationPerfTest.java

@@ -62,18 +62,14 @@ public class DeserializationPerfTest extends ZKTestCase {
 
         assertEquals(count, dserTree.getNodeCount());
 
-        LOG.info("Deserialized "
-                         + count
-                         + " nodes in "
-                         + durationms
-                         + " ms ("
-                         + pernodeus
-                         + "us/node), depth="
-                         + depth
-                         + " width="
-                         + width
-                         + " datalen="
-                         + len);
+        LOG.info(
+            "Deserialized {} nodes in {} ms ({}us/node), depth={} width={} datalen={}",
+            count,
+            durationms,
+            pernodeus,
+            depth,
+            width,
+            len);
     }
 
     @Test

+ 1 - 1
zookeeper-server/src/test/java/org/apache/zookeeper/server/InvalidSnapshotTest.java

@@ -61,7 +61,7 @@ public class InvalidSnapshotTest extends ClientBase {
 
         // now corrupt the snapshot
         File snapFile = factory.zkServer.getTxnLogFactory().findMostRecentSnapshot();
-        LOG.info("Corrupting " + snapFile);
+        LOG.info("Corrupting {}", snapFile);
         RandomAccessFile raf = new RandomAccessFile(snapFile, "rws");
         raf.setLength(3);
         raf.close();

+ 2 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java

@@ -529,7 +529,7 @@ public class PurgeTxnTest extends ZKTestCase {
                             znodes.add(mynode);
                             zk.create(mynode, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
                         } catch (Exception e) {
-                            LOG.error("Unexpected exception occurred!", e);
+                            LOG.error("Unexpected exception", e);
                         }
                         if (i == 200) {
                             doPurge.countDown();
@@ -547,7 +547,7 @@ public class PurgeTxnTest extends ZKTestCase {
         try {
             assertTrue("ZkClient ops is not finished!", finished.await(OP_TIMEOUT_IN_MILLIS, TimeUnit.MILLISECONDS));
         } catch (InterruptedException ie) {
-            LOG.error("Unexpected exception occurred!", ie);
+            LOG.error("Unexpected exception", ie);
             fail("Unexpected exception occurred!");
         }
         return znodes;

+ 8 - 6
zookeeper-server/src/test/java/org/apache/zookeeper/server/SerializationPerfTest.java

@@ -69,12 +69,14 @@ public class SerializationPerfTest extends ZKTestCase {
         long end = System.nanoTime();
         long durationms = (end - start) / 1000000L;
         long pernodeus = ((end - start) / 1000L) / count;
-        LOG.info("Serialized "
-                 + count + " nodes in "
-                 + durationms + " ms ("
-                 + pernodeus + "us/node), depth=" + depth
-                 + " width=" + width
-                 + " datalen=" + len);
+        LOG.info(
+            "Serialized {} nodes in {} ms ({}us/node), depth={} width={} datalen={}",
+            count,
+            durationms,
+            pernodeus,
+            depth,
+            width,
+            len);
     }
 
     @Test

+ 1 - 1
zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerStartupTest.java

@@ -161,7 +161,7 @@ public class ZooKeeperServerStartupTest extends ZKTestCase {
 
     private void verify(String cmd, String expected) throws IOException, SSLContextException {
         String resp = sendRequest(cmd);
-        LOG.info("cmd " + cmd + " expected " + expected + " got " + resp);
+        LOG.info("cmd {} expected {} got {}", cmd, expected, resp);
         assertTrue("Unexpected response", resp.contains(expected));
     }
 

+ 2 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java

@@ -86,7 +86,7 @@ public class JettyAdminServerTest extends ZKTestCase {
                 "zookeeper.ssl.quorum.trustStore.location",
                 x509TestContext.getTrustStoreFile(KeyStoreFileType.PEM).getAbsolutePath());
         } catch (Exception e) {
-            LOG.info("Problems encountered while setting up encryption for Jetty admin server test: " + e);
+            LOG.info("Problems encountered while setting up encryption for Jetty admin server test", e);
         }
         System.setProperty("zookeeper.ssl.quorum.keyStore.password", "");
         System.setProperty("zookeeper.ssl.quorum.keyStore.type", "PEM");
@@ -111,7 +111,7 @@ public class JettyAdminServerTest extends ZKTestCase {
             sc = SSLContext.getInstance("SSL");
             sc.init(null, trustAllCerts, new java.security.SecureRandom());
         } catch (Exception e) {
-            LOG.error("Failed to customize encryption for HTTPS: e");
+            LOG.error("Failed to customize encryption for HTTPS", e);
         }
 
         // Create all-trusting hostname verifier

+ 4 - 4
zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnLogTest.java

@@ -184,14 +184,14 @@ public class FileTxnLogTest extends ZKTestCase {
         int zxid = 1;
         for (int i = 0; i < 4; i++) {
             log.append(new TxnHeader(0, 0, zxid++, 0, 0), record);
-            LOG.debug("Current log size: " + log.getCurrentLogSize());
+            LOG.debug("Current log size: {}", log.getCurrentLogSize());
         }
         log.commit();
-        LOG.info("Current log size: " + log.getCurrentLogSize());
+        LOG.info("Current log size: {}", log.getCurrentLogSize());
         assertTrue(log.getCurrentLogSize() > (zxid - 1) * NODE_SIZE);
         for (int i = 0; i < 4; i++) {
             log.append(new TxnHeader(0, 0, zxid++, 0, 0), record);
-            LOG.debug("Current log size: " + log.getCurrentLogSize());
+            LOG.debug("Current log size: {}", log.getCurrentLogSize());
         }
         log.commit();
         LOG.info("Current log size: " + log.getCurrentLogSize());
@@ -229,7 +229,7 @@ public class FileTxnLogTest extends ZKTestCase {
         // We will create enough txn to generate 3 logs
         long txnCount = LOG_SIZE_LIMIT / NODE_SIZE / 2 * 5;
 
-        LOG.info("Creating " + txnCount + " txns");
+        LOG.info("Creating {} txns", txnCount);
 
         try {
             for (long i = 0; i < txnCount; i++) {

+ 5 - 5
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java

@@ -201,7 +201,7 @@ public class CnxManagerTest extends ZKTestCase {
         int deadPort = PortAssignment.unique();
         String deadAddress = "10.1.1." + b;
 
-        LOG.info("This is the dead address I'm trying: " + deadAddress);
+        LOG.info("This is the dead address I'm trying: {}", deadAddress);
 
         peers.put(2L, new QuorumServer(2, new InetSocketAddress(deadAddress, deadPort), new InetSocketAddress(deadAddress, PortAssignment.unique()), new InetSocketAddress(deadAddress, PortAssignment.unique())));
         peerTmpdir[2] = ClientBase.createTmpDir();
@@ -246,14 +246,14 @@ public class CnxManagerTest extends ZKTestCase {
         }
 
         int port = peers.get(peer.getId()).electionAddr.getPort();
-        LOG.info("Election port: " + port);
+        LOG.info("Election port: {}", port);
 
         Thread.sleep(1000);
 
         SocketChannel sc = SocketChannel.open();
         sc.socket().connect(peers.get(1L).electionAddr, 5000);
 
-        InetSocketAddress otherAddr = peers.get(Long.valueOf(2)).electionAddr;
+        InetSocketAddress otherAddr = peers.get(2L).electionAddr;
         DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream());
         dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION);
         dout.writeLong(2);
@@ -336,7 +336,7 @@ public class CnxManagerTest extends ZKTestCase {
             LOG.error("Null listener when initializing cnx manager");
         }
         int port = peers.get(peer.getId()).electionAddr.getPort();
-        LOG.info("Election port: " + port);
+        LOG.info("Election port: {}", port);
 
         Thread.sleep(1000);
 
@@ -383,7 +383,7 @@ public class CnxManagerTest extends ZKTestCase {
             LOG.error("Null listener when initializing cnx manager");
         }
         int port = peers.get(peer.getId()).electionAddr.getPort();
-        LOG.info("Election port: " + port);
+        LOG.info("Election port: {}", port);
         Thread.sleep(1000);
 
         Socket sock = new Socket();

+ 2 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorConcurrencyTest.java

@@ -207,7 +207,7 @@ public class CommitProcessorConcurrencyTest extends ZKTestCase {
         Thread.sleep(1000);
         shouldBeProcessed.removeAll(processedRequests);
         for (Request r : shouldBeProcessed) {
-            LOG.error("Did not process " + r);
+            LOG.error("Did not process {}", r);
         }
         assertTrue("Not all requests were processed", shouldBeProcessed.isEmpty());
         assertFalse("Processed a wrong request", shouldNotBeProcessed.removeAll(processedRequests));
@@ -251,7 +251,7 @@ public class CommitProcessorConcurrencyTest extends ZKTestCase {
 
         shouldBeInPending.removeAll(processor.pendingRequests.get(writeReq.sessionId));
         for (Request r : shouldBeInPending) {
-            LOG.error("Should be in pending " + r);
+            LOG.error("Should be in pending {}", r);
         }
         assertTrue("Not all requests moved to pending from queuedRequests", shouldBeInPending.isEmpty());
 

+ 1 - 3
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java

@@ -466,9 +466,7 @@ public class CommitProcessorTest extends ZKTestCase {
         }
 
         private void validateRequest(Request request) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Got request " + request);
-            }
+            LOG.debug("Got request {}", request);
 
             // Zxids should always be in order for write requests
             if (request.getHdr() != null) {

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác