You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ca...@apache.org on 2016/01/13 20:48:52 UTC

[01/13] cassandra git commit: Make sure client gets tombstone overwhelmed warning

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 582bdba4b -> dbf6e62c3
  refs/heads/cassandra-3.0 0f995a2dc -> 94e7ef177
  refs/heads/cassandra-3.3 a7feb80d6 -> 93f652939
  refs/heads/trunk 3f053121f -> 5090afc7a


Make sure client gets tombstone overwhelmed warning

patch by Carl Yeksigian; reviewed by Josh McKenzie for CASSANDRA-9465


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

Branch: refs/heads/cassandra-2.2
Commit: dbf6e62c382d62f9c1727ecf5afb90d131a81775
Parents: 582bdba
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 13:22:36 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:35:59 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 229 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 229 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cql3/statements/BatchStatement.java         |  11 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../cassandra/db/filter/SliceQueryFilter.java   |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   8 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cassandra/service/ClientWarningsTest.java   |  43 ++++
 21 files changed, 529 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11f2529..6530956 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.5
+ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
  * Fix error streaming section more than 2GB (CASSANDRA-10961)
  * (cqlsh) Also apply --connect-timeout to control connection
    timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
new file mode 100644
index 0000000..088b43e
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -0,0 +1,229 @@
+/*
+ * 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.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.tracing.Tracing.isTracing;
+
+public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+{
+    private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+
+    protected abstract void addTask(FutureTask<?> futureTask);
+    protected abstract void onCompletion();
+
+    /** Task Submission / Creation / Objects **/
+
+    public <T> FutureTask<T> submit(Callable<T> task)
+    {
+        return submit(newTaskFor(task));
+    }
+
+    public FutureTask<?> submit(Runnable task)
+    {
+        return submit(newTaskFor(task, null));
+    }
+
+    public <T> FutureTask<T> submit(Runnable task, T result)
+    {
+        return submit(newTaskFor(task, result));
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+    {
+        return newTaskFor(runnable, result, ExecutorLocals.create());
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+    {
+        if (locals != null)
+        {
+            if (runnable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) runnable;
+            return new LocalSessionFutureTask<T>(runnable, result, locals);
+        }
+        if (runnable instanceof FutureTask)
+            return (FutureTask<T>) runnable;
+        return new FutureTask<>(runnable, result);
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+    {
+        if (isTracing())
+        {
+            if (callable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) callable;
+            return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+        }
+        if (callable instanceof FutureTask)
+            return (FutureTask<T>) callable;
+        return new FutureTask<>(callable);
+    }
+
+    private class LocalSessionFutureTask<T> extends FutureTask<T>
+    {
+        private final ExecutorLocals locals;
+
+        public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+        {
+            super(callable);
+            this.locals = locals;
+        }
+
+        public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+        {
+            super(runnable, result);
+            this.locals = locals;
+        }
+
+        public void run()
+        {
+            ExecutorLocals old = ExecutorLocals.create();
+            ExecutorLocals.set(locals);
+            try
+            {
+                super.run();
+            }
+            finally
+            {
+                ExecutorLocals.set(old);
+            }
+        }
+    }
+
+    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+    {
+        private boolean failure;
+        private Object result = this;
+        private final Callable<T> callable;
+
+        public FutureTask(Callable<T> callable)
+        {
+            this.callable = callable;
+        }
+        public FutureTask(Runnable runnable, T result)
+        {
+            this(Executors.callable(runnable, result));
+        }
+
+        public void run()
+        {
+            try
+            {
+                result = callable.call();
+            }
+            catch (Throwable t)
+            {
+                JVMStabilityInspector.inspectThrowable(t);
+                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                result = t;
+                failure = true;
+            }
+            finally
+            {
+                signalAll();
+                onCompletion();
+            }
+        }
+
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            return false;
+        }
+
+        public boolean isCancelled()
+        {
+            return false;
+        }
+
+        public boolean isDone()
+        {
+            return isSignaled();
+        }
+
+        public T get() throws InterruptedException, ExecutionException
+        {
+            await();
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+
+        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            await(timeout, unit);
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+    }
+
+    private <T> FutureTask<T> submit(FutureTask<T> task)
+    {
+        addTask(task);
+        return task;
+    }
+
+    public void execute(Runnable command)
+    {
+        addTask(newTaskFor(command, ExecutorLocals.create()));
+    }
+
+    public void execute(Runnable command, ExecutorLocals locals)
+    {
+        addTask(newTaskFor(command, null, locals));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
deleted file mode 100644
index fb753b0..0000000
--- a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.concurrent;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-public abstract class AbstractTracingAwareExecutorService implements TracingAwareExecutorService
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractTracingAwareExecutorService.class);
-
-    protected abstract void addTask(FutureTask<?> futureTask);
-    protected abstract void onCompletion();
-
-    /** Task Submission / Creation / Objects **/
-
-    public <T> FutureTask<T> submit(Callable<T> task)
-    {
-        return submit(newTaskFor(task));
-    }
-
-    public FutureTask<?> submit(Runnable task)
-    {
-        return submit(newTaskFor(task, null));
-    }
-
-    public <T> FutureTask<T> submit(Runnable task, T result)
-    {
-        return submit(newTaskFor(task, result));
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
-    {
-        return newTaskFor(runnable, result, Tracing.instance.get());
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, TraceState traceState)
-    {
-        if (traceState != null)
-        {
-            if (runnable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) runnable;
-            return new TraceSessionFutureTask<T>(runnable, result, traceState);
-        }
-        if (runnable instanceof FutureTask)
-            return (FutureTask<T>) runnable;
-        return new FutureTask<>(runnable, result);
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing())
-        {
-            if (callable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) callable;
-            return new TraceSessionFutureTask<T>(callable, Tracing.instance.get());
-        }
-        if (callable instanceof FutureTask)
-            return (FutureTask<T>) callable;
-        return new FutureTask<>(callable);
-    }
-
-    private class TraceSessionFutureTask<T> extends FutureTask<T>
-    {
-        private final TraceState state;
-
-        public TraceSessionFutureTask(Callable<T> callable, TraceState state)
-        {
-            super(callable);
-            this.state = state;
-        }
-
-        public TraceSessionFutureTask(Runnable runnable, T result, TraceState state)
-        {
-            super(runnable, result);
-            this.state = state;
-        }
-
-        public void run()
-        {
-            TraceState oldState = Tracing.instance.get();
-            Tracing.instance.set(state);
-            try
-            {
-                super.run();
-            }
-            finally
-            {
-                Tracing.instance.set(oldState);
-            }
-        }
-    }
-
-    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
-    {
-        private boolean failure;
-        private Object result = this;
-        private final Callable<T> callable;
-
-        public FutureTask(Callable<T> callable)
-        {
-            this.callable = callable;
-        }
-        public FutureTask(Runnable runnable, T result)
-        {
-            this(Executors.callable(runnable, result));
-        }
-
-        public void run()
-        {
-            try
-            {
-                result = callable.call();
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
-                result = t;
-                failure = true;
-            }
-            finally
-            {
-                signalAll();
-                onCompletion();
-            }
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return isSignaled();
-        }
-
-        public T get() throws InterruptedException, ExecutionException
-        {
-            await();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-
-        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            await(timeout, unit);
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-    }
-
-    private <T> FutureTask<T> submit(FutureTask<T> task)
-    {
-        addTask(task);
-        return task;
-    }
-
-    public void execute(Runnable command)
-    {
-        addTask(newTaskFor(command, null));
-    }
-
-    public void execute(Runnable command, TraceState state)
-    {
-        addTask(newTaskFor(command, null, state));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index a6d0049..1fb0690 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -44,7 +44,7 @@ import static org.apache.cassandra.tracing.Tracing.isTracing;
  *   threads and the queue is full, we want the enqueuer to block.  But to allow the number of threads to drop if a
  *   stage is less busy, core thread timeout is enabled.
  */
-public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
 {
     protected static final Logger logger = LoggerFactory.getLogger(DebuggableThreadPoolExecutor.class);
     public static final RejectedExecutionHandler blockingExecutionHandler = new RejectedExecutionHandler()
@@ -146,11 +146,11 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     protected void onFinalAccept(Runnable task) {}
     protected void onFinalRejection(Runnable task) {}
 
-    public void execute(Runnable command, TraceState state)
+    public void execute(Runnable command, ExecutorLocals locals)
     {
-        super.execute(state == null || command instanceof TraceSessionWrapper
+        super.execute(locals == null || command instanceof LocalSessionWrapper
                       ? command
-                      : new TraceSessionWrapper<Object>(command, state));
+                      : new LocalSessionWrapper<Object>(command, locals));
     }
 
     public void maybeExecuteImmediately(Runnable command)
@@ -162,17 +162,17 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     public void execute(Runnable command)
     {
-        super.execute(isTracing() && !(command instanceof TraceSessionWrapper)
-                      ? new TraceSessionWrapper<Object>(Executors.callable(command, null))
+        super.execute(isTracing() && !(command instanceof LocalSessionWrapper)
+                      ? new LocalSessionWrapper<Object>(Executors.callable(command, null))
                       : command);
     }
 
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T result)
     {
-        if (isTracing() && !(runnable instanceof TraceSessionWrapper))
+        if (isTracing() && !(runnable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(Executors.callable(runnable, result));
+            return new LocalSessionWrapper<T>(Executors.callable(runnable, result));
         }
         return super.newTaskFor(runnable, result);
     }
@@ -180,9 +180,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
     {
-        if (isTracing() && !(callable instanceof TraceSessionWrapper))
+        if (isTracing() && !(callable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(callable);
+            return new LocalSessionWrapper<T>(callable);
         }
         return super.newTaskFor(callable);
     }
@@ -198,9 +198,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
 
     protected static void maybeResetTraceSessionWrapper(Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
+        if (r instanceof LocalSessionWrapper)
         {
-            TraceSessionWrapper tsw = (TraceSessionWrapper) r;
+            LocalSessionWrapper tsw = (LocalSessionWrapper) r;
             // we have to reset trace state as its presence is what denotes the current thread is tracing
             // and if left this thread might start tracing unrelated tasks
             tsw.reset();
@@ -210,8 +210,8 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected void beforeExecute(Thread t, Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
-            ((TraceSessionWrapper) r).setupContext();
+        if (r instanceof LocalSessionWrapper)
+            ((LocalSessionWrapper) r).setupContext();
 
         super.beforeExecute(t, r);
     }
@@ -278,35 +278,35 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     }
 
     /**
-     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the TraceSessionContext and move
-     * it into the worker thread.
+     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the ExecutorLocals and move
+     * them into the worker thread.
      *
      * @param <T>
      */
-    private static class TraceSessionWrapper<T> extends FutureTask<T>
+    private static class LocalSessionWrapper<T> extends FutureTask<T>
     {
-        private final TraceState state;
+        private final ExecutorLocals locals;
 
-        public TraceSessionWrapper(Callable<T> callable)
+        public LocalSessionWrapper(Callable<T> callable)
         {
             super(callable);
-            state = Tracing.instance.get();
+            locals = ExecutorLocals.create();
         }
 
-        public TraceSessionWrapper(Runnable command, TraceState state)
+        public LocalSessionWrapper(Runnable command, ExecutorLocals locals)
         {
             super(command, null);
-            this.state = state;
+            this.locals = locals;
         }
 
         private void setupContext()
         {
-            Tracing.instance.set(state);
+            ExecutorLocals.set(locals);
         }
 
         private void reset()
         {
-            Tracing.instance.set(null);
+            ExecutorLocals.set(null);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
new file mode 100644
index 0000000..47826f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
@@ -0,0 +1,44 @@
+/*
+ * 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.concurrent;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+public interface ExecutorLocal<T>
+{
+    ExecutorLocal[] all = { Tracing.instance, ClientWarn.instance };
+
+    /**
+     * This is called when scheduling the task, and also before calling {@link ExecutorLocal#set(T)} when running on a
+     * executor thread.
+     *
+     * @return The thread-local value that we want to copy across executor boundaries; may be null if not set.
+     */
+    T get();
+
+    /**
+     * Before a task has been run, this will be called with the value from the thread that scheduled the task, and after
+     * the task is finished, the value that was previously retrieved from this thread is restored.
+     *
+     * @param value Value to use for the executor local state; may be null.
+     */
+    void set(T value);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
new file mode 100644
index 0000000..8e6d6ea
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
@@ -0,0 +1,84 @@
+/*
+ * 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.concurrent;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+/*
+ * This class only knows about Tracing and ClientWarn, so if any different executor locals are added, it must be
+ * updated.
+ *
+ * We don't enumerate the ExecutorLocal.all array each time because it would be much slower.
+ */
+public class ExecutorLocals
+{
+    private static final ExecutorLocal<TraceState> tracing = Tracing.instance;
+    private static final ExecutorLocal<ClientWarn.State> clientWarn = ClientWarn.instance;
+
+    public final TraceState traceState;
+    public final ClientWarn.State clientWarnState;
+
+    private ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
+    {
+        this.traceState = traceState;
+        this.clientWarnState = clientWarnState;
+    }
+
+    static
+    {
+        assert Arrays.equals(ExecutorLocal.all, new ExecutorLocal[]{ tracing, clientWarn })
+        : "ExecutorLocals has not been updated to reflect new ExecutorLocal.all";
+    }
+
+    /**
+     * This creates a new ExecutorLocals object based on what is already set.
+     *
+     * @return an ExecutorLocals object which has the trace state and client warn state captured if either has been set,
+     *         or null if both are unset. The null result short-circuits logic in
+     *         {@link AbstractLocalAwareExecutorService#newTaskFor(Runnable, Object, ExecutorLocals)}, preventing
+     *         unnecessarily calling {@link ExecutorLocals#set(ExecutorLocals)}.
+     */
+    public static ExecutorLocals create()
+    {
+        TraceState traceState = tracing.get();
+        ClientWarn.State clientWarnState = clientWarn.get();
+        if (traceState == null && clientWarnState == null)
+            return null;
+        else
+            return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static ExecutorLocals create(TraceState traceState)
+    {
+        ClientWarn.State clientWarnState = clientWarn.get();
+        return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static void set(ExecutorLocals locals)
+    {
+        TraceState traceState = locals == null ? null : locals.traceState;
+        ClientWarn.State clientWarnState = locals == null ? null : locals.clientWarnState;
+        tracing.set(traceState);
+        clientWarn.set(clientWarnState);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
new file mode 100644
index 0000000..5577d59
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * 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.concurrent;
+
+import java.util.concurrent.ExecutorService;
+
+public interface LocalAwareExecutorService extends ExecutorService
+{
+    // we need a way to inject a TraceState directly into the Executor context without going through
+    // the global Tracing sessions; see CASSANDRA-5668
+    public void execute(Runnable command, ExecutorLocals locals);
+
+    // permits executing in the context of the submitting thread
+    public void maybeExecuteImmediately(Runnable command);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index d9a0fa8..8b12b82 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.cassandra.metrics.SEPMetrics;
@@ -30,7 +29,7 @@ import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 
-public class SEPExecutor extends AbstractTracingAwareExecutorService
+public class SEPExecutor extends AbstractLocalAwareExecutorService
 {
     private final SharedExecutorPool pool;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index 8c18c44..dfd7011 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -103,7 +103,7 @@ public class SharedExecutorPool
             schedule(Work.SPINNING);
     }
 
-    public TracingAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
+    public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
     {
         SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name);
         executors.add(executor);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 4f03fd5..114795e 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -39,7 +39,7 @@ public class StageManager
 {
     private static final Logger logger = LoggerFactory.getLogger(StageManager.class);
 
-    private static final EnumMap<Stage, TracingAwareExecutorService> stages = new EnumMap<Stage, TracingAwareExecutorService>(Stage.class);
+    private static final EnumMap<Stage, LocalAwareExecutorService> stages = new EnumMap<Stage, LocalAwareExecutorService>(Stage.class);
 
     public static final long KEEPALIVE = 60; // seconds to keep "extra" threads alive for when idle
 
@@ -87,7 +87,7 @@ public class StageManager
                                                 stage.getJmxType());
     }
 
-    private static TracingAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
+    private static LocalAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
     {
         return SharedExecutorPool.SHARED.newExecutor(numThreads, Integer.MAX_VALUE, stage.getJmxType(), stage.getJmxName());
     }
@@ -96,7 +96,7 @@ public class StageManager
      * Retrieve a stage from the StageManager
      * @param stage name of the stage to be retrieved.
      */
-    public static TracingAwareExecutorService getStage(Stage stage)
+    public static LocalAwareExecutorService getStage(Stage stage)
     {
         return stages.get(stage);
     }
@@ -116,16 +116,16 @@ public class StageManager
      * A TPE that disallows submit so that we don't need to worry about unwrapping exceptions on the
      * tracing stage.  See CASSANDRA-1123 for background.
      */
-    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
     {
         public ExecuteOnlyExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
         {
             super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
         }
 
-        public void execute(Runnable command, TraceState state)
+        public void execute(Runnable command, ExecutorLocals locals)
         {
-            assert state == null;
+            assert locals == null;
             super.execute(command);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
deleted file mode 100644
index f580fea..0000000
--- a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 
- * 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.concurrent;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.cassandra.tracing.TraceState;
-
-public interface TracingAwareExecutorService extends ExecutorService
-{
-    // we need a way to inject a TraceState directly into the Executor context without going through
-    // the global Tracing sessions; see CASSANDRA-5668
-    public void execute(Runnable command, TraceState state);
-
-    // permits executing in the context of the submitting thread
-    public void maybeExecuteImmediately(Runnable command);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 43a80bb..a289ad1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -273,7 +273,7 @@ public class BatchStatement implements CQLStatement
             {
                 logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
             }
-            ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {ksCfPairs, size, warnThreshold, size - warnThreshold, ""}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
         }
     }
 
@@ -305,8 +305,13 @@ public class BatchStatement implements CQLStatement
                              keySet.size(), keySet.size() == 1 ? "" : "s",
                              ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
 
-            ClientWarn.warn(MessageFormatter.arrayFormat(unloggedBatchWarning, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
-                                                    ksCfPairs.size() == 1 ? "" : "s", ksCfPairs}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
+                                                                  new Object[]{
+                                                                              keySet.size(),
+                                                                              keySet.size() == 1 ? "" : "s",
+                                                                              ksCfPairs.size() == 1 ? "" : "s",
+                                                                              ksCfPairs
+                                                                  }).getMessage());
 
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 5f142ce..5cfa94b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -251,12 +251,12 @@ public class SelectStatement implements CQLStatement
         if (!restrictions.hasPartitionKeyRestrictions())
         {
             logger.warn("Aggregation query used without partition key");
-            ClientWarn.warn("Aggregation query used without partition key");
+            ClientWarn.instance.warn("Aggregation query used without partition key");
         }
         else if (restrictions.keyIsInRelation())
         {
             logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-            ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+            ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
         }
 
         Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index e1a68e7..d2f0bf4 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -303,7 +303,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
                                        container.metadata().getKeyValidator().getString(key.getKey()),
                                        count,
                                        getSlicesInfo(container));
-            ClientWarn.warn(msg);
+            ClientWarn.instance.warn(msg);
             logger.warn(msg);
         }
         Tracing.trace("Read {} live and {} tombstone cells{}",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 61e58c2..459923b 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -43,10 +43,12 @@ import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.concurrent.ExecutorLocals;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.*;
@@ -801,10 +803,10 @@ public final class MessagingService implements MessagingServiceMBean
                 return;
 
         Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
-        TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+        LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
         assert stage != null : "No stage for message type " + message.verb;
 
-        stage.execute(runnable, state);
+        stage.execute(runnable, ExecutorLocals.create(state));
     }
 
     public void setCallbackForTests(int messageId, CallbackInfo callback)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/ClientWarn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java
index 2ed0a6c..ddad197 100644
--- a/src/java/org/apache/cassandra/service/ClientWarn.java
+++ b/src/java/org/apache/cassandra/service/ClientWarn.java
@@ -20,54 +20,68 @@ package org.apache.cassandra.service;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class ClientWarn
+public class ClientWarn implements ExecutorLocal<ClientWarn.State>
 {
     private static final String TRUNCATED = " [truncated]";
-    private static final ThreadLocal<ClientWarn> warnLocal = new ThreadLocal<>();
-
-    private final List<String> warnings = new ArrayList<>();
+    private static final ThreadLocal<ClientWarn.State> warnLocal = new ThreadLocal<>();
+    public static ClientWarn instance = new ClientWarn();
 
     private ClientWarn()
     {
     }
 
-    public static void warn(String text)
-    {
-        ClientWarn warner = warnLocal.get();
-        if (warner != null)
-            warner.add(text);
+    public State get() {
+        return warnLocal.get();
     }
 
-    private void add(String warning)
-    {
-        if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
-            warnings.add(maybeTruncate(warning));
+    public void set(State value) {
+        warnLocal.set(value);
     }
 
-    private static String maybeTruncate(String warning)
+    public void warn(String text)
     {
-        return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
-             ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
-             : warning;
+        State state = warnLocal.get();
+        if (state != null)
+            state.add(text);
     }
 
-    public static void captureWarnings()
+    public void captureWarnings()
     {
-        warnLocal.set(new ClientWarn());
+        warnLocal.set(new State());
     }
 
-    public static List<String> getWarnings()
+    public List<String> getWarnings()
     {
-        ClientWarn warner = warnLocal.get();
-        if (warner == null || warner.warnings.isEmpty())
+        State state = warnLocal.get();
+        if (state == null || state.warnings.isEmpty())
             return null;
-        return warner.warnings;
+        return state.warnings;
     }
 
-    public static void resetWarnings()
+    public void resetWarnings()
     {
         warnLocal.remove();
     }
+
+    public static class State
+    {
+        private final List<String> warnings = new ArrayList<>();
+
+        private void add(String warning)
+        {
+            if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
+                warnings.add(maybeTruncate(warning));
+        }
+
+        private static String maybeTruncate(String warning)
+        {
+            return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
+                   ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
+                   : warning;
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 88253e3..841e980 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -1842,7 +1842,7 @@ public class StorageProxy implements StorageProxyMBean
                     if (filteredEndpoints.size() == 1
                         && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
                     {
-                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler), Tracing.instance.get());
+                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index ccc2637..bf9cee7 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.net.MessageIn;
@@ -41,7 +42,7 @@ import org.apache.cassandra.utils.UUIDGen;
  * A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
  * have multiple local and remote events before it is completed. All events and sessions are stored at keyspace.
  */
-public class Tracing
+public class Tracing implements ExecutorLocal<TraceState>
 {
     public static final String TRACE_HEADER = "TraceSession";
     public static final String TRACE_TYPE = "TraceType";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index ab794df..01a0794 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -499,14 +499,14 @@ public abstract class Message
                 assert request.connection() instanceof ServerConnection;
                 connection = (ServerConnection)request.connection();
                 if (connection.getVersion() >= Server.VERSION_4)
-                    ClientWarn.captureWarnings();
+                    ClientWarn.instance.captureWarnings();
 
                 QueryState qstate = connection.validateNewMessage(request.type, connection.getVersion(), request.getStreamId());
 
                 logger.trace("Received: {}, v={}", request, connection.getVersion());
                 response = request.execute(qstate);
                 response.setStreamId(request.getStreamId());
-                response.setWarnings(ClientWarn.getWarnings());
+                response.setWarnings(ClientWarn.instance.getWarnings());
                 response.attach(connection);
                 connection.applyStateTransition(request.type, response.type);
             }
@@ -519,7 +519,7 @@ public abstract class Message
             }
             finally
             {
-                ClientWarn.resetWarnings();
+                ClientWarn.instance.resetWarnings();
             }
 
             logger.trace("Responding: {}, v={}", response, connection.getVersion());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
index 4ecd6a7..289f3e3 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
 import io.netty.util.concurrent.AbstractEventExecutor;
 import io.netty.util.concurrent.EventExecutorGroup;
 import io.netty.util.concurrent.Future;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED;
@@ -32,7 +32,7 @@ public class RequestThreadPoolExecutor extends AbstractEventExecutor
 {
     private final static int MAX_QUEUED_REQUESTS = 128;
     private final static String THREAD_FACTORY_ID = "Native-Transport-Requests";
-    private final TracingAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
+    private final LocalAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
                                                                            MAX_QUEUED_REQUESTS,
                                                                            "transport",
                                                                            THREAD_FACTORY_ID);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index cfd5f7a..d22a8f6 100644
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@ -24,6 +24,8 @@ import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.transport.Server;
@@ -79,6 +81,47 @@ public class ClientWarningsTest extends CQLTester
     }
 
     @Test
+    public void testTombstoneWarning() throws Exception
+    {
+        final int iterations = 10000;
+        createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+            store.forceBlockingFlush();
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            store.forceBlockingFlush();
+
+            {
+                QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                    KEYSPACE,
+                                                                    currentTable()), QueryOptions.DEFAULT);
+                Message.Response resp = client.execute(query);
+                assertEquals(1, resp.getWarnings().size());
+            }
+        }
+    }
+
+    @Test
     public void testLargeBatchWithProtoV2() throws Exception
     {
         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");


[08/13] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ca...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.0
Commit: 94e7ef17757235cb35df70ff9a2a63e1d29d6c41
Parents: 0f995a2 dbf6e62
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 14:45:13 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:45:13 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 230 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 230 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cassandra/cql3/functions/UDFunction.java    |   2 +-
 .../cql3/statements/BatchStatement.java         |   9 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   7 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |   6 +-
 .../cassandra/service/ClientWarningsTest.java   |  58 +++++
 24 files changed, 545 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 614d5b4,6530956..a37ec99
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,21 @@@
 -2.2.5
 +3.0.3
 + * Fix AssertionError when removing from list using UPDATE (CASSANDRA-10954)
 + * Fix UnsupportedOperationException when reading old sstable with range
 +   tombstone (CASSANDRA-10743)
 + * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
 + * Fix potential assertion error during compaction (CASSANDRA-10944)
 + * Fix counting of received sstables in streaming (CASSANDRA-10949)
 + * Implement hints compression (CASSANDRA-9428)
 + * Fix potential assertion error when reading static columns (CASSANDRA-10903)
 + * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711)
 + * Avoid building PartitionUpdate in toString (CASSANDRA-10897)
 + * Reduce heap spent when receiving many SSTables (CASSANDRA-10797)
 + * Add back support for 3rd party auth providers to bulk loader (CASSANDRA-10873)
 + * Eliminate the dependency on jgrapht for UDT resolution (CASSANDRA-10653)
 + * (Hadoop) Close Clusters and Sessions in Hadoop Input/Output classes (CASSANDRA-10837)
 + * Fix sstableloader not working with upper case keyspace name (CASSANDRA-10806)
 +Merged from 2.2:
+  * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
   * Fix error streaming section more than 2GB (CASSANDRA-10961)
   * (cqlsh) Also apply --connect-timeout to control connection
     timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
index 0000000,088b43e..f47d8ac
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@@ -1,0 -1,229 +1,230 @@@
+ /*
+  * 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.concurrent;
+ 
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.TimeoutException;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import org.apache.cassandra.tracing.TraceState;
+ import org.apache.cassandra.tracing.Tracing;
+ import org.apache.cassandra.utils.concurrent.SimpleCondition;
+ import org.apache.cassandra.utils.JVMStabilityInspector;
+ 
+ import static org.apache.cassandra.tracing.Tracing.isTracing;
+ 
+ public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+ {
+     private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+ 
+     protected abstract void addTask(FutureTask<?> futureTask);
+     protected abstract void onCompletion();
+ 
+     /** Task Submission / Creation / Objects **/
+ 
+     public <T> FutureTask<T> submit(Callable<T> task)
+     {
+         return submit(newTaskFor(task));
+     }
+ 
+     public FutureTask<?> submit(Runnable task)
+     {
+         return submit(newTaskFor(task, null));
+     }
+ 
+     public <T> FutureTask<T> submit(Runnable task, T result)
+     {
+         return submit(newTaskFor(task, result));
+     }
+ 
+     public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+     {
+         return newTaskFor(runnable, result, ExecutorLocals.create());
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+     {
+         if (locals != null)
+         {
+             if (runnable instanceof LocalSessionFutureTask)
+                 return (LocalSessionFutureTask<T>) runnable;
+             return new LocalSessionFutureTask<T>(runnable, result, locals);
+         }
+         if (runnable instanceof FutureTask)
+             return (FutureTask<T>) runnable;
+         return new FutureTask<>(runnable, result);
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+     {
+         if (isTracing())
+         {
+             if (callable instanceof LocalSessionFutureTask)
+                 return (LocalSessionFutureTask<T>) callable;
+             return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+         }
+         if (callable instanceof FutureTask)
+             return (FutureTask<T>) callable;
+         return new FutureTask<>(callable);
+     }
+ 
+     private class LocalSessionFutureTask<T> extends FutureTask<T>
+     {
+         private final ExecutorLocals locals;
+ 
+         public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+         {
+             super(callable);
+             this.locals = locals;
+         }
+ 
+         public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+         {
+             super(runnable, result);
+             this.locals = locals;
+         }
+ 
+         public void run()
+         {
+             ExecutorLocals old = ExecutorLocals.create();
+             ExecutorLocals.set(locals);
+             try
+             {
+                 super.run();
+             }
+             finally
+             {
+                 ExecutorLocals.set(old);
+             }
+         }
+     }
+ 
+     class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+     {
+         private boolean failure;
+         private Object result = this;
+         private final Callable<T> callable;
+ 
+         public FutureTask(Callable<T> callable)
+         {
+             this.callable = callable;
+         }
+         public FutureTask(Runnable runnable, T result)
+         {
+             this(Executors.callable(runnable, result));
+         }
+ 
+         public void run()
+         {
+             try
+             {
+                 result = callable.call();
+             }
+             catch (Throwable t)
+             {
+                 JVMStabilityInspector.inspectThrowable(t);
+                 logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                 result = t;
+                 failure = true;
+             }
+             finally
+             {
+                 signalAll();
+                 onCompletion();
+             }
+         }
+ 
+         public boolean cancel(boolean mayInterruptIfRunning)
+         {
+             return false;
+         }
+ 
+         public boolean isCancelled()
+         {
+             return false;
+         }
+ 
+         public boolean isDone()
+         {
+             return isSignaled();
+         }
+ 
+         public T get() throws InterruptedException, ExecutionException
+         {
+             await();
+             Object result = this.result;
+             if (failure)
+                 throw new ExecutionException((Throwable) result);
+             return (T) result;
+         }
+ 
+         public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+         {
 -            await(timeout, unit);
++            if (!await(timeout, unit))
++                throw new TimeoutException();
+             Object result = this.result;
+             if (failure)
+                 throw new ExecutionException((Throwable) result);
+             return (T) result;
+         }
+     }
+ 
+     private <T> FutureTask<T> submit(FutureTask<T> task)
+     {
+         addTask(task);
+         return task;
+     }
+ 
+     public void execute(Runnable command)
+     {
+         addTask(newTaskFor(command, ExecutorLocals.create()));
+     }
+ 
+     public void execute(Runnable command, ExecutorLocals locals)
+     {
+         addTask(newTaskFor(command, null, locals));
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 04a4c3d,1e5cea6..fa0d306
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@@ -269,164 -143,11 +269,164 @@@ public abstract class UDFunction extend
              return null;
  
          long tStart = System.nanoTime();
 -        ByteBuffer result = executeUserDefined(protocolVersion, parameters);
 -        Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 -        return result;
 +        parameters = makeEmptyParametersNull(parameters);
 +
 +        try
 +        {
 +            // Using async UDF execution is expensive (adds about 100us overhead per invocation on a Core-i7 MBPr).
 +            ByteBuffer result = DatabaseDescriptor.enableUserDefinedFunctionsThreads()
 +                                ? executeAsync(protocolVersion, parameters)
 +                                : executeUserDefined(protocolVersion, parameters);
 +
 +            Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 +            return result;
 +        }
 +        catch (InvalidRequestException e)
 +        {
 +            throw e;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.trace("Invocation of user-defined function '{}' failed", this, t);
 +            if (t instanceof VirtualMachineError)
 +                throw (VirtualMachineError) t;
 +            throw FunctionExecutionException.create(this, t);
 +        }
      }
  
 +    public static void assertUdfsEnabled(String language)
 +    {
 +        if (!DatabaseDescriptor.enableUserDefinedFunctions())
 +            throw new InvalidRequestException("User-defined functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable");
 +        if (!"java".equalsIgnoreCase(language) && !DatabaseDescriptor.enableScriptedUserDefinedFunctions())
 +            throw new InvalidRequestException("Scripted user-defined functions are disabled in cassandra.yaml - set enable_scripted_user_defined_functions=true to enable if you are aware of the security risks");
 +    }
 +
 +    static void initializeThread()
 +    {
 +        // Get the TypeCodec stuff in Java Driver initialized.
 +        // This is to get the classes loaded outside of the restricted sandbox's security context of a UDF.
 +        UDHelper.codecFor(DataType.inet()).format(InetAddress.getLoopbackAddress());
 +        UDHelper.codecFor(DataType.ascii()).format("");
 +    }
 +
 +    private static final class ThreadIdAndCpuTime extends CompletableFuture<Object>
 +    {
 +        long threadId;
 +        long cpuTime;
 +
 +        ThreadIdAndCpuTime()
 +        {
 +            // Looks weird?
 +            // This call "just" links this class to java.lang.management - otherwise UDFs (script UDFs) might fail due to
 +            //      java.security.AccessControlException: access denied: ("java.lang.RuntimePermission" "accessClassInPackage.java.lang.management")
 +            // because class loading would be deferred until setup() is executed - but setup() is called with
 +            // limited privileges.
 +            threadMXBean.getCurrentThreadCpuTime();
 +        }
 +
 +        void setup()
 +        {
 +            this.threadId = Thread.currentThread().getId();
 +            this.cpuTime = threadMXBean.getCurrentThreadCpuTime();
 +            complete(null);
 +        }
 +    }
 +
 +    private ByteBuffer executeAsync(int protocolVersion, List<ByteBuffer> parameters)
 +    {
 +        ThreadIdAndCpuTime threadIdAndCpuTime = new ThreadIdAndCpuTime();
 +
 +        Future<ByteBuffer> future = executor().submit(() -> {
 +            threadIdAndCpuTime.setup();
 +            return executeUserDefined(protocolVersion, parameters);
 +        });
 +
 +        try
 +        {
 +            if (DatabaseDescriptor.getUserDefinedFunctionWarnTimeout() > 0)
 +                try
 +                {
 +                    return future.get(DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +                }
 +                catch (TimeoutException e)
 +                {
 +
 +                    // log and emit a warning that UDF execution took long
 +                    String warn = String.format("User defined function %s ran longer than %dms", this, DatabaseDescriptor.getUserDefinedFunctionWarnTimeout());
 +                    logger.warn(warn);
-                     ClientWarn.warn(warn);
++                    ClientWarn.instance.warn(warn);
 +                }
 +
 +            // retry with difference of warn-timeout to fail-timeout
 +            return future.get(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            Thread.currentThread().interrupt();
 +            throw new RuntimeException(e);
 +        }
 +        catch (ExecutionException e)
 +        {
 +            Throwable c = e.getCause();
 +            if (c instanceof RuntimeException)
 +                throw (RuntimeException) c;
 +            throw new RuntimeException(c);
 +        }
 +        catch (TimeoutException e)
 +        {
 +            // retry a last time with the difference of UDF-fail-timeout to consumed CPU time (just in case execution hit a badly timed GC)
 +            try
 +            {
 +                //The threadIdAndCpuTime shouldn't take a long time to be set so this should return immediately
 +                threadIdAndCpuTime.get(1, TimeUnit.SECONDS);
 +
 +                long cpuTimeMillis = threadMXBean.getThreadCpuTime(threadIdAndCpuTime.threadId) - threadIdAndCpuTime.cpuTime;
 +                cpuTimeMillis /= 1000000L;
 +
 +                return future.get(Math.max(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - cpuTimeMillis, 0L),
 +                                  TimeUnit.MILLISECONDS);
 +            }
 +            catch (InterruptedException e1)
 +            {
 +                Thread.currentThread().interrupt();
 +                throw new RuntimeException(e);
 +            }
 +            catch (ExecutionException e1)
 +            {
 +                Throwable c = e.getCause();
 +                if (c instanceof RuntimeException)
 +                    throw (RuntimeException) c;
 +                throw new RuntimeException(c);
 +            }
 +            catch (TimeoutException e1)
 +            {
 +                TimeoutException cause = new TimeoutException(String.format("User defined function %s ran longer than %dms%s",
 +                                                                            this,
 +                                                                            DatabaseDescriptor.getUserDefinedFunctionFailTimeout(),
 +                                                                            DatabaseDescriptor.getUserFunctionTimeoutPolicy() == Config.UserFunctionTimeoutPolicy.ignore
 +                                                                            ? "" : " - will stop Cassandra VM"));
 +                FunctionExecutionException fe = FunctionExecutionException.create(this, cause);
 +                JVMStabilityInspector.userFunctionTimeout(cause);
 +                throw fe;
 +            }
 +        }
 +    }
 +
 +    private List<ByteBuffer> makeEmptyParametersNull(List<ByteBuffer> parameters)
 +    {
 +        List<ByteBuffer> r = new ArrayList<>(parameters.size());
 +        for (int i = 0; i < parameters.size(); i++)
 +        {
 +            ByteBuffer param = parameters.get(i);
 +            r.add(UDHelper.isNullOrEmpty(argTypes.get(i), param)
 +                  ? null : param);
 +        }
 +        return r;
 +    }
 +
 +    protected abstract ExecutorService executor();
 +
      public boolean isCallableWrtNullable(List<ByteBuffer> parameters)
      {
          if (!calledOnNullInput)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 3979597,a289ad1..47396fb
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@@ -226,35 -179,67 +226,35 @@@ public class BatchStatement implements 
          for (int i = 0; i < statements.size(); i++)
          {
              ModificationStatement statement = statements.get(i);
 +            if (isLogged() && statement.cfm.params.gcGraceSeconds == 0)
 +            {
 +                if (tablesWithZeroGcGs == null)
 +                    tablesWithZeroGcGs = new HashSet<>();
 +                tablesWithZeroGcGs.add(String.format("%s.%s", statement.cfm.ksName, statement.cfm.cfName));
 +            }
              QueryOptions statementOptions = options.forStatement(i);
              long timestamp = attrs.getTimestamp(now, statementOptions);
 -            addStatementMutations(statement, statementOptions, local, timestamp, mutations);
 +            statement.addUpdates(collector, statementOptions, local, timestamp);
          }
 -        return unzipMutations(mutations);
 -    }
 -
 -    private Collection<? extends IMutation> unzipMutations(Map<String, Map<ByteBuffer, IMutation>> mutations)
 -    {
 -
 -        // The case where all statement where on the same keyspace is pretty common
 -        if (mutations.size() == 1)
 -            return mutations.values().iterator().next().values();
 -
  
 -        List<IMutation> ms = new ArrayList<>();
 -        for (Map<ByteBuffer, IMutation> ksMap : mutations.values())
 -            ms.addAll(ksMap.values());
 -
 -        return ms;
 -    }
 -
 -    private void addStatementMutations(ModificationStatement statement,
 -                                       QueryOptions options,
 -                                       boolean local,
 -                                       long now,
 -                                       Map<String, Map<ByteBuffer, IMutation>> mutations)
 -    throws RequestExecutionException, RequestValidationException
 -    {
 -        String ksName = statement.keyspace();
 -        Map<ByteBuffer, IMutation> ksMap = mutations.get(ksName);
 -        if (ksMap == null)
 +        if (tablesWithZeroGcGs != null)
          {
 -            ksMap = new HashMap<>();
 -            mutations.put(ksName, ksMap);
 +            String suffix = tablesWithZeroGcGs.size() == 1 ? "" : "s";
 +            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, LOGGED_BATCH_LOW_GCGS_WARNING,
 +                             suffix, tablesWithZeroGcGs);
-             ClientWarn.warn(MessageFormatter.arrayFormat(LOGGED_BATCH_LOW_GCGS_WARNING, new Object[] { suffix, tablesWithZeroGcGs })
-                                             .getMessage());
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(LOGGED_BATCH_LOW_GCGS_WARNING, new Object[] { suffix, tablesWithZeroGcGs })
++                                                     .getMessage());
          }
  
 -        // The following does the same than statement.getMutations(), but we inline it here because
 -        // we don't want to recreate mutations every time as this is particularly inefficient when applying
 -        // multiple batch to the same partition (see #6737).
 -        List<ByteBuffer> keys = statement.buildPartitionKeyNames(options);
 -        Composite clusteringPrefix = statement.createClusteringPrefix(options);
 -        UpdateParameters params = statement.makeUpdateParameters(keys, clusteringPrefix, options, local, now);
 -
 -        for (ByteBuffer key : keys)
 -        {
 -            IMutation mutation = ksMap.get(key);
 -            Mutation mut;
 -            if (mutation == null)
 -            {
 -                mut = new Mutation(ksName, key);
 -                mutation = statement.cfm.isCounter() ? new CounterMutation(mut, options.getConsistency()) : mut;
 -                ksMap.put(key, mutation);
 -            }
 -            else
 -            {
 -                mut = statement.cfm.isCounter() ? ((CounterMutation) mutation).getMutation() : (Mutation) mutation;
 -            }
 +        collector.validateIndexedColumns();
 +        return collector.toMutations();
 +    }
  
 -            statement.addUpdateForKey(mut.addOrGet(statement.cfm), key, clusteringPrefix, params);
 -        }
 +    private int updatedRows()
 +    {
 +        // Note: it's possible for 2 statements to actually apply to the same row, but that's just an estimation
 +        // for sizing our PartitionUpdate backing array, so it's good enough.
 +        return statements.size();
      }
  
      /**
@@@ -286,9 -271,9 +286,9 @@@
              }
              else if (logger.isWarnEnabled())
              {
 -                logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
 +                logger.warn(format, tableNames, size, warnThreshold, size - warnThreshold, "");
              }
-             ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage());
 -            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage());
          }
      }
  
@@@ -311,17 -298,21 +311,16 @@@
              }
  
              // CASSANDRA-9303: If we only have local mutations we do not warn
 -            if (localMutationsOnly)
 +            if (localPartitionsOnly)
                  return;
  
 -            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, unloggedBatchWarning,
 -                             keySet.size(), keySet.size() == 1 ? "" : "s",
 -                             ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
  
 -            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
 -                                                                  new Object[]{
 -                                                                              keySet.size(),
 -                                                                              keySet.size() == 1 ? "" : "s",
 -                                                                              ksCfPairs.size() == 1 ? "" : "s",
 -                                                                              ksCfPairs
 -                                                                  }).getMessage());
 +            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, UNLOGGED_BATCH_WARNING,
 +                             keySet.size(), keySet.size() == 1 ? "" : "s",
 +                             tableNames.size() == 1 ? "" : "s", tableNames);
  
-             ClientWarn.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
 +                                                    tableNames.size() == 1 ? "" : "s", tableNames}).getMessage());
- 
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 4017ce6,0000000..5af4887
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@@ -1,330 -1,0 +1,330 @@@
 +/*
 + * 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.cql3.statements;
 +
 +import java.util.*;
 +import java.util.stream.Collectors;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.auth.Permission;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.ViewDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 +import org.apache.cassandra.cql3.selection.RawSelector;
 +import org.apache.cassandra.cql3.selection.Selectable;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.ReversedType;
 +import org.apache.cassandra.db.view.View;
 +import org.apache.cassandra.exceptions.AlreadyExistsException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.exceptions.RequestValidationException;
 +import org.apache.cassandra.exceptions.UnauthorizedException;
 +import org.apache.cassandra.schema.TableParams;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.ClientWarn;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.thrift.ThriftValidation;
 +import org.apache.cassandra.transport.Event;
 +
 +public class CreateViewStatement extends SchemaAlteringStatement
 +{
 +    private final CFName baseName;
 +    private final List<RawSelector> selectClause;
 +    private final WhereClause whereClause;
 +    private final List<ColumnIdentifier.Raw> partitionKeys;
 +    private final List<ColumnIdentifier.Raw> clusteringKeys;
 +    public final CFProperties properties = new CFProperties();
 +    private final boolean ifNotExists;
 +
 +    public CreateViewStatement(CFName viewName,
 +                               CFName baseName,
 +                               List<RawSelector> selectClause,
 +                               WhereClause whereClause,
 +                               List<ColumnIdentifier.Raw> partitionKeys,
 +                               List<ColumnIdentifier.Raw> clusteringKeys,
 +                               boolean ifNotExists)
 +    {
 +        super(viewName);
 +        this.baseName = baseName;
 +        this.selectClause = selectClause;
 +        this.whereClause = whereClause;
 +        this.partitionKeys = partitionKeys;
 +        this.clusteringKeys = clusteringKeys;
 +        this.ifNotExists = ifNotExists;
 +    }
 +
 +
 +    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
 +    {
 +        if (!baseName.hasKeyspace())
 +            baseName.setKeyspace(keyspace(), true);
 +        state.hasColumnFamilyAccess(keyspace(), baseName.getColumnFamily(), Permission.ALTER);
 +    }
 +
 +    public void validate(ClientState state) throws RequestValidationException
 +    {
 +        // We do validation in announceMigration to reduce doubling up of work
 +    }
 +
 +    private interface AddColumn {
 +        void add(ColumnIdentifier identifier, AbstractType<?> type);
 +    }
 +
 +    private void add(CFMetaData baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
 +    {
 +        for (ColumnIdentifier column : columns)
 +        {
 +            AbstractType<?> type = baseCfm.getColumnDefinition(column).type;
 +            if (properties.definedOrdering.containsKey(column))
 +            {
 +                boolean desc = properties.definedOrdering.get(column);
 +                if (!desc && type.isReversed())
 +                {
 +                    type = ((ReversedType)type).baseType;
 +                }
 +                else if (desc && !type.isReversed())
 +                {
 +                    type = ReversedType.getInstance(type);
 +                }
 +            }
 +            adder.add(column, type);
 +        }
 +    }
 +
 +    public Event.SchemaChange announceMigration(boolean isLocalOnly) throws RequestValidationException
 +    {
 +        // We need to make sure that:
 +        //  - primary key includes all columns in base table's primary key
 +        //  - make sure that the select statement does not have anything other than columns
 +        //    and their names match the base table's names
 +        //  - make sure that primary key does not include any collections
 +        //  - make sure there is no where clause in the select statement
 +        //  - make sure there is not currently a table or view
 +        //  - make sure baseTable gcGraceSeconds > 0
 +
 +        properties.validate();
 +
 +        if (properties.useCompactStorage)
 +            throw new InvalidRequestException("Cannot use 'COMPACT STORAGE' when defining a materialized view");
 +
 +        // We enforce the keyspace because if the RF is different, the logic to wait for a
 +        // specific replica would break
 +        if (!baseName.getKeyspace().equals(keyspace()))
 +            throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
 +
 +        CFMetaData cfm = ThriftValidation.validateColumnFamily(baseName.getKeyspace(), baseName.getColumnFamily());
 +
 +        if (cfm.isCounter())
 +            throw new InvalidRequestException("Materialized views are not supported on counter tables");
 +        if (cfm.isView())
 +            throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
 +
 +        if (cfm.params.gcGraceSeconds == 0)
 +        {
 +            throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
 +                                                            "'%s' with gc_grace_seconds of 0, since this value is " +
 +                                                            "used to TTL undelivered updates. Setting gc_grace_seconds" +
 +                                                            " too low might cause undelivered updates to expire " +
 +                                                            "before being replayed.", cfName.getColumnFamily(),
 +                                                            baseName.getColumnFamily()));
 +        }
 +
 +        Set<ColumnIdentifier> included = new HashSet<>();
 +        for (RawSelector selector : selectClause)
 +        {
 +            Selectable.Raw selectable = selector.selectable;
 +            if (selectable instanceof Selectable.WithFieldSelection.Raw)
 +                throw new InvalidRequestException("Cannot select out a part of type when defining a materialized view");
 +            if (selectable instanceof Selectable.WithFunction.Raw)
 +                throw new InvalidRequestException("Cannot use function when defining a materialized view");
 +            if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
 +                throw new InvalidRequestException("Cannot use function when defining a materialized view");
 +            ColumnIdentifier identifier = (ColumnIdentifier) selectable.prepare(cfm);
 +            if (selector.alias != null)
 +                throw new InvalidRequestException(String.format("Cannot alias column '%s' as '%s' when defining a materialized view", identifier.toString(), selector.alias.toString()));
 +
 +            ColumnDefinition cdef = cfm.getColumnDefinition(identifier);
 +
 +            if (cdef == null)
 +                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
 +
 +            if (cdef.isStatic())
-                 ClientWarn.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
++                ClientWarn.instance.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
 +            else
 +                included.add(identifier);
 +        }
 +
 +        Set<ColumnIdentifier.Raw> targetPrimaryKeys = new HashSet<>();
 +        for (ColumnIdentifier.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
 +        {
 +            if (!targetPrimaryKeys.add(identifier))
 +                throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
 +
 +            ColumnDefinition cdef = cfm.getColumnDefinition(identifier.prepare(cfm));
 +
 +            if (cdef == null)
 +                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
 +
 +            if (cfm.getColumnDefinition(identifier.prepare(cfm)).type.isMultiCell())
 +                throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
 +
 +            if (cdef.isStatic())
 +                throw new InvalidRequestException(String.format("Cannot use Static column '%s' in PRIMARY KEY of materialized view", identifier));
 +        }
 +
 +        // build the select statement
 +        Map<ColumnIdentifier.Raw, Boolean> orderings = Collections.emptyMap();
 +        SelectStatement.Parameters parameters = new SelectStatement.Parameters(orderings, false, true, false);
 +        SelectStatement.RawStatement rawSelect = new SelectStatement.RawStatement(baseName, parameters, selectClause, whereClause, null);
 +
 +        ClientState state = ClientState.forInternalCalls();
 +        state.setKeyspace(keyspace());
 +
 +        rawSelect.prepareKeyspace(state);
 +        rawSelect.setBoundVariables(getBoundVariables());
 +
 +        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
 +        SelectStatement select = (SelectStatement) prepared.statement;
 +        StatementRestrictions restrictions = select.getRestrictions();
 +
 +        if (!prepared.boundNames.isEmpty())
 +            throw new InvalidRequestException("Cannot use query parameters in CREATE MATERIALIZED VIEW statements");
 +
 +        if (!restrictions.nonPKRestrictedColumns(false).isEmpty())
 +        {
 +            throw new InvalidRequestException(String.format(
 +                    "Non-primary key columns cannot be restricted in the SELECT statement used for materialized view " +
 +                    "creation (got restrictions on: %s)",
 +                    restrictions.nonPKRestrictedColumns(false).stream().map(def -> def.name.toString()).collect(Collectors.joining(", "))));
 +        }
 +
 +        String whereClauseText = View.relationsToWhereClause(whereClause.relations);
 +
 +        Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
 +        for (ColumnDefinition definition : Iterables.concat(cfm.partitionKeyColumns(), cfm.clusteringColumns()))
 +            basePrimaryKeyCols.add(definition.name);
 +
 +        List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
 +        List<ColumnIdentifier> targetPartitionKeys = new ArrayList<>();
 +
 +        // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
 +        boolean hasNonPKColumn = false;
 +        for (ColumnIdentifier.Raw raw : partitionKeys)
 +            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 +
 +        for (ColumnIdentifier.Raw raw : clusteringKeys)
 +            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 +
 +        // We need to include all of the primary key columns from the base table in order to make sure that we do not
 +        // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
 +        // the view because if we need to generate a tombstone, we have no way of knowing which value is currently being
 +        // used in the view and whether or not to generate a tombstone. In order to not surprise our users, we require
 +        // that they include all of the columns. We provide them with a list of all of the columns left to include.
 +        boolean missingClusteringColumns = false;
 +        StringBuilder columnNames = new StringBuilder();
 +        List<ColumnIdentifier> includedColumns = new ArrayList<>();
 +        for (ColumnDefinition def : cfm.allColumns())
 +        {
 +            ColumnIdentifier identifier = def.name;
 +
 +            if ((included.isEmpty() || included.contains(identifier))
 +                && !targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier)
 +                && !def.isStatic())
 +            {
 +                includedColumns.add(identifier);
 +            }
 +            if (!def.isPrimaryKeyColumn()) continue;
 +
 +            if (!targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier))
 +            {
 +                if (missingClusteringColumns)
 +                    columnNames.append(',');
 +                else
 +                    missingClusteringColumns = true;
 +                columnNames.append(identifier);
 +            }
 +        }
 +        if (missingClusteringColumns)
 +            throw new InvalidRequestException(String.format("Cannot create Materialized View %s without primary key columns from base %s (%s)",
 +                                                            columnFamily(), baseName.getColumnFamily(), columnNames.toString()));
 +
 +        if (targetPartitionKeys.isEmpty())
 +            throw new InvalidRequestException("Must select at least a column for a Materialized View");
 +
 +        if (targetClusteringColumns.isEmpty())
 +            throw new InvalidRequestException("No columns are defined for Materialized View other than primary key");
 +
 +        CFMetaData.Builder cfmBuilder = CFMetaData.Builder.createView(keyspace(), columnFamily());
 +        add(cfm, targetPartitionKeys, cfmBuilder::addPartitionKey);
 +        add(cfm, targetClusteringColumns, cfmBuilder::addClusteringColumn);
 +        add(cfm, includedColumns, cfmBuilder::addRegularColumn);
 +        cfmBuilder.withId(properties.properties.getId());
 +        TableParams params = properties.properties.asNewTableParams();
 +        CFMetaData viewCfm = cfmBuilder.build().params(params);
 +        ViewDefinition definition = new ViewDefinition(keyspace(),
 +                                                       columnFamily(),
 +                                                       Schema.instance.getId(keyspace(), baseName.getColumnFamily()),
 +                                                       baseName.getColumnFamily(),
 +                                                       included.isEmpty(),
 +                                                       rawSelect,
 +                                                       whereClauseText,
 +                                                       viewCfm);
 +
 +        try
 +        {
 +            MigrationManager.announceNewView(definition, isLocalOnly);
 +            return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
 +        }
 +        catch (AlreadyExistsException e)
 +        {
 +            if (ifNotExists)
 +                return null;
 +            throw e;
 +        }
 +    }
 +
 +    private static boolean getColumnIdentifier(CFMetaData cfm,
 +                                               Set<ColumnIdentifier> basePK,
 +                                               boolean hasNonPKColumn,
 +                                               ColumnIdentifier.Raw raw,
 +                                               List<ColumnIdentifier> columns,
 +                                               StatementRestrictions restrictions)
 +    {
 +        ColumnIdentifier identifier = raw.prepare(cfm);
 +        ColumnDefinition def = cfm.getColumnDefinition(identifier);
 +
 +        boolean isPk = basePK.contains(identifier);
 +        if (!isPk && hasNonPKColumn)
 +            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", identifier));
 +
 +        // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
 +        // because we will never allow a single partition key to be NULL
 +        boolean isSinglePartitionKey = cfm.getColumnDefinition(identifier).isPartitionKey()
 +                                       && cfm.partitionKeyColumns().size() == 1;
 +        if (!isSinglePartitionKey && !restrictions.isRestricted(def))
 +            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", identifier));
 +
 +        columns.add(identifier);
 +        return !isPk;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index a9bb121,5cfa94b..904adca
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -359,21 -256,19 +359,21 @@@ public class SelectStatement implement
          else if (restrictions.keyIsInRelation())
          {
              logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-             ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+             ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
          }
  
 -        Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);
 +        Selection.ResultSetBuilder result = selection.resultSetBuilder(parameters.isJson);
          while (!pager.isExhausted())
          {
 -            for (Row row : pager.fetchPage(pageSize))
 +            try (PartitionIterator iter = pager.fetchPage(pageSize))
              {
 -                // Not columns match the query, skip
 -                if (row.cf == null)
 -                    continue;
 -
 -                processColumnFamily(row.key.getKey(), row.cf, options, now, result);
 +                while (iter.hasNext())
 +                {
 +                    try (RowIterator partition = iter.next())
 +                    {
 +                        processPartition(partition, options, result, nowInSec);
 +                    }
 +                }
              }
          }
          return new ResultMessage.Rows(result.build(options.getProtocolVersion()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index 3f0695c,cd86336..668a189
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -232,713 -95,55 +232,713 @@@ public abstract class ReadCommand imple
          return this;
      }
  
 -    public String getColumnFamilyName()
 +    /**
 +     * Sets the digest version, for when digest for that command is requested.
 +     * <p>
 +     * Note that we allow setting this independently of setting the command as a digest query as
 +     * this allows us to use the command as a carrier of the digest version even if we only call
 +     * setIsDigestQuery on some copy of it.
 +     *
 +     * @param digestVersion the version for the digest is this command is used for digest query..
 +     * @return this read command.
 +     */
 +    public ReadCommand setDigestVersion(int digestVersion)
      {
 -        return cfName;
 +        this.digestVersion = digestVersion;
 +        return this;
      }
  
 +    /**
 +     * Whether this query is for thrift or not.
 +     *
 +     * @return whether this query is for thrift.
 +     */
 +    public boolean isForThrift()
 +    {
 +        return isForThrift;
 +    }
 +
 +    /**
 +     * The clustering index filter this command to use for the provided key.
 +     * <p>
 +     * Note that that method should only be called on a key actually queried by this command
 +     * and in practice, this will almost always return the same filter, but for the sake of
 +     * paging, the filter on the first key of a range command might be slightly different.
 +     *
 +     * @param key a partition key queried by this command.
 +     *
 +     * @return the {@code ClusteringIndexFilter} to use for the partition of key {@code key}.
 +     */
 +    public abstract ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key);
 +
 +    /**
 +     * Returns a copy of this command.
 +     *
 +     * @return a copy of this command.
 +     */
      public abstract ReadCommand copy();
  
 -    public abstract Row getRow(Keyspace keyspace);
 +    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 +
 +    protected abstract int oldestUnrepairedTombstone();
 +
 +    public ReadResponse createResponse(UnfilteredPartitionIterator iterator, ColumnFilter selection)
 +    {
 +        return isDigestQuery()
 +             ? ReadResponse.createDigestResponse(iterator, digestVersion)
 +             : ReadResponse.createDataResponse(iterator, selection);
 +    }
 +
 +    public long indexSerializedSize(int version)
 +    {
 +        if (index.isPresent())
 +            return IndexMetadata.serializer.serializedSize(index.get(), version);
 +        else
 +            return 0;
 +    }
 +
 +    public Index getIndex(ColumnFamilyStore cfs)
 +    {
 +        // if we've already consulted the index manager, and it returned a valid index
 +        // the result should be cached here.
 +        if(index.isPresent())
 +            return cfs.indexManager.getIndex(index.get());
 +
 +        // if no cached index is present, but we've already consulted the index manager
 +        // then no registered index is suitable for this command, so just return null.
 +        if (indexManagerQueried)
 +            return null;
 +
 +        // do the lookup, set the flag to indicate so and cache the result if not null
 +        Index selected = cfs.indexManager.getBestIndexFor(this);
 +        indexManagerQueried = true;
  
 -    public abstract IDiskAtomFilter filter();
 +        if (selected == null)
 +            return null;
  
 -    public String getKeyspace()
 +        index = Optional.of(selected.getIndexMetadata());
 +        return selected;
 +    }
 +
 +    /**
 +     * Executes this command on the local host.
 +     *
 +     * @param orderGroup the operation group spanning this command
 +     *
 +     * @return an iterator over the result of executing this command locally.
 +     */
 +    @SuppressWarnings("resource") // The result iterator is closed upon exceptions (we know it's fine to potentially not close the intermediary
 +                                  // iterators created inside the try as long as we do close the original resultIterator), or by closing the result.
 +    public UnfilteredPartitionIterator executeLocally(ReadOrderGroup orderGroup)
      {
 -        return ksName;
 +        long startTimeNanos = System.nanoTime();
 +
 +        ColumnFamilyStore cfs = Keyspace.openAndGetStore(metadata());
 +        Index index = getIndex(cfs);
 +
 +        Index.Searcher searcher = null;
 +        if (index != null)
 +        {
 +            if (!cfs.indexManager.isIndexQueryable(index))
 +                throw new IndexNotAvailableException(index);
 +
 +            searcher = index.searcherFor(this);
 +            Tracing.trace("Executing read on {}.{} using index {}", cfs.metadata.ksName, cfs.metadata.cfName, index.getIndexMetadata().name);
 +        }
 +
 +        UnfilteredPartitionIterator resultIterator = searcher == null
 +                                         ? queryStorage(cfs, orderGroup)
 +                                         : searcher.search(orderGroup);
 +
 +        try
 +        {
 +            resultIterator = withMetricsRecording(withoutPurgeableTombstones(resultIterator, cfs), cfs.metric, startTimeNanos);
 +
 +            // If we've used a 2ndary index, we know the result already satisfy the primary expression used, so
 +            // no point in checking it again.
 +            RowFilter updatedFilter = searcher == null
 +                                    ? rowFilter()
 +                                    : index.getPostIndexQueryFilter(rowFilter());
 +
 +            // TODO: We'll currently do filtering by the rowFilter here because it's convenient. However,
 +            // we'll probably want to optimize by pushing it down the layer (like for dropped columns) as it
 +            // would be more efficient (the sooner we discard stuff we know we don't care, the less useless
 +            // processing we do on it).
 +            return limits().filter(updatedFilter.filter(resultIterator, nowInSec()), nowInSec());
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            resultIterator.close();
 +            throw e;
 +        }
      }
  
 -    // maybeGenerateRetryCommand is used to generate a retry for short reads
 -    public ReadCommand maybeGenerateRetryCommand(RowDataResolver resolver, Row row)
 +    protected abstract void recordLatency(TableMetrics metric, long latencyNanos);
 +
 +    public PartitionIterator executeInternal(ReadOrderGroup orderGroup)
      {
 -        return null;
 +        return UnfilteredPartitionIterators.filter(executeLocally(orderGroup), nowInSec());
      }
  
 -    // maybeTrim removes columns from a response that is too long
 -    public Row maybeTrim(Row row)
 +    public ReadOrderGroup startOrderGroup()
      {
 -        return row;
 +        return ReadOrderGroup.forCommand(this);
      }
  
 -    public long getTimeout()
 +    /**
 +     * Wraps the provided iterator so that metrics on what is scanned by the command are recorded.
 +     * This also log warning/trow TombstoneOverwhelmingException if appropriate.
 +     */
 +    private UnfilteredPartitionIterator withMetricsRecording(UnfilteredPartitionIterator iter, final TableMetrics metric, final long startTimeNanos)
      {
 -        return DatabaseDescriptor.getReadRpcTimeout();
 +        class MetricRecording extends Transformation<UnfilteredRowIterator>
 +        {
 +            private final int failureThreshold = DatabaseDescriptor.getTombstoneFailureThreshold();
 +            private final int warningThreshold = DatabaseDescriptor.getTombstoneWarnThreshold();
 +
 +            private final boolean respectTombstoneThresholds = !Schema.isSystemKeyspace(ReadCommand.this.metadata().ksName);
 +
 +            private int liveRows = 0;
 +            private int tombstones = 0;
 +
 +            private DecoratedKey currentKey;
 +
 +            @Override
 +            public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator iter)
 +            {
 +                currentKey = iter.partitionKey();
 +                return Transformation.apply(iter, this);
 +            }
 +
 +            @Override
 +            public Row applyToStatic(Row row)
 +            {
 +                return applyToRow(row);
 +            }
 +
 +            @Override
 +            public Row applyToRow(Row row)
 +            {
 +                if (row.hasLiveData(ReadCommand.this.nowInSec()))
 +                    ++liveRows;
 +
 +                for (Cell cell : row.cells())
 +                {
 +                    if (!cell.isLive(ReadCommand.this.nowInSec()))
 +                        countTombstone(row.clustering());
 +                }
 +                return row;
 +            }
 +
 +            @Override
 +            public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +            {
 +                countTombstone(marker.clustering());
 +                return marker;
 +            }
 +
 +            private void countTombstone(ClusteringPrefix clustering)
 +            {
 +                ++tombstones;
 +                if (tombstones > failureThreshold && respectTombstoneThresholds)
 +                {
 +                    String query = ReadCommand.this.toCQLString();
 +                    Tracing.trace("Scanned over {} tombstones for query {}; query aborted (see tombstone_failure_threshold)", failureThreshold, query);
 +                    throw new TombstoneOverwhelmingException(tombstones, query, ReadCommand.this.metadata(), currentKey, clustering);
 +                }
 +            }
 +
 +            @Override
 +            public void onClose()
 +            {
 +                recordLatency(metric, System.nanoTime() - startTimeNanos);
 +
 +                metric.tombstoneScannedHistogram.update(tombstones);
 +                metric.liveScannedHistogram.update(liveRows);
 +
 +                boolean warnTombstones = tombstones > warningThreshold && respectTombstoneThresholds;
 +                if (warnTombstones)
 +                {
 +                    String msg = String.format("Read %d live rows and %d tombstone cells for query %1.512s (see tombstone_warn_threshold)", liveRows, tombstones, ReadCommand.this.toCQLString());
-                     ClientWarn.warn(msg);
++                    ClientWarn.instance.warn(msg);
 +                    logger.warn(msg);
 +                }
 +
 +                Tracing.trace("Read {} live and {} tombstone cells{}", liveRows, tombstones, (warnTombstones ? " (see tombstone_warn_threshold)" : ""));
 +            }
 +        };
 +
 +        return Transformation.apply(iter, new MetricRecording());
      }
 -}
  
 -class ReadCommandSerializer implements IVersionedSerializer<ReadCommand>
 -{
 -    public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +    /**
 +     * Creates a message for this command.
 +     */
 +    public abstract MessageOut<ReadCommand> createMessage(int version);
 +
 +    protected abstract void appendCQLWhereClause(StringBuilder sb);
 +
 +    // Skip purgeable tombstones. We do this because it's safe to do (post-merge of the memtable and sstable at least), it
 +    // can save us some bandwith, and avoid making us throw a TombstoneOverwhelmingException for purgeable tombstones (which
 +    // are to some extend an artefact of compaction lagging behind and hence counting them is somewhat unintuitive).
 +    protected UnfilteredPartitionIterator withoutPurgeableTombstones(UnfilteredPartitionIterator iterator, ColumnFamilyStore cfs)
 +    {
 +        final boolean isForThrift = iterator.isForThrift();
 +        class WithoutPurgeableTombstones extends PurgeFunction
 +        {
 +            public WithoutPurgeableTombstones()
 +            {
 +                super(isForThrift, nowInSec(), cfs.gcBefore(nowInSec()), oldestUnrepairedTombstone(), cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
 +            }
 +
 +            protected long getMaxPurgeableTimestamp()
 +            {
 +                return Long.MAX_VALUE;
 +            }
 +        }
 +        return Transformation.apply(iterator, new WithoutPurgeableTombstones());
 +    }
 +
 +    /**
 +     * Recreate the CQL string corresponding to this query.
 +     * <p>
 +     * Note that in general the returned string will not be exactly the original user string, first
 +     * because there isn't always a single syntax for a given query,  but also because we don't have
 +     * all the information needed (we know the non-PK columns queried but not the PK ones as internally
 +     * we query them all). So this shouldn't be relied too strongly, but this should be good enough for
 +     * debugging purpose which is what this is for.
 +     */
 +    public String toCQLString()
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        sb.append("SELECT ").append(columnFilter());
 +        sb.append(" FROM ").append(metadata().ksName).append('.').append(metadata.cfName);
 +        appendCQLWhereClause(sb);
 +
 +        if (limits() != DataLimits.NONE)
 +            sb.append(' ').append(limits());
 +        return sb.toString();
 +    }
 +
 +    private static class Serializer implements IVersionedSerializer<ReadCommand>
 +    {
 +        private static int digestFlag(boolean isDigest)
 +        {
 +            return isDigest ? 0x01 : 0;
 +        }
 +
 +        private static boolean isDigest(int flags)
 +        {
 +            return (flags & 0x01) != 0;
 +        }
 +
 +        private static int thriftFlag(boolean isForThrift)
 +        {
 +            return isForThrift ? 0x02 : 0;
 +        }
 +
 +        private static boolean isForThrift(int flags)
 +        {
 +            return (flags & 0x02) != 0;
 +        }
 +
 +        private static int indexFlag(boolean hasIndex)
 +        {
 +            return hasIndex ? 0x04 : 0;
 +        }
 +
 +        private static boolean hasIndex(int flags)
 +        {
 +            return (flags & 0x04) != 0;
 +        }
 +
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
 +            assert version >= MessagingService.VERSION_30;
 +
 +            out.writeByte(command.kind.ordinal());
 +            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(command.index.isPresent()));
 +            if (command.isDigestQuery())
 +                out.writeUnsignedVInt(command.digestVersion());
 +            CFMetaData.serializer.serialize(command.metadata(), out, version);
 +            out.writeInt(command.nowInSec());
 +            ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
 +            RowFilter.serializer.serialize(command.rowFilter(), out, version);
 +            DataLimits.serializer.serialize(command.limits(), out, version);
 +            if (command.index.isPresent())
 +                IndexMetadata.serializer.serialize(command.index.get(), out, version);
 +
 +            command.serializeSelection(out, version);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            if (version < MessagingService.VERSION_30)
 +                return legacyReadCommandSerializer.deserialize(in, version);
 +
 +            Kind kind = Kind.values()[in.readByte()];
 +            int flags = in.readByte();
 +            boolean isDigest = isDigest(flags);
 +            boolean isForThrift = isForThrift(flags);
 +            boolean hasIndex = hasIndex(flags);
 +            int digestVersion = isDigest ? (int)in.readUnsignedVInt() : 0;
 +            CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
 +            int nowInSec = in.readInt();
 +            ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
 +            RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
 +            DataLimits limits = DataLimits.serializer.deserialize(in, version);
 +            Optional<IndexMetadata> index = hasIndex
 +                                            ? deserializeIndexMetadata(in, version, metadata)
 +                                            : Optional.empty();
 +
 +            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
 +        }
 +
 +        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
 +        {
 +            try
 +            {
 +                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
 +            }
 +            catch (UnknownIndexException e)
 +            {
 +                String message = String.format("Couldn't find a defined index on %s.%s with the id %s. " +
 +                                               "If an index was just created, this is likely due to the schema not " +
 +                                               "being fully propagated. Local read will proceed without using the " +
 +                                               "index. Please wait for schema agreement after index creation.",
 +                                               cfm.ksName, cfm.cfName, e.indexId.toString());
 +                logger.info(message);
 +                return Optional.empty();
 +            }
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
 +            assert version >= MessagingService.VERSION_30;
 +
 +            return 2 // kind + flags
 +                 + (command.isDigestQuery() ? TypeSizes.sizeofUnsignedVInt(command.digestVersion()) : 0)
 +                 + CFMetaData.serializer.serializedSize(command.metadata(), version)
 +                 + TypeSizes.sizeof(command.nowInSec())
 +                 + ColumnFilter.serializer.serializedSize(command.columnFilter(), version)
 +                 + RowFilter.serializer.serializedSize(command.rowFilter(), version)
 +                 + DataLimits.serializer.serializedSize(command.limits(), version)
 +                 + command.selectionSerializedSize(version)
 +                 + command.indexSerializedSize(version);
 +        }
 +    }
 +
 +    // Dispatch to either Serializer or LegacyRangeSliceCommandSerializer. Only useful as long as we maintain pre-3.0
 +    // compatibility
 +    private static class RangeSliceSerializer implements IVersionedSerializer<ReadCommand>
 +    {
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            if (version < MessagingService.VERSION_30)
 +                legacyRangeSliceCommandSerializer.serialize(command, out, version);
 +            else
 +                serializer.serialize(command, out, version);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            return version < MessagingService.VERSION_30
 +                 ? legacyRangeSliceCommandSerializer.deserialize(in, version)
 +                 : serializer.deserialize(in, version);
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            return version < MessagingService.VERSION_30
 +                 ? legacyRangeSliceCommandSerializer.serializedSize(command, version)
 +                 : serializer.serializedSize(command, version);
 +        }
 +    }
 +
 +    private enum LegacyType
 +    {
 +        GET_BY_NAMES((byte)1),
 +        GET_SLICES((byte)2);
 +
 +        public final byte serializedValue;
 +
 +        LegacyType(byte b)
 +        {
 +            this.serializedValue = b;
 +        }
 +
 +        public static LegacyType fromPartitionFilterKind(ClusteringIndexFilter.Kind kind)
 +        {
 +            return kind == ClusteringIndexFilter.Kind.SLICE
 +                   ? GET_SLICES
 +                   : GET_BY_NAMES;
 +        }
 +
 +        public static LegacyType fromSerializedValue(byte b)
 +        {
 +            return b == 1 ? GET_BY_NAMES : GET_SLICES;
 +        }
 +    }
 +
 +    /**
 +     * Serializer for pre-3.0 RangeSliceCommands.
 +     */
 +    private static class LegacyRangeSliceCommandSerializer implements IVersionedSerializer<ReadCommand>
      {
 -        out.writeByte(command.commandType.serializedValue);
 -        switch (command.commandType)
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            assert version < MessagingService.VERSION_30;
 +
 +            PartitionRangeReadCommand rangeCommand = (PartitionRangeReadCommand) command;
 +            assert !rangeCommand.dataRange().isPaging();
 +
 +            // convert pre-3.0 incompatible names filters to slice filters
 +            rangeCommand = maybeConvertNamesToSlice(rangeCommand);
 +
 +            CFMetaData metadata = rangeCommand.metadata();
 +
 +            out.writeUTF(metadata.ksName);
 +            out.writeUTF(metadata.cfName);
 +            out.writeLong(rangeCommand.nowInSec() * 1000L);  // convert from seconds to millis
 +
 +            // begin DiskAtomFilterSerializer.serialize()
 +            if (rangeCommand.isNamesQuery())
 +            {
 +                out.writeByte(1);  // 0 for slices, 1 for names
 +                ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                LegacyReadCommandSerializer.serializeNamesFilter(rangeCommand, filter, out);
 +            }
 +            else
 +            {
 +                out.writeByte(0);  // 0 for slices, 1 for names
 +
 +                // slice filter serialization
 +                ClusteringIndexSliceFilter filter = (ClusteringIndexSliceFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +
 +                boolean makeStaticSlice = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() && !filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                LegacyReadCommandSerializer.serializeSlices(out, filter.requestedSlices(), filter.isReversed(), makeStaticSlice, metadata);
 +
 +                out.writeBoolean(filter.isReversed());
 +
 +                // limit
 +                DataLimits.Kind kind = rangeCommand.limits().kind();
 +                boolean isDistinct = (kind == DataLimits.Kind.CQL_LIMIT || kind == DataLimits.Kind.CQL_PAGING_LIMIT) && rangeCommand.limits().perPartitionCount() == 1;
 +                if (isDistinct)
 +                    out.writeInt(1);
 +                else
 +                    out.writeInt(LegacyReadCommandSerializer.updateLimitForQuery(rangeCommand.limits().count(), filter.requestedSlices()));
 +
 +                int compositesToGroup;
 +                boolean selectsStatics = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() || filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                if (kind == DataLimits.Kind.THRIFT_LIMIT)
 +                    compositesToGroup = -1;
 +                else if (isDistinct && !selectsStatics)
 +                    compositesToGroup = -2;  // for DISTINCT queries (CASSANDRA-8490)
 +                else
 +                    compositesToGroup = metadata.isDense() ? -1 : metadata.clusteringColumns().size();
 +
 +                out.writeInt(compositesToGroup);
 +            }
 +
 +            serializeRowFilter(out, rangeCommand.rowFilter());
 +            AbstractBounds.rowPositionSerializer.serialize(rangeCommand.dataRange().keyRange(), out, version);
 +
 +            // maxResults
 +            out.writeInt(rangeCommand.limits().count());
 +
 +            // countCQL3Rows
 +            if (rangeCommand.isForThrift() || rangeCommand.limits().perPartitionCount() == 1)  // if for Thrift or DISTINCT
 +                out.writeBoolean(false);
 +            else
 +                out.writeBoolean(true);
 +
 +            // isPaging
 +            out.writeBoolean(false);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            assert version < MessagingService.VERSION_30;
 +
 +            String keyspace = in.readUTF();
 +            String columnFamily = in.readUTF();
 +
 +            CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
 +            if (metadata == null)
 +            {
 +                String message = String.format("Got legacy range command for nonexistent table %s.%s.", keyspace, columnFamily);
 +                throw new UnknownColumnFamilyException(message, null);
 +            }
 +
 +            int nowInSec = (int) (in.readLong() / 1000);  // convert from millis to seconds
 +
 +            ClusteringIndexFilter filter;
 +            ColumnFilter selection;
 +            int compositesToGroup = 0;
 +            int perPartitionLimit = -1;
 +            byte readType = in.readByte();  // 0 for slices, 1 for names
 +            if (readType == 1)
 +            {
 +                Pair<ColumnFilter, ClusteringIndexNamesFilter> selectionAndFilter = LegacyReadCommandSerializer.deserializeNamesSelectionAndFilter(in, metadata);
 +                selection = selectionAndFilter.left;
 +                filter = selectionAndFilter.right;
 +            }
 +            else
 +            {
 +                Pair<ClusteringIndexSliceFilter, Boolean> p = LegacyReadCommandSerializer.deserializeSlicePartitionFilter(in, metadata);
 +                filter = p.left;
 +                perPartitionLimit = in.readInt();
 +                compositesToGroup = in.readInt();
 +                selection = getColumnSelectionForSlice(p.right, compositesToGroup, metadata);
 +            }
 +
 +            RowFilter rowFilter = deserializeRowFilter(in, metadata);
 +
 +            AbstractBounds<PartitionPosition> keyRange = AbstractBounds.rowPositionSerializer.deserialize(in, metadata.partitioner, version);
 +            int maxResults = in.readInt();
 +
 +            in.readBoolean();  // countCQL3Rows (not needed)
 +            in.readBoolean();  // isPaging (not needed)
 +
 +            boolean selectsStatics = (!selection.fetchedColumns().statics.isEmpty() || filter.selects(Clustering.STATIC_CLUSTERING));
 +            boolean isDistinct = compositesToGroup == -2 || (perPartitionLimit == 1 && selectsStatics);
 +            DataLimits limits;
 +            if (isDistinct)
 +                limits = DataLimits.distinctLimits(maxResults);
 +            else if (compositesToGroup == -1)
 +                limits = DataLimits.thriftLimits(maxResults, perPartitionLimit);
 +            else
 +                limits = DataLimits.cqlLimits(maxResults);
 +
 +            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter), Optional.empty());
 +        }
 +
 +        static void serializeRowFilter(DataOutputPlus out, RowFilter rowFilter) throws IOException
 +        {
 +            ArrayList<RowFilter.Expression> indexExpressions = Lists.newArrayList(rowFilter.iterator());
 +            out.writeInt(indexExpressions.size());
 +            for (RowFilter.Expression expression : indexExpressions)
 +            {
 +                ByteBufferUtil.writeWithShortLength(expression.column().name.bytes, out);
 +                expression.operator().writeTo(out);
 +                ByteBufferUtil.writeWithShortLength(expression.getIndexValue(), out);
 +            }
 +        }
 +
 +        static RowFilter deserializeRowFilter(DataInputPlus in, CFMetaData metadata) throws IOException
 +        {
 +            int numRowFilters = in.readInt();
 +            if (numRowFilters == 0)
 +                return RowFilter.NONE;
 +
 +            RowFilter rowFilter = RowFilter.create(numRowFilters);
 +            for (int i = 0; i < numRowFilters; i++)
 +            {
 +                ByteBuffer columnName = ByteBufferUtil.readWithShortLength(in);
 +                ColumnDefinition column = metadata.getColumnDefinition(columnName);
 +                Operator op = Operator.readFrom(in);
 +                ByteBuffer indexValue = ByteBufferUtil.readWithShortLength(in);
 +                rowFilter.add(column, op, indexValue);
 +            }
 +            return rowFilter;
 +        }
 +
 +        static long serializedRowFilterSize(RowFilter rowFilter)
 +        {
 +            long size = TypeSizes.sizeof(0);  // rowFilterCount
 +            for (RowFilter.Expression expression : rowFilter)
 +            {
 +                size += ByteBufferUtil.serializedSizeWithShortLength(expression.column().name.bytes);
 +                size += TypeSizes.sizeof(0);  // operator int value
 +                size += ByteBufferUtil.serializedSizeWithShortLength(expression.getIndexValue());
 +            }
 +            return size;
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            assert version < MessagingService.VERSION_30;
 +            assert command.kind == Kind.PARTITION_RANGE;
 +
 +            PartitionRangeReadCommand rangeCommand = (PartitionRangeReadCommand) command;
 +            rangeCommand = maybeConvertNamesToSlice(rangeCommand);
 +            CFMetaData metadata = rangeCommand.metadata();
 +
 +            long size = TypeSizes.sizeof(metadata.ksName);
 +            size += TypeSizes.sizeof(metadata.cfName);
 +            size += TypeSizes.sizeof((long) rangeCommand.nowInSec());
 +
 +            size += 1;  // single byte flag: 0 for slices, 1 for names
 +            if (rangeCommand.isNamesQuery())
 +            {
 +                PartitionColumns columns = rangeCommand.columnFilter().fetchedColumns();
 +                ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                size += LegacyReadCommandSerializer.serializedNamesFilterSize(filter, metadata, columns);
 +            }
 +            else
 +            {
 +                ClusteringIndexSliceFilter filter = (ClusteringIndexSliceFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                boolean makeStaticSlice = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() && !filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                size += LegacyReadCommandSerializer.serializedSlicesSize(filter.requestedSlices(), makeStaticSlice, metadata);
 +                size += TypeSizes.sizeof(filter.isReversed());
 +                size += TypeSizes.sizeof(rangeCommand.limits().perPartitionCount());
 +                size += TypeSizes.sizeof(0); // compositesToGroup
 +            }
 +
 +            if (rangeCommand.rowFilter().equals(RowFilter.NONE))
 +            {
 +                size += TypeSizes.sizeof(0);
 +            }
 +            else
 +            {
 +                ArrayList<RowFilter.Expression> indexExpressions = Lists.newArrayList(rangeCommand.rowFilter().iterator());
 +                size += TypeSizes.sizeof(indexExpressions.size());
 +                for (RowFilter.Expression expression : indexExpressions)
 +                {
 +                    size += ByteBufferUtil.serializedSizeWithShortLength(expression.column().name.bytes);
 +                    size += TypeSizes.sizeof(expression.operator().ordinal());
 +                    size += ByteBufferUtil.serializedSizeWithShortLength(expression.getIndexValue());
 +                }
 +            }
 +
 +            size += AbstractBounds.rowPositionSerializer.serializedSize(rangeCommand.dataRange().keyRange(), version);
 +            size += TypeSizes.sizeof(rangeCommand.limits().count());
 +            size += TypeSizes.sizeof(!rangeCommand.isForThrift());
 +            return size + TypeSizes.sizeof(rangeCommand.dataRange().isPaging());
 +        }
 +
 +        static PartitionRangeReadCommand maybeConvertNamesToSlice(PartitionRangeReadCommand command)
          {
 -            case GET_BY_NAMES:
 -                SliceByNamesReadCommand.serializer.serialize(command, out, version);
 -                break;
 -            case GET_SLICES:
 -                SliceFromReadCommand.serializer.serialize(command, out, version);
 -                break;
 -            default:
 -                throw new AssertionError();
 +            if (!command.dataRange().isNamesQuery())
 +                return command;
 +
 +            CFMetaData metadata = command.metadata();
 +            if (!LegacyReadCommandSerializer.shouldConvertNamesToSlice(metadata, command.columnFilter().fetchedColumns()))
 +                return command;
 +
 +            ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) command.dataRange().clusteringIndexFilter;
 +            ClusteringIndexSliceFilter sliceFilter = LegacyReadCommandSerializer.convertNamesFilterToSliceFilter(filter, metadata);
 +            DataRange newRange = new DataRange(command.dataRange().keyRange(), sliceFilter);
 +            return new PartitionRangeReadCommand(
 +                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
 +                    command.columnFilter(), command.rowFilter(), command.limits(), newRange, Optional.empty());
 +        }
 +
 +        static ColumnFilter getColumnSelectionForSlice(boolean selectsStatics, int compositesToGroup, CFMetaData metadata)
 +        {
 +            // A value of -2 indicates this is a DISTINCT query that doesn't select static columns, only partition keys.
 +            // In that case, we'll basically be querying the first row of the partition, but we must make sure we include
 +            // all columns so we get at least one cell if there is a live row as it would confuse pre-3.0 nodes otherwise.
 +            if (compositesToGroup == -2)
 +                return ColumnFilter.all(metadata);
 +
 +            // if a slice query from a pre-3.0 node doesn't cover statics, we shouldn't select them at all
 +            PartitionColumns columns = selectsStatics
 +                                     ? metadata.partitionColumns()
 +                                     : metadata.partitionColumns().withoutStatics();
 +            return ColumnFilter.selectionBuilder().addAll(columns).build();
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/MessagingService.java
index ccc900b,459923b..d416dca
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@@ -39,8 -39,12 +39,9 @@@ import com.google.common.collect.Lists
  import com.google.common.collect.Sets;
  
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
 -
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
 -
 -import org.apache.cassandra.concurrent.ExecutorLocal;
+ import org.apache.cassandra.concurrent.ExecutorLocals;
  import org.apache.cassandra.concurrent.ScheduledExecutors;
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;


[13/13] cassandra git commit: Merge branch 'cassandra-3.3' into trunk

Posted by ca...@apache.org.
Merge branch 'cassandra-3.3' into trunk


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

Branch: refs/heads/trunk
Commit: 5090afc7a8f7c49532ff441217614e808876be19
Parents: 3f05312 93f6529
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 14:47:41 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:47:41 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 230 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 230 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cassandra/cql3/functions/UDFunction.java    |   2 +-
 .../cql3/statements/BatchStatement.java         |   9 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   7 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |   6 +-
 .../cassandra/service/ClientWarningsTest.java   |  58 +++++
 24 files changed, 545 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5090afc7/CHANGES.txt
----------------------------------------------------------------------


[07/13] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageProxy.java
index e2fa270,841e980..89ac0bb
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@@ -1941,201 -1711,252 +1941,201 @@@ public class StorageProxy implements St
          }
      }
  
 -    public static List<Row> getRangeSlice(AbstractRangeCommand command, ConsistencyLevel consistency_level)
 -    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    private static class SingleRangeResponse extends AbstractIterator<RowIterator> implements PartitionIterator
      {
 -        Tracing.trace("Computing ranges to query");
 -        long startTime = System.nanoTime();
 +        private final ReadCallback handler;
 +        private PartitionIterator result;
  
 -        Keyspace keyspace = Keyspace.open(command.keyspace);
 -        List<Row> rows;
 -        // now scan until we have enough results
 -        try
 +        private SingleRangeResponse(ReadCallback handler)
          {
 -            int liveRowCount = 0;
 -            boolean countLiveRows = command.countCQL3Rows() || command.ignoredTombstonedPartitions();
 -            rows = new ArrayList<>();
 +            this.handler = handler;
 +        }
  
 -            // when dealing with LocalStrategy keyspaces, we can skip the range splitting and merging (which can be
 -            // expensive in clusters with vnodes)
 -            List<? extends AbstractBounds<RowPosition>> ranges;
 -            if (keyspace.getReplicationStrategy() instanceof LocalStrategy)
 -                ranges = command.keyRange.unwrap();
 -            else
 -                ranges = getRestrictedRanges(command.keyRange);
 -
 -            // determine the number of rows to be fetched and the concurrency factor
 -            int rowsToBeFetched = command.limit();
 -            int concurrencyFactor;
 -            if (command.requiresScanningAllRanges())
 -            {
 -                // all nodes must be queried
 -                rowsToBeFetched *= ranges.size();
 -                concurrencyFactor = ranges.size();
 -                logger.debug("Requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {}",
 -                              ranges.size(), concurrencyFactor);
 +        private void waitForResponse() throws ReadTimeoutException
 +        {
 +            if (result != null)
 +                return;
 +
 +            try
 +            {
 +                result = handler.get();
              }
 -            else
 +            catch (DigestMismatchException e)
              {
 -                // our estimate of how many result rows there will be per-range
 -                float resultRowsPerRange = estimateResultRowsPerRange(command, keyspace);
 -                // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 -                // fetch enough rows in the first round
 -                resultRowsPerRange -= resultRowsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 -                concurrencyFactor = resultRowsPerRange == 0.0
 -                                  ? 1
 -                                  : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit() / resultRowsPerRange)));
 -
 -                logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             resultRowsPerRange,
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)",
 -                              ranges.size(),
 -                              concurrencyFactor,
 -                              resultRowsPerRange);
 -            }
 -
 -            boolean haveSufficientRows = false;
 -            int i = 0;
 -            AbstractBounds<RowPosition> nextRange = null;
 -            List<InetAddress> nextEndpoints = null;
 -            List<InetAddress> nextFilteredEndpoints = null;
 -            while (i < ranges.size())
 -            {
 -                List<Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>>> scanHandlers = new ArrayList<>(concurrencyFactor);
 -                int concurrentFetchStartingIndex = i;
 -                int concurrentRequests = 0;
 -                while ((i - concurrentFetchStartingIndex) < concurrencyFactor)
 -                {
 -                    AbstractBounds<RowPosition> range = nextRange == null
 -                                                      ? ranges.get(i)
 -                                                      : nextRange;
 -                    List<InetAddress> liveEndpoints = nextEndpoints == null
 -                                                    ? getLiveSortedEndpoints(keyspace, range.right)
 -                                                    : nextEndpoints;
 -                    List<InetAddress> filteredEndpoints = nextFilteredEndpoints == null
 -                                                        ? consistency_level.filterForQuery(keyspace, liveEndpoints)
 -                                                        : nextFilteredEndpoints;
 -                    ++i;
 -                    ++concurrentRequests;
 -
 -                    // getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take
 -                    // the replication factor into account. If the intersection of live endpoints for 2 consecutive ranges
 -                    // still meets the CL requirements, then we can merge both ranges into the same RangeSliceCommand.
 -                    while (i < ranges.size())
 -                    {
 -                        nextRange = ranges.get(i);
 -                        nextEndpoints = getLiveSortedEndpoints(keyspace, nextRange.right);
 -                        nextFilteredEndpoints = consistency_level.filterForQuery(keyspace, nextEndpoints);
 -
 -                        // If the current range right is the min token, we should stop merging because CFS.getRangeSlice
 -                        // don't know how to deal with a wrapping range.
 -                        // Note: it would be slightly more efficient to have CFS.getRangeSlice on the destination nodes unwraps
 -                        // the range if necessary and deal with it. However, we can't start sending wrapped range without breaking
 -                        // wire compatibility, so It's likely easier not to bother;
 -                        if (range.right.isMinimum())
 -                            break;
 -
 -                        List<InetAddress> merged = intersection(liveEndpoints, nextEndpoints);
 -
 -                        // Check if there is enough endpoint for the merge to be possible.
 -                        if (!consistency_level.isSufficientLiveNodes(keyspace, merged))
 -                            break;
 -
 -                        List<InetAddress> filteredMerged = consistency_level.filterForQuery(keyspace, merged);
 -
 -                        // Estimate whether merging will be a win or not
 -                        if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, filteredEndpoints, nextFilteredEndpoints))
 -                            break;
 -
 -                        // If we get there, merge this range and the next one
 -                        range = range.withNewRight(nextRange.right);
 -                        liveEndpoints = merged;
 -                        filteredEndpoints = filteredMerged;
 -                        ++i;
 -                    }
 +                throw new AssertionError(e); // no digests in range slices yet
 +            }
 +        }
  
 -                    AbstractRangeCommand nodeCmd = command.forSubRange(range);
 +        protected RowIterator computeNext()
 +        {
 +            waitForResponse();
 +            return result.hasNext() ? result.next() : endOfData();
 +        }
  
 -                    // collect replies and resolve according to consistency level
 -                    RangeSliceResponseResolver resolver = new RangeSliceResponseResolver(nodeCmd.keyspace, command.timestamp);
 -                    List<InetAddress> minimalEndpoints = filteredEndpoints.subList(0, Math.min(filteredEndpoints.size(), consistency_level.blockFor(keyspace)));
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = new ReadCallback<>(resolver, consistency_level, nodeCmd, minimalEndpoints);
 -                    handler.assureSufficientLiveNodes();
 -                    resolver.setSources(filteredEndpoints);
 -                    if (filteredEndpoints.size() == 1
 -                        && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
 -                    {
 -                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
 -                    }
 -                    else
 -                    {
 -                        MessageOut<? extends AbstractRangeCommand> message = nodeCmd.createMessage();
 -                        for (InetAddress endpoint : filteredEndpoints)
 -                        {
 -                            Tracing.trace("Enqueuing request to {}", endpoint);
 -                            MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
 -                        }
 -                    }
 -                    scanHandlers.add(Pair.create(nodeCmd, handler));
 -                }
 -                Tracing.trace("Submitted {} concurrent range requests covering {} ranges", concurrentRequests, i - concurrentFetchStartingIndex);
 +        public void close()
 +        {
 +            if (result != null)
 +                result.close();
 +        }
 +    }
  
 -                List<AsyncOneResponse> repairResponses = new ArrayList<>();
 -                for (Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>> cmdPairHandler : scanHandlers)
 -                {
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = cmdPairHandler.right;
 -                    RangeSliceResponseResolver resolver = (RangeSliceResponseResolver)handler.resolver;
 +    private static class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
 +    {
 +        private final Iterator<RangeForQuery> ranges;
 +        private final int totalRangeCount;
 +        private final PartitionRangeReadCommand command;
 +        private final Keyspace keyspace;
 +        private final ConsistencyLevel consistency;
  
 -                    try
 -                    {
 -                        for (Row row : handler.get())
 -                        {
 -                            rows.add(row);
 -                            if (countLiveRows)
 -                                liveRowCount += row.getLiveCount(command.predicate, command.timestamp);
 -                        }
 -                        repairResponses.addAll(resolver.repairResults);
 -                    }
 -                    catch (ReadTimeoutException|ReadFailureException ex)
 -                    {
 -                        // we timed out or failed waiting for responses
 -                        int blockFor = consistency_level.blockFor(keyspace);
 -                        int responseCount = resolver.responses.size();
 -                        String gotData = responseCount > 0
 -                                         ? resolver.isDataPresent() ? " (including data)" : " (only digests)"
 -                                         : "";
 -
 -                        boolean isTimeout = ex instanceof ReadTimeoutException;
 -                        if (Tracing.isTracing())
 -                        {
 -                            Tracing.trace("{}; received {} of {} responses{} for range {} of {}",
 -                                          (isTimeout ? "Timed out" : "Failed"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        else if (logger.isDebugEnabled())
 -                        {
 -                            logger.debug("Range slice {}; received {} of {} responses{} for range {} of {}",
 -                                         (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        throw ex;
 -                    }
 -                    catch (DigestMismatchException e)
 -                    {
 -                        throw new AssertionError(e); // no digests in range slices yet
 -                    }
 +        private final long startTime;
 +        private DataLimits.Counter counter;
 +        private PartitionIterator sentQueryIterator;
  
 -                    // if we're done, great, otherwise, move to the next range
 -                    int count = countLiveRows ? liveRowCount : rows.size();
 -                    if (count >= rowsToBeFetched)
 -                    {
 -                        haveSufficientRows = true;
 -                        break;
 -                    }
 -                }
 +        private int concurrencyFactor;
 +        // The two following "metric" are maintained to improve the concurrencyFactor
 +        // when it was not good enough initially.
 +        private int liveReturned;
 +        private int rangesQueried;
  
 -                try
 -                {
 -                    FBUtilities.waitOnFutures(repairResponses, DatabaseDescriptor.getWriteRpcTimeout());
 -                }
 -                catch (TimeoutException ex)
 +        public RangeCommandIterator(RangeIterator ranges, PartitionRangeReadCommand command, int concurrencyFactor, Keyspace keyspace, ConsistencyLevel consistency)
 +        {
 +            this.command = command;
 +            this.concurrencyFactor = concurrencyFactor;
 +            this.startTime = System.nanoTime();
 +            this.ranges = new RangeMerger(ranges, keyspace, consistency);
 +            this.totalRangeCount = ranges.rangeCount();
 +            this.consistency = consistency;
 +            this.keyspace = keyspace;
 +        }
 +
 +        public RowIterator computeNext()
 +        {
 +            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
 +            {
 +                // If we don't have more range to handle, we're done
 +                if (!ranges.hasNext())
 +                    return endOfData();
 +
 +                // else, sends the next batch of concurrent queries (after having close the previous iterator)
 +                if (sentQueryIterator != null)
                  {
 -                    // We got all responses, but timed out while repairing
 -                    int blockFor = consistency_level.blockFor(keyspace);
 -                    if (Tracing.isTracing())
 -                        Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    else
 -                        logger.debug("Range slice timeout while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    throw new ReadTimeoutException(consistency_level, blockFor-1, blockFor, true);
 +                    liveReturned += counter.counted();
 +                    sentQueryIterator.close();
 +
 +                    // It's not the first batch of queries and we're not done, so we we can use what has been
 +                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
 +                    updateConcurrencyFactor();
                  }
 +                sentQueryIterator = sendNextRequests();
 +            }
 +
 +            return sentQueryIterator.next();
 +        }
 +
 +        private void updateConcurrencyFactor()
 +        {
 +            if (liveReturned == 0)
 +            {
 +                // we haven't actually gotten any results, so query all remaining ranges at once
 +                concurrencyFactor = totalRangeCount - rangesQueried;
 +                return;
 +            }
  
 -                if (haveSufficientRows)
 -                    return command.postReconciliationProcessing(rows);
 +            // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
 +            // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
 +            int remainingRows = command.limits().count() - liveReturned;
 +            float rowsPerRange = (float)liveReturned / (float)rangesQueried;
 +            concurrencyFactor = Math.max(1, Math.min(totalRangeCount - rangesQueried, Math.round(remainingRows / rowsPerRange)));
 +            logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 +                         rowsPerRange, (int) remainingRows, concurrencyFactor);
 +        }
 +
 +        private SingleRangeResponse query(RangeForQuery toQuery)
 +        {
 +            PartitionRangeReadCommand rangeCommand = command.forSubRange(toQuery.range);
 +
 +            DataResolver resolver = new DataResolver(keyspace, rangeCommand, consistency, toQuery.filteredEndpoints.size());
 +
 +            int blockFor = consistency.blockFor(keyspace);
 +            int minResponses = Math.min(toQuery.filteredEndpoints.size(), blockFor);
 +            List<InetAddress> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
 +            ReadCallback handler = new ReadCallback(resolver, consistency, rangeCommand, minimalEndpoints);
  
 -                // we didn't get enough rows in our concurrent fetch; recalculate our concurrency factor
 -                // based on the results we've seen so far (as long as we still have ranges left to query)
 -                if (i < ranges.size())
 +            handler.assureSufficientLiveNodes();
 +
 +            if (toQuery.filteredEndpoints.size() == 1 && canDoLocalRequest(toQuery.filteredEndpoints.get(0)))
 +            {
-                 StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler), Tracing.instance.get());
++                StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler));
 +            }
 +            else
 +            {
 +                for (InetAddress endpoint : toQuery.filteredEndpoints)
                  {
 -                    float fetchedRows = countLiveRows ? liveRowCount : rows.size();
 -                    float remainingRows = rowsToBeFetched - fetchedRows;
 -                    float actualRowsPerRange;
 -                    if (fetchedRows == 0.0)
 -                    {
 -                        // we haven't actually gotten any results, so query all remaining ranges at once
 -                        actualRowsPerRange = 0.0f;
 -                        concurrencyFactor = ranges.size() - i;
 -                    }
 -                    else
 -                    {
 -                        actualRowsPerRange = fetchedRows / i;
 -                        concurrencyFactor = Math.max(1, Math.min(ranges.size() - i, Math.round(remainingRows / actualRowsPerRange)));
 -                    }
 -                    logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 -                                 actualRowsPerRange, (int) remainingRows, concurrencyFactor);
 +                    MessageOut<ReadCommand> message = rangeCommand.createMessage(MessagingService.instance().getVersion(endpoint));
 +                    Tracing.trace("Enqueuing request to {}", endpoint);
 +                    MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
                  }
              }
 +
 +            return new SingleRangeResponse(handler);
          }
 -        finally
 +
 +        private PartitionIterator sendNextRequests()
          {
 -            long latency = System.nanoTime() - startTime;
 -            rangeMetrics.addNano(latency);
 -            Keyspace.open(command.keyspace).getColumnFamilyStore(command.columnFamily).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            List<PartitionIterator> concurrentQueries = new ArrayList<>(concurrencyFactor);
 +            for (int i = 0; i < concurrencyFactor && ranges.hasNext(); i++)
 +            {
 +                concurrentQueries.add(query(ranges.next()));
 +                ++rangesQueried;
 +            }
 +
 +            Tracing.trace("Submitted {} concurrent range requests", concurrentQueries.size());
 +            // We want to count the results for the sake of updating the concurrency factor (see updateConcurrencyFactor) but we don't want to
 +            // enforce any particular limit at this point (this could break code than rely on postReconciliationProcessing), hence the DataLimits.NONE.
 +            counter = DataLimits.NONE.newCounter(command.nowInSec(), true);
 +            return counter.applyTo(PartitionIterators.concat(concurrentQueries));
          }
 -        return command.postReconciliationProcessing(rows);
 +
 +        public void close()
 +        {
 +            try
 +            {
 +                if (sentQueryIterator != null)
 +                    sentQueryIterator.close();
 +            }
 +            finally
 +            {
 +                long latency = System.nanoTime() - startTime;
 +                rangeMetrics.addNano(latency);
 +                Keyspace.openAndGetStore(command.metadata()).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            }
 +        }
 +    }
 +
 +    @SuppressWarnings("resource")
 +    public static PartitionIterator getRangeSlice(PartitionRangeReadCommand command, ConsistencyLevel consistencyLevel)
 +    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    {
 +        Tracing.trace("Computing ranges to query");
 +
 +        Keyspace keyspace = Keyspace.open(command.metadata().ksName);
 +        RangeIterator ranges = new RangeIterator(command, keyspace, consistencyLevel);
 +
 +        // our estimate of how many result rows there will be per-range
 +        float resultsPerRange = estimateResultsPerRange(command, keyspace);
 +        // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 +        // fetch enough rows in the first round
 +        resultsPerRange -= resultsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 +        int concurrencyFactor = resultsPerRange == 0.0
 +                              ? 1
 +                              : Math.max(1, Math.min(ranges.rangeCount(), (int) Math.ceil(command.limits().count() / resultsPerRange)));
 +        logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 +                     resultsPerRange, command.limits().count(), ranges.rangeCount(), concurrencyFactor);
 +        Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)", ranges.rangeCount(), concurrencyFactor, resultsPerRange);
 +
 +        // Note that in general, a RangeCommandIterator will honor the command limit for each range, but will not enforce it globally.
 +
 +        return command.limits().filter(command.postReconciliationProcessing(new RangeCommandIterator(ranges, command, concurrencyFactor, keyspace, consistencyLevel)), command.nowInSec());
      }
  
      public Map<String, List<String>> getSchemaVersions()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 704a6c9,bcfe871..6d54e36
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -2263,226 -2686,4 +2263,226 @@@ public class UFTest extends CQLTeste
                             "AS 'return 0;'");
          }
      }
 +
 +    @Test
 +    public void testSecurityPermissions() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, dval double)");
 +        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
 +
 +        // Java UDFs
 +
 +        try
 +        {
 +            String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                          "RETURNS NULL ON NULL INPUT " +
 +                                          "RETURNS double " +
 +                                          "LANGUAGE JAVA\n" +
 +                                          "AS 'System.getProperty(\"foo.bar.baz\"); return 0d;';");
 +            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +            Assert.fail();
 +        }
 +        catch (FunctionExecutionException e)
 +        {
 +            assertAccessControlException("System.getProperty(\"foo.bar.baz\"); return 0d;", e);
 +        }
 +
 +        String[][] typesAndSources =
 +        {
 +        {"",                        "try { Class.forName(\"" + UDHelper.class.getName() + "\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
 +        {"sun.misc.Unsafe",         "sun.misc.Unsafe.getUnsafe(); return 0d;"},
 +        {"",                        "try { Class.forName(\"sun.misc.Unsafe\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
 +        {"java.nio.file.FileSystems", "try {" +
 +                                    "     java.nio.file.FileSystems.getDefault(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.nio.channels.FileChannel", "try {" +
 +                                    "     java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")).close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.nio.channels.SocketChannel", "try {" +
 +                                    "     java.nio.channels.SocketChannel.open().close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.io.FileInputStream", "try {" +
 +                                    "     new java.io.FileInputStream(\"./foobar\").close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.lang.Runtime",       "try {" +
 +                                    "     java.lang.Runtime.getRuntime(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"org.apache.cassandra.service.StorageService",
 +                                    "try {" +
 +                                    "     org.apache.cassandra.service.StorageService v = org.apache.cassandra.service.StorageService.instance; v.isInShutdownHook(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.net.ServerSocket",   "try {" +
 +                                    "     new java.net.ServerSocket().bind(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.io.FileOutputStream","try {" +
 +                                    "     new java.io.FileOutputStream(\".foo\"); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.lang.Runtime",       "try {" +
 +                                    "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'}
 +        };
 +
 +        for (String[] typeAndSource : typesAndSources)
 +        {
 +            assertInvalidMessage(typeAndSource[0] + " cannot be resolved",
 +                                 "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".invalid_class_access(val double) " +
 +                                 "RETURNS NULL ON NULL INPUT " +
 +                                 "RETURNS double " +
 +                                 "LANGUAGE JAVA\n" +
 +                                 "AS '" + typeAndSource[1] + "';");
 +        }
 +
 +        // JavaScript UDFs
 +
 +        try
 +        {
 +            String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                          "RETURNS NULL ON NULL INPUT " +
 +                                          "RETURNS double " +
 +                                          "LANGUAGE javascript\n" +
 +                                          "AS 'org.apache.cassandra.service.StorageService.instance.isInShutdownHook(); 0;';");
 +            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +            Assert.fail("Javascript security check failed");
 +        }
 +        catch (FunctionExecutionException e)
 +        {
 +            assertAccessControlException("", e);
 +        }
 +
 +        String[] javascript =
 +        {
 +        "java.lang.management.ManagmentFactory.getThreadMXBean(); 0;",
 +        "new java.io.FileInputStream(\"/tmp/foo\"); 0;",
 +        "new java.io.FileOutputStream(\"/tmp/foo\"); 0;",
 +        "java.nio.file.FileSystems.getDefault().createFileExclusively(\"./foo_bar_baz\"); 0;",
 +        "java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")); 0;",
 +        "java.nio.channels.SocketChannel.open(); 0;",
 +        "new java.net.ServerSocket().bind(null); 0;",
 +        "var thread = new java.lang.Thread(); thread.start(); 0;",
 +        "java.lang.System.getProperty(\"foo.bar.baz\"); 0;",
 +        "java.lang.Class.forName(\"java.lang.System\"); 0;",
 +        "java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); 0;",
 +        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
 +        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
 +        // TODO these (ugly) calls are still possible - these can consume CPU (as one could do with an evil loop, too)
 +//        "java.lang.Runtime.getRuntime().traceMethodCalls(true); 0;",
 +//        "java.lang.Runtime.getRuntime().gc(); 0;",
 +//        "java.lang.Runtime.getRuntime(); 0;",
 +        };
 +
 +        for (String script : javascript)
 +        {
 +            try
 +            {
 +                String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                              "RETURNS NULL ON NULL INPUT " +
 +                                              "RETURNS double " +
 +                                              "LANGUAGE javascript\n" +
 +                                              "AS '" + script + "';");
 +                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +                Assert.fail("Javascript security check failed: " + script);
 +            }
 +            catch (FunctionExecutionException e)
 +            {
 +                assertAccessControlException(script, e);
 +            }
 +        }
 +    }
 +
 +    private static void assertAccessControlException(String script, FunctionExecutionException e)
 +    {
 +        for (Throwable t = e; t != null && t != t.getCause(); t = t.getCause())
 +            if (t instanceof AccessControlException)
 +                return;
 +        Assert.fail("no AccessControlException for " + script + " (got " + e + ')');
 +    }
 +
 +    @Test
 +    public void testAmokUDF() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, dval double)");
 +        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
 +
 +        long udfWarnTimeout = DatabaseDescriptor.getUserDefinedFunctionWarnTimeout();
 +        long udfFailTimeout = DatabaseDescriptor.getUserDefinedFunctionFailTimeout();
 +        int maxTries = 5;
 +        for (int i = 1; i <= maxTries; i++)
 +        {
 +            try
 +            {
 +                // short timeout
 +                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(10);
 +                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(250);
 +                // don't kill the unit test... - default policy is "die"
 +                DatabaseDescriptor.setUserFunctionTimeoutPolicy(Config.UserFunctionTimeoutPolicy.ignore);
 +
-                 ClientWarn.captureWarnings();
++                ClientWarn.instance.captureWarnings();
 +                String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                              "RETURNS NULL ON NULL INPUT " +
 +                                              "RETURNS double " +
 +                                              "LANGUAGE JAVA\n" +
 +                                              "AS 'long t=System.currentTimeMillis()+110; while (t>System.currentTimeMillis()) { }; return 0d;'");
 +                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
-                 List<String> warnings = ClientWarn.getWarnings();
++                List<String> warnings = ClientWarn.instance.getWarnings();
 +                Assert.assertNotNull(warnings);
 +                Assert.assertFalse(warnings.isEmpty());
-                 ClientWarn.resetWarnings();
++                ClientWarn.instance.resetWarnings();
 +
 +                // Java UDF
 +
 +                fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS double " +
 +                                       "LANGUAGE JAVA\n" +
 +                                       "AS 'long t=System.currentTimeMillis()+500; while (t>System.currentTimeMillis()) { }; return 0d;';");
 +                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +
 +                // Javascript UDF
 +
 +                fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS double " +
 +                                       "LANGUAGE JAVASCRIPT\n" +
 +                                       "AS 'var t=java.lang.System.currentTimeMillis()+500; while (t>java.lang.System.currentTimeMillis()) { }; 0;';");
 +                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +
 +                return;
 +            }
 +            catch (Error | RuntimeException e)
 +            {
 +                if (i == maxTries)
 +                    throw e;
 +            }
 +            finally
 +            {
 +                // reset to defaults
 +                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(udfWarnTimeout);
 +                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(udfFailTimeout);
 +            }
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index c3f2629,d22a8f6..cf14d55
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@@ -23,8 -22,11 +23,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.cql3.QueryOptions;
  import org.apache.cassandra.cql3.CQLTester;
 +import org.apache.cassandra.cql3.QueryOptions;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Keyspace;
 -import org.apache.cassandra.dht.ByteOrderedPartitioner;
  import org.apache.cassandra.transport.Message;
  import org.apache.cassandra.transport.Server;
  import org.apache.cassandra.transport.SimpleClient;
@@@ -76,6 -80,62 +78,62 @@@ public class ClientWarningsTest extend
          }
      }
  
+     @Test
+     public void testTombstoneWarning() throws Exception
+     {
+         final int iterations = 10000;
+         createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+ 
+         try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+         {
+             client.connect(false);
+ 
+             for (int i = 0; i < iterations; i++)
+             {
+                 QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                     KEYSPACE,
+                                                                     currentTable(),
+                                                                     i), QueryOptions.DEFAULT);
+                 client.execute(query);
+             }
+             ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+             store.forceBlockingFlush();
+ 
+             for (int i = 0; i < iterations; i++)
+             {
+                 QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                     KEYSPACE,
+                                                                     currentTable(),
+                                                                     i), QueryOptions.DEFAULT);
+                 client.execute(query);
+             }
+             store.forceBlockingFlush();
+ 
+             {
+                 QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                     KEYSPACE,
+                                                                     currentTable()), QueryOptions.DEFAULT);
+                 Message.Response resp = client.execute(query);
+                 assertEquals(1, resp.getWarnings().size());
+             }
+         }
+     }
+ 
+     @Test
+     public void testLargeBatchWithProtoV2() throws Exception
+     {
+         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");
+ 
 -        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_2))
++        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_3))
+         {
+             client.connect(false);
+ 
+             QueryMessage query = new QueryMessage(createBatchStatement(DatabaseDescriptor.getBatchSizeWarnThreshold()), QueryOptions.DEFAULT);
+             Message.Response resp = client.execute(query);
+             assertNull(resp.getWarnings());
+         }
+     }
+ 
      private String createBatchStatement(int minSize)
      {
          return String.format("BEGIN UNLOGGED BATCH INSERT INTO %s.%s (pk, v) VALUES (1, '%s') APPLY BATCH;",


[03/13] cassandra git commit: Make sure client gets tombstone overwhelmed warning

Posted by ca...@apache.org.
Make sure client gets tombstone overwhelmed warning

patch by Carl Yeksigian; reviewed by Josh McKenzie for CASSANDRA-9465


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

Branch: refs/heads/cassandra-3.3
Commit: dbf6e62c382d62f9c1727ecf5afb90d131a81775
Parents: 582bdba
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 13:22:36 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:35:59 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 229 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 229 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cql3/statements/BatchStatement.java         |  11 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../cassandra/db/filter/SliceQueryFilter.java   |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   8 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cassandra/service/ClientWarningsTest.java   |  43 ++++
 21 files changed, 529 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11f2529..6530956 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.5
+ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
  * Fix error streaming section more than 2GB (CASSANDRA-10961)
  * (cqlsh) Also apply --connect-timeout to control connection
    timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
new file mode 100644
index 0000000..088b43e
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -0,0 +1,229 @@
+/*
+ * 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.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.tracing.Tracing.isTracing;
+
+public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+{
+    private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+
+    protected abstract void addTask(FutureTask<?> futureTask);
+    protected abstract void onCompletion();
+
+    /** Task Submission / Creation / Objects **/
+
+    public <T> FutureTask<T> submit(Callable<T> task)
+    {
+        return submit(newTaskFor(task));
+    }
+
+    public FutureTask<?> submit(Runnable task)
+    {
+        return submit(newTaskFor(task, null));
+    }
+
+    public <T> FutureTask<T> submit(Runnable task, T result)
+    {
+        return submit(newTaskFor(task, result));
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+    {
+        return newTaskFor(runnable, result, ExecutorLocals.create());
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+    {
+        if (locals != null)
+        {
+            if (runnable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) runnable;
+            return new LocalSessionFutureTask<T>(runnable, result, locals);
+        }
+        if (runnable instanceof FutureTask)
+            return (FutureTask<T>) runnable;
+        return new FutureTask<>(runnable, result);
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+    {
+        if (isTracing())
+        {
+            if (callable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) callable;
+            return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+        }
+        if (callable instanceof FutureTask)
+            return (FutureTask<T>) callable;
+        return new FutureTask<>(callable);
+    }
+
+    private class LocalSessionFutureTask<T> extends FutureTask<T>
+    {
+        private final ExecutorLocals locals;
+
+        public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+        {
+            super(callable);
+            this.locals = locals;
+        }
+
+        public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+        {
+            super(runnable, result);
+            this.locals = locals;
+        }
+
+        public void run()
+        {
+            ExecutorLocals old = ExecutorLocals.create();
+            ExecutorLocals.set(locals);
+            try
+            {
+                super.run();
+            }
+            finally
+            {
+                ExecutorLocals.set(old);
+            }
+        }
+    }
+
+    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+    {
+        private boolean failure;
+        private Object result = this;
+        private final Callable<T> callable;
+
+        public FutureTask(Callable<T> callable)
+        {
+            this.callable = callable;
+        }
+        public FutureTask(Runnable runnable, T result)
+        {
+            this(Executors.callable(runnable, result));
+        }
+
+        public void run()
+        {
+            try
+            {
+                result = callable.call();
+            }
+            catch (Throwable t)
+            {
+                JVMStabilityInspector.inspectThrowable(t);
+                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                result = t;
+                failure = true;
+            }
+            finally
+            {
+                signalAll();
+                onCompletion();
+            }
+        }
+
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            return false;
+        }
+
+        public boolean isCancelled()
+        {
+            return false;
+        }
+
+        public boolean isDone()
+        {
+            return isSignaled();
+        }
+
+        public T get() throws InterruptedException, ExecutionException
+        {
+            await();
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+
+        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            await(timeout, unit);
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+    }
+
+    private <T> FutureTask<T> submit(FutureTask<T> task)
+    {
+        addTask(task);
+        return task;
+    }
+
+    public void execute(Runnable command)
+    {
+        addTask(newTaskFor(command, ExecutorLocals.create()));
+    }
+
+    public void execute(Runnable command, ExecutorLocals locals)
+    {
+        addTask(newTaskFor(command, null, locals));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
deleted file mode 100644
index fb753b0..0000000
--- a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.concurrent;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-public abstract class AbstractTracingAwareExecutorService implements TracingAwareExecutorService
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractTracingAwareExecutorService.class);
-
-    protected abstract void addTask(FutureTask<?> futureTask);
-    protected abstract void onCompletion();
-
-    /** Task Submission / Creation / Objects **/
-
-    public <T> FutureTask<T> submit(Callable<T> task)
-    {
-        return submit(newTaskFor(task));
-    }
-
-    public FutureTask<?> submit(Runnable task)
-    {
-        return submit(newTaskFor(task, null));
-    }
-
-    public <T> FutureTask<T> submit(Runnable task, T result)
-    {
-        return submit(newTaskFor(task, result));
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
-    {
-        return newTaskFor(runnable, result, Tracing.instance.get());
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, TraceState traceState)
-    {
-        if (traceState != null)
-        {
-            if (runnable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) runnable;
-            return new TraceSessionFutureTask<T>(runnable, result, traceState);
-        }
-        if (runnable instanceof FutureTask)
-            return (FutureTask<T>) runnable;
-        return new FutureTask<>(runnable, result);
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing())
-        {
-            if (callable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) callable;
-            return new TraceSessionFutureTask<T>(callable, Tracing.instance.get());
-        }
-        if (callable instanceof FutureTask)
-            return (FutureTask<T>) callable;
-        return new FutureTask<>(callable);
-    }
-
-    private class TraceSessionFutureTask<T> extends FutureTask<T>
-    {
-        private final TraceState state;
-
-        public TraceSessionFutureTask(Callable<T> callable, TraceState state)
-        {
-            super(callable);
-            this.state = state;
-        }
-
-        public TraceSessionFutureTask(Runnable runnable, T result, TraceState state)
-        {
-            super(runnable, result);
-            this.state = state;
-        }
-
-        public void run()
-        {
-            TraceState oldState = Tracing.instance.get();
-            Tracing.instance.set(state);
-            try
-            {
-                super.run();
-            }
-            finally
-            {
-                Tracing.instance.set(oldState);
-            }
-        }
-    }
-
-    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
-    {
-        private boolean failure;
-        private Object result = this;
-        private final Callable<T> callable;
-
-        public FutureTask(Callable<T> callable)
-        {
-            this.callable = callable;
-        }
-        public FutureTask(Runnable runnable, T result)
-        {
-            this(Executors.callable(runnable, result));
-        }
-
-        public void run()
-        {
-            try
-            {
-                result = callable.call();
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
-                result = t;
-                failure = true;
-            }
-            finally
-            {
-                signalAll();
-                onCompletion();
-            }
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return isSignaled();
-        }
-
-        public T get() throws InterruptedException, ExecutionException
-        {
-            await();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-
-        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            await(timeout, unit);
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-    }
-
-    private <T> FutureTask<T> submit(FutureTask<T> task)
-    {
-        addTask(task);
-        return task;
-    }
-
-    public void execute(Runnable command)
-    {
-        addTask(newTaskFor(command, null));
-    }
-
-    public void execute(Runnable command, TraceState state)
-    {
-        addTask(newTaskFor(command, null, state));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index a6d0049..1fb0690 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -44,7 +44,7 @@ import static org.apache.cassandra.tracing.Tracing.isTracing;
  *   threads and the queue is full, we want the enqueuer to block.  But to allow the number of threads to drop if a
  *   stage is less busy, core thread timeout is enabled.
  */
-public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
 {
     protected static final Logger logger = LoggerFactory.getLogger(DebuggableThreadPoolExecutor.class);
     public static final RejectedExecutionHandler blockingExecutionHandler = new RejectedExecutionHandler()
@@ -146,11 +146,11 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     protected void onFinalAccept(Runnable task) {}
     protected void onFinalRejection(Runnable task) {}
 
-    public void execute(Runnable command, TraceState state)
+    public void execute(Runnable command, ExecutorLocals locals)
     {
-        super.execute(state == null || command instanceof TraceSessionWrapper
+        super.execute(locals == null || command instanceof LocalSessionWrapper
                       ? command
-                      : new TraceSessionWrapper<Object>(command, state));
+                      : new LocalSessionWrapper<Object>(command, locals));
     }
 
     public void maybeExecuteImmediately(Runnable command)
@@ -162,17 +162,17 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     public void execute(Runnable command)
     {
-        super.execute(isTracing() && !(command instanceof TraceSessionWrapper)
-                      ? new TraceSessionWrapper<Object>(Executors.callable(command, null))
+        super.execute(isTracing() && !(command instanceof LocalSessionWrapper)
+                      ? new LocalSessionWrapper<Object>(Executors.callable(command, null))
                       : command);
     }
 
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T result)
     {
-        if (isTracing() && !(runnable instanceof TraceSessionWrapper))
+        if (isTracing() && !(runnable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(Executors.callable(runnable, result));
+            return new LocalSessionWrapper<T>(Executors.callable(runnable, result));
         }
         return super.newTaskFor(runnable, result);
     }
@@ -180,9 +180,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
     {
-        if (isTracing() && !(callable instanceof TraceSessionWrapper))
+        if (isTracing() && !(callable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(callable);
+            return new LocalSessionWrapper<T>(callable);
         }
         return super.newTaskFor(callable);
     }
@@ -198,9 +198,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
 
     protected static void maybeResetTraceSessionWrapper(Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
+        if (r instanceof LocalSessionWrapper)
         {
-            TraceSessionWrapper tsw = (TraceSessionWrapper) r;
+            LocalSessionWrapper tsw = (LocalSessionWrapper) r;
             // we have to reset trace state as its presence is what denotes the current thread is tracing
             // and if left this thread might start tracing unrelated tasks
             tsw.reset();
@@ -210,8 +210,8 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected void beforeExecute(Thread t, Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
-            ((TraceSessionWrapper) r).setupContext();
+        if (r instanceof LocalSessionWrapper)
+            ((LocalSessionWrapper) r).setupContext();
 
         super.beforeExecute(t, r);
     }
@@ -278,35 +278,35 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     }
 
     /**
-     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the TraceSessionContext and move
-     * it into the worker thread.
+     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the ExecutorLocals and move
+     * them into the worker thread.
      *
      * @param <T>
      */
-    private static class TraceSessionWrapper<T> extends FutureTask<T>
+    private static class LocalSessionWrapper<T> extends FutureTask<T>
     {
-        private final TraceState state;
+        private final ExecutorLocals locals;
 
-        public TraceSessionWrapper(Callable<T> callable)
+        public LocalSessionWrapper(Callable<T> callable)
         {
             super(callable);
-            state = Tracing.instance.get();
+            locals = ExecutorLocals.create();
         }
 
-        public TraceSessionWrapper(Runnable command, TraceState state)
+        public LocalSessionWrapper(Runnable command, ExecutorLocals locals)
         {
             super(command, null);
-            this.state = state;
+            this.locals = locals;
         }
 
         private void setupContext()
         {
-            Tracing.instance.set(state);
+            ExecutorLocals.set(locals);
         }
 
         private void reset()
         {
-            Tracing.instance.set(null);
+            ExecutorLocals.set(null);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
new file mode 100644
index 0000000..47826f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
@@ -0,0 +1,44 @@
+/*
+ * 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.concurrent;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+public interface ExecutorLocal<T>
+{
+    ExecutorLocal[] all = { Tracing.instance, ClientWarn.instance };
+
+    /**
+     * This is called when scheduling the task, and also before calling {@link ExecutorLocal#set(T)} when running on a
+     * executor thread.
+     *
+     * @return The thread-local value that we want to copy across executor boundaries; may be null if not set.
+     */
+    T get();
+
+    /**
+     * Before a task has been run, this will be called with the value from the thread that scheduled the task, and after
+     * the task is finished, the value that was previously retrieved from this thread is restored.
+     *
+     * @param value Value to use for the executor local state; may be null.
+     */
+    void set(T value);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
new file mode 100644
index 0000000..8e6d6ea
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
@@ -0,0 +1,84 @@
+/*
+ * 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.concurrent;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+/*
+ * This class only knows about Tracing and ClientWarn, so if any different executor locals are added, it must be
+ * updated.
+ *
+ * We don't enumerate the ExecutorLocal.all array each time because it would be much slower.
+ */
+public class ExecutorLocals
+{
+    private static final ExecutorLocal<TraceState> tracing = Tracing.instance;
+    private static final ExecutorLocal<ClientWarn.State> clientWarn = ClientWarn.instance;
+
+    public final TraceState traceState;
+    public final ClientWarn.State clientWarnState;
+
+    private ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
+    {
+        this.traceState = traceState;
+        this.clientWarnState = clientWarnState;
+    }
+
+    static
+    {
+        assert Arrays.equals(ExecutorLocal.all, new ExecutorLocal[]{ tracing, clientWarn })
+        : "ExecutorLocals has not been updated to reflect new ExecutorLocal.all";
+    }
+
+    /**
+     * This creates a new ExecutorLocals object based on what is already set.
+     *
+     * @return an ExecutorLocals object which has the trace state and client warn state captured if either has been set,
+     *         or null if both are unset. The null result short-circuits logic in
+     *         {@link AbstractLocalAwareExecutorService#newTaskFor(Runnable, Object, ExecutorLocals)}, preventing
+     *         unnecessarily calling {@link ExecutorLocals#set(ExecutorLocals)}.
+     */
+    public static ExecutorLocals create()
+    {
+        TraceState traceState = tracing.get();
+        ClientWarn.State clientWarnState = clientWarn.get();
+        if (traceState == null && clientWarnState == null)
+            return null;
+        else
+            return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static ExecutorLocals create(TraceState traceState)
+    {
+        ClientWarn.State clientWarnState = clientWarn.get();
+        return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static void set(ExecutorLocals locals)
+    {
+        TraceState traceState = locals == null ? null : locals.traceState;
+        ClientWarn.State clientWarnState = locals == null ? null : locals.clientWarnState;
+        tracing.set(traceState);
+        clientWarn.set(clientWarnState);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
new file mode 100644
index 0000000..5577d59
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * 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.concurrent;
+
+import java.util.concurrent.ExecutorService;
+
+public interface LocalAwareExecutorService extends ExecutorService
+{
+    // we need a way to inject a TraceState directly into the Executor context without going through
+    // the global Tracing sessions; see CASSANDRA-5668
+    public void execute(Runnable command, ExecutorLocals locals);
+
+    // permits executing in the context of the submitting thread
+    public void maybeExecuteImmediately(Runnable command);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index d9a0fa8..8b12b82 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.cassandra.metrics.SEPMetrics;
@@ -30,7 +29,7 @@ import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 
-public class SEPExecutor extends AbstractTracingAwareExecutorService
+public class SEPExecutor extends AbstractLocalAwareExecutorService
 {
     private final SharedExecutorPool pool;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index 8c18c44..dfd7011 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -103,7 +103,7 @@ public class SharedExecutorPool
             schedule(Work.SPINNING);
     }
 
-    public TracingAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
+    public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
     {
         SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name);
         executors.add(executor);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 4f03fd5..114795e 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -39,7 +39,7 @@ public class StageManager
 {
     private static final Logger logger = LoggerFactory.getLogger(StageManager.class);
 
-    private static final EnumMap<Stage, TracingAwareExecutorService> stages = new EnumMap<Stage, TracingAwareExecutorService>(Stage.class);
+    private static final EnumMap<Stage, LocalAwareExecutorService> stages = new EnumMap<Stage, LocalAwareExecutorService>(Stage.class);
 
     public static final long KEEPALIVE = 60; // seconds to keep "extra" threads alive for when idle
 
@@ -87,7 +87,7 @@ public class StageManager
                                                 stage.getJmxType());
     }
 
-    private static TracingAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
+    private static LocalAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
     {
         return SharedExecutorPool.SHARED.newExecutor(numThreads, Integer.MAX_VALUE, stage.getJmxType(), stage.getJmxName());
     }
@@ -96,7 +96,7 @@ public class StageManager
      * Retrieve a stage from the StageManager
      * @param stage name of the stage to be retrieved.
      */
-    public static TracingAwareExecutorService getStage(Stage stage)
+    public static LocalAwareExecutorService getStage(Stage stage)
     {
         return stages.get(stage);
     }
@@ -116,16 +116,16 @@ public class StageManager
      * A TPE that disallows submit so that we don't need to worry about unwrapping exceptions on the
      * tracing stage.  See CASSANDRA-1123 for background.
      */
-    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
     {
         public ExecuteOnlyExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
         {
             super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
         }
 
-        public void execute(Runnable command, TraceState state)
+        public void execute(Runnable command, ExecutorLocals locals)
         {
-            assert state == null;
+            assert locals == null;
             super.execute(command);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
deleted file mode 100644
index f580fea..0000000
--- a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 
- * 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.concurrent;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.cassandra.tracing.TraceState;
-
-public interface TracingAwareExecutorService extends ExecutorService
-{
-    // we need a way to inject a TraceState directly into the Executor context without going through
-    // the global Tracing sessions; see CASSANDRA-5668
-    public void execute(Runnable command, TraceState state);
-
-    // permits executing in the context of the submitting thread
-    public void maybeExecuteImmediately(Runnable command);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 43a80bb..a289ad1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -273,7 +273,7 @@ public class BatchStatement implements CQLStatement
             {
                 logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
             }
-            ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {ksCfPairs, size, warnThreshold, size - warnThreshold, ""}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
         }
     }
 
@@ -305,8 +305,13 @@ public class BatchStatement implements CQLStatement
                              keySet.size(), keySet.size() == 1 ? "" : "s",
                              ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
 
-            ClientWarn.warn(MessageFormatter.arrayFormat(unloggedBatchWarning, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
-                                                    ksCfPairs.size() == 1 ? "" : "s", ksCfPairs}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
+                                                                  new Object[]{
+                                                                              keySet.size(),
+                                                                              keySet.size() == 1 ? "" : "s",
+                                                                              ksCfPairs.size() == 1 ? "" : "s",
+                                                                              ksCfPairs
+                                                                  }).getMessage());
 
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 5f142ce..5cfa94b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -251,12 +251,12 @@ public class SelectStatement implements CQLStatement
         if (!restrictions.hasPartitionKeyRestrictions())
         {
             logger.warn("Aggregation query used without partition key");
-            ClientWarn.warn("Aggregation query used without partition key");
+            ClientWarn.instance.warn("Aggregation query used without partition key");
         }
         else if (restrictions.keyIsInRelation())
         {
             logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-            ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+            ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
         }
 
         Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index e1a68e7..d2f0bf4 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -303,7 +303,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
                                        container.metadata().getKeyValidator().getString(key.getKey()),
                                        count,
                                        getSlicesInfo(container));
-            ClientWarn.warn(msg);
+            ClientWarn.instance.warn(msg);
             logger.warn(msg);
         }
         Tracing.trace("Read {} live and {} tombstone cells{}",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 61e58c2..459923b 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -43,10 +43,12 @@ import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.concurrent.ExecutorLocals;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.*;
@@ -801,10 +803,10 @@ public final class MessagingService implements MessagingServiceMBean
                 return;
 
         Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
-        TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+        LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
         assert stage != null : "No stage for message type " + message.verb;
 
-        stage.execute(runnable, state);
+        stage.execute(runnable, ExecutorLocals.create(state));
     }
 
     public void setCallbackForTests(int messageId, CallbackInfo callback)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/ClientWarn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java
index 2ed0a6c..ddad197 100644
--- a/src/java/org/apache/cassandra/service/ClientWarn.java
+++ b/src/java/org/apache/cassandra/service/ClientWarn.java
@@ -20,54 +20,68 @@ package org.apache.cassandra.service;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class ClientWarn
+public class ClientWarn implements ExecutorLocal<ClientWarn.State>
 {
     private static final String TRUNCATED = " [truncated]";
-    private static final ThreadLocal<ClientWarn> warnLocal = new ThreadLocal<>();
-
-    private final List<String> warnings = new ArrayList<>();
+    private static final ThreadLocal<ClientWarn.State> warnLocal = new ThreadLocal<>();
+    public static ClientWarn instance = new ClientWarn();
 
     private ClientWarn()
     {
     }
 
-    public static void warn(String text)
-    {
-        ClientWarn warner = warnLocal.get();
-        if (warner != null)
-            warner.add(text);
+    public State get() {
+        return warnLocal.get();
     }
 
-    private void add(String warning)
-    {
-        if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
-            warnings.add(maybeTruncate(warning));
+    public void set(State value) {
+        warnLocal.set(value);
     }
 
-    private static String maybeTruncate(String warning)
+    public void warn(String text)
     {
-        return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
-             ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
-             : warning;
+        State state = warnLocal.get();
+        if (state != null)
+            state.add(text);
     }
 
-    public static void captureWarnings()
+    public void captureWarnings()
     {
-        warnLocal.set(new ClientWarn());
+        warnLocal.set(new State());
     }
 
-    public static List<String> getWarnings()
+    public List<String> getWarnings()
     {
-        ClientWarn warner = warnLocal.get();
-        if (warner == null || warner.warnings.isEmpty())
+        State state = warnLocal.get();
+        if (state == null || state.warnings.isEmpty())
             return null;
-        return warner.warnings;
+        return state.warnings;
     }
 
-    public static void resetWarnings()
+    public void resetWarnings()
     {
         warnLocal.remove();
     }
+
+    public static class State
+    {
+        private final List<String> warnings = new ArrayList<>();
+
+        private void add(String warning)
+        {
+            if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
+                warnings.add(maybeTruncate(warning));
+        }
+
+        private static String maybeTruncate(String warning)
+        {
+            return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
+                   ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
+                   : warning;
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 88253e3..841e980 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -1842,7 +1842,7 @@ public class StorageProxy implements StorageProxyMBean
                     if (filteredEndpoints.size() == 1
                         && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
                     {
-                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler), Tracing.instance.get());
+                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index ccc2637..bf9cee7 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.net.MessageIn;
@@ -41,7 +42,7 @@ import org.apache.cassandra.utils.UUIDGen;
  * A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
  * have multiple local and remote events before it is completed. All events and sessions are stored at keyspace.
  */
-public class Tracing
+public class Tracing implements ExecutorLocal<TraceState>
 {
     public static final String TRACE_HEADER = "TraceSession";
     public static final String TRACE_TYPE = "TraceType";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index ab794df..01a0794 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -499,14 +499,14 @@ public abstract class Message
                 assert request.connection() instanceof ServerConnection;
                 connection = (ServerConnection)request.connection();
                 if (connection.getVersion() >= Server.VERSION_4)
-                    ClientWarn.captureWarnings();
+                    ClientWarn.instance.captureWarnings();
 
                 QueryState qstate = connection.validateNewMessage(request.type, connection.getVersion(), request.getStreamId());
 
                 logger.trace("Received: {}, v={}", request, connection.getVersion());
                 response = request.execute(qstate);
                 response.setStreamId(request.getStreamId());
-                response.setWarnings(ClientWarn.getWarnings());
+                response.setWarnings(ClientWarn.instance.getWarnings());
                 response.attach(connection);
                 connection.applyStateTransition(request.type, response.type);
             }
@@ -519,7 +519,7 @@ public abstract class Message
             }
             finally
             {
-                ClientWarn.resetWarnings();
+                ClientWarn.instance.resetWarnings();
             }
 
             logger.trace("Responding: {}, v={}", response, connection.getVersion());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
index 4ecd6a7..289f3e3 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
 import io.netty.util.concurrent.AbstractEventExecutor;
 import io.netty.util.concurrent.EventExecutorGroup;
 import io.netty.util.concurrent.Future;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED;
@@ -32,7 +32,7 @@ public class RequestThreadPoolExecutor extends AbstractEventExecutor
 {
     private final static int MAX_QUEUED_REQUESTS = 128;
     private final static String THREAD_FACTORY_ID = "Native-Transport-Requests";
-    private final TracingAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
+    private final LocalAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
                                                                            MAX_QUEUED_REQUESTS,
                                                                            "transport",
                                                                            THREAD_FACTORY_ID);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index cfd5f7a..d22a8f6 100644
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@ -24,6 +24,8 @@ import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.transport.Server;
@@ -79,6 +81,47 @@ public class ClientWarningsTest extends CQLTester
     }
 
     @Test
+    public void testTombstoneWarning() throws Exception
+    {
+        final int iterations = 10000;
+        createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+            store.forceBlockingFlush();
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            store.forceBlockingFlush();
+
+            {
+                QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                    KEYSPACE,
+                                                                    currentTable()), QueryOptions.DEFAULT);
+                Message.Response resp = client.execute(query);
+                assertEquals(1, resp.getWarnings().size());
+            }
+        }
+    }
+
+    @Test
     public void testLargeBatchWithProtoV2() throws Exception
     {
         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");


[12/13] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3

Posted by ca...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.3


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

Branch: refs/heads/trunk
Commit: 93f6529393baa83bd232c48f94b1805a996ceb9a
Parents: a7feb80 94e7ef1
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 14:46:34 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:46:34 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 230 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 230 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cassandra/cql3/functions/UDFunction.java    |   2 +-
 .../cql3/statements/BatchStatement.java         |   9 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   7 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |   6 +-
 .../cassandra/service/ClientWarningsTest.java   |  58 +++++
 24 files changed, 545 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index aae5efe,a37ec99..85bc100
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -7,29 -4,6 +7,30 @@@ Merged from 3.0
     tombstone (CASSANDRA-10743)
   * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
   * Fix potential assertion error during compaction (CASSANDRA-10944)
 +Merged from 2.2:
++ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
 + * Fix error streaming section more than 2GB (CASSANDRA-10961)
 + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975)
 + * Enable GC logging by default (CASSANDRA-10140)
 + * Optimize pending range computation (CASSANDRA-9258)
 + * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
 + * drop/alter user should be case sensitive (CASSANDRA-10817)
 +Merged from 2.1:
 + * (cqlsh) Add request timeout option to cqlsh (CASSANDRA-10686)
 + * Avoid AssertionError while submitting hint with LWT (CASSANDRA-10477)
 + * If CompactionMetadata is not in stats file, use index summary instead (CASSANDRA-10676)
 + * Retry sending gossip syn multiple times during shadow round (CASSANDRA-8072)
 + * Fix pending range calculation during moves (CASSANDRA-10887)
 + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-8708)
 +
 +
 +3.2
 + * Make sure tokens don't exist in several data directories (CASSANDRA-6696)
 + * Add requireAuthorization method to IAuthorizer (CASSANDRA-10852)
 + * Move static JVM options to conf/jvm.options file (CASSANDRA-10494)
 + * Fix CassandraVersion to accept x.y version string (CASSANDRA-10931)
 + * Add forceUserDefinedCleanup to allow more flexible cleanup (CASSANDRA-10708)
 + * (cqlsh) allow setting TTL with COPY (CASSANDRA-9494)
   * Fix counting of received sstables in streaming (CASSANDRA-10949)
   * Implement hints compression (CASSANDRA-9428)
   * Fix potential assertion error when reading static columns (CASSANDRA-10903)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/MessagingService.java
index d95c49b,d416dca..76f4967
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@@ -820,11 -821,11 +821,11 @@@ public final class MessagingService imp
              if (!ms.allowIncomingMessage(message, id))
                  return;
  
 -        Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
 +        Runnable runnable = new MessageDeliveryTask(message, id);
-         TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+         LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
          assert stage != null : "No stage for message type " + message.verb;
  
-         stage.execute(runnable, state);
+         stage.execute(runnable, ExecutorLocals.create(state));
      }
  
      public void setCallbackForTests(int messageId, CallbackInfo callback)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------


[02/13] cassandra git commit: Make sure client gets tombstone overwhelmed warning

Posted by ca...@apache.org.
Make sure client gets tombstone overwhelmed warning

patch by Carl Yeksigian; reviewed by Josh McKenzie for CASSANDRA-9465


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

Branch: refs/heads/cassandra-3.0
Commit: dbf6e62c382d62f9c1727ecf5afb90d131a81775
Parents: 582bdba
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 13:22:36 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:35:59 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 229 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 229 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cql3/statements/BatchStatement.java         |  11 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../cassandra/db/filter/SliceQueryFilter.java   |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   8 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cassandra/service/ClientWarningsTest.java   |  43 ++++
 21 files changed, 529 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11f2529..6530956 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.5
+ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
  * Fix error streaming section more than 2GB (CASSANDRA-10961)
  * (cqlsh) Also apply --connect-timeout to control connection
    timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
new file mode 100644
index 0000000..088b43e
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -0,0 +1,229 @@
+/*
+ * 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.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.tracing.Tracing.isTracing;
+
+public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+{
+    private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+
+    protected abstract void addTask(FutureTask<?> futureTask);
+    protected abstract void onCompletion();
+
+    /** Task Submission / Creation / Objects **/
+
+    public <T> FutureTask<T> submit(Callable<T> task)
+    {
+        return submit(newTaskFor(task));
+    }
+
+    public FutureTask<?> submit(Runnable task)
+    {
+        return submit(newTaskFor(task, null));
+    }
+
+    public <T> FutureTask<T> submit(Runnable task, T result)
+    {
+        return submit(newTaskFor(task, result));
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+    {
+        return newTaskFor(runnable, result, ExecutorLocals.create());
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+    {
+        if (locals != null)
+        {
+            if (runnable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) runnable;
+            return new LocalSessionFutureTask<T>(runnable, result, locals);
+        }
+        if (runnable instanceof FutureTask)
+            return (FutureTask<T>) runnable;
+        return new FutureTask<>(runnable, result);
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+    {
+        if (isTracing())
+        {
+            if (callable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) callable;
+            return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+        }
+        if (callable instanceof FutureTask)
+            return (FutureTask<T>) callable;
+        return new FutureTask<>(callable);
+    }
+
+    private class LocalSessionFutureTask<T> extends FutureTask<T>
+    {
+        private final ExecutorLocals locals;
+
+        public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+        {
+            super(callable);
+            this.locals = locals;
+        }
+
+        public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+        {
+            super(runnable, result);
+            this.locals = locals;
+        }
+
+        public void run()
+        {
+            ExecutorLocals old = ExecutorLocals.create();
+            ExecutorLocals.set(locals);
+            try
+            {
+                super.run();
+            }
+            finally
+            {
+                ExecutorLocals.set(old);
+            }
+        }
+    }
+
+    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+    {
+        private boolean failure;
+        private Object result = this;
+        private final Callable<T> callable;
+
+        public FutureTask(Callable<T> callable)
+        {
+            this.callable = callable;
+        }
+        public FutureTask(Runnable runnable, T result)
+        {
+            this(Executors.callable(runnable, result));
+        }
+
+        public void run()
+        {
+            try
+            {
+                result = callable.call();
+            }
+            catch (Throwable t)
+            {
+                JVMStabilityInspector.inspectThrowable(t);
+                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                result = t;
+                failure = true;
+            }
+            finally
+            {
+                signalAll();
+                onCompletion();
+            }
+        }
+
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            return false;
+        }
+
+        public boolean isCancelled()
+        {
+            return false;
+        }
+
+        public boolean isDone()
+        {
+            return isSignaled();
+        }
+
+        public T get() throws InterruptedException, ExecutionException
+        {
+            await();
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+
+        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            await(timeout, unit);
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+    }
+
+    private <T> FutureTask<T> submit(FutureTask<T> task)
+    {
+        addTask(task);
+        return task;
+    }
+
+    public void execute(Runnable command)
+    {
+        addTask(newTaskFor(command, ExecutorLocals.create()));
+    }
+
+    public void execute(Runnable command, ExecutorLocals locals)
+    {
+        addTask(newTaskFor(command, null, locals));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
deleted file mode 100644
index fb753b0..0000000
--- a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.concurrent;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-public abstract class AbstractTracingAwareExecutorService implements TracingAwareExecutorService
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractTracingAwareExecutorService.class);
-
-    protected abstract void addTask(FutureTask<?> futureTask);
-    protected abstract void onCompletion();
-
-    /** Task Submission / Creation / Objects **/
-
-    public <T> FutureTask<T> submit(Callable<T> task)
-    {
-        return submit(newTaskFor(task));
-    }
-
-    public FutureTask<?> submit(Runnable task)
-    {
-        return submit(newTaskFor(task, null));
-    }
-
-    public <T> FutureTask<T> submit(Runnable task, T result)
-    {
-        return submit(newTaskFor(task, result));
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
-    {
-        return newTaskFor(runnable, result, Tracing.instance.get());
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, TraceState traceState)
-    {
-        if (traceState != null)
-        {
-            if (runnable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) runnable;
-            return new TraceSessionFutureTask<T>(runnable, result, traceState);
-        }
-        if (runnable instanceof FutureTask)
-            return (FutureTask<T>) runnable;
-        return new FutureTask<>(runnable, result);
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing())
-        {
-            if (callable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) callable;
-            return new TraceSessionFutureTask<T>(callable, Tracing.instance.get());
-        }
-        if (callable instanceof FutureTask)
-            return (FutureTask<T>) callable;
-        return new FutureTask<>(callable);
-    }
-
-    private class TraceSessionFutureTask<T> extends FutureTask<T>
-    {
-        private final TraceState state;
-
-        public TraceSessionFutureTask(Callable<T> callable, TraceState state)
-        {
-            super(callable);
-            this.state = state;
-        }
-
-        public TraceSessionFutureTask(Runnable runnable, T result, TraceState state)
-        {
-            super(runnable, result);
-            this.state = state;
-        }
-
-        public void run()
-        {
-            TraceState oldState = Tracing.instance.get();
-            Tracing.instance.set(state);
-            try
-            {
-                super.run();
-            }
-            finally
-            {
-                Tracing.instance.set(oldState);
-            }
-        }
-    }
-
-    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
-    {
-        private boolean failure;
-        private Object result = this;
-        private final Callable<T> callable;
-
-        public FutureTask(Callable<T> callable)
-        {
-            this.callable = callable;
-        }
-        public FutureTask(Runnable runnable, T result)
-        {
-            this(Executors.callable(runnable, result));
-        }
-
-        public void run()
-        {
-            try
-            {
-                result = callable.call();
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
-                result = t;
-                failure = true;
-            }
-            finally
-            {
-                signalAll();
-                onCompletion();
-            }
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return isSignaled();
-        }
-
-        public T get() throws InterruptedException, ExecutionException
-        {
-            await();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-
-        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            await(timeout, unit);
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-    }
-
-    private <T> FutureTask<T> submit(FutureTask<T> task)
-    {
-        addTask(task);
-        return task;
-    }
-
-    public void execute(Runnable command)
-    {
-        addTask(newTaskFor(command, null));
-    }
-
-    public void execute(Runnable command, TraceState state)
-    {
-        addTask(newTaskFor(command, null, state));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index a6d0049..1fb0690 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -44,7 +44,7 @@ import static org.apache.cassandra.tracing.Tracing.isTracing;
  *   threads and the queue is full, we want the enqueuer to block.  But to allow the number of threads to drop if a
  *   stage is less busy, core thread timeout is enabled.
  */
-public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
 {
     protected static final Logger logger = LoggerFactory.getLogger(DebuggableThreadPoolExecutor.class);
     public static final RejectedExecutionHandler blockingExecutionHandler = new RejectedExecutionHandler()
@@ -146,11 +146,11 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     protected void onFinalAccept(Runnable task) {}
     protected void onFinalRejection(Runnable task) {}
 
-    public void execute(Runnable command, TraceState state)
+    public void execute(Runnable command, ExecutorLocals locals)
     {
-        super.execute(state == null || command instanceof TraceSessionWrapper
+        super.execute(locals == null || command instanceof LocalSessionWrapper
                       ? command
-                      : new TraceSessionWrapper<Object>(command, state));
+                      : new LocalSessionWrapper<Object>(command, locals));
     }
 
     public void maybeExecuteImmediately(Runnable command)
@@ -162,17 +162,17 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     public void execute(Runnable command)
     {
-        super.execute(isTracing() && !(command instanceof TraceSessionWrapper)
-                      ? new TraceSessionWrapper<Object>(Executors.callable(command, null))
+        super.execute(isTracing() && !(command instanceof LocalSessionWrapper)
+                      ? new LocalSessionWrapper<Object>(Executors.callable(command, null))
                       : command);
     }
 
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T result)
     {
-        if (isTracing() && !(runnable instanceof TraceSessionWrapper))
+        if (isTracing() && !(runnable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(Executors.callable(runnable, result));
+            return new LocalSessionWrapper<T>(Executors.callable(runnable, result));
         }
         return super.newTaskFor(runnable, result);
     }
@@ -180,9 +180,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
     {
-        if (isTracing() && !(callable instanceof TraceSessionWrapper))
+        if (isTracing() && !(callable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(callable);
+            return new LocalSessionWrapper<T>(callable);
         }
         return super.newTaskFor(callable);
     }
@@ -198,9 +198,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
 
     protected static void maybeResetTraceSessionWrapper(Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
+        if (r instanceof LocalSessionWrapper)
         {
-            TraceSessionWrapper tsw = (TraceSessionWrapper) r;
+            LocalSessionWrapper tsw = (LocalSessionWrapper) r;
             // we have to reset trace state as its presence is what denotes the current thread is tracing
             // and if left this thread might start tracing unrelated tasks
             tsw.reset();
@@ -210,8 +210,8 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected void beforeExecute(Thread t, Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
-            ((TraceSessionWrapper) r).setupContext();
+        if (r instanceof LocalSessionWrapper)
+            ((LocalSessionWrapper) r).setupContext();
 
         super.beforeExecute(t, r);
     }
@@ -278,35 +278,35 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     }
 
     /**
-     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the TraceSessionContext and move
-     * it into the worker thread.
+     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the ExecutorLocals and move
+     * them into the worker thread.
      *
      * @param <T>
      */
-    private static class TraceSessionWrapper<T> extends FutureTask<T>
+    private static class LocalSessionWrapper<T> extends FutureTask<T>
     {
-        private final TraceState state;
+        private final ExecutorLocals locals;
 
-        public TraceSessionWrapper(Callable<T> callable)
+        public LocalSessionWrapper(Callable<T> callable)
         {
             super(callable);
-            state = Tracing.instance.get();
+            locals = ExecutorLocals.create();
         }
 
-        public TraceSessionWrapper(Runnable command, TraceState state)
+        public LocalSessionWrapper(Runnable command, ExecutorLocals locals)
         {
             super(command, null);
-            this.state = state;
+            this.locals = locals;
         }
 
         private void setupContext()
         {
-            Tracing.instance.set(state);
+            ExecutorLocals.set(locals);
         }
 
         private void reset()
         {
-            Tracing.instance.set(null);
+            ExecutorLocals.set(null);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
new file mode 100644
index 0000000..47826f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
@@ -0,0 +1,44 @@
+/*
+ * 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.concurrent;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+public interface ExecutorLocal<T>
+{
+    ExecutorLocal[] all = { Tracing.instance, ClientWarn.instance };
+
+    /**
+     * This is called when scheduling the task, and also before calling {@link ExecutorLocal#set(T)} when running on a
+     * executor thread.
+     *
+     * @return The thread-local value that we want to copy across executor boundaries; may be null if not set.
+     */
+    T get();
+
+    /**
+     * Before a task has been run, this will be called with the value from the thread that scheduled the task, and after
+     * the task is finished, the value that was previously retrieved from this thread is restored.
+     *
+     * @param value Value to use for the executor local state; may be null.
+     */
+    void set(T value);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
new file mode 100644
index 0000000..8e6d6ea
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
@@ -0,0 +1,84 @@
+/*
+ * 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.concurrent;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+/*
+ * This class only knows about Tracing and ClientWarn, so if any different executor locals are added, it must be
+ * updated.
+ *
+ * We don't enumerate the ExecutorLocal.all array each time because it would be much slower.
+ */
+public class ExecutorLocals
+{
+    private static final ExecutorLocal<TraceState> tracing = Tracing.instance;
+    private static final ExecutorLocal<ClientWarn.State> clientWarn = ClientWarn.instance;
+
+    public final TraceState traceState;
+    public final ClientWarn.State clientWarnState;
+
+    private ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
+    {
+        this.traceState = traceState;
+        this.clientWarnState = clientWarnState;
+    }
+
+    static
+    {
+        assert Arrays.equals(ExecutorLocal.all, new ExecutorLocal[]{ tracing, clientWarn })
+        : "ExecutorLocals has not been updated to reflect new ExecutorLocal.all";
+    }
+
+    /**
+     * This creates a new ExecutorLocals object based on what is already set.
+     *
+     * @return an ExecutorLocals object which has the trace state and client warn state captured if either has been set,
+     *         or null if both are unset. The null result short-circuits logic in
+     *         {@link AbstractLocalAwareExecutorService#newTaskFor(Runnable, Object, ExecutorLocals)}, preventing
+     *         unnecessarily calling {@link ExecutorLocals#set(ExecutorLocals)}.
+     */
+    public static ExecutorLocals create()
+    {
+        TraceState traceState = tracing.get();
+        ClientWarn.State clientWarnState = clientWarn.get();
+        if (traceState == null && clientWarnState == null)
+            return null;
+        else
+            return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static ExecutorLocals create(TraceState traceState)
+    {
+        ClientWarn.State clientWarnState = clientWarn.get();
+        return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static void set(ExecutorLocals locals)
+    {
+        TraceState traceState = locals == null ? null : locals.traceState;
+        ClientWarn.State clientWarnState = locals == null ? null : locals.clientWarnState;
+        tracing.set(traceState);
+        clientWarn.set(clientWarnState);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
new file mode 100644
index 0000000..5577d59
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * 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.concurrent;
+
+import java.util.concurrent.ExecutorService;
+
+public interface LocalAwareExecutorService extends ExecutorService
+{
+    // we need a way to inject a TraceState directly into the Executor context without going through
+    // the global Tracing sessions; see CASSANDRA-5668
+    public void execute(Runnable command, ExecutorLocals locals);
+
+    // permits executing in the context of the submitting thread
+    public void maybeExecuteImmediately(Runnable command);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index d9a0fa8..8b12b82 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.cassandra.metrics.SEPMetrics;
@@ -30,7 +29,7 @@ import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 
-public class SEPExecutor extends AbstractTracingAwareExecutorService
+public class SEPExecutor extends AbstractLocalAwareExecutorService
 {
     private final SharedExecutorPool pool;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index 8c18c44..dfd7011 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -103,7 +103,7 @@ public class SharedExecutorPool
             schedule(Work.SPINNING);
     }
 
-    public TracingAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
+    public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
     {
         SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name);
         executors.add(executor);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 4f03fd5..114795e 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -39,7 +39,7 @@ public class StageManager
 {
     private static final Logger logger = LoggerFactory.getLogger(StageManager.class);
 
-    private static final EnumMap<Stage, TracingAwareExecutorService> stages = new EnumMap<Stage, TracingAwareExecutorService>(Stage.class);
+    private static final EnumMap<Stage, LocalAwareExecutorService> stages = new EnumMap<Stage, LocalAwareExecutorService>(Stage.class);
 
     public static final long KEEPALIVE = 60; // seconds to keep "extra" threads alive for when idle
 
@@ -87,7 +87,7 @@ public class StageManager
                                                 stage.getJmxType());
     }
 
-    private static TracingAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
+    private static LocalAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
     {
         return SharedExecutorPool.SHARED.newExecutor(numThreads, Integer.MAX_VALUE, stage.getJmxType(), stage.getJmxName());
     }
@@ -96,7 +96,7 @@ public class StageManager
      * Retrieve a stage from the StageManager
      * @param stage name of the stage to be retrieved.
      */
-    public static TracingAwareExecutorService getStage(Stage stage)
+    public static LocalAwareExecutorService getStage(Stage stage)
     {
         return stages.get(stage);
     }
@@ -116,16 +116,16 @@ public class StageManager
      * A TPE that disallows submit so that we don't need to worry about unwrapping exceptions on the
      * tracing stage.  See CASSANDRA-1123 for background.
      */
-    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
     {
         public ExecuteOnlyExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
         {
             super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
         }
 
-        public void execute(Runnable command, TraceState state)
+        public void execute(Runnable command, ExecutorLocals locals)
         {
-            assert state == null;
+            assert locals == null;
             super.execute(command);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
deleted file mode 100644
index f580fea..0000000
--- a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 
- * 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.concurrent;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.cassandra.tracing.TraceState;
-
-public interface TracingAwareExecutorService extends ExecutorService
-{
-    // we need a way to inject a TraceState directly into the Executor context without going through
-    // the global Tracing sessions; see CASSANDRA-5668
-    public void execute(Runnable command, TraceState state);
-
-    // permits executing in the context of the submitting thread
-    public void maybeExecuteImmediately(Runnable command);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 43a80bb..a289ad1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -273,7 +273,7 @@ public class BatchStatement implements CQLStatement
             {
                 logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
             }
-            ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {ksCfPairs, size, warnThreshold, size - warnThreshold, ""}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
         }
     }
 
@@ -305,8 +305,13 @@ public class BatchStatement implements CQLStatement
                              keySet.size(), keySet.size() == 1 ? "" : "s",
                              ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
 
-            ClientWarn.warn(MessageFormatter.arrayFormat(unloggedBatchWarning, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
-                                                    ksCfPairs.size() == 1 ? "" : "s", ksCfPairs}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
+                                                                  new Object[]{
+                                                                              keySet.size(),
+                                                                              keySet.size() == 1 ? "" : "s",
+                                                                              ksCfPairs.size() == 1 ? "" : "s",
+                                                                              ksCfPairs
+                                                                  }).getMessage());
 
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 5f142ce..5cfa94b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -251,12 +251,12 @@ public class SelectStatement implements CQLStatement
         if (!restrictions.hasPartitionKeyRestrictions())
         {
             logger.warn("Aggregation query used without partition key");
-            ClientWarn.warn("Aggregation query used without partition key");
+            ClientWarn.instance.warn("Aggregation query used without partition key");
         }
         else if (restrictions.keyIsInRelation())
         {
             logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-            ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+            ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
         }
 
         Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index e1a68e7..d2f0bf4 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -303,7 +303,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
                                        container.metadata().getKeyValidator().getString(key.getKey()),
                                        count,
                                        getSlicesInfo(container));
-            ClientWarn.warn(msg);
+            ClientWarn.instance.warn(msg);
             logger.warn(msg);
         }
         Tracing.trace("Read {} live and {} tombstone cells{}",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 61e58c2..459923b 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -43,10 +43,12 @@ import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.concurrent.ExecutorLocals;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.*;
@@ -801,10 +803,10 @@ public final class MessagingService implements MessagingServiceMBean
                 return;
 
         Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
-        TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+        LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
         assert stage != null : "No stage for message type " + message.verb;
 
-        stage.execute(runnable, state);
+        stage.execute(runnable, ExecutorLocals.create(state));
     }
 
     public void setCallbackForTests(int messageId, CallbackInfo callback)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/ClientWarn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java
index 2ed0a6c..ddad197 100644
--- a/src/java/org/apache/cassandra/service/ClientWarn.java
+++ b/src/java/org/apache/cassandra/service/ClientWarn.java
@@ -20,54 +20,68 @@ package org.apache.cassandra.service;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class ClientWarn
+public class ClientWarn implements ExecutorLocal<ClientWarn.State>
 {
     private static final String TRUNCATED = " [truncated]";
-    private static final ThreadLocal<ClientWarn> warnLocal = new ThreadLocal<>();
-
-    private final List<String> warnings = new ArrayList<>();
+    private static final ThreadLocal<ClientWarn.State> warnLocal = new ThreadLocal<>();
+    public static ClientWarn instance = new ClientWarn();
 
     private ClientWarn()
     {
     }
 
-    public static void warn(String text)
-    {
-        ClientWarn warner = warnLocal.get();
-        if (warner != null)
-            warner.add(text);
+    public State get() {
+        return warnLocal.get();
     }
 
-    private void add(String warning)
-    {
-        if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
-            warnings.add(maybeTruncate(warning));
+    public void set(State value) {
+        warnLocal.set(value);
     }
 
-    private static String maybeTruncate(String warning)
+    public void warn(String text)
     {
-        return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
-             ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
-             : warning;
+        State state = warnLocal.get();
+        if (state != null)
+            state.add(text);
     }
 
-    public static void captureWarnings()
+    public void captureWarnings()
     {
-        warnLocal.set(new ClientWarn());
+        warnLocal.set(new State());
     }
 
-    public static List<String> getWarnings()
+    public List<String> getWarnings()
     {
-        ClientWarn warner = warnLocal.get();
-        if (warner == null || warner.warnings.isEmpty())
+        State state = warnLocal.get();
+        if (state == null || state.warnings.isEmpty())
             return null;
-        return warner.warnings;
+        return state.warnings;
     }
 
-    public static void resetWarnings()
+    public void resetWarnings()
     {
         warnLocal.remove();
     }
+
+    public static class State
+    {
+        private final List<String> warnings = new ArrayList<>();
+
+        private void add(String warning)
+        {
+            if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
+                warnings.add(maybeTruncate(warning));
+        }
+
+        private static String maybeTruncate(String warning)
+        {
+            return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
+                   ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
+                   : warning;
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 88253e3..841e980 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -1842,7 +1842,7 @@ public class StorageProxy implements StorageProxyMBean
                     if (filteredEndpoints.size() == 1
                         && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
                     {
-                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler), Tracing.instance.get());
+                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index ccc2637..bf9cee7 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.net.MessageIn;
@@ -41,7 +42,7 @@ import org.apache.cassandra.utils.UUIDGen;
  * A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
  * have multiple local and remote events before it is completed. All events and sessions are stored at keyspace.
  */
-public class Tracing
+public class Tracing implements ExecutorLocal<TraceState>
 {
     public static final String TRACE_HEADER = "TraceSession";
     public static final String TRACE_TYPE = "TraceType";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index ab794df..01a0794 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -499,14 +499,14 @@ public abstract class Message
                 assert request.connection() instanceof ServerConnection;
                 connection = (ServerConnection)request.connection();
                 if (connection.getVersion() >= Server.VERSION_4)
-                    ClientWarn.captureWarnings();
+                    ClientWarn.instance.captureWarnings();
 
                 QueryState qstate = connection.validateNewMessage(request.type, connection.getVersion(), request.getStreamId());
 
                 logger.trace("Received: {}, v={}", request, connection.getVersion());
                 response = request.execute(qstate);
                 response.setStreamId(request.getStreamId());
-                response.setWarnings(ClientWarn.getWarnings());
+                response.setWarnings(ClientWarn.instance.getWarnings());
                 response.attach(connection);
                 connection.applyStateTransition(request.type, response.type);
             }
@@ -519,7 +519,7 @@ public abstract class Message
             }
             finally
             {
-                ClientWarn.resetWarnings();
+                ClientWarn.instance.resetWarnings();
             }
 
             logger.trace("Responding: {}, v={}", response, connection.getVersion());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
index 4ecd6a7..289f3e3 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
 import io.netty.util.concurrent.AbstractEventExecutor;
 import io.netty.util.concurrent.EventExecutorGroup;
 import io.netty.util.concurrent.Future;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED;
@@ -32,7 +32,7 @@ public class RequestThreadPoolExecutor extends AbstractEventExecutor
 {
     private final static int MAX_QUEUED_REQUESTS = 128;
     private final static String THREAD_FACTORY_ID = "Native-Transport-Requests";
-    private final TracingAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
+    private final LocalAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
                                                                            MAX_QUEUED_REQUESTS,
                                                                            "transport",
                                                                            THREAD_FACTORY_ID);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index cfd5f7a..d22a8f6 100644
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@ -24,6 +24,8 @@ import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.transport.Server;
@@ -79,6 +81,47 @@ public class ClientWarningsTest extends CQLTester
     }
 
     @Test
+    public void testTombstoneWarning() throws Exception
+    {
+        final int iterations = 10000;
+        createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+            store.forceBlockingFlush();
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            store.forceBlockingFlush();
+
+            {
+                QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                    KEYSPACE,
+                                                                    currentTable()), QueryOptions.DEFAULT);
+                Message.Response resp = client.execute(query);
+                assertEquals(1, resp.getWarnings().size());
+            }
+        }
+    }
+
+    @Test
     public void testLargeBatchWithProtoV2() throws Exception
     {
         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");


[05/13] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageProxy.java
index e2fa270,841e980..89ac0bb
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@@ -1941,201 -1711,252 +1941,201 @@@ public class StorageProxy implements St
          }
      }
  
 -    public static List<Row> getRangeSlice(AbstractRangeCommand command, ConsistencyLevel consistency_level)
 -    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    private static class SingleRangeResponse extends AbstractIterator<RowIterator> implements PartitionIterator
      {
 -        Tracing.trace("Computing ranges to query");
 -        long startTime = System.nanoTime();
 +        private final ReadCallback handler;
 +        private PartitionIterator result;
  
 -        Keyspace keyspace = Keyspace.open(command.keyspace);
 -        List<Row> rows;
 -        // now scan until we have enough results
 -        try
 +        private SingleRangeResponse(ReadCallback handler)
          {
 -            int liveRowCount = 0;
 -            boolean countLiveRows = command.countCQL3Rows() || command.ignoredTombstonedPartitions();
 -            rows = new ArrayList<>();
 +            this.handler = handler;
 +        }
  
 -            // when dealing with LocalStrategy keyspaces, we can skip the range splitting and merging (which can be
 -            // expensive in clusters with vnodes)
 -            List<? extends AbstractBounds<RowPosition>> ranges;
 -            if (keyspace.getReplicationStrategy() instanceof LocalStrategy)
 -                ranges = command.keyRange.unwrap();
 -            else
 -                ranges = getRestrictedRanges(command.keyRange);
 -
 -            // determine the number of rows to be fetched and the concurrency factor
 -            int rowsToBeFetched = command.limit();
 -            int concurrencyFactor;
 -            if (command.requiresScanningAllRanges())
 -            {
 -                // all nodes must be queried
 -                rowsToBeFetched *= ranges.size();
 -                concurrencyFactor = ranges.size();
 -                logger.debug("Requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {}",
 -                              ranges.size(), concurrencyFactor);
 +        private void waitForResponse() throws ReadTimeoutException
 +        {
 +            if (result != null)
 +                return;
 +
 +            try
 +            {
 +                result = handler.get();
              }
 -            else
 +            catch (DigestMismatchException e)
              {
 -                // our estimate of how many result rows there will be per-range
 -                float resultRowsPerRange = estimateResultRowsPerRange(command, keyspace);
 -                // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 -                // fetch enough rows in the first round
 -                resultRowsPerRange -= resultRowsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 -                concurrencyFactor = resultRowsPerRange == 0.0
 -                                  ? 1
 -                                  : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit() / resultRowsPerRange)));
 -
 -                logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             resultRowsPerRange,
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)",
 -                              ranges.size(),
 -                              concurrencyFactor,
 -                              resultRowsPerRange);
 -            }
 -
 -            boolean haveSufficientRows = false;
 -            int i = 0;
 -            AbstractBounds<RowPosition> nextRange = null;
 -            List<InetAddress> nextEndpoints = null;
 -            List<InetAddress> nextFilteredEndpoints = null;
 -            while (i < ranges.size())
 -            {
 -                List<Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>>> scanHandlers = new ArrayList<>(concurrencyFactor);
 -                int concurrentFetchStartingIndex = i;
 -                int concurrentRequests = 0;
 -                while ((i - concurrentFetchStartingIndex) < concurrencyFactor)
 -                {
 -                    AbstractBounds<RowPosition> range = nextRange == null
 -                                                      ? ranges.get(i)
 -                                                      : nextRange;
 -                    List<InetAddress> liveEndpoints = nextEndpoints == null
 -                                                    ? getLiveSortedEndpoints(keyspace, range.right)
 -                                                    : nextEndpoints;
 -                    List<InetAddress> filteredEndpoints = nextFilteredEndpoints == null
 -                                                        ? consistency_level.filterForQuery(keyspace, liveEndpoints)
 -                                                        : nextFilteredEndpoints;
 -                    ++i;
 -                    ++concurrentRequests;
 -
 -                    // getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take
 -                    // the replication factor into account. If the intersection of live endpoints for 2 consecutive ranges
 -                    // still meets the CL requirements, then we can merge both ranges into the same RangeSliceCommand.
 -                    while (i < ranges.size())
 -                    {
 -                        nextRange = ranges.get(i);
 -                        nextEndpoints = getLiveSortedEndpoints(keyspace, nextRange.right);
 -                        nextFilteredEndpoints = consistency_level.filterForQuery(keyspace, nextEndpoints);
 -
 -                        // If the current range right is the min token, we should stop merging because CFS.getRangeSlice
 -                        // don't know how to deal with a wrapping range.
 -                        // Note: it would be slightly more efficient to have CFS.getRangeSlice on the destination nodes unwraps
 -                        // the range if necessary and deal with it. However, we can't start sending wrapped range without breaking
 -                        // wire compatibility, so It's likely easier not to bother;
 -                        if (range.right.isMinimum())
 -                            break;
 -
 -                        List<InetAddress> merged = intersection(liveEndpoints, nextEndpoints);
 -
 -                        // Check if there is enough endpoint for the merge to be possible.
 -                        if (!consistency_level.isSufficientLiveNodes(keyspace, merged))
 -                            break;
 -
 -                        List<InetAddress> filteredMerged = consistency_level.filterForQuery(keyspace, merged);
 -
 -                        // Estimate whether merging will be a win or not
 -                        if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, filteredEndpoints, nextFilteredEndpoints))
 -                            break;
 -
 -                        // If we get there, merge this range and the next one
 -                        range = range.withNewRight(nextRange.right);
 -                        liveEndpoints = merged;
 -                        filteredEndpoints = filteredMerged;
 -                        ++i;
 -                    }
 +                throw new AssertionError(e); // no digests in range slices yet
 +            }
 +        }
  
 -                    AbstractRangeCommand nodeCmd = command.forSubRange(range);
 +        protected RowIterator computeNext()
 +        {
 +            waitForResponse();
 +            return result.hasNext() ? result.next() : endOfData();
 +        }
  
 -                    // collect replies and resolve according to consistency level
 -                    RangeSliceResponseResolver resolver = new RangeSliceResponseResolver(nodeCmd.keyspace, command.timestamp);
 -                    List<InetAddress> minimalEndpoints = filteredEndpoints.subList(0, Math.min(filteredEndpoints.size(), consistency_level.blockFor(keyspace)));
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = new ReadCallback<>(resolver, consistency_level, nodeCmd, minimalEndpoints);
 -                    handler.assureSufficientLiveNodes();
 -                    resolver.setSources(filteredEndpoints);
 -                    if (filteredEndpoints.size() == 1
 -                        && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
 -                    {
 -                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
 -                    }
 -                    else
 -                    {
 -                        MessageOut<? extends AbstractRangeCommand> message = nodeCmd.createMessage();
 -                        for (InetAddress endpoint : filteredEndpoints)
 -                        {
 -                            Tracing.trace("Enqueuing request to {}", endpoint);
 -                            MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
 -                        }
 -                    }
 -                    scanHandlers.add(Pair.create(nodeCmd, handler));
 -                }
 -                Tracing.trace("Submitted {} concurrent range requests covering {} ranges", concurrentRequests, i - concurrentFetchStartingIndex);
 +        public void close()
 +        {
 +            if (result != null)
 +                result.close();
 +        }
 +    }
  
 -                List<AsyncOneResponse> repairResponses = new ArrayList<>();
 -                for (Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>> cmdPairHandler : scanHandlers)
 -                {
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = cmdPairHandler.right;
 -                    RangeSliceResponseResolver resolver = (RangeSliceResponseResolver)handler.resolver;
 +    private static class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
 +    {
 +        private final Iterator<RangeForQuery> ranges;
 +        private final int totalRangeCount;
 +        private final PartitionRangeReadCommand command;
 +        private final Keyspace keyspace;
 +        private final ConsistencyLevel consistency;
  
 -                    try
 -                    {
 -                        for (Row row : handler.get())
 -                        {
 -                            rows.add(row);
 -                            if (countLiveRows)
 -                                liveRowCount += row.getLiveCount(command.predicate, command.timestamp);
 -                        }
 -                        repairResponses.addAll(resolver.repairResults);
 -                    }
 -                    catch (ReadTimeoutException|ReadFailureException ex)
 -                    {
 -                        // we timed out or failed waiting for responses
 -                        int blockFor = consistency_level.blockFor(keyspace);
 -                        int responseCount = resolver.responses.size();
 -                        String gotData = responseCount > 0
 -                                         ? resolver.isDataPresent() ? " (including data)" : " (only digests)"
 -                                         : "";
 -
 -                        boolean isTimeout = ex instanceof ReadTimeoutException;
 -                        if (Tracing.isTracing())
 -                        {
 -                            Tracing.trace("{}; received {} of {} responses{} for range {} of {}",
 -                                          (isTimeout ? "Timed out" : "Failed"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        else if (logger.isDebugEnabled())
 -                        {
 -                            logger.debug("Range slice {}; received {} of {} responses{} for range {} of {}",
 -                                         (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        throw ex;
 -                    }
 -                    catch (DigestMismatchException e)
 -                    {
 -                        throw new AssertionError(e); // no digests in range slices yet
 -                    }
 +        private final long startTime;
 +        private DataLimits.Counter counter;
 +        private PartitionIterator sentQueryIterator;
  
 -                    // if we're done, great, otherwise, move to the next range
 -                    int count = countLiveRows ? liveRowCount : rows.size();
 -                    if (count >= rowsToBeFetched)
 -                    {
 -                        haveSufficientRows = true;
 -                        break;
 -                    }
 -                }
 +        private int concurrencyFactor;
 +        // The two following "metric" are maintained to improve the concurrencyFactor
 +        // when it was not good enough initially.
 +        private int liveReturned;
 +        private int rangesQueried;
  
 -                try
 -                {
 -                    FBUtilities.waitOnFutures(repairResponses, DatabaseDescriptor.getWriteRpcTimeout());
 -                }
 -                catch (TimeoutException ex)
 +        public RangeCommandIterator(RangeIterator ranges, PartitionRangeReadCommand command, int concurrencyFactor, Keyspace keyspace, ConsistencyLevel consistency)
 +        {
 +            this.command = command;
 +            this.concurrencyFactor = concurrencyFactor;
 +            this.startTime = System.nanoTime();
 +            this.ranges = new RangeMerger(ranges, keyspace, consistency);
 +            this.totalRangeCount = ranges.rangeCount();
 +            this.consistency = consistency;
 +            this.keyspace = keyspace;
 +        }
 +
 +        public RowIterator computeNext()
 +        {
 +            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
 +            {
 +                // If we don't have more range to handle, we're done
 +                if (!ranges.hasNext())
 +                    return endOfData();
 +
 +                // else, sends the next batch of concurrent queries (after having close the previous iterator)
 +                if (sentQueryIterator != null)
                  {
 -                    // We got all responses, but timed out while repairing
 -                    int blockFor = consistency_level.blockFor(keyspace);
 -                    if (Tracing.isTracing())
 -                        Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    else
 -                        logger.debug("Range slice timeout while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    throw new ReadTimeoutException(consistency_level, blockFor-1, blockFor, true);
 +                    liveReturned += counter.counted();
 +                    sentQueryIterator.close();
 +
 +                    // It's not the first batch of queries and we're not done, so we we can use what has been
 +                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
 +                    updateConcurrencyFactor();
                  }
 +                sentQueryIterator = sendNextRequests();
 +            }
 +
 +            return sentQueryIterator.next();
 +        }
 +
 +        private void updateConcurrencyFactor()
 +        {
 +            if (liveReturned == 0)
 +            {
 +                // we haven't actually gotten any results, so query all remaining ranges at once
 +                concurrencyFactor = totalRangeCount - rangesQueried;
 +                return;
 +            }
  
 -                if (haveSufficientRows)
 -                    return command.postReconciliationProcessing(rows);
 +            // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
 +            // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
 +            int remainingRows = command.limits().count() - liveReturned;
 +            float rowsPerRange = (float)liveReturned / (float)rangesQueried;
 +            concurrencyFactor = Math.max(1, Math.min(totalRangeCount - rangesQueried, Math.round(remainingRows / rowsPerRange)));
 +            logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 +                         rowsPerRange, (int) remainingRows, concurrencyFactor);
 +        }
 +
 +        private SingleRangeResponse query(RangeForQuery toQuery)
 +        {
 +            PartitionRangeReadCommand rangeCommand = command.forSubRange(toQuery.range);
 +
 +            DataResolver resolver = new DataResolver(keyspace, rangeCommand, consistency, toQuery.filteredEndpoints.size());
 +
 +            int blockFor = consistency.blockFor(keyspace);
 +            int minResponses = Math.min(toQuery.filteredEndpoints.size(), blockFor);
 +            List<InetAddress> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
 +            ReadCallback handler = new ReadCallback(resolver, consistency, rangeCommand, minimalEndpoints);
  
 -                // we didn't get enough rows in our concurrent fetch; recalculate our concurrency factor
 -                // based on the results we've seen so far (as long as we still have ranges left to query)
 -                if (i < ranges.size())
 +            handler.assureSufficientLiveNodes();
 +
 +            if (toQuery.filteredEndpoints.size() == 1 && canDoLocalRequest(toQuery.filteredEndpoints.get(0)))
 +            {
-                 StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler), Tracing.instance.get());
++                StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler));
 +            }
 +            else
 +            {
 +                for (InetAddress endpoint : toQuery.filteredEndpoints)
                  {
 -                    float fetchedRows = countLiveRows ? liveRowCount : rows.size();
 -                    float remainingRows = rowsToBeFetched - fetchedRows;
 -                    float actualRowsPerRange;
 -                    if (fetchedRows == 0.0)
 -                    {
 -                        // we haven't actually gotten any results, so query all remaining ranges at once
 -                        actualRowsPerRange = 0.0f;
 -                        concurrencyFactor = ranges.size() - i;
 -                    }
 -                    else
 -                    {
 -                        actualRowsPerRange = fetchedRows / i;
 -                        concurrencyFactor = Math.max(1, Math.min(ranges.size() - i, Math.round(remainingRows / actualRowsPerRange)));
 -                    }
 -                    logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 -                                 actualRowsPerRange, (int) remainingRows, concurrencyFactor);
 +                    MessageOut<ReadCommand> message = rangeCommand.createMessage(MessagingService.instance().getVersion(endpoint));
 +                    Tracing.trace("Enqueuing request to {}", endpoint);
 +                    MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
                  }
              }
 +
 +            return new SingleRangeResponse(handler);
          }
 -        finally
 +
 +        private PartitionIterator sendNextRequests()
          {
 -            long latency = System.nanoTime() - startTime;
 -            rangeMetrics.addNano(latency);
 -            Keyspace.open(command.keyspace).getColumnFamilyStore(command.columnFamily).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            List<PartitionIterator> concurrentQueries = new ArrayList<>(concurrencyFactor);
 +            for (int i = 0; i < concurrencyFactor && ranges.hasNext(); i++)
 +            {
 +                concurrentQueries.add(query(ranges.next()));
 +                ++rangesQueried;
 +            }
 +
 +            Tracing.trace("Submitted {} concurrent range requests", concurrentQueries.size());
 +            // We want to count the results for the sake of updating the concurrency factor (see updateConcurrencyFactor) but we don't want to
 +            // enforce any particular limit at this point (this could break code than rely on postReconciliationProcessing), hence the DataLimits.NONE.
 +            counter = DataLimits.NONE.newCounter(command.nowInSec(), true);
 +            return counter.applyTo(PartitionIterators.concat(concurrentQueries));
          }
 -        return command.postReconciliationProcessing(rows);
 +
 +        public void close()
 +        {
 +            try
 +            {
 +                if (sentQueryIterator != null)
 +                    sentQueryIterator.close();
 +            }
 +            finally
 +            {
 +                long latency = System.nanoTime() - startTime;
 +                rangeMetrics.addNano(latency);
 +                Keyspace.openAndGetStore(command.metadata()).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            }
 +        }
 +    }
 +
 +    @SuppressWarnings("resource")
 +    public static PartitionIterator getRangeSlice(PartitionRangeReadCommand command, ConsistencyLevel consistencyLevel)
 +    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    {
 +        Tracing.trace("Computing ranges to query");
 +
 +        Keyspace keyspace = Keyspace.open(command.metadata().ksName);
 +        RangeIterator ranges = new RangeIterator(command, keyspace, consistencyLevel);
 +
 +        // our estimate of how many result rows there will be per-range
 +        float resultsPerRange = estimateResultsPerRange(command, keyspace);
 +        // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 +        // fetch enough rows in the first round
 +        resultsPerRange -= resultsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 +        int concurrencyFactor = resultsPerRange == 0.0
 +                              ? 1
 +                              : Math.max(1, Math.min(ranges.rangeCount(), (int) Math.ceil(command.limits().count() / resultsPerRange)));
 +        logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 +                     resultsPerRange, command.limits().count(), ranges.rangeCount(), concurrencyFactor);
 +        Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)", ranges.rangeCount(), concurrencyFactor, resultsPerRange);
 +
 +        // Note that in general, a RangeCommandIterator will honor the command limit for each range, but will not enforce it globally.
 +
 +        return command.limits().filter(command.postReconciliationProcessing(new RangeCommandIterator(ranges, command, concurrencyFactor, keyspace, consistencyLevel)), command.nowInSec());
      }
  
      public Map<String, List<String>> getSchemaVersions()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 704a6c9,bcfe871..6d54e36
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -2263,226 -2686,4 +2263,226 @@@ public class UFTest extends CQLTeste
                             "AS 'return 0;'");
          }
      }
 +
 +    @Test
 +    public void testSecurityPermissions() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, dval double)");
 +        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
 +
 +        // Java UDFs
 +
 +        try
 +        {
 +            String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                          "RETURNS NULL ON NULL INPUT " +
 +                                          "RETURNS double " +
 +                                          "LANGUAGE JAVA\n" +
 +                                          "AS 'System.getProperty(\"foo.bar.baz\"); return 0d;';");
 +            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +            Assert.fail();
 +        }
 +        catch (FunctionExecutionException e)
 +        {
 +            assertAccessControlException("System.getProperty(\"foo.bar.baz\"); return 0d;", e);
 +        }
 +
 +        String[][] typesAndSources =
 +        {
 +        {"",                        "try { Class.forName(\"" + UDHelper.class.getName() + "\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
 +        {"sun.misc.Unsafe",         "sun.misc.Unsafe.getUnsafe(); return 0d;"},
 +        {"",                        "try { Class.forName(\"sun.misc.Unsafe\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
 +        {"java.nio.file.FileSystems", "try {" +
 +                                    "     java.nio.file.FileSystems.getDefault(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.nio.channels.FileChannel", "try {" +
 +                                    "     java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")).close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.nio.channels.SocketChannel", "try {" +
 +                                    "     java.nio.channels.SocketChannel.open().close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.io.FileInputStream", "try {" +
 +                                    "     new java.io.FileInputStream(\"./foobar\").close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.lang.Runtime",       "try {" +
 +                                    "     java.lang.Runtime.getRuntime(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"org.apache.cassandra.service.StorageService",
 +                                    "try {" +
 +                                    "     org.apache.cassandra.service.StorageService v = org.apache.cassandra.service.StorageService.instance; v.isInShutdownHook(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.net.ServerSocket",   "try {" +
 +                                    "     new java.net.ServerSocket().bind(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.io.FileOutputStream","try {" +
 +                                    "     new java.io.FileOutputStream(\".foo\"); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.lang.Runtime",       "try {" +
 +                                    "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'}
 +        };
 +
 +        for (String[] typeAndSource : typesAndSources)
 +        {
 +            assertInvalidMessage(typeAndSource[0] + " cannot be resolved",
 +                                 "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".invalid_class_access(val double) " +
 +                                 "RETURNS NULL ON NULL INPUT " +
 +                                 "RETURNS double " +
 +                                 "LANGUAGE JAVA\n" +
 +                                 "AS '" + typeAndSource[1] + "';");
 +        }
 +
 +        // JavaScript UDFs
 +
 +        try
 +        {
 +            String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                          "RETURNS NULL ON NULL INPUT " +
 +                                          "RETURNS double " +
 +                                          "LANGUAGE javascript\n" +
 +                                          "AS 'org.apache.cassandra.service.StorageService.instance.isInShutdownHook(); 0;';");
 +            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +            Assert.fail("Javascript security check failed");
 +        }
 +        catch (FunctionExecutionException e)
 +        {
 +            assertAccessControlException("", e);
 +        }
 +
 +        String[] javascript =
 +        {
 +        "java.lang.management.ManagmentFactory.getThreadMXBean(); 0;",
 +        "new java.io.FileInputStream(\"/tmp/foo\"); 0;",
 +        "new java.io.FileOutputStream(\"/tmp/foo\"); 0;",
 +        "java.nio.file.FileSystems.getDefault().createFileExclusively(\"./foo_bar_baz\"); 0;",
 +        "java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")); 0;",
 +        "java.nio.channels.SocketChannel.open(); 0;",
 +        "new java.net.ServerSocket().bind(null); 0;",
 +        "var thread = new java.lang.Thread(); thread.start(); 0;",
 +        "java.lang.System.getProperty(\"foo.bar.baz\"); 0;",
 +        "java.lang.Class.forName(\"java.lang.System\"); 0;",
 +        "java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); 0;",
 +        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
 +        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
 +        // TODO these (ugly) calls are still possible - these can consume CPU (as one could do with an evil loop, too)
 +//        "java.lang.Runtime.getRuntime().traceMethodCalls(true); 0;",
 +//        "java.lang.Runtime.getRuntime().gc(); 0;",
 +//        "java.lang.Runtime.getRuntime(); 0;",
 +        };
 +
 +        for (String script : javascript)
 +        {
 +            try
 +            {
 +                String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                              "RETURNS NULL ON NULL INPUT " +
 +                                              "RETURNS double " +
 +                                              "LANGUAGE javascript\n" +
 +                                              "AS '" + script + "';");
 +                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +                Assert.fail("Javascript security check failed: " + script);
 +            }
 +            catch (FunctionExecutionException e)
 +            {
 +                assertAccessControlException(script, e);
 +            }
 +        }
 +    }
 +
 +    private static void assertAccessControlException(String script, FunctionExecutionException e)
 +    {
 +        for (Throwable t = e; t != null && t != t.getCause(); t = t.getCause())
 +            if (t instanceof AccessControlException)
 +                return;
 +        Assert.fail("no AccessControlException for " + script + " (got " + e + ')');
 +    }
 +
 +    @Test
 +    public void testAmokUDF() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, dval double)");
 +        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
 +
 +        long udfWarnTimeout = DatabaseDescriptor.getUserDefinedFunctionWarnTimeout();
 +        long udfFailTimeout = DatabaseDescriptor.getUserDefinedFunctionFailTimeout();
 +        int maxTries = 5;
 +        for (int i = 1; i <= maxTries; i++)
 +        {
 +            try
 +            {
 +                // short timeout
 +                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(10);
 +                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(250);
 +                // don't kill the unit test... - default policy is "die"
 +                DatabaseDescriptor.setUserFunctionTimeoutPolicy(Config.UserFunctionTimeoutPolicy.ignore);
 +
-                 ClientWarn.captureWarnings();
++                ClientWarn.instance.captureWarnings();
 +                String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                              "RETURNS NULL ON NULL INPUT " +
 +                                              "RETURNS double " +
 +                                              "LANGUAGE JAVA\n" +
 +                                              "AS 'long t=System.currentTimeMillis()+110; while (t>System.currentTimeMillis()) { }; return 0d;'");
 +                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
-                 List<String> warnings = ClientWarn.getWarnings();
++                List<String> warnings = ClientWarn.instance.getWarnings();
 +                Assert.assertNotNull(warnings);
 +                Assert.assertFalse(warnings.isEmpty());
-                 ClientWarn.resetWarnings();
++                ClientWarn.instance.resetWarnings();
 +
 +                // Java UDF
 +
 +                fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS double " +
 +                                       "LANGUAGE JAVA\n" +
 +                                       "AS 'long t=System.currentTimeMillis()+500; while (t>System.currentTimeMillis()) { }; return 0d;';");
 +                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +
 +                // Javascript UDF
 +
 +                fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS double " +
 +                                       "LANGUAGE JAVASCRIPT\n" +
 +                                       "AS 'var t=java.lang.System.currentTimeMillis()+500; while (t>java.lang.System.currentTimeMillis()) { }; 0;';");
 +                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +
 +                return;
 +            }
 +            catch (Error | RuntimeException e)
 +            {
 +                if (i == maxTries)
 +                    throw e;
 +            }
 +            finally
 +            {
 +                // reset to defaults
 +                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(udfWarnTimeout);
 +                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(udfFailTimeout);
 +            }
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index c3f2629,d22a8f6..cf14d55
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@@ -23,8 -22,11 +23,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.cql3.QueryOptions;
  import org.apache.cassandra.cql3.CQLTester;
 +import org.apache.cassandra.cql3.QueryOptions;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Keyspace;
 -import org.apache.cassandra.dht.ByteOrderedPartitioner;
  import org.apache.cassandra.transport.Message;
  import org.apache.cassandra.transport.Server;
  import org.apache.cassandra.transport.SimpleClient;
@@@ -76,6 -80,62 +78,62 @@@ public class ClientWarningsTest extend
          }
      }
  
+     @Test
+     public void testTombstoneWarning() throws Exception
+     {
+         final int iterations = 10000;
+         createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+ 
+         try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+         {
+             client.connect(false);
+ 
+             for (int i = 0; i < iterations; i++)
+             {
+                 QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                     KEYSPACE,
+                                                                     currentTable(),
+                                                                     i), QueryOptions.DEFAULT);
+                 client.execute(query);
+             }
+             ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+             store.forceBlockingFlush();
+ 
+             for (int i = 0; i < iterations; i++)
+             {
+                 QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                     KEYSPACE,
+                                                                     currentTable(),
+                                                                     i), QueryOptions.DEFAULT);
+                 client.execute(query);
+             }
+             store.forceBlockingFlush();
+ 
+             {
+                 QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                     KEYSPACE,
+                                                                     currentTable()), QueryOptions.DEFAULT);
+                 Message.Response resp = client.execute(query);
+                 assertEquals(1, resp.getWarnings().size());
+             }
+         }
+     }
+ 
+     @Test
+     public void testLargeBatchWithProtoV2() throws Exception
+     {
+         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");
+ 
 -        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_2))
++        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_3))
+         {
+             client.connect(false);
+ 
+             QueryMessage query = new QueryMessage(createBatchStatement(DatabaseDescriptor.getBatchSizeWarnThreshold()), QueryOptions.DEFAULT);
+             Message.Response resp = client.execute(query);
+             assertNull(resp.getWarnings());
+         }
+     }
+ 
      private String createBatchStatement(int minSize)
      {
          return String.format("BEGIN UNLOGGED BATCH INSERT INTO %s.%s (pk, v) VALUES (1, '%s') APPLY BATCH;",


[10/13] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ca...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: 94e7ef17757235cb35df70ff9a2a63e1d29d6c41
Parents: 0f995a2 dbf6e62
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 14:45:13 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:45:13 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 230 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 230 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cassandra/cql3/functions/UDFunction.java    |   2 +-
 .../cql3/statements/BatchStatement.java         |   9 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   7 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |   6 +-
 .../cassandra/service/ClientWarningsTest.java   |  58 +++++
 24 files changed, 545 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 614d5b4,6530956..a37ec99
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,21 @@@
 -2.2.5
 +3.0.3
 + * Fix AssertionError when removing from list using UPDATE (CASSANDRA-10954)
 + * Fix UnsupportedOperationException when reading old sstable with range
 +   tombstone (CASSANDRA-10743)
 + * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
 + * Fix potential assertion error during compaction (CASSANDRA-10944)
 + * Fix counting of received sstables in streaming (CASSANDRA-10949)
 + * Implement hints compression (CASSANDRA-9428)
 + * Fix potential assertion error when reading static columns (CASSANDRA-10903)
 + * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711)
 + * Avoid building PartitionUpdate in toString (CASSANDRA-10897)
 + * Reduce heap spent when receiving many SSTables (CASSANDRA-10797)
 + * Add back support for 3rd party auth providers to bulk loader (CASSANDRA-10873)
 + * Eliminate the dependency on jgrapht for UDT resolution (CASSANDRA-10653)
 + * (Hadoop) Close Clusters and Sessions in Hadoop Input/Output classes (CASSANDRA-10837)
 + * Fix sstableloader not working with upper case keyspace name (CASSANDRA-10806)
 +Merged from 2.2:
+  * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
   * Fix error streaming section more than 2GB (CASSANDRA-10961)
   * (cqlsh) Also apply --connect-timeout to control connection
     timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
index 0000000,088b43e..f47d8ac
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@@ -1,0 -1,229 +1,230 @@@
+ /*
+  * 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.concurrent;
+ 
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.TimeoutException;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import org.apache.cassandra.tracing.TraceState;
+ import org.apache.cassandra.tracing.Tracing;
+ import org.apache.cassandra.utils.concurrent.SimpleCondition;
+ import org.apache.cassandra.utils.JVMStabilityInspector;
+ 
+ import static org.apache.cassandra.tracing.Tracing.isTracing;
+ 
+ public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+ {
+     private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+ 
+     protected abstract void addTask(FutureTask<?> futureTask);
+     protected abstract void onCompletion();
+ 
+     /** Task Submission / Creation / Objects **/
+ 
+     public <T> FutureTask<T> submit(Callable<T> task)
+     {
+         return submit(newTaskFor(task));
+     }
+ 
+     public FutureTask<?> submit(Runnable task)
+     {
+         return submit(newTaskFor(task, null));
+     }
+ 
+     public <T> FutureTask<T> submit(Runnable task, T result)
+     {
+         return submit(newTaskFor(task, result));
+     }
+ 
+     public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+     {
+         return newTaskFor(runnable, result, ExecutorLocals.create());
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+     {
+         if (locals != null)
+         {
+             if (runnable instanceof LocalSessionFutureTask)
+                 return (LocalSessionFutureTask<T>) runnable;
+             return new LocalSessionFutureTask<T>(runnable, result, locals);
+         }
+         if (runnable instanceof FutureTask)
+             return (FutureTask<T>) runnable;
+         return new FutureTask<>(runnable, result);
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+     {
+         if (isTracing())
+         {
+             if (callable instanceof LocalSessionFutureTask)
+                 return (LocalSessionFutureTask<T>) callable;
+             return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+         }
+         if (callable instanceof FutureTask)
+             return (FutureTask<T>) callable;
+         return new FutureTask<>(callable);
+     }
+ 
+     private class LocalSessionFutureTask<T> extends FutureTask<T>
+     {
+         private final ExecutorLocals locals;
+ 
+         public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+         {
+             super(callable);
+             this.locals = locals;
+         }
+ 
+         public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+         {
+             super(runnable, result);
+             this.locals = locals;
+         }
+ 
+         public void run()
+         {
+             ExecutorLocals old = ExecutorLocals.create();
+             ExecutorLocals.set(locals);
+             try
+             {
+                 super.run();
+             }
+             finally
+             {
+                 ExecutorLocals.set(old);
+             }
+         }
+     }
+ 
+     class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+     {
+         private boolean failure;
+         private Object result = this;
+         private final Callable<T> callable;
+ 
+         public FutureTask(Callable<T> callable)
+         {
+             this.callable = callable;
+         }
+         public FutureTask(Runnable runnable, T result)
+         {
+             this(Executors.callable(runnable, result));
+         }
+ 
+         public void run()
+         {
+             try
+             {
+                 result = callable.call();
+             }
+             catch (Throwable t)
+             {
+                 JVMStabilityInspector.inspectThrowable(t);
+                 logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                 result = t;
+                 failure = true;
+             }
+             finally
+             {
+                 signalAll();
+                 onCompletion();
+             }
+         }
+ 
+         public boolean cancel(boolean mayInterruptIfRunning)
+         {
+             return false;
+         }
+ 
+         public boolean isCancelled()
+         {
+             return false;
+         }
+ 
+         public boolean isDone()
+         {
+             return isSignaled();
+         }
+ 
+         public T get() throws InterruptedException, ExecutionException
+         {
+             await();
+             Object result = this.result;
+             if (failure)
+                 throw new ExecutionException((Throwable) result);
+             return (T) result;
+         }
+ 
+         public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+         {
 -            await(timeout, unit);
++            if (!await(timeout, unit))
++                throw new TimeoutException();
+             Object result = this.result;
+             if (failure)
+                 throw new ExecutionException((Throwable) result);
+             return (T) result;
+         }
+     }
+ 
+     private <T> FutureTask<T> submit(FutureTask<T> task)
+     {
+         addTask(task);
+         return task;
+     }
+ 
+     public void execute(Runnable command)
+     {
+         addTask(newTaskFor(command, ExecutorLocals.create()));
+     }
+ 
+     public void execute(Runnable command, ExecutorLocals locals)
+     {
+         addTask(newTaskFor(command, null, locals));
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 04a4c3d,1e5cea6..fa0d306
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@@ -269,164 -143,11 +269,164 @@@ public abstract class UDFunction extend
              return null;
  
          long tStart = System.nanoTime();
 -        ByteBuffer result = executeUserDefined(protocolVersion, parameters);
 -        Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 -        return result;
 +        parameters = makeEmptyParametersNull(parameters);
 +
 +        try
 +        {
 +            // Using async UDF execution is expensive (adds about 100us overhead per invocation on a Core-i7 MBPr).
 +            ByteBuffer result = DatabaseDescriptor.enableUserDefinedFunctionsThreads()
 +                                ? executeAsync(protocolVersion, parameters)
 +                                : executeUserDefined(protocolVersion, parameters);
 +
 +            Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 +            return result;
 +        }
 +        catch (InvalidRequestException e)
 +        {
 +            throw e;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.trace("Invocation of user-defined function '{}' failed", this, t);
 +            if (t instanceof VirtualMachineError)
 +                throw (VirtualMachineError) t;
 +            throw FunctionExecutionException.create(this, t);
 +        }
      }
  
 +    public static void assertUdfsEnabled(String language)
 +    {
 +        if (!DatabaseDescriptor.enableUserDefinedFunctions())
 +            throw new InvalidRequestException("User-defined functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable");
 +        if (!"java".equalsIgnoreCase(language) && !DatabaseDescriptor.enableScriptedUserDefinedFunctions())
 +            throw new InvalidRequestException("Scripted user-defined functions are disabled in cassandra.yaml - set enable_scripted_user_defined_functions=true to enable if you are aware of the security risks");
 +    }
 +
 +    static void initializeThread()
 +    {
 +        // Get the TypeCodec stuff in Java Driver initialized.
 +        // This is to get the classes loaded outside of the restricted sandbox's security context of a UDF.
 +        UDHelper.codecFor(DataType.inet()).format(InetAddress.getLoopbackAddress());
 +        UDHelper.codecFor(DataType.ascii()).format("");
 +    }
 +
 +    private static final class ThreadIdAndCpuTime extends CompletableFuture<Object>
 +    {
 +        long threadId;
 +        long cpuTime;
 +
 +        ThreadIdAndCpuTime()
 +        {
 +            // Looks weird?
 +            // This call "just" links this class to java.lang.management - otherwise UDFs (script UDFs) might fail due to
 +            //      java.security.AccessControlException: access denied: ("java.lang.RuntimePermission" "accessClassInPackage.java.lang.management")
 +            // because class loading would be deferred until setup() is executed - but setup() is called with
 +            // limited privileges.
 +            threadMXBean.getCurrentThreadCpuTime();
 +        }
 +
 +        void setup()
 +        {
 +            this.threadId = Thread.currentThread().getId();
 +            this.cpuTime = threadMXBean.getCurrentThreadCpuTime();
 +            complete(null);
 +        }
 +    }
 +
 +    private ByteBuffer executeAsync(int protocolVersion, List<ByteBuffer> parameters)
 +    {
 +        ThreadIdAndCpuTime threadIdAndCpuTime = new ThreadIdAndCpuTime();
 +
 +        Future<ByteBuffer> future = executor().submit(() -> {
 +            threadIdAndCpuTime.setup();
 +            return executeUserDefined(protocolVersion, parameters);
 +        });
 +
 +        try
 +        {
 +            if (DatabaseDescriptor.getUserDefinedFunctionWarnTimeout() > 0)
 +                try
 +                {
 +                    return future.get(DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +                }
 +                catch (TimeoutException e)
 +                {
 +
 +                    // log and emit a warning that UDF execution took long
 +                    String warn = String.format("User defined function %s ran longer than %dms", this, DatabaseDescriptor.getUserDefinedFunctionWarnTimeout());
 +                    logger.warn(warn);
-                     ClientWarn.warn(warn);
++                    ClientWarn.instance.warn(warn);
 +                }
 +
 +            // retry with difference of warn-timeout to fail-timeout
 +            return future.get(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            Thread.currentThread().interrupt();
 +            throw new RuntimeException(e);
 +        }
 +        catch (ExecutionException e)
 +        {
 +            Throwable c = e.getCause();
 +            if (c instanceof RuntimeException)
 +                throw (RuntimeException) c;
 +            throw new RuntimeException(c);
 +        }
 +        catch (TimeoutException e)
 +        {
 +            // retry a last time with the difference of UDF-fail-timeout to consumed CPU time (just in case execution hit a badly timed GC)
 +            try
 +            {
 +                //The threadIdAndCpuTime shouldn't take a long time to be set so this should return immediately
 +                threadIdAndCpuTime.get(1, TimeUnit.SECONDS);
 +
 +                long cpuTimeMillis = threadMXBean.getThreadCpuTime(threadIdAndCpuTime.threadId) - threadIdAndCpuTime.cpuTime;
 +                cpuTimeMillis /= 1000000L;
 +
 +                return future.get(Math.max(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - cpuTimeMillis, 0L),
 +                                  TimeUnit.MILLISECONDS);
 +            }
 +            catch (InterruptedException e1)
 +            {
 +                Thread.currentThread().interrupt();
 +                throw new RuntimeException(e);
 +            }
 +            catch (ExecutionException e1)
 +            {
 +                Throwable c = e.getCause();
 +                if (c instanceof RuntimeException)
 +                    throw (RuntimeException) c;
 +                throw new RuntimeException(c);
 +            }
 +            catch (TimeoutException e1)
 +            {
 +                TimeoutException cause = new TimeoutException(String.format("User defined function %s ran longer than %dms%s",
 +                                                                            this,
 +                                                                            DatabaseDescriptor.getUserDefinedFunctionFailTimeout(),
 +                                                                            DatabaseDescriptor.getUserFunctionTimeoutPolicy() == Config.UserFunctionTimeoutPolicy.ignore
 +                                                                            ? "" : " - will stop Cassandra VM"));
 +                FunctionExecutionException fe = FunctionExecutionException.create(this, cause);
 +                JVMStabilityInspector.userFunctionTimeout(cause);
 +                throw fe;
 +            }
 +        }
 +    }
 +
 +    private List<ByteBuffer> makeEmptyParametersNull(List<ByteBuffer> parameters)
 +    {
 +        List<ByteBuffer> r = new ArrayList<>(parameters.size());
 +        for (int i = 0; i < parameters.size(); i++)
 +        {
 +            ByteBuffer param = parameters.get(i);
 +            r.add(UDHelper.isNullOrEmpty(argTypes.get(i), param)
 +                  ? null : param);
 +        }
 +        return r;
 +    }
 +
 +    protected abstract ExecutorService executor();
 +
      public boolean isCallableWrtNullable(List<ByteBuffer> parameters)
      {
          if (!calledOnNullInput)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 3979597,a289ad1..47396fb
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@@ -226,35 -179,67 +226,35 @@@ public class BatchStatement implements 
          for (int i = 0; i < statements.size(); i++)
          {
              ModificationStatement statement = statements.get(i);
 +            if (isLogged() && statement.cfm.params.gcGraceSeconds == 0)
 +            {
 +                if (tablesWithZeroGcGs == null)
 +                    tablesWithZeroGcGs = new HashSet<>();
 +                tablesWithZeroGcGs.add(String.format("%s.%s", statement.cfm.ksName, statement.cfm.cfName));
 +            }
              QueryOptions statementOptions = options.forStatement(i);
              long timestamp = attrs.getTimestamp(now, statementOptions);
 -            addStatementMutations(statement, statementOptions, local, timestamp, mutations);
 +            statement.addUpdates(collector, statementOptions, local, timestamp);
          }
 -        return unzipMutations(mutations);
 -    }
 -
 -    private Collection<? extends IMutation> unzipMutations(Map<String, Map<ByteBuffer, IMutation>> mutations)
 -    {
 -
 -        // The case where all statement where on the same keyspace is pretty common
 -        if (mutations.size() == 1)
 -            return mutations.values().iterator().next().values();
 -
  
 -        List<IMutation> ms = new ArrayList<>();
 -        for (Map<ByteBuffer, IMutation> ksMap : mutations.values())
 -            ms.addAll(ksMap.values());
 -
 -        return ms;
 -    }
 -
 -    private void addStatementMutations(ModificationStatement statement,
 -                                       QueryOptions options,
 -                                       boolean local,
 -                                       long now,
 -                                       Map<String, Map<ByteBuffer, IMutation>> mutations)
 -    throws RequestExecutionException, RequestValidationException
 -    {
 -        String ksName = statement.keyspace();
 -        Map<ByteBuffer, IMutation> ksMap = mutations.get(ksName);
 -        if (ksMap == null)
 +        if (tablesWithZeroGcGs != null)
          {
 -            ksMap = new HashMap<>();
 -            mutations.put(ksName, ksMap);
 +            String suffix = tablesWithZeroGcGs.size() == 1 ? "" : "s";
 +            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, LOGGED_BATCH_LOW_GCGS_WARNING,
 +                             suffix, tablesWithZeroGcGs);
-             ClientWarn.warn(MessageFormatter.arrayFormat(LOGGED_BATCH_LOW_GCGS_WARNING, new Object[] { suffix, tablesWithZeroGcGs })
-                                             .getMessage());
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(LOGGED_BATCH_LOW_GCGS_WARNING, new Object[] { suffix, tablesWithZeroGcGs })
++                                                     .getMessage());
          }
  
 -        // The following does the same than statement.getMutations(), but we inline it here because
 -        // we don't want to recreate mutations every time as this is particularly inefficient when applying
 -        // multiple batch to the same partition (see #6737).
 -        List<ByteBuffer> keys = statement.buildPartitionKeyNames(options);
 -        Composite clusteringPrefix = statement.createClusteringPrefix(options);
 -        UpdateParameters params = statement.makeUpdateParameters(keys, clusteringPrefix, options, local, now);
 -
 -        for (ByteBuffer key : keys)
 -        {
 -            IMutation mutation = ksMap.get(key);
 -            Mutation mut;
 -            if (mutation == null)
 -            {
 -                mut = new Mutation(ksName, key);
 -                mutation = statement.cfm.isCounter() ? new CounterMutation(mut, options.getConsistency()) : mut;
 -                ksMap.put(key, mutation);
 -            }
 -            else
 -            {
 -                mut = statement.cfm.isCounter() ? ((CounterMutation) mutation).getMutation() : (Mutation) mutation;
 -            }
 +        collector.validateIndexedColumns();
 +        return collector.toMutations();
 +    }
  
 -            statement.addUpdateForKey(mut.addOrGet(statement.cfm), key, clusteringPrefix, params);
 -        }
 +    private int updatedRows()
 +    {
 +        // Note: it's possible for 2 statements to actually apply to the same row, but that's just an estimation
 +        // for sizing our PartitionUpdate backing array, so it's good enough.
 +        return statements.size();
      }
  
      /**
@@@ -286,9 -271,9 +286,9 @@@
              }
              else if (logger.isWarnEnabled())
              {
 -                logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
 +                logger.warn(format, tableNames, size, warnThreshold, size - warnThreshold, "");
              }
-             ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage());
 -            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage());
          }
      }
  
@@@ -311,17 -298,21 +311,16 @@@
              }
  
              // CASSANDRA-9303: If we only have local mutations we do not warn
 -            if (localMutationsOnly)
 +            if (localPartitionsOnly)
                  return;
  
 -            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, unloggedBatchWarning,
 -                             keySet.size(), keySet.size() == 1 ? "" : "s",
 -                             ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
  
 -            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
 -                                                                  new Object[]{
 -                                                                              keySet.size(),
 -                                                                              keySet.size() == 1 ? "" : "s",
 -                                                                              ksCfPairs.size() == 1 ? "" : "s",
 -                                                                              ksCfPairs
 -                                                                  }).getMessage());
 +            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, UNLOGGED_BATCH_WARNING,
 +                             keySet.size(), keySet.size() == 1 ? "" : "s",
 +                             tableNames.size() == 1 ? "" : "s", tableNames);
  
-             ClientWarn.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
 +                                                    tableNames.size() == 1 ? "" : "s", tableNames}).getMessage());
- 
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 4017ce6,0000000..5af4887
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@@ -1,330 -1,0 +1,330 @@@
 +/*
 + * 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.cql3.statements;
 +
 +import java.util.*;
 +import java.util.stream.Collectors;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.auth.Permission;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.ViewDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 +import org.apache.cassandra.cql3.selection.RawSelector;
 +import org.apache.cassandra.cql3.selection.Selectable;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.ReversedType;
 +import org.apache.cassandra.db.view.View;
 +import org.apache.cassandra.exceptions.AlreadyExistsException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.exceptions.RequestValidationException;
 +import org.apache.cassandra.exceptions.UnauthorizedException;
 +import org.apache.cassandra.schema.TableParams;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.ClientWarn;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.thrift.ThriftValidation;
 +import org.apache.cassandra.transport.Event;
 +
 +public class CreateViewStatement extends SchemaAlteringStatement
 +{
 +    private final CFName baseName;
 +    private final List<RawSelector> selectClause;
 +    private final WhereClause whereClause;
 +    private final List<ColumnIdentifier.Raw> partitionKeys;
 +    private final List<ColumnIdentifier.Raw> clusteringKeys;
 +    public final CFProperties properties = new CFProperties();
 +    private final boolean ifNotExists;
 +
 +    public CreateViewStatement(CFName viewName,
 +                               CFName baseName,
 +                               List<RawSelector> selectClause,
 +                               WhereClause whereClause,
 +                               List<ColumnIdentifier.Raw> partitionKeys,
 +                               List<ColumnIdentifier.Raw> clusteringKeys,
 +                               boolean ifNotExists)
 +    {
 +        super(viewName);
 +        this.baseName = baseName;
 +        this.selectClause = selectClause;
 +        this.whereClause = whereClause;
 +        this.partitionKeys = partitionKeys;
 +        this.clusteringKeys = clusteringKeys;
 +        this.ifNotExists = ifNotExists;
 +    }
 +
 +
 +    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
 +    {
 +        if (!baseName.hasKeyspace())
 +            baseName.setKeyspace(keyspace(), true);
 +        state.hasColumnFamilyAccess(keyspace(), baseName.getColumnFamily(), Permission.ALTER);
 +    }
 +
 +    public void validate(ClientState state) throws RequestValidationException
 +    {
 +        // We do validation in announceMigration to reduce doubling up of work
 +    }
 +
 +    private interface AddColumn {
 +        void add(ColumnIdentifier identifier, AbstractType<?> type);
 +    }
 +
 +    private void add(CFMetaData baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
 +    {
 +        for (ColumnIdentifier column : columns)
 +        {
 +            AbstractType<?> type = baseCfm.getColumnDefinition(column).type;
 +            if (properties.definedOrdering.containsKey(column))
 +            {
 +                boolean desc = properties.definedOrdering.get(column);
 +                if (!desc && type.isReversed())
 +                {
 +                    type = ((ReversedType)type).baseType;
 +                }
 +                else if (desc && !type.isReversed())
 +                {
 +                    type = ReversedType.getInstance(type);
 +                }
 +            }
 +            adder.add(column, type);
 +        }
 +    }
 +
 +    public Event.SchemaChange announceMigration(boolean isLocalOnly) throws RequestValidationException
 +    {
 +        // We need to make sure that:
 +        //  - primary key includes all columns in base table's primary key
 +        //  - make sure that the select statement does not have anything other than columns
 +        //    and their names match the base table's names
 +        //  - make sure that primary key does not include any collections
 +        //  - make sure there is no where clause in the select statement
 +        //  - make sure there is not currently a table or view
 +        //  - make sure baseTable gcGraceSeconds > 0
 +
 +        properties.validate();
 +
 +        if (properties.useCompactStorage)
 +            throw new InvalidRequestException("Cannot use 'COMPACT STORAGE' when defining a materialized view");
 +
 +        // We enforce the keyspace because if the RF is different, the logic to wait for a
 +        // specific replica would break
 +        if (!baseName.getKeyspace().equals(keyspace()))
 +            throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
 +
 +        CFMetaData cfm = ThriftValidation.validateColumnFamily(baseName.getKeyspace(), baseName.getColumnFamily());
 +
 +        if (cfm.isCounter())
 +            throw new InvalidRequestException("Materialized views are not supported on counter tables");
 +        if (cfm.isView())
 +            throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
 +
 +        if (cfm.params.gcGraceSeconds == 0)
 +        {
 +            throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
 +                                                            "'%s' with gc_grace_seconds of 0, since this value is " +
 +                                                            "used to TTL undelivered updates. Setting gc_grace_seconds" +
 +                                                            " too low might cause undelivered updates to expire " +
 +                                                            "before being replayed.", cfName.getColumnFamily(),
 +                                                            baseName.getColumnFamily()));
 +        }
 +
 +        Set<ColumnIdentifier> included = new HashSet<>();
 +        for (RawSelector selector : selectClause)
 +        {
 +            Selectable.Raw selectable = selector.selectable;
 +            if (selectable instanceof Selectable.WithFieldSelection.Raw)
 +                throw new InvalidRequestException("Cannot select out a part of type when defining a materialized view");
 +            if (selectable instanceof Selectable.WithFunction.Raw)
 +                throw new InvalidRequestException("Cannot use function when defining a materialized view");
 +            if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
 +                throw new InvalidRequestException("Cannot use function when defining a materialized view");
 +            ColumnIdentifier identifier = (ColumnIdentifier) selectable.prepare(cfm);
 +            if (selector.alias != null)
 +                throw new InvalidRequestException(String.format("Cannot alias column '%s' as '%s' when defining a materialized view", identifier.toString(), selector.alias.toString()));
 +
 +            ColumnDefinition cdef = cfm.getColumnDefinition(identifier);
 +
 +            if (cdef == null)
 +                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
 +
 +            if (cdef.isStatic())
-                 ClientWarn.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
++                ClientWarn.instance.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
 +            else
 +                included.add(identifier);
 +        }
 +
 +        Set<ColumnIdentifier.Raw> targetPrimaryKeys = new HashSet<>();
 +        for (ColumnIdentifier.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
 +        {
 +            if (!targetPrimaryKeys.add(identifier))
 +                throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
 +
 +            ColumnDefinition cdef = cfm.getColumnDefinition(identifier.prepare(cfm));
 +
 +            if (cdef == null)
 +                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
 +
 +            if (cfm.getColumnDefinition(identifier.prepare(cfm)).type.isMultiCell())
 +                throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
 +
 +            if (cdef.isStatic())
 +                throw new InvalidRequestException(String.format("Cannot use Static column '%s' in PRIMARY KEY of materialized view", identifier));
 +        }
 +
 +        // build the select statement
 +        Map<ColumnIdentifier.Raw, Boolean> orderings = Collections.emptyMap();
 +        SelectStatement.Parameters parameters = new SelectStatement.Parameters(orderings, false, true, false);
 +        SelectStatement.RawStatement rawSelect = new SelectStatement.RawStatement(baseName, parameters, selectClause, whereClause, null);
 +
 +        ClientState state = ClientState.forInternalCalls();
 +        state.setKeyspace(keyspace());
 +
 +        rawSelect.prepareKeyspace(state);
 +        rawSelect.setBoundVariables(getBoundVariables());
 +
 +        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
 +        SelectStatement select = (SelectStatement) prepared.statement;
 +        StatementRestrictions restrictions = select.getRestrictions();
 +
 +        if (!prepared.boundNames.isEmpty())
 +            throw new InvalidRequestException("Cannot use query parameters in CREATE MATERIALIZED VIEW statements");
 +
 +        if (!restrictions.nonPKRestrictedColumns(false).isEmpty())
 +        {
 +            throw new InvalidRequestException(String.format(
 +                    "Non-primary key columns cannot be restricted in the SELECT statement used for materialized view " +
 +                    "creation (got restrictions on: %s)",
 +                    restrictions.nonPKRestrictedColumns(false).stream().map(def -> def.name.toString()).collect(Collectors.joining(", "))));
 +        }
 +
 +        String whereClauseText = View.relationsToWhereClause(whereClause.relations);
 +
 +        Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
 +        for (ColumnDefinition definition : Iterables.concat(cfm.partitionKeyColumns(), cfm.clusteringColumns()))
 +            basePrimaryKeyCols.add(definition.name);
 +
 +        List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
 +        List<ColumnIdentifier> targetPartitionKeys = new ArrayList<>();
 +
 +        // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
 +        boolean hasNonPKColumn = false;
 +        for (ColumnIdentifier.Raw raw : partitionKeys)
 +            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 +
 +        for (ColumnIdentifier.Raw raw : clusteringKeys)
 +            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 +
 +        // We need to include all of the primary key columns from the base table in order to make sure that we do not
 +        // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
 +        // the view because if we need to generate a tombstone, we have no way of knowing which value is currently being
 +        // used in the view and whether or not to generate a tombstone. In order to not surprise our users, we require
 +        // that they include all of the columns. We provide them with a list of all of the columns left to include.
 +        boolean missingClusteringColumns = false;
 +        StringBuilder columnNames = new StringBuilder();
 +        List<ColumnIdentifier> includedColumns = new ArrayList<>();
 +        for (ColumnDefinition def : cfm.allColumns())
 +        {
 +            ColumnIdentifier identifier = def.name;
 +
 +            if ((included.isEmpty() || included.contains(identifier))
 +                && !targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier)
 +                && !def.isStatic())
 +            {
 +                includedColumns.add(identifier);
 +            }
 +            if (!def.isPrimaryKeyColumn()) continue;
 +
 +            if (!targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier))
 +            {
 +                if (missingClusteringColumns)
 +                    columnNames.append(',');
 +                else
 +                    missingClusteringColumns = true;
 +                columnNames.append(identifier);
 +            }
 +        }
 +        if (missingClusteringColumns)
 +            throw new InvalidRequestException(String.format("Cannot create Materialized View %s without primary key columns from base %s (%s)",
 +                                                            columnFamily(), baseName.getColumnFamily(), columnNames.toString()));
 +
 +        if (targetPartitionKeys.isEmpty())
 +            throw new InvalidRequestException("Must select at least a column for a Materialized View");
 +
 +        if (targetClusteringColumns.isEmpty())
 +            throw new InvalidRequestException("No columns are defined for Materialized View other than primary key");
 +
 +        CFMetaData.Builder cfmBuilder = CFMetaData.Builder.createView(keyspace(), columnFamily());
 +        add(cfm, targetPartitionKeys, cfmBuilder::addPartitionKey);
 +        add(cfm, targetClusteringColumns, cfmBuilder::addClusteringColumn);
 +        add(cfm, includedColumns, cfmBuilder::addRegularColumn);
 +        cfmBuilder.withId(properties.properties.getId());
 +        TableParams params = properties.properties.asNewTableParams();
 +        CFMetaData viewCfm = cfmBuilder.build().params(params);
 +        ViewDefinition definition = new ViewDefinition(keyspace(),
 +                                                       columnFamily(),
 +                                                       Schema.instance.getId(keyspace(), baseName.getColumnFamily()),
 +                                                       baseName.getColumnFamily(),
 +                                                       included.isEmpty(),
 +                                                       rawSelect,
 +                                                       whereClauseText,
 +                                                       viewCfm);
 +
 +        try
 +        {
 +            MigrationManager.announceNewView(definition, isLocalOnly);
 +            return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
 +        }
 +        catch (AlreadyExistsException e)
 +        {
 +            if (ifNotExists)
 +                return null;
 +            throw e;
 +        }
 +    }
 +
 +    private static boolean getColumnIdentifier(CFMetaData cfm,
 +                                               Set<ColumnIdentifier> basePK,
 +                                               boolean hasNonPKColumn,
 +                                               ColumnIdentifier.Raw raw,
 +                                               List<ColumnIdentifier> columns,
 +                                               StatementRestrictions restrictions)
 +    {
 +        ColumnIdentifier identifier = raw.prepare(cfm);
 +        ColumnDefinition def = cfm.getColumnDefinition(identifier);
 +
 +        boolean isPk = basePK.contains(identifier);
 +        if (!isPk && hasNonPKColumn)
 +            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", identifier));
 +
 +        // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
 +        // because we will never allow a single partition key to be NULL
 +        boolean isSinglePartitionKey = cfm.getColumnDefinition(identifier).isPartitionKey()
 +                                       && cfm.partitionKeyColumns().size() == 1;
 +        if (!isSinglePartitionKey && !restrictions.isRestricted(def))
 +            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", identifier));
 +
 +        columns.add(identifier);
 +        return !isPk;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index a9bb121,5cfa94b..904adca
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -359,21 -256,19 +359,21 @@@ public class SelectStatement implement
          else if (restrictions.keyIsInRelation())
          {
              logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-             ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+             ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
          }
  
 -        Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);
 +        Selection.ResultSetBuilder result = selection.resultSetBuilder(parameters.isJson);
          while (!pager.isExhausted())
          {
 -            for (Row row : pager.fetchPage(pageSize))
 +            try (PartitionIterator iter = pager.fetchPage(pageSize))
              {
 -                // Not columns match the query, skip
 -                if (row.cf == null)
 -                    continue;
 -
 -                processColumnFamily(row.key.getKey(), row.cf, options, now, result);
 +                while (iter.hasNext())
 +                {
 +                    try (RowIterator partition = iter.next())
 +                    {
 +                        processPartition(partition, options, result, nowInSec);
 +                    }
 +                }
              }
          }
          return new ResultMessage.Rows(result.build(options.getProtocolVersion()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index 3f0695c,cd86336..668a189
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -232,713 -95,55 +232,713 @@@ public abstract class ReadCommand imple
          return this;
      }
  
 -    public String getColumnFamilyName()
 +    /**
 +     * Sets the digest version, for when digest for that command is requested.
 +     * <p>
 +     * Note that we allow setting this independently of setting the command as a digest query as
 +     * this allows us to use the command as a carrier of the digest version even if we only call
 +     * setIsDigestQuery on some copy of it.
 +     *
 +     * @param digestVersion the version for the digest is this command is used for digest query..
 +     * @return this read command.
 +     */
 +    public ReadCommand setDigestVersion(int digestVersion)
      {
 -        return cfName;
 +        this.digestVersion = digestVersion;
 +        return this;
      }
  
 +    /**
 +     * Whether this query is for thrift or not.
 +     *
 +     * @return whether this query is for thrift.
 +     */
 +    public boolean isForThrift()
 +    {
 +        return isForThrift;
 +    }
 +
 +    /**
 +     * The clustering index filter this command to use for the provided key.
 +     * <p>
 +     * Note that that method should only be called on a key actually queried by this command
 +     * and in practice, this will almost always return the same filter, but for the sake of
 +     * paging, the filter on the first key of a range command might be slightly different.
 +     *
 +     * @param key a partition key queried by this command.
 +     *
 +     * @return the {@code ClusteringIndexFilter} to use for the partition of key {@code key}.
 +     */
 +    public abstract ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key);
 +
 +    /**
 +     * Returns a copy of this command.
 +     *
 +     * @return a copy of this command.
 +     */
      public abstract ReadCommand copy();
  
 -    public abstract Row getRow(Keyspace keyspace);
 +    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 +
 +    protected abstract int oldestUnrepairedTombstone();
 +
 +    public ReadResponse createResponse(UnfilteredPartitionIterator iterator, ColumnFilter selection)
 +    {
 +        return isDigestQuery()
 +             ? ReadResponse.createDigestResponse(iterator, digestVersion)
 +             : ReadResponse.createDataResponse(iterator, selection);
 +    }
 +
 +    public long indexSerializedSize(int version)
 +    {
 +        if (index.isPresent())
 +            return IndexMetadata.serializer.serializedSize(index.get(), version);
 +        else
 +            return 0;
 +    }
 +
 +    public Index getIndex(ColumnFamilyStore cfs)
 +    {
 +        // if we've already consulted the index manager, and it returned a valid index
 +        // the result should be cached here.
 +        if(index.isPresent())
 +            return cfs.indexManager.getIndex(index.get());
 +
 +        // if no cached index is present, but we've already consulted the index manager
 +        // then no registered index is suitable for this command, so just return null.
 +        if (indexManagerQueried)
 +            return null;
 +
 +        // do the lookup, set the flag to indicate so and cache the result if not null
 +        Index selected = cfs.indexManager.getBestIndexFor(this);
 +        indexManagerQueried = true;
  
 -    public abstract IDiskAtomFilter filter();
 +        if (selected == null)
 +            return null;
  
 -    public String getKeyspace()
 +        index = Optional.of(selected.getIndexMetadata());
 +        return selected;
 +    }
 +
 +    /**
 +     * Executes this command on the local host.
 +     *
 +     * @param orderGroup the operation group spanning this command
 +     *
 +     * @return an iterator over the result of executing this command locally.
 +     */
 +    @SuppressWarnings("resource") // The result iterator is closed upon exceptions (we know it's fine to potentially not close the intermediary
 +                                  // iterators created inside the try as long as we do close the original resultIterator), or by closing the result.
 +    public UnfilteredPartitionIterator executeLocally(ReadOrderGroup orderGroup)
      {
 -        return ksName;
 +        long startTimeNanos = System.nanoTime();
 +
 +        ColumnFamilyStore cfs = Keyspace.openAndGetStore(metadata());
 +        Index index = getIndex(cfs);
 +
 +        Index.Searcher searcher = null;
 +        if (index != null)
 +        {
 +            if (!cfs.indexManager.isIndexQueryable(index))
 +                throw new IndexNotAvailableException(index);
 +
 +            searcher = index.searcherFor(this);
 +            Tracing.trace("Executing read on {}.{} using index {}", cfs.metadata.ksName, cfs.metadata.cfName, index.getIndexMetadata().name);
 +        }
 +
 +        UnfilteredPartitionIterator resultIterator = searcher == null
 +                                         ? queryStorage(cfs, orderGroup)
 +                                         : searcher.search(orderGroup);
 +
 +        try
 +        {
 +            resultIterator = withMetricsRecording(withoutPurgeableTombstones(resultIterator, cfs), cfs.metric, startTimeNanos);
 +
 +            // If we've used a 2ndary index, we know the result already satisfy the primary expression used, so
 +            // no point in checking it again.
 +            RowFilter updatedFilter = searcher == null
 +                                    ? rowFilter()
 +                                    : index.getPostIndexQueryFilter(rowFilter());
 +
 +            // TODO: We'll currently do filtering by the rowFilter here because it's convenient. However,
 +            // we'll probably want to optimize by pushing it down the layer (like for dropped columns) as it
 +            // would be more efficient (the sooner we discard stuff we know we don't care, the less useless
 +            // processing we do on it).
 +            return limits().filter(updatedFilter.filter(resultIterator, nowInSec()), nowInSec());
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            resultIterator.close();
 +            throw e;
 +        }
      }
  
 -    // maybeGenerateRetryCommand is used to generate a retry for short reads
 -    public ReadCommand maybeGenerateRetryCommand(RowDataResolver resolver, Row row)
 +    protected abstract void recordLatency(TableMetrics metric, long latencyNanos);
 +
 +    public PartitionIterator executeInternal(ReadOrderGroup orderGroup)
      {
 -        return null;
 +        return UnfilteredPartitionIterators.filter(executeLocally(orderGroup), nowInSec());
      }
  
 -    // maybeTrim removes columns from a response that is too long
 -    public Row maybeTrim(Row row)
 +    public ReadOrderGroup startOrderGroup()
      {
 -        return row;
 +        return ReadOrderGroup.forCommand(this);
      }
  
 -    public long getTimeout()
 +    /**
 +     * Wraps the provided iterator so that metrics on what is scanned by the command are recorded.
 +     * This also log warning/trow TombstoneOverwhelmingException if appropriate.
 +     */
 +    private UnfilteredPartitionIterator withMetricsRecording(UnfilteredPartitionIterator iter, final TableMetrics metric, final long startTimeNanos)
      {
 -        return DatabaseDescriptor.getReadRpcTimeout();
 +        class MetricRecording extends Transformation<UnfilteredRowIterator>
 +        {
 +            private final int failureThreshold = DatabaseDescriptor.getTombstoneFailureThreshold();
 +            private final int warningThreshold = DatabaseDescriptor.getTombstoneWarnThreshold();
 +
 +            private final boolean respectTombstoneThresholds = !Schema.isSystemKeyspace(ReadCommand.this.metadata().ksName);
 +
 +            private int liveRows = 0;
 +            private int tombstones = 0;
 +
 +            private DecoratedKey currentKey;
 +
 +            @Override
 +            public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator iter)
 +            {
 +                currentKey = iter.partitionKey();
 +                return Transformation.apply(iter, this);
 +            }
 +
 +            @Override
 +            public Row applyToStatic(Row row)
 +            {
 +                return applyToRow(row);
 +            }
 +
 +            @Override
 +            public Row applyToRow(Row row)
 +            {
 +                if (row.hasLiveData(ReadCommand.this.nowInSec()))
 +                    ++liveRows;
 +
 +                for (Cell cell : row.cells())
 +                {
 +                    if (!cell.isLive(ReadCommand.this.nowInSec()))
 +                        countTombstone(row.clustering());
 +                }
 +                return row;
 +            }
 +
 +            @Override
 +            public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +            {
 +                countTombstone(marker.clustering());
 +                return marker;
 +            }
 +
 +            private void countTombstone(ClusteringPrefix clustering)
 +            {
 +                ++tombstones;
 +                if (tombstones > failureThreshold && respectTombstoneThresholds)
 +                {
 +                    String query = ReadCommand.this.toCQLString();
 +                    Tracing.trace("Scanned over {} tombstones for query {}; query aborted (see tombstone_failure_threshold)", failureThreshold, query);
 +                    throw new TombstoneOverwhelmingException(tombstones, query, ReadCommand.this.metadata(), currentKey, clustering);
 +                }
 +            }
 +
 +            @Override
 +            public void onClose()
 +            {
 +                recordLatency(metric, System.nanoTime() - startTimeNanos);
 +
 +                metric.tombstoneScannedHistogram.update(tombstones);
 +                metric.liveScannedHistogram.update(liveRows);
 +
 +                boolean warnTombstones = tombstones > warningThreshold && respectTombstoneThresholds;
 +                if (warnTombstones)
 +                {
 +                    String msg = String.format("Read %d live rows and %d tombstone cells for query %1.512s (see tombstone_warn_threshold)", liveRows, tombstones, ReadCommand.this.toCQLString());
-                     ClientWarn.warn(msg);
++                    ClientWarn.instance.warn(msg);
 +                    logger.warn(msg);
 +                }
 +
 +                Tracing.trace("Read {} live and {} tombstone cells{}", liveRows, tombstones, (warnTombstones ? " (see tombstone_warn_threshold)" : ""));
 +            }
 +        };
 +
 +        return Transformation.apply(iter, new MetricRecording());
      }
 -}
  
 -class ReadCommandSerializer implements IVersionedSerializer<ReadCommand>
 -{
 -    public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +    /**
 +     * Creates a message for this command.
 +     */
 +    public abstract MessageOut<ReadCommand> createMessage(int version);
 +
 +    protected abstract void appendCQLWhereClause(StringBuilder sb);
 +
 +    // Skip purgeable tombstones. We do this because it's safe to do (post-merge of the memtable and sstable at least), it
 +    // can save us some bandwith, and avoid making us throw a TombstoneOverwhelmingException for purgeable tombstones (which
 +    // are to some extend an artefact of compaction lagging behind and hence counting them is somewhat unintuitive).
 +    protected UnfilteredPartitionIterator withoutPurgeableTombstones(UnfilteredPartitionIterator iterator, ColumnFamilyStore cfs)
 +    {
 +        final boolean isForThrift = iterator.isForThrift();
 +        class WithoutPurgeableTombstones extends PurgeFunction
 +        {
 +            public WithoutPurgeableTombstones()
 +            {
 +                super(isForThrift, nowInSec(), cfs.gcBefore(nowInSec()), oldestUnrepairedTombstone(), cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
 +            }
 +
 +            protected long getMaxPurgeableTimestamp()
 +            {
 +                return Long.MAX_VALUE;
 +            }
 +        }
 +        return Transformation.apply(iterator, new WithoutPurgeableTombstones());
 +    }
 +
 +    /**
 +     * Recreate the CQL string corresponding to this query.
 +     * <p>
 +     * Note that in general the returned string will not be exactly the original user string, first
 +     * because there isn't always a single syntax for a given query,  but also because we don't have
 +     * all the information needed (we know the non-PK columns queried but not the PK ones as internally
 +     * we query them all). So this shouldn't be relied too strongly, but this should be good enough for
 +     * debugging purpose which is what this is for.
 +     */
 +    public String toCQLString()
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        sb.append("SELECT ").append(columnFilter());
 +        sb.append(" FROM ").append(metadata().ksName).append('.').append(metadata.cfName);
 +        appendCQLWhereClause(sb);
 +
 +        if (limits() != DataLimits.NONE)
 +            sb.append(' ').append(limits());
 +        return sb.toString();
 +    }
 +
 +    private static class Serializer implements IVersionedSerializer<ReadCommand>
 +    {
 +        private static int digestFlag(boolean isDigest)
 +        {
 +            return isDigest ? 0x01 : 0;
 +        }
 +
 +        private static boolean isDigest(int flags)
 +        {
 +            return (flags & 0x01) != 0;
 +        }
 +
 +        private static int thriftFlag(boolean isForThrift)
 +        {
 +            return isForThrift ? 0x02 : 0;
 +        }
 +
 +        private static boolean isForThrift(int flags)
 +        {
 +            return (flags & 0x02) != 0;
 +        }
 +
 +        private static int indexFlag(boolean hasIndex)
 +        {
 +            return hasIndex ? 0x04 : 0;
 +        }
 +
 +        private static boolean hasIndex(int flags)
 +        {
 +            return (flags & 0x04) != 0;
 +        }
 +
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
 +            assert version >= MessagingService.VERSION_30;
 +
 +            out.writeByte(command.kind.ordinal());
 +            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(command.index.isPresent()));
 +            if (command.isDigestQuery())
 +                out.writeUnsignedVInt(command.digestVersion());
 +            CFMetaData.serializer.serialize(command.metadata(), out, version);
 +            out.writeInt(command.nowInSec());
 +            ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
 +            RowFilter.serializer.serialize(command.rowFilter(), out, version);
 +            DataLimits.serializer.serialize(command.limits(), out, version);
 +            if (command.index.isPresent())
 +                IndexMetadata.serializer.serialize(command.index.get(), out, version);
 +
 +            command.serializeSelection(out, version);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            if (version < MessagingService.VERSION_30)
 +                return legacyReadCommandSerializer.deserialize(in, version);
 +
 +            Kind kind = Kind.values()[in.readByte()];
 +            int flags = in.readByte();
 +            boolean isDigest = isDigest(flags);
 +            boolean isForThrift = isForThrift(flags);
 +            boolean hasIndex = hasIndex(flags);
 +            int digestVersion = isDigest ? (int)in.readUnsignedVInt() : 0;
 +            CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
 +            int nowInSec = in.readInt();
 +            ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
 +            RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
 +            DataLimits limits = DataLimits.serializer.deserialize(in, version);
 +            Optional<IndexMetadata> index = hasIndex
 +                                            ? deserializeIndexMetadata(in, version, metadata)
 +                                            : Optional.empty();
 +
 +            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
 +        }
 +
 +        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
 +        {
 +            try
 +            {
 +                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
 +            }
 +            catch (UnknownIndexException e)
 +            {
 +                String message = String.format("Couldn't find a defined index on %s.%s with the id %s. " +
 +                                               "If an index was just created, this is likely due to the schema not " +
 +                                               "being fully propagated. Local read will proceed without using the " +
 +                                               "index. Please wait for schema agreement after index creation.",
 +                                               cfm.ksName, cfm.cfName, e.indexId.toString());
 +                logger.info(message);
 +                return Optional.empty();
 +            }
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
 +            assert version >= MessagingService.VERSION_30;
 +
 +            return 2 // kind + flags
 +                 + (command.isDigestQuery() ? TypeSizes.sizeofUnsignedVInt(command.digestVersion()) : 0)
 +                 + CFMetaData.serializer.serializedSize(command.metadata(), version)
 +                 + TypeSizes.sizeof(command.nowInSec())
 +                 + ColumnFilter.serializer.serializedSize(command.columnFilter(), version)
 +                 + RowFilter.serializer.serializedSize(command.rowFilter(), version)
 +                 + DataLimits.serializer.serializedSize(command.limits(), version)
 +                 + command.selectionSerializedSize(version)
 +                 + command.indexSerializedSize(version);
 +        }
 +    }
 +
 +    // Dispatch to either Serializer or LegacyRangeSliceCommandSerializer. Only useful as long as we maintain pre-3.0
 +    // compatibility
 +    private static class RangeSliceSerializer implements IVersionedSerializer<ReadCommand>
 +    {
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            if (version < MessagingService.VERSION_30)
 +                legacyRangeSliceCommandSerializer.serialize(command, out, version);
 +            else
 +                serializer.serialize(command, out, version);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            return version < MessagingService.VERSION_30
 +                 ? legacyRangeSliceCommandSerializer.deserialize(in, version)
 +                 : serializer.deserialize(in, version);
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            return version < MessagingService.VERSION_30
 +                 ? legacyRangeSliceCommandSerializer.serializedSize(command, version)
 +                 : serializer.serializedSize(command, version);
 +        }
 +    }
 +
 +    private enum LegacyType
 +    {
 +        GET_BY_NAMES((byte)1),
 +        GET_SLICES((byte)2);
 +
 +        public final byte serializedValue;
 +
 +        LegacyType(byte b)
 +        {
 +            this.serializedValue = b;
 +        }
 +
 +        public static LegacyType fromPartitionFilterKind(ClusteringIndexFilter.Kind kind)
 +        {
 +            return kind == ClusteringIndexFilter.Kind.SLICE
 +                   ? GET_SLICES
 +                   : GET_BY_NAMES;
 +        }
 +
 +        public static LegacyType fromSerializedValue(byte b)
 +        {
 +            return b == 1 ? GET_BY_NAMES : GET_SLICES;
 +        }
 +    }
 +
 +    /**
 +     * Serializer for pre-3.0 RangeSliceCommands.
 +     */
 +    private static class LegacyRangeSliceCommandSerializer implements IVersionedSerializer<ReadCommand>
      {
 -        out.writeByte(command.commandType.serializedValue);
 -        switch (command.commandType)
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            assert version < MessagingService.VERSION_30;
 +
 +            PartitionRangeReadCommand rangeCommand = (PartitionRangeReadCommand) command;
 +            assert !rangeCommand.dataRange().isPaging();
 +
 +            // convert pre-3.0 incompatible names filters to slice filters
 +            rangeCommand = maybeConvertNamesToSlice(rangeCommand);
 +
 +            CFMetaData metadata = rangeCommand.metadata();
 +
 +            out.writeUTF(metadata.ksName);
 +            out.writeUTF(metadata.cfName);
 +            out.writeLong(rangeCommand.nowInSec() * 1000L);  // convert from seconds to millis
 +
 +            // begin DiskAtomFilterSerializer.serialize()
 +            if (rangeCommand.isNamesQuery())
 +            {
 +                out.writeByte(1);  // 0 for slices, 1 for names
 +                ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                LegacyReadCommandSerializer.serializeNamesFilter(rangeCommand, filter, out);
 +            }
 +            else
 +            {
 +                out.writeByte(0);  // 0 for slices, 1 for names
 +
 +                // slice filter serialization
 +                ClusteringIndexSliceFilter filter = (ClusteringIndexSliceFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +
 +                boolean makeStaticSlice = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() && !filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                LegacyReadCommandSerializer.serializeSlices(out, filter.requestedSlices(), filter.isReversed(), makeStaticSlice, metadata);
 +
 +                out.writeBoolean(filter.isReversed());
 +
 +                // limit
 +                DataLimits.Kind kind = rangeCommand.limits().kind();
 +                boolean isDistinct = (kind == DataLimits.Kind.CQL_LIMIT || kind == DataLimits.Kind.CQL_PAGING_LIMIT) && rangeCommand.limits().perPartitionCount() == 1;
 +                if (isDistinct)
 +                    out.writeInt(1);
 +                else
 +                    out.writeInt(LegacyReadCommandSerializer.updateLimitForQuery(rangeCommand.limits().count(), filter.requestedSlices()));
 +
 +                int compositesToGroup;
 +                boolean selectsStatics = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() || filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                if (kind == DataLimits.Kind.THRIFT_LIMIT)
 +                    compositesToGroup = -1;
 +                else if (isDistinct && !selectsStatics)
 +                    compositesToGroup = -2;  // for DISTINCT queries (CASSANDRA-8490)
 +                else
 +                    compositesToGroup = metadata.isDense() ? -1 : metadata.clusteringColumns().size();
 +
 +                out.writeInt(compositesToGroup);
 +            }
 +
 +            serializeRowFilter(out, rangeCommand.rowFilter());
 +            AbstractBounds.rowPositionSerializer.serialize(rangeCommand.dataRange().keyRange(), out, version);
 +
 +            // maxResults
 +            out.writeInt(rangeCommand.limits().count());
 +
 +            // countCQL3Rows
 +            if (rangeCommand.isForThrift() || rangeCommand.limits().perPartitionCount() == 1)  // if for Thrift or DISTINCT
 +                out.writeBoolean(false);
 +            else
 +                out.writeBoolean(true);
 +
 +            // isPaging
 +            out.writeBoolean(false);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            assert version < MessagingService.VERSION_30;
 +
 +            String keyspace = in.readUTF();
 +            String columnFamily = in.readUTF();
 +
 +            CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
 +            if (metadata == null)
 +            {
 +                String message = String.format("Got legacy range command for nonexistent table %s.%s.", keyspace, columnFamily);
 +                throw new UnknownColumnFamilyException(message, null);
 +            }
 +
 +            int nowInSec = (int) (in.readLong() / 1000);  // convert from millis to seconds
 +
 +            ClusteringIndexFilter filter;
 +            ColumnFilter selection;
 +            int compositesToGroup = 0;
 +            int perPartitionLimit = -1;
 +            byte readType = in.readByte();  // 0 for slices, 1 for names
 +            if (readType == 1)
 +            {
 +                Pair<ColumnFilter, ClusteringIndexNamesFilter> selectionAndFilter = LegacyReadCommandSerializer.deserializeNamesSelectionAndFilter(in, metadata);
 +                selection = selectionAndFilter.left;
 +                filter = selectionAndFilter.right;
 +            }
 +            else
 +            {
 +                Pair<ClusteringIndexSliceFilter, Boolean> p = LegacyReadCommandSerializer.deserializeSlicePartitionFilter(in, metadata);
 +                filter = p.left;
 +                perPartitionLimit = in.readInt();
 +                compositesToGroup = in.readInt();
 +                selection = getColumnSelectionForSlice(p.right, compositesToGroup, metadata);
 +            }
 +
 +            RowFilter rowFilter = deserializeRowFilter(in, metadata);
 +
 +            AbstractBounds<PartitionPosition> keyRange = AbstractBounds.rowPositionSerializer.deserialize(in, metadata.partitioner, version);
 +            int maxResults = in.readInt();
 +
 +            in.readBoolean();  // countCQL3Rows (not needed)
 +            in.readBoolean();  // isPaging (not needed)
 +
 +            boolean selectsStatics = (!selection.fetchedColumns().statics.isEmpty() || filter.selects(Clustering.STATIC_CLUSTERING));
 +            boolean isDistinct = compositesToGroup == -2 || (perPartitionLimit == 1 && selectsStatics);
 +            DataLimits limits;
 +            if (isDistinct)
 +                limits = DataLimits.distinctLimits(maxResults);
 +            else if (compositesToGroup == -1)
 +                limits = DataLimits.thriftLimits(maxResults, perPartitionLimit);
 +            else
 +                limits = DataLimits.cqlLimits(maxResults);
 +
 +            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter), Optional.empty());
 +        }
 +
 +        static void serializeRowFilter(DataOutputPlus out, RowFilter rowFilter) throws IOException
 +        {
 +            ArrayList<RowFilter.Expression> indexExpressions = Lists.newArrayList(rowFilter.iterator());
 +            out.writeInt(indexExpressions.size());
 +            for (RowFilter.Expression expression : indexExpressions)
 +            {
 +                ByteBufferUtil.writeWithShortLength(expression.column().name.bytes, out);
 +                expression.operator().writeTo(out);
 +                ByteBufferUtil.writeWithShortLength(expression.getIndexValue(), out);
 +            }
 +        }
 +
 +        static RowFilter deserializeRowFilter(DataInputPlus in, CFMetaData metadata) throws IOException
 +        {
 +            int numRowFilters = in.readInt();
 +            if (numRowFilters == 0)
 +                return RowFilter.NONE;
 +
 +            RowFilter rowFilter = RowFilter.create(numRowFilters);
 +            for (int i = 0; i < numRowFilters; i++)
 +            {
 +                ByteBuffer columnName = ByteBufferUtil.readWithShortLength(in);
 +                ColumnDefinition column = metadata.getColumnDefinition(columnName);
 +                Operator op = Operator.readFrom(in);
 +                ByteBuffer indexValue = ByteBufferUtil.readWithShortLength(in);
 +                rowFilter.add(column, op, indexValue);
 +            }
 +            return rowFilter;
 +        }
 +
 +        static long serializedRowFilterSize(RowFilter rowFilter)
 +        {
 +            long size = TypeSizes.sizeof(0);  // rowFilterCount
 +            for (RowFilter.Expression expression : rowFilter)
 +            {
 +                size += ByteBufferUtil.serializedSizeWithShortLength(expression.column().name.bytes);
 +                size += TypeSizes.sizeof(0);  // operator int value
 +                size += ByteBufferUtil.serializedSizeWithShortLength(expression.getIndexValue());
 +            }
 +            return size;
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            assert version < MessagingService.VERSION_30;
 +            assert command.kind == Kind.PARTITION_RANGE;
 +
 +            PartitionRangeReadCommand rangeCommand = (PartitionRangeReadCommand) command;
 +            rangeCommand = maybeConvertNamesToSlice(rangeCommand);
 +            CFMetaData metadata = rangeCommand.metadata();
 +
 +            long size = TypeSizes.sizeof(metadata.ksName);
 +            size += TypeSizes.sizeof(metadata.cfName);
 +            size += TypeSizes.sizeof((long) rangeCommand.nowInSec());
 +
 +            size += 1;  // single byte flag: 0 for slices, 1 for names
 +            if (rangeCommand.isNamesQuery())
 +            {
 +                PartitionColumns columns = rangeCommand.columnFilter().fetchedColumns();
 +                ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                size += LegacyReadCommandSerializer.serializedNamesFilterSize(filter, metadata, columns);
 +            }
 +            else
 +            {
 +                ClusteringIndexSliceFilter filter = (ClusteringIndexSliceFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                boolean makeStaticSlice = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() && !filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                size += LegacyReadCommandSerializer.serializedSlicesSize(filter.requestedSlices(), makeStaticSlice, metadata);
 +                size += TypeSizes.sizeof(filter.isReversed());
 +                size += TypeSizes.sizeof(rangeCommand.limits().perPartitionCount());
 +                size += TypeSizes.sizeof(0); // compositesToGroup
 +            }
 +
 +            if (rangeCommand.rowFilter().equals(RowFilter.NONE))
 +            {
 +                size += TypeSizes.sizeof(0);
 +            }
 +            else
 +            {
 +                ArrayList<RowFilter.Expression> indexExpressions = Lists.newArrayList(rangeCommand.rowFilter().iterator());
 +                size += TypeSizes.sizeof(indexExpressions.size());
 +                for (RowFilter.Expression expression : indexExpressions)
 +                {
 +                    size += ByteBufferUtil.serializedSizeWithShortLength(expression.column().name.bytes);
 +                    size += TypeSizes.sizeof(expression.operator().ordinal());
 +                    size += ByteBufferUtil.serializedSizeWithShortLength(expression.getIndexValue());
 +                }
 +            }
 +
 +            size += AbstractBounds.rowPositionSerializer.serializedSize(rangeCommand.dataRange().keyRange(), version);
 +            size += TypeSizes.sizeof(rangeCommand.limits().count());
 +            size += TypeSizes.sizeof(!rangeCommand.isForThrift());
 +            return size + TypeSizes.sizeof(rangeCommand.dataRange().isPaging());
 +        }
 +
 +        static PartitionRangeReadCommand maybeConvertNamesToSlice(PartitionRangeReadCommand command)
          {
 -            case GET_BY_NAMES:
 -                SliceByNamesReadCommand.serializer.serialize(command, out, version);
 -                break;
 -            case GET_SLICES:
 -                SliceFromReadCommand.serializer.serialize(command, out, version);
 -                break;
 -            default:
 -                throw new AssertionError();
 +            if (!command.dataRange().isNamesQuery())
 +                return command;
 +
 +            CFMetaData metadata = command.metadata();
 +            if (!LegacyReadCommandSerializer.shouldConvertNamesToSlice(metadata, command.columnFilter().fetchedColumns()))
 +                return command;
 +
 +            ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) command.dataRange().clusteringIndexFilter;
 +            ClusteringIndexSliceFilter sliceFilter = LegacyReadCommandSerializer.convertNamesFilterToSliceFilter(filter, metadata);
 +            DataRange newRange = new DataRange(command.dataRange().keyRange(), sliceFilter);
 +            return new PartitionRangeReadCommand(
 +                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
 +                    command.columnFilter(), command.rowFilter(), command.limits(), newRange, Optional.empty());
 +        }
 +
 +        static ColumnFilter getColumnSelectionForSlice(boolean selectsStatics, int compositesToGroup, CFMetaData metadata)
 +        {
 +            // A value of -2 indicates this is a DISTINCT query that doesn't select static columns, only partition keys.
 +            // In that case, we'll basically be querying the first row of the partition, but we must make sure we include
 +            // all columns so we get at least one cell if there is a live row as it would confuse pre-3.0 nodes otherwise.
 +            if (compositesToGroup == -2)
 +                return ColumnFilter.all(metadata);
 +
 +            // if a slice query from a pre-3.0 node doesn't cover statics, we shouldn't select them at all
 +            PartitionColumns columns = selectsStatics
 +                                     ? metadata.partitionColumns()
 +                                     : metadata.partitionColumns().withoutStatics();
 +            return ColumnFilter.selectionBuilder().addAll(columns).build();
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/MessagingService.java
index ccc900b,459923b..d416dca
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@@ -39,8 -39,12 +39,9 @@@ import com.google.common.collect.Lists
  import com.google.common.collect.Sets;
  
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
 -
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
 -
 -import org.apache.cassandra.concurrent.ExecutorLocal;
+ import org.apache.cassandra.concurrent.ExecutorLocals;
  import org.apache.cassandra.concurrent.ScheduledExecutors;
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;


[04/13] cassandra git commit: Make sure client gets tombstone overwhelmed warning

Posted by ca...@apache.org.
Make sure client gets tombstone overwhelmed warning

patch by Carl Yeksigian; reviewed by Josh McKenzie for CASSANDRA-9465


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

Branch: refs/heads/trunk
Commit: dbf6e62c382d62f9c1727ecf5afb90d131a81775
Parents: 582bdba
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 13:22:36 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:35:59 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 229 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 229 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cql3/statements/BatchStatement.java         |  11 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../cassandra/db/filter/SliceQueryFilter.java   |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   8 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cassandra/service/ClientWarningsTest.java   |  43 ++++
 21 files changed, 529 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11f2529..6530956 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.5
+ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
  * Fix error streaming section more than 2GB (CASSANDRA-10961)
  * (cqlsh) Also apply --connect-timeout to control connection
    timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
new file mode 100644
index 0000000..088b43e
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -0,0 +1,229 @@
+/*
+ * 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.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.tracing.Tracing.isTracing;
+
+public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+{
+    private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+
+    protected abstract void addTask(FutureTask<?> futureTask);
+    protected abstract void onCompletion();
+
+    /** Task Submission / Creation / Objects **/
+
+    public <T> FutureTask<T> submit(Callable<T> task)
+    {
+        return submit(newTaskFor(task));
+    }
+
+    public FutureTask<?> submit(Runnable task)
+    {
+        return submit(newTaskFor(task, null));
+    }
+
+    public <T> FutureTask<T> submit(Runnable task, T result)
+    {
+        return submit(newTaskFor(task, result));
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+    {
+        return newTaskFor(runnable, result, ExecutorLocals.create());
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+    {
+        if (locals != null)
+        {
+            if (runnable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) runnable;
+            return new LocalSessionFutureTask<T>(runnable, result, locals);
+        }
+        if (runnable instanceof FutureTask)
+            return (FutureTask<T>) runnable;
+        return new FutureTask<>(runnable, result);
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+    {
+        if (isTracing())
+        {
+            if (callable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) callable;
+            return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+        }
+        if (callable instanceof FutureTask)
+            return (FutureTask<T>) callable;
+        return new FutureTask<>(callable);
+    }
+
+    private class LocalSessionFutureTask<T> extends FutureTask<T>
+    {
+        private final ExecutorLocals locals;
+
+        public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+        {
+            super(callable);
+            this.locals = locals;
+        }
+
+        public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+        {
+            super(runnable, result);
+            this.locals = locals;
+        }
+
+        public void run()
+        {
+            ExecutorLocals old = ExecutorLocals.create();
+            ExecutorLocals.set(locals);
+            try
+            {
+                super.run();
+            }
+            finally
+            {
+                ExecutorLocals.set(old);
+            }
+        }
+    }
+
+    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+    {
+        private boolean failure;
+        private Object result = this;
+        private final Callable<T> callable;
+
+        public FutureTask(Callable<T> callable)
+        {
+            this.callable = callable;
+        }
+        public FutureTask(Runnable runnable, T result)
+        {
+            this(Executors.callable(runnable, result));
+        }
+
+        public void run()
+        {
+            try
+            {
+                result = callable.call();
+            }
+            catch (Throwable t)
+            {
+                JVMStabilityInspector.inspectThrowable(t);
+                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                result = t;
+                failure = true;
+            }
+            finally
+            {
+                signalAll();
+                onCompletion();
+            }
+        }
+
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            return false;
+        }
+
+        public boolean isCancelled()
+        {
+            return false;
+        }
+
+        public boolean isDone()
+        {
+            return isSignaled();
+        }
+
+        public T get() throws InterruptedException, ExecutionException
+        {
+            await();
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+
+        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            await(timeout, unit);
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+    }
+
+    private <T> FutureTask<T> submit(FutureTask<T> task)
+    {
+        addTask(task);
+        return task;
+    }
+
+    public void execute(Runnable command)
+    {
+        addTask(newTaskFor(command, ExecutorLocals.create()));
+    }
+
+    public void execute(Runnable command, ExecutorLocals locals)
+    {
+        addTask(newTaskFor(command, null, locals));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
deleted file mode 100644
index fb753b0..0000000
--- a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.concurrent;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-public abstract class AbstractTracingAwareExecutorService implements TracingAwareExecutorService
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractTracingAwareExecutorService.class);
-
-    protected abstract void addTask(FutureTask<?> futureTask);
-    protected abstract void onCompletion();
-
-    /** Task Submission / Creation / Objects **/
-
-    public <T> FutureTask<T> submit(Callable<T> task)
-    {
-        return submit(newTaskFor(task));
-    }
-
-    public FutureTask<?> submit(Runnable task)
-    {
-        return submit(newTaskFor(task, null));
-    }
-
-    public <T> FutureTask<T> submit(Runnable task, T result)
-    {
-        return submit(newTaskFor(task, result));
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
-    {
-        return newTaskFor(runnable, result, Tracing.instance.get());
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, TraceState traceState)
-    {
-        if (traceState != null)
-        {
-            if (runnable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) runnable;
-            return new TraceSessionFutureTask<T>(runnable, result, traceState);
-        }
-        if (runnable instanceof FutureTask)
-            return (FutureTask<T>) runnable;
-        return new FutureTask<>(runnable, result);
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing())
-        {
-            if (callable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) callable;
-            return new TraceSessionFutureTask<T>(callable, Tracing.instance.get());
-        }
-        if (callable instanceof FutureTask)
-            return (FutureTask<T>) callable;
-        return new FutureTask<>(callable);
-    }
-
-    private class TraceSessionFutureTask<T> extends FutureTask<T>
-    {
-        private final TraceState state;
-
-        public TraceSessionFutureTask(Callable<T> callable, TraceState state)
-        {
-            super(callable);
-            this.state = state;
-        }
-
-        public TraceSessionFutureTask(Runnable runnable, T result, TraceState state)
-        {
-            super(runnable, result);
-            this.state = state;
-        }
-
-        public void run()
-        {
-            TraceState oldState = Tracing.instance.get();
-            Tracing.instance.set(state);
-            try
-            {
-                super.run();
-            }
-            finally
-            {
-                Tracing.instance.set(oldState);
-            }
-        }
-    }
-
-    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
-    {
-        private boolean failure;
-        private Object result = this;
-        private final Callable<T> callable;
-
-        public FutureTask(Callable<T> callable)
-        {
-            this.callable = callable;
-        }
-        public FutureTask(Runnable runnable, T result)
-        {
-            this(Executors.callable(runnable, result));
-        }
-
-        public void run()
-        {
-            try
-            {
-                result = callable.call();
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
-                result = t;
-                failure = true;
-            }
-            finally
-            {
-                signalAll();
-                onCompletion();
-            }
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return isSignaled();
-        }
-
-        public T get() throws InterruptedException, ExecutionException
-        {
-            await();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-
-        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            await(timeout, unit);
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-    }
-
-    private <T> FutureTask<T> submit(FutureTask<T> task)
-    {
-        addTask(task);
-        return task;
-    }
-
-    public void execute(Runnable command)
-    {
-        addTask(newTaskFor(command, null));
-    }
-
-    public void execute(Runnable command, TraceState state)
-    {
-        addTask(newTaskFor(command, null, state));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index a6d0049..1fb0690 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -44,7 +44,7 @@ import static org.apache.cassandra.tracing.Tracing.isTracing;
  *   threads and the queue is full, we want the enqueuer to block.  But to allow the number of threads to drop if a
  *   stage is less busy, core thread timeout is enabled.
  */
-public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
 {
     protected static final Logger logger = LoggerFactory.getLogger(DebuggableThreadPoolExecutor.class);
     public static final RejectedExecutionHandler blockingExecutionHandler = new RejectedExecutionHandler()
@@ -146,11 +146,11 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     protected void onFinalAccept(Runnable task) {}
     protected void onFinalRejection(Runnable task) {}
 
-    public void execute(Runnable command, TraceState state)
+    public void execute(Runnable command, ExecutorLocals locals)
     {
-        super.execute(state == null || command instanceof TraceSessionWrapper
+        super.execute(locals == null || command instanceof LocalSessionWrapper
                       ? command
-                      : new TraceSessionWrapper<Object>(command, state));
+                      : new LocalSessionWrapper<Object>(command, locals));
     }
 
     public void maybeExecuteImmediately(Runnable command)
@@ -162,17 +162,17 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     public void execute(Runnable command)
     {
-        super.execute(isTracing() && !(command instanceof TraceSessionWrapper)
-                      ? new TraceSessionWrapper<Object>(Executors.callable(command, null))
+        super.execute(isTracing() && !(command instanceof LocalSessionWrapper)
+                      ? new LocalSessionWrapper<Object>(Executors.callable(command, null))
                       : command);
     }
 
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T result)
     {
-        if (isTracing() && !(runnable instanceof TraceSessionWrapper))
+        if (isTracing() && !(runnable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(Executors.callable(runnable, result));
+            return new LocalSessionWrapper<T>(Executors.callable(runnable, result));
         }
         return super.newTaskFor(runnable, result);
     }
@@ -180,9 +180,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
     {
-        if (isTracing() && !(callable instanceof TraceSessionWrapper))
+        if (isTracing() && !(callable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(callable);
+            return new LocalSessionWrapper<T>(callable);
         }
         return super.newTaskFor(callable);
     }
@@ -198,9 +198,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
 
     protected static void maybeResetTraceSessionWrapper(Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
+        if (r instanceof LocalSessionWrapper)
         {
-            TraceSessionWrapper tsw = (TraceSessionWrapper) r;
+            LocalSessionWrapper tsw = (LocalSessionWrapper) r;
             // we have to reset trace state as its presence is what denotes the current thread is tracing
             // and if left this thread might start tracing unrelated tasks
             tsw.reset();
@@ -210,8 +210,8 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     @Override
     protected void beforeExecute(Thread t, Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
-            ((TraceSessionWrapper) r).setupContext();
+        if (r instanceof LocalSessionWrapper)
+            ((LocalSessionWrapper) r).setupContext();
 
         super.beforeExecute(t, r);
     }
@@ -278,35 +278,35 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
     }
 
     /**
-     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the TraceSessionContext and move
-     * it into the worker thread.
+     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the ExecutorLocals and move
+     * them into the worker thread.
      *
      * @param <T>
      */
-    private static class TraceSessionWrapper<T> extends FutureTask<T>
+    private static class LocalSessionWrapper<T> extends FutureTask<T>
     {
-        private final TraceState state;
+        private final ExecutorLocals locals;
 
-        public TraceSessionWrapper(Callable<T> callable)
+        public LocalSessionWrapper(Callable<T> callable)
         {
             super(callable);
-            state = Tracing.instance.get();
+            locals = ExecutorLocals.create();
         }
 
-        public TraceSessionWrapper(Runnable command, TraceState state)
+        public LocalSessionWrapper(Runnable command, ExecutorLocals locals)
         {
             super(command, null);
-            this.state = state;
+            this.locals = locals;
         }
 
         private void setupContext()
         {
-            Tracing.instance.set(state);
+            ExecutorLocals.set(locals);
         }
 
         private void reset()
         {
-            Tracing.instance.set(null);
+            ExecutorLocals.set(null);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
new file mode 100644
index 0000000..47826f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
@@ -0,0 +1,44 @@
+/*
+ * 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.concurrent;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+public interface ExecutorLocal<T>
+{
+    ExecutorLocal[] all = { Tracing.instance, ClientWarn.instance };
+
+    /**
+     * This is called when scheduling the task, and also before calling {@link ExecutorLocal#set(T)} when running on a
+     * executor thread.
+     *
+     * @return The thread-local value that we want to copy across executor boundaries; may be null if not set.
+     */
+    T get();
+
+    /**
+     * Before a task has been run, this will be called with the value from the thread that scheduled the task, and after
+     * the task is finished, the value that was previously retrieved from this thread is restored.
+     *
+     * @param value Value to use for the executor local state; may be null.
+     */
+    void set(T value);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
new file mode 100644
index 0000000..8e6d6ea
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
@@ -0,0 +1,84 @@
+/*
+ * 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.concurrent;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+/*
+ * This class only knows about Tracing and ClientWarn, so if any different executor locals are added, it must be
+ * updated.
+ *
+ * We don't enumerate the ExecutorLocal.all array each time because it would be much slower.
+ */
+public class ExecutorLocals
+{
+    private static final ExecutorLocal<TraceState> tracing = Tracing.instance;
+    private static final ExecutorLocal<ClientWarn.State> clientWarn = ClientWarn.instance;
+
+    public final TraceState traceState;
+    public final ClientWarn.State clientWarnState;
+
+    private ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
+    {
+        this.traceState = traceState;
+        this.clientWarnState = clientWarnState;
+    }
+
+    static
+    {
+        assert Arrays.equals(ExecutorLocal.all, new ExecutorLocal[]{ tracing, clientWarn })
+        : "ExecutorLocals has not been updated to reflect new ExecutorLocal.all";
+    }
+
+    /**
+     * This creates a new ExecutorLocals object based on what is already set.
+     *
+     * @return an ExecutorLocals object which has the trace state and client warn state captured if either has been set,
+     *         or null if both are unset. The null result short-circuits logic in
+     *         {@link AbstractLocalAwareExecutorService#newTaskFor(Runnable, Object, ExecutorLocals)}, preventing
+     *         unnecessarily calling {@link ExecutorLocals#set(ExecutorLocals)}.
+     */
+    public static ExecutorLocals create()
+    {
+        TraceState traceState = tracing.get();
+        ClientWarn.State clientWarnState = clientWarn.get();
+        if (traceState == null && clientWarnState == null)
+            return null;
+        else
+            return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static ExecutorLocals create(TraceState traceState)
+    {
+        ClientWarn.State clientWarnState = clientWarn.get();
+        return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static void set(ExecutorLocals locals)
+    {
+        TraceState traceState = locals == null ? null : locals.traceState;
+        ClientWarn.State clientWarnState = locals == null ? null : locals.clientWarnState;
+        tracing.set(traceState);
+        clientWarn.set(clientWarnState);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
new file mode 100644
index 0000000..5577d59
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * 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.concurrent;
+
+import java.util.concurrent.ExecutorService;
+
+public interface LocalAwareExecutorService extends ExecutorService
+{
+    // we need a way to inject a TraceState directly into the Executor context without going through
+    // the global Tracing sessions; see CASSANDRA-5668
+    public void execute(Runnable command, ExecutorLocals locals);
+
+    // permits executing in the context of the submitting thread
+    public void maybeExecuteImmediately(Runnable command);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index d9a0fa8..8b12b82 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.cassandra.metrics.SEPMetrics;
@@ -30,7 +29,7 @@ import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 
-public class SEPExecutor extends AbstractTracingAwareExecutorService
+public class SEPExecutor extends AbstractLocalAwareExecutorService
 {
     private final SharedExecutorPool pool;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index 8c18c44..dfd7011 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -103,7 +103,7 @@ public class SharedExecutorPool
             schedule(Work.SPINNING);
     }
 
-    public TracingAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
+    public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
     {
         SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name);
         executors.add(executor);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 4f03fd5..114795e 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -39,7 +39,7 @@ public class StageManager
 {
     private static final Logger logger = LoggerFactory.getLogger(StageManager.class);
 
-    private static final EnumMap<Stage, TracingAwareExecutorService> stages = new EnumMap<Stage, TracingAwareExecutorService>(Stage.class);
+    private static final EnumMap<Stage, LocalAwareExecutorService> stages = new EnumMap<Stage, LocalAwareExecutorService>(Stage.class);
 
     public static final long KEEPALIVE = 60; // seconds to keep "extra" threads alive for when idle
 
@@ -87,7 +87,7 @@ public class StageManager
                                                 stage.getJmxType());
     }
 
-    private static TracingAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
+    private static LocalAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
     {
         return SharedExecutorPool.SHARED.newExecutor(numThreads, Integer.MAX_VALUE, stage.getJmxType(), stage.getJmxName());
     }
@@ -96,7 +96,7 @@ public class StageManager
      * Retrieve a stage from the StageManager
      * @param stage name of the stage to be retrieved.
      */
-    public static TracingAwareExecutorService getStage(Stage stage)
+    public static LocalAwareExecutorService getStage(Stage stage)
     {
         return stages.get(stage);
     }
@@ -116,16 +116,16 @@ public class StageManager
      * A TPE that disallows submit so that we don't need to worry about unwrapping exceptions on the
      * tracing stage.  See CASSANDRA-1123 for background.
      */
-    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
     {
         public ExecuteOnlyExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
         {
             super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
         }
 
-        public void execute(Runnable command, TraceState state)
+        public void execute(Runnable command, ExecutorLocals locals)
         {
-            assert state == null;
+            assert locals == null;
             super.execute(command);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
deleted file mode 100644
index f580fea..0000000
--- a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 
- * 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.concurrent;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.cassandra.tracing.TraceState;
-
-public interface TracingAwareExecutorService extends ExecutorService
-{
-    // we need a way to inject a TraceState directly into the Executor context without going through
-    // the global Tracing sessions; see CASSANDRA-5668
-    public void execute(Runnable command, TraceState state);
-
-    // permits executing in the context of the submitting thread
-    public void maybeExecuteImmediately(Runnable command);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 43a80bb..a289ad1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -273,7 +273,7 @@ public class BatchStatement implements CQLStatement
             {
                 logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
             }
-            ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {ksCfPairs, size, warnThreshold, size - warnThreshold, ""}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
         }
     }
 
@@ -305,8 +305,13 @@ public class BatchStatement implements CQLStatement
                              keySet.size(), keySet.size() == 1 ? "" : "s",
                              ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
 
-            ClientWarn.warn(MessageFormatter.arrayFormat(unloggedBatchWarning, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
-                                                    ksCfPairs.size() == 1 ? "" : "s", ksCfPairs}).getMessage());
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
+                                                                  new Object[]{
+                                                                              keySet.size(),
+                                                                              keySet.size() == 1 ? "" : "s",
+                                                                              ksCfPairs.size() == 1 ? "" : "s",
+                                                                              ksCfPairs
+                                                                  }).getMessage());
 
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 5f142ce..5cfa94b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -251,12 +251,12 @@ public class SelectStatement implements CQLStatement
         if (!restrictions.hasPartitionKeyRestrictions())
         {
             logger.warn("Aggregation query used without partition key");
-            ClientWarn.warn("Aggregation query used without partition key");
+            ClientWarn.instance.warn("Aggregation query used without partition key");
         }
         else if (restrictions.keyIsInRelation())
         {
             logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-            ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+            ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
         }
 
         Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index e1a68e7..d2f0bf4 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -303,7 +303,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
                                        container.metadata().getKeyValidator().getString(key.getKey()),
                                        count,
                                        getSlicesInfo(container));
-            ClientWarn.warn(msg);
+            ClientWarn.instance.warn(msg);
             logger.warn(msg);
         }
         Tracing.trace("Read {} live and {} tombstone cells{}",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 61e58c2..459923b 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -43,10 +43,12 @@ import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.concurrent.ExecutorLocals;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.*;
@@ -801,10 +803,10 @@ public final class MessagingService implements MessagingServiceMBean
                 return;
 
         Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
-        TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+        LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
         assert stage != null : "No stage for message type " + message.verb;
 
-        stage.execute(runnable, state);
+        stage.execute(runnable, ExecutorLocals.create(state));
     }
 
     public void setCallbackForTests(int messageId, CallbackInfo callback)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/ClientWarn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java
index 2ed0a6c..ddad197 100644
--- a/src/java/org/apache/cassandra/service/ClientWarn.java
+++ b/src/java/org/apache/cassandra/service/ClientWarn.java
@@ -20,54 +20,68 @@ package org.apache.cassandra.service;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class ClientWarn
+public class ClientWarn implements ExecutorLocal<ClientWarn.State>
 {
     private static final String TRUNCATED = " [truncated]";
-    private static final ThreadLocal<ClientWarn> warnLocal = new ThreadLocal<>();
-
-    private final List<String> warnings = new ArrayList<>();
+    private static final ThreadLocal<ClientWarn.State> warnLocal = new ThreadLocal<>();
+    public static ClientWarn instance = new ClientWarn();
 
     private ClientWarn()
     {
     }
 
-    public static void warn(String text)
-    {
-        ClientWarn warner = warnLocal.get();
-        if (warner != null)
-            warner.add(text);
+    public State get() {
+        return warnLocal.get();
     }
 
-    private void add(String warning)
-    {
-        if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
-            warnings.add(maybeTruncate(warning));
+    public void set(State value) {
+        warnLocal.set(value);
     }
 
-    private static String maybeTruncate(String warning)
+    public void warn(String text)
     {
-        return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
-             ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
-             : warning;
+        State state = warnLocal.get();
+        if (state != null)
+            state.add(text);
     }
 
-    public static void captureWarnings()
+    public void captureWarnings()
     {
-        warnLocal.set(new ClientWarn());
+        warnLocal.set(new State());
     }
 
-    public static List<String> getWarnings()
+    public List<String> getWarnings()
     {
-        ClientWarn warner = warnLocal.get();
-        if (warner == null || warner.warnings.isEmpty())
+        State state = warnLocal.get();
+        if (state == null || state.warnings.isEmpty())
             return null;
-        return warner.warnings;
+        return state.warnings;
     }
 
-    public static void resetWarnings()
+    public void resetWarnings()
     {
         warnLocal.remove();
     }
+
+    public static class State
+    {
+        private final List<String> warnings = new ArrayList<>();
+
+        private void add(String warning)
+        {
+            if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
+                warnings.add(maybeTruncate(warning));
+        }
+
+        private static String maybeTruncate(String warning)
+        {
+            return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
+                   ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
+                   : warning;
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 88253e3..841e980 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -1842,7 +1842,7 @@ public class StorageProxy implements StorageProxyMBean
                     if (filteredEndpoints.size() == 1
                         && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
                     {
-                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler), Tracing.instance.get());
+                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index ccc2637..bf9cee7 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.net.MessageIn;
@@ -41,7 +42,7 @@ import org.apache.cassandra.utils.UUIDGen;
  * A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
  * have multiple local and remote events before it is completed. All events and sessions are stored at keyspace.
  */
-public class Tracing
+public class Tracing implements ExecutorLocal<TraceState>
 {
     public static final String TRACE_HEADER = "TraceSession";
     public static final String TRACE_TYPE = "TraceType";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index ab794df..01a0794 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -499,14 +499,14 @@ public abstract class Message
                 assert request.connection() instanceof ServerConnection;
                 connection = (ServerConnection)request.connection();
                 if (connection.getVersion() >= Server.VERSION_4)
-                    ClientWarn.captureWarnings();
+                    ClientWarn.instance.captureWarnings();
 
                 QueryState qstate = connection.validateNewMessage(request.type, connection.getVersion(), request.getStreamId());
 
                 logger.trace("Received: {}, v={}", request, connection.getVersion());
                 response = request.execute(qstate);
                 response.setStreamId(request.getStreamId());
-                response.setWarnings(ClientWarn.getWarnings());
+                response.setWarnings(ClientWarn.instance.getWarnings());
                 response.attach(connection);
                 connection.applyStateTransition(request.type, response.type);
             }
@@ -519,7 +519,7 @@ public abstract class Message
             }
             finally
             {
-                ClientWarn.resetWarnings();
+                ClientWarn.instance.resetWarnings();
             }
 
             logger.trace("Responding: {}, v={}", response, connection.getVersion());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
index 4ecd6a7..289f3e3 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
 import io.netty.util.concurrent.AbstractEventExecutor;
 import io.netty.util.concurrent.EventExecutorGroup;
 import io.netty.util.concurrent.Future;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED;
@@ -32,7 +32,7 @@ public class RequestThreadPoolExecutor extends AbstractEventExecutor
 {
     private final static int MAX_QUEUED_REQUESTS = 128;
     private final static String THREAD_FACTORY_ID = "Native-Transport-Requests";
-    private final TracingAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
+    private final LocalAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
                                                                            MAX_QUEUED_REQUESTS,
                                                                            "transport",
                                                                            THREAD_FACTORY_ID);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbf6e62c/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index cfd5f7a..d22a8f6 100644
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@ -24,6 +24,8 @@ import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.transport.Server;
@@ -79,6 +81,47 @@ public class ClientWarningsTest extends CQLTester
     }
 
     @Test
+    public void testTombstoneWarning() throws Exception
+    {
+        final int iterations = 10000;
+        createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+            store.forceBlockingFlush();
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            store.forceBlockingFlush();
+
+            {
+                QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                    KEYSPACE,
+                                                                    currentTable()), QueryOptions.DEFAULT);
+                Message.Response resp = client.execute(query);
+                assertEquals(1, resp.getWarnings().size());
+            }
+        }
+    }
+
+    @Test
     public void testLargeBatchWithProtoV2() throws Exception
     {
         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");


[09/13] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageProxy.java
index e2fa270,841e980..89ac0bb
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@@ -1941,201 -1711,252 +1941,201 @@@ public class StorageProxy implements St
          }
      }
  
 -    public static List<Row> getRangeSlice(AbstractRangeCommand command, ConsistencyLevel consistency_level)
 -    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    private static class SingleRangeResponse extends AbstractIterator<RowIterator> implements PartitionIterator
      {
 -        Tracing.trace("Computing ranges to query");
 -        long startTime = System.nanoTime();
 +        private final ReadCallback handler;
 +        private PartitionIterator result;
  
 -        Keyspace keyspace = Keyspace.open(command.keyspace);
 -        List<Row> rows;
 -        // now scan until we have enough results
 -        try
 +        private SingleRangeResponse(ReadCallback handler)
          {
 -            int liveRowCount = 0;
 -            boolean countLiveRows = command.countCQL3Rows() || command.ignoredTombstonedPartitions();
 -            rows = new ArrayList<>();
 +            this.handler = handler;
 +        }
  
 -            // when dealing with LocalStrategy keyspaces, we can skip the range splitting and merging (which can be
 -            // expensive in clusters with vnodes)
 -            List<? extends AbstractBounds<RowPosition>> ranges;
 -            if (keyspace.getReplicationStrategy() instanceof LocalStrategy)
 -                ranges = command.keyRange.unwrap();
 -            else
 -                ranges = getRestrictedRanges(command.keyRange);
 -
 -            // determine the number of rows to be fetched and the concurrency factor
 -            int rowsToBeFetched = command.limit();
 -            int concurrencyFactor;
 -            if (command.requiresScanningAllRanges())
 -            {
 -                // all nodes must be queried
 -                rowsToBeFetched *= ranges.size();
 -                concurrencyFactor = ranges.size();
 -                logger.debug("Requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {}",
 -                              ranges.size(), concurrencyFactor);
 +        private void waitForResponse() throws ReadTimeoutException
 +        {
 +            if (result != null)
 +                return;
 +
 +            try
 +            {
 +                result = handler.get();
              }
 -            else
 +            catch (DigestMismatchException e)
              {
 -                // our estimate of how many result rows there will be per-range
 -                float resultRowsPerRange = estimateResultRowsPerRange(command, keyspace);
 -                // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 -                // fetch enough rows in the first round
 -                resultRowsPerRange -= resultRowsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 -                concurrencyFactor = resultRowsPerRange == 0.0
 -                                  ? 1
 -                                  : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit() / resultRowsPerRange)));
 -
 -                logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 -                             resultRowsPerRange,
 -                             command.limit(),
 -                             ranges.size(),
 -                             concurrencyFactor);
 -                Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)",
 -                              ranges.size(),
 -                              concurrencyFactor,
 -                              resultRowsPerRange);
 -            }
 -
 -            boolean haveSufficientRows = false;
 -            int i = 0;
 -            AbstractBounds<RowPosition> nextRange = null;
 -            List<InetAddress> nextEndpoints = null;
 -            List<InetAddress> nextFilteredEndpoints = null;
 -            while (i < ranges.size())
 -            {
 -                List<Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>>> scanHandlers = new ArrayList<>(concurrencyFactor);
 -                int concurrentFetchStartingIndex = i;
 -                int concurrentRequests = 0;
 -                while ((i - concurrentFetchStartingIndex) < concurrencyFactor)
 -                {
 -                    AbstractBounds<RowPosition> range = nextRange == null
 -                                                      ? ranges.get(i)
 -                                                      : nextRange;
 -                    List<InetAddress> liveEndpoints = nextEndpoints == null
 -                                                    ? getLiveSortedEndpoints(keyspace, range.right)
 -                                                    : nextEndpoints;
 -                    List<InetAddress> filteredEndpoints = nextFilteredEndpoints == null
 -                                                        ? consistency_level.filterForQuery(keyspace, liveEndpoints)
 -                                                        : nextFilteredEndpoints;
 -                    ++i;
 -                    ++concurrentRequests;
 -
 -                    // getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take
 -                    // the replication factor into account. If the intersection of live endpoints for 2 consecutive ranges
 -                    // still meets the CL requirements, then we can merge both ranges into the same RangeSliceCommand.
 -                    while (i < ranges.size())
 -                    {
 -                        nextRange = ranges.get(i);
 -                        nextEndpoints = getLiveSortedEndpoints(keyspace, nextRange.right);
 -                        nextFilteredEndpoints = consistency_level.filterForQuery(keyspace, nextEndpoints);
 -
 -                        // If the current range right is the min token, we should stop merging because CFS.getRangeSlice
 -                        // don't know how to deal with a wrapping range.
 -                        // Note: it would be slightly more efficient to have CFS.getRangeSlice on the destination nodes unwraps
 -                        // the range if necessary and deal with it. However, we can't start sending wrapped range without breaking
 -                        // wire compatibility, so It's likely easier not to bother;
 -                        if (range.right.isMinimum())
 -                            break;
 -
 -                        List<InetAddress> merged = intersection(liveEndpoints, nextEndpoints);
 -
 -                        // Check if there is enough endpoint for the merge to be possible.
 -                        if (!consistency_level.isSufficientLiveNodes(keyspace, merged))
 -                            break;
 -
 -                        List<InetAddress> filteredMerged = consistency_level.filterForQuery(keyspace, merged);
 -
 -                        // Estimate whether merging will be a win or not
 -                        if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, filteredEndpoints, nextFilteredEndpoints))
 -                            break;
 -
 -                        // If we get there, merge this range and the next one
 -                        range = range.withNewRight(nextRange.right);
 -                        liveEndpoints = merged;
 -                        filteredEndpoints = filteredMerged;
 -                        ++i;
 -                    }
 +                throw new AssertionError(e); // no digests in range slices yet
 +            }
 +        }
  
 -                    AbstractRangeCommand nodeCmd = command.forSubRange(range);
 +        protected RowIterator computeNext()
 +        {
 +            waitForResponse();
 +            return result.hasNext() ? result.next() : endOfData();
 +        }
  
 -                    // collect replies and resolve according to consistency level
 -                    RangeSliceResponseResolver resolver = new RangeSliceResponseResolver(nodeCmd.keyspace, command.timestamp);
 -                    List<InetAddress> minimalEndpoints = filteredEndpoints.subList(0, Math.min(filteredEndpoints.size(), consistency_level.blockFor(keyspace)));
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = new ReadCallback<>(resolver, consistency_level, nodeCmd, minimalEndpoints);
 -                    handler.assureSufficientLiveNodes();
 -                    resolver.setSources(filteredEndpoints);
 -                    if (filteredEndpoints.size() == 1
 -                        && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
 -                    {
 -                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
 -                    }
 -                    else
 -                    {
 -                        MessageOut<? extends AbstractRangeCommand> message = nodeCmd.createMessage();
 -                        for (InetAddress endpoint : filteredEndpoints)
 -                        {
 -                            Tracing.trace("Enqueuing request to {}", endpoint);
 -                            MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
 -                        }
 -                    }
 -                    scanHandlers.add(Pair.create(nodeCmd, handler));
 -                }
 -                Tracing.trace("Submitted {} concurrent range requests covering {} ranges", concurrentRequests, i - concurrentFetchStartingIndex);
 +        public void close()
 +        {
 +            if (result != null)
 +                result.close();
 +        }
 +    }
  
 -                List<AsyncOneResponse> repairResponses = new ArrayList<>();
 -                for (Pair<AbstractRangeCommand, ReadCallback<RangeSliceReply, Iterable<Row>>> cmdPairHandler : scanHandlers)
 -                {
 -                    ReadCallback<RangeSliceReply, Iterable<Row>> handler = cmdPairHandler.right;
 -                    RangeSliceResponseResolver resolver = (RangeSliceResponseResolver)handler.resolver;
 +    private static class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
 +    {
 +        private final Iterator<RangeForQuery> ranges;
 +        private final int totalRangeCount;
 +        private final PartitionRangeReadCommand command;
 +        private final Keyspace keyspace;
 +        private final ConsistencyLevel consistency;
  
 -                    try
 -                    {
 -                        for (Row row : handler.get())
 -                        {
 -                            rows.add(row);
 -                            if (countLiveRows)
 -                                liveRowCount += row.getLiveCount(command.predicate, command.timestamp);
 -                        }
 -                        repairResponses.addAll(resolver.repairResults);
 -                    }
 -                    catch (ReadTimeoutException|ReadFailureException ex)
 -                    {
 -                        // we timed out or failed waiting for responses
 -                        int blockFor = consistency_level.blockFor(keyspace);
 -                        int responseCount = resolver.responses.size();
 -                        String gotData = responseCount > 0
 -                                         ? resolver.isDataPresent() ? " (including data)" : " (only digests)"
 -                                         : "";
 -
 -                        boolean isTimeout = ex instanceof ReadTimeoutException;
 -                        if (Tracing.isTracing())
 -                        {
 -                            Tracing.trace("{}; received {} of {} responses{} for range {} of {}",
 -                                          (isTimeout ? "Timed out" : "Failed"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        else if (logger.isDebugEnabled())
 -                        {
 -                            logger.debug("Range slice {}; received {} of {} responses{} for range {} of {}",
 -                                         (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData, i, ranges.size());
 -                        }
 -                        throw ex;
 -                    }
 -                    catch (DigestMismatchException e)
 -                    {
 -                        throw new AssertionError(e); // no digests in range slices yet
 -                    }
 +        private final long startTime;
 +        private DataLimits.Counter counter;
 +        private PartitionIterator sentQueryIterator;
  
 -                    // if we're done, great, otherwise, move to the next range
 -                    int count = countLiveRows ? liveRowCount : rows.size();
 -                    if (count >= rowsToBeFetched)
 -                    {
 -                        haveSufficientRows = true;
 -                        break;
 -                    }
 -                }
 +        private int concurrencyFactor;
 +        // The two following "metric" are maintained to improve the concurrencyFactor
 +        // when it was not good enough initially.
 +        private int liveReturned;
 +        private int rangesQueried;
  
 -                try
 -                {
 -                    FBUtilities.waitOnFutures(repairResponses, DatabaseDescriptor.getWriteRpcTimeout());
 -                }
 -                catch (TimeoutException ex)
 +        public RangeCommandIterator(RangeIterator ranges, PartitionRangeReadCommand command, int concurrencyFactor, Keyspace keyspace, ConsistencyLevel consistency)
 +        {
 +            this.command = command;
 +            this.concurrencyFactor = concurrencyFactor;
 +            this.startTime = System.nanoTime();
 +            this.ranges = new RangeMerger(ranges, keyspace, consistency);
 +            this.totalRangeCount = ranges.rangeCount();
 +            this.consistency = consistency;
 +            this.keyspace = keyspace;
 +        }
 +
 +        public RowIterator computeNext()
 +        {
 +            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
 +            {
 +                // If we don't have more range to handle, we're done
 +                if (!ranges.hasNext())
 +                    return endOfData();
 +
 +                // else, sends the next batch of concurrent queries (after having close the previous iterator)
 +                if (sentQueryIterator != null)
                  {
 -                    // We got all responses, but timed out while repairing
 -                    int blockFor = consistency_level.blockFor(keyspace);
 -                    if (Tracing.isTracing())
 -                        Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    else
 -                        logger.debug("Range slice timeout while read-repairing after receiving all {} data and digest responses", blockFor);
 -                    throw new ReadTimeoutException(consistency_level, blockFor-1, blockFor, true);
 +                    liveReturned += counter.counted();
 +                    sentQueryIterator.close();
 +
 +                    // It's not the first batch of queries and we're not done, so we we can use what has been
 +                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
 +                    updateConcurrencyFactor();
                  }
 +                sentQueryIterator = sendNextRequests();
 +            }
 +
 +            return sentQueryIterator.next();
 +        }
 +
 +        private void updateConcurrencyFactor()
 +        {
 +            if (liveReturned == 0)
 +            {
 +                // we haven't actually gotten any results, so query all remaining ranges at once
 +                concurrencyFactor = totalRangeCount - rangesQueried;
 +                return;
 +            }
  
 -                if (haveSufficientRows)
 -                    return command.postReconciliationProcessing(rows);
 +            // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
 +            // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
 +            int remainingRows = command.limits().count() - liveReturned;
 +            float rowsPerRange = (float)liveReturned / (float)rangesQueried;
 +            concurrencyFactor = Math.max(1, Math.min(totalRangeCount - rangesQueried, Math.round(remainingRows / rowsPerRange)));
 +            logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 +                         rowsPerRange, (int) remainingRows, concurrencyFactor);
 +        }
 +
 +        private SingleRangeResponse query(RangeForQuery toQuery)
 +        {
 +            PartitionRangeReadCommand rangeCommand = command.forSubRange(toQuery.range);
 +
 +            DataResolver resolver = new DataResolver(keyspace, rangeCommand, consistency, toQuery.filteredEndpoints.size());
 +
 +            int blockFor = consistency.blockFor(keyspace);
 +            int minResponses = Math.min(toQuery.filteredEndpoints.size(), blockFor);
 +            List<InetAddress> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
 +            ReadCallback handler = new ReadCallback(resolver, consistency, rangeCommand, minimalEndpoints);
  
 -                // we didn't get enough rows in our concurrent fetch; recalculate our concurrency factor
 -                // based on the results we've seen so far (as long as we still have ranges left to query)
 -                if (i < ranges.size())
 +            handler.assureSufficientLiveNodes();
 +
 +            if (toQuery.filteredEndpoints.size() == 1 && canDoLocalRequest(toQuery.filteredEndpoints.get(0)))
 +            {
-                 StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler), Tracing.instance.get());
++                StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler));
 +            }
 +            else
 +            {
 +                for (InetAddress endpoint : toQuery.filteredEndpoints)
                  {
 -                    float fetchedRows = countLiveRows ? liveRowCount : rows.size();
 -                    float remainingRows = rowsToBeFetched - fetchedRows;
 -                    float actualRowsPerRange;
 -                    if (fetchedRows == 0.0)
 -                    {
 -                        // we haven't actually gotten any results, so query all remaining ranges at once
 -                        actualRowsPerRange = 0.0f;
 -                        concurrencyFactor = ranges.size() - i;
 -                    }
 -                    else
 -                    {
 -                        actualRowsPerRange = fetchedRows / i;
 -                        concurrencyFactor = Math.max(1, Math.min(ranges.size() - i, Math.round(remainingRows / actualRowsPerRange)));
 -                    }
 -                    logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
 -                                 actualRowsPerRange, (int) remainingRows, concurrencyFactor);
 +                    MessageOut<ReadCommand> message = rangeCommand.createMessage(MessagingService.instance().getVersion(endpoint));
 +                    Tracing.trace("Enqueuing request to {}", endpoint);
 +                    MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
                  }
              }
 +
 +            return new SingleRangeResponse(handler);
          }
 -        finally
 +
 +        private PartitionIterator sendNextRequests()
          {
 -            long latency = System.nanoTime() - startTime;
 -            rangeMetrics.addNano(latency);
 -            Keyspace.open(command.keyspace).getColumnFamilyStore(command.columnFamily).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            List<PartitionIterator> concurrentQueries = new ArrayList<>(concurrencyFactor);
 +            for (int i = 0; i < concurrencyFactor && ranges.hasNext(); i++)
 +            {
 +                concurrentQueries.add(query(ranges.next()));
 +                ++rangesQueried;
 +            }
 +
 +            Tracing.trace("Submitted {} concurrent range requests", concurrentQueries.size());
 +            // We want to count the results for the sake of updating the concurrency factor (see updateConcurrencyFactor) but we don't want to
 +            // enforce any particular limit at this point (this could break code than rely on postReconciliationProcessing), hence the DataLimits.NONE.
 +            counter = DataLimits.NONE.newCounter(command.nowInSec(), true);
 +            return counter.applyTo(PartitionIterators.concat(concurrentQueries));
          }
 -        return command.postReconciliationProcessing(rows);
 +
 +        public void close()
 +        {
 +            try
 +            {
 +                if (sentQueryIterator != null)
 +                    sentQueryIterator.close();
 +            }
 +            finally
 +            {
 +                long latency = System.nanoTime() - startTime;
 +                rangeMetrics.addNano(latency);
 +                Keyspace.openAndGetStore(command.metadata()).metric.coordinatorScanLatency.update(latency, TimeUnit.NANOSECONDS);
 +            }
 +        }
 +    }
 +
 +    @SuppressWarnings("resource")
 +    public static PartitionIterator getRangeSlice(PartitionRangeReadCommand command, ConsistencyLevel consistencyLevel)
 +    throws UnavailableException, ReadFailureException, ReadTimeoutException
 +    {
 +        Tracing.trace("Computing ranges to query");
 +
 +        Keyspace keyspace = Keyspace.open(command.metadata().ksName);
 +        RangeIterator ranges = new RangeIterator(command, keyspace, consistencyLevel);
 +
 +        // our estimate of how many result rows there will be per-range
 +        float resultsPerRange = estimateResultsPerRange(command, keyspace);
 +        // underestimate how many rows we will get per-range in order to increase the likelihood that we'll
 +        // fetch enough rows in the first round
 +        resultsPerRange -= resultsPerRange * CONCURRENT_SUBREQUESTS_MARGIN;
 +        int concurrencyFactor = resultsPerRange == 0.0
 +                              ? 1
 +                              : Math.max(1, Math.min(ranges.rangeCount(), (int) Math.ceil(command.limits().count() / resultsPerRange)));
 +        logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
 +                     resultsPerRange, command.limits().count(), ranges.rangeCount(), concurrencyFactor);
 +        Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)", ranges.rangeCount(), concurrencyFactor, resultsPerRange);
 +
 +        // Note that in general, a RangeCommandIterator will honor the command limit for each range, but will not enforce it globally.
 +
 +        return command.limits().filter(command.postReconciliationProcessing(new RangeCommandIterator(ranges, command, concurrencyFactor, keyspace, consistencyLevel)), command.nowInSec());
      }
  
      public Map<String, List<String>> getSchemaVersions()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 704a6c9,bcfe871..6d54e36
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -2263,226 -2686,4 +2263,226 @@@ public class UFTest extends CQLTeste
                             "AS 'return 0;'");
          }
      }
 +
 +    @Test
 +    public void testSecurityPermissions() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, dval double)");
 +        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
 +
 +        // Java UDFs
 +
 +        try
 +        {
 +            String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                          "RETURNS NULL ON NULL INPUT " +
 +                                          "RETURNS double " +
 +                                          "LANGUAGE JAVA\n" +
 +                                          "AS 'System.getProperty(\"foo.bar.baz\"); return 0d;';");
 +            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +            Assert.fail();
 +        }
 +        catch (FunctionExecutionException e)
 +        {
 +            assertAccessControlException("System.getProperty(\"foo.bar.baz\"); return 0d;", e);
 +        }
 +
 +        String[][] typesAndSources =
 +        {
 +        {"",                        "try { Class.forName(\"" + UDHelper.class.getName() + "\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
 +        {"sun.misc.Unsafe",         "sun.misc.Unsafe.getUnsafe(); return 0d;"},
 +        {"",                        "try { Class.forName(\"sun.misc.Unsafe\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
 +        {"java.nio.file.FileSystems", "try {" +
 +                                    "     java.nio.file.FileSystems.getDefault(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.nio.channels.FileChannel", "try {" +
 +                                    "     java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")).close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.nio.channels.SocketChannel", "try {" +
 +                                    "     java.nio.channels.SocketChannel.open().close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.io.FileInputStream", "try {" +
 +                                    "     new java.io.FileInputStream(\"./foobar\").close(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.lang.Runtime",       "try {" +
 +                                    "     java.lang.Runtime.getRuntime(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"org.apache.cassandra.service.StorageService",
 +                                    "try {" +
 +                                    "     org.apache.cassandra.service.StorageService v = org.apache.cassandra.service.StorageService.instance; v.isInShutdownHook(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.net.ServerSocket",   "try {" +
 +                                    "     new java.net.ServerSocket().bind(); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.io.FileOutputStream","try {" +
 +                                    "     new java.io.FileOutputStream(\".foo\"); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'},
 +        {"java.lang.Runtime",       "try {" +
 +                                    "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
 +                                    "} catch (Exception t) {" +
 +                                    "     throw new RuntimeException(t);" +
 +                                    '}'}
 +        };
 +
 +        for (String[] typeAndSource : typesAndSources)
 +        {
 +            assertInvalidMessage(typeAndSource[0] + " cannot be resolved",
 +                                 "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".invalid_class_access(val double) " +
 +                                 "RETURNS NULL ON NULL INPUT " +
 +                                 "RETURNS double " +
 +                                 "LANGUAGE JAVA\n" +
 +                                 "AS '" + typeAndSource[1] + "';");
 +        }
 +
 +        // JavaScript UDFs
 +
 +        try
 +        {
 +            String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                          "RETURNS NULL ON NULL INPUT " +
 +                                          "RETURNS double " +
 +                                          "LANGUAGE javascript\n" +
 +                                          "AS 'org.apache.cassandra.service.StorageService.instance.isInShutdownHook(); 0;';");
 +            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +            Assert.fail("Javascript security check failed");
 +        }
 +        catch (FunctionExecutionException e)
 +        {
 +            assertAccessControlException("", e);
 +        }
 +
 +        String[] javascript =
 +        {
 +        "java.lang.management.ManagmentFactory.getThreadMXBean(); 0;",
 +        "new java.io.FileInputStream(\"/tmp/foo\"); 0;",
 +        "new java.io.FileOutputStream(\"/tmp/foo\"); 0;",
 +        "java.nio.file.FileSystems.getDefault().createFileExclusively(\"./foo_bar_baz\"); 0;",
 +        "java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")); 0;",
 +        "java.nio.channels.SocketChannel.open(); 0;",
 +        "new java.net.ServerSocket().bind(null); 0;",
 +        "var thread = new java.lang.Thread(); thread.start(); 0;",
 +        "java.lang.System.getProperty(\"foo.bar.baz\"); 0;",
 +        "java.lang.Class.forName(\"java.lang.System\"); 0;",
 +        "java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); 0;",
 +        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
 +        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
 +        // TODO these (ugly) calls are still possible - these can consume CPU (as one could do with an evil loop, too)
 +//        "java.lang.Runtime.getRuntime().traceMethodCalls(true); 0;",
 +//        "java.lang.Runtime.getRuntime().gc(); 0;",
 +//        "java.lang.Runtime.getRuntime(); 0;",
 +        };
 +
 +        for (String script : javascript)
 +        {
 +            try
 +            {
 +                String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                              "RETURNS NULL ON NULL INPUT " +
 +                                              "RETURNS double " +
 +                                              "LANGUAGE javascript\n" +
 +                                              "AS '" + script + "';");
 +                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +                Assert.fail("Javascript security check failed: " + script);
 +            }
 +            catch (FunctionExecutionException e)
 +            {
 +                assertAccessControlException(script, e);
 +            }
 +        }
 +    }
 +
 +    private static void assertAccessControlException(String script, FunctionExecutionException e)
 +    {
 +        for (Throwable t = e; t != null && t != t.getCause(); t = t.getCause())
 +            if (t instanceof AccessControlException)
 +                return;
 +        Assert.fail("no AccessControlException for " + script + " (got " + e + ')');
 +    }
 +
 +    @Test
 +    public void testAmokUDF() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, dval double)");
 +        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
 +
 +        long udfWarnTimeout = DatabaseDescriptor.getUserDefinedFunctionWarnTimeout();
 +        long udfFailTimeout = DatabaseDescriptor.getUserDefinedFunctionFailTimeout();
 +        int maxTries = 5;
 +        for (int i = 1; i <= maxTries; i++)
 +        {
 +            try
 +            {
 +                // short timeout
 +                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(10);
 +                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(250);
 +                // don't kill the unit test... - default policy is "die"
 +                DatabaseDescriptor.setUserFunctionTimeoutPolicy(Config.UserFunctionTimeoutPolicy.ignore);
 +
-                 ClientWarn.captureWarnings();
++                ClientWarn.instance.captureWarnings();
 +                String fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                              "RETURNS NULL ON NULL INPUT " +
 +                                              "RETURNS double " +
 +                                              "LANGUAGE JAVA\n" +
 +                                              "AS 'long t=System.currentTimeMillis()+110; while (t>System.currentTimeMillis()) { }; return 0d;'");
 +                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
-                 List<String> warnings = ClientWarn.getWarnings();
++                List<String> warnings = ClientWarn.instance.getWarnings();
 +                Assert.assertNotNull(warnings);
 +                Assert.assertFalse(warnings.isEmpty());
-                 ClientWarn.resetWarnings();
++                ClientWarn.instance.resetWarnings();
 +
 +                // Java UDF
 +
 +                fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS double " +
 +                                       "LANGUAGE JAVA\n" +
 +                                       "AS 'long t=System.currentTimeMillis()+500; while (t>System.currentTimeMillis()) { }; return 0d;';");
 +                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +
 +                // Javascript UDF
 +
 +                fName = createFunction(KEYSPACE_PER_TEST, "double",
 +                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS double " +
 +                                       "LANGUAGE JAVASCRIPT\n" +
 +                                       "AS 'var t=java.lang.System.currentTimeMillis()+500; while (t>java.lang.System.currentTimeMillis()) { }; 0;';");
 +                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
 +
 +                return;
 +            }
 +            catch (Error | RuntimeException e)
 +            {
 +                if (i == maxTries)
 +                    throw e;
 +            }
 +            finally
 +            {
 +                // reset to defaults
 +                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(udfWarnTimeout);
 +                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(udfFailTimeout);
 +            }
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index c3f2629,d22a8f6..cf14d55
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@@ -23,8 -22,11 +23,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.cql3.QueryOptions;
  import org.apache.cassandra.cql3.CQLTester;
 +import org.apache.cassandra.cql3.QueryOptions;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Keyspace;
 -import org.apache.cassandra.dht.ByteOrderedPartitioner;
  import org.apache.cassandra.transport.Message;
  import org.apache.cassandra.transport.Server;
  import org.apache.cassandra.transport.SimpleClient;
@@@ -76,6 -80,62 +78,62 @@@ public class ClientWarningsTest extend
          }
      }
  
+     @Test
+     public void testTombstoneWarning() throws Exception
+     {
+         final int iterations = 10000;
+         createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+ 
+         try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+         {
+             client.connect(false);
+ 
+             for (int i = 0; i < iterations; i++)
+             {
+                 QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                     KEYSPACE,
+                                                                     currentTable(),
+                                                                     i), QueryOptions.DEFAULT);
+                 client.execute(query);
+             }
+             ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+             store.forceBlockingFlush();
+ 
+             for (int i = 0; i < iterations; i++)
+             {
+                 QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                     KEYSPACE,
+                                                                     currentTable(),
+                                                                     i), QueryOptions.DEFAULT);
+                 client.execute(query);
+             }
+             store.forceBlockingFlush();
+ 
+             {
+                 QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                     KEYSPACE,
+                                                                     currentTable()), QueryOptions.DEFAULT);
+                 Message.Response resp = client.execute(query);
+                 assertEquals(1, resp.getWarnings().size());
+             }
+         }
+     }
+ 
+     @Test
+     public void testLargeBatchWithProtoV2() throws Exception
+     {
+         createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");
+ 
 -        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_2))
++        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_3))
+         {
+             client.connect(false);
+ 
+             QueryMessage query = new QueryMessage(createBatchStatement(DatabaseDescriptor.getBatchSizeWarnThreshold()), QueryOptions.DEFAULT);
+             Message.Response resp = client.execute(query);
+             assertNull(resp.getWarnings());
+         }
+     }
+ 
      private String createBatchStatement(int minSize)
      {
          return String.format("BEGIN UNLOGGED BATCH INSERT INTO %s.%s (pk, v) VALUES (1, '%s') APPLY BATCH;",


[06/13] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ca...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.3
Commit: 94e7ef17757235cb35df70ff9a2a63e1d29d6c41
Parents: 0f995a2 dbf6e62
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 14:45:13 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:45:13 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 230 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 230 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cassandra/cql3/functions/UDFunction.java    |   2 +-
 .../cql3/statements/BatchStatement.java         |   9 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   7 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |   6 +-
 .../cassandra/service/ClientWarningsTest.java   |  58 +++++
 24 files changed, 545 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 614d5b4,6530956..a37ec99
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,21 @@@
 -2.2.5
 +3.0.3
 + * Fix AssertionError when removing from list using UPDATE (CASSANDRA-10954)
 + * Fix UnsupportedOperationException when reading old sstable with range
 +   tombstone (CASSANDRA-10743)
 + * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
 + * Fix potential assertion error during compaction (CASSANDRA-10944)
 + * Fix counting of received sstables in streaming (CASSANDRA-10949)
 + * Implement hints compression (CASSANDRA-9428)
 + * Fix potential assertion error when reading static columns (CASSANDRA-10903)
 + * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711)
 + * Avoid building PartitionUpdate in toString (CASSANDRA-10897)
 + * Reduce heap spent when receiving many SSTables (CASSANDRA-10797)
 + * Add back support for 3rd party auth providers to bulk loader (CASSANDRA-10873)
 + * Eliminate the dependency on jgrapht for UDT resolution (CASSANDRA-10653)
 + * (Hadoop) Close Clusters and Sessions in Hadoop Input/Output classes (CASSANDRA-10837)
 + * Fix sstableloader not working with upper case keyspace name (CASSANDRA-10806)
 +Merged from 2.2:
+  * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
   * Fix error streaming section more than 2GB (CASSANDRA-10961)
   * (cqlsh) Also apply --connect-timeout to control connection
     timeout (CASSANDRA-10959)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
index 0000000,088b43e..f47d8ac
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@@ -1,0 -1,229 +1,230 @@@
+ /*
+  * 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.concurrent;
+ 
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.TimeoutException;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import org.apache.cassandra.tracing.TraceState;
+ import org.apache.cassandra.tracing.Tracing;
+ import org.apache.cassandra.utils.concurrent.SimpleCondition;
+ import org.apache.cassandra.utils.JVMStabilityInspector;
+ 
+ import static org.apache.cassandra.tracing.Tracing.isTracing;
+ 
+ public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+ {
+     private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+ 
+     protected abstract void addTask(FutureTask<?> futureTask);
+     protected abstract void onCompletion();
+ 
+     /** Task Submission / Creation / Objects **/
+ 
+     public <T> FutureTask<T> submit(Callable<T> task)
+     {
+         return submit(newTaskFor(task));
+     }
+ 
+     public FutureTask<?> submit(Runnable task)
+     {
+         return submit(newTaskFor(task, null));
+     }
+ 
+     public <T> FutureTask<T> submit(Runnable task, T result)
+     {
+         return submit(newTaskFor(task, result));
+     }
+ 
+     public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+     {
+         throw new UnsupportedOperationException();
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+     {
+         return newTaskFor(runnable, result, ExecutorLocals.create());
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+     {
+         if (locals != null)
+         {
+             if (runnable instanceof LocalSessionFutureTask)
+                 return (LocalSessionFutureTask<T>) runnable;
+             return new LocalSessionFutureTask<T>(runnable, result, locals);
+         }
+         if (runnable instanceof FutureTask)
+             return (FutureTask<T>) runnable;
+         return new FutureTask<>(runnable, result);
+     }
+ 
+     protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+     {
+         if (isTracing())
+         {
+             if (callable instanceof LocalSessionFutureTask)
+                 return (LocalSessionFutureTask<T>) callable;
+             return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+         }
+         if (callable instanceof FutureTask)
+             return (FutureTask<T>) callable;
+         return new FutureTask<>(callable);
+     }
+ 
+     private class LocalSessionFutureTask<T> extends FutureTask<T>
+     {
+         private final ExecutorLocals locals;
+ 
+         public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+         {
+             super(callable);
+             this.locals = locals;
+         }
+ 
+         public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+         {
+             super(runnable, result);
+             this.locals = locals;
+         }
+ 
+         public void run()
+         {
+             ExecutorLocals old = ExecutorLocals.create();
+             ExecutorLocals.set(locals);
+             try
+             {
+                 super.run();
+             }
+             finally
+             {
+                 ExecutorLocals.set(old);
+             }
+         }
+     }
+ 
+     class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+     {
+         private boolean failure;
+         private Object result = this;
+         private final Callable<T> callable;
+ 
+         public FutureTask(Callable<T> callable)
+         {
+             this.callable = callable;
+         }
+         public FutureTask(Runnable runnable, T result)
+         {
+             this(Executors.callable(runnable, result));
+         }
+ 
+         public void run()
+         {
+             try
+             {
+                 result = callable.call();
+             }
+             catch (Throwable t)
+             {
+                 JVMStabilityInspector.inspectThrowable(t);
+                 logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                 result = t;
+                 failure = true;
+             }
+             finally
+             {
+                 signalAll();
+                 onCompletion();
+             }
+         }
+ 
+         public boolean cancel(boolean mayInterruptIfRunning)
+         {
+             return false;
+         }
+ 
+         public boolean isCancelled()
+         {
+             return false;
+         }
+ 
+         public boolean isDone()
+         {
+             return isSignaled();
+         }
+ 
+         public T get() throws InterruptedException, ExecutionException
+         {
+             await();
+             Object result = this.result;
+             if (failure)
+                 throw new ExecutionException((Throwable) result);
+             return (T) result;
+         }
+ 
+         public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+         {
 -            await(timeout, unit);
++            if (!await(timeout, unit))
++                throw new TimeoutException();
+             Object result = this.result;
+             if (failure)
+                 throw new ExecutionException((Throwable) result);
+             return (T) result;
+         }
+     }
+ 
+     private <T> FutureTask<T> submit(FutureTask<T> task)
+     {
+         addTask(task);
+         return task;
+     }
+ 
+     public void execute(Runnable command)
+     {
+         addTask(newTaskFor(command, ExecutorLocals.create()));
+     }
+ 
+     public void execute(Runnable command, ExecutorLocals locals)
+     {
+         addTask(newTaskFor(command, null, locals));
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 04a4c3d,1e5cea6..fa0d306
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@@ -269,164 -143,11 +269,164 @@@ public abstract class UDFunction extend
              return null;
  
          long tStart = System.nanoTime();
 -        ByteBuffer result = executeUserDefined(protocolVersion, parameters);
 -        Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 -        return result;
 +        parameters = makeEmptyParametersNull(parameters);
 +
 +        try
 +        {
 +            // Using async UDF execution is expensive (adds about 100us overhead per invocation on a Core-i7 MBPr).
 +            ByteBuffer result = DatabaseDescriptor.enableUserDefinedFunctionsThreads()
 +                                ? executeAsync(protocolVersion, parameters)
 +                                : executeUserDefined(protocolVersion, parameters);
 +
 +            Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
 +            return result;
 +        }
 +        catch (InvalidRequestException e)
 +        {
 +            throw e;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.trace("Invocation of user-defined function '{}' failed", this, t);
 +            if (t instanceof VirtualMachineError)
 +                throw (VirtualMachineError) t;
 +            throw FunctionExecutionException.create(this, t);
 +        }
      }
  
 +    public static void assertUdfsEnabled(String language)
 +    {
 +        if (!DatabaseDescriptor.enableUserDefinedFunctions())
 +            throw new InvalidRequestException("User-defined functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable");
 +        if (!"java".equalsIgnoreCase(language) && !DatabaseDescriptor.enableScriptedUserDefinedFunctions())
 +            throw new InvalidRequestException("Scripted user-defined functions are disabled in cassandra.yaml - set enable_scripted_user_defined_functions=true to enable if you are aware of the security risks");
 +    }
 +
 +    static void initializeThread()
 +    {
 +        // Get the TypeCodec stuff in Java Driver initialized.
 +        // This is to get the classes loaded outside of the restricted sandbox's security context of a UDF.
 +        UDHelper.codecFor(DataType.inet()).format(InetAddress.getLoopbackAddress());
 +        UDHelper.codecFor(DataType.ascii()).format("");
 +    }
 +
 +    private static final class ThreadIdAndCpuTime extends CompletableFuture<Object>
 +    {
 +        long threadId;
 +        long cpuTime;
 +
 +        ThreadIdAndCpuTime()
 +        {
 +            // Looks weird?
 +            // This call "just" links this class to java.lang.management - otherwise UDFs (script UDFs) might fail due to
 +            //      java.security.AccessControlException: access denied: ("java.lang.RuntimePermission" "accessClassInPackage.java.lang.management")
 +            // because class loading would be deferred until setup() is executed - but setup() is called with
 +            // limited privileges.
 +            threadMXBean.getCurrentThreadCpuTime();
 +        }
 +
 +        void setup()
 +        {
 +            this.threadId = Thread.currentThread().getId();
 +            this.cpuTime = threadMXBean.getCurrentThreadCpuTime();
 +            complete(null);
 +        }
 +    }
 +
 +    private ByteBuffer executeAsync(int protocolVersion, List<ByteBuffer> parameters)
 +    {
 +        ThreadIdAndCpuTime threadIdAndCpuTime = new ThreadIdAndCpuTime();
 +
 +        Future<ByteBuffer> future = executor().submit(() -> {
 +            threadIdAndCpuTime.setup();
 +            return executeUserDefined(protocolVersion, parameters);
 +        });
 +
 +        try
 +        {
 +            if (DatabaseDescriptor.getUserDefinedFunctionWarnTimeout() > 0)
 +                try
 +                {
 +                    return future.get(DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +                }
 +                catch (TimeoutException e)
 +                {
 +
 +                    // log and emit a warning that UDF execution took long
 +                    String warn = String.format("User defined function %s ran longer than %dms", this, DatabaseDescriptor.getUserDefinedFunctionWarnTimeout());
 +                    logger.warn(warn);
-                     ClientWarn.warn(warn);
++                    ClientWarn.instance.warn(warn);
 +                }
 +
 +            // retry with difference of warn-timeout to fail-timeout
 +            return future.get(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - DatabaseDescriptor.getUserDefinedFunctionWarnTimeout(), TimeUnit.MILLISECONDS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            Thread.currentThread().interrupt();
 +            throw new RuntimeException(e);
 +        }
 +        catch (ExecutionException e)
 +        {
 +            Throwable c = e.getCause();
 +            if (c instanceof RuntimeException)
 +                throw (RuntimeException) c;
 +            throw new RuntimeException(c);
 +        }
 +        catch (TimeoutException e)
 +        {
 +            // retry a last time with the difference of UDF-fail-timeout to consumed CPU time (just in case execution hit a badly timed GC)
 +            try
 +            {
 +                //The threadIdAndCpuTime shouldn't take a long time to be set so this should return immediately
 +                threadIdAndCpuTime.get(1, TimeUnit.SECONDS);
 +
 +                long cpuTimeMillis = threadMXBean.getThreadCpuTime(threadIdAndCpuTime.threadId) - threadIdAndCpuTime.cpuTime;
 +                cpuTimeMillis /= 1000000L;
 +
 +                return future.get(Math.max(DatabaseDescriptor.getUserDefinedFunctionFailTimeout() - cpuTimeMillis, 0L),
 +                                  TimeUnit.MILLISECONDS);
 +            }
 +            catch (InterruptedException e1)
 +            {
 +                Thread.currentThread().interrupt();
 +                throw new RuntimeException(e);
 +            }
 +            catch (ExecutionException e1)
 +            {
 +                Throwable c = e.getCause();
 +                if (c instanceof RuntimeException)
 +                    throw (RuntimeException) c;
 +                throw new RuntimeException(c);
 +            }
 +            catch (TimeoutException e1)
 +            {
 +                TimeoutException cause = new TimeoutException(String.format("User defined function %s ran longer than %dms%s",
 +                                                                            this,
 +                                                                            DatabaseDescriptor.getUserDefinedFunctionFailTimeout(),
 +                                                                            DatabaseDescriptor.getUserFunctionTimeoutPolicy() == Config.UserFunctionTimeoutPolicy.ignore
 +                                                                            ? "" : " - will stop Cassandra VM"));
 +                FunctionExecutionException fe = FunctionExecutionException.create(this, cause);
 +                JVMStabilityInspector.userFunctionTimeout(cause);
 +                throw fe;
 +            }
 +        }
 +    }
 +
 +    private List<ByteBuffer> makeEmptyParametersNull(List<ByteBuffer> parameters)
 +    {
 +        List<ByteBuffer> r = new ArrayList<>(parameters.size());
 +        for (int i = 0; i < parameters.size(); i++)
 +        {
 +            ByteBuffer param = parameters.get(i);
 +            r.add(UDHelper.isNullOrEmpty(argTypes.get(i), param)
 +                  ? null : param);
 +        }
 +        return r;
 +    }
 +
 +    protected abstract ExecutorService executor();
 +
      public boolean isCallableWrtNullable(List<ByteBuffer> parameters)
      {
          if (!calledOnNullInput)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 3979597,a289ad1..47396fb
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@@ -226,35 -179,67 +226,35 @@@ public class BatchStatement implements 
          for (int i = 0; i < statements.size(); i++)
          {
              ModificationStatement statement = statements.get(i);
 +            if (isLogged() && statement.cfm.params.gcGraceSeconds == 0)
 +            {
 +                if (tablesWithZeroGcGs == null)
 +                    tablesWithZeroGcGs = new HashSet<>();
 +                tablesWithZeroGcGs.add(String.format("%s.%s", statement.cfm.ksName, statement.cfm.cfName));
 +            }
              QueryOptions statementOptions = options.forStatement(i);
              long timestamp = attrs.getTimestamp(now, statementOptions);
 -            addStatementMutations(statement, statementOptions, local, timestamp, mutations);
 +            statement.addUpdates(collector, statementOptions, local, timestamp);
          }
 -        return unzipMutations(mutations);
 -    }
 -
 -    private Collection<? extends IMutation> unzipMutations(Map<String, Map<ByteBuffer, IMutation>> mutations)
 -    {
 -
 -        // The case where all statement where on the same keyspace is pretty common
 -        if (mutations.size() == 1)
 -            return mutations.values().iterator().next().values();
 -
  
 -        List<IMutation> ms = new ArrayList<>();
 -        for (Map<ByteBuffer, IMutation> ksMap : mutations.values())
 -            ms.addAll(ksMap.values());
 -
 -        return ms;
 -    }
 -
 -    private void addStatementMutations(ModificationStatement statement,
 -                                       QueryOptions options,
 -                                       boolean local,
 -                                       long now,
 -                                       Map<String, Map<ByteBuffer, IMutation>> mutations)
 -    throws RequestExecutionException, RequestValidationException
 -    {
 -        String ksName = statement.keyspace();
 -        Map<ByteBuffer, IMutation> ksMap = mutations.get(ksName);
 -        if (ksMap == null)
 +        if (tablesWithZeroGcGs != null)
          {
 -            ksMap = new HashMap<>();
 -            mutations.put(ksName, ksMap);
 +            String suffix = tablesWithZeroGcGs.size() == 1 ? "" : "s";
 +            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, LOGGED_BATCH_LOW_GCGS_WARNING,
 +                             suffix, tablesWithZeroGcGs);
-             ClientWarn.warn(MessageFormatter.arrayFormat(LOGGED_BATCH_LOW_GCGS_WARNING, new Object[] { suffix, tablesWithZeroGcGs })
-                                             .getMessage());
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(LOGGED_BATCH_LOW_GCGS_WARNING, new Object[] { suffix, tablesWithZeroGcGs })
++                                                     .getMessage());
          }
  
 -        // The following does the same than statement.getMutations(), but we inline it here because
 -        // we don't want to recreate mutations every time as this is particularly inefficient when applying
 -        // multiple batch to the same partition (see #6737).
 -        List<ByteBuffer> keys = statement.buildPartitionKeyNames(options);
 -        Composite clusteringPrefix = statement.createClusteringPrefix(options);
 -        UpdateParameters params = statement.makeUpdateParameters(keys, clusteringPrefix, options, local, now);
 -
 -        for (ByteBuffer key : keys)
 -        {
 -            IMutation mutation = ksMap.get(key);
 -            Mutation mut;
 -            if (mutation == null)
 -            {
 -                mut = new Mutation(ksName, key);
 -                mutation = statement.cfm.isCounter() ? new CounterMutation(mut, options.getConsistency()) : mut;
 -                ksMap.put(key, mutation);
 -            }
 -            else
 -            {
 -                mut = statement.cfm.isCounter() ? ((CounterMutation) mutation).getMutation() : (Mutation) mutation;
 -            }
 +        collector.validateIndexedColumns();
 +        return collector.toMutations();
 +    }
  
 -            statement.addUpdateForKey(mut.addOrGet(statement.cfm), key, clusteringPrefix, params);
 -        }
 +    private int updatedRows()
 +    {
 +        // Note: it's possible for 2 statements to actually apply to the same row, but that's just an estimation
 +        // for sizing our PartitionUpdate backing array, so it's good enough.
 +        return statements.size();
      }
  
      /**
@@@ -286,9 -271,9 +286,9 @@@
              }
              else if (logger.isWarnEnabled())
              {
 -                logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
 +                logger.warn(format, tableNames, size, warnThreshold, size - warnThreshold, "");
              }
-             ClientWarn.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage());
 -            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage());
          }
      }
  
@@@ -311,17 -298,21 +311,16 @@@
              }
  
              // CASSANDRA-9303: If we only have local mutations we do not warn
 -            if (localMutationsOnly)
 +            if (localPartitionsOnly)
                  return;
  
 -            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, unloggedBatchWarning,
 -                             keySet.size(), keySet.size() == 1 ? "" : "s",
 -                             ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
  
 -            ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
 -                                                                  new Object[]{
 -                                                                              keySet.size(),
 -                                                                              keySet.size() == 1 ? "" : "s",
 -                                                                              ksCfPairs.size() == 1 ? "" : "s",
 -                                                                              ksCfPairs
 -                                                                  }).getMessage());
 +            NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, UNLOGGED_BATCH_WARNING,
 +                             keySet.size(), keySet.size() == 1 ? "" : "s",
 +                             tableNames.size() == 1 ? "" : "s", tableNames);
  
-             ClientWarn.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
++            ClientWarn.instance.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), keySet.size() == 1 ? "" : "s",
 +                                                    tableNames.size() == 1 ? "" : "s", tableNames}).getMessage());
- 
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 4017ce6,0000000..5af4887
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@@ -1,330 -1,0 +1,330 @@@
 +/*
 + * 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.cql3.statements;
 +
 +import java.util.*;
 +import java.util.stream.Collectors;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.auth.Permission;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.ViewDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 +import org.apache.cassandra.cql3.selection.RawSelector;
 +import org.apache.cassandra.cql3.selection.Selectable;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.ReversedType;
 +import org.apache.cassandra.db.view.View;
 +import org.apache.cassandra.exceptions.AlreadyExistsException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.exceptions.RequestValidationException;
 +import org.apache.cassandra.exceptions.UnauthorizedException;
 +import org.apache.cassandra.schema.TableParams;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.ClientWarn;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.thrift.ThriftValidation;
 +import org.apache.cassandra.transport.Event;
 +
 +public class CreateViewStatement extends SchemaAlteringStatement
 +{
 +    private final CFName baseName;
 +    private final List<RawSelector> selectClause;
 +    private final WhereClause whereClause;
 +    private final List<ColumnIdentifier.Raw> partitionKeys;
 +    private final List<ColumnIdentifier.Raw> clusteringKeys;
 +    public final CFProperties properties = new CFProperties();
 +    private final boolean ifNotExists;
 +
 +    public CreateViewStatement(CFName viewName,
 +                               CFName baseName,
 +                               List<RawSelector> selectClause,
 +                               WhereClause whereClause,
 +                               List<ColumnIdentifier.Raw> partitionKeys,
 +                               List<ColumnIdentifier.Raw> clusteringKeys,
 +                               boolean ifNotExists)
 +    {
 +        super(viewName);
 +        this.baseName = baseName;
 +        this.selectClause = selectClause;
 +        this.whereClause = whereClause;
 +        this.partitionKeys = partitionKeys;
 +        this.clusteringKeys = clusteringKeys;
 +        this.ifNotExists = ifNotExists;
 +    }
 +
 +
 +    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
 +    {
 +        if (!baseName.hasKeyspace())
 +            baseName.setKeyspace(keyspace(), true);
 +        state.hasColumnFamilyAccess(keyspace(), baseName.getColumnFamily(), Permission.ALTER);
 +    }
 +
 +    public void validate(ClientState state) throws RequestValidationException
 +    {
 +        // We do validation in announceMigration to reduce doubling up of work
 +    }
 +
 +    private interface AddColumn {
 +        void add(ColumnIdentifier identifier, AbstractType<?> type);
 +    }
 +
 +    private void add(CFMetaData baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
 +    {
 +        for (ColumnIdentifier column : columns)
 +        {
 +            AbstractType<?> type = baseCfm.getColumnDefinition(column).type;
 +            if (properties.definedOrdering.containsKey(column))
 +            {
 +                boolean desc = properties.definedOrdering.get(column);
 +                if (!desc && type.isReversed())
 +                {
 +                    type = ((ReversedType)type).baseType;
 +                }
 +                else if (desc && !type.isReversed())
 +                {
 +                    type = ReversedType.getInstance(type);
 +                }
 +            }
 +            adder.add(column, type);
 +        }
 +    }
 +
 +    public Event.SchemaChange announceMigration(boolean isLocalOnly) throws RequestValidationException
 +    {
 +        // We need to make sure that:
 +        //  - primary key includes all columns in base table's primary key
 +        //  - make sure that the select statement does not have anything other than columns
 +        //    and their names match the base table's names
 +        //  - make sure that primary key does not include any collections
 +        //  - make sure there is no where clause in the select statement
 +        //  - make sure there is not currently a table or view
 +        //  - make sure baseTable gcGraceSeconds > 0
 +
 +        properties.validate();
 +
 +        if (properties.useCompactStorage)
 +            throw new InvalidRequestException("Cannot use 'COMPACT STORAGE' when defining a materialized view");
 +
 +        // We enforce the keyspace because if the RF is different, the logic to wait for a
 +        // specific replica would break
 +        if (!baseName.getKeyspace().equals(keyspace()))
 +            throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
 +
 +        CFMetaData cfm = ThriftValidation.validateColumnFamily(baseName.getKeyspace(), baseName.getColumnFamily());
 +
 +        if (cfm.isCounter())
 +            throw new InvalidRequestException("Materialized views are not supported on counter tables");
 +        if (cfm.isView())
 +            throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
 +
 +        if (cfm.params.gcGraceSeconds == 0)
 +        {
 +            throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
 +                                                            "'%s' with gc_grace_seconds of 0, since this value is " +
 +                                                            "used to TTL undelivered updates. Setting gc_grace_seconds" +
 +                                                            " too low might cause undelivered updates to expire " +
 +                                                            "before being replayed.", cfName.getColumnFamily(),
 +                                                            baseName.getColumnFamily()));
 +        }
 +
 +        Set<ColumnIdentifier> included = new HashSet<>();
 +        for (RawSelector selector : selectClause)
 +        {
 +            Selectable.Raw selectable = selector.selectable;
 +            if (selectable instanceof Selectable.WithFieldSelection.Raw)
 +                throw new InvalidRequestException("Cannot select out a part of type when defining a materialized view");
 +            if (selectable instanceof Selectable.WithFunction.Raw)
 +                throw new InvalidRequestException("Cannot use function when defining a materialized view");
 +            if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
 +                throw new InvalidRequestException("Cannot use function when defining a materialized view");
 +            ColumnIdentifier identifier = (ColumnIdentifier) selectable.prepare(cfm);
 +            if (selector.alias != null)
 +                throw new InvalidRequestException(String.format("Cannot alias column '%s' as '%s' when defining a materialized view", identifier.toString(), selector.alias.toString()));
 +
 +            ColumnDefinition cdef = cfm.getColumnDefinition(identifier);
 +
 +            if (cdef == null)
 +                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
 +
 +            if (cdef.isStatic())
-                 ClientWarn.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
++                ClientWarn.instance.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
 +            else
 +                included.add(identifier);
 +        }
 +
 +        Set<ColumnIdentifier.Raw> targetPrimaryKeys = new HashSet<>();
 +        for (ColumnIdentifier.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
 +        {
 +            if (!targetPrimaryKeys.add(identifier))
 +                throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
 +
 +            ColumnDefinition cdef = cfm.getColumnDefinition(identifier.prepare(cfm));
 +
 +            if (cdef == null)
 +                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
 +
 +            if (cfm.getColumnDefinition(identifier.prepare(cfm)).type.isMultiCell())
 +                throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
 +
 +            if (cdef.isStatic())
 +                throw new InvalidRequestException(String.format("Cannot use Static column '%s' in PRIMARY KEY of materialized view", identifier));
 +        }
 +
 +        // build the select statement
 +        Map<ColumnIdentifier.Raw, Boolean> orderings = Collections.emptyMap();
 +        SelectStatement.Parameters parameters = new SelectStatement.Parameters(orderings, false, true, false);
 +        SelectStatement.RawStatement rawSelect = new SelectStatement.RawStatement(baseName, parameters, selectClause, whereClause, null);
 +
 +        ClientState state = ClientState.forInternalCalls();
 +        state.setKeyspace(keyspace());
 +
 +        rawSelect.prepareKeyspace(state);
 +        rawSelect.setBoundVariables(getBoundVariables());
 +
 +        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
 +        SelectStatement select = (SelectStatement) prepared.statement;
 +        StatementRestrictions restrictions = select.getRestrictions();
 +
 +        if (!prepared.boundNames.isEmpty())
 +            throw new InvalidRequestException("Cannot use query parameters in CREATE MATERIALIZED VIEW statements");
 +
 +        if (!restrictions.nonPKRestrictedColumns(false).isEmpty())
 +        {
 +            throw new InvalidRequestException(String.format(
 +                    "Non-primary key columns cannot be restricted in the SELECT statement used for materialized view " +
 +                    "creation (got restrictions on: %s)",
 +                    restrictions.nonPKRestrictedColumns(false).stream().map(def -> def.name.toString()).collect(Collectors.joining(", "))));
 +        }
 +
 +        String whereClauseText = View.relationsToWhereClause(whereClause.relations);
 +
 +        Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
 +        for (ColumnDefinition definition : Iterables.concat(cfm.partitionKeyColumns(), cfm.clusteringColumns()))
 +            basePrimaryKeyCols.add(definition.name);
 +
 +        List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
 +        List<ColumnIdentifier> targetPartitionKeys = new ArrayList<>();
 +
 +        // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
 +        boolean hasNonPKColumn = false;
 +        for (ColumnIdentifier.Raw raw : partitionKeys)
 +            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 +
 +        for (ColumnIdentifier.Raw raw : clusteringKeys)
 +            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 +
 +        // We need to include all of the primary key columns from the base table in order to make sure that we do not
 +        // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
 +        // the view because if we need to generate a tombstone, we have no way of knowing which value is currently being
 +        // used in the view and whether or not to generate a tombstone. In order to not surprise our users, we require
 +        // that they include all of the columns. We provide them with a list of all of the columns left to include.
 +        boolean missingClusteringColumns = false;
 +        StringBuilder columnNames = new StringBuilder();
 +        List<ColumnIdentifier> includedColumns = new ArrayList<>();
 +        for (ColumnDefinition def : cfm.allColumns())
 +        {
 +            ColumnIdentifier identifier = def.name;
 +
 +            if ((included.isEmpty() || included.contains(identifier))
 +                && !targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier)
 +                && !def.isStatic())
 +            {
 +                includedColumns.add(identifier);
 +            }
 +            if (!def.isPrimaryKeyColumn()) continue;
 +
 +            if (!targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier))
 +            {
 +                if (missingClusteringColumns)
 +                    columnNames.append(',');
 +                else
 +                    missingClusteringColumns = true;
 +                columnNames.append(identifier);
 +            }
 +        }
 +        if (missingClusteringColumns)
 +            throw new InvalidRequestException(String.format("Cannot create Materialized View %s without primary key columns from base %s (%s)",
 +                                                            columnFamily(), baseName.getColumnFamily(), columnNames.toString()));
 +
 +        if (targetPartitionKeys.isEmpty())
 +            throw new InvalidRequestException("Must select at least a column for a Materialized View");
 +
 +        if (targetClusteringColumns.isEmpty())
 +            throw new InvalidRequestException("No columns are defined for Materialized View other than primary key");
 +
 +        CFMetaData.Builder cfmBuilder = CFMetaData.Builder.createView(keyspace(), columnFamily());
 +        add(cfm, targetPartitionKeys, cfmBuilder::addPartitionKey);
 +        add(cfm, targetClusteringColumns, cfmBuilder::addClusteringColumn);
 +        add(cfm, includedColumns, cfmBuilder::addRegularColumn);
 +        cfmBuilder.withId(properties.properties.getId());
 +        TableParams params = properties.properties.asNewTableParams();
 +        CFMetaData viewCfm = cfmBuilder.build().params(params);
 +        ViewDefinition definition = new ViewDefinition(keyspace(),
 +                                                       columnFamily(),
 +                                                       Schema.instance.getId(keyspace(), baseName.getColumnFamily()),
 +                                                       baseName.getColumnFamily(),
 +                                                       included.isEmpty(),
 +                                                       rawSelect,
 +                                                       whereClauseText,
 +                                                       viewCfm);
 +
 +        try
 +        {
 +            MigrationManager.announceNewView(definition, isLocalOnly);
 +            return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
 +        }
 +        catch (AlreadyExistsException e)
 +        {
 +            if (ifNotExists)
 +                return null;
 +            throw e;
 +        }
 +    }
 +
 +    private static boolean getColumnIdentifier(CFMetaData cfm,
 +                                               Set<ColumnIdentifier> basePK,
 +                                               boolean hasNonPKColumn,
 +                                               ColumnIdentifier.Raw raw,
 +                                               List<ColumnIdentifier> columns,
 +                                               StatementRestrictions restrictions)
 +    {
 +        ColumnIdentifier identifier = raw.prepare(cfm);
 +        ColumnDefinition def = cfm.getColumnDefinition(identifier);
 +
 +        boolean isPk = basePK.contains(identifier);
 +        if (!isPk && hasNonPKColumn)
 +            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", identifier));
 +
 +        // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
 +        // because we will never allow a single partition key to be NULL
 +        boolean isSinglePartitionKey = cfm.getColumnDefinition(identifier).isPartitionKey()
 +                                       && cfm.partitionKeyColumns().size() == 1;
 +        if (!isSinglePartitionKey && !restrictions.isRestricted(def))
 +            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", identifier));
 +
 +        columns.add(identifier);
 +        return !isPk;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index a9bb121,5cfa94b..904adca
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -359,21 -256,19 +359,21 @@@ public class SelectStatement implement
          else if (restrictions.keyIsInRelation())
          {
              logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
-             ClientWarn.warn("Aggregation query used on multiple partition keys (IN restriction)");
+             ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
          }
  
 -        Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);
 +        Selection.ResultSetBuilder result = selection.resultSetBuilder(parameters.isJson);
          while (!pager.isExhausted())
          {
 -            for (Row row : pager.fetchPage(pageSize))
 +            try (PartitionIterator iter = pager.fetchPage(pageSize))
              {
 -                // Not columns match the query, skip
 -                if (row.cf == null)
 -                    continue;
 -
 -                processColumnFamily(row.key.getKey(), row.cf, options, now, result);
 +                while (iter.hasNext())
 +                {
 +                    try (RowIterator partition = iter.next())
 +                    {
 +                        processPartition(partition, options, result, nowInSec);
 +                    }
 +                }
              }
          }
          return new ResultMessage.Rows(result.build(options.getProtocolVersion()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index 3f0695c,cd86336..668a189
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -232,713 -95,55 +232,713 @@@ public abstract class ReadCommand imple
          return this;
      }
  
 -    public String getColumnFamilyName()
 +    /**
 +     * Sets the digest version, for when digest for that command is requested.
 +     * <p>
 +     * Note that we allow setting this independently of setting the command as a digest query as
 +     * this allows us to use the command as a carrier of the digest version even if we only call
 +     * setIsDigestQuery on some copy of it.
 +     *
 +     * @param digestVersion the version for the digest is this command is used for digest query..
 +     * @return this read command.
 +     */
 +    public ReadCommand setDigestVersion(int digestVersion)
      {
 -        return cfName;
 +        this.digestVersion = digestVersion;
 +        return this;
      }
  
 +    /**
 +     * Whether this query is for thrift or not.
 +     *
 +     * @return whether this query is for thrift.
 +     */
 +    public boolean isForThrift()
 +    {
 +        return isForThrift;
 +    }
 +
 +    /**
 +     * The clustering index filter this command to use for the provided key.
 +     * <p>
 +     * Note that that method should only be called on a key actually queried by this command
 +     * and in practice, this will almost always return the same filter, but for the sake of
 +     * paging, the filter on the first key of a range command might be slightly different.
 +     *
 +     * @param key a partition key queried by this command.
 +     *
 +     * @return the {@code ClusteringIndexFilter} to use for the partition of key {@code key}.
 +     */
 +    public abstract ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key);
 +
 +    /**
 +     * Returns a copy of this command.
 +     *
 +     * @return a copy of this command.
 +     */
      public abstract ReadCommand copy();
  
 -    public abstract Row getRow(Keyspace keyspace);
 +    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 +
 +    protected abstract int oldestUnrepairedTombstone();
 +
 +    public ReadResponse createResponse(UnfilteredPartitionIterator iterator, ColumnFilter selection)
 +    {
 +        return isDigestQuery()
 +             ? ReadResponse.createDigestResponse(iterator, digestVersion)
 +             : ReadResponse.createDataResponse(iterator, selection);
 +    }
 +
 +    public long indexSerializedSize(int version)
 +    {
 +        if (index.isPresent())
 +            return IndexMetadata.serializer.serializedSize(index.get(), version);
 +        else
 +            return 0;
 +    }
 +
 +    public Index getIndex(ColumnFamilyStore cfs)
 +    {
 +        // if we've already consulted the index manager, and it returned a valid index
 +        // the result should be cached here.
 +        if(index.isPresent())
 +            return cfs.indexManager.getIndex(index.get());
 +
 +        // if no cached index is present, but we've already consulted the index manager
 +        // then no registered index is suitable for this command, so just return null.
 +        if (indexManagerQueried)
 +            return null;
 +
 +        // do the lookup, set the flag to indicate so and cache the result if not null
 +        Index selected = cfs.indexManager.getBestIndexFor(this);
 +        indexManagerQueried = true;
  
 -    public abstract IDiskAtomFilter filter();
 +        if (selected == null)
 +            return null;
  
 -    public String getKeyspace()
 +        index = Optional.of(selected.getIndexMetadata());
 +        return selected;
 +    }
 +
 +    /**
 +     * Executes this command on the local host.
 +     *
 +     * @param orderGroup the operation group spanning this command
 +     *
 +     * @return an iterator over the result of executing this command locally.
 +     */
 +    @SuppressWarnings("resource") // The result iterator is closed upon exceptions (we know it's fine to potentially not close the intermediary
 +                                  // iterators created inside the try as long as we do close the original resultIterator), or by closing the result.
 +    public UnfilteredPartitionIterator executeLocally(ReadOrderGroup orderGroup)
      {
 -        return ksName;
 +        long startTimeNanos = System.nanoTime();
 +
 +        ColumnFamilyStore cfs = Keyspace.openAndGetStore(metadata());
 +        Index index = getIndex(cfs);
 +
 +        Index.Searcher searcher = null;
 +        if (index != null)
 +        {
 +            if (!cfs.indexManager.isIndexQueryable(index))
 +                throw new IndexNotAvailableException(index);
 +
 +            searcher = index.searcherFor(this);
 +            Tracing.trace("Executing read on {}.{} using index {}", cfs.metadata.ksName, cfs.metadata.cfName, index.getIndexMetadata().name);
 +        }
 +
 +        UnfilteredPartitionIterator resultIterator = searcher == null
 +                                         ? queryStorage(cfs, orderGroup)
 +                                         : searcher.search(orderGroup);
 +
 +        try
 +        {
 +            resultIterator = withMetricsRecording(withoutPurgeableTombstones(resultIterator, cfs), cfs.metric, startTimeNanos);
 +
 +            // If we've used a 2ndary index, we know the result already satisfy the primary expression used, so
 +            // no point in checking it again.
 +            RowFilter updatedFilter = searcher == null
 +                                    ? rowFilter()
 +                                    : index.getPostIndexQueryFilter(rowFilter());
 +
 +            // TODO: We'll currently do filtering by the rowFilter here because it's convenient. However,
 +            // we'll probably want to optimize by pushing it down the layer (like for dropped columns) as it
 +            // would be more efficient (the sooner we discard stuff we know we don't care, the less useless
 +            // processing we do on it).
 +            return limits().filter(updatedFilter.filter(resultIterator, nowInSec()), nowInSec());
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            resultIterator.close();
 +            throw e;
 +        }
      }
  
 -    // maybeGenerateRetryCommand is used to generate a retry for short reads
 -    public ReadCommand maybeGenerateRetryCommand(RowDataResolver resolver, Row row)
 +    protected abstract void recordLatency(TableMetrics metric, long latencyNanos);
 +
 +    public PartitionIterator executeInternal(ReadOrderGroup orderGroup)
      {
 -        return null;
 +        return UnfilteredPartitionIterators.filter(executeLocally(orderGroup), nowInSec());
      }
  
 -    // maybeTrim removes columns from a response that is too long
 -    public Row maybeTrim(Row row)
 +    public ReadOrderGroup startOrderGroup()
      {
 -        return row;
 +        return ReadOrderGroup.forCommand(this);
      }
  
 -    public long getTimeout()
 +    /**
 +     * Wraps the provided iterator so that metrics on what is scanned by the command are recorded.
 +     * This also log warning/trow TombstoneOverwhelmingException if appropriate.
 +     */
 +    private UnfilteredPartitionIterator withMetricsRecording(UnfilteredPartitionIterator iter, final TableMetrics metric, final long startTimeNanos)
      {
 -        return DatabaseDescriptor.getReadRpcTimeout();
 +        class MetricRecording extends Transformation<UnfilteredRowIterator>
 +        {
 +            private final int failureThreshold = DatabaseDescriptor.getTombstoneFailureThreshold();
 +            private final int warningThreshold = DatabaseDescriptor.getTombstoneWarnThreshold();
 +
 +            private final boolean respectTombstoneThresholds = !Schema.isSystemKeyspace(ReadCommand.this.metadata().ksName);
 +
 +            private int liveRows = 0;
 +            private int tombstones = 0;
 +
 +            private DecoratedKey currentKey;
 +
 +            @Override
 +            public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator iter)
 +            {
 +                currentKey = iter.partitionKey();
 +                return Transformation.apply(iter, this);
 +            }
 +
 +            @Override
 +            public Row applyToStatic(Row row)
 +            {
 +                return applyToRow(row);
 +            }
 +
 +            @Override
 +            public Row applyToRow(Row row)
 +            {
 +                if (row.hasLiveData(ReadCommand.this.nowInSec()))
 +                    ++liveRows;
 +
 +                for (Cell cell : row.cells())
 +                {
 +                    if (!cell.isLive(ReadCommand.this.nowInSec()))
 +                        countTombstone(row.clustering());
 +                }
 +                return row;
 +            }
 +
 +            @Override
 +            public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +            {
 +                countTombstone(marker.clustering());
 +                return marker;
 +            }
 +
 +            private void countTombstone(ClusteringPrefix clustering)
 +            {
 +                ++tombstones;
 +                if (tombstones > failureThreshold && respectTombstoneThresholds)
 +                {
 +                    String query = ReadCommand.this.toCQLString();
 +                    Tracing.trace("Scanned over {} tombstones for query {}; query aborted (see tombstone_failure_threshold)", failureThreshold, query);
 +                    throw new TombstoneOverwhelmingException(tombstones, query, ReadCommand.this.metadata(), currentKey, clustering);
 +                }
 +            }
 +
 +            @Override
 +            public void onClose()
 +            {
 +                recordLatency(metric, System.nanoTime() - startTimeNanos);
 +
 +                metric.tombstoneScannedHistogram.update(tombstones);
 +                metric.liveScannedHistogram.update(liveRows);
 +
 +                boolean warnTombstones = tombstones > warningThreshold && respectTombstoneThresholds;
 +                if (warnTombstones)
 +                {
 +                    String msg = String.format("Read %d live rows and %d tombstone cells for query %1.512s (see tombstone_warn_threshold)", liveRows, tombstones, ReadCommand.this.toCQLString());
-                     ClientWarn.warn(msg);
++                    ClientWarn.instance.warn(msg);
 +                    logger.warn(msg);
 +                }
 +
 +                Tracing.trace("Read {} live and {} tombstone cells{}", liveRows, tombstones, (warnTombstones ? " (see tombstone_warn_threshold)" : ""));
 +            }
 +        };
 +
 +        return Transformation.apply(iter, new MetricRecording());
      }
 -}
  
 -class ReadCommandSerializer implements IVersionedSerializer<ReadCommand>
 -{
 -    public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +    /**
 +     * Creates a message for this command.
 +     */
 +    public abstract MessageOut<ReadCommand> createMessage(int version);
 +
 +    protected abstract void appendCQLWhereClause(StringBuilder sb);
 +
 +    // Skip purgeable tombstones. We do this because it's safe to do (post-merge of the memtable and sstable at least), it
 +    // can save us some bandwith, and avoid making us throw a TombstoneOverwhelmingException for purgeable tombstones (which
 +    // are to some extend an artefact of compaction lagging behind and hence counting them is somewhat unintuitive).
 +    protected UnfilteredPartitionIterator withoutPurgeableTombstones(UnfilteredPartitionIterator iterator, ColumnFamilyStore cfs)
 +    {
 +        final boolean isForThrift = iterator.isForThrift();
 +        class WithoutPurgeableTombstones extends PurgeFunction
 +        {
 +            public WithoutPurgeableTombstones()
 +            {
 +                super(isForThrift, nowInSec(), cfs.gcBefore(nowInSec()), oldestUnrepairedTombstone(), cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
 +            }
 +
 +            protected long getMaxPurgeableTimestamp()
 +            {
 +                return Long.MAX_VALUE;
 +            }
 +        }
 +        return Transformation.apply(iterator, new WithoutPurgeableTombstones());
 +    }
 +
 +    /**
 +     * Recreate the CQL string corresponding to this query.
 +     * <p>
 +     * Note that in general the returned string will not be exactly the original user string, first
 +     * because there isn't always a single syntax for a given query,  but also because we don't have
 +     * all the information needed (we know the non-PK columns queried but not the PK ones as internally
 +     * we query them all). So this shouldn't be relied too strongly, but this should be good enough for
 +     * debugging purpose which is what this is for.
 +     */
 +    public String toCQLString()
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        sb.append("SELECT ").append(columnFilter());
 +        sb.append(" FROM ").append(metadata().ksName).append('.').append(metadata.cfName);
 +        appendCQLWhereClause(sb);
 +
 +        if (limits() != DataLimits.NONE)
 +            sb.append(' ').append(limits());
 +        return sb.toString();
 +    }
 +
 +    private static class Serializer implements IVersionedSerializer<ReadCommand>
 +    {
 +        private static int digestFlag(boolean isDigest)
 +        {
 +            return isDigest ? 0x01 : 0;
 +        }
 +
 +        private static boolean isDigest(int flags)
 +        {
 +            return (flags & 0x01) != 0;
 +        }
 +
 +        private static int thriftFlag(boolean isForThrift)
 +        {
 +            return isForThrift ? 0x02 : 0;
 +        }
 +
 +        private static boolean isForThrift(int flags)
 +        {
 +            return (flags & 0x02) != 0;
 +        }
 +
 +        private static int indexFlag(boolean hasIndex)
 +        {
 +            return hasIndex ? 0x04 : 0;
 +        }
 +
 +        private static boolean hasIndex(int flags)
 +        {
 +            return (flags & 0x04) != 0;
 +        }
 +
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
 +            assert version >= MessagingService.VERSION_30;
 +
 +            out.writeByte(command.kind.ordinal());
 +            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(command.index.isPresent()));
 +            if (command.isDigestQuery())
 +                out.writeUnsignedVInt(command.digestVersion());
 +            CFMetaData.serializer.serialize(command.metadata(), out, version);
 +            out.writeInt(command.nowInSec());
 +            ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
 +            RowFilter.serializer.serialize(command.rowFilter(), out, version);
 +            DataLimits.serializer.serialize(command.limits(), out, version);
 +            if (command.index.isPresent())
 +                IndexMetadata.serializer.serialize(command.index.get(), out, version);
 +
 +            command.serializeSelection(out, version);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            if (version < MessagingService.VERSION_30)
 +                return legacyReadCommandSerializer.deserialize(in, version);
 +
 +            Kind kind = Kind.values()[in.readByte()];
 +            int flags = in.readByte();
 +            boolean isDigest = isDigest(flags);
 +            boolean isForThrift = isForThrift(flags);
 +            boolean hasIndex = hasIndex(flags);
 +            int digestVersion = isDigest ? (int)in.readUnsignedVInt() : 0;
 +            CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
 +            int nowInSec = in.readInt();
 +            ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
 +            RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
 +            DataLimits limits = DataLimits.serializer.deserialize(in, version);
 +            Optional<IndexMetadata> index = hasIndex
 +                                            ? deserializeIndexMetadata(in, version, metadata)
 +                                            : Optional.empty();
 +
 +            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
 +        }
 +
 +        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
 +        {
 +            try
 +            {
 +                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
 +            }
 +            catch (UnknownIndexException e)
 +            {
 +                String message = String.format("Couldn't find a defined index on %s.%s with the id %s. " +
 +                                               "If an index was just created, this is likely due to the schema not " +
 +                                               "being fully propagated. Local read will proceed without using the " +
 +                                               "index. Please wait for schema agreement after index creation.",
 +                                               cfm.ksName, cfm.cfName, e.indexId.toString());
 +                logger.info(message);
 +                return Optional.empty();
 +            }
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
 +            assert version >= MessagingService.VERSION_30;
 +
 +            return 2 // kind + flags
 +                 + (command.isDigestQuery() ? TypeSizes.sizeofUnsignedVInt(command.digestVersion()) : 0)
 +                 + CFMetaData.serializer.serializedSize(command.metadata(), version)
 +                 + TypeSizes.sizeof(command.nowInSec())
 +                 + ColumnFilter.serializer.serializedSize(command.columnFilter(), version)
 +                 + RowFilter.serializer.serializedSize(command.rowFilter(), version)
 +                 + DataLimits.serializer.serializedSize(command.limits(), version)
 +                 + command.selectionSerializedSize(version)
 +                 + command.indexSerializedSize(version);
 +        }
 +    }
 +
 +    // Dispatch to either Serializer or LegacyRangeSliceCommandSerializer. Only useful as long as we maintain pre-3.0
 +    // compatibility
 +    private static class RangeSliceSerializer implements IVersionedSerializer<ReadCommand>
 +    {
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            if (version < MessagingService.VERSION_30)
 +                legacyRangeSliceCommandSerializer.serialize(command, out, version);
 +            else
 +                serializer.serialize(command, out, version);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            return version < MessagingService.VERSION_30
 +                 ? legacyRangeSliceCommandSerializer.deserialize(in, version)
 +                 : serializer.deserialize(in, version);
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            return version < MessagingService.VERSION_30
 +                 ? legacyRangeSliceCommandSerializer.serializedSize(command, version)
 +                 : serializer.serializedSize(command, version);
 +        }
 +    }
 +
 +    private enum LegacyType
 +    {
 +        GET_BY_NAMES((byte)1),
 +        GET_SLICES((byte)2);
 +
 +        public final byte serializedValue;
 +
 +        LegacyType(byte b)
 +        {
 +            this.serializedValue = b;
 +        }
 +
 +        public static LegacyType fromPartitionFilterKind(ClusteringIndexFilter.Kind kind)
 +        {
 +            return kind == ClusteringIndexFilter.Kind.SLICE
 +                   ? GET_SLICES
 +                   : GET_BY_NAMES;
 +        }
 +
 +        public static LegacyType fromSerializedValue(byte b)
 +        {
 +            return b == 1 ? GET_BY_NAMES : GET_SLICES;
 +        }
 +    }
 +
 +    /**
 +     * Serializer for pre-3.0 RangeSliceCommands.
 +     */
 +    private static class LegacyRangeSliceCommandSerializer implements IVersionedSerializer<ReadCommand>
      {
 -        out.writeByte(command.commandType.serializedValue);
 -        switch (command.commandType)
 +        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
 +        {
 +            assert version < MessagingService.VERSION_30;
 +
 +            PartitionRangeReadCommand rangeCommand = (PartitionRangeReadCommand) command;
 +            assert !rangeCommand.dataRange().isPaging();
 +
 +            // convert pre-3.0 incompatible names filters to slice filters
 +            rangeCommand = maybeConvertNamesToSlice(rangeCommand);
 +
 +            CFMetaData metadata = rangeCommand.metadata();
 +
 +            out.writeUTF(metadata.ksName);
 +            out.writeUTF(metadata.cfName);
 +            out.writeLong(rangeCommand.nowInSec() * 1000L);  // convert from seconds to millis
 +
 +            // begin DiskAtomFilterSerializer.serialize()
 +            if (rangeCommand.isNamesQuery())
 +            {
 +                out.writeByte(1);  // 0 for slices, 1 for names
 +                ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                LegacyReadCommandSerializer.serializeNamesFilter(rangeCommand, filter, out);
 +            }
 +            else
 +            {
 +                out.writeByte(0);  // 0 for slices, 1 for names
 +
 +                // slice filter serialization
 +                ClusteringIndexSliceFilter filter = (ClusteringIndexSliceFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +
 +                boolean makeStaticSlice = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() && !filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                LegacyReadCommandSerializer.serializeSlices(out, filter.requestedSlices(), filter.isReversed(), makeStaticSlice, metadata);
 +
 +                out.writeBoolean(filter.isReversed());
 +
 +                // limit
 +                DataLimits.Kind kind = rangeCommand.limits().kind();
 +                boolean isDistinct = (kind == DataLimits.Kind.CQL_LIMIT || kind == DataLimits.Kind.CQL_PAGING_LIMIT) && rangeCommand.limits().perPartitionCount() == 1;
 +                if (isDistinct)
 +                    out.writeInt(1);
 +                else
 +                    out.writeInt(LegacyReadCommandSerializer.updateLimitForQuery(rangeCommand.limits().count(), filter.requestedSlices()));
 +
 +                int compositesToGroup;
 +                boolean selectsStatics = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() || filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                if (kind == DataLimits.Kind.THRIFT_LIMIT)
 +                    compositesToGroup = -1;
 +                else if (isDistinct && !selectsStatics)
 +                    compositesToGroup = -2;  // for DISTINCT queries (CASSANDRA-8490)
 +                else
 +                    compositesToGroup = metadata.isDense() ? -1 : metadata.clusteringColumns().size();
 +
 +                out.writeInt(compositesToGroup);
 +            }
 +
 +            serializeRowFilter(out, rangeCommand.rowFilter());
 +            AbstractBounds.rowPositionSerializer.serialize(rangeCommand.dataRange().keyRange(), out, version);
 +
 +            // maxResults
 +            out.writeInt(rangeCommand.limits().count());
 +
 +            // countCQL3Rows
 +            if (rangeCommand.isForThrift() || rangeCommand.limits().perPartitionCount() == 1)  // if for Thrift or DISTINCT
 +                out.writeBoolean(false);
 +            else
 +                out.writeBoolean(true);
 +
 +            // isPaging
 +            out.writeBoolean(false);
 +        }
 +
 +        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
 +        {
 +            assert version < MessagingService.VERSION_30;
 +
 +            String keyspace = in.readUTF();
 +            String columnFamily = in.readUTF();
 +
 +            CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
 +            if (metadata == null)
 +            {
 +                String message = String.format("Got legacy range command for nonexistent table %s.%s.", keyspace, columnFamily);
 +                throw new UnknownColumnFamilyException(message, null);
 +            }
 +
 +            int nowInSec = (int) (in.readLong() / 1000);  // convert from millis to seconds
 +
 +            ClusteringIndexFilter filter;
 +            ColumnFilter selection;
 +            int compositesToGroup = 0;
 +            int perPartitionLimit = -1;
 +            byte readType = in.readByte();  // 0 for slices, 1 for names
 +            if (readType == 1)
 +            {
 +                Pair<ColumnFilter, ClusteringIndexNamesFilter> selectionAndFilter = LegacyReadCommandSerializer.deserializeNamesSelectionAndFilter(in, metadata);
 +                selection = selectionAndFilter.left;
 +                filter = selectionAndFilter.right;
 +            }
 +            else
 +            {
 +                Pair<ClusteringIndexSliceFilter, Boolean> p = LegacyReadCommandSerializer.deserializeSlicePartitionFilter(in, metadata);
 +                filter = p.left;
 +                perPartitionLimit = in.readInt();
 +                compositesToGroup = in.readInt();
 +                selection = getColumnSelectionForSlice(p.right, compositesToGroup, metadata);
 +            }
 +
 +            RowFilter rowFilter = deserializeRowFilter(in, metadata);
 +
 +            AbstractBounds<PartitionPosition> keyRange = AbstractBounds.rowPositionSerializer.deserialize(in, metadata.partitioner, version);
 +            int maxResults = in.readInt();
 +
 +            in.readBoolean();  // countCQL3Rows (not needed)
 +            in.readBoolean();  // isPaging (not needed)
 +
 +            boolean selectsStatics = (!selection.fetchedColumns().statics.isEmpty() || filter.selects(Clustering.STATIC_CLUSTERING));
 +            boolean isDistinct = compositesToGroup == -2 || (perPartitionLimit == 1 && selectsStatics);
 +            DataLimits limits;
 +            if (isDistinct)
 +                limits = DataLimits.distinctLimits(maxResults);
 +            else if (compositesToGroup == -1)
 +                limits = DataLimits.thriftLimits(maxResults, perPartitionLimit);
 +            else
 +                limits = DataLimits.cqlLimits(maxResults);
 +
 +            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter), Optional.empty());
 +        }
 +
 +        static void serializeRowFilter(DataOutputPlus out, RowFilter rowFilter) throws IOException
 +        {
 +            ArrayList<RowFilter.Expression> indexExpressions = Lists.newArrayList(rowFilter.iterator());
 +            out.writeInt(indexExpressions.size());
 +            for (RowFilter.Expression expression : indexExpressions)
 +            {
 +                ByteBufferUtil.writeWithShortLength(expression.column().name.bytes, out);
 +                expression.operator().writeTo(out);
 +                ByteBufferUtil.writeWithShortLength(expression.getIndexValue(), out);
 +            }
 +        }
 +
 +        static RowFilter deserializeRowFilter(DataInputPlus in, CFMetaData metadata) throws IOException
 +        {
 +            int numRowFilters = in.readInt();
 +            if (numRowFilters == 0)
 +                return RowFilter.NONE;
 +
 +            RowFilter rowFilter = RowFilter.create(numRowFilters);
 +            for (int i = 0; i < numRowFilters; i++)
 +            {
 +                ByteBuffer columnName = ByteBufferUtil.readWithShortLength(in);
 +                ColumnDefinition column = metadata.getColumnDefinition(columnName);
 +                Operator op = Operator.readFrom(in);
 +                ByteBuffer indexValue = ByteBufferUtil.readWithShortLength(in);
 +                rowFilter.add(column, op, indexValue);
 +            }
 +            return rowFilter;
 +        }
 +
 +        static long serializedRowFilterSize(RowFilter rowFilter)
 +        {
 +            long size = TypeSizes.sizeof(0);  // rowFilterCount
 +            for (RowFilter.Expression expression : rowFilter)
 +            {
 +                size += ByteBufferUtil.serializedSizeWithShortLength(expression.column().name.bytes);
 +                size += TypeSizes.sizeof(0);  // operator int value
 +                size += ByteBufferUtil.serializedSizeWithShortLength(expression.getIndexValue());
 +            }
 +            return size;
 +        }
 +
 +        public long serializedSize(ReadCommand command, int version)
 +        {
 +            assert version < MessagingService.VERSION_30;
 +            assert command.kind == Kind.PARTITION_RANGE;
 +
 +            PartitionRangeReadCommand rangeCommand = (PartitionRangeReadCommand) command;
 +            rangeCommand = maybeConvertNamesToSlice(rangeCommand);
 +            CFMetaData metadata = rangeCommand.metadata();
 +
 +            long size = TypeSizes.sizeof(metadata.ksName);
 +            size += TypeSizes.sizeof(metadata.cfName);
 +            size += TypeSizes.sizeof((long) rangeCommand.nowInSec());
 +
 +            size += 1;  // single byte flag: 0 for slices, 1 for names
 +            if (rangeCommand.isNamesQuery())
 +            {
 +                PartitionColumns columns = rangeCommand.columnFilter().fetchedColumns();
 +                ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                size += LegacyReadCommandSerializer.serializedNamesFilterSize(filter, metadata, columns);
 +            }
 +            else
 +            {
 +                ClusteringIndexSliceFilter filter = (ClusteringIndexSliceFilter) rangeCommand.dataRange().clusteringIndexFilter;
 +                boolean makeStaticSlice = !rangeCommand.columnFilter().fetchedColumns().statics.isEmpty() && !filter.requestedSlices().selects(Clustering.STATIC_CLUSTERING);
 +                size += LegacyReadCommandSerializer.serializedSlicesSize(filter.requestedSlices(), makeStaticSlice, metadata);
 +                size += TypeSizes.sizeof(filter.isReversed());
 +                size += TypeSizes.sizeof(rangeCommand.limits().perPartitionCount());
 +                size += TypeSizes.sizeof(0); // compositesToGroup
 +            }
 +
 +            if (rangeCommand.rowFilter().equals(RowFilter.NONE))
 +            {
 +                size += TypeSizes.sizeof(0);
 +            }
 +            else
 +            {
 +                ArrayList<RowFilter.Expression> indexExpressions = Lists.newArrayList(rangeCommand.rowFilter().iterator());
 +                size += TypeSizes.sizeof(indexExpressions.size());
 +                for (RowFilter.Expression expression : indexExpressions)
 +                {
 +                    size += ByteBufferUtil.serializedSizeWithShortLength(expression.column().name.bytes);
 +                    size += TypeSizes.sizeof(expression.operator().ordinal());
 +                    size += ByteBufferUtil.serializedSizeWithShortLength(expression.getIndexValue());
 +                }
 +            }
 +
 +            size += AbstractBounds.rowPositionSerializer.serializedSize(rangeCommand.dataRange().keyRange(), version);
 +            size += TypeSizes.sizeof(rangeCommand.limits().count());
 +            size += TypeSizes.sizeof(!rangeCommand.isForThrift());
 +            return size + TypeSizes.sizeof(rangeCommand.dataRange().isPaging());
 +        }
 +
 +        static PartitionRangeReadCommand maybeConvertNamesToSlice(PartitionRangeReadCommand command)
          {
 -            case GET_BY_NAMES:
 -                SliceByNamesReadCommand.serializer.serialize(command, out, version);
 -                break;
 -            case GET_SLICES:
 -                SliceFromReadCommand.serializer.serialize(command, out, version);
 -                break;
 -            default:
 -                throw new AssertionError();
 +            if (!command.dataRange().isNamesQuery())
 +                return command;
 +
 +            CFMetaData metadata = command.metadata();
 +            if (!LegacyReadCommandSerializer.shouldConvertNamesToSlice(metadata, command.columnFilter().fetchedColumns()))
 +                return command;
 +
 +            ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) command.dataRange().clusteringIndexFilter;
 +            ClusteringIndexSliceFilter sliceFilter = LegacyReadCommandSerializer.convertNamesFilterToSliceFilter(filter, metadata);
 +            DataRange newRange = new DataRange(command.dataRange().keyRange(), sliceFilter);
 +            return new PartitionRangeReadCommand(
 +                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
 +                    command.columnFilter(), command.rowFilter(), command.limits(), newRange, Optional.empty());
 +        }
 +
 +        static ColumnFilter getColumnSelectionForSlice(boolean selectsStatics, int compositesToGroup, CFMetaData metadata)
 +        {
 +            // A value of -2 indicates this is a DISTINCT query that doesn't select static columns, only partition keys.
 +            // In that case, we'll basically be querying the first row of the partition, but we must make sure we include
 +            // all columns so we get at least one cell if there is a live row as it would confuse pre-3.0 nodes otherwise.
 +            if (compositesToGroup == -2)
 +                return ColumnFilter.all(metadata);
 +
 +            // if a slice query from a pre-3.0 node doesn't cover statics, we shouldn't select them at all
 +            PartitionColumns columns = selectsStatics
 +                                     ? metadata.partitionColumns()
 +                                     : metadata.partitionColumns().withoutStatics();
 +            return ColumnFilter.selectionBuilder().addAll(columns).build();
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/94e7ef17/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/MessagingService.java
index ccc900b,459923b..d416dca
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@@ -39,8 -39,12 +39,9 @@@ import com.google.common.collect.Lists
  import com.google.common.collect.Sets;
  
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
 -
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
 -
 -import org.apache.cassandra.concurrent.ExecutorLocal;
+ import org.apache.cassandra.concurrent.ExecutorLocals;
  import org.apache.cassandra.concurrent.ScheduledExecutors;
  import org.apache.cassandra.concurrent.Stage;
  import org.apache.cassandra.concurrent.StageManager;


[11/13] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3

Posted by ca...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.3


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

Branch: refs/heads/cassandra-3.3
Commit: 93f6529393baa83bd232c48f94b1805a996ceb9a
Parents: a7feb80 94e7ef1
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 13 14:46:34 2016 -0500
Committer: Carl Yeksigian <ca...@apache.org>
Committed: Wed Jan 13 14:46:34 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../AbstractLocalAwareExecutorService.java      | 230 +++++++++++++++++++
 .../AbstractTracingAwareExecutorService.java    | 230 -------------------
 .../DebuggableThreadPoolExecutor.java           |  48 ++--
 .../cassandra/concurrent/ExecutorLocal.java     |  44 ++++
 .../cassandra/concurrent/ExecutorLocals.java    |  84 +++++++
 .../concurrent/LocalAwareExecutorService.java   |  34 +++
 .../cassandra/concurrent/SEPExecutor.java       |   3 +-
 .../concurrent/SharedExecutorPool.java          |   2 +-
 .../cassandra/concurrent/StageManager.java      |  12 +-
 .../concurrent/TracingAwareExecutorService.java |  36 ---
 .../cassandra/cql3/functions/UDFunction.java    |   2 +-
 .../cql3/statements/BatchStatement.java         |   9 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   2 +-
 .../apache/cassandra/net/MessagingService.java  |   7 +-
 .../apache/cassandra/service/ClientWarn.java    |  62 +++--
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   3 +-
 .../org/apache/cassandra/transport/Message.java |   6 +-
 .../transport/RequestThreadPoolExecutor.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |   6 +-
 .../cassandra/service/ClientWarningsTest.java   |  58 +++++
 24 files changed, 545 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index aae5efe,a37ec99..85bc100
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -7,29 -4,6 +7,30 @@@ Merged from 3.0
     tombstone (CASSANDRA-10743)
   * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
   * Fix potential assertion error during compaction (CASSANDRA-10944)
 +Merged from 2.2:
++ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
 + * Fix error streaming section more than 2GB (CASSANDRA-10961)
 + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975)
 + * Enable GC logging by default (CASSANDRA-10140)
 + * Optimize pending range computation (CASSANDRA-9258)
 + * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
 + * drop/alter user should be case sensitive (CASSANDRA-10817)
 +Merged from 2.1:
 + * (cqlsh) Add request timeout option to cqlsh (CASSANDRA-10686)
 + * Avoid AssertionError while submitting hint with LWT (CASSANDRA-10477)
 + * If CompactionMetadata is not in stats file, use index summary instead (CASSANDRA-10676)
 + * Retry sending gossip syn multiple times during shadow round (CASSANDRA-8072)
 + * Fix pending range calculation during moves (CASSANDRA-10887)
 + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-8708)
 +
 +
 +3.2
 + * Make sure tokens don't exist in several data directories (CASSANDRA-6696)
 + * Add requireAuthorization method to IAuthorizer (CASSANDRA-10852)
 + * Move static JVM options to conf/jvm.options file (CASSANDRA-10494)
 + * Fix CassandraVersion to accept x.y version string (CASSANDRA-10931)
 + * Add forceUserDefinedCleanup to allow more flexible cleanup (CASSANDRA-10708)
 + * (cqlsh) allow setting TTL with COPY (CASSANDRA-9494)
   * Fix counting of received sstables in streaming (CASSANDRA-10949)
   * Implement hints compression (CASSANDRA-9428)
   * Fix potential assertion error when reading static columns (CASSANDRA-10903)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/MessagingService.java
index d95c49b,d416dca..76f4967
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@@ -820,11 -821,11 +821,11 @@@ public final class MessagingService imp
              if (!ms.allowIncomingMessage(message, id))
                  return;
  
 -        Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
 +        Runnable runnable = new MessageDeliveryTask(message, id);
-         TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+         LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
          assert stage != null : "No stage for message type " + message.verb;
  
-         stage.execute(runnable, state);
+         stage.execute(runnable, ExecutorLocals.create(state));
      }
  
      public void setCallbackForTests(int messageId, CallbackInfo callback)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/93f65293/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------