[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17714255#comment-17714255 ] Abe Ratnofsky commented on CASSANDRA-18108: --- [~zagol] Thanks for providing more info on your fuzzer. Could we move that to a new ticket to avoid distracting from this particular issue: https://issues.apache.org/jira/browse/CASSANDRA-18468 > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17712920#comment-17712920 ] Yongle Zhang commented on CASSANDRA-18108: -- @[~aratnofsky] Hi Abe, Ke has been working on the fuzzer and has made a lot of progress very recently. We are wondering if you and the community are interested in it. Here is a general description of the fuzzer: # Architecture ## It sets up a docker container cluster running Cassandra inside a VM ([Nyx-Net|[https://github.com/RUB-SysSec/nyx-net])] that supports fast checkpoint and restore (<1 second with the memory footprint of a docker cluster). ## The fuzzer runs outside the VM, generates Cassandra cluster configurations, use these configurations to start up the cluster inside VM, and checkpoints a snapshot once C* cluster is ready to serve requests, then the fuzzer generates tests and send to each node in the docker cluster to be executed, and restores after a number of tests. # Feedback collector ## The fuzzer modifies Jacoco to collect branch coverage as feedback metric (our modification is to add branch count, which does not exist in Jacoco). We tested some other JVM feedback collector but they are slow. The modified Jacoco's instrumentation to performance is negligible. # Input generation and mutation ## The fuzzer uses grammar-based, type-aware input mutation: (1) each test it generates contains ~20 valid C* SQL commands, (2) mutating String and Collections have type-specific strategies (e.g., concatenation, collection.add()). # Fault injection ## It supports injecting (1) node crash, (2) network link failure between nodes, (3) network partition. # Error detection ## It detects crash, exception (new exceptions), error log (new error logs), and buckets the triggered failures. # Upgrade test: the fuzzer supports testing full-stop upgrade and rolling-upgrade with differential test oracles and upgrade-specific feedback metrics. > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17646651#comment-17646651 ] Aleksey Yeschenko commented on CASSANDRA-18108: --- [~maedhroz] It should be perfectly safe, seeing as you are essentially changing an alias for an component that is positionally set in stone. There is no reason to forbid it, and it has never been problematic. If there is a bug somewhere, it's the bug that needs to be fixed. > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17646228#comment-17646228 ] Caleb Rackliffe commented on CASSANDRA-18108: - [~aleksey] Am I thinking about this incorrectly? Should renaming a key component actually be safe? > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17646226#comment-17646226 ] Ke Han commented on CASSANDRA-18108: [~maedhroz] That makes sense. Thanks. > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645916#comment-17645916 ] Caleb Rackliffe commented on CASSANDRA-18108: - [~kehan5800] I think this is going to be a case where we just need to prohibit changing the names of primary key elements. You'll notice that {{PartitionColumns}} was renamed {{RegularAndStaticColumns}} in CASSANDRA-9425 to more clearly indicate that is isn't meant to deal w/ key elements. (The assertion from the trace you've posted is making sure {{add()}} is passed either a regular or static column.) > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645501#comment-17645501 ] Ke Han commented on CASSANDRA-18108: [~maedhroz] Thanks for the reply! Yes, renaming the primary key column is necessary to trigger the bug. > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645500#comment-17645500 ] Ke Han commented on CASSANDRA-18108: [~aratnofsky] That sounds great! Sure. I'll open a ticket explaining our fuzzing mechanism on a high level and I'll gradually fill in the details. > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645493#comment-17645493 ] Abe Ratnofsky commented on CASSANDRA-18108: --- [~kehan5800] if you can post more info about your fuzzer mechanism, that would be fantastic. We may be able to incorporate your tooling into our release process. Could you create a ticket with more info on that, so we don't distract from this particular bug? So far, each of these reports has been fantastic. If you're not sure, you can always ask on the dev mailing list though. > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645492#comment-17645492 ] Ke Han commented on CASSANDRA-18108: [~aratnofsky] Thanks for asking. Yes. We are fuzzing Cassandra. Currently, we do manual investigations for our results, filter out false positives, and only open tickets for the ones we think could be true bugs. May I ask what could be the most helpful approach to report the issues for the community? Will it be better if we report and update our results under one umbrella ticket? > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645489#comment-17645489 ] Caleb Rackliffe commented on CASSANDRA-18108: - Does this only occur when you're renaming a primary key column? > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18108) Data loss after a system restart/upgrade (3.11.14)
[ https://issues.apache.org/jira/browse/CASSANDRA-18108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645487#comment-17645487 ] Abe Ratnofsky commented on CASSANDRA-18108: --- Hey Ke - thanks for reporting this. I noticed you also reported CASSANDRA-18105. Can you share more about the type of investigation you're doing? Are you fuzzing Cassandra? > Data loss after a system restart/upgrade (3.11.14) > -- > > Key: CASSANDRA-18108 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18108 > Project: Cassandra > Issue Type: Bug >Reporter: Ke Han >Priority: Normal > > When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during > the upgrade process. This bug can also be triggered if simply performing a > system restart. > h1. Steps to reproduce > Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute > the following cqlsh commands. > {code:java} > CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) > WITH speculative_retry = 'ALWAYS'; > INSERT INTO ks.tb (c1, c2) VALUES (2,'val'); > ALTER TABLE ks.tb DROP c2 ; > ALTER TABLE ks.tb RENAME c1 TO c2; {code} > Then execute a SELECT command, we get the correct data > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > +-- > 2 | null > (1 rows){code} > Flush and stop the Cassandra daemon. > {code:java} > bin/nodetool flush > bin/nodetool stopdaemon{code} > Then restart the node. > {code:java} > bin/cassandra{code} > Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost. > {code:java} > cqlsh> SELECT * FROM ks.tb; > c2 | c3 > + > (0 rows){code} > > During the node restart, we found an error log about initializing the table, > but it didn't prevent the system from starting up. > {code:java} > INFO [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - > Initializing ks.tb > ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - > Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161) > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522) > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385) > at > org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84) > at java.lang.Thread.run(Thread.java:750) {code} > > This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 > and execute the SELECT command in the new version. (*The token_num > configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, > all the other configurations are using default values) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org