[ https://issues.apache.org/jira/browse/CASSANDRA-5925?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jeremy Hanna updated CASSANDRA-5925: ------------------------------------ Labels: LWT (was: ) > Race condition in update lightweight transaction > ------------------------------------------------ > > Key: CASSANDRA-5925 > URL: https://issues.apache.org/jira/browse/CASSANDRA-5925 > Project: Cassandra > Issue Type: Bug > Environment: 3 node Cassandra 2.0.0-rc2 cluster. Java driver 1.0.2. > Reporter: Phil Persad > Assignee: Sylvain Lebresne > Priority: Major > Labels: LWT > Fix For: 2.0.0 > > 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 was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org