cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-5925) Race condition in update lightweight transaction
Date Tue, 27 Aug 2013 13:54:52 GMT

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

Sylvain Lebresne updated CASSANDRA-5925:
----------------------------------------

    Attachment: 5925.txt

Thanks for the test.

There is indeed 2 problems:
# the NPE while loading the paxos state. Contrarily to my first reading, the fact that savePaxosProposal
doesn't save the inProgress ballot is not only a problem due to expiration. We may call that
method if our own inProgress is older than the proposal, so given a node received a proposal
without having seen the prepare first (and without having a previous state), we'd end up with
a state where just the 'proposal' column is set. Note that technically, I don't think it breaks
Paxos not to update the ballot when saving a proposal so we could just fix loadPaxosState
to not NPE in that case, but it feels saner/simpler to me to write the proposal ballot when
we save a proposal value.
# the reason Phil test fails is different however. The problem is that when we were building
the 'expected' ColumnFamily for the cas() call in CQL3, we were using the full parameters
of the statement, including (which was the problem) the TTL. In that test, the TTL is 1 second,
so it's possible (and even not that unlikely since we only have up to 1 second accuracy internally)
that when we were comparing 'expected' to 'current' the expected column was considered deleted.
So, if when the 2nd update to a row was processed the first one had expired (again, not unlikely
given the 1 second ttl), the 2nd CAS update was comparing both a deleted 'expected' and a
deleted 'current', thus succeeding.  So anyway, the fix is to not use the TTL for the conditions
in ModificationStatement.

So attaching patch that fix both issue.

                
> Race condition in update lightweight transaction
> ------------------------------------------------
>
>                 Key: CASSANDRA-5925
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-5925
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: 3 node Cassandra 2.0.0-rc2 cluster. Java driver 1.0.2.
>            Reporter: Phil Persad
>            Assignee: Sylvain Lebresne
>             Fix For: 2.0.1
>
>         Attachments: 5925.txt, TokenConsumptionTest.java
>
>
> I'm building some tests for a Cassandra PoC.  One scenario I need to test is consumption
of 1 time tokens.  These tokens must be consumed exactly once.  The cluster involved is a
3 node cluster.  All queries are run with ConsistencyLevel.QUORUM. I'm using the following
queries:
> CREATE KEYSPACE IF NOT EXISTS test WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor'
: 3 };
> CREATE TABLE IF NOT EXISTS tkns (tkn blob, consumed boolean, PRIMARY KEY (tkn));
> INSERT INTO tkns (tkn, consumed) VALUES (?,FALSE) USING TTL 30;
> UPDATE tkns USING TTL 1 SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE;
> I use the '[applied]' column in the result set of the update statement to determine whether
the token has been successfully consumed or if the token is being replayed.
> My test involves concurrently executing many sets of 1 insert and 2 update statements
(using Session#execute on BoundStatemnts) then checking to make sure that only one of the
updates was applied.
> When I run this test with relatively few iterations (~100) my results are  what I expect
(exactly 1 update succeeds).  At ~1000 iterations, I start seeing both updates reporting success
in 1-2% of cases.  While my test is running, I see corresponding error entries in the Cassandra
log:
> ERROR 15:34:53,583 Exception in thread Thread[MutationStage:522,5,main]
> java.lang.NullPointerException
> ERROR 15:34:53,584 Exception in thread Thread[MutationStage:474,5,main]
> java.lang.NullPointerException
> ERROR 15:34:53,584 Exception in thread Thread[MutationStage:536,5,main]
> java.lang.NullPointerException
> ERROR 15:34:53,729 Exception in thread Thread[MutationStage:480,5,main]
> java.lang.NullPointerException
> ERROR 15:34:53,729 Exception in thread Thread[MutationStage:534,5,main]
> java.lang.NullPointerException
> Thanks.
> Update:
> I'm not sure what's going on with the logging the the dev release.  I grabbed the rc2
source and built that.  The resultant log is a bit more informative:
> ERROR 11:53:38,967 Exception in thread Thread[MutationStage:114,5,main]
> java.lang.NullPointerException
> 	at org.apache.cassandra.serializers.UUIDSerializer.deserialize(UUIDSerializer.java:32)
> 	at org.apache.cassandra.serializers.UUIDSerializer.deserialize(UUIDSerializer.java:26)
> 	at org.apache.cassandra.db.marshal.AbstractType.compose(AbstractType.java:142)
> 	at org.apache.cassandra.cql3.UntypedResultSet$Row.getUUID(UntypedResultSet.java:131)
> 	at org.apache.cassandra.db.SystemKeyspace.loadPaxosState(SystemKeyspace.java:785)
> 	at org.apache.cassandra.service.paxos.PaxosState.commit(PaxosState.java:118)
> 	at org.apache.cassandra.service.paxos.CommitVerbHandler.doVerb(CommitVerbHandler.java:34)
> 	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> 	at java.lang.Thread.run(Thread.java:722)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message