You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by az...@apache.org on 2021/11/13 17:04:49 UTC

[cassandra] branch cassandra-17062_auth_caches_metrics created (now 9df7ddc)

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

azotcsit pushed a change to branch cassandra-17062_auth_caches_metrics
in repository https://gitbox.apache.org/repos/asf/cassandra.git.


      at 9df7ddc  Expose Auth Caches metrics

This branch includes the following new commits:

     new 9df7ddc  Expose Auth Caches metrics

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


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


[cassandra] 01/01: Expose Auth Caches metrics

Posted by az...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

azotcsit pushed a commit to branch cassandra-17062_auth_caches_metrics
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 9df7ddc0a33188ec8bc5f617bc87cd69ca105edc
Author: Aleksei Zotov <az...@gmail.com>
AuthorDate: Sat Nov 13 21:04:22 2021 +0400

    Expose Auth Caches metrics
    
    patch by Aleksei Zotov; reviewed by XXX for CASSANDRA-17062
---
 doc/source/new/virtualtables.rst                   | 27 ++++--
 doc/source/operating/metrics.rst                   |  9 +-
 src/java/org/apache/cassandra/auth/AuthCache.java  | 53 ++++++++++--
 src/java/org/apache/cassandra/cache/CacheSize.java | 15 ++++
 .../org/apache/cassandra/cache/ChunkCache.java     |  2 +-
 .../apache/cassandra/db/virtual/CachesTable.java   | 37 ++++++---
 .../org/apache/cassandra/auth/AuthCacheTest.java   | 38 +++++++++
 .../cassandra/db/virtual/CachesTableTest.java      | 95 ++++++++++++++++++++++
 8 files changed, 247 insertions(+), 29 deletions(-)

diff --git a/doc/source/new/virtualtables.rst b/doc/source/new/virtualtables.rst
index 50a37c2..5cf50e9 100644
--- a/doc/source/new/virtualtables.rst
+++ b/doc/source/new/virtualtables.rst
@@ -203,19 +203,28 @@ The virtual tables may be described with ``DESCRIBE`` statement. The DDL listed
 
 Caches Virtual Table
 ********************
-The ``caches`` virtual table lists information about the caches. The four caches presently created are chunks, counters, keys and rows. A query on the ``caches`` virtual table returns the following details:
+The ``caches`` virtual table lists information about the caches. A query on the ``caches`` virtual table returns the following details:
 
 ::
 
  cqlsh:system_views> SELECT * FROM system_views.caches;
- name     | capacity_bytes | entry_count | hit_count | hit_ratio | recent_hit_rate_per_second | recent_request_rate_per_second | request_count | size_bytes
- ---------+----------------+-------------+-----------+-----------+----------------------------+--------------------------------+---------------+------------
-   chunks |      229638144 |          29 |       166 |      0.83 |                          5 |                              6 |           200 |     475136
- counters |       26214400 |           0 |         0 |       NaN |                          0 |                              0 |             0 |          0
-     keys |       52428800 |          14 |       124 |  0.873239 |                          4 |                              4 |           142 |       1248
-     rows |              0 |           0 |         0 |       NaN |                          0 |                              0 |             0 |          0
-
- (4 rows)
+ name                | capacity_bytes | entry_count | hit_count | hit_ratio | recent_hit_rate_per_second | recent_request_rate_per_second | request_count | size_bytes
+ --------------------+----------------+-------------+-----------+-----------+----------------------------+--------------------------------+---------------+------------
+              chunks |      229638144 |          29 |       166 |      0.83 |                          5 |                              6 |           200 |     475136
+            counters |       26214400 |           0 |         0 |       NaN |                          0 |                              0 |             0 |          0
+         credentials |           1000 |           1 |         2 |         1 |                          0 |                              0 |             2 |          1
+     jmx_permissions |           1000 |           0 |         0 |       NaN |                          0 |                              0 |             0 |          0
+                keys |       52428800 |          14 |       124 |  0.873239 |                          4 |                              4 |           142 |       1248
+ network_permissions |           1000 |           1 |        18 |         1 |                          2 |                              2 |            18 |          1
+         permissions |           1000 |           1 |         1 |         1 |                          0 |                              0 |             1 |          1
+               roles |           1000 |           1 |         9 |         1 |                          0 |                              0 |             9 |          1
+                rows |              0 |           0 |         0 |       NaN |                          0 |                              0 |             0 |          0
+
+ (8 rows)
+
+.. NOTE::
+   * chunk cache is only available if it is enabled.
+   * auth caches are only available if corresponding authorizers and authenticators are used.
 
 Settings Virtual Table
 **********************
diff --git a/doc/source/operating/metrics.rst b/doc/source/operating/metrics.rst
index e3af955..7f7e851 100644
--- a/doc/source/operating/metrics.rst
+++ b/doc/source/operating/metrics.rst
@@ -384,12 +384,19 @@ Name                         Description
 ============================ ===========
 ChunkCache                   In process uncompressed page cache.
 CounterCache                 Keeps hot counters in memory for performance.
+CredentialsCache             Auth cache for credentials.
+JmxPermissionsCache          Auth cache for JMX permissions.
 KeyCache                     Cache for partition to sstable offsets.
+NetworkPermissionsCache      Auth cache for network permissions.
+PermissionsCache             Auth cache for permissions.
+RolesCache                   Auth cache for roles.
 RowCache                     Cache for rows kept in memory.
 ============================ ===========
 
 .. NOTE::
-    Misses and MissLatency are only defined for the ChunkCache
+   * MissLatency is only defined for the ChunkCache.
+   * ChunkCache MBean is only available if the cache is enabled.
+   * AuthCache MBeans are only available if corresponding authorizers and authenticators are used.
 
 CQL Metrics
 ^^^^^^^^^^^
diff --git a/src/java/org/apache/cassandra/auth/AuthCache.java b/src/java/org/apache/cassandra/auth/AuthCache.java
index 04d3784..d96e098 100644
--- a/src/java/org/apache/cassandra/auth/AuthCache.java
+++ b/src/java/org/apache/cassandra/auth/AuthCache.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.auth;
 
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledFuture;
@@ -32,31 +31,35 @@ import java.util.function.Function;
 import java.util.function.IntConsumer;
 import java.util.function.IntSupplier;
 
+import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.github.benmanes.caffeine.cache.Policy;
+import org.apache.cassandra.cache.CacheSize;
 import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Shutdownable;
+import org.apache.cassandra.metrics.CacheMetrics;
 import org.apache.cassandra.utils.ExecutorUtils;
 import org.apache.cassandra.utils.MBeanWrapper;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
 
-public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
+public class AuthCache<K, V> implements AuthCacheMBean, CacheSize, Shutdownable
 {
     private static final Logger logger = LoggerFactory.getLogger(AuthCache.class);
 
     public static final String MBEAN_NAME_BASE = "org.apache.cassandra.auth:type=";
 
+    @SuppressWarnings("rawtypes")
     private volatile ScheduledFuture cacheRefresher = null;
 
     // Keep a handle on created instances so their executors can be terminated cleanly
-    private static final Set<Shutdownable> REGISTRY = new HashSet<>(4);
+    private static final Set<Shutdownable> REGISTRY = Sets.newHashSetWithExpectedSize(5);
 
     public static void shutdownAllAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
     {
@@ -87,6 +90,8 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
     // values until the natural expiry time.
     private final BiPredicate<K, V> invalidateCondition;
 
+    private final CacheMetrics metrics;
+
     /**
      * @param name Used for MBean
      * @param setValidityDelegate Used to set cache validity period. See {@link Policy#expireAfterWrite()}
@@ -166,6 +171,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
         this.loadFunction = checkNotNull(loadFunction);
         this.enableCache = checkNotNull(cacheEnabledDelegate);
         this.invalidateCondition = checkNotNull(invalidationCondition);
+        this.metrics = new CacheMetrics(name, this);
         init();
     }
 
@@ -204,7 +210,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
     /**
      * Retrieve a value from the cache. Will call {@link LoadingCache#get(Object)} which will
      * "load" the value if it's not present, thus populating the key.
-     * @param k
+     * @param k key
      * @return The current value of {@code K} if cached or loaded.
      *
      * See {@link LoadingCache#get(Object)} for possible exceptions.
@@ -214,7 +220,13 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
         if (cache == null)
             return loadFunction.apply(k);
 
-        return cache.get(k);
+        metrics.requests.mark();
+        V v = cache.get(k);
+        if (v != null)
+            metrics.hits.mark();
+        else
+            metrics.misses.mark();
+        return v;
     }
 
     /**
@@ -273,7 +285,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
 
     /**
      * Set maximum number of entries in the cache.
-     * @param maxEntries
+     * @param maxEntries max number of entries
      */
     public synchronized void setMaxEntries(int maxEntries)
     {
@@ -303,6 +315,11 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
         cache = initCache(cache);
     }
 
+    public CacheMetrics getMetrics()
+    {
+        return metrics;
+    }
+
     /**
      * (Re-)initialise the underlying cache. Will update validity, max entries, and update interval if
      * any have changed. The underlying {@link LoadingCache} will be initiated based on the provided {@code loadFunction}.
@@ -382,4 +399,28 @@ public class AuthCache<K, V> implements AuthCacheMBean, Shutdownable
     {
         return cacheRefreshExecutor.awaitTermination(timeout, units);
     }
+
+    @Override
+    public long capacity()
+    {
+        return getMaxEntries();
+    }
+
+    @Override
+    public void setCapacity(long capacity)
+    {
+        setMaxEntries(Math.toIntExact(capacity));
+    }
+
+    @Override
+    public int size()
+    {
+        return Math.toIntExact(cache.estimatedSize());
+    }
+
+    @Override
+    public long weightedSize()
+    {
+        return cache.estimatedSize();
+    }
 }
diff --git a/src/java/org/apache/cassandra/cache/CacheSize.java b/src/java/org/apache/cassandra/cache/CacheSize.java
index 71365bb..accff5e 100644
--- a/src/java/org/apache/cassandra/cache/CacheSize.java
+++ b/src/java/org/apache/cassandra/cache/CacheSize.java
@@ -23,12 +23,27 @@ package org.apache.cassandra.cache;
 public interface CacheSize
 {
 
+    /**
+     * Returns the maximum total weighted or unweighted size (number of entries) of this cache, depending on how the
+     * cache was constructed.
+     */
     long capacity();
 
+    /**
+     * Specifies the maximum total size of this cache. This value may be interpreted as the weighted or unweighted
+     * (number of entries) threshold size based on how this cache was constructed.
+     */
     void setCapacity(long capacity);
 
+    /**
+     * Returns the approximate number of entries in this cache.
+     */
     int size();
 
+    /**
+     * Returns the approximate accumulated weight of entries in this cache. If this cache does not use a weighted size
+     * bound, then it equals to the approximate number of entries.
+     */
     long weightedSize();
 
 }
diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java
index c53810a..fed7ffd 100644
--- a/src/java/org/apache/cassandra/cache/ChunkCache.java
+++ b/src/java/org/apache/cassandra/cache/ChunkCache.java
@@ -165,7 +165,7 @@ public class ChunkCache
         buffer.release();
     }
 
-    public void close()
+    public void clear()
     {
         cache.invalidateAll();
     }
diff --git a/src/java/org/apache/cassandra/db/virtual/CachesTable.java b/src/java/org/apache/cassandra/db/virtual/CachesTable.java
index 5a265e6..e9c9ea3 100644
--- a/src/java/org/apache/cassandra/db/virtual/CachesTable.java
+++ b/src/java/org/apache/cassandra/db/virtual/CachesTable.java
@@ -17,7 +17,13 @@
  */
 package org.apache.cassandra.db.virtual;
 
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.auth.PasswordAuthenticator;
+import org.apache.cassandra.auth.Roles;
+import org.apache.cassandra.auth.jmx.AuthorizationProxy;
 import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.metrics.CacheMetrics;
@@ -27,14 +33,14 @@ import org.apache.cassandra.service.CacheService;
 final class CachesTable extends AbstractVirtualTable
 {
     private static final String NAME = "name";
-    private static final String CAPACITY_BYTES = "capacity_bytes";
-    private static final String SIZE_BYTES = "size_bytes";
+    private static final String CAPACITY = "capacity";
     private static final String ENTRY_COUNT = "entry_count";
-    private static final String REQUEST_COUNT = "request_count";
     private static final String HIT_COUNT = "hit_count";
     private static final String HIT_RATIO = "hit_ratio";
     private static final String RECENT_REQUEST_RATE_PER_SECOND = "recent_request_rate_per_second";
     private static final String RECENT_HIT_RATE_PER_SECOND = "recent_hit_rate_per_second";
+    private static final String REQUEST_COUNT = "request_count";
+    private static final String SIZE = "size";
 
     CachesTable(String keyspace)
     {
@@ -43,38 +49,45 @@ final class CachesTable extends AbstractVirtualTable
                            .kind(TableMetadata.Kind.VIRTUAL)
                            .partitioner(new LocalPartitioner(UTF8Type.instance))
                            .addPartitionKeyColumn(NAME, UTF8Type.instance)
-                           .addRegularColumn(CAPACITY_BYTES, LongType.instance)
-                           .addRegularColumn(SIZE_BYTES, LongType.instance)
+                           .addRegularColumn(CAPACITY, LongType.instance)
                            .addRegularColumn(ENTRY_COUNT, Int32Type.instance)
-                           .addRegularColumn(REQUEST_COUNT, LongType.instance)
                            .addRegularColumn(HIT_COUNT, LongType.instance)
                            .addRegularColumn(HIT_RATIO, DoubleType.instance)
-                           .addRegularColumn(RECENT_REQUEST_RATE_PER_SECOND, LongType.instance)
                            .addRegularColumn(RECENT_HIT_RATE_PER_SECOND, LongType.instance)
+                           .addRegularColumn(RECENT_REQUEST_RATE_PER_SECOND, LongType.instance)
+                           .addRegularColumn(REQUEST_COUNT, LongType.instance)
+                           .addRegularColumn(SIZE, LongType.instance)
                            .build());
     }
 
     private void addRow(SimpleDataSet result, String name, CacheMetrics metrics)
     {
         result.row(name)
-              .column(CAPACITY_BYTES, metrics.capacity.getValue())
-              .column(SIZE_BYTES, metrics.size.getValue())
+              .column(CAPACITY, metrics.capacity.getValue())
               .column(ENTRY_COUNT, metrics.entries.getValue())
-              .column(REQUEST_COUNT, metrics.requests.getCount())
               .column(HIT_COUNT, metrics.hits.getCount())
               .column(HIT_RATIO, metrics.hitRate.getValue())
+              .column(RECENT_HIT_RATE_PER_SECOND, (long) metrics.hits.getFifteenMinuteRate())
               .column(RECENT_REQUEST_RATE_PER_SECOND, (long) metrics.requests.getFifteenMinuteRate())
-              .column(RECENT_HIT_RATE_PER_SECOND, (long) metrics.hits.getFifteenMinuteRate());
+              .column(REQUEST_COUNT, metrics.requests.getCount())
+              .column(SIZE, metrics.size.getValue());
     }
 
     public DataSet data()
     {
         SimpleDataSet result = new SimpleDataSet(metadata());
 
-        if (null != ChunkCache.instance)
+        if (ChunkCache.instance != null)
             addRow(result, "chunks", ChunkCache.instance.metrics);
         addRow(result, "counters", CacheService.instance.counterCache.getMetrics());
+        IAuthenticator authenticator = DatabaseDescriptor.getAuthenticator();
+        if (authenticator instanceof PasswordAuthenticator)
+            addRow(result, "credentials", ((PasswordAuthenticator) authenticator).getCredentialsCache().getMetrics());
+        addRow(result, "jmx_permissions", AuthorizationProxy.jmxPermissionsCache.getMetrics());
         addRow(result, "keys", CacheService.instance.keyCache.getMetrics());
+        addRow(result, "network_permissions", AuthenticatedUser.networkPermissionsCache.getMetrics());
+        addRow(result, "permissions", AuthenticatedUser.permissionsCache.getMetrics());
+        addRow(result, "roles", Roles.cache.getMetrics());
         addRow(result, "rows", CacheService.instance.rowCache.getMetrics());
 
         return result;
diff --git a/test/unit/org/apache/cassandra/auth/AuthCacheTest.java b/test/unit/org/apache/cassandra/auth/AuthCacheTest.java
index bf3d7bb..909b2e6 100644
--- a/test/unit/org/apache/cassandra/auth/AuthCacheTest.java
+++ b/test/unit/org/apache/cassandra/auth/AuthCacheTest.java
@@ -263,6 +263,31 @@ public class AuthCacheTest
         assertEquals(1, loadCounter);
     }
 
+    @Test
+    public void testMetricsOnCacheEnabled()
+    {
+        TestCache<String, Integer> authCache = new TestCache<>(this::countingLoaderForEvenNumbers, this::setValidity, () -> validity, () -> isCacheEnabled);
+        authCache.get("10");
+        authCache.get("11");
+
+        assertThat(authCache.getMetrics().requests.getCount()).isEqualTo(2L);
+        assertThat(authCache.getMetrics().hits.getCount()).isEqualTo(1L);
+        assertThat(authCache.getMetrics().misses.getCount()).isEqualTo(1L);
+    }
+
+    @Test
+    public void testMetricsOnCacheDisabled()
+    {
+        isCacheEnabled = false;
+        TestCache<String, Integer> authCache = new TestCache<>(this::countingLoaderForEvenNumbers, this::setValidity, () -> validity, () -> isCacheEnabled);
+        authCache.get("10");
+        authCache.get("11");
+
+        assertThat(authCache.getMetrics().requests.getCount()).isEqualTo(0L);
+        assertThat(authCache.getMetrics().hits.getCount()).isEqualTo(0L);
+        assertThat(authCache.getMetrics().misses.getCount()).isEqualTo(0L);
+    }
+
     private void setValidity(int validity)
     {
         this.validity = validity;
@@ -284,6 +309,19 @@ public class AuthCacheTest
         return loadedValue;
     }
 
+    /**
+     * Loads the key if it represents an even number.
+     */
+    private Integer countingLoaderForEvenNumbers(String s)
+    {
+        Integer loadedValue = countingLoader(s);
+
+        if (loadedValue % 2 == 0)
+            return loadedValue;
+        else
+            return null;
+    }
+
     private static class TestCache<K, V> extends AuthCache<K, V>
     {
         private static int nameCounter = 0; // Allow us to create many instances of cache with same name prefix
diff --git a/test/unit/org/apache/cassandra/db/virtual/CachesTableTest.java b/test/unit/org/apache/cassandra/db/virtual/CachesTableTest.java
new file mode 100644
index 0000000..d87d6f0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/virtual/CachesTableTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.db.virtual;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.auth.PasswordAuthenticator;
+import org.apache.cassandra.auth.Roles;
+import org.apache.cassandra.auth.jmx.AuthorizationProxy;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.service.CacheService;
+
+public class CachesTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @SuppressWarnings("FieldCanBeLocal")
+    private CachesTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+        CQLTester.requireAuthentication();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new CachesTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    public void testSelectAllWhenMetricsAreZeroed() throws Throwable
+    {
+        resetAllCaches();
+
+        assertRows(execute("SELECT * FROM vts.caches"),
+                   row("chunks",         503316480L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("counters",         5242880L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("credentials",         1000L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("jmx_permissions",     1000L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("keys",            11534336L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("network_permissions", 1000L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("permissions",         1000L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("roles",               1000L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L),
+                   row("rows",            16777216L, 0, 0L, Double.NaN, 0L, 0L, 0L, 0L));
+    }
+
+    private void resetAllCaches()
+    {
+        ChunkCache.instance.clear();
+        ChunkCache.instance.metrics.reset();
+        CacheService.instance.counterCache.clear();
+        CacheService.instance.counterCache.getMetrics().reset();
+        PasswordAuthenticator.CredentialsCache credentialsCache = ((PasswordAuthenticator) DatabaseDescriptor.getAuthenticator()).getCredentialsCache();
+        credentialsCache.invalidate();
+        credentialsCache.getMetrics().reset();
+        AuthorizationProxy.jmxPermissionsCache.invalidate();
+        AuthorizationProxy.jmxPermissionsCache.getMetrics().reset();
+        CacheService.instance.keyCache.clear();
+        CacheService.instance.keyCache.getMetrics().reset();
+        AuthenticatedUser.networkPermissionsCache.invalidate();
+        AuthenticatedUser.networkPermissionsCache.getMetrics().reset();
+        AuthenticatedUser.permissionsCache.invalidate();
+        AuthenticatedUser.permissionsCache.getMetrics().reset();
+        Roles.cache.invalidate();
+        Roles.cache.getMetrics().reset();
+        CacheService.instance.rowCache.clear();
+        CacheService.instance.rowCache.getMetrics().reset();
+    }
+}

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