Repository: cassandra Updated Branches: refs/heads/cassandra-3.X 4fd05e5b5 -> 78de13bcc refs/heads/trunk 394a47453 -> 6b9a8061f
Avoid shutting down socket input/output on StreamSession since this is not supported on SSL Patch by Paulo Motta; Reviewed by Sam Tunnicliffe for CASSANDRA-12903 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/78de13bc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/78de13bc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/78de13bc Branch: refs/heads/cassandra-3.X Commit: 78de13bcc7ea10f92d74a3db7ca8ba515904cbef Parents: 4fd05e5 Author: Paulo Motta <[email protected]> Authored: Mon Nov 14 11:02:47 2016 -0200 Committer: Sam Tunnicliffe <[email protected]> Committed: Tue Nov 15 10:50:08 2016 +0000 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../org/apache/cassandra/streaming/ConnectionHandler.java | 9 --------- 2 files changed, 1 insertion(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/78de13bc/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 78d6325..32f6d9b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.10 + * Don't shut down socket input/output on StreamSession (CASSANDRA-12903) * Fix Murmur3PartitionerTest (CASSANDRA-12858) * Move cqlsh syntax rules into separate module and allow easier customization (CASSANDRA-12897) * Fix CommitLogSegmentManagerTest (CASSANDRA-12283) http://git-wip-us.apache.org/repos/asf/cassandra/blob/78de13bc/src/java/org/apache/cassandra/streaming/ConnectionHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java index f2e0f9c..556748d 100644 --- a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java +++ b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java @@ -61,7 +61,6 @@ public class ConnectionHandler private static final Logger logger = LoggerFactory.getLogger(ConnectionHandler.class); private final StreamSession session; - private int incomingSocketTimeout; private IncomingMessageHandler incoming; private OutgoingMessageHandler outgoing; @@ -86,12 +85,10 @@ public class ConnectionHandler logger.debug("[Stream #{}] Sending stream init for incoming stream", session.planId()); Socket incomingSocket = session.createConnection(); incoming.start(incomingSocket, StreamMessage.CURRENT_VERSION, true); - incomingSocket.shutdownOutput(); logger.debug("[Stream #{}] Sending stream init for outgoing stream", session.planId()); Socket outgoingSocket = session.createConnection(); outgoing.start(outgoingSocket, StreamMessage.CURRENT_VERSION, true); - outgoingSocket.shutdownInput(); } /** @@ -104,15 +101,9 @@ public class ConnectionHandler public void initiateOnReceivingSide(IncomingStreamingConnection connection, boolean isForOutgoing, int version) throws IOException { if (isForOutgoing) - { outgoing.start(connection, version); - outgoing.socket.shutdownInput(); - } else - { incoming.start(connection, version); - incoming.socket.shutdownOutput(); - } } public ListenableFuture<?> close()
