Merge branch 'cassandra-2.2' into cassandra-3.0

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

Branch: refs/heads/trunk
Commit: c9d2b7addcfefb4cfa2bfcb96215153030d9d624
Parents: 232e12b b9ff7fe
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Mar 10 13:01:30 2016 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 10 13:01:30 2016 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 conf/cassandra.yaml                             |  7 +++----
 .../net/IncomingStreamingConnection.java        |  5 +++++
 .../cassandra/streaming/ConnectionHandler.java  | 21 +++++++++++++-------
 .../cassandra/streaming/StreamSession.java      |  2 ++
 5 files changed, 25 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9d2b7ad/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 7b93083,045b867..4c00994
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -56,8 -26,8 +56,9 @@@ Merged from 2.2
   * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
   * Fix paging on DISTINCT queries repeats result when first row in partition 
changes
     (CASSANDRA-10010)
 + * cqlsh: change default encoding to UTF-8 (CASSANDRA-11124)
  Merged from 2.1:
+  * Fix streaming_socket_timeout_in_ms not enforced (CASSANDRA-11286)
   * Avoid dropping message too quickly due to missing unit conversion 
(CASSANDRA-11302)
   * COPY FROM on large datasets: fix progress report and debug performance 
(CASSANDRA-11053)
   * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9d2b7ad/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
index e58d227,842676d..19e0671
--- a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
@@@ -25,8 -27,7 +25,9 @@@ import java.util.Set
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.streaming.StreamResultFuture;
  import org.apache.cassandra.streaming.messages.StreamInitMessage;
  import org.apache.cassandra.streaming.messages.StreamMessage;
@@@ -60,9 -60,13 +61,13 @@@ public class IncomingStreamingConnectio
              if (version != StreamMessage.CURRENT_VERSION)
                  throw new IOException(String.format("Received stream using 
protocol version %d (my version %d). Terminating connection", version, 
MessagingService.current_version));
  
 -            DataInput input = new DataInputStream(socket.getInputStream());
 +            DataInputPlus input = new 
DataInputStreamPlus(socket.getInputStream());
              StreamInitMessage init = 
StreamInitMessage.serializer.deserialize(input, version);
  
+             //Set SO_TIMEOUT on follower side
+             if (!init.isForOutgoing)
+                 
socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
+ 
              // The initiator makes two connections, one for incoming and one 
for outgoing.
              // The receiving side distinguish two connections by looking at 
StreamInitMessage#isForOutgoing.
              // Note: we cannot use the same socket for incoming and outgoing 
streams because we want to

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9d2b7ad/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------

Reply via email to