You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2016/01/06 23:43:16 UTC

cassandra git commit: Add nodetool gettimeout and settimeout commands

Repository: cassandra
Updated Branches:
  refs/heads/trunk 9a88f8550 -> 85ad12275


Add nodetool gettimeout and settimeout commands

Patch by Jeremy Hanna; reviewed by Tyler Hobbs for CASSANDRA-10953


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

Branch: refs/heads/trunk
Commit: 85ad12275f5012ed77af4ac4b0921d95e74a7d8e
Parents: 9a88f85
Author: Jeremy Hanna <je...@gmail.com>
Authored: Wed Jan 6 16:42:32 2016 -0600
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Wed Jan 6 16:42:32 2016 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  5 ++
 .../cassandra/config/DatabaseDescriptor.java    |  5 ++
 .../cassandra/service/StorageService.java       | 88 ++++++++++++++++++++
 .../cassandra/service/StorageServiceMBean.java  | 24 ++++++
 .../org/apache/cassandra/tools/NodeProbe.java   | 64 ++++++++++++++
 .../org/apache/cassandra/tools/NodeTool.java    |  2 +
 .../cassandra/tools/nodetool/GetTimeout.java    | 52 ++++++++++++
 .../cassandra/tools/nodetool/SetTimeout.java    | 55 ++++++++++++
 8 files changed, 295 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index db336e5..d684dcf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,8 +1,13 @@
+3.4
+ * Add nodetool gettimeout and settimeout commands (CASSANDRA-10953)
+
+
 3.3
 Merged from 3.0:
  * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
  * Fix potential assertion error during compaction (CASSANDRA-10944)
 
+
 3.2
  * Make sure tokens don't exist in several data directories (CASSANDRA-6696)
  * Add requireAuthorization method to IAuthorizer (CASSANDRA-10852)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/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 c82e930..116d92e 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1794,6 +1794,11 @@ public class DatabaseDescriptor
         conf.counter_cache_keys_to_save = counterCacheKeysToSave;
     }
 
+    public static void setStreamingSocketTimeout(int value)
+    {
+        conf.streaming_socket_timeout_in_ms = value;
+    }
+
     public static int getStreamingSocketTimeout()
     {
         return conf.streaming_socket_timeout_in_ms;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 24bebae..d4cd738 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -1137,6 +1137,94 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
     }
 
+    public void setRpcTimeout(long value)
+    {
+        DatabaseDescriptor.setRpcTimeout(value);
+        logger.info("set rpc timeout to {} ms", value);
+    }
+
+    public long getRpcTimeout()
+    {
+        return DatabaseDescriptor.getRpcTimeout();
+    }
+
+    public void setReadRpcTimeout(long value)
+    {
+        DatabaseDescriptor.setReadRpcTimeout(value);
+        logger.info("set read rpc timeout to {} ms", value);
+    }
+
+    public long getReadRpcTimeout()
+    {
+        return DatabaseDescriptor.getReadRpcTimeout();
+    }
+
+    public void setRangeRpcTimeout(long value)
+    {
+        DatabaseDescriptor.setRangeRpcTimeout(value);
+        logger.info("set range rpc timeout to {} ms", value);
+    }
+
+    public long getRangeRpcTimeout()
+    {
+        return DatabaseDescriptor.getRangeRpcTimeout();
+    }
+
+    public void setWriteRpcTimeout(long value)
+    {
+        DatabaseDescriptor.setWriteRpcTimeout(value);
+        logger.info("set write rpc timeout to {} ms", value);
+    }
+
+    public long getWriteRpcTimeout()
+    {
+        return DatabaseDescriptor.getWriteRpcTimeout();
+    }
+
+    public void setCounterWriteRpcTimeout(long value)
+    {
+        DatabaseDescriptor.setCounterWriteRpcTimeout(value);
+        logger.info("set counter write rpc timeout to {} ms", value);
+    }
+
+    public long getCounterWriteRpcTimeout()
+    {
+        return DatabaseDescriptor.getCounterWriteRpcTimeout();
+    }
+
+    public void setCasContentionTimeout(long value)
+    {
+        DatabaseDescriptor.setCasContentionTimeout(value);
+        logger.info("set cas contention rpc timeout to {} ms", value);
+    }
+
+    public long getCasContentionTimeout()
+    {
+        return DatabaseDescriptor.getCasContentionTimeout();
+    }
+
+    public void setTruncateRpcTimeout(long value)
+    {
+        DatabaseDescriptor.setTruncateRpcTimeout(value);
+        logger.info("set truncate rpc timeout to {} ms", value);
+    }
+
+    public long getTruncateRpcTimeout()
+    {
+        return DatabaseDescriptor.getTruncateRpcTimeout();
+    }
+
+    public void setStreamingSocketTimeout(int value)
+    {
+        DatabaseDescriptor.setStreamingSocketTimeout(value);
+        logger.info("set streaming socket timeout to {} ms", value);
+    }
+
+    public int getStreamingSocketTimeout()
+    {
+        return DatabaseDescriptor.getStreamingSocketTimeout();
+    }
+
     public void setStreamThroughputMbPerSec(int value)
     {
         DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(value);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index eef34c0..4328544 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -483,6 +483,30 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void joinRing() throws IOException;
     public boolean isJoined();
 
+    public void setRpcTimeout(long value);
+    public long getRpcTimeout();
+
+    public void setReadRpcTimeout(long value);
+    public long getReadRpcTimeout();
+
+    public void setRangeRpcTimeout(long value);
+    public long getRangeRpcTimeout();
+
+    public void setWriteRpcTimeout(long value);
+    public long getWriteRpcTimeout();
+
+    public void setCounterWriteRpcTimeout(long value);
+    public long getCounterWriteRpcTimeout();
+
+    public void setCasContentionTimeout(long value);
+    public long getCasContentionTimeout();
+
+    public void setTruncateRpcTimeout(long value);
+    public long getTruncateRpcTimeout();
+
+    public void setStreamingSocketTimeout(int value);
+    public int getStreamingSocketTimeout();
+
     public void setStreamThroughputMbPerSec(int value);
     public int getStreamThroughputMbPerSec();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 6f1c753..e121192 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -89,6 +89,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.cassandra.tools.nodetool.GetTimeout;
 
 /**
  * JMX client operations for Cassandra.
@@ -957,6 +958,31 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.getCompactionThroughputMbPerSec();
     }
 
+    public long getTimeout(String type)
+    {
+        switch (type)
+        {
+            case "misc":
+                return ssProxy.getRpcTimeout();
+            case "read":
+                return ssProxy.getReadRpcTimeout();
+            case "range":
+                return ssProxy.getRangeRpcTimeout();
+            case "write":
+                return ssProxy.getWriteRpcTimeout();
+            case "counterwrite":
+                return ssProxy.getCounterWriteRpcTimeout();
+            case "cascontention":
+                return ssProxy.getCasContentionTimeout();
+            case "truncate":
+                return ssProxy.getTruncateRpcTimeout();
+            case "streamingsocket":
+                return (long) ssProxy.getStreamingSocketTimeout();
+            default:
+                throw new RuntimeException("Timeout type requires one of (" + GetTimeout.TIMEOUT_TYPES + ")");
+        }
+    }
+
     public int getStreamThroughput()
     {
         return ssProxy.getStreamThroughputMbPerSec();
@@ -994,6 +1020,44 @@ public class NodeProbe implements AutoCloseable
         compactionProxy.stopCompaction(string);
     }
 
+    public void setTimeout(String type, long value)
+    {
+        if (value < 0)
+            throw new RuntimeException("timeout must be non-negative");
+
+        switch (type)
+        {
+            case "misc":
+                ssProxy.setRpcTimeout(value);
+                break;
+            case "read":
+                ssProxy.setReadRpcTimeout(value);
+                break;
+            case "range":
+                ssProxy.setRangeRpcTimeout(value);
+                break;
+            case "write":
+                ssProxy.setWriteRpcTimeout(value);
+                break;
+            case "counterwrite":
+                ssProxy.setCounterWriteRpcTimeout(value);
+                break;
+            case "cascontention":
+                ssProxy.setCasContentionTimeout(value);
+                break;
+            case "truncate":
+                ssProxy.setTruncateRpcTimeout(value);
+                break;
+            case "streamingsocket":
+                if (value > Integer.MAX_VALUE)
+                    throw new RuntimeException("streamingsocket timeout must be less than " + Integer.MAX_VALUE);
+                ssProxy.setStreamingSocketTimeout((int) value);
+                break;
+            default:
+                throw new RuntimeException("Timeout type requires one of (" + GetTimeout.TIMEOUT_TYPES + ")");
+        }
+    }
+
     public void stopById(String compactionId)
     {
         compactionProxy.stopCompactionById(compactionId);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 9728356..22a6006 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -79,6 +79,7 @@ public class NodeTool
                 GcStats.class,
                 GetCompactionThreshold.class,
                 GetCompactionThroughput.class,
+                GetTimeout.class,
                 GetStreamThroughput.class,
                 GetTraceProbability.class,
                 GetEndpoints.class,
@@ -102,6 +103,7 @@ public class NodeTool
                 SetHintedHandoffThrottleInKB.class,
                 SetCompactionThreshold.class,
                 SetCompactionThroughput.class,
+                SetTimeout.class,
                 SetStreamThroughput.class,
                 SetTraceProbability.class,
                 Snapshot.class,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java b/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
new file mode 100644
index 0000000..b12c9a7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
@@ -0,0 +1,52 @@
+/*
+ * 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.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@Command(name = "gettimeout", description = "Print the timeout of the given type in ms")
+public class GetTimeout extends NodeToolCmd
+{
+    public static final String TIMEOUT_TYPES = "read, range, write, counterwrite, cascontention, truncate, streamingsocket, misc (general rpc_timeout_in_ms)";
+
+    @Arguments(usage = "<timeout_type>", description = "The timeout type, one of (" + TIMEOUT_TYPES + ")")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 1, "gettimeout requires a timeout type, one of (" + TIMEOUT_TYPES + ")");
+        try
+        {
+            System.out.println("Current timeout for type " + args.get(0) + ": " + probe.getTimeout(args.get(0)) + " ms");
+        } catch (Exception e)
+        {
+            throw new IllegalArgumentException(e.getMessage());
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/85ad1227/src/java/org/apache/cassandra/tools/nodetool/SetTimeout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetTimeout.java b/src/java/org/apache/cassandra/tools/nodetool/SetTimeout.java
new file mode 100644
index 0000000..0b99efd
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetTimeout.java
@@ -0,0 +1,55 @@
+/*
+ * 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.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@Command(name = "settimeout", description = "Set the specified timeout in ms, or 0 to disable timeout")
+public class SetTimeout extends NodeToolCmd
+{
+    @Arguments(usage = "<timeout_type> <timeout_in_ms>", description = "Timeout type followed by value in ms " +
+            "(0 disables socket streaming timeout). Type should be one of (" + GetTimeout.TIMEOUT_TYPES + ")",
+            required = true)
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 2, "Timeout type followed by value in ms (0 disables socket streaming timeout)." +
+                " Type should be one of (" + GetTimeout.TIMEOUT_TYPES + ")");
+
+        try
+        {
+            String type = args.get(0);
+            long timeout = Long.parseLong(args.get(1));
+            probe.setTimeout(type, timeout);
+        } catch (Exception e)
+        {
+            throw new IllegalArgumentException(e.getMessage());
+        }
+    }
+}