You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2017/02/06 16:47:42 UTC

[jira] [Assigned] (CASSANDRA-13109) Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0

     [ https://issues.apache.org/jira/browse/CASSANDRA-13109?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne reassigned CASSANDRA-13109:
--------------------------------------------

    Assignee: Samuel Klock

> Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0
> -----------------------------------------------------------------------
>
>                 Key: CASSANDRA-13109
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13109
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Samuel Klock
>            Assignee: Samuel Klock
>         Attachments: 13109-3.0.txt
>
>
> We've observed this upgrading from 2.1.15 to 3.0.8 and from 2.1.16 to 3.0.10: some lightweight transactions executed on upgraded nodes fail with a read failure.  The following conditions seem relevant to this occurring:
> * The transaction must be conditioned on the current value of at least one column, e.g., {{IF NOT EXISTS}} transactions don't seem to be affected.
> * There should be a collection column (in our case, a map) defined on the table on which the transaction is executed.
> * The transaction should be executed before sstables on the node are upgraded.  The failure does not occur after the sstables have been upgraded (whether via {{nodetool upgradesstables}} or effectively via compaction).
> * Upgraded nodes seem to be able to participate in lightweight transactions as long as they're not the coordinator.
> * The values in the row being manipulated by the transaction must have been consistently manipulated by lightweight transactions (perhaps the existence of Paxos state for the partition is somehow relevant?).
> * In 3.0.10, it _seems_ to be necessary to have the partition split across multiple legacy sstables.  This was not necessary to reproduce the bug in 3.0.8 or .9.
> For applications affected by this bug, a possible workaround is to prevent nodes being upgraded from coordinating requests until sstables have been upgraded.
> We're able to reproduce this when upgrading from 2.1.16 to 3.0.10 with the following steps on a single-node cluster using a mostly pristine {{cassandra.yaml}} from the source distribution.
> # Start Cassandra-2.1.16 on the node.
> # Create a table with a collection column and insert some data into it.
> {code:sql}
> CREATE KEYSPACE test WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1};
> CREATE TABLE test.test (key TEXT PRIMARY KEY, cas_target TEXT, some_collection MAP<TEXT, TEXT>);
> INSERT INTO test.test (key, cas_target, some_collection) VALUES ('key', 'value', {}) IF NOT EXISTS;
> {code}
> # Flush the row to an sstable: {{nodetool flush}}.
> # Update the row:
> {code:sql}
> UPDATE test.test SET cas_target = 'newvalue', some_collection = {} WHERE key = 'key' IF cas_target = 'value';
> {code}
> # Drain the node: {{nodetool drain}}
> # Stop the node, upgrade to 3.0.10, and start the node.
> # Attempt to update the row again:
> {code:sql}
> UPDATE test.test SET cas_target = 'lastvalue' WHERE key = 'key' IF cas_target = 'newvalue';
> {code}
> Using {{cqlsh}}, if the error is reproduced, the following output will be returned:
> {code:sql}
> $ ./cqlsh <<< "UPDATE test.test SET cas_target = 'newvalue', some_collection = {} WHERE key = 'key' IF cas_target = 'value';"
> (start: 2016-12-22 10:14:27 EST)
> <stdin>:2:ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] message="Operation failed - received 0 responses and 1 failures" info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 'consistency': 'QUORUM'}
> {code}
> and the following stack trace will be present in the system log:
> {noformat}
> WARN  15:14:28 Uncaught exception on thread Thread[SharedPool-Worker-10,10,main]: {}
> java.lang.RuntimeException: java.lang.NullPointerException
> 	at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2476) ~[main/:na]
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101]
> 	at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) ~[main/:na]
> 	at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) [main/:na]
> 	at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> Caused by: java.lang.NullPointerException: null
> 	at org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:617) ~[main/:na]
> 	at org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:569) ~[main/:na]
> 	at org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:220) ~[main/:na]
> 	at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:159) ~[main/:na]
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[main/:na]
> 	at org.apache.cassandra.db.rows.Row$Merger.merge(Row.java:546) ~[main/:na]
> 	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:563) ~[main/:na]
> 	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:527) ~[main/:na]
> 	at org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:220) ~[main/:na]
> 	at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:159) ~[main/:na]
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[main/:na]
> 	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:509) ~[main/:na]
> 	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:369) ~[main/:na]
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[main/:na]
> 	at org.apache.cassandra.db.partitions.AbstractBTreePartition.build(AbstractBTreePartition.java:334) ~[main/:na]
> 	at org.apache.cassandra.db.partitions.ImmutableBTreePartition.create(ImmutableBTreePartition.java:111) ~[main/:na]
> 	at org.apache.cassandra.db.partitions.ImmutableBTreePartition.create(ImmutableBTreePartition.java:94) ~[main/:na]
> 	at org.apache.cassandra.db.SinglePartitionReadCommand.add(SinglePartitionReadCommand.java:810) ~[main/:na]
> 	at org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:760) ~[main/:na]
> 	at org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:519) ~[main/:na]
> 	at org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:496) ~[main/:na]
> 	at org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:358) ~[main/:na]
> 	at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:394) ~[main/:na]
> 	at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1794) ~[main/:na]
> 	at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2472) ~[main/:na]
> 	... 5 common frames omitted
> {noformat}
> Under both 3.0.8 and .9, the {{nodetool flush}} and additional {{UPDATE}} statement before upgrading to 3.0 are not necessary to reproduce this.  In that case (when Cassandra only has to read the data from one sstable?), a different stack trace appears in the log.  Here's a sample from 3.0.8:
> {noformat}
>  WARN [SharedPool-Worker-3] 2016-12-13 15:19:48,863 AbstractLocalAwareExecutorService.java (line 169) Uncaught exception on thread Thread[SharedPool-Worker-3,5,main]: {}
> java.lang.RuntimeException: java.lang.IllegalStateException: [ColumnDefinition{name=REDACTED, type=org.apache.cassandra.db.marshal.UTF8Type, kind=REGULAR, position=-1}, ColumnDefinition{name=REDACTED2, type=org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.UTF8Type), kind=REGULAR, position=-1}] is not a subset of [REDACTED]
>         at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2453) ~[main/:na]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101]
>         at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) ~[main/:na]
>         at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) [main/:na]
>         at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> Caused by: java.lang.IllegalStateException: [ColumnDefinition{name=REDACTED, type=org.apache.cassandra.db.marshal.UTF8Type, kind=REGULAR, position=-1}, ColumnDefinition{name=REDACTED2, type=org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.UTF8Type), kind=REGULAR, position=-1}] is not a subset of [REDACTED]
>         at org.apache.cassandra.db.Columns$Serializer.encodeBitmap(Columns.java:531) ~[main/:na]
>         at org.apache.cassandra.db.Columns$Serializer.serializeSubset(Columns.java:465) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:178) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:108) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:96) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:132) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77) ~[main/:na]
>         at org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:300) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:134) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:127) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:123) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) ~[main/:na]
>         at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:289) ~[main/:na]
>         at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1796) ~[main/:na]
>         at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2449) ~[main/:na]
>         ... 5 common frames omitted
>  WARN [SharedPool-Worker-1] 2016-12-13 15:19:48,943 AbstractLocalAwareExecutorService.java (line 169) Uncaught exception on thread Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.IllegalStateException: [ColumnDefinition{name=REDACTED, type=org.apache.cassandra.db.marshal.UTF8Type, kind=REGULAR, position=-1}, ColumnDefinition{name=REDACTED2, type=org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.UTF8Type), kind=REGULAR, position=-1}] is not a subset of [REDACTED]
>         at org.apache.cassandra.db.Columns$Serializer.encodeBitmap(Columns.java:531) ~[main/:na]
>         at org.apache.cassandra.db.Columns$Serializer.serializeSubset(Columns.java:465) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:178) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:108) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:96) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:132) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87) ~[main/:na]
>         at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77) ~[main/:na]
>         at org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:300) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:134) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:127) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:123) ~[main/:na]
>         at org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) ~[main/:na]
>         at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:289) ~[main/:na]
>         at org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:47) ~[main/:na]
>         at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) ~[main/:na]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101]
>         at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) ~[main/:na]
>         at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) [main/:na]
>         at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> {noformat}
> It's not clear to us what changed in 3.0.10 to make this behavior somewhat more difficult to reproduce.
> We spent some time trying to track down the cause in 3.0.8, and we've identified a very small patch (which I will attach to this issue) that _seems_ to fix it.  The problem appears to be that the logic that reads data from legacy sstables can pull range tombstones covering collection columns that weren't requested, which then breaks downstream logic that doesn't expect those tombstones to be present in the data.  The patch attempts to include those tombstones only if they're explicitly requested.  However, there's enough going on in that logic that it's not clear to us whether the change is safe, so it is definitely in need of review from someone knowledgable about what that area of the code is intended to do.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)