You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2015/05/04 18:44:14 UTC

[1/4] cassandra git commit: Introduce NoSpamLogger

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 739f3e37c -> e6f027979


Introduce NoSpamLogger

patch by ariel; reviewed by benedict for CASSANDRA-9029


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5bffaf85
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5bffaf85
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5bffaf85

Branch: refs/heads/cassandra-2.1
Commit: 5bffaf850ca3e978baaa8664acc65612d7460d3f
Parents: 739f3e3
Author: Ariel Weisberg <ar...@datastax.com>
Authored: Fri Apr 3 23:27:10 2015 +0100
Committer: T Jake Luciani <ja...@apache.org>
Committed: Mon May 4 12:21:59 2015 -0400

----------------------------------------------------------------------
 .../apache/cassandra/utils/NoSpamLogger.java    | 238 +++++++++++++++++++
 .../cassandra/utils/NoSpamLoggerTest.java       | 174 ++++++++++++++
 2 files changed, 412 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5bffaf85/src/java/org/apache/cassandra/utils/NoSpamLogger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NoSpamLogger.java b/src/java/org/apache/cassandra/utils/NoSpamLogger.java
new file mode 100644
index 0000000..9f5d5ce
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NoSpamLogger.java
@@ -0,0 +1,238 @@
+/*
+ * 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.utils;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+import org.slf4j.Logger;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Logging that limits each log statement to firing based on time since the statement last fired.
+ *
+ * Every logger has a unique timer per statement. Minimum time between logging is set for each statement
+ * the first time it is used and a subsequent attempt to request that statement with a different minimum time will
+ * result in the original time being used. No warning is provided if there is a mismatch.
+ *
+ * If the statement is cached and used to log directly then only a volatile read will be required in the common case.
+ * If the Logger is cached then there is a single concurrent hash map lookup + the volatile read.
+ * If neither the logger nor the statement is cached then it is two concurrent hash map lookups + the volatile read.
+ *
+ */
+public class NoSpamLogger
+{
+    /**
+     * Levels for programmatically specifying the severity of a log statement
+     */
+    public enum Level
+    {
+        INFO, WARN, ERROR;
+    }
+
+    @VisibleForTesting
+    static interface Clock
+    {
+        long nanoTime();
+    }
+
+    @VisibleForTesting
+    static Clock CLOCK = new Clock()
+    {
+        public long nanoTime()
+        {
+            return System.nanoTime();
+        }
+    };
+
+    public class NoSpamLogStatement extends AtomicLong
+    {
+        private static final long serialVersionUID = 1L;
+
+        private final String statement;
+        private final long minIntervalNanos;
+
+        public NoSpamLogStatement(String statement, long minIntervalNanos)
+        {
+            this.statement = statement;
+            this.minIntervalNanos = minIntervalNanos;
+        }
+
+        private boolean shouldLog(long nowNanos)
+        {
+            long expected = get();
+            return nowNanos - expected >= minIntervalNanos && compareAndSet(expected, nowNanos);
+        }
+
+        public void log(Level l, long nowNanos, Object... objects)
+        {
+            if (!shouldLog(nowNanos)) return;
+
+            switch (l)
+            {
+            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();
+            }
+        }
+
+        public void info(long nowNanos, Object... objects)
+        {
+            log(Level.INFO, nowNanos, objects);
+        }
+
+        public void info(Object... objects)
+        {
+            info(CLOCK.nanoTime(), objects);
+        }
+
+        public void warn(long nowNanos, Object... objects)
+        {
+            log(Level.WARN, nowNanos, objects);
+        }
+
+        public void warn(String s, Object... objects)
+        {
+            warn(CLOCK.nanoTime(), s, objects);
+        }
+
+        public void error(long nowNanos, Object... objects)
+        {
+            log(Level.ERROR, nowNanos, objects);
+        }
+
+        public void error(Object... objects)
+        {
+            error(CLOCK.nanoTime(), objects);
+        }
+    }
+
+    private static final NonBlockingHashMap<Logger, NoSpamLogger> wrappedLoggers = new NonBlockingHashMap<>();
+
+    @VisibleForTesting
+    static void clearWrappedLoggersForTest()
+    {
+        wrappedLoggers.clear();
+    }
+
+    public static NoSpamLogger getLogger(Logger logger, long minInterval, TimeUnit unit)
+    {
+        NoSpamLogger wrapped = wrappedLoggers.get(logger);
+        if (wrapped == null)
+        {
+            wrapped = new NoSpamLogger(logger, minInterval, unit);
+            NoSpamLogger temp = wrappedLoggers.putIfAbsent(logger, wrapped);
+            if (temp != null)
+                wrapped = temp;
+        }
+        return wrapped;
+    }
+
+    public static void log(Logger logger, Level level, long minInterval, TimeUnit unit, String message, Object... objects)
+    {
+        log(logger, level, minInterval, unit, CLOCK.nanoTime(), message, objects);
+    }
+
+    public static void log(Logger logger, Level level, long minInterval, TimeUnit unit, long nowNanos, String message, Object... objects)
+    {
+        NoSpamLogger wrapped = getLogger(logger, minInterval, unit);
+        NoSpamLogStatement statement = wrapped.getStatement(message);
+        statement.log(level, nowNanos, objects);
+    }
+
+    public static NoSpamLogStatement getStatement(Logger logger, String message, long minInterval, TimeUnit unit) {
+        NoSpamLogger wrapped = getLogger(logger, minInterval, unit);
+        return wrapped.getStatement(message);
+    }
+
+    private final Logger wrapped;
+    private final long minIntervalNanos;
+    private final NonBlockingHashMap<String, NoSpamLogStatement> lastMessage = new NonBlockingHashMap<>();
+
+    private NoSpamLogger(Logger wrapped, long minInterval, TimeUnit timeUnit)
+    {
+        this.wrapped = wrapped;
+        minIntervalNanos = timeUnit.toNanos(minInterval);
+    }
+
+    public void info(long nowNanos, String s, Object... objects)
+    {
+        log( Level.INFO, s, nowNanos, objects);
+    }
+
+    public void info(String s, Object... objects)
+    {
+        info(CLOCK.nanoTime(), s, objects);
+    }
+
+    public void warn(long nowNanos, String s, Object... objects)
+    {
+        log( Level.WARN, s, nowNanos, objects);
+    }
+
+    public void warn(String s, Object... objects)
+    {
+        warn(CLOCK.nanoTime(), s, objects);
+    }
+
+    public void error(long nowNanos, String s, Object... objects)
+    {
+        log( Level.ERROR, s, nowNanos, objects);
+    }
+
+    public void error(String s, Object... objects)
+    {
+        error(CLOCK.nanoTime(), s, objects);
+    }
+
+    public void log(Level l, String s, long nowNanos, Object... objects) {
+        getStatement(s, minIntervalNanos).log(l, nowNanos, objects);
+    }
+
+    public NoSpamLogStatement getStatement(String s)
+    {
+        return getStatement(s, minIntervalNanos);
+    }
+
+    public NoSpamLogStatement getStatement(String s, long minInterval, TimeUnit unit) {
+        return getStatement(s, unit.toNanos(minInterval));
+    }
+
+    public NoSpamLogStatement getStatement(String s, long minIntervalNanos)
+    {
+        NoSpamLogStatement statement = lastMessage.get(s);
+        if (statement == null)
+        {
+            statement = new NoSpamLogStatement(s, minIntervalNanos);
+            NoSpamLogStatement temp = lastMessage.putIfAbsent(s, statement);
+            if (temp != null)
+                statement = temp;
+        }
+        return statement;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5bffaf85/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
new file mode 100644
index 0000000..3ad7ab8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
@@ -0,0 +1,174 @@
+/*
+* 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.utils;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.utils.NoSpamLogger.Level;
+import org.apache.cassandra.utils.NoSpamLogger.NoSpamLogStatement;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.helpers.SubstituteLogger;
+
+
+public class NoSpamLoggerTest
+{
+    Map<Level, List<Pair<String, Object[]>>> logged = new HashMap<>();
+
+   Logger mock = new SubstituteLogger(null)
+   {
+
+       @Override
+       public void info(String statement, Object... args)
+       {
+           logged.get(Level.INFO).add(Pair.create(statement, args));
+       }
+
+       @Override
+       public void warn(String statement, Object... args)
+       {
+           logged.get(Level.WARN).add(Pair.create(statement, args));
+       }
+
+       @Override
+       public void error(String statement, Object... args)
+       {
+           logged.get(Level.ERROR).add(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 long now;
+
+   @BeforeClass
+   public static void setUpClass() throws Exception
+   {
+       NoSpamLogger.CLOCK = new NoSpamLogger.Clock()
+       {
+        @Override
+        public long nanoTime()
+        {
+            return now;
+        }
+       };
+   }
+
+   @Before
+   public void setUp() throws Exception
+   {
+       logged.put(Level.INFO, new ArrayList<Pair<String, Object[]>>());
+       logged.put(Level.WARN, new ArrayList<Pair<String, Object[]>>());
+       logged.put(Level.ERROR, new ArrayList<Pair<String, Object[]>>());
+   }
+
+   @Test
+   public void testNoSpamLogger() throws Exception
+   {
+       testLevel(Level.INFO);
+       testLevel(Level.WARN);
+       testLevel(Level.ERROR);
+   }
+
+   private void testLevel(Level l) throws Exception
+   {
+       setUp();
+       now = 5;
+       NoSpamLogger.clearWrappedLoggersForTest();
+
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, "swizzle{}", "a");
+
+       assertEquals(1, logged.get(l).size());
+
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, "swizzle{}", "a");
+
+       assertEquals(1, logged.get(l).size());
+
+       now += 5;
+
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, "swizzle{}", "a");
+
+       assertEquals(2, logged.get(l).size());
+   }
+
+   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);
+
+       logger.info("swizzle{}", "a");
+       logger.info("swizzle{}", "a");
+       logger.warn("swizzle{}", "a");
+       logger.error("swizzle{}", "a");
+
+       assertLoggedSizes(1, 0, 0);
+
+       NoSpamLogStatement statement = logger.getStatement("swizzle2{}", 10, TimeUnit.NANOSECONDS);
+       statement.warn("a");
+       //now is 5 so it won't log
+       assertLoggedSizes(1, 0, 0);
+
+       now = 10;
+       statement.warn("a");
+       assertLoggedSizes(1, 1, 0);
+
+   }
+
+   @Test
+   public void testNoSpamLoggerStatementDirect() throws Exception
+   {
+       NoSpamLogger.NoSpamLogStatement statement = NoSpamLogger.getStatement( mock, "swizzle{}", 5, TimeUnit.NANOSECONDS);
+
+       now = 5;
+
+       statement.info("swizzle{}", "a");
+       statement.info("swizzle{}", "a");
+       statement.warn("swizzle{}", "a");
+       statement.error("swizzle{}", "a");
+
+       assertLoggedSizes(1, 0, 0);
+   }
+}


[4/4] cassandra git commit: backport 9029 to 2.1

Posted by ja...@apache.org.
backport 9029 to 2.1


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e6f02797
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e6f02797
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e6f02797

Branch: refs/heads/cassandra-2.1
Commit: e6f027979a3ec4221438bd2a21db8053cb3c1ad7
Parents: 8ec1da2
Author: T Jake Luciani <ja...@apache.org>
Authored: Mon May 4 12:42:10 2015 -0400
Committer: T Jake Luciani <ja...@apache.org>
Committed: Mon May 4 12:42:10 2015 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/utils/NoSpamLoggerTest.java       | 141 ++++++++++++++++++-
 2 files changed, 139 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e6f02797/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0593e2b..e7689ab 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.6
+ * Add support for rate limiting log messages (CASSANDRA-9029)
  * Log the partition key with tombstone warnings (CASSANDRA-8561)
  * Reduce runWithCompactionsDisabled poll interval to 1ms (CASSANDRA-9271)
  * Fix PITR commitlog replay (CASSANDRA-9195)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e6f02797/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
index 0a5a005..0d6c8b1 100644
--- a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
+++ b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
@@ -32,34 +32,169 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
-import org.slf4j.helpers.SubstituteLogger;
+import org.slf4j.helpers.MarkerIgnoringBase;
 
 
 public class NoSpamLoggerTest
 {
     Map<Level, Queue<Pair<String, Object[]>>> logged = new HashMap<>();
 
-   Logger mock = new SubstituteLogger(null)
+   Logger mock = new MarkerIgnoringBase()
    {
 
+       public boolean isTraceEnabled()
+       {
+           return false;
+       }
+
+       public void trace(String s)
+       {
+
+       }
+
+       public void trace(String s, Object o)
+       {
+
+       }
+
+       public void trace(String s, Object o, Object o1)
+       {
+
+       }
+
+       public void trace(String s, Object... objects)
+       {
+
+       }
+
+       public void trace(String s, Throwable throwable)
+       {
+
+       }
+
+       public boolean isDebugEnabled()
+       {
+           return false;
+       }
+
+       public void debug(String s)
+       {
+
+       }
+
+       public void debug(String s, Object o)
+       {
+
+       }
+
+       public void debug(String s, Object o, Object o1)
+       {
+
+       }
+
+       public void debug(String s, Object... objects)
+       {
+
+       }
+
+       public void debug(String s, Throwable throwable)
+       {
+
+       }
+
+       public boolean isInfoEnabled()
+       {
+           return false;
+       }
+
+       public void info(String s)
+       {
+
+       }
+
+       public void info(String s, Object o)
+       {
+
+       }
+
+       public void info(String s, Object o, Object o1)
+       {
+
+       }
+
        @Override
        public void info(String statement, Object... args)
        {
            logged.get(Level.INFO).offer(Pair.create(statement, args));
        }
 
+       public void info(String s, Throwable throwable)
+       {
+
+       }
+
+       public boolean isWarnEnabled()
+       {
+           return false;
+       }
+
+       public void warn(String s)
+       {
+
+       }
+
+       public void warn(String s, Object o)
+       {
+
+       }
+
        @Override
        public void warn(String statement, Object... args)
        {
            logged.get(Level.WARN).offer(Pair.create(statement, args));
        }
 
+       public void warn(String s, Object o, Object o1)
+       {
+
+       }
+
+       public void warn(String s, Throwable throwable)
+       {
+
+       }
+
+       public boolean isErrorEnabled()
+       {
+           return false;
+       }
+
+       public void error(String s)
+       {
+
+       }
+
+       public void error(String s, Object o)
+       {
+
+       }
+
+       public void error(String s, Object o, Object o1)
+       {
+
+       }
+
        @Override
        public void error(String statement, Object... args)
        {
            logged.get(Level.ERROR).offer(Pair.create(statement, args));
        }
 
+       public void error(String s, Throwable throwable)
+       {
+
+       }
+
        @Override
        public int hashCode()
        {
@@ -123,7 +258,7 @@ public class NoSpamLoggerTest
 
        now += 5;
 
-       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param);
+       NoSpamLogger.log(mock, l, 5, TimeUnit.NANOSECONDS, statement, param);
 
        assertEquals(2, logged.get(l).size());
    }


[3/4] cassandra git commit: Ninja fix CASSANDRA-9029

Posted by ja...@apache.org.
Ninja fix CASSANDRA-9029


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8ec1da21
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8ec1da21
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8ec1da21

Branch: refs/heads/cassandra-2.1
Commit: 8ec1da211830762ebf571f12d9cbd505d2a1fada
Parents: a654944
Author: Ariel Weisberg <ar...@weisberg.ws>
Authored: Tue Apr 7 01:01:16 2015 +0200
Committer: T Jake Luciani <ja...@apache.org>
Committed: Mon May 4 12:35:08 2015 -0400

----------------------------------------------------------------------
 test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ec1da21/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
index ca1d6d3..0a5a005 100644
--- a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
+++ b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
@@ -191,6 +191,8 @@ public class NoSpamLoggerTest
    @Test
    public void testLoggedResult() throws Exception
    {
+       now = 5;
+
        NoSpamLogger.log( mock, Level.INFO, 5,  TimeUnit.NANOSECONDS, statement, param);
        checkMock(Level.INFO);
 


[2/4] cassandra git commit: Ninja fix CASSANDRA-9029

Posted by ja...@apache.org.
Ninja fix CASSANDRA-9029


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a6549440
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a6549440
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a6549440

Branch: refs/heads/cassandra-2.1
Commit: a6549440f30997273f0b1a073b1493684715c43b
Parents: 5bffaf8
Author: Ariel Weisberg <ar...@weisberg.ws>
Authored: Mon Apr 6 23:00:00 2015 +0200
Committer: T Jake Luciani <ja...@apache.org>
Committed: Mon May 4 12:34:08 2015 -0400

----------------------------------------------------------------------
 .../apache/cassandra/utils/NoSpamLogger.java    |  35 +++---
 .../cassandra/utils/NoSpamLoggerTest.java       | 115 +++++++++++++++----
 2 files changed, 109 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6549440/src/java/org/apache/cassandra/utils/NoSpamLogger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NoSpamLogger.java b/src/java/org/apache/cassandra/utils/NoSpamLogger.java
index 9f5d5ce..3cc8b5e 100644
--- a/src/java/org/apache/cassandra/utils/NoSpamLogger.java
+++ b/src/java/org/apache/cassandra/utils/NoSpamLogger.java
@@ -103,32 +103,32 @@ public class NoSpamLogger
 
         public void info(long nowNanos, Object... objects)
         {
-            log(Level.INFO, nowNanos, objects);
+            NoSpamLogStatement.this.log(Level.INFO, nowNanos, objects);
         }
 
         public void info(Object... objects)
         {
-            info(CLOCK.nanoTime(), objects);
+            NoSpamLogStatement.this.info(CLOCK.nanoTime(), objects);
         }
 
         public void warn(long nowNanos, Object... objects)
         {
-            log(Level.WARN, nowNanos, objects);
+            NoSpamLogStatement.this.log(Level.WARN, nowNanos, objects);
         }
 
-        public void warn(String s, Object... objects)
+        public void warn(Object... objects)
         {
-            warn(CLOCK.nanoTime(), s, objects);
+            NoSpamLogStatement.this.warn(CLOCK.nanoTime(), objects);
         }
 
         public void error(long nowNanos, Object... objects)
         {
-            log(Level.ERROR, nowNanos, objects);
+            NoSpamLogStatement.this.log(Level.ERROR, nowNanos, objects);
         }
 
         public void error(Object... objects)
         {
-            error(CLOCK.nanoTime(), objects);
+            NoSpamLogStatement.this.error(CLOCK.nanoTime(), objects);
         }
     }
 
@@ -165,7 +165,8 @@ public class NoSpamLogger
         statement.log(level, nowNanos, objects);
     }
 
-    public static NoSpamLogStatement getStatement(Logger logger, String message, long minInterval, TimeUnit unit) {
+    public static NoSpamLogStatement getStatement(Logger logger, String message, long minInterval, TimeUnit unit)
+    {
         NoSpamLogger wrapped = getLogger(logger, minInterval, unit);
         return wrapped.getStatement(message);
     }
@@ -182,45 +183,45 @@ public class NoSpamLogger
 
     public void info(long nowNanos, String s, Object... objects)
     {
-        log( Level.INFO, s, nowNanos, objects);
+        NoSpamLogger.this.log( Level.INFO, s, nowNanos, objects);
     }
 
     public void info(String s, Object... objects)
     {
-        info(CLOCK.nanoTime(), s, objects);
+        NoSpamLogger.this.info(CLOCK.nanoTime(), s, objects);
     }
 
     public void warn(long nowNanos, String s, Object... objects)
     {
-        log( Level.WARN, s, nowNanos, objects);
+        NoSpamLogger.this.log( Level.WARN, s, nowNanos, objects);
     }
 
     public void warn(String s, Object... objects)
     {
-        warn(CLOCK.nanoTime(), s, objects);
+        NoSpamLogger.this.warn(CLOCK.nanoTime(), s, objects);
     }
 
     public void error(long nowNanos, String s, Object... objects)
     {
-        log( Level.ERROR, s, nowNanos, objects);
+        NoSpamLogger.this.log( Level.ERROR, s, nowNanos, objects);
     }
 
     public void error(String s, Object... objects)
     {
-        error(CLOCK.nanoTime(), s, objects);
+        NoSpamLogger.this.error(CLOCK.nanoTime(), s, objects);
     }
 
     public void log(Level l, String s, long nowNanos, Object... objects) {
-        getStatement(s, minIntervalNanos).log(l, nowNanos, objects);
+        NoSpamLogger.this.getStatement(s, minIntervalNanos).log(l, nowNanos, objects);
     }
 
     public NoSpamLogStatement getStatement(String s)
     {
-        return getStatement(s, minIntervalNanos);
+        return NoSpamLogger.this.getStatement(s, minIntervalNanos);
     }
 
     public NoSpamLogStatement getStatement(String s, long minInterval, TimeUnit unit) {
-        return getStatement(s, unit.toNanos(minInterval));
+        return NoSpamLogger.this.getStatement(s, unit.toNanos(minInterval));
     }
 
     public NoSpamLogStatement getStatement(String s, long minIntervalNanos)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a6549440/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
index 3ad7ab8..ca1d6d3 100644
--- a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
+++ b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
@@ -20,10 +20,10 @@ package org.apache.cassandra.utils;
 
 import static org.junit.Assert.*;
 
-import java.util.ArrayList;
+import java.util.ArrayDeque;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Queue;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.utils.NoSpamLogger.Level;
@@ -37,7 +37,7 @@ import org.slf4j.helpers.SubstituteLogger;
 
 public class NoSpamLoggerTest
 {
-    Map<Level, List<Pair<String, Object[]>>> logged = new HashMap<>();
+    Map<Level, Queue<Pair<String, Object[]>>> logged = new HashMap<>();
 
    Logger mock = new SubstituteLogger(null)
    {
@@ -45,19 +45,19 @@ public class NoSpamLoggerTest
        @Override
        public void info(String statement, Object... args)
        {
-           logged.get(Level.INFO).add(Pair.create(statement, args));
+           logged.get(Level.INFO).offer(Pair.create(statement, args));
        }
 
        @Override
        public void warn(String statement, Object... args)
        {
-           logged.get(Level.WARN).add(Pair.create(statement, args));
+           logged.get(Level.WARN).offer(Pair.create(statement, args));
        }
 
        @Override
        public void error(String statement, Object... args)
        {
-           logged.get(Level.ERROR).add(Pair.create(statement, args));
+           logged.get(Level.ERROR).offer(Pair.create(statement, args));
        }
 
        @Override
@@ -74,6 +74,8 @@ public class NoSpamLoggerTest
    };
 
 
+   static final String statement = "swizzle{}";
+   static final String param = "";
    static long now;
 
    @BeforeClass
@@ -92,9 +94,10 @@ public class NoSpamLoggerTest
    @Before
    public void setUp() throws Exception
    {
-       logged.put(Level.INFO, new ArrayList<Pair<String, Object[]>>());
-       logged.put(Level.WARN, new ArrayList<Pair<String, Object[]>>());
-       logged.put(Level.ERROR, new ArrayList<Pair<String, Object[]>>());
+       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();
    }
 
    @Test
@@ -109,19 +112,18 @@ public class NoSpamLoggerTest
    {
        setUp();
        now = 5;
-       NoSpamLogger.clearWrappedLoggersForTest();
 
-       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, "swizzle{}", "a");
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param);
 
        assertEquals(1, logged.get(l).size());
 
-       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, "swizzle{}", "a");
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param);
 
        assertEquals(1, logged.get(l).size());
 
        now += 5;
 
-       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, "swizzle{}", "a");
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param);
 
        assertEquals(2, logged.get(l).size());
    }
@@ -139,20 +141,20 @@ public class NoSpamLoggerTest
        now = 5;
        NoSpamLogger logger = NoSpamLogger.getLogger( mock, 5, TimeUnit.NANOSECONDS);
 
-       logger.info("swizzle{}", "a");
-       logger.info("swizzle{}", "a");
-       logger.warn("swizzle{}", "a");
-       logger.error("swizzle{}", "a");
+       logger.info(statement, param);
+       logger.info(statement, param);
+       logger.warn(statement, param);
+       logger.error(statement, param);
 
        assertLoggedSizes(1, 0, 0);
 
        NoSpamLogStatement statement = logger.getStatement("swizzle2{}", 10, TimeUnit.NANOSECONDS);
-       statement.warn("a");
+       statement.warn(param);
        //now is 5 so it won't log
        assertLoggedSizes(1, 0, 0);
 
        now = 10;
-       statement.warn("a");
+       statement.warn(param);
        assertLoggedSizes(1, 1, 0);
 
    }
@@ -160,15 +162,80 @@ public class NoSpamLoggerTest
    @Test
    public void testNoSpamLoggerStatementDirect() throws Exception
    {
-       NoSpamLogger.NoSpamLogStatement statement = NoSpamLogger.getStatement( mock, "swizzle{}", 5, TimeUnit.NANOSECONDS);
+       NoSpamLogger.NoSpamLogStatement nospam = NoSpamLogger.getStatement( mock, statement, 5, TimeUnit.NANOSECONDS);
 
        now = 5;
 
-       statement.info("swizzle{}", "a");
-       statement.info("swizzle{}", "a");
-       statement.warn("swizzle{}", "a");
-       statement.error("swizzle{}", "a");
+       nospam.info(statement, param);
+       nospam.info(statement, param);
+       nospam.warn(statement, param);
+       nospam.error(statement, param);
 
        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());
+   }
+
+   /*
+    * Make sure that what is passed to the underlying logger is the correct set of objects
+    */
+   @Test
+   public void testLoggedResult() throws Exception
+   {
+       NoSpamLogger.log( mock, Level.INFO, 5,  TimeUnit.NANOSECONDS, statement, param);
+       checkMock(Level.INFO);
+
+       now = 10;
+
+       NoSpamLogger.log( mock, Level.WARN, 5,  TimeUnit.NANOSECONDS, statement, param);
+       checkMock(Level.WARN);
+
+       now = 15;
+
+       NoSpamLogger.log( mock, Level.ERROR, 5,  TimeUnit.NANOSECONDS, statement, param);
+       checkMock(Level.ERROR);
+
+       now = 20;
+
+       NoSpamLogger logger = NoSpamLogger.getLogger(mock, 5, TimeUnit.NANOSECONDS);
+
+       logger.info(statement, param);
+       checkMock(Level.INFO);
+
+       now = 25;
+
+       logger.warn(statement, param);
+       checkMock(Level.WARN);
+
+       now = 30;
+
+       logger.error(statement, param);
+       checkMock(Level.ERROR);
+
+       NoSpamLogger.NoSpamLogStatement nospamStatement = logger.getStatement(statement);
+
+       now = 35;
+
+       nospamStatement.info(param);
+       checkMock(Level.INFO);
+
+       now = 40;
+
+       nospamStatement.warn(param);
+       checkMock(Level.WARN);
+
+       now = 45;
+
+       nospamStatement.error(param);
+       checkMock(Level.ERROR);
+   }
 }