You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2020/11/09 14:03:57 UTC

[cassandra] branch trunk updated: Add a ratelimiter to snapshot creation and deletion

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new 85f69cf  Add a ratelimiter to snapshot creation and deletion
85f69cf is described below

commit 85f69cf394a8035280a741b117e7dfc71fb848c3
Author: Jeff Jirsa <jj...@apple.com>
AuthorDate: Sat Nov 9 21:01:37 2019 -0800

    Add a ratelimiter to snapshot creation and deletion
    
    Patch by Jeff Jirsa; Reviewed by Aleksey Yeschenko, Chris Lohfink,
    maxwellguo for CASSANDRA-13019
---
 CHANGES.txt                                        |  1 +
 NEWS.txt                                           |  1 +
 conf/cassandra.yaml                                |  7 ++++
 src/java/org/apache/cassandra/config/Config.java   |  1 +
 .../cassandra/config/DatabaseDescriptor.java       | 22 ++++++++++
 .../org/apache/cassandra/db/ColumnFamilyStore.java | 35 ++++++++++++----
 src/java/org/apache/cassandra/db/Directories.java  |  5 ++-
 src/java/org/apache/cassandra/db/Keyspace.java     | 12 ++++--
 .../org/apache/cassandra/db/SystemKeyspace.java    |  2 +-
 .../org/apache/cassandra/io/util/FileUtils.java    | 47 +++++++++++++++++++---
 .../apache/cassandra/service/StorageService.java   | 19 ++++++++-
 .../cassandra/service/StorageServiceMBean.java     | 16 ++++++++
 src/java/org/apache/cassandra/tools/NodeProbe.java | 10 +++++
 src/java/org/apache/cassandra/tools/NodeTool.java  |  2 +
 .../tools/nodetool/GetSnapshotThrottle.java        | 36 +++++++++++++++++
 .../tools/nodetool/SetSnapshotThrottle.java        | 36 +++++++++++++++++
 16 files changed, 230 insertions(+), 22 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 635fa71..f90ffd4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-beta4
+ * Add a ratelimiter to snapshot creation and deletion (CASSANDRA-13019)
  * Produce consistent tombstone for reads to avoid digest mistmatch (CASSANDRA-15369)
  * Fix SSTableloader issue when restoring a table named backups (CASSANDRA-16235)
  * Invalid serialized size for responses caused by increasing message time by 1ms which caused extra bytes in size calculation (CASSANDRA-16103)
diff --git a/NEWS.txt b/NEWS.txt
index 4b5264f..7d16a97 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -112,6 +112,7 @@ New features
       Added --python option to cqlsh so users can specify the path to their chosen Python interpreter.
       See CASSANDRA-10190 for details.
     - Support for server side DESCRIBE statements has been added. See CASSANDRA-14825
+    - It is now possible to rate limit snapshot creation/clearing. See CASSANDRA-13019
 
 Upgrading
 ---------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 37b18f9..11e54ad 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -791,6 +791,13 @@ snapshot_before_compaction: false
 # lose data on truncation or drop.
 auto_snapshot: true
 
+# The act of creating or clearing a snapshot involves creating or removing
+# potentially tens of thousands of links, which can cause significant performance
+# impact, especially on consumer grade SSDs. A non-zero value here can
+# be used to throttle these links to avoid negative performance impact of
+# taking and clearing snapshots
+snapshot_links_per_second: 0
+
 # Granularity of the collation index of rows within a partition.
 # Increase if your rows are large, or if you have a very large
 # number of rows per partition.  The competing goals are these:
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 26854da..759a41a 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -204,6 +204,7 @@ public class Config
 
     public boolean snapshot_before_compaction = false;
     public boolean auto_snapshot = true;
+    public volatile long snapshot_links_per_second = 0;
 
     /* if the size of columns or super-columns are more than this, indexing will kick in */
     public int column_index_size_in_kb = 64;
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 5fbb220..2be169a 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -34,6 +34,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.primitives.Ints;
 import com.google.common.primitives.Longs;
+import com.google.common.util.concurrent.RateLimiter;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -772,6 +773,9 @@ public class DatabaseDescriptor
             }
         }
 
+        if (conf.snapshot_links_per_second < 0)
+            throw new ConfigurationException("snapshot_links_per_second must be >= 0");
+
         if (conf.max_value_size_in_mb <= 0)
             throw new ConfigurationException("max_value_size_in_mb must be positive", false);
         else if (conf.max_value_size_in_mb >= 2048)
@@ -2294,6 +2298,24 @@ public class DatabaseDescriptor
         return conf.auto_snapshot;
     }
 
+    public static long getSnapshotLinksPerSecond()
+    {
+        return conf.snapshot_links_per_second == 0 ? Long.MAX_VALUE : conf.snapshot_links_per_second;
+    }
+
+    public static void setSnapshotLinksPerSecond(long throttle)
+    {
+        if (throttle < 0)
+            throw new IllegalArgumentException("Invalid throttle for snapshot_links_per_second: must be positive");
+
+        conf.snapshot_links_per_second = throttle;
+    }
+
+    public static RateLimiter getSnapshotRateLimiter()
+    {
+        return RateLimiter.create(getSnapshotLinksPerSecond());
+    }
+
     public static boolean isAutoBootstrap()
     {
         return Boolean.parseBoolean(System.getProperty(Config.PROPERTY_PREFIX + "auto_bootstrap", Boolean.toString(conf.auto_bootstrap)));
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 1fc3eba..d0d6ed1 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1760,14 +1760,17 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public void snapshotWithoutFlush(String snapshotName)
     {
-        snapshotWithoutFlush(snapshotName, null, false);
+        snapshotWithoutFlush(snapshotName, null, false, null);
     }
 
     /**
      * @param ephemeral If this flag is set to true, the snapshot will be cleaned during next startup
      */
-    public Set<SSTableReader> snapshotWithoutFlush(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral)
+    public Set<SSTableReader> snapshotWithoutFlush(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral, RateLimiter rateLimiter)
     {
+        if (rateLimiter == null)
+            rateLimiter = DatabaseDescriptor.getSnapshotRateLimiter();
+
         Set<SSTableReader> snapshottedSSTables = new HashSet<>();
         final JSONArray filesJSONArr = new JSONArray();
         for (ColumnFamilyStore cfs : concatWithIndexes())
@@ -1777,6 +1780,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 for (SSTableReader ssTable : currentView.sstables)
                 {
                     File snapshotDirectory = Directories.getSnapshotDirectory(ssTable.descriptor, snapshotName);
+                    rateLimiter.acquire(SSTableReader.componentsFor(ssTable.descriptor).size());
                     ssTable.createLinks(snapshotDirectory.getPath()); // hard links
                     filesJSONArr.add(ssTable.descriptor.relativeFilenameFor(Component.DATA));
 
@@ -1864,10 +1868,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     protected static void clearEphemeralSnapshots(Directories directories)
     {
+        RateLimiter clearSnapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter();
+
         for (String ephemeralSnapshot : directories.listEphemeralSnapshots())
         {
             logger.trace("Clearing ephemeral snapshot {} leftover from previous session.", ephemeralSnapshot);
-            Directories.clearSnapshot(ephemeralSnapshot, directories.getCFDirectories());
+            Directories.clearSnapshot(ephemeralSnapshot, directories.getCFDirectories(), clearSnapshotRateLimiter);
         }
     }
 
@@ -1918,7 +1924,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public Set<SSTableReader> snapshot(String snapshotName)
     {
-        return snapshot(snapshotName, false);
+        return snapshot(snapshotName, false, null);
     }
 
     /**
@@ -1926,10 +1932,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      *
      * @param snapshotName the name of the associated with the snapshot
      * @param skipFlush Skip blocking flush of memtable
+     * @param rateLimiter Rate limiter for hardlinks-per-second
      */
-    public Set<SSTableReader> snapshot(String snapshotName, boolean skipFlush)
+    public Set<SSTableReader> snapshot(String snapshotName, boolean skipFlush, RateLimiter rateLimiter)
     {
-        return snapshot(snapshotName, null, false, skipFlush);
+        return snapshot(snapshotName, null, false, skipFlush, rateLimiter);
     }
 
 
@@ -1939,11 +1946,21 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public Set<SSTableReader> snapshot(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral, boolean skipFlush)
     {
+        return snapshot(snapshotName, predicate, ephemeral, skipFlush, null);
+    }
+
+    /**
+     * @param ephemeral If this flag is set to true, the snapshot will be cleaned up during next startup
+     * @param skipFlush Skip blocking flush of memtable
+     * @param rateLimiter Rate limiter for hardlinks-per-second
+     */
+    public Set<SSTableReader> snapshot(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral, boolean skipFlush, RateLimiter rateLimiter)
+    {
         if (!skipFlush)
         {
             forceBlockingFlush();
         }
-        return snapshotWithoutFlush(snapshotName, predicate, ephemeral);
+        return snapshotWithoutFlush(snapshotName, predicate, ephemeral, rateLimiter);
     }
 
     public boolean snapshotExists(String snapshotName)
@@ -1964,8 +1981,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public void clearSnapshot(String snapshotName)
     {
+        RateLimiter clearSnapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter();
+
         List<File> snapshotDirs = getDirectories().getCFDirectories();
-        Directories.clearSnapshot(snapshotName, snapshotDirs);
+        Directories.clearSnapshot(snapshotName, snapshotDirs, clearSnapshotRateLimiter);
     }
     /**
      *
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 889f3a5..9a620a2 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -32,6 +32,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.RateLimiter;
 
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -916,7 +917,7 @@ public class Directories
         return false;
     }
 
-    public static void clearSnapshot(String snapshotName, List<File> snapshotDirectories)
+    public static void clearSnapshot(String snapshotName, List<File> snapshotDirectories, RateLimiter snapshotRateLimiter)
     {
         // If snapshotName is empty or null, we will delete the entire snapshot directory
         String tag = snapshotName == null ? "" : snapshotName;
@@ -928,7 +929,7 @@ public class Directories
                 logger.trace("Removing snapshot directory {}", snapshotDir);
                 try
                 {
-                    FileUtils.deleteRecursive(snapshotDir);
+                    FileUtils.deleteRecursiveWithThrottle(snapshotDir, snapshotRateLimiter);
                 }
                 catch (FSWriteError e)
                 {
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 1b05a36..ead01fb 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -37,6 +37,7 @@ import java.util.stream.Stream;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.RateLimiter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -242,9 +243,10 @@ public class Keyspace
      * @param snapshotName     the tag associated with the name of the snapshot.  This value may not be null
      * @param columnFamilyName the column family to snapshot or all on null
      * @param skipFlush Skip blocking flush of memtable
+     * @param rateLimiter Rate limiter for hardlinks-per-second
      * @throws IOException if the column family doesn't exist
      */
-    public void snapshot(String snapshotName, String columnFamilyName, boolean skipFlush) throws IOException
+    public void snapshot(String snapshotName, String columnFamilyName, boolean skipFlush, RateLimiter rateLimiter) throws IOException
     {
         assert snapshotName != null;
         boolean tookSnapShot = false;
@@ -253,7 +255,7 @@ public class Keyspace
             if (columnFamilyName == null || cfStore.name.equals(columnFamilyName))
             {
                 tookSnapShot = true;
-                cfStore.snapshot(snapshotName, skipFlush);
+                cfStore.snapshot(snapshotName, skipFlush, rateLimiter);
             }
         }
 
@@ -271,7 +273,7 @@ public class Keyspace
      */
     public void snapshot(String snapshotName, String columnFamilyName) throws IOException
     {
-        snapshot(snapshotName, columnFamilyName, false);
+        snapshot(snapshotName, columnFamilyName, false, null);
     }
 
     /**
@@ -318,8 +320,10 @@ public class Keyspace
      */
     public static void clearSnapshot(String snapshotName, String keyspace)
     {
+        RateLimiter clearSnapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter();
+
         List<File> snapshotDirs = Directories.getKSChildDirectories(keyspace);
-        Directories.clearSnapshot(snapshotName, snapshotDirs);
+        Directories.clearSnapshot(snapshotName, snapshotDirs, clearSnapshotRateLimiter);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 655c7a0..bb6ab4a 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -1449,7 +1449,7 @@ public final class SystemKeyspace
                                                                              previous,
                                                                              next));
             for (String keyspace : SchemaConstants.LOCAL_SYSTEM_KEYSPACE_NAMES)
-                Keyspace.open(keyspace).snapshot(snapshotName, null);
+                Keyspace.open(keyspace).snapshot(snapshotName, null, false, null);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 67840c4..bcd848a 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -38,9 +38,11 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
+import com.google.common.util.concurrent.RateLimiter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,6 +54,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.SyncUtil;
 
 import static com.google.common.base.Throwables.propagate;
@@ -64,6 +67,8 @@ public final class FileUtils
     public static final Charset CHARSET = StandardCharsets.UTF_8;
 
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
+    private static final NoSpamLogger nospam1m = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
     public static final long ONE_KB = 1024;
     public static final long ONE_MB = 1024 * ONE_KB;
     public static final long ONE_GB = 1024 * ONE_MB;
@@ -186,16 +191,24 @@ public final class FileUtils
 
     public static Throwable deleteWithConfirm(String filePath, Throwable accumulate)
     {
-        return deleteWithConfirm(new File(filePath), accumulate);
+        return deleteWithConfirm(new File(filePath), accumulate, null);
     }
 
     public static Throwable deleteWithConfirm(File file, Throwable accumulate)
     {
+        return deleteWithConfirm(file, accumulate, null);
+    }
+    
+    public static Throwable deleteWithConfirm(File file, Throwable accumulate, RateLimiter rateLimiter)
+    {
         try
         {
-            if (!StorageService.instance.isDaemonSetupCompleted())
-                logger.info("Deleting file during startup: {}", file);
-
+            if (rateLimiter != null)
+            {
+                double throttled = rateLimiter.acquire();
+                if (throttled > 0.0)
+                    nospam1m.warn("Throttling file deletion: waited {} seconds to delete {}", throttled, file);
+            }
             Files.delete(file.toPath());
         }
         catch (Throwable t)
@@ -219,7 +232,12 @@ public final class FileUtils
 
     public static void deleteWithConfirm(File file)
     {
-        maybeFail(deleteWithConfirm(file, null));
+        maybeFail(deleteWithConfirm(file, null, null));
+    }
+
+    public static void deleteWithConfirmWithThrottle(File file, RateLimiter rateLimiter)
+    {
+        maybeFail(deleteWithConfirm(file, null, rateLimiter));
     }
 
     public static void renameWithOutConfirm(String from, String to)
@@ -558,6 +576,25 @@ public final class FileUtils
      * @param dir Directory to be deleted
      * @throws FSWriteError if any part of the tree cannot be deleted
      */
+    public static void deleteRecursiveWithThrottle(File dir, RateLimiter rateLimiter)
+    {
+        if (dir.isDirectory())
+        {
+            String[] children = dir.list();
+            for (String child : children)
+                deleteRecursiveWithThrottle(new File(dir, child), rateLimiter);
+        }
+
+        // The directory is now empty so now it can be smoked
+        deleteWithConfirmWithThrottle(dir, rateLimiter);
+    }
+
+
+    /**
+     * Deletes all files and subdirectories under "dir".
+     * @param dir Directory to be deleted
+     * @throws FSWriteError if any part of the tree cannot be deleted
+     */
     public static void deleteRecursive(File dir)
     {
         if (dir.isDirectory())
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 47f82b8..ca2bff2 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -3621,8 +3621,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 throw new IOException("Snapshot " + tag + " already exists.");
 
 
+        RateLimiter snapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter();
+
         for (Keyspace keyspace : keyspaces)
-            keyspace.snapshot(tag, null, skipFlush);
+            keyspace.snapshot(tag, null, skipFlush, snapshotRateLimiter);
     }
 
     /**
@@ -3682,10 +3684,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             }
         }
 
+        RateLimiter snapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter();
+
         for (Entry<Keyspace, List<String>> entry : keyspaceColumnfamily.entrySet())
         {
             for (String table : entry.getValue())
-                entry.getKey().snapshot(tag, table, skipFlush);
+                entry.getKey().snapshot(tag, table, skipFlush, snapshotRateLimiter);
         }
 
     }
@@ -3773,6 +3777,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return total;
     }
 
+    public void setSnapshotLinksPerSecond(long throttle)
+    {
+        logger.info("Setting snapshot throttle to {}", throttle);
+        DatabaseDescriptor.setSnapshotLinksPerSecond(throttle);
+    }
+
+    public long getSnapshotLinksPerSecond()
+    {
+        return DatabaseDescriptor.getSnapshotLinksPerSecond();
+    }
+
     public void refreshSizeEstimates() throws ExecutionException
     {
         cleanupSizeEstimates();
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 58402e1..618a37e 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -267,6 +267,22 @@ public interface StorageServiceMBean extends NotificationEmitter
     public long trueSnapshotsSize();
 
     /**
+     * Set the current hardlink-per-second throttle for snapshots
+     * A setting of zero indicates no throttling
+     *
+     * @param throttle
+     */
+    public void setSnapshotLinksPerSecond(long throttle);
+
+    /**
+     * Get the current hardlink-per-second throttle for snapshots
+     * A setting of zero indicates no throttling.
+     *
+     * @return snapshot links-per-second throttle
+     */
+    public long getSnapshotLinksPerSecond();
+
+    /**
      * Forces refresh of values stored in system.size_estimates of all column families.
      */
     public void refreshSizeEstimates() throws ExecutionException;
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 16d7f73..1697325 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -641,6 +641,16 @@ public class NodeProbe implements AutoCloseable
         return memProxy.getHeapMemoryUsage();
     }
 
+    public long getSnapshotLinksPerSecond()
+    {
+        return ssProxy.getSnapshotLinksPerSecond();
+    }
+
+    public void setSnapshotLinksPerSecond(long throttle)
+    {
+        ssProxy.setSnapshotLinksPerSecond(throttle);
+    }
+
     /**
      * Take a snapshot of all the keyspaces, optionally specifying only a specific column family.
      *
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index c3cebd8..b796e02 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -174,6 +174,8 @@ public class NodeTool
                 SetMaxHintWindow.class,
                 Snapshot.class,
                 ListSnapshots.class,
+                GetSnapshotThrottle.class,
+                SetSnapshotThrottle.class,
                 Status.class,
                 StatusBinary.class,
                 StatusGossip.class,
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java b/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java
new file mode 100644
index 0000000..0e9bdc1
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetSnapshotThrottle.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getsnapshotthrottle", description = "Print the snapshot_links_per_second throttle for snapshot/clearsnapshot")
+public class GetSnapshotThrottle extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        long throttle = probe.getSnapshotLinksPerSecond();
+        if (throttle > 0)
+            System.out.println("Current snapshot throttle: " + throttle + " links/s");
+        else
+            System.out.println("Snapshot throttle is disabled");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java b/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java
new file mode 100644
index 0000000..a4c49b8
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetSnapshotThrottle.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setsnapshotthrottle", description = "Set the snapshot_links_per_second cap for snapshot and clearsnapshot throttling")
+public class SetSnapshotThrottle extends NodeToolCmd
+{
+    @Arguments(title = "setsnapshotthrottle", usage = "<throttle>", description = "Value represents hardlinks per second ( snapshot_links_per_second ) , 0 to disable throttling", required = true)
+    private Long snapshotThrottle = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setSnapshotLinksPerSecond(snapshotThrottle);
+    }
+}
\ No newline at end of file


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