You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Kurt Greaves (JIRA)" <ji...@apache.org> on 2018/03/13 03:34:00 UTC

[jira] [Commented] (CASSANDRA-14215) Cassandra does not respect hint window for CAS

    [ https://issues.apache.org/jira/browse/CASSANDRA-14215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16396489#comment-16396489 ] 

Kurt Greaves commented on CASSANDRA-14215:
------------------------------------------

Thanks [~iamaleksey].

Created CASSANDRA-14309 for hint window patch.

Branches are the same for C*

|[3.0|https://github.com/apache/cassandra/compare/cassandra-3.0...kgreav:14215-3.0]|
|[3.11|https://github.com/apache/cassandra/compare/cassandra-3.11...kgreav:14215-3.11]|
|[trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:14215-trunk-1]| 

I've split out the dtest for CAS.
|[dtest|https://github.com/apache/cassandra-dtest/compare/master...kgreav:14215-2]|

> Cassandra does not respect hint window for CAS
> ----------------------------------------------
>
>                 Key: CASSANDRA-14215
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14215
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Hints, Streaming and Messaging
>            Reporter: Arijit Banerjee
>            Assignee: Kurt Greaves
>            Priority: Major
>
> On Cassandra 3.0.9, it was observed that Cassandra continues to write hints even though a node remains down (and does not come up) for longer than the default 3 hour window.
>  
> After doing "nodetool setlogginglevel org.apache.cassandra TRACE", we see the following log line in cassandra (debug) logs:
>  StorageProxy.java:2625 - Adding hints for [/10.0.100.84]
>  
> One possible code path seems to be:
> cas -> commitPaxos(proposal, consistencyForCommit, true); -> submitHint (in StorageProxy.java)
>  
> The "true" parameter above explicitly states that a hint should be recorded and ignores the time window calculation performed by the shouldHint method invoked in other code paths. Is there a reason for this behavior?
>  
> Edit: There are actually two stacks that seem to be producing hints, the "cas" and "syncWriteBatchedMutations" methods. I have posted them below.
>  
> A third issue seems to be that Cassandra seems to reset the timer which counts how long a node has been down after a restart. Thus if Cassandra is restarted on a good node, it continues to accumulate hints for a down node over the next three hours.
>  
> {code:java}
> WARN [SharedPool-Worker-14] 2018-02-06 22:15:51,136 StorageProxy.java:2636 - Adding hints for [/10.0.100.84] with stack trace: java.lang.Throwable: at org.apache.cassandra.service.StorageProxy.stackTrace(StorageProxy.java:2608) at org.apache.cassandra.service.StorageProxy.submitHint(StorageProxy.java:2617) at org.apache.cassandra.service.StorageProxy.submitHint(StorageProxy.java:2603) at org.apache.cassandra.service.StorageProxy.commitPaxos(StorageProxy.java:540) at org.apache.cassandra.service.StorageProxy.cas(StorageProxy.java:282) at org.apache.cassandra.cql3.statements.ModificationStatement.executeWithCondition(ModificationStatement.java:432) at org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:407) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) at java.lang.Thread.run(Thread.java:748) WARN
> {code}
> {code:java}
> [SharedPool-Worker-8] 2018-02-06 22:15:51,153 StorageProxy.java:2636 - Adding hints for [/10.0.100.84] with stack trace: java.lang.Throwable: at org.apache.cassandra.service.StorageProxy.stackTrace(StorageProxy.java:2608) at org.apache.cassandra.service.StorageProxy.submitHint(StorageProxy.java:2617) at org.apache.cassandra.service.StorageProxy.sendToHintedEndpoints(StorageProxy.java:1247) at org.apache.cassandra.service.StorageProxy.syncWriteBatchedMutations(StorageProxy.java:1014) at org.apache.cassandra.service.StorageProxy.mutateAtomically(StorageProxy.java:899) at org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:834) at org.apache.cassandra.cql3.statements.BatchStatement.executeWithoutConditions(BatchStatement.java:365) at org.apache.cassandra.cql3.statements.BatchStatement.execute(BatchStatement.java:343) at org.apache.cassandra.cql3.statements.BatchStatement.execute(BatchStatement.java:329) at org.apache.cassandra.cql3.statements.BatchStatement.execute(BatchStatement.java:324) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) at java.lang.Thread.run(Thread.java:748)
> {code}
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org