You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2021/11/18 21:48:46 UTC

[cassandra] branch trunk updated: Log queries that fail on timeout or unavailable errors up to once per minute by default

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

maedhroz pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 33fd2dc  Log queries that fail on timeout or unavailable errors up to once per minute by default
33fd2dc is described below

commit 33fd2dc817f56116af0b4b7d2cd5365ce52948d9
Author: Caleb Rackliffe <ca...@gmail.com>
AuthorDate: Wed Nov 17 16:43:00 2021 -0600

    Log queries that fail on timeout or unavailable errors up to once per minute by default
    
    patch by Caleb Rackliffe and Marcus Eriksson; reviewed by David Capwell and Yifan Cai for CASSANDRA-17159
    
    Co-authored-by: Caleb Rackliffe <ca...@gmail.com>
    Co-authored-by: Marcus Eriksson <ma...@apache.org>
---
 CHANGES.txt                                        |   1 +
 .../config/CassandraRelevantProperties.java        |   1 +
 .../org/apache/cassandra/service/StorageProxy.java |  24 +-
 .../service/reads/range/RangeCommandIterator.java  |   6 +-
 .../org/apache/cassandra/utils/NoSpamLogger.java   |  48 ++-
 .../distributed/test/FailureLoggingTest.java       | 177 +++++++++++
 .../apache/cassandra/utils/NoSpamLoggerTest.java   | 350 +++++++++++----------
 7 files changed, 431 insertions(+), 176 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index f533407..676e92b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Log queries that fail on timeout or unavailable errors up to once per minute by default (CASSANDRA-17159)
  * Refactor normal/preview/IR repair to standardize repair cleanup and error handling of failed RepairJobs (CASSANDRA-17069)
  * Log missing peers in StartupClusterConnectivityChecker (CASSANDRA-17130)
  * Introduce separate rate limiting settings for entire SSTable streaming (CASSANDRA-17065)
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index 9b247ca..43db1c3 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -233,6 +233,7 @@ public enum CassandraRelevantProperties
     DETERMINISM_SSTABLE_COMPRESSION_DEFAULT("cassandra.sstable_compression_default", "true"),
     DETERMINISM_CONSISTENT_DIRECTORY_LISTINGS("cassandra.consistent_directory_listings", "false"),
     DETERMINISM_UNSAFE_UUID_NODE("cassandra.unsafe.deterministicuuidnode", "false"),
+    FAILURE_LOGGING_INTERVAL_SECONDS("cassandra.request_failure_log_interval_seconds", "60"),
 
     // properties to disable certain behaviours for testing
     DISABLE_GOSSIP_ENDPOINT_REMOVAL("cassandra.gossip.disable_endpoint_removal"),
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 6f83b20..06c9cbd 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -37,6 +37,7 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheLoader;
@@ -49,6 +50,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.batchlog.Batch;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -130,6 +132,7 @@ import org.apache.cassandra.utils.Clock;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MBeanWrapper;
 import org.apache.cassandra.utils.MonotonicClock;
+import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.CountDownLatch;
@@ -163,6 +166,8 @@ public class StorageProxy implements StorageProxyMBean
 
     public static final String UNREACHABLE = "UNREACHABLE";
 
+    private static final int FAILURE_LOGGING_INTERVAL_SECONDS = CassandraRelevantProperties.FAILURE_LOGGING_INTERVAL_SECONDS.getInt();
+
     private static final WritePerformer standardWritePerformer;
     private static final WritePerformer counterWritePerformer;
     private static final WritePerformer counterWriteOnCoordinatorPerformer;
@@ -1758,7 +1763,9 @@ public class StorageProxy implements StorageProxyMBean
         {
             readMetrics.unavailables.mark();
             readMetricsForLevel(consistencyLevel).unavailables.mark();
-            throw new IsBootstrappingException();
+            IsBootstrappingException exception = new IsBootstrappingException();
+            logRequestException(exception, group.queries);
+            throw exception;
         }
 
         if (DatabaseDescriptor.getEnablePartitionDenylist() && DatabaseDescriptor.getEnableDenylistReads())
@@ -1838,6 +1845,7 @@ public class StorageProxy implements StorageProxyMBean
             readMetrics.unavailables.mark();
             casReadMetrics.unavailables.mark();
             readMetricsForLevel(consistencyLevel).unavailables.mark();
+            logRequestException(e, group.queries);
             throw e;
         }
         catch (ReadTimeoutException e)
@@ -1845,6 +1853,7 @@ public class StorageProxy implements StorageProxyMBean
             readMetrics.timeouts.mark();
             casReadMetrics.timeouts.mark();
             readMetricsForLevel(consistencyLevel).timeouts.mark();
+            logRequestException(e, group.queries);
             throw e;
         }
         catch (ReadAbortException e)
@@ -1894,12 +1903,14 @@ public class StorageProxy implements StorageProxyMBean
         {
             readMetrics.unavailables.mark();
             readMetricsForLevel(consistencyLevel).unavailables.mark();
+            logRequestException(e, group.queries);
             throw e;
         }
         catch (ReadTimeoutException e)
         {
             readMetrics.timeouts.mark();
             readMetricsForLevel(consistencyLevel).timeouts.mark();
+            logRequestException(e, group.queries);
             throw e;
         }
         catch (ReadAbortException e)
@@ -2455,6 +2466,17 @@ public class StorageProxy implements StorageProxyMBean
         abstract protected void runMayThrow() throws Exception;
     }
 
+    public static void logRequestException(Exception exception, Collection<? extends ReadCommand> commands)
+    {
+        NoSpamLogger.log(logger, NoSpamLogger.Level.INFO, FAILURE_LOGGING_INTERVAL_SECONDS, TimeUnit.SECONDS,
+                         "\"{}\" while executing {}",
+                         () -> new Object[]
+                               {
+                                   exception.getMessage(),
+                                   commands.stream().map(ReadCommand::toCQLString).collect(Collectors.joining("; "))
+                               });
+    }
+
     /**
      * HintRunnable will decrease totalHintsInProgress and targetHints when finished.
      * It is the caller's responsibility to increment them initially.
diff --git a/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java b/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
index 60309bd..b353b4b 100644
--- a/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
+++ b/src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.service.reads.range;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
@@ -54,7 +55,8 @@ import org.apache.cassandra.utils.CloseableIterator;
 
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
-class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
+@VisibleForTesting
+public class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeCommandIterator.class);
 
@@ -125,11 +127,13 @@ class RangeCommandIterator extends AbstractIterator<RowIterator> implements Part
         catch (UnavailableException e)
         {
             rangeMetrics.unavailables.mark();
+            StorageProxy.logRequestException(e, Collections.singleton(command));
             throw e;
         }
         catch (ReadTimeoutException e)
         {
             rangeMetrics.timeouts.mark();
+            StorageProxy.logRequestException(e, Collections.singleton(command));
             throw e;
         }
         catch (ReadAbortException e)
diff --git a/src/java/org/apache/cassandra/utils/NoSpamLogger.java b/src/java/org/apache/cassandra/utils/NoSpamLogger.java
index 5a05a47..9b62e21 100644
--- a/src/java/org/apache/cassandra/utils/NoSpamLogger.java
+++ b/src/java/org/apache/cassandra/utils/NoSpamLogger.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.utils;
 
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
 
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.slf4j.Logger;
@@ -46,7 +47,7 @@ public class NoSpamLogger
      */
     public enum Level
     {
-        INFO, WARN, ERROR;
+        INFO, WARN, ERROR
     }
 
     @VisibleForTesting
@@ -84,21 +85,31 @@ public class NoSpamLogger
             return nowNanos >= expected && compareAndSet(expected, nowNanos + minIntervalNanos);
         }
 
+        public boolean log(Level l, long nowNanos, Supplier<Object[]> objects)
+        {
+            if (!shouldLog(nowNanos)) return false;
+            return logNoCheck(l, objects.get());
+        }
+
         public boolean log(Level l, long nowNanos, Object... objects)
         {
             if (!shouldLog(nowNanos)) return false;
+            return logNoCheck(l, objects);
+        }
 
+        private boolean logNoCheck(Level l, Object... objects)
+        {
             switch (l)
             {
-            case INFO:
-                wrapped.info(statement, objects);
-                break;
-            case WARN:
-                wrapped.warn(statement, objects);
-                break;
-            case ERROR:
-                wrapped.error(statement, objects);
-                break;
+                case INFO:
+                    wrapped.info(statement, objects);
+                    break;
+                case WARN:
+                    wrapped.warn(statement, objects);
+                    break;
+                case ERROR:
+                    wrapped.error(statement, objects);
+                    break;
                 default:
                     throw new AssertionError();
             }
@@ -174,6 +185,23 @@ public class NoSpamLogger
         return statement.log(level, nowNanos, objects);
     }
 
+    public static boolean log(Logger logger, Level level, long minInterval, TimeUnit unit, String message, Supplier<Object[]> objects)
+    {
+        return log(logger, level, message, minInterval, unit, CLOCK.nanoTime(), message, objects);
+    }
+
+    public static boolean log(Logger logger, Level level, String key, long minInterval, TimeUnit unit, String message, Supplier<Object[]> objects)
+    {
+        return log(logger, level, key, minInterval, unit, CLOCK.nanoTime(), message, objects);
+    }
+
+    public static boolean log(Logger logger, Level level, String key, long minInterval, TimeUnit unit, long nowNanos, String message, Supplier<Object[]> objects)
+    {
+        NoSpamLogger wrapped = getLogger(logger, minInterval, unit);
+        NoSpamLogStatement statement = wrapped.getStatement(key, message);
+        return statement.log(level, nowNanos, objects);
+    }
+
     public static NoSpamLogStatement getStatement(Logger logger, String message, long minInterval, TimeUnit unit)
     {
         NoSpamLogger wrapped = getLogger(logger, minInterval, unit);
diff --git a/test/distributed/org/apache/cassandra/distributed/test/FailureLoggingTest.java b/test/distributed/org/apache/cassandra/distributed/test/FailureLoggingTest.java
new file mode 100644
index 0000000..ff8234d
--- /dev/null
+++ b/test/distributed/org/apache/cassandra/distributed/test/FailureLoggingTest.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.util.List;
+
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.LogResult;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.reads.range.RangeCommandIterator;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class FailureLoggingTest extends TestBaseImpl
+{
+    private static Cluster cluster;
+    
+    @BeforeClass
+    public static void setUpCluster() throws IOException
+    {
+        CassandraRelevantProperties.FAILURE_LOGGING_INTERVAL_SECONDS.setInt(0);
+        cluster = init(Cluster.build(1).withInstanceInitializer(BBRequestFailures::install).start());
+        cluster.schemaChange("create table "+KEYSPACE+".tbl (id int primary key, i int)");
+    }
+
+    @AfterClass
+    public static void tearDownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void resetBootstrappingState()
+    {
+        cluster.get(1).callOnInstance(() -> BBRequestFailures.bootstrapping = false);
+        
+    }
+
+    @Test
+    public void testRequestBootstrapFail() throws Throwable
+    {
+        cluster.get(1).callOnInstance(() -> BBRequestFailures.bootstrapping = true);
+        long mark = cluster.get(1).logs().mark();
+
+        try
+        {
+            cluster.coordinator(1).execute("select * from " + KEYSPACE + ".tbl where id = 55", ConsistencyLevel.ALL);
+            fail("Query should fail");
+        }
+        catch (RuntimeException e)
+        {
+            LogResult<List<String>> result = cluster.get(1).logs().grep(mark, "while executing SELECT");
+            assertEquals(1, result.getResult().size());
+            assertTrue(result.getResult().get(0).contains("Cannot read from a bootstrapping node"));
+        }
+    }
+
+    @Test
+    public void testRangeRequestFail() throws Throwable
+    {
+        long mark = cluster.get(1).logs().mark();
+
+        try
+        {
+            cluster.coordinator(1).execute("select * from " + KEYSPACE + ".tbl", ConsistencyLevel.ALL);
+            fail("Query should fail");
+        }
+        catch (RuntimeException e)
+        {
+            LogResult<List<String>> result = cluster.get(1).logs().grep(mark, "while executing SELECT");
+            assertEquals(1, result.getResult().size());
+            assertTrue(result.getResult().get(0).contains("Cannot achieve consistency level"));
+        }
+    }
+
+    @Test
+    public void testReadRequestFail() throws Throwable
+    {
+        long mark = cluster.get(1).logs().mark();
+
+        try
+        {
+            cluster.coordinator(1).execute("select * from " + KEYSPACE + ".tbl where id = 55", ConsistencyLevel.ALL);
+            fail("Query should fail");
+        }
+        catch (RuntimeException e)
+        {
+            LogResult<List<String>> result = cluster.get(1).logs().grep(mark, "while executing SELECT");
+            assertEquals(1, result.getResult().size());
+            assertTrue(result.getResult().get(0).contains("Cannot achieve consistency level"));
+        }
+    }
+
+    public static class BBRequestFailures
+    {
+        static volatile boolean bootstrapping = false;
+        
+        static void install(ClassLoader cl, int nodeNumber)
+        {
+            ByteBuddy bb = new ByteBuddy();
+            
+            bb.redefine(StorageService.class)
+              .method(named("isBootstrapMode"))
+              .intercept(MethodDelegation.to(BBRequestFailures.class))
+              .make()
+              .load(cl, ClassLoadingStrategy.Default.INJECTION);
+
+            bb.redefine(RangeCommandIterator.class)
+              .method(named("sendNextRequests"))
+              .intercept(MethodDelegation.to(BBRequestFailures.class))
+              .make()
+              .load(cl, ClassLoadingStrategy.Default.INJECTION);
+
+            bb.redefine(StorageProxy.class)
+              .method(named("fetchRows"))
+              .intercept(MethodDelegation.to(BBRequestFailures.class))
+              .make()
+              .load(cl, ClassLoadingStrategy.Default.INJECTION);
+        }
+
+        @SuppressWarnings("unused")
+        public static boolean isBootstrapMode()
+        {
+            return bootstrapping;
+        }
+
+        @SuppressWarnings("unused")
+        public static PartitionIterator sendNextRequests()
+        {
+            throw UnavailableException.create(org.apache.cassandra.db.ConsistencyLevel.ALL, 1, 0);
+        }
+
+        @SuppressWarnings("unused")
+        public static PartitionIterator fetchRows(List<SinglePartitionReadCommand> commands, 
+                                                  org.apache.cassandra.db.ConsistencyLevel consistencyLevel, 
+                                                  long queryStartNanoTime)
+        {
+            throw UnavailableException.create(org.apache.cassandra.db.ConsistencyLevel.ALL, 1, 0);
+        }
+    }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
index 58e6ea0..73aef09 100644
--- a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
+++ b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
@@ -18,13 +18,13 @@
 */
 package org.apache.cassandra.utils;
 
-import static org.junit.Assert.*;
-
 import java.util.ArrayDeque;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
 
 import org.apache.cassandra.utils.NoSpamLogger.Level;
 import org.apache.cassandra.utils.NoSpamLogger.NoSpamLogStatement;
@@ -34,146 +34,138 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.helpers.SubstituteLogger;
 
+import static org.junit.Assert.*;
 
 public class NoSpamLoggerTest
 {
     Map<Level, Queue<Pair<String, Object[]>>> logged = new HashMap<>();
 
-   Logger mock = new SubstituteLogger(null, null, true)
-   {
-
-       @Override
-       public void info(String statement, Object... args)
-       {
-           logged.get(Level.INFO).offer(Pair.create(statement, args));
-       }
-
-       @Override
-       public void warn(String statement, Object... args)
-       {
-           logged.get(Level.WARN).offer(Pair.create(statement, args));
-       }
-
-       @Override
-       public void error(String statement, Object... args)
-       {
-           logged.get(Level.ERROR).offer(Pair.create(statement, args));
-       }
-
-       @Override
-       public int hashCode()
-       {
-           return 42;//It's a valid hash code
-       }
-
-       @Override
-       public boolean equals(Object o)
-       {
-           return this == o;
-       }
-   };
-
-
-   static final String statement = "swizzle{}";
-   static final String param = "";
-   static long now;
-
-   @BeforeClass
-   public static void setUpClass() throws Exception
-   {
-       NoSpamLogger.CLOCK = new NoSpamLogger.Clock()
-       {
+    Logger mock = new SubstituteLogger(null, null, true)
+    {
         @Override
-        public long nanoTime()
+        public void info(String statement, Object... args)
         {
-            return now;
+            logged.get(Level.INFO).offer(Pair.create(statement, args));
         }
-       };
-   }
 
-   @Before
-   public void setUp() throws Exception
-   {
-       logged.put(Level.INFO, new ArrayDeque<Pair<String, Object[]>>());
-       logged.put(Level.WARN, new ArrayDeque<Pair<String, Object[]>>());
-       logged.put(Level.ERROR, new ArrayDeque<Pair<String, Object[]>>());
-       NoSpamLogger.clearWrappedLoggersForTest();
-   }
+        @Override
+        public void warn(String statement, Object... args)
+        {
+            logged.get(Level.WARN).offer(Pair.create(statement, args));
+        }
 
-   @Test
-   public void testNoSpamLogger() throws Exception
-   {
-       testLevel(Level.INFO);
-       testLevel(Level.WARN);
-       testLevel(Level.ERROR);
-   }
+        @Override
+        public void error(String statement, Object... args)
+        {
+            logged.get(Level.ERROR).offer(Pair.create(statement, args));
+        }
 
-   private void testLevel(Level l) throws Exception
-   {
-       setUp();
-       now = 5;
+        @Override
+        public int hashCode()
+        {
+            return 42; //It's a valid hash code
+        }
 
-       assertTrue(NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param));
+        @Override
+        public boolean equals(Object o)
+        {
+            return this == o;
+        }
+    };
 
-       assertEquals(1, logged.get(l).size());
+    static final String statement = "swizzle{}";
+    static final String param = "";
+    static long now;
 
-       assertFalse(NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param));
+    @BeforeClass
+    public static void setUpClass() throws Exception
+    {
+        NoSpamLogger.CLOCK = () -> now;
+    }
 
-       assertEquals(1, logged.get(l).size());
+    @Before
+    public void setUp() throws Exception
+    {
+        logged.put(Level.INFO, new ArrayDeque<Pair<String, Object[]>>());
+        logged.put(Level.WARN, new ArrayDeque<Pair<String, Object[]>>());
+        logged.put(Level.ERROR, new ArrayDeque<Pair<String, Object[]>>());
+        NoSpamLogger.clearWrappedLoggersForTest();
+    }
 
-       now += 5;
+    @Test
+    public void testNoSpamLogger() throws Exception
+    {
+        testLevel(Level.INFO);
+        testLevel(Level.WARN);
+        testLevel(Level.ERROR);
+    }
 
-       assertTrue(NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param));
+    private void testLevel(Level l) throws Exception
+    {
+        setUp();
+        now = 5;
 
-       assertEquals(2, logged.get(l).size());
+        assertTrue(NoSpamLogger.log(mock, l, 5, TimeUnit.NANOSECONDS, statement, param));
 
-       assertTrue(NoSpamLogger.log( mock, l, "key", 5,  TimeUnit.NANOSECONDS, statement, param));
+        assertEquals(1, logged.get(l).size());
 
-       assertEquals(3, logged.get(l).size());
+        assertFalse(NoSpamLogger.log(mock, l, 5, TimeUnit.NANOSECONDS, statement, param));
 
-       assertFalse(NoSpamLogger.log( mock, l, "key", 5,  TimeUnit.NANOSECONDS, statement, param));
+        assertEquals(1, logged.get(l).size());
 
-       assertEquals(3, logged.get(l).size());
-   }
+        now += 5;
 
-   private void assertLoggedSizes(int info, int warn, int error)
-   {
-       assertEquals(info, logged.get(Level.INFO).size());
-       assertEquals(warn, logged.get(Level.WARN).size());
-       assertEquals(error, logged.get(Level.ERROR).size());
-   }
+        assertTrue(NoSpamLogger.log(mock, l, 5, TimeUnit.NANOSECONDS, statement, param));
 
-   @Test
-   public void testNoSpamLoggerDirect() throws Exception
-   {
-       now = 5;
-       NoSpamLogger logger = NoSpamLogger.getLogger( mock, 5, TimeUnit.NANOSECONDS);
+        assertEquals(2, logged.get(l).size());
 
-       assertTrue(logger.info(statement, param));
-       assertFalse(logger.info(statement, param));
-       assertFalse(logger.warn(statement, param));
-       assertFalse(logger.error(statement, param));
+        assertTrue(NoSpamLogger.log(mock, l, "key", 5, TimeUnit.NANOSECONDS, statement, param));
 
-       assertLoggedSizes(1, 0, 0);
+        assertEquals(3, logged.get(l).size());
 
-       NoSpamLogStatement statement = logger.getStatement("swizzle2{}", 10, TimeUnit.NANOSECONDS);
-       assertTrue(statement.warn(param)); // since a statement of this key hasn't logged yet
-       assertLoggedSizes(1, 1, 0);
+        assertFalse(NoSpamLogger.log(mock, l, "key", 5, TimeUnit.NANOSECONDS, statement, param));
 
-       now = 10;
-       assertFalse(statement.warn(param)); // we logged it above
-       assertLoggedSizes(1, 1, 0);
+        assertEquals(3, logged.get(l).size());
+    }
 
-       now = 15;
-       assertTrue(statement.warn(param)); // First log was at 5, now past the interval
-       assertLoggedSizes(1, 2, 0);
-   }
+    private void assertLoggedSizes(int info, int warn, int error)
+    {
+        assertEquals(info, logged.get(Level.INFO).size());
+        assertEquals(warn, logged.get(Level.WARN).size());
+        assertEquals(error, logged.get(Level.ERROR).size());
+    }
+
+    @Test
+    public void testNoSpamLoggerDirect() throws Exception
+    {
+        now = 5;
+        NoSpamLogger logger = NoSpamLogger.getLogger(mock, 5, TimeUnit.NANOSECONDS);
+
+        assertTrue(logger.info(statement, param));
+        assertFalse(logger.info(statement, param));
+        assertFalse(logger.warn(statement, param));
+        assertFalse(logger.error(statement, param));
+
+        assertLoggedSizes(1, 0, 0);
+
+        NoSpamLogStatement statement = logger.getStatement("swizzle2{}", 10, TimeUnit.NANOSECONDS);
+        assertTrue(statement.warn(param)); // since a statement of this key hasn't logged yet
+        assertLoggedSizes(1, 1, 0);
+
+        now = 10;
+        assertFalse(statement.warn(param)); // we logged it above
+        assertLoggedSizes(1, 1, 0);
+
+        now = 15;
+        assertTrue(statement.warn(param)); // First log was at 5, now past the interval
+        assertLoggedSizes(1, 2, 0);
+    }
 
     @Test
     public void testNegativeNowNanos() throws Exception
     {
         now = -6;
-        NoSpamLogger logger = NoSpamLogger.getLogger( mock, 5, TimeUnit.NANOSECONDS);
+        NoSpamLogger logger = NoSpamLogger.getLogger(mock, 5, TimeUnit.NANOSECONDS);
 
         assertTrue(logger.info(statement, param));
         assertFalse(logger.info(statement, param));
@@ -204,84 +196,114 @@ public class NoSpamLoggerTest
     }
 
     @Test
-   public void testNoSpamLoggerStatementDirect() throws Exception
-   {
-       NoSpamLogger.NoSpamLogStatement nospam = NoSpamLogger.getStatement( mock, statement, 5, TimeUnit.NANOSECONDS);
+    public void testNoSpamLoggerStatementDirect()
+    {
+        NoSpamLogger.NoSpamLogStatement nospam = NoSpamLogger.getStatement(mock, statement, 5, TimeUnit.NANOSECONDS);
 
-       now = 5;
+        now = 5;
 
-       assertTrue(nospam.info(statement, param));
-       assertFalse(nospam.info(statement, param));
-       assertFalse(nospam.warn(statement, param));
-       assertFalse(nospam.error(statement, param));
+        assertTrue(nospam.info(statement, param));
+        assertFalse(nospam.info(statement, param));
+        assertFalse(nospam.warn(statement, param));
+        assertFalse(nospam.error(statement, param));
 
-       assertLoggedSizes(1, 0, 0);
-   }
+        assertLoggedSizes(1, 0, 0);
+    }
 
-   private void checkMock(Level l)
-   {
-       Pair<String, Object[]> p = logged.get(l).poll();
-       assertNotNull(p);
-       assertEquals(statement, p.left);
-       Object objs[] = p.right;
-       assertEquals(1, objs.length);
-       assertEquals(param, objs[0]);
-       assertTrue(logged.get(l).isEmpty());
-   }
+    private void checkMock(Level l)
+    {
+        Pair<String, Object[]> p = logged.get(l).poll();
+        assertNotNull(p);
+        assertEquals(statement, p.left);
+        Object[] objs = p.right;
+        assertEquals(1, objs.length);
+        assertEquals(param, objs[0]);
+        assertTrue(logged.get(l).isEmpty());
+    }
 
-   /*
-    * Make sure that what is passed to the underlying logger is the correct set of objects
-    */
-   @Test
-   public void testLoggedResult() throws Exception
-   {
-       now = 5;
+    /*
+     * Make sure that what is passed to the underlying logger is the correct set of objects
+     */
+    @Test
+    public void testLoggedResult()
+    {
+        now = 5;
 
-       assertTrue(NoSpamLogger.log( mock, Level.INFO, 5,  TimeUnit.NANOSECONDS, statement, param));
-       checkMock(Level.INFO);
+        assertTrue(NoSpamLogger.log(mock, Level.INFO, 5, TimeUnit.NANOSECONDS, statement, param));
+        checkMock(Level.INFO);
 
-       now = 10;
+        now = 10;
 
-       assertTrue(NoSpamLogger.log( mock, Level.WARN, 5,  TimeUnit.NANOSECONDS, statement, param));
-       checkMock(Level.WARN);
+        assertTrue(NoSpamLogger.log(mock, Level.WARN, 5, TimeUnit.NANOSECONDS, statement, param));
+        checkMock(Level.WARN);
 
-       now = 15;
+        now = 15;
 
-       assertTrue(NoSpamLogger.log( mock, Level.ERROR, 5,  TimeUnit.NANOSECONDS, statement, param));
-       checkMock(Level.ERROR);
+        assertTrue(NoSpamLogger.log(mock, Level.ERROR, 5, TimeUnit.NANOSECONDS, statement, param));
+        checkMock(Level.ERROR);
 
-       now = 20;
+        now = 20;
 
-       NoSpamLogger logger = NoSpamLogger.getLogger(mock, 5, TimeUnit.NANOSECONDS);
+        NoSpamLogger logger = NoSpamLogger.getLogger(mock, 5, TimeUnit.NANOSECONDS);
 
-       assertTrue(logger.info(statement, param));
-       checkMock(Level.INFO);
+        assertTrue(logger.info(statement, param));
+        checkMock(Level.INFO);
 
-       now = 25;
+        now = 25;
 
-       assertTrue(logger.warn(statement, param));
-       checkMock(Level.WARN);
+        assertTrue(logger.warn(statement, param));
+        checkMock(Level.WARN);
 
-       now = 30;
+        now = 30;
 
-       assertTrue(logger.error(statement, param));
-       checkMock(Level.ERROR);
+        assertTrue(logger.error(statement, param));
+        checkMock(Level.ERROR);
+
+        NoSpamLogger.NoSpamLogStatement nospamStatement = logger.getStatement(statement);
 
-       NoSpamLogger.NoSpamLogStatement nospamStatement = logger.getStatement(statement);
+        now = 35;
 
-       now = 35;
+        assertTrue(nospamStatement.info(param));
+        checkMock(Level.INFO);
 
-       assertTrue(nospamStatement.info(param));
-       checkMock(Level.INFO);
+        now = 40;
 
-       now = 40;
+        assertTrue(nospamStatement.warn(param));
+        checkMock(Level.WARN);
 
-       assertTrue(nospamStatement.warn(param));
-       checkMock(Level.WARN);
+        now = 45;
 
-       now = 45;
+        assertTrue(nospamStatement.error(param));
+        checkMock(Level.ERROR);
+    }
 
-       assertTrue(nospamStatement.error(param));
-       checkMock(Level.ERROR);
-   }
+    @Test
+    public void testSupplierLogging()
+    {
+        AtomicInteger evaluationTimes = new AtomicInteger();
+        Object [] params = new Object[] {"hello"};
+        Supplier<Object[]> paramSupplier = () -> {
+            evaluationTimes.incrementAndGet();
+            return params;
+        };
+
+        now = 5;
+
+        NoSpamLogger.log(mock, Level.INFO, 5, TimeUnit.NANOSECONDS, "TESTING {}", paramSupplier);
+        assertEquals(1, evaluationTimes.get());
+        Pair<String, Object[]> loggedMsg = logged.get(Level.INFO).remove();
+        assertEquals("TESTING {}", loggedMsg.left);
+        assertArrayEquals(params, loggedMsg.right);
+
+        NoSpamLogger.log(mock, Level.INFO, 5, TimeUnit.NANOSECONDS, "TESTING {}", paramSupplier);
+        assertEquals(1, evaluationTimes.get());
+        assertTrue(logged.get(Level.INFO).isEmpty());
+
+        now = 10;
+        NoSpamLogger.log(mock, Level.INFO, 5, TimeUnit.NANOSECONDS, "TESTING {}", paramSupplier);
+        assertEquals(2, evaluationTimes.get());
+        loggedMsg = logged.get(Level.INFO).remove();
+        assertEquals("TESTING {}", loggedMsg.left);
+        assertArrayEquals(params, loggedMsg.right);
+    }
 }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org