Repository: cassandra
Updated Branches:
  refs/heads/trunk c277fc56b -> f3609995c


Correct the messaging versions used in ThirdHandshakeMessage.

patch by jasobrown; reviewed by Ariel Weisberg for CASSANDRA-14896


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

Branch: refs/heads/trunk
Commit: f3609995c09570d523527d9bd0fd69c2bc65d986
Parents: c277fc5
Author: Jason Brown <jasedbr...@gmail.com>
Authored: Thu Nov 29 14:19:02 2018 -0800
Committer: Jason Brown <jasedbr...@gmail.com>
Committed: Fri Nov 30 13:16:07 2018 -0800

----------------------------------------------------------------------
 .../org/apache/cassandra/net/async/HandshakeProtocol.java    | 8 +++++++-
 .../apache/cassandra/net/async/OutboundHandshakeHandler.java | 2 +-
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3609995/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java 
b/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
index 327b20e..ebf26bd 100644
--- a/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
+++ b/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java
@@ -214,6 +214,7 @@ public class HandshakeProtocol
      * The third message of the handshake, sent by the connection initiator on 
reception of {@link SecondHandshakeMessage}.
      * This message contains:
      *   1) the connection initiator's messaging version (4 bytes) - {@link 
org.apache.cassandra.net.MessagingService#current_version}.
+     *      This indicates the max messaging version supported by this node.
      *   2) the connection initiator's broadcast address as encoded by {@link 
org.apache.cassandra.net.CompactEndpointSerializationHelper}.
      *      This can be either 5 bytes for an IPv4 address, or 17 bytes for an 
IPv6 one.
      * <p>
@@ -230,6 +231,9 @@ public class HandshakeProtocol
          */
         private static final int MIN_LENGTH = 9;
 
+        /**
+         * The internode messaging version of the peer; used for serializing 
to a version the peer understands.
+         */
         final int messagingVersion;
         final InetAddressAndPort address;
 
@@ -245,7 +249,9 @@ public class HandshakeProtocol
             int bufLength = Ints.checkedCast(Integer.BYTES + 
CompactEndpointSerializationHelper.instance.serializedSize(address, 
messagingVersion));
             ByteBuf buffer = allocator.directBuffer(bufLength, bufLength);
             buffer.writerIndex(0);
-            buffer.writeInt(messagingVersion);
+
+            // the max messaging version supported by the local node (not 
#messagingVersion)
+            buffer.writeInt(MessagingService.current_version);
             try
             {
                 DataOutputPlus dop = new ByteBufDataOutputPlus(buffer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3609995/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java 
b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
index c555bed..3ccbf49 100644
--- a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
+++ b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java
@@ -177,7 +177,7 @@ public class OutboundHandshakeHandler extends 
ByteToMessageDecoder
 
         try
         {
-            ctx.writeAndFlush(new 
ThirdHandshakeMessage(MessagingService.current_version, 
connectionId.local()).encode(ctx.alloc()));
+            ctx.writeAndFlush(new ThirdHandshakeMessage(peerMessagingVersion, 
connectionId.local()).encode(ctx.alloc()));
             ChannelWriter channelWriter = setupPipeline(ctx.channel(), 
peerMessagingVersion);
             callback.accept(HandshakeResult.success(channelWriter, 
peerMessagingVersion));
         }


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

Reply via email to