Cleanup StartupClusterConnectivityChecker and PING Verb patch by jasobrown; reviewed by Ariel Weisberg for CASSANDRA-14447
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/06b3521a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/06b3521a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/06b3521a Branch: refs/heads/cassandra-3.11 Commit: 06b3521acdb21dd3d85902d59146b9d08ad7d752 Parents: 6efa99c Author: Jason Brown <jasedbr...@gmail.com> Authored: Fri Apr 6 11:47:35 2018 -0700 Committer: Jason Brown <jasedbr...@gmail.com> Committed: Thu May 17 10:34:31 2018 -0700 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/net/MessagingService.java | 15 ++++- .../org/apache/cassandra/net/PingMessage.java | 60 ++++++++++++++++++++ 3 files changed, 73 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/06b3521a/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 3da808a..1293bd4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.17 + * Cleanup StartupClusterConnectivityChecker and PING Verb (CASSANDRA-14447) * Fix deprecated repair error notifications from 3.x clusters to legacy JMX clients (CASSANDRA-13121) * Cassandra not starting when using enhanced startup scripts in windows (CASSANDRA-14418) * Fix progress stats and units in compactionstats (CASSANDRA-12244) http://git-wip-us.apache.org/repos/asf/cassandra/blob/06b3521a/src/java/org/apache/cassandra/net/MessagingService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java index e0f77b7..047f51f 100644 --- a/src/java/org/apache/cassandra/net/MessagingService.java +++ b/src/java/org/apache/cassandra/net/MessagingService.java @@ -147,13 +147,20 @@ public final class MessagingService implements MessagingServiceMBean PAXOS_PROPOSE, PAXOS_COMMIT, @Deprecated PAGED_RANGE, - // remember to add new verbs at the end, since we serialize by ordinal - UNUSED_1, + PING, + + // UNUSED verbs were used as padding for backward/forward compatability before 4.0, + // but it wasn't quite as bullet/future proof as needed. We still need to keep these entries + // around, at least for a major rev or two (post-4.0). see CASSANDRA-13993 for a discussion. + // For now, though, the UNUSED are legacy values (placeholders, basically) that should only be used + // for correctly adding VERBs that need to be emergency additions to 3.0/3.11. + // We can reclaim them (their id's, to be correct) in future versions, if desired, though. UNUSED_2, UNUSED_3, UNUSED_4, UNUSED_5, ; + // remember to add new verbs at the end, since we serialize by ordinal // This is to support a "late" choice of the verb based on the messaging service version. // See CASSANDRA-12249 for more details. @@ -210,9 +217,10 @@ public final class MessagingService implements MessagingServiceMBean put(Verb.SNAPSHOT, Stage.MISC); put(Verb.ECHO, Stage.GOSSIP); - put(Verb.UNUSED_1, Stage.INTERNAL_RESPONSE); put(Verb.UNUSED_2, Stage.INTERNAL_RESPONSE); put(Verb.UNUSED_3, Stage.INTERNAL_RESPONSE); + + put(Verb.PING, Stage.READ); }}; /** @@ -251,6 +259,7 @@ public final class MessagingService implements MessagingServiceMBean put(Verb.HINT, HintMessage.serializer); put(Verb.BATCH_STORE, Batch.serializer); put(Verb.BATCH_REMOVE, UUIDSerializer.serializer); + put(Verb.PING, PingMessage.serializer); }}; /** http://git-wip-us.apache.org/repos/asf/cassandra/blob/06b3521a/src/java/org/apache/cassandra/net/PingMessage.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/net/PingMessage.java b/src/java/org/apache/cassandra/net/PingMessage.java new file mode 100644 index 0000000..8eaf23e --- /dev/null +++ b/src/java/org/apache/cassandra/net/PingMessage.java @@ -0,0 +1,60 @@ +/* + * 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.net; + +import java.io.IOException; + +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +/** + * A backport of the version from 4.0, intentionnaly added as versions 4.0 or greater aren't guaranteed + * to know the c* versions they communicate with before they connect. + * + * It is intentional that no {@link IVerbHandler} is provided as we do not want process the message; + * the intent is to not break the stream by leaving it in an unclean state, with unconsumed bytes. + * We do, however, assign a {@link org.apache.cassandra.concurrent.StageManager} stage + * to maintain proper message flow. + * See CASSANDRA-13393 for a discussion. + */ +public class PingMessage +{ + public static IVersionedSerializer<PingMessage> serializer = new PingMessageSerializer(); + + public static class PingMessageSerializer implements IVersionedSerializer<PingMessage> + { + public void serialize(PingMessage t, DataOutputPlus out, int version) + { + throw new UnsupportedOperationException(); + } + + public PingMessage deserialize(DataInputPlus in, int version) throws IOException + { + // throw away the one byte of the payload + in.readByte(); + return new PingMessage(); + } + + public long serializedSize(PingMessage t, int version) + { + return 1; + } + } +} \ No newline at end of file --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org