[
https://issues.apache.org/jira/browse/CASSANDRA-18109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17645649#comment-17645649
]
Ke Han edited comment on CASSANDRA-18109 at 12/10/22 5:04 PM:
--------------------------------------------------------------
Thanks for the reply!
[~jmckenzie] The
[code|https://github.com/apache/cassandra/blob/cassandra-3.11.14/src/java/org/apache/cassandra/schema/SchemaKeyspace.java#L437-L450]
indeed tries to skip cdc column during the Schema serialization at [line
448|https://github.com/apache/cassandra/blob/cassandra-3.11.14/src/java/org/apache/cassandra/schema/SchemaKeyspace.java#L448].
But this issue is different. It happens during rolling upgrade when N0 and N1
are in 3.11.14 with cdc enabled, while N2 is in 3.0.28 with default settings
(cdc disabled).
Therefore, the function actually returns at [line
441|https://github.com/apache/cassandra/blob/cassandra-3.11.14/src/java/org/apache/cassandra/schema/SchemaKeyspace.java#L441]
for the 3.11.14 nodes. cdc column is still serialized and sent to 3.0.28
nodes. That causes the error.
[~brandon.williams] Thanks for the suggestions! I agree we shouldn't enable new
features until the cluster upgrade is completed. It might cause incompatibility
between versions.
was (Author: JIRAUSER289562):
Thanks for the reply!
[~jmckenzie] The
[code|https://github.com/apache/cassandra/blob/cassandra-3.11.14/src/java/org/apache/cassandra/schema/SchemaKeyspace.java#L437-L450]
indeed tries to skip cdc column during the Schema serialization at [line
448|https://github.com/apache/cassandra/blob/cassandra-3.11.14/src/java/org/apache/cassandra/schema/SchemaKeyspace.java#L448].
But this issue is different. It happens during rolling upgrade when N0 and N1
are in 3.11.14 with cdc enabled, while N2 is in 3.0.28 with default settings
(cdc disabled).
Therefore, the function actually returns at [line
441|https://github.com/apache/cassandra/blob/cassandra-3.11.14/src/java/org/apache/cassandra/schema/SchemaKeyspace.java#L441]
for the 3.11.14 nodes. cdc column is still serialized and sent to 3.0.28
nodes. That will cause the error log.
[~brandon.williams] Thanks for the suggestions! I agree we shouldn't enable new
features until the cluster upgrade is completed. It might cause incompatibility
between versions.
> Unknown column cdc during deserialization ERROR when upgrading from 3.0.28 to
> 3.11.14
> -------------------------------------------------------------------------------------
>
> Key: CASSANDRA-18109
> URL: https://issues.apache.org/jira/browse/CASSANDRA-18109
> Project: Cassandra
> Issue Type: Bug
> Reporter: Ke Han
> Priority: Normal
> Attachments: N0-system.log, N1-system.log, N2-system.log
>
>
> When we performed a rolling upgrade from 3.0.28 to 3.11.14, we encountered
> the following error message during the upgrade process. It doesn't prevent
> the upgrade process, but it's in ERROR log level and causes concerns for the
> upgrade process.
> {code:java}
> ERROR [MessagingService-Incoming-/192.168.107.2] 2022-12-10 04:01:12,449
> CassandraDaemon.java:265 - Exception in thread
> Thread[MessagingService-Incoming-/192.168.107.2,5,main]
> java.lang.RuntimeException: Unknown column cdc during deserialization
> at
> org.apache.cassandra.db.Columns$Serializer.deserialize(Columns.java:445)
> at
> org.apache.cassandra.db.SerializationHeader$Serializer.deserializeForMessaging(SerializationHeader.java:446)
> at
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.deserializeHeader(UnfilteredRowIteratorSerializer.java:190)
> at
> org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize30(PartitionUpdate.java:734)
> at
> org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:722)
> at
> org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:337)
> at
> org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:346)
> at
> org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:574)
> at
> org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:557)
> at org.apache.cassandra.net.MessageIn.read(MessageIn.java:98)
> at
> org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:216)
> at
> org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:193)
> at
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:96)
> INFO [MigrationStage:1] 2022-12-10 04:01:13,394
> MigrationCoordinator.java:523 - Sending schema pull request to /192.168.107.2
> at 1670644873394 with timeout 10000
> ERROR [MessagingService-Incoming-/192.168.107.2] 2022-12-10 04:01:13,427
> CassandraDaemon.java:265 - Exception in thread
> Thread[MessagingService-Incoming-/192.168.107.2,5,main]
> java.lang.RuntimeException: Unknown column cdc during deserialization
> at
> org.apache.cassandra.db.Columns$Serializer.deserialize(Columns.java:445)
> at
> org.apache.cassandra.db.SerializationHeader$Serializer.deserializeForMessaging(SerializationHeader.java:446)
> at
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.deserializeHeader(UnfilteredRowIteratorSerializer.java:190)
> at
> org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize30(PartitionUpdate.java:734)
> at
> org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:722)
> at
> org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:337)
> at
> org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:346)
> at
> org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:574)
> at
> org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:557)
> at org.apache.cassandra.net.MessageIn.read(MessageIn.java:98)
> at
> org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:216)
> at
> org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:193)
> at
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:96){code}
> h1. Steps to reproduce
> Configure a 3-node Cassandra cluster (N0, N1 and N2) in 3.0.28 using the
> default configuration setting. Configure N0 as the seed node.
> For the new version configuration (3.11.14), set *cdc_enabled: true* in
> cassandra.yaml. All the other configurations use default values.
> Perform a rolling upgrade.
> # Start up the 3-node cluster in 3.0.28.
> # Drain, shutdown and upgrade N0 to 3.11.14.
> # Drain, shutdown and upgrade N1 to 3.11.14.
> # Drain, shutdown and upgrade N2 to 3.11.14.
> The +Unknown column cdc ERROR message+ occurs in *N2* once both N0 and N1 are
> upgraded.
> I attached the system.log for our three-node cluster.
> * N0 (Seed): 192.168.86.2
> * N1: 192.168.86.3
> * N2: 192.168.86.4
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]