cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aweisb...@apache.org
Subject cassandra git commit: Partitioned outbound internode TCP connections can occur when nodes restart
Date Thu, 01 Nov 2018 22:36:08 GMT
Repository: cassandra
Updated Branches:
  refs/heads/trunk 877b08eaf -> bfbc5274f


Partitioned outbound internode TCP connections can occur when nodes restart

Patch by Joseph Lynch; Reviewed by Ariel Weisberg for CASSANDRA-14358


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

Branch: refs/heads/trunk
Commit: bfbc5274f2b3a5af2cbbe9679f0e78f1066ef638
Parents: 877b08e
Author: Joseph Lynch <joe.e.lynch@gmail.com>
Authored: Sat Apr 7 12:05:37 2018 -0700
Committer: Ariel Weisberg <aweisberg@apple.com>
Committed: Thu Nov 1 17:35:23 2018 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  8 +++++
 conf/cassandra.yaml                             | 12 +++++++
 .../org/apache/cassandra/config/Config.java     |  8 +++++
 .../cassandra/config/DatabaseDescriptor.java    | 20 +++++++++++
 .../cassandra/net/async/NettyFactory.java       |  9 ++++-
 .../net/async/OutboundConnectionParams.java     | 36 +++++++++++++++++---
 .../net/async/OutboundMessagingConnection.java  |  7 ++--
 .../cassandra/service/StorageService.java       | 22 ++++++++++++
 .../cassandra/service/StorageServiceMBean.java  |  6 ++++
 .../streaming/DefaultConnectionFactory.java     |  5 +++
 .../org/apache/cassandra/tools/NodeProbe.java   | 11 +++++-
 .../cassandra/tools/nodetool/GetTimeout.java    |  2 +-
 .../net/async/OutboundConnectionParamsTest.java | 35 +++++++++++++++++--
 14 files changed, 171 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b7c0398..41b3da9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Partitioned outbound internode TCP connections can occur when nodes restart (CASSANDRA-14358)
  * Don't write to system_distributed.repair_history, system_traces.sessions, system_traces.events
in mixed version 3.X/4.0 clusters (CASSANDRA-14841)
  * Avoid running query to self through messaging service (CASSANDRA-14807)
  * Allow using custom script for chronicle queue BinLog archival (CASSANDRA-14373)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3267c91..0d211a3 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -185,6 +185,14 @@ Upgrading
     - Background repair has been removed. dclocal_read_repair_chance and
       read_repair_chance table options have been removed and are now rejected.
       See CASSANDRA-13910 for details.
+    - Internode TCP connections that do not ack segments for 30s will now
+      be automatically detected and closed via the Linux TCP_USER_TIMEOUT
+      socket option. This should be exceedingly rare, but AWS networks (and
+      other stateful firewalls) apparently suffer from this issue. You can
+      tune the timeouts on TCP connection and segment ack via the
+      `cassandra.yaml:internode_tcp_connect_timeout_in_ms` and
+      `cassandra.yaml:internode_tcp_user_timeout_in_ms` options respectively.
+      See CASSANDRA-14358 for details.
 
 Materialized Views
 -------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index d72b2e8..0a92d4c 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -847,6 +847,18 @@ truncate_request_timeout_in_ms: 60000
 # Lowest acceptable value is 10 ms.
 request_timeout_in_ms: 10000
 
+# Defensive settings for protecting Cassandra from true network partitions.
+# See (CASSANDRA-14358) for details.
+#
+# The amount of time to wait for internode tcp connections to establish.
+# internode_tcp_connect_timeout_in_ms = 2000
+#
+# The amount of time unacknowledged data is allowed on a connection before we throw out the
connection
+# Note this is only supported on Linux + epoll, and it appears to behave oddly above a setting
of 30000
+# (it takes much longer than 30s) as of Linux 4.12. If you want something that high set this
to 0
+# which picks up the OS default and configure the net.ipv4.tcp_retries2 sysctl to be ~8.
+# internode_tcp_user_timeout_in_ms = 30000
+
 # How long before a node logs slow queries. Select queries that take longer than
 # this timeout to execute, will generate an aggregated log message, so that slow queries
 # can be identified. Set this value to zero to disable slow query logging.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/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 9049131..1e80108 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -145,6 +145,14 @@ public class Config
     public boolean rpc_keepalive = true;
     public int internode_send_buff_size_in_bytes = 0;
     public int internode_recv_buff_size_in_bytes = 0;
+    // Defensive settings for protecting Cassandra from true network partitions. See (CASSANDRA-14358)
for details.
+    // The amount of time to wait for internode tcp connections to establish.
+    public int internode_tcp_connect_timeout_in_ms = 2000;
+    // The amount of time unacknowledged data is allowed on a connection before we throw
out the connection
+    // Note this is only supported on Linux + epoll, and it appears to behave oddly above
a setting of 30000
+    // (it takes much longer than 30s) as of Linux 4.12. If you want something that high
set this to 0
+    // (which picks up the OS default) and configure the net.ipv4.tcp_retries2 sysctl to
be ~8.
+    public int internode_tcp_user_timeout_in_ms = 30000;
 
     public boolean start_native_transport = true;
     public int native_transport_port = 9042;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/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 de87de5..1b11a91 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1803,6 +1803,26 @@ public class DatabaseDescriptor
         return conf.internode_recv_buff_size_in_bytes;
     }
 
+    public static int getInternodeTcpConnectTimeoutInMS()
+    {
+        return conf.internode_tcp_connect_timeout_in_ms;
+    }
+
+    public static void setInternodeTcpConnectTimeoutInMS(int value)
+    {
+        conf.internode_tcp_connect_timeout_in_ms = value;
+    }
+
+    public static int getInternodeTcpUserTimeoutInMS()
+    {
+        return conf.internode_tcp_user_timeout_in_ms;
+    }
+
+    public static void setInternodeTcpUserTimeoutInMS(int value)
+    {
+        conf.internode_tcp_user_timeout_in_ms = value;
+    }
+
     public static boolean startNativeTransport()
     {
         return conf.start_native_transport;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/src/java/org/apache/cassandra/net/async/NettyFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/NettyFactory.java b/src/java/org/apache/cassandra/net/async/NettyFactory.java
index 5bbac45..989e33c 100644
--- a/src/java/org/apache/cassandra/net/async/NettyFactory.java
+++ b/src/java/org/apache/cassandra/net/async/NettyFactory.java
@@ -21,6 +21,7 @@ import io.netty.channel.ChannelOption;
 import io.netty.channel.ChannelPipeline;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.ServerChannel;
+import io.netty.channel.epoll.EpollChannelOption;
 import io.netty.channel.epoll.EpollEventLoopGroup;
 import io.netty.channel.epoll.EpollServerSocketChannel;
 import io.netty.channel.epoll.EpollSocketChannel;
@@ -200,6 +201,9 @@ public final class NettyFactory
                                                          .childOption(ChannelOption.SO_SNDBUF,
INBOUND_CHANNEL_SEND_BUFFER_SIZE)
                                                          .childHandler(initializer);
 
+        if (useEpoll)
+            bootstrap.childOption(EpollChannelOption.TCP_USER_TIMEOUT, DatabaseDescriptor.getInternodeTcpUserTimeoutInMS());
+
         if (receiveBufferSize > 0)
             bootstrap.childOption(ChannelOption.SO_RCVBUF, receiveBufferSize);
 
@@ -326,7 +330,7 @@ public final class NettyFactory
         Class<? extends Channel> transport = useEpoll ? EpollSocketChannel.class :
NioSocketChannel.class;
         Bootstrap bootstrap = new Bootstrap().group(params.mode == Mode.MESSAGING ? outboundGroup
: streamingGroup)
                               .channel(transport)
-                              .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 2000)
+                              .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, params.tcpConnectTimeoutInMS)
                               .option(ChannelOption.SO_KEEPALIVE, true)
                               .option(ChannelOption.SO_REUSEADDR, true)
                               .option(ChannelOption.SO_SNDBUF, params.sendBufferSize)
@@ -334,6 +338,9 @@ public final class NettyFactory
                               .option(ChannelOption.TCP_NODELAY, params.tcpNoDelay)
                               .option(ChannelOption.WRITE_BUFFER_WATER_MARK, params.waterMark)
                               .handler(new OutboundInitializer(params));
+        if (useEpoll)
+            bootstrap.option(EpollChannelOption.TCP_USER_TIMEOUT, params.tcpUserTimeoutInMS);
+
         InetAddressAndPort remoteAddress = params.connectionId.connectionAddress();
         bootstrap.remoteAddress(new InetSocketAddress(remoteAddress.address, remoteAddress.port));
         return bootstrap;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java b/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
index 282480e..64968c6 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java
@@ -25,6 +25,7 @@ import java.util.function.Supplier;
 import com.google.common.base.Preconditions;
 
 import io.netty.channel.WriteBufferWaterMark;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult;
 import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
@@ -48,6 +49,8 @@ public class OutboundConnectionParams
     final Consumer<MessageResult> messageResultConsumer;
     final WriteBufferWaterMark waterMark;
     final int protocolVersion;
+    final int tcpConnectTimeoutInMS;
+    final int tcpUserTimeoutInMS;
 
     private OutboundConnectionParams(OutboundConnectionIdentifier connectionId,
                                      Consumer<HandshakeResult> callback,
@@ -60,7 +63,9 @@ public class OutboundConnectionParams
                                      Supplier<QueuedMessage> backlogSupplier,
                                      Consumer<MessageResult> messageResultConsumer,
                                      WriteBufferWaterMark waterMark,
-                                     int protocolVersion)
+                                     int protocolVersion,
+                                     int tcpConnectTimeoutInMS,
+                                     int tcpUserTimeoutInMS)
     {
         this.connectionId = connectionId;
         this.callback = callback;
@@ -74,6 +79,8 @@ public class OutboundConnectionParams
         this.messageResultConsumer = messageResultConsumer;
         this.waterMark = waterMark;
         this.protocolVersion = protocolVersion;
+        this.tcpConnectTimeoutInMS = tcpConnectTimeoutInMS;
+        this.tcpUserTimeoutInMS = tcpUserTimeoutInMS;
     }
 
     public static Builder builder()
@@ -99,10 +106,15 @@ public class OutboundConnectionParams
         private Supplier<QueuedMessage> backlogSupplier;
         private Consumer<MessageResult> messageResultConsumer;
         private WriteBufferWaterMark waterMark = WriteBufferWaterMark.DEFAULT;
-        int protocolVersion;
+        private int protocolVersion;
+        private int tcpConnectTimeoutInMS;
+        private int tcpUserTimeoutInMS;
 
         private Builder()
-        {   }
+        {
+            this.tcpConnectTimeoutInMS = DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS();
+            this.tcpUserTimeoutInMS = DatabaseDescriptor.getInternodeTcpUserTimeoutInMS();
+        }
 
         private Builder(OutboundConnectionParams params)
         {
@@ -116,6 +128,8 @@ public class OutboundConnectionParams
             this.tcpNoDelay = params.tcpNoDelay;
             this.backlogSupplier = params.backlogSupplier;
             this.messageResultConsumer = params.messageResultConsumer;
+            this.tcpConnectTimeoutInMS = params.tcpConnectTimeoutInMS;
+            this.tcpUserTimeoutInMS = params.tcpUserTimeoutInMS;
         }
 
         public Builder connectionId(OutboundConnectionIdentifier connectionId)
@@ -190,13 +204,27 @@ public class OutboundConnectionParams
             return this;
         }
 
+        public Builder tcpConnectTimeoutInMS(int tcpConnectTimeoutInMS)
+        {
+            this.tcpConnectTimeoutInMS = tcpConnectTimeoutInMS;
+            return this;
+        }
+
+        public Builder tcpUserTimeoutInMS(int tcpUserTimeoutInMS)
+        {
+            this.tcpUserTimeoutInMS = tcpUserTimeoutInMS;
+            return this;
+        }
+
         public OutboundConnectionParams build()
         {
             Preconditions.checkArgument(protocolVersion > 0, "illegal protocol version:
" + protocolVersion);
             Preconditions.checkArgument(sendBufferSize > 0 && sendBufferSize <
1 << 20, "illegal send buffer size: " + sendBufferSize);
+            Preconditions.checkArgument(tcpUserTimeoutInMS >= 0, "tcp user timeout must
be non negative: " + tcpUserTimeoutInMS);
+            Preconditions.checkArgument(tcpConnectTimeoutInMS > 0, "tcp connect timeout
must be positive: " + tcpConnectTimeoutInMS);
 
             return new OutboundConnectionParams(connectionId, callback, encryptionOptions,
mode, compress, coalescingStrategy, sendBufferSize,
-                                                tcpNoDelay, backlogSupplier, messageResultConsumer,
waterMark, protocolVersion);
+                                                tcpNoDelay, backlogSupplier, messageResultConsumer,
waterMark, protocolVersion, tcpConnectTimeoutInMS, tcpUserTimeoutInMS);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java b/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
index 064131b..265ece9 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java
@@ -19,8 +19,6 @@
 package org.apache.cassandra.net.async;
 
 import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.util.Optional;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -323,6 +321,9 @@ public class OutboundMessagingConnection
                              ? DatabaseDescriptor.getInternodeSendBufferSize()
                              : OutboundConnectionParams.DEFAULT_SEND_BUFFER_SIZE;
 
+        int tcpConnectTimeout = DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS();
+        int tcpUserTimeout = DatabaseDescriptor.getInternodeTcpUserTimeoutInMS();
+
         OutboundConnectionParams params = OutboundConnectionParams.builder()
                                                                   .connectionId(connectionId)
                                                                   .callback(this::finishHandshake)
@@ -332,6 +333,8 @@ public class OutboundMessagingConnection
                                                                   .coalescingStrategy(coalescingStrategy)
                                                                   .sendBufferSize(sendBufferSize)
                                                                   .tcpNoDelay(tcpNoDelay)
+                                                                  .tcpConnectTimeoutInMS(tcpConnectTimeout)
+                                                                  .tcpUserTimeoutInMS(tcpUserTimeout)
                                                                   .backlogSupplier(() ->
nextBackloggedMessage())
                                                                   .messageResultConsumer(this::handleMessageResult)
                                                                   .protocolVersion(targetVersion)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/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 7cd99de..e7ca4be 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -1370,6 +1370,28 @@ public class StorageService extends NotificationBroadcasterSupport
implements IE
         return DatabaseDescriptor.getWriteRpcTimeout();
     }
 
+    public void setInternodeTcpConnectTimeoutInMS(int value)
+    {
+        DatabaseDescriptor.setInternodeTcpConnectTimeoutInMS(value);
+        logger.info("set internode tcp connect timeout to {} ms", value);
+    }
+
+    public int getInternodeTcpConnectTimeoutInMS()
+    {
+        return DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS();
+    }
+
+    public void setInternodeTcpUserTimeoutInMS(int value)
+    {
+        DatabaseDescriptor.setInternodeTcpUserTimeoutInMS(value);
+        logger.info("set internode tcp user timeout to {} ms", value);
+    }
+
+    public int getInternodeTcpUserTimeoutInMS()
+    {
+        return DatabaseDescriptor.getInternodeTcpUserTimeoutInMS();
+    }
+
     public void setCounterWriteRpcTimeout(long value)
     {
         DatabaseDescriptor.setCounterWriteRpcTimeout(value);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/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 4e6295a..d11e997 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -529,6 +529,12 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void setWriteRpcTimeout(long value);
     public long getWriteRpcTimeout();
 
+    public void setInternodeTcpConnectTimeoutInMS(int value);
+    public int getInternodeTcpConnectTimeoutInMS();
+
+    public void setInternodeTcpUserTimeoutInMS(int value);
+    public int getInternodeTcpUserTimeoutInMS();
+
     public void setCounterWriteRpcTimeout(long value);
     public long getCounterWriteRpcTimeout();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
index 38c25dc..b192803 100644
--- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
@@ -70,12 +70,17 @@ public class DefaultConnectionFactory implements StreamConnectionFactory
                              ? DatabaseDescriptor.getInternodeSendBufferSize()
                              : OutboundConnectionParams.DEFAULT_SEND_BUFFER_SIZE;
 
+        int tcpConnectTimeout = DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS();
+        int tcpUserTimeout = DatabaseDescriptor.getInternodeTcpUserTimeoutInMS();
+
         OutboundConnectionParams params = OutboundConnectionParams.builder()
                                                                   .connectionId(connectionId)
                                                                   .encryptionOptions(encryptionOptions)
                                                                   .mode(NettyFactory.Mode.STREAMING)
                                                                   .protocolVersion(protocolVersion)
                                                                   .sendBufferSize(sendBufferSize)
+                                                                  .tcpConnectTimeoutInMS(tcpConnectTimeout)
+                                                                  .tcpUserTimeoutInMS(tcpUserTimeout)
                                                                   .waterMark(waterMark)
                                                                   .build();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/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 54187d1..76cb7e8 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -90,7 +90,6 @@ import com.codahale.metrics.JmxReporter;
 import com.google.common.base.Function;
 import com.google.common.base.Strings;
 import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
@@ -1150,6 +1149,10 @@ public class NodeProbe implements AutoCloseable
                 return ssProxy.getCasContentionTimeout();
             case "truncate":
                 return ssProxy.getTruncateRpcTimeout();
+            case "internodeconnect":
+                return ssProxy.getInternodeTcpConnectTimeoutInMS();
+            case "internodeuser":
+                return ssProxy.getInternodeTcpUserTimeoutInMS();
             default:
                 throw new RuntimeException("Timeout type requires one of (" + GetTimeout.TIMEOUT_TYPES
+ ")");
         }
@@ -1234,6 +1237,12 @@ public class NodeProbe implements AutoCloseable
             case "truncate":
                 ssProxy.setTruncateRpcTimeout(value);
                 break;
+            case "internodeconnect":
+                ssProxy.setInternodeTcpConnectTimeoutInMS((int) value);
+                break;
+            case "internodeuser":
+                ssProxy.setInternodeTcpUserTimeoutInMS((int) value);
+                break;
             default:
                 throw new RuntimeException("Timeout type requires one of (" + GetTimeout.TIMEOUT_TYPES
+ ")");
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/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
index deac8a3..9f99ac6 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetTimeout.java
@@ -31,7 +31,7 @@ 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, misc (general rpc_timeout_in_ms)";
+    public static final String TIMEOUT_TYPES = "read, range, write, counterwrite, cascontention,
truncate, internodeconnect, internodeuser, misc (general rpc_timeout_in_ms)";
 
     @Arguments(usage = "<timeout_type>", description = "The timeout type, one of ("
+ TIMEOUT_TYPES + ")")
     private List<String> args = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfbc5274/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java b/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
index 0ce4968..23a4a68 100644
--- a/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
+++ b/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java
@@ -18,19 +18,50 @@
 
 package org.apache.cassandra.net.async;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+
 public class OutboundConnectionParamsTest
 {
+    static int version;
+
+    @BeforeClass
+    public static void before()
+    {
+        DatabaseDescriptor.daemonInitialization();
+        version = MessagingService.current_version;
+    }
+
     @Test (expected = IllegalArgumentException.class)
     public void build_SendSizeLessThanZero()
     {
-        OutboundConnectionParams.builder().sendBufferSize(-1).build();
+        OutboundConnectionParams.builder().protocolVersion(version).sendBufferSize(-1).build();
     }
 
     @Test (expected = IllegalArgumentException.class)
     public void build_SendSizeHuge()
     {
-        OutboundConnectionParams.builder().sendBufferSize(1 << 30).build();
+        OutboundConnectionParams.builder().protocolVersion(version).sendBufferSize(1 <<
30).build();
+    }
+
+    @Test (expected = IllegalArgumentException.class)
+    public void build_TcpConnectTimeoutLessThanZero()
+    {
+        OutboundConnectionParams.builder().protocolVersion(version).tcpConnectTimeoutInMS(-1).build();
+    }
+
+    @Test (expected = IllegalArgumentException.class)
+    public void build_TcpUserTimeoutLessThanZero()
+    {
+        OutboundConnectionParams.builder().protocolVersion(version).tcpUserTimeoutInMS(-1).build();
+    }
+
+    @Test
+    public void build_TcpUserTimeoutEqualsZero()
+    {
+        OutboundConnectionParams.builder().protocolVersion(version).tcpUserTimeoutInMS(0).build();
     }
 }


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


Mime
View raw message