You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/01/09 02:09:33 UTC

[1/3] cassandra git commit: Introduce background cache refreshing to permissions cache

Repository: cassandra
Updated Branches:
  refs/heads/trunk e41231933 -> 2fc3a8934


Introduce background cache refreshing to permissions cache

patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-8194


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

Branch: refs/heads/trunk
Commit: e750ab238e07daa61180d2451ba90f819a4cf5a1
Parents: bd3c47c
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Fri Jan 9 04:02:32 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Jan 9 04:02:32 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 conf/cassandra.yaml                             |   8 ++
 src/java/org/apache/cassandra/auth/Auth.java    |  55 ++--------
 .../org/apache/cassandra/auth/AuthMBean.java    |  27 -----
 .../apache/cassandra/auth/PermissionsCache.java | 108 +++++++++++++++++++
 .../org/apache/cassandra/config/Config.java     |   2 +
 .../cassandra/config/DatabaseDescriptor.java    |  10 +-
 .../apache/cassandra/service/ClientState.java   |  15 +--
 8 files changed, 138 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9ccbf45..adb374a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.0.12:
+ * Introduce background cache refreshing to permissions cache
+   (CASSANDRA-8194)
  * Fix race condition in StreamTransferTask that could lead to
    infinite loops and premature sstable deletion (CASSANDRA-7704)
  * Add an extra version check to MigrationTask (CASSANDRA-8462)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 5eaffc2..45290aa 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -79,6 +79,14 @@ authorizer: AllowAllAuthorizer
 # Will be disabled automatically for AllowAllAuthorizer.
 permissions_validity_in_ms: 2000
 
+# Refresh interval for permissions cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If permissions_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as permissions_validity_in_ms.
+# permissions_update_interval_in_ms: 1000
+
 # The partitioner is responsible for distributing groups of rows (by
 # partition key) across nodes in the cluster.  You should leave this
 # alone for new clusters.  The partitioner can NOT be changed without

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/src/java/org/apache/cassandra/auth/Auth.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/Auth.java b/src/java/org/apache/cassandra/auth/Auth.java
index 94d4b3d..465643d 100644
--- a/src/java/org/apache/cassandra/auth/Auth.java
+++ b/src/java/org/apache/cassandra/auth/Auth.java
@@ -20,9 +20,6 @@ package org.apache.cassandra.auth;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.StringUtils;
@@ -32,9 +29,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.RequestExecutionException;
@@ -43,9 +40,8 @@ import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
 
-public class Auth implements AuthMBean
+public class Auth
 {
     private static final Logger logger = LoggerFactory.getLogger(Auth.class);
 
@@ -57,8 +53,10 @@ public class Auth implements AuthMBean
     public static final String USERS_CF = "users";
 
     // User-level permissions cache.
-    public static volatile LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> permissionsCache = initPermissionsCache(null);
-
+    private static final PermissionsCache permissionsCache = new PermissionsCache(DatabaseDescriptor.getPermissionsValidity(),
+                                                                                  DatabaseDescriptor.getPermissionsUpdateInterval(),
+                                                                                  DatabaseDescriptor.getPermissionsCacheMaxEntries(),
+                                                                                  DatabaseDescriptor.getAuthorizer());
 
     private static final String USERS_CF_SCHEMA = String.format("CREATE TABLE %s.%s ("
                                                                 + "name text,"
@@ -71,44 +69,9 @@ public class Auth implements AuthMBean
 
     private static SelectStatement selectUserStatement;
 
-    public int getPermissionsValidity()
-    {
-        return DatabaseDescriptor.getPermissionsValidity();
-    }
-
-    public void setPermissionsValidity(int timeoutInMs)
-    {
-        DatabaseDescriptor.setPermissionsValidity(timeoutInMs);
-        permissionsCache = initPermissionsCache(permissionsCache);
-    }
-
-    public void invalidatePermissionsCache()
-    {
-        permissionsCache = initPermissionsCache(null);
-    }
-
-    private static LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> initPermissionsCache(LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> oldCache)
+    public static Set<Permission> getPermissions(AuthenticatedUser user, IResource resource)
     {
-        if (DatabaseDescriptor.getAuthorizer() instanceof AllowAllAuthorizer)
-            return null;
-
-        int validityPeriod = DatabaseDescriptor.getPermissionsValidity();
-        if (validityPeriod <= 0)
-            return null;
-
-        LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> newCache =
-            CacheBuilder.newBuilder().expireAfterWrite(validityPeriod, TimeUnit.MILLISECONDS)
-                    .build(new CacheLoader<Pair<AuthenticatedUser, IResource>, Set<Permission>>()
-                    {
-                        public Set<Permission> load(Pair<AuthenticatedUser, IResource> userResource)
-                        {
-                            return DatabaseDescriptor.getAuthorizer().authorize(userResource.left,
-                                    userResource.right);
-                        }
-                    });
-        if (oldCache != null)
-            newCache.putAll(oldCache.asMap());
-        return newCache;
+        return permissionsCache.getPermissions(user, resource);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/src/java/org/apache/cassandra/auth/AuthMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthMBean.java b/src/java/org/apache/cassandra/auth/AuthMBean.java
deleted file mode 100644
index db11f21..0000000
--- a/src/java/org/apache/cassandra/auth/AuthMBean.java
+++ /dev/null
@@ -1,27 +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.auth;
-
-public interface AuthMBean
-{
-    public int getPermissionsValidity();
-
-    public void setPermissionsValidity(int timeoutInMs);
-
-    public void invalidatePermissionsCache();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/src/java/org/apache/cassandra/auth/PermissionsCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java b/src/java/org/apache/cassandra/auth/PermissionsCache.java
new file mode 100644
index 0000000..9e0dfa9
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java
@@ -0,0 +1,108 @@
+/*
+ * 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.auth;
+
+import java.util.Set;
+import java.util.concurrent.*;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListenableFutureTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.utils.Pair;
+
+public class PermissionsCache
+{
+    private static final Logger logger = LoggerFactory.getLogger(PermissionsCache.class);
+
+    private final ThreadPoolExecutor cacheRefreshExecutor = new DebuggableThreadPoolExecutor("PermissionsCacheRefresh",
+                                                                                             Thread.NORM_PRIORITY);
+    private final IAuthorizer authorizer;
+    private final LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> cache;
+
+    public PermissionsCache(int validityPeriod, int updateInterval, int maxEntries, IAuthorizer authorizer)
+    {
+        this.authorizer = authorizer;
+        this.cache = initCache(validityPeriod, updateInterval, maxEntries);
+    }
+
+    public Set<Permission> getPermissions(AuthenticatedUser user, IResource resource)
+    {
+        if (cache == null)
+            return authorizer.authorize(user, resource);
+
+        try
+        {
+            return cache.get(Pair.create(user, resource));
+        }
+        catch (ExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> initCache(int validityPeriod,
+                                                                                        int updateInterval,
+                                                                                        int maxEntries)
+    {
+        if (authorizer instanceof AllowAllAuthorizer)
+            return null;
+
+        if (validityPeriod <= 0)
+            return null;
+
+        return CacheBuilder.newBuilder()
+                           .refreshAfterWrite(updateInterval, TimeUnit.MILLISECONDS)
+                           .expireAfterWrite(validityPeriod, TimeUnit.MILLISECONDS)
+                           .maximumSize(maxEntries)
+                           .build(new CacheLoader<Pair<AuthenticatedUser, IResource>, Set<Permission>>()
+                           {
+                               public Set<Permission> load(Pair<AuthenticatedUser, IResource> userResource)
+                               {
+                                   return authorizer.authorize(userResource.left, userResource.right);
+                               }
+
+                               public ListenableFuture<Set<Permission>> reload(final Pair<AuthenticatedUser, IResource> userResource,
+                                                                               final Set<Permission> oldValue)
+                               {
+                                   ListenableFutureTask<Set<Permission>> task = ListenableFutureTask.create(new Callable<Set<Permission>>()
+                                   {
+                                       public Set<Permission>call() throws Exception
+                                       {
+                                           try
+                                           {
+                                               return authorizer.authorize(userResource.left, userResource.right);
+                                           }
+                                           catch (Exception e)
+                                           {
+                                               logger.debug("Error performing async refresh of user permissions", e);
+                                               throw e;
+                                           }
+                                       }
+                                   });
+                                   cacheRefreshExecutor.execute(task);
+                                   return task;
+                               }
+                           });
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index aab5025..4dd71aa 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -43,6 +43,8 @@ public class Config
     public String authenticator;
     public String authorizer;
     public int permissions_validity_in_ms = 2000;
+    public int permissions_cache_max_entries = 1000;
+    public int permissions_update_interval_in_ms = -1;
 
     /* Hashing strategy Random or OPHF */
     public String partitioner;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 92ef79a..2bfdb16 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -578,11 +578,17 @@ public class DatabaseDescriptor
         return conf.permissions_validity_in_ms;
     }
 
-    public static void setPermissionsValidity(int timeout)
+    public static int getPermissionsCacheMaxEntries()
     {
-        conf.permissions_validity_in_ms = timeout;
+        return conf.permissions_cache_max_entries;
     }
 
+    public static int getPermissionsUpdateInterval()
+    {
+        return conf.permissions_update_interval_in_ms == -1
+             ? conf.permissions_validity_in_ms
+             : conf.permissions_update_interval_in_ms;
+    }
 
     public static int getThriftFramedTransportSize()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e750ab23/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 7611a14..63c9431 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.service;
 
 import java.net.SocketAddress;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
@@ -40,7 +39,6 @@ import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.SemanticVersion;
 
 /**
@@ -313,17 +311,6 @@ public class ClientState
 
     private Set<Permission> authorize(IResource resource)
     {
-        // AllowAllAuthorizer or manually disabled caching.
-        if (Auth.permissionsCache == null)
-            return DatabaseDescriptor.getAuthorizer().authorize(user, resource);
-
-        try
-        {
-            return Auth.permissionsCache.get(Pair.create(user, resource));
-        }
-        catch (ExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return Auth.getPermissions(user, resource);
     }
 }


[3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 2fc3a8934b211af762f9b3bec4c5b2dc46d6e0f5
Parents: e412319 8c5a959
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri Jan 9 04:08:30 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Jan 9 04:08:30 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 conf/cassandra.yaml                             |   8 ++
 src/java/org/apache/cassandra/auth/Auth.java    |  55 ++--------
 .../org/apache/cassandra/auth/AuthMBean.java    |  27 -----
 .../apache/cassandra/auth/PermissionsCache.java | 108 +++++++++++++++++++
 .../org/apache/cassandra/config/Config.java     |   2 +
 .../cassandra/config/DatabaseDescriptor.java    |  10 +-
 .../apache/cassandra/service/ClientState.java   |  15 +--
 8 files changed, 138 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2fc3a893/conf/cassandra.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2fc3a893/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2fc3a893/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2fc3a893/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------


[2/3] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by al...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	CHANGES.txt
	src/java/org/apache/cassandra/auth/Auth.java
	src/java/org/apache/cassandra/service/ClientState.java


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

Branch: refs/heads/trunk
Commit: 8c5a959c97729c5fbd536bf0f47cf6330c0bddbc
Parents: 5674a96 e750ab2
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri Jan 9 04:08:05 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Jan 9 04:08:05 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 conf/cassandra.yaml                             |   8 ++
 src/java/org/apache/cassandra/auth/Auth.java    |  55 ++--------
 .../org/apache/cassandra/auth/AuthMBean.java    |  27 -----
 .../apache/cassandra/auth/PermissionsCache.java | 108 +++++++++++++++++++
 .../org/apache/cassandra/config/Config.java     |   2 +
 .../cassandra/config/DatabaseDescriptor.java    |  10 +-
 .../apache/cassandra/service/ClientState.java   |  15 +--
 8 files changed, 138 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5a959c/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index dac555b,adb374a..57c2f49
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,55 -1,6 +1,57 @@@
 -2.0.12:
 +2.1.3
 + * Fix case-sensitivity of index name on CREATE and DROP INDEX
 +   statements (CASSANDRA-8365)
 + * Better detection/logging for corruption in compressed sstables (CASSANDRA-8192)
 + * Use the correct repairedAt value when closing writer (CASSANDRA-8570)
 + * (cqlsh) Handle a schema mismatch being detected on startup (CASSANDRA-8512)
 + * Properly calculate expected write size during compaction (CASSANDRA-8532)
 + * Invalidate affected prepared statements when a table's columns
 +   are altered (CASSANDRA-7910)
 + * Stress - user defined writes should populate sequentally (CASSANDRA-8524)
 + * Fix regression in SSTableRewriter causing some rows to become unreadable 
 +   during compaction (CASSANDRA-8429)
 + * Run major compactions for repaired/unrepaired in parallel (CASSANDRA-8510)
 + * (cqlsh) Fix compression options in DESCRIBE TABLE output when compression
 +   is disabled (CASSANDRA-8288)
 + * (cqlsh) Fix DESCRIBE output after keyspaces are altered (CASSANDRA-7623)
 + * Make sure we set lastCompactedKey correctly (CASSANDRA-8463)
 + * (cqlsh) Fix output of CONSISTENCY command (CASSANDRA-8507)
 + * (cqlsh) Fixed the handling of LIST statements (CASSANDRA-8370)
 + * Make sstablescrub check leveled manifest again (CASSANDRA-8432)
 + * Check first/last keys in sstable when giving out positions (CASSANDRA-8458)
 + * Disable mmap on Windows (CASSANDRA-6993)
 + * Add missing ConsistencyLevels to cassandra-stress (CASSANDRA-8253)
 + * Add auth support to cassandra-stress (CASSANDRA-7985)
 + * Fix ArrayIndexOutOfBoundsException when generating error message
 +   for some CQL syntax errors (CASSANDRA-8455)
 + * Scale memtable slab allocation logarithmically (CASSANDRA-7882)
 + * cassandra-stress simultaneous inserts over same seed (CASSANDRA-7964)
 + * Reduce cassandra-stress sampling memory requirements (CASSANDRA-7926)
 + * Ensure memtable flush cannot expire commit log entries from its future (CASSANDRA-8383)
 + * Make read "defrag" async to reclaim memtables (CASSANDRA-8459)
 + * Remove tmplink files for offline compactions (CASSANDRA-8321)
 + * Reduce maxHintsInProgress (CASSANDRA-8415)
 + * BTree updates may call provided update function twice (CASSANDRA-8018)
 + * Release sstable references after anticompaction (CASSANDRA-8386)
 + * Handle abort() in SSTableRewriter properly (CASSANDRA-8320)
 + * Fix high size calculations for prepared statements (CASSANDRA-8231)
 + * Centralize shared executors (CASSANDRA-8055)
 + * Fix filtering for CONTAINS (KEY) relations on frozen collection
 +   clustering columns when the query is restricted to a single
 +   partition (CASSANDRA-8203)
 + * Do more aggressive entire-sstable TTL expiry checks (CASSANDRA-8243)
 + * Add more log info if readMeter is null (CASSANDRA-8238)
 + * add check of the system wall clock time at startup (CASSANDRA-8305)
 + * Support for frozen collections (CASSANDRA-7859)
 + * Fix overflow on histogram computation (CASSANDRA-8028)
 + * Have paxos reuse the timestamp generation of normal queries (CASSANDRA-7801)
 + * Fix incremental repair not remove parent session on remote (CASSANDRA-8291)
 + * Improve JBOD disk utilization (CASSANDRA-7386)
 + * Log failed host when preparing incremental repair (CASSANDRA-8228)
 + * Force config client mode in CQLSSTableWriter (CASSANDRA-8281)
 +Merged from 2.0:
+  * Introduce background cache refreshing to permissions cache
+    (CASSANDRA-8194)
   * Fix race condition in StreamTransferTask that could lead to
     infinite loops and premature sstable deletion (CASSANDRA-7704)
   * Add an extra version check to MigrationTask (CASSANDRA-8462)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5a959c/conf/cassandra.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5a959c/src/java/org/apache/cassandra/auth/Auth.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/auth/Auth.java
index 0c3b0fe,465643d..05e5061
--- a/src/java/org/apache/cassandra/auth/Auth.java
+++ b/src/java/org/apache/cassandra/auth/Auth.java
@@@ -34,11 -29,9 +31,11 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.KSMetaData;
  import org.apache.cassandra.config.Schema;
- import org.apache.cassandra.cql3.UntypedResultSet;
- import org.apache.cassandra.cql3.QueryProcessor;
  import org.apache.cassandra.cql3.QueryOptions;
+ import org.apache.cassandra.cql3.QueryProcessor;
+ import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.cql3.statements.CFStatement;
 +import org.apache.cassandra.cql3.statements.CreateTableStatement;
  import org.apache.cassandra.cql3.statements.SelectStatement;
  import org.apache.cassandra.db.ConsistencyLevel;
  import org.apache.cassandra.exceptions.RequestExecutionException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5a959c/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5a959c/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5a959c/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/ClientState.java
index e1df1bd,63c9431..5ea3d0c
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@@ -19,8 -19,6 +19,7 @@@ package org.apache.cassandra.service
  
  import java.net.SocketAddress;
  import java.util.*;
- import java.util.concurrent.ExecutionException;
 +import java.util.concurrent.atomic.AtomicLong;
  
  import com.google.common.collect.Iterables;
  import com.google.common.collect.Sets;
@@@ -41,8 -39,6 +40,7 @@@ import org.apache.cassandra.exceptions.
  import org.apache.cassandra.tracing.Tracing;
  import org.apache.cassandra.thrift.ThriftValidation;
  import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.JVMStabilityInspector;
- import org.apache.cassandra.utils.Pair;
  import org.apache.cassandra.utils.SemanticVersion;
  
  /**