[jira] [Updated] (CASSANDRA-14463) Prevent the generation of new tokens when using replace_address flag
[ https://issues.apache.org/jira/browse/CASSANDRA-14463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kurt Greaves updated CASSANDRA-14463: - Status: Patch Available (was: Open) > Prevent the generation of new tokens when using replace_address flag > > > Key: CASSANDRA-14463 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14463 > Project: Cassandra > Issue Type: Bug >Reporter: Vincent White >Assignee: Vincent White >Priority: Minor > > This is a follow up to/replacement of CASSANDRA-14073. > The behaviour that I want to avoid is someone trying to replace a node with > the replace_address flag and mistakenly having that node listed in its own > seed list which causes the node to generate a new set of random tokens before > joining the ring. > Currently anytime an unbootstrapped node is listed in its own seed list and > initial_token isn't set in the yaml, Cassandra will generate a new set of > random tokens and join the ring regardless of whether it was replacing a > previous node or not. > We could simply check for this configuration and refuse to start but I it's > probably better (particularly for 3.0.X) if it's handled in the same manner > as skipping streaming with the allow_unsafe_replace flag that was introduced > in 3.X . This would still allow 3.0.X users the ability to re-bootstrap nodes > without needing to re-stream all the data to the node again, which can be > useful. > We currently handle replacing without streaming different;y between 3.0.X and > 3.X. In 3.X we have the allow_unsafe_replace JVM flag to allow the use of > auto_bootstrap: false in combination with the replace_address option. But in > 3.0.X to perform the replacement of a node with the same IP address without > streaming I believe you need to: > * Set replace_address (because the address is already in gossip) > * Include the node in its own seed list (to skip bootstrapping/streaming) > * Set the initial_token to the token/s owned by the previous node (to > prevent it generating new tokens. > I believe if 3.0.X simply refused to start when a node has itself in its seed > list and replace_address set this will completely block this operation. > Example patches to fix this edge case using allow_unsafe_replace: > > ||Branch|| > |[3.0.x|https://github.com/apache/cassandra/compare/trunk...vincewhite:30-no_clobber]| > |[3.x|https://github.com/apache/cassandra/compare/trunk...vincewhite:311-no_clobber]| -- 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
[jira] [Commented] (CASSANDRA-14525) streaming failure during bootstrap makes new node into inconsistent state
[ https://issues.apache.org/jira/browse/CASSANDRA-14525?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16620099#comment-16620099 ] Kurt Greaves commented on CASSANDRA-14525: -- Looks good to me. Probably need another run of the dtests though :/ > streaming failure during bootstrap makes new node into inconsistent state > - > > Key: CASSANDRA-14525 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14525 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Jaydeepkumar Chovatia >Assignee: Jaydeepkumar Chovatia >Priority: Major > Fix For: 4.0, 2.2.x, 3.0.x > > > If bootstrap fails for newly joining node (most common reason is due to > streaming failure) then Cassandra state remains in {{joining}} state which is > fine but Cassandra also enables Native transport which makes overall state > inconsistent. This further creates NullPointer exception if auth is enabled > on the new node, please find reproducible steps here: > For example if bootstrap fails due to streaming errors like > {quote}java.util.concurrent.ExecutionException: > org.apache.cassandra.streaming.StreamException: Stream failed > at > com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:299) > ~[guava-18.0.jar:na] > at > com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:286) > ~[guava-18.0.jar:na] > at > com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) > ~[guava-18.0.jar:na] > at > org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1256) > [apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:894) > [apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:660) > [apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:573) > [apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:330) > [apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567) > [apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:695) > [apache-cassandra-3.0.16.jar:3.0.16] > Caused by: org.apache.cassandra.streaming.StreamException: Stream failed > at > org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) > ~[guava-18.0.jar:na] > at > com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457) > ~[guava-18.0.jar:na] > at > com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) > ~[guava-18.0.jar:na] > at > com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) > ~[guava-18.0.jar:na] > at > com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) > ~[guava-18.0.jar:na] > at > org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:211) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:187) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:440) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:540) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:307) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > ~[apache-cassandra-3.0.16.jar:3.0.16] > at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_121] > {quote} > then variable [StorageService.java::dataAvailable > |https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/service/StorageService.java#L892] > will be {{false}}. Since {{dataAvailable}} is {{false}} hence it will not > call [StorageService.java::finishJoiningRing > |https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/service/StorageService.java#L933] > and as a result > [StorageService.java::doAuthSetup|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/service/StorageService.java#L999] > will not be
[jira] [Assigned] (CASSANDRA-14463) Prevent the generation of new tokens when using replace_address flag
[ https://issues.apache.org/jira/browse/CASSANDRA-14463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kurt Greaves reassigned CASSANDRA-14463: Assignee: Vincent White > Prevent the generation of new tokens when using replace_address flag > > > Key: CASSANDRA-14463 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14463 > Project: Cassandra > Issue Type: Bug >Reporter: Vincent White >Assignee: Vincent White >Priority: Minor > > This is a follow up to/replacement of CASSANDRA-14073. > The behaviour that I want to avoid is someone trying to replace a node with > the replace_address flag and mistakenly having that node listed in its own > seed list which causes the node to generate a new set of random tokens before > joining the ring. > Currently anytime an unbootstrapped node is listed in its own seed list and > initial_token isn't set in the yaml, Cassandra will generate a new set of > random tokens and join the ring regardless of whether it was replacing a > previous node or not. > We could simply check for this configuration and refuse to start but I it's > probably better (particularly for 3.0.X) if it's handled in the same manner > as skipping streaming with the allow_unsafe_replace flag that was introduced > in 3.X . This would still allow 3.0.X users the ability to re-bootstrap nodes > without needing to re-stream all the data to the node again, which can be > useful. > We currently handle replacing without streaming different;y between 3.0.X and > 3.X. In 3.X we have the allow_unsafe_replace JVM flag to allow the use of > auto_bootstrap: false in combination with the replace_address option. But in > 3.0.X to perform the replacement of a node with the same IP address without > streaming I believe you need to: > * Set replace_address (because the address is already in gossip) > * Include the node in its own seed list (to skip bootstrapping/streaming) > * Set the initial_token to the token/s owned by the previous node (to > prevent it generating new tokens. > I believe if 3.0.X simply refused to start when a node has itself in its seed > list and replace_address set this will completely block this operation. > Example patches to fix this edge case using allow_unsafe_replace: > > ||Branch|| > |[3.0.x|https://github.com/apache/cassandra/compare/trunk...vincewhite:30-no_clobber]| > |[3.x|https://github.com/apache/cassandra/compare/trunk...vincewhite:311-no_clobber]| -- 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
[jira] [Assigned] (CASSANDRA-14721) sstabledump displays incorrect value for "position" key
[ https://issues.apache.org/jira/browse/CASSANDRA-14721?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kurt Greaves reassigned CASSANDRA-14721: Assignee: Cameron Zemek > sstabledump displays incorrect value for "position" key > --- > > Key: CASSANDRA-14721 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14721 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Damien Stevenson >Assignee: Cameron Zemek >Priority: Minor > Attachments: cassandra-dump.patch > > > When partitions with multiple rows are displayed using sstabledump, the > "position" value the first row of each partition is incorrect. > For example: > {code:java} > sstabledump mc-1-big-Data.db > [ > { > "partition" : { > "key" : [ "1", "24" ], > "position" : 0 > }, > "rows" : [ > { > "type" : "row", > "position" : 66, > "clustering" : [ "2013-12-10 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.290086Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 8 }, > { "name" : "chanceofrain", "value" : 0.1 }, > { "name" : "feelslike", "value" : 8 }, > { "name" : "humidity", "value" : 0.76 }, > { "name" : "wind", "value" : 10.0 } > ] > }, > { > "type" : "row", > "position" : 66, > "clustering" : [ "2013-12-11 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.295369Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 4 }, > { "name" : "chanceofrain", "value" : 0.3 }, > { "name" : "feelslike", "value" : 4 }, > { "name" : "humidity", "value" : 0.9 }, > { "name" : "wind", "value" : 12.0 } > ] > }, > { > "type" : "row", > "position" : 105, > "clustering" : [ "2013-12-12 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.300841Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 3 }, > { "name" : "chanceofrain", "value" : 0.2 }, > { "name" : "feelslike", "value" : 3 }, > { "name" : "humidity", "value" : 0.68 }, > { "name" : "wind", "value" : 6.0 } > ] > } > ] > } > ] > {code} > The expected output is: > {code:java} > [ > { > "partition" : { > "key" : [ "1", "24" ], > "position" : 0 > }, > "rows" : [ > { > "type" : "row", > "position" : 28, > "clustering" : [ "2013-12-10 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.290086Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 8 }, > { "name" : "chanceofrain", "value" : 0.1 }, > { "name" : "feelslike", "value" : 8 }, > { "name" : "humidity", "value" : 0.76 }, > { "name" : "wind", "value" : 10.0 } > ] > }, > { > "type" : "row", > "position" : 66, > "clustering" : [ "2013-12-11 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.295369Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 4 }, > { "name" : "chanceofrain", "value" : 0.3 }, > { "name" : "feelslike", "value" : 4 }, > { "name" : "humidity", "value" : 0.9 }, > { "name" : "wind", "value" : 12.0 } > ] > }, > { > "type" : "row", > "position" : 105, > "clustering" : [ "2013-12-12 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.300841Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 3 }, > { "name" : "chanceofrain", "value" : 0.2 }, > { "name" : "feelslike", "value" : 3 }, > { "name" : "humidity", "value" : 0.68 }, > { "name" : "wind", "value" : 6.0 } > ] > } > ] > } > ] > {code} -- 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
[jira] [Updated] (CASSANDRA-14721) sstabledump displays incorrect value for "position" key
[ https://issues.apache.org/jira/browse/CASSANDRA-14721?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kurt Greaves updated CASSANDRA-14721: - Status: Patch Available (was: Open) > sstabledump displays incorrect value for "position" key > --- > > Key: CASSANDRA-14721 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14721 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Damien Stevenson >Assignee: Cameron Zemek >Priority: Minor > Attachments: cassandra-dump.patch > > > When partitions with multiple rows are displayed using sstabledump, the > "position" value the first row of each partition is incorrect. > For example: > {code:java} > sstabledump mc-1-big-Data.db > [ > { > "partition" : { > "key" : [ "1", "24" ], > "position" : 0 > }, > "rows" : [ > { > "type" : "row", > "position" : 66, > "clustering" : [ "2013-12-10 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.290086Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 8 }, > { "name" : "chanceofrain", "value" : 0.1 }, > { "name" : "feelslike", "value" : 8 }, > { "name" : "humidity", "value" : 0.76 }, > { "name" : "wind", "value" : 10.0 } > ] > }, > { > "type" : "row", > "position" : 66, > "clustering" : [ "2013-12-11 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.295369Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 4 }, > { "name" : "chanceofrain", "value" : 0.3 }, > { "name" : "feelslike", "value" : 4 }, > { "name" : "humidity", "value" : 0.9 }, > { "name" : "wind", "value" : 12.0 } > ] > }, > { > "type" : "row", > "position" : 105, > "clustering" : [ "2013-12-12 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.300841Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 3 }, > { "name" : "chanceofrain", "value" : 0.2 }, > { "name" : "feelslike", "value" : 3 }, > { "name" : "humidity", "value" : 0.68 }, > { "name" : "wind", "value" : 6.0 } > ] > } > ] > } > ] > {code} > The expected output is: > {code:java} > [ > { > "partition" : { > "key" : [ "1", "24" ], > "position" : 0 > }, > "rows" : [ > { > "type" : "row", > "position" : 28, > "clustering" : [ "2013-12-10 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.290086Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 8 }, > { "name" : "chanceofrain", "value" : 0.1 }, > { "name" : "feelslike", "value" : 8 }, > { "name" : "humidity", "value" : 0.76 }, > { "name" : "wind", "value" : 10.0 } > ] > }, > { > "type" : "row", > "position" : 66, > "clustering" : [ "2013-12-11 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.295369Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 4 }, > { "name" : "chanceofrain", "value" : 0.3 }, > { "name" : "feelslike", "value" : 4 }, > { "name" : "humidity", "value" : 0.9 }, > { "name" : "wind", "value" : 12.0 } > ] > }, > { > "type" : "row", > "position" : 105, > "clustering" : [ "2013-12-12 00:00:00.000Z" ], > "liveness_info" : { "tstamp" : "2018-09-12T05:01:09.300841Z" }, > "cells" : [ > { "name" : "centigrade", "value" : 3 }, > { "name" : "chanceofrain", "value" : 0.2 }, > { "name" : "feelslike", "value" : 3 }, > { "name" : "humidity", "value" : 0.68 }, > { "name" : "wind", "value" : 6.0 } > ] > } > ] > } > ] > {code} -- 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
[jira] [Commented] (CASSANDRA-14553) Document troubleshooting page
[ https://issues.apache.org/jira/browse/CASSANDRA-14553?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16620006#comment-16620006 ] Jordan West commented on CASSANDRA-14553: - Thanks [~jolynch]. I will definitely take a look. Also pinging [~djoshi3] who is interested in parts of this. > Document troubleshooting page > - > > Key: CASSANDRA-14553 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14553 > Project: Cassandra > Issue Type: Improvement > Components: Documentation and Website >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Minor > Labels: Documentation > > Currently the [troubleshooting > docs|http://cassandra.apache.org/doc/latest/troubleshooting/] are blank. As > much as I like to believe Cassandra never has any problems I was thinking of > writing up a troubleshooting page focussing on: > # Finding the hosts(s) that are behaving badly (common error messages) > # Which logs exist, where they are, and what to look for in which log > (common error messages, gc logs, etc) > # Which nodetool commands can give you more information > # Java/Operating systems tools that can help dive deep into performance > issues (jstat, top, iostat, cachestat, etc) > Since this is going to be a fairly lengthy page I wanted to get a jira going > in case someone else had ideas or had already started. Also if there are any > large areas I missed above please comment here and I can include them. > [~cscotta] fyi -- 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
[jira] [Comment Edited] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619974#comment-16619974 ] Joseph Lynch edited comment on CASSANDRA-14747 at 9/19/18 2:08 AM: --- Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}} s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. We think a few things are going on here: # When the outbound queues get backed up we enter various long (sometimes infinite) loops. We're working on stopping those # Since we're multiplexing multiple nodes onto one outbound thread, we can have multi-tenant queues where one slow consumer hurts other nodes as well. We're working on a fix for this. was (Author: jolynch): Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}} s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > i-0ed2acd2dfacab7c1-after-looping-fixes.svg, > ttop_NettyOutbound-Thread_spinning.txt, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Updated] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joseph Lynch updated CASSANDRA-14747: - Attachment: ttop_NettyOutbound-Thread_spinning.txt > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > i-0ed2acd2dfacab7c1-after-looping-fixes.svg, > ttop_NettyOutbound-Thread_spinning.txt, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Comment Edited] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619974#comment-16619974 ] Joseph Lynch edited comment on CASSANDRA-14747 at 9/19/18 2:05 AM: --- Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}} s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. was (Author: jolynch): Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}}s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > i-0ed2acd2dfacab7c1-after-looping-fixes.svg, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Updated] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joseph Lynch updated CASSANDRA-14747: - Attachment: i-0ed2acd2dfacab7c1-after-looping-fixes.svg > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > i-0ed2acd2dfacab7c1-after-looping-fixes.svg, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Comment Edited] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619974#comment-16619974 ] Joseph Lynch edited comment on CASSANDRA-14747 at 9/19/18 2:04 AM: --- Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}}s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. was (Author: jolynch): Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}}s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Commented] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619974#comment-16619974 ] Joseph Lynch commented on CASSANDRA-14747: -- Things went much better today, after the queue fixes we no longer ran out of memory, but the {{MessagingService-NettyOutbound-Thread}}s would be pinned at 100% cpu. We (Jason, Jordan, myself, etc) tracked it down to various unfortunate looping behaviors in the {{OutboundMessagingConnection}} class. We're following up with various fixes to these queueing problems. I've attached flame graphs and ttop outputs showing what's going on on the latest version of {{jasobrown/14503-collab}} branch. > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Created] (CASSANDRA-14765) Evaluate Recovery Time on Single Token Cluster Test
Joseph Lynch created CASSANDRA-14765: Summary: Evaluate Recovery Time on Single Token Cluster Test Key: CASSANDRA-14765 URL: https://issues.apache.org/jira/browse/CASSANDRA-14765 Project: Cassandra Issue Type: Sub-task Reporter: Joseph Lynch *Setup:* * Cassandra: 6 (2*3 rack) node i3.8xlarge AWS instance (32 cpu cores, 240GB ram) running cassandra trunk with Jason's 14503 changes vs the same footprint running 3.0.17 * One datacenter, single tokens * No compression, encryption, or coalescing turned on *Test #1:* ndbench loaded ~150GB of data per node into a LCS table. Then we killed a node and let a new node stream. With a single token this should be a worst case recovery scenario (only a few peers to stream from). *Result:* As the table used LCS and we didn't not have encryption on, the zero copy transfer was used via CASSANDRA-14556. We recovered *150GB in 5 minutes,* going at a consistent rate of about 3 gigabit per second. Theoretically we should be able to get 10 gigabit, but this is still something like an estimated 16x improvement over 3.0.x. We're still running the 3.0.x test for a hard comparison. *Follow Ups:* We need to get more rigorous measurements (over more terminations), as well as finishing the 3.0.x test. [~sumanth.pasupuleti] and [~djoshi] are driving this. -- 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
[jira] [Updated] (CASSANDRA-14764) Evaluate 12 Node Breaking Point, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14764?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joseph Lynch updated CASSANDRA-14764: - Attachment: i-07cd92e844d66d801-after-queue-bound.svg ttop.txt > Evaluate 12 Node Breaking Point, compression=none, encryption=none, > coalescing=off > -- > > Key: CASSANDRA-14764 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14764 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Priority: Major > Attachments: i-07cd92e844d66d801-after-queue-bound.svg, ttop.txt > > > *Setup:* > * Cassandra: 12 (2*6) node i3.xlarge AWS instance (4 cpu cores, 30GB ram) > running cassandra trunk off of jasobrown/14503 jdd7ec5a2 (Jasons patched > internode messaging branch) vs the same footprint running 3.0.17 > * Two datacenters with 100ms latency between them > * No compression, encryption, or coalescing turned on > *Test #1:* > ndbench sent 1.5k QPS at a coordinator level to one datacenter (RF=3*2 = 6 so > 3k global replica QPS) of 4kb single partition BATCH mutations at LOCAL_ONE. > This represents about 250 QPS per coordinator in the first datacenter or 60 > QPS per core. The goal was to observe P99 write and read latencies under > various QPS. > *Result:* > The good news is since the CASSANDRA-14503 changes, instead of keeping the > mutations on heap we put the message into hints instead and don't run out of > memory. The bad news is that the {{MessagingService-NettyOutbound-Thread's}} > would occasionally enter a degraded state where they would just spin on a > core. I've attached flame graphs showing the CPU state as [~jasobrown] > applied fixes to the {{OutboundMessagingConnection}} class. > *Follow Ups:* > [~jasobrown] has committed a number of fixes onto his > {{jasobrown/14503-collab}} branch including: > 1. Limiting the amount of time spent dequeuing messages if they are expired > (previously if messages entered the queue faster than we could dequeue them > we'd just inifinte loop on the consumer side) > 2. Don't call {{dequeueMessages}} from within {{dequeueMessages}} created > callbacks. > We're continuing to use CPU flamegraphs to figure out where we're looping and > fixing bugs as we find them. -- 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
[jira] [Created] (CASSANDRA-14764) Evaluate 12 Node Breaking Point, compression=none, encryption=none, coalescing=off
Joseph Lynch created CASSANDRA-14764: Summary: Evaluate 12 Node Breaking Point, compression=none, encryption=none, coalescing=off Key: CASSANDRA-14764 URL: https://issues.apache.org/jira/browse/CASSANDRA-14764 Project: Cassandra Issue Type: Sub-task Reporter: Joseph Lynch *Setup:* * Cassandra: 12 (2*6) node i3.xlarge AWS instance (4 cpu cores, 30GB ram) running cassandra trunk off of jasobrown/14503 jdd7ec5a2 (Jasons patched internode messaging branch) vs the same footprint running 3.0.17 * Two datacenters with 100ms latency between them * No compression, encryption, or coalescing turned on *Test #1:* ndbench sent 1.5k QPS at a coordinator level to one datacenter (RF=3*2 = 6 so 3k global replica QPS) of 4kb single partition BATCH mutations at LOCAL_ONE. This represents about 250 QPS per coordinator in the first datacenter or 60 QPS per core. The goal was to observe P99 write and read latencies under various QPS. *Result:* The good news is since the CASSANDRA-14503 changes, instead of keeping the mutations on heap we put the message into hints instead and don't run out of memory. The bad news is that the {{MessagingService-NettyOutbound-Thread's}} would occasionally enter a degraded state where they would just spin on a core. I've attached flame graphs showing the CPU state as [~jasobrown] applied fixes to the {{OutboundMessagingConnection}} class. *Follow Ups:* [~jasobrown] has committed a number of fixes onto his {{jasobrown/14503-collab}} branch including: 1. Limiting the amount of time spent dequeuing messages if they are expired (previously if messages entered the queue faster than we could dequeue them we'd just inifinte loop on the consumer side) 2. Don't call {{dequeueMessages}} from within {{dequeueMessages}} created callbacks. We're continuing to use CPU flamegraphs to figure out where we're looping and fixing bugs as we find them. -- 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
[jira] [Updated] (CASSANDRA-14746) Ensure Netty Internode Messaging Refactor is Solid
[ https://issues.apache.org/jira/browse/CASSANDRA-14746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joseph Lynch updated CASSANDRA-14746: - Description: Before we release 4.0 let's ensure that the internode messaging refactor is 100% solid. As internode messaging is naturally used in many code paths and widely configurable we have a large number of cluster configurations and test configurations that must be vetted. We plan to vary the following: * Version of Cassandra 3.0.17 vs 4.0-alpha * Cluster sizes with *multi-dc* deployments ranging from 6 - 100 nodes * Client request rates varying between 1k QPS and 100k QPS of varying sizes and shapes (BATCH, INSERT, SELECT point, SELECT range, etc ...) * Internode compression * Internode SSL (as well as openssl vs jdk) * Internode Coalescing options We are looking to measure the following as appropriate: * Latency distributions of reads and writes (lower is better) * Scaling limit, aka maximum throughput before violating p99 latency deadline of 10ms @ LOCAL_QUORUM, on a fixed hardware deployment for 100% writes, 100% reads and 50-50 writes+reads (higher is better) * Thread counts (lower is better) * Context switches (lower is better) * On-CPU time of tasks (higher periods without context switch is better) * GC allocation rates / throughput for a fixed size heap (lower allocation better) * Streaming recovery time for a single node failure, i.e. can Cassandra saturate the NIC The goal is that 4.0 should have better latency, more throughput, fewer threads, fewer context switches, less GC allocation, and faster recovery time. I'm putting Jason Brown as the reviewer since he implemented most of the internode refactor. Current collaborators driving this QA task: Dinesh Joshi, Jordan West, Joey Lynch (Netflix), Vinay Chella (Netflix) Owning committer(s): Jason Brown was: Before we release 4.0 let's ensure that the internode messaging refactor is 100% solid. As internode messaging is naturally used in many code paths and widely configurable we have a large number of cluster configurations and test configurations that must be vetted. We plan to vary the following: * Version of Cassandra 3.0.17 vs 4.0-alpha * Cluster sizes with *multi-dc* deployments ranging from 6 - 100 nodes * Client request rates varying between 1k QPS and 100k QPS of varying sizes and shapes (BATCH, INSERT, SELECT point, SELECT range, etc ...) * Internode compression * Internode SSL (as well as openssl vs jdk) * Internode Coalescing options We are looking to measure the following as appropriate: * Latency distributions of reads and writes (lower is better) * Scaling limit, aka maximum throughput before violating p99 latency deadline of 100ms, on a fixed hardware deployment for 100% writes, 100% reads and 50-50 writes+reads (higher is better) * Thread counts (lower is better) * Context switches (lower is better) * On-CPU time of tasks (higher periods without context switch is better) * GC allocation rates / throughput for a fixed size heap (lower allocation better) * Streaming recovery time for a single node failure, i.e. can Cassandra saturate the NIC The goal is that 4.0 should have better latency, more throughput, fewer threads, fewer context switches, less GC allocation, and faster recovery time. I'm putting Jason Brown as the reviewer since he implemented most of the internode refactor. Current collaborators driving this QA task: Dinesh Joshi (Apple), Jordan West (Apple), Joey Lynch (Netflix), Vinay Chella (Netflix) Owning committer(s): Jason Brown (Apple) > Ensure Netty Internode Messaging Refactor is Solid > -- > > Key: CASSANDRA-14746 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14746 > Project: Cassandra > Issue Type: Improvement > Components: Streaming and Messaging >Reporter: Joseph Lynch >Priority: Major > Labels: 4.0-QA > Fix For: 4.0 > > > Before we release 4.0 let's ensure that the internode messaging refactor is > 100% solid. As internode messaging is naturally used in many code paths and > widely configurable we have a large number of cluster configurations and test > configurations that must be vetted. > We plan to vary the following: > * Version of Cassandra 3.0.17 vs 4.0-alpha > * Cluster sizes with *multi-dc* deployments ranging from 6 - 100 nodes > * Client request rates varying between 1k QPS and 100k QPS of varying sizes > and shapes (BATCH, INSERT, SELECT point, SELECT range, etc ...) > * Internode compression > * Internode SSL (as well as openssl vs jdk) > * Internode Coalescing options > We are looking to measure the following as appropriate: > * Latency distributions of reads and writes (lower is better) > * Scaling limit, aka maximum throughput before
[jira] [Commented] (CASSANDRA-14631) Add RSS support for Cassandra blog
[ https://issues.apache.org/jira/browse/CASSANDRA-14631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619941#comment-16619941 ] Nate McCall commented on CASSANDRA-14631: - That works for me - thanks, [~beckje01]. I'll get this put together later today (ie. set everything else down to concentrate on interacting with svn). > Add RSS support for Cassandra blog > -- > > Key: CASSANDRA-14631 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14631 > Project: Cassandra > Issue Type: Improvement > Components: Documentation and Website >Reporter: Jacques-Henri Berthemet >Assignee: Jeff Beck >Priority: Major > Labels: blog > Attachments: 14631-site.txt, Screen Shot 2018-08-17 at 5.32.08 > PM.png, Screen Shot 2018-08-17 at 5.32.25 PM.png > > > It would be convenient to add RSS support to Cassandra blog: > [http://cassandra.apache.org/blog/2018/08/07/faster_streaming_in_cassandra.html] > And maybe also for other resources like new versions, but this ticket is > about blog. > > {quote}From: Scott Andreas > Sent: Wednesday, August 08, 2018 6:53 PM > To: [d...@cassandra.apache.org|mailto:d...@cassandra.apache.org] > Subject: Re: Apache Cassandra Blog is now live > > Please feel free to file a ticket (label: Documentation and Website). > > It looks like Jekyll, the static site generator used to build the website, > has a plugin that generates Atom feeds if someone would like to work on > adding one: [https://github.com/jekyll/jekyll-feed] > {quote} -- 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
[jira] [Commented] (CASSANDRA-14553) Document troubleshooting page
[ https://issues.apache.org/jira/browse/CASSANDRA-14553?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619923#comment-16619923 ] Joseph Lynch commented on CASSANDRA-14553: -- [~jrwest] or [~rustyrazorblade] if you or someone has time to review these docs that would be really great. Happy to refactor any and everything :) > Document troubleshooting page > - > > Key: CASSANDRA-14553 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14553 > Project: Cassandra > Issue Type: Improvement > Components: Documentation and Website >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Minor > Labels: Documentation > > Currently the [troubleshooting > docs|http://cassandra.apache.org/doc/latest/troubleshooting/] are blank. As > much as I like to believe Cassandra never has any problems I was thinking of > writing up a troubleshooting page focussing on: > # Finding the hosts(s) that are behaving badly (common error messages) > # Which logs exist, where they are, and what to look for in which log > (common error messages, gc logs, etc) > # Which nodetool commands can give you more information > # Java/Operating systems tools that can help dive deep into performance > issues (jstat, top, iostat, cachestat, etc) > Since this is going to be a fairly lengthy page I wanted to get a jira going > in case someone else had ideas or had already started. Also if there are any > large areas I missed above please comment here and I can include them. > [~cscotta] fyi -- 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
[jira] [Updated] (CASSANDRA-14763) Fail incremental repair prepare phase if it encounters sstables from un-finalized sessions
[ https://issues.apache.org/jira/browse/CASSANDRA-14763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Blake Eggleston updated CASSANDRA-14763: Reviewer: Marcus Eriksson Status: Patch Available (was: Open) [trunk|https://github.com/bdeggleston/cassandra/tree/14763] [circle|https://circleci.com/workflow-run/03a36dfd-74b9-40e7-9284-b079f931f991] > Fail incremental repair prepare phase if it encounters sstables from > un-finalized sessions > -- > > Key: CASSANDRA-14763 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14763 > Project: Cassandra > Issue Type: Bug > Components: Repair >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Major > Fix For: 4.0 > > > Raised in CASSANDRA-14685. If we encounter sstables from other IR sessions > during an IR prepare phase, we should fail the new session. If we don't, the > expectation that all data received before a repair session is consistent when > it completes wouldn't always be true. > In more detail: > We don’t have a foolproof way of determining if a repair session has hung. To > prevent hung repair sessions from locking up sstables indefinitely, > incremental repair sessions will auto-fail after 24 hours. During this time, > the sstables for this session will remain isolated from the rest of the data > set. Afterwards, the sstables are moved back into the unrepaired set. > > During the prepare phase of an incremental repair, we isolate the data to be > repaired. However, we ignore other sstables marked pending repair for the > same token range. I think the intention here was to prevent a hung repair > from locking up incremental repairs for 24 hours without manual intervention. > Assuming the session succeeds, it’s data will be moved to repaired. _However > the data from a hung session will eventually be moved back to unrepaired._ > This means that you can’t use the most recent successful incremental repair > as the high water mark for fully repaired data. -- 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
[jira] [Updated] (CASSANDRA-14763) Fail incremental repair prepare phase if it encounters sstables from un-finalized sessions
[ https://issues.apache.org/jira/browse/CASSANDRA-14763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] C. Scott Andreas updated CASSANDRA-14763: - Component/s: Repair > Fail incremental repair prepare phase if it encounters sstables from > un-finalized sessions > -- > > Key: CASSANDRA-14763 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14763 > Project: Cassandra > Issue Type: Bug > Components: Repair >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Major > Fix For: 4.0 > > > Raised in CASSANDRA-14685. If we encounter sstables from other IR sessions > during an IR prepare phase, we should fail the new session. If we don't, the > expectation that all data received before a repair session is consistent when > it completes wouldn't always be true. > In more detail: > We don’t have a foolproof way of determining if a repair session has hung. To > prevent hung repair sessions from locking up sstables indefinitely, > incremental repair sessions will auto-fail after 24 hours. During this time, > the sstables for this session will remain isolated from the rest of the data > set. Afterwards, the sstables are moved back into the unrepaired set. > > During the prepare phase of an incremental repair, we isolate the data to be > repaired. However, we ignore other sstables marked pending repair for the > same token range. I think the intention here was to prevent a hung repair > from locking up incremental repairs for 24 hours without manual intervention. > Assuming the session succeeds, it’s data will be moved to repaired. _However > the data from a hung session will eventually be moved back to unrepaired._ > This means that you can’t use the most recent successful incremental repair > as the high water mark for fully repaired data. -- 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
[jira] [Created] (CASSANDRA-14763) Fail incremental repair prepare phase if it encounters sstables from un-finalized sessions
Blake Eggleston created CASSANDRA-14763: --- Summary: Fail incremental repair prepare phase if it encounters sstables from un-finalized sessions Key: CASSANDRA-14763 URL: https://issues.apache.org/jira/browse/CASSANDRA-14763 Project: Cassandra Issue Type: Bug Reporter: Blake Eggleston Assignee: Blake Eggleston Fix For: 4.0 Raised in CASSANDRA-14685. If we encounter sstables from other IR sessions during an IR prepare phase, we should fail the new session. If we don't, the expectation that all data received before a repair session is consistent when it completes wouldn't always be true. In more detail: We don’t have a foolproof way of determining if a repair session has hung. To prevent hung repair sessions from locking up sstables indefinitely, incremental repair sessions will auto-fail after 24 hours. During this time, the sstables for this session will remain isolated from the rest of the data set. Afterwards, the sstables are moved back into the unrepaired set. During the prepare phase of an incremental repair, we isolate the data to be repaired. However, we ignore other sstables marked pending repair for the same token range. I think the intention here was to prevent a hung repair from locking up incremental repairs for 24 hours without manual intervention. Assuming the session succeeds, it’s data will be moved to repaired. _However the data from a hung session will eventually be moved back to unrepaired._ This means that you can’t use the most recent successful incremental repair as the high water mark for fully repaired data. -- 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
[jira] [Commented] (CASSANDRA-14395) C* Management process
[ https://issues.apache.org/jira/browse/CASSANDRA-14395?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619604#comment-16619604 ] Vinay Chella commented on CASSANDRA-14395: -- We ([~jolynch] & I) gave a talk at [Distributed Data Summit - 2018|http://distributeddatasummit.com/2018-sf] to share more context around sidecar for C* and the operational simplicities that it brings with. Here are the slides if anyone is interested - [link|https://www.slideshare.net/VinayKumarChella/looking-towards-an-official-cassandra-sidecar-netflix] Attaching the slides (pdf) to this ticket as well - [^Looking towards an Official Cassandra Sidecar - Netflix.pdf] > C* Management process > - > > Key: CASSANDRA-14395 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14395 > Project: Cassandra > Issue Type: New Feature >Reporter: Dinesh Joshi >Assignee: Dinesh Joshi >Priority: Major > Attachments: Looking towards an Official Cassandra Sidecar - > Netflix.pdf > > > I would like to propose amending Cassandra's architecture to include a > management process. The detailed description is here: > https://docs.google.com/document/d/1UV9pE81NaIUF3g4L1wxq09nT11AkSQcMijgLFwGsY3s/edit > I'd like to propose seeding this with a few simple use-cases such as Health > Checks, Bulk Commands with a simple REST API interface. -- 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
[jira] [Updated] (CASSANDRA-14395) C* Management process
[ https://issues.apache.org/jira/browse/CASSANDRA-14395?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vinay Chella updated CASSANDRA-14395: - Attachment: Looking towards an Official Cassandra Sidecar - Netflix.pdf > C* Management process > - > > Key: CASSANDRA-14395 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14395 > Project: Cassandra > Issue Type: New Feature >Reporter: Dinesh Joshi >Assignee: Dinesh Joshi >Priority: Major > Attachments: Looking towards an Official Cassandra Sidecar - > Netflix.pdf > > > I would like to propose amending Cassandra's architecture to include a > management process. The detailed description is here: > https://docs.google.com/document/d/1UV9pE81NaIUF3g4L1wxq09nT11AkSQcMijgLFwGsY3s/edit > I'd like to propose seeding this with a few simple use-cases such as Health > Checks, Bulk Commands with a simple REST API interface. -- 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
[jira] [Commented] (CASSANDRA-14747) Evaluate 200 node, compression=none, encryption=none, coalescing=off
[ https://issues.apache.org/jira/browse/CASSANDRA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619566#comment-16619566 ] Joseph Lynch commented on CASSANDRA-14747: -- We're re-running the test today with the 14503 branch pulled in, will record results here. > Evaluate 200 node, compression=none, encryption=none, coalescing=off > - > > Key: CASSANDRA-14747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14747 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joseph Lynch >Assignee: Joseph Lynch >Priority: Major > Attachments: 3.0.17-QPS.png, 4.0.1-QPS.png, > 4.0_errors_showing_heap_pressure.txt, > 4.0_heap_histogram_showing_many_MessageOuts.txt, > useast1c-i-0e1ddfe8b2f769060-mutation-flame.svg > > > Tracks evaluating a 200 node cluster with all internode settings off (no > compression, no encryption, no coalescing). -- 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
[jira] [Commented] (CASSANDRA-14713) Add docker testing image to cassandra-builds
[ https://issues.apache.org/jira/browse/CASSANDRA-14713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619501#comment-16619501 ] Stefan Podkowinski commented on CASSANDRA-14713: I've managed to * update the Dockerfile provided by Michael to Ubuntu 18, replaced provided binaries with public packages, added Java 11 support ([result|https://github.com/spodkowinski/cassandra-builds/commit/5e90f42a040110a2c4cf672b4365daed24e3d8ec]) * uploaded image to [hub repo|https://hub.docker.com/r/spod/cassandra-testing-ubuntu18-java11/] * made used docker image in cassandra-builds configurable (old image by Michael still default), now also builds multiversion jar if Java 11 is present (as in new image) Next step would be to keep an eye on builds.apache.org and see if there are any issues and if tests can be run against the new image as before. If that's being the case, we can start making the new image the default for b.a.o and also circleCI. Afterwards we'd be more flexible in adding and picking new docker images for different JDKs and Linux environments. We'd also be able to optionally run tests against Java 11, as it's now also part of the image. > Add docker testing image to cassandra-builds > > > Key: CASSANDRA-14713 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14713 > Project: Cassandra > Issue Type: New Feature > Components: Testing >Reporter: Stefan Podkowinski >Assignee: Stefan Podkowinski >Priority: Major > Attachments: Dockerfile > > > Tests executed on builds.apache.org ({{docker/jenkins/jenkinscommand.sh}}) > and circleCI ({{.circleci/config.yml}}) will currently use the same > [cassandra-test|https://hub.docker.com/r/kjellman/cassandra-test/] docker > image ([github|https://github.com/mkjellman/cassandra-test-docker]) by > [~mkjellman]. > We should manage this image on our own as part of cassandra-builds, to keep > it updated. There's also a [Apache > user|https://hub.docker.com/u/apache/?page=1] on docker hub for publishing > images. -- 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
cassandra-dtest git commit: Ninja fix, add @since('4.0') for transient replication tests
Repository: cassandra-dtest Updated Branches: refs/heads/master 0d9c98ee1 -> 97529ccfb Ninja fix, add @since('4.0') for transient replication tests Project: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/commit/97529ccf Tree: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/tree/97529ccf Diff: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/diff/97529ccf Branch: refs/heads/master Commit: 97529ccfb338a986c6ca596d4e989dbf53a05551 Parents: 0d9c98e Author: Ariel Weisberg Authored: Tue Sep 18 13:51:46 2018 -0400 Committer: Ariel Weisberg Committed: Tue Sep 18 13:51:46 2018 -0400 -- transient_replication_ring_test.py | 27 --- transient_replication_test.py | 11 ++- 2 files changed, 14 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/97529ccf/transient_replication_ring_test.py -- diff --git a/transient_replication_ring_test.py b/transient_replication_ring_test.py index a3b596e..feea876 100644 --- a/transient_replication_ring_test.py +++ b/transient_replication_ring_test.py @@ -10,11 +10,13 @@ from tools.assertions import (assert_all) from flaky import flaky -from cassandra.metadata import BytesToken, OrderedDict +from cassandra.metadata import OrderedDict import pytest from itertools import chain from tools.misc import new_node +since = pytest.mark.since + logging.getLogger('cassandra').setLevel(logging.CRITICAL) NODELOCAL = 11 @@ -46,6 +48,8 @@ def patch_start(startable): startable.start = types.MethodType(new_start, startable) + +@since('4.0') class TestTransientReplicationRing(Tester): keyspace = "ks" @@ -125,14 +129,9 @@ class TestTransientReplicationRing(Tester): session.execute("CREATE KEYSPACE %s WITH REPLICATION={%s}" % (self.keyspace, replication_params)) print("CREATE KEYSPACE %s WITH REPLICATION={%s}" % (self.keyspace, replication_params)) -self.create_table(session) - -def create_table(self, session, never_speculate=False): -if never_speculate: -session.execute("CREATE TABLE %s.%s (pk varchar, ck int, value int, PRIMARY KEY (pk, ck)) WITH speculative_retry = 'NEVER' AND read_repair = 'NONE'" % (self.keyspace, self.table)) -else: -session.execute("CREATE TABLE %s.%s (pk varchar, ck int, value int, PRIMARY KEY (pk, ck)) WITH read_repair = 'NONE'" % (self.keyspace, self.table)) -print(str(self.node1.run_cqlsh("describe table %s.%s" % (self.keyspace, self.table +session.execute( +"CREATE TABLE %s.%s (pk varchar, ck int, value int, PRIMARY KEY (pk, ck)) WITH speculative_retry = 'NEVER' AND speculative_write_threshold = 'NEVER' AND read_repair = 'NONE'" % ( +self.keyspace, self.table)) def quorum(self, session, stmt_str): return session.execute(SimpleStatement(stmt_str, consistency_level=ConsistencyLevel.QUORUM)) @@ -150,8 +149,6 @@ class TestTransientReplicationRing(Tester): def test_bootstrap_and_cleanup(self): """Test bootstrapping a new node across a mix of repaired and unrepaired data""" main_session = self.patient_cql_connection(self.node1) -self.table = 'tbl2' -self.create_table(main_session, never_speculate=True) nodes = [self.node1, self.node2, self.node3] for i in range(0, 40, 2): @@ -232,8 +229,6 @@ class TestTransientReplicationRing(Tester): patch_start(node4) node4.start(wait_for_binary_proto=True, wait_other_notice=True) main_session = self.patient_cql_connection(self.node1) -self.table = 'tbl2' -self.create_table(main_session, never_speculate=True) nodes = [self.node1, self.node2, self.node3, node4] for i in range(0, 40, 2): @@ -345,8 +340,6 @@ class TestTransientReplicationRing(Tester): patch_start(node4) node4.start(wait_for_binary_proto=True, wait_other_notice=True) main_session = self.patient_cql_connection(self.node1) -self.table = 'tbl2' -self.create_table(main_session, never_speculate=True) nodes = [self.node1, self.node2, self.node3, node4] for i in range(0, 40, 2): @@ -408,8 +401,6 @@ class TestTransientReplicationRing(Tester): patch_start(node4) node4.start(wait_for_binary_proto=True, wait_other_notice=True) main_session = self.patient_cql_connection(self.node1) -self.table = 'tbl2' -self.create_table(main_session, never_speculate=True) nodes = [self.node1, self.node2, self.node3] #We want the node being removed to have no data on it so nodetool remove always gets all the necessary
[jira] [Commented] (CASSANDRA-14145) Detecting data resurrection during read
[ https://issues.apache.org/jira/browse/CASSANDRA-14145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619445#comment-16619445 ] ASF GitHub Bot commented on CASSANDRA-14145: Github user jrwest commented on a diff in the pull request: https://github.com/apache/cassandra-dtest/pull/37#discussion_r218529082 --- Diff: repair_tests/incremental_repair_test.py --- @@ -207,6 +208,7 @@ def test_manual_session_fail(self): self.fixture_dtest_setup.setup_overrides.cluster_options = ImmutableMapping({'hinted_handoff_enabled': 'false', 'num_tokens': 1, 'commitlog_sync_period_in_ms': 500}) +self.fixture_dtest_setup.init_default_config() --- End diff -- Yea the proxying to the fixture is a bit non-intuitive. iirc it was added to make the transition from nosetest -> pytest easier. > Detecting data resurrection during read > > > Key: CASSANDRA-14145 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14145 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Sam Tunnicliffe >Priority: Minor > Labels: pull-request-available > Fix For: 4.0 > > > We have seen several bugs in which deleted data gets resurrected. We should > try to see if we can detect this on the read path and possibly fix it. Here > are a few examples which brought back data > A replica lost an sstable on startup which caused one replica to lose the > tombstone and not the data. This tombstone was past gc grace which means this > could resurrect data. We can detect such invalid states by looking at other > replicas. > If we are running incremental repair, Cassandra will keep repaired and > non-repaired data separate. Every-time incremental repair will run, it will > move the data from non-repaired to repaired. Repaired data across all > replicas should be 100% consistent. > Here is an example of how we can detect and mitigate the issue in most cases. > Say we have 3 machines, A,B and C. All these machines will have data split > b/w repaired and non-repaired. > 1. Machine A due to some bug bring backs data D. This data D is in repaired > dataset. All other replicas will have data D and tombstone T > 2. Read for data D comes from application which involve replicas A and B. The > data being read involves data which is in repaired state. A will respond > back to co-ordinator with data D and B will send nothing as tombstone is past > gc grace. This will cause digest mismatch. > 3. This patch will only kick in when there is a digest mismatch. Co-ordinator > will ask both replicas to send back all data like we do today but with this > patch, replicas will respond back what data it is returning is coming from > repaired vs non-repaired. If data coming from repaired does not match, we > know there is a something wrong!! At this time, co-ordinator cannot determine > if replica A has resurrected some data or replica B has lost some data. We > can still log error in the logs saying we hit an invalid state. > 4. Besides the log, we can take this further and even correct the response to > the query. After logging an invalid state, we can ask replica A and B (and > also C if alive) to send back all data for this including gcable tombstones. > If any machine returns a tombstone which is after this data, we know we > cannot return this data. This way we can avoid returning data which has been > deleted. > Some Challenges with this > 1. When data will be moved from non-repaired to repaired, there could be a > race here. We can look at which incremental repairs have promoted things on > which replica to avoid false positives. > 2. If the third replica is down and live replica does not have any tombstone, > we wont be able to break the tie in deciding whether data was actually > deleted or resurrected. > 3. If the read is for latest data only, we wont be able to detect it as the > read will be served from non-repaired data. > 4. If the replica where we lose a tombstone is the last replica to compact > the tombstone, we wont be able to decide if data is coming back or rest of > the replicas has lost that data. But we will still detect something is wrong. > 5. We wont affect 99.9% of the read queries as we only do extra work during > digest mismatch. > 6. CL.ONE reads will not be able to detect this. -- 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:
[jira] [Created] (CASSANDRA-14762) Transient node receives full data requests in dtests
Ariel Weisberg created CASSANDRA-14762: -- Summary: Transient node receives full data requests in dtests Key: CASSANDRA-14762 URL: https://issues.apache.org/jira/browse/CASSANDRA-14762 Project: Cassandra Issue Type: Bug Components: Core Reporter: Ariel Weisberg Fix For: 4.0 I saw this running them on my laptop with rapid write protection disabled. Attached is a patch for disabling rapid write protection in the transient dtests. {noformat} .Exception in thread Thread-19: Traceback (most recent call last): File "/usr/local/Cellar/python/3.6.4_4/Frameworks/Python.framework/Versions/3.6/lib/python3.6/threading.py", line 916, in _bootstrap_inner self.run() File "/Users/aweisberg/repos/cassandra-dtest/venv/src/ccm/ccmlib/cluster.py", line 180, in run self.scan_and_report() File "/Users/aweisberg/repos/cassandra-dtest/venv/src/ccm/ccmlib/cluster.py", line 173, in scan_and_report on_error_call(errordata) File "/Users/aweisberg/repos/cassandra-dtest/dtest_setup.py", line 137, in _log_error_handler pytest.fail("Error details: \n{message}".format(message=message)) File "/Users/aweisberg/repos/cassandra-dtest/venv/lib/python3.6/site-packages/_pytest/outcomes.py", line 96, in fail raise Failed(msg=msg, pytrace=pytrace) Failed: Error details: Errors seen in logs for: node3 node3: ERROR [ReadStage-1] 2018-09-18 12:28:48,344 AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread Thread[ReadStage-1,5,main] org.apache.cassandra.exceptions.InvalidRequestException: Attempted to serve transient data request from full node in org.apache.cassandra.db.ReadCommandVerbHandler@3c55e0ff at org.apache.cassandra.db.ReadCommandVerbHandler.validateTransientStatus(ReadCommandVerbHandler.java:104) at org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:53) at org.apache.cassandra.net.MessageDeliveryTask.process(MessageDeliveryTask.java:92) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:54) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134) at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:110) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:748) {noformat} -- 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
[jira] [Created] (CASSANDRA-14761) Rename speculative_write_threshold to something more appropriate
Ariel Weisberg created CASSANDRA-14761: -- Summary: Rename speculative_write_threshold to something more appropriate Key: CASSANDRA-14761 URL: https://issues.apache.org/jira/browse/CASSANDRA-14761 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Ariel Weisberg Fix For: 4.0 It's not really speculative. This commit is where it was last named and shows what to update https://github.com/aweisberg/cassandra/commit/e1df8e977d942a1b0da7c2a7554149c781d0e6c3 -- 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
cassandra-builds git commit: Make docker image for dtests configurable
Repository: cassandra-builds Updated Branches: refs/heads/master aecee8ef5 -> 6fd11d6e2 Make docker image for dtests configurable Also build against Java 11 if on trunk and Java 11 is available (Java 8 is still used for test execution) Project: http://git-wip-us.apache.org/repos/asf/cassandra-builds/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra-builds/commit/6fd11d6e Tree: http://git-wip-us.apache.org/repos/asf/cassandra-builds/tree/6fd11d6e Diff: http://git-wip-us.apache.org/repos/asf/cassandra-builds/diff/6fd11d6e Branch: refs/heads/master Commit: 6fd11d6e2f24cbd619fe229044f5e93626455413 Parents: aecee8e Author: Stefan Podkowinski Authored: Mon Sep 17 15:22:57 2018 +0200 Committer: Stefan Podkowinski Committed: Tue Sep 18 19:35:19 2018 +0200 -- build-scripts/cassandra-dtest-pytest.sh | 12 docker/jenkins/jenkinscommand.sh | 11 --- jenkins-dsl/cassandra_job_dsl_seed.groovy | 9 +++-- 3 files changed, 27 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra-builds/blob/6fd11d6e/build-scripts/cassandra-dtest-pytest.sh -- diff --git a/build-scripts/cassandra-dtest-pytest.sh b/build-scripts/cassandra-dtest-pytest.sh index 7eae11e..9602cf9 100755 --- a/build-scripts/cassandra-dtest-pytest.sh +++ b/build-scripts/cassandra-dtest-pytest.sh @@ -21,6 +21,13 @@ export CASSANDRA_DIR=${WORKSPACE} #Have Cassandra skip all fsyncs to improve test performance and reliability export CASSANDRA_SKIP_SYNC=true +# set JAVA_HOME environment to enable multi-version jar files for >4.0 +# both JAVA8/11_HOME env variables must exist +grep -q _build_multi_java $CASSANDRA_DIR/build.xml +if [ $? -eq 0 -a -n "$JAVA8_HOME" -a -n "$JAVA11_HOME" ]; then + export JAVA_HOME="$JAVA11_HOME" +fi + # Loop to prevent failure due to maven-ant-tasks not downloading a jar.. for x in $(seq 1 3); do ant clean jar @@ -35,6 +42,11 @@ if [ "${RETURN}" -ne "0" ]; then exit ${RETURN} fi +# restore JAVA_HOME to Java 8 version we intent to run tests with +if [ -n "$JAVA8_HOME" ]; then + export JAVA_HOME="$JAVA8_HOME" +fi + # Set up venv with dtest dependencies set -e # enable immediate exit if venv setup fails virtualenv --python=python3 --no-site-packages venv http://git-wip-us.apache.org/repos/asf/cassandra-builds/blob/6fd11d6e/docker/jenkins/jenkinscommand.sh -- diff --git a/docker/jenkins/jenkinscommand.sh b/docker/jenkins/jenkinscommand.sh index 8a29dd7..e342ed4 100644 --- a/docker/jenkins/jenkinscommand.sh +++ b/docker/jenkins/jenkinscommand.sh @@ -1,15 +1,20 @@ #!/bin/sh -DOCKER_IMAGE="kjellman/cassandra-test:0.4.4" +if [ "$#" -lt 7 ]; then + echo "Usage: jenkinscommand.sh GITHUB_USER BRANCH DTEST_REPO_URL DTEST_BRANCH BUILDS_REPO_URL BUILDS_BRANCH DOCKER_IMAGE [target]" + exit 1 +fi BUILDSREPO=$5 BUILDSBRANCH=$6 +DOCKER_IMAGE=$7 +TARGET=$8 cat > env.list
[jira] [Updated] (CASSANDRA-14757) GCInspector "Error accessing field of java.nio.Bits" under java11
[ https://issues.apache.org/jira/browse/CASSANDRA-14757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-14757: - Status: Patch Available (was: In Progress) > GCInspector "Error accessing field of java.nio.Bits" under java11 > - > > Key: CASSANDRA-14757 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14757 > Project: Cassandra > Issue Type: Bug > Components: Metrics >Reporter: Jason Brown >Assignee: Robert Stupp >Priority: Trivial > Labels: Java11, pull-request-available > Fix For: 4.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Running under java11, {{GCInspector}} throws the following exception: > {noformat} > DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing > field of java.nio.Bits > java.lang.NoSuchFieldException: totalCapacity > at java.base/java.lang.Class.getDeclaredField(Class.java:2412) > at > org.apache.cassandra.service.GCInspector.(GCInspector.java:72) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) > {noformat} > This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} > from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} > somewhere between java8 and java11. > Note: this is a rather harmless error, as we only look at > {{Bits.totalCapacity}} for metrics collection on how much direct memory is > being used by {{ByteBuffer}}s. If we fail to read the field, we simply return > -1 for the metric value. -- 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
[jira] [Updated] (CASSANDRA-14757) GCInspector "Error accessing field of java.nio.Bits" under java11
[ https://issues.apache.org/jira/browse/CASSANDRA-14757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated CASSANDRA-14757: --- Labels: Java11 pull-request-available (was: Java11) > GCInspector "Error accessing field of java.nio.Bits" under java11 > - > > Key: CASSANDRA-14757 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14757 > Project: Cassandra > Issue Type: Bug > Components: Metrics >Reporter: Jason Brown >Assignee: Robert Stupp >Priority: Trivial > Labels: Java11, pull-request-available > Fix For: 4.0 > > > Running under java11, {{GCInspector}} throws the following exception: > {noformat} > DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing > field of java.nio.Bits > java.lang.NoSuchFieldException: totalCapacity > at java.base/java.lang.Class.getDeclaredField(Class.java:2412) > at > org.apache.cassandra.service.GCInspector.(GCInspector.java:72) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) > {noformat} > This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} > from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} > somewhere between java8 and java11. > Note: this is a rather harmless error, as we only look at > {{Bits.totalCapacity}} for metrics collection on how much direct memory is > being used by {{ByteBuffer}}s. If we fail to read the field, we simply return > -1 for the metric value. -- 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
[jira] [Commented] (CASSANDRA-14685) Incremental repair 4.0 : SSTables remain locked forever if the coordinator dies during streaming
[ https://issues.apache.org/jira/browse/CASSANDRA-14685?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619399#comment-16619399 ] Blake Eggleston commented on CASSANDRA-14685: - bq. I guess it would be worth failing a repair if some SSTables with overlapping token ranges are still part of another repair session. Doing that would cause a hung session to lock up incremental repair for a range until the session finishes or is manually failed. However, if you’re attempting to have no more than a few hours of data unrepaired at any time, that’s probably preferable to silently keeping some data marked pending repair and reintroducing it back to the unrepaired pool after a day. [~krummas], [~jjirsa], [~mbyrd]: do you have any thoughts on this? Seems like failing new repair sessions until sstables from non-finalized sessions is the right thing to do here. > Incremental repair 4.0 : SSTables remain locked forever if the coordinator > dies during streaming > - > > Key: CASSANDRA-14685 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14685 > Project: Cassandra > Issue Type: Bug > Components: Repair >Reporter: Alexander Dejanovski >Assignee: Jason Brown >Priority: Critical > > The changes in CASSANDRA-9143 modified the way incremental repair performs by > applying the following sequence of events : > * Anticompaction is executed on all replicas for all SSTables overlapping > the repaired ranges > * Anticompacted SSTables are then marked as "Pending repair" and cannot be > compacted anymore, nor part of another repair session > * Merkle trees are generated and compared > * Streaming takes place if needed > * Anticompaction is committed and "pending repair" table are marked as > repaired if it succeeded, or they are released if the repair session failed. > If the repair coordinator dies during the streaming phase, *the SSTables on > the replicas will remain in "pending repair" state and will never be eligible > for repair or compaction*, even after all the nodes in the cluster are > restarted. > Steps to reproduce (I've used Jason's 13938 branch that fixes streaming > errors) : > {noformat} > ccm create inc-repair-issue -v github:jasobrown/13938 -n 3 > # Allow jmx access and remove all rpc_ settings in yaml > for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra-env.sh; > do > sed -i'' -e > 's/com.sun.management.jmxremote.authenticate=true/com.sun.management.jmxremote.authenticate=false/g' > $f > done > for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra.yaml; > do > grep -v "rpc_" $f > ${f}.tmp > cat ${f}.tmp > $f > done > ccm start > {noformat} > I used [tlp-stress|https://github.com/thelastpickle/tlp-stress] to generate a > few 10s of MBs of data (killed it after some time). Obviously > cassandra-stress works as well : > {noformat} > bin/tlp-stress run BasicTimeSeries -i 1M -p 1M -t 2 --rate 5000 > --replication "{'class':'SimpleStrategy', 'replication_factor':2}" > --compaction "{'class': 'SizeTieredCompactionStrategy'}" --host > 127.0.0.1 > {noformat} > Flush and delete all SSTables in node1 : > {noformat} > ccm node1 nodetool flush > ccm node1 stop > rm -f ~/.ccm/inc-repair-issue/node1/data0/tlp_stress/sensor*/*.* > ccm node1 start{noformat} > Then throttle streaming throughput to 1MB/s so we have time to take node1 > down during the streaming phase and run repair: > {noformat} > ccm node1 nodetool setstreamthroughput 1 > ccm node2 nodetool setstreamthroughput 1 > ccm node3 nodetool setstreamthroughput 1 > ccm node1 nodetool repair tlp_stress > {noformat} > Once streaming starts, shut down node1 and start it again : > {noformat} > ccm node1 stop > ccm node1 start > {noformat} > Run repair again : > {noformat} > ccm node1 nodetool repair tlp_stress > {noformat} > The command will return very quickly, showing that it skipped all sstables : > {noformat} > [2018-08-31 19:05:16,292] Repair completed successfully > [2018-08-31 19:05:16,292] Repair command #1 finished in 2 seconds > $ ccm node1 nodetool status > Datacenter: datacenter1 > === > Status=Up/Down > |/ State=Normal/Leaving/Joining/Moving > -- AddressLoad Tokens OwnsHost ID >Rack > UN 127.0.0.1 228,64 KiB 256 ? > 437dc9cd-b1a1-41a5-961e-cfc99763e29f rack1 > UN 127.0.0.2 60,09 MiB 256 ? > fbcbbdbb-e32a-4716-8230-8ca59aa93e62 rack1 > UN 127.0.0.3 57,59 MiB 256 ? > a0b1bcc6-0fad-405a-b0bf-180a0ca31dd0 rack1 > {noformat} > sstablemetadata will then show that nodes 2 and 3 have SSTables still in > "pending repair" state : > {noformat} > ~/.ccm/repository/gitCOLONtrunk/tools/bin/sstablemetadata
[jira] [Commented] (CASSANDRA-14631) Add RSS support for Cassandra blog
[ https://issues.apache.org/jira/browse/CASSANDRA-14631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619385#comment-16619385 ] Jeff Beck commented on CASSANDRA-14631: --- Anything else we want to do with this? > Add RSS support for Cassandra blog > -- > > Key: CASSANDRA-14631 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14631 > Project: Cassandra > Issue Type: Improvement > Components: Documentation and Website >Reporter: Jacques-Henri Berthemet >Assignee: Jeff Beck >Priority: Major > Labels: blog > Attachments: 14631-site.txt, Screen Shot 2018-08-17 at 5.32.08 > PM.png, Screen Shot 2018-08-17 at 5.32.25 PM.png > > > It would be convenient to add RSS support to Cassandra blog: > [http://cassandra.apache.org/blog/2018/08/07/faster_streaming_in_cassandra.html] > And maybe also for other resources like new versions, but this ticket is > about blog. > > {quote}From: Scott Andreas > Sent: Wednesday, August 08, 2018 6:53 PM > To: [d...@cassandra.apache.org|mailto:d...@cassandra.apache.org] > Subject: Re: Apache Cassandra Blog is now live > > Please feel free to file a ticket (label: Documentation and Website). > > It looks like Jekyll, the static site generator used to build the website, > has a plugin that generates Atom feeds if someone would like to work on > adding one: [https://github.com/jekyll/jekyll-feed] > {quote} -- 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
[jira] [Assigned] (CASSANDRA-14757) GCInspector "Error accessing field of java.nio.Bits" under java11
[ https://issues.apache.org/jira/browse/CASSANDRA-14757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp reassigned CASSANDRA-14757: Assignee: Robert Stupp > GCInspector "Error accessing field of java.nio.Bits" under java11 > - > > Key: CASSANDRA-14757 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14757 > Project: Cassandra > Issue Type: Bug > Components: Metrics >Reporter: Jason Brown >Assignee: Robert Stupp >Priority: Trivial > Labels: Java11 > Fix For: 4.0 > > > Running under java11, {{GCInspector}} throws the following exception: > {noformat} > DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing > field of java.nio.Bits > java.lang.NoSuchFieldException: totalCapacity > at java.base/java.lang.Class.getDeclaredField(Class.java:2412) > at > org.apache.cassandra.service.GCInspector.(GCInspector.java:72) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) > {noformat} > This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} > from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} > somewhere between java8 and java11. > Note: this is a rather harmless error, as we only look at > {{Bits.totalCapacity}} for metrics collection on how much direct memory is > being used by {{ByteBuffer}}s. If we fail to read the field, we simply return > -1 for the metric value. -- 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
[jira] [Commented] (CASSANDRA-14760) CVE-2018-10237 Security vulnerability in 3.11.3
[ https://issues.apache.org/jira/browse/CASSANDRA-14760?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619378#comment-16619378 ] Jason Brown commented on CASSANDRA-14760: - The CVE seems to apply to only to: - AtomicDoubleArray (when serialized with Java serialization) - CompoundOrdering (when serialized with GWT serialization) Cassandra uses neither of those classes, nor do we use Java nor GWT serialization. Thus, it's not clear this CVE is a problem for us. > CVE-2018-10237 Security vulnerability in 3.11.3 > --- > > Key: CASSANDRA-14760 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14760 > Project: Cassandra > Issue Type: Bug >Reporter: John F. Gbruoski >Priority: Major > > As described in the CVE, Guava 11.0 through 24.x before 24.1.1 have a > security exposure. Cassandra 3.11.3 uses Guava 18.0. Can Cassandra 3.11 be > patched to support Guava 24.1.1 or later? -- 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
[jira] [Commented] (CASSANDRA-14685) Incremental repair 4.0 : SSTables remain locked forever if the coordinator dies during streaming
[ https://issues.apache.org/jira/browse/CASSANDRA-14685?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619368#comment-16619368 ] Jason Brown commented on CASSANDRA-14685: - {quote}One weird behavior of streaming is that when the coordinator goes down, "nodetool netstats" still shows progress on the replicas until it reaches 100% and it stays like this. It even starts streaming new files although the target node is still down {quote} I discovered that, as well, when investigating this one. I have a working fix for it, as well as CASSANDRA-14520, and am working out the kinks. Hoping to get it out ASAP.. > Incremental repair 4.0 : SSTables remain locked forever if the coordinator > dies during streaming > - > > Key: CASSANDRA-14685 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14685 > Project: Cassandra > Issue Type: Bug > Components: Repair >Reporter: Alexander Dejanovski >Assignee: Jason Brown >Priority: Critical > > The changes in CASSANDRA-9143 modified the way incremental repair performs by > applying the following sequence of events : > * Anticompaction is executed on all replicas for all SSTables overlapping > the repaired ranges > * Anticompacted SSTables are then marked as "Pending repair" and cannot be > compacted anymore, nor part of another repair session > * Merkle trees are generated and compared > * Streaming takes place if needed > * Anticompaction is committed and "pending repair" table are marked as > repaired if it succeeded, or they are released if the repair session failed. > If the repair coordinator dies during the streaming phase, *the SSTables on > the replicas will remain in "pending repair" state and will never be eligible > for repair or compaction*, even after all the nodes in the cluster are > restarted. > Steps to reproduce (I've used Jason's 13938 branch that fixes streaming > errors) : > {noformat} > ccm create inc-repair-issue -v github:jasobrown/13938 -n 3 > # Allow jmx access and remove all rpc_ settings in yaml > for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra-env.sh; > do > sed -i'' -e > 's/com.sun.management.jmxremote.authenticate=true/com.sun.management.jmxremote.authenticate=false/g' > $f > done > for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra.yaml; > do > grep -v "rpc_" $f > ${f}.tmp > cat ${f}.tmp > $f > done > ccm start > {noformat} > I used [tlp-stress|https://github.com/thelastpickle/tlp-stress] to generate a > few 10s of MBs of data (killed it after some time). Obviously > cassandra-stress works as well : > {noformat} > bin/tlp-stress run BasicTimeSeries -i 1M -p 1M -t 2 --rate 5000 > --replication "{'class':'SimpleStrategy', 'replication_factor':2}" > --compaction "{'class': 'SizeTieredCompactionStrategy'}" --host > 127.0.0.1 > {noformat} > Flush and delete all SSTables in node1 : > {noformat} > ccm node1 nodetool flush > ccm node1 stop > rm -f ~/.ccm/inc-repair-issue/node1/data0/tlp_stress/sensor*/*.* > ccm node1 start{noformat} > Then throttle streaming throughput to 1MB/s so we have time to take node1 > down during the streaming phase and run repair: > {noformat} > ccm node1 nodetool setstreamthroughput 1 > ccm node2 nodetool setstreamthroughput 1 > ccm node3 nodetool setstreamthroughput 1 > ccm node1 nodetool repair tlp_stress > {noformat} > Once streaming starts, shut down node1 and start it again : > {noformat} > ccm node1 stop > ccm node1 start > {noformat} > Run repair again : > {noformat} > ccm node1 nodetool repair tlp_stress > {noformat} > The command will return very quickly, showing that it skipped all sstables : > {noformat} > [2018-08-31 19:05:16,292] Repair completed successfully > [2018-08-31 19:05:16,292] Repair command #1 finished in 2 seconds > $ ccm node1 nodetool status > Datacenter: datacenter1 > === > Status=Up/Down > |/ State=Normal/Leaving/Joining/Moving > -- AddressLoad Tokens OwnsHost ID >Rack > UN 127.0.0.1 228,64 KiB 256 ? > 437dc9cd-b1a1-41a5-961e-cfc99763e29f rack1 > UN 127.0.0.2 60,09 MiB 256 ? > fbcbbdbb-e32a-4716-8230-8ca59aa93e62 rack1 > UN 127.0.0.3 57,59 MiB 256 ? > a0b1bcc6-0fad-405a-b0bf-180a0ca31dd0 rack1 > {noformat} > sstablemetadata will then show that nodes 2 and 3 have SSTables still in > "pending repair" state : > {noformat} > ~/.ccm/repository/gitCOLONtrunk/tools/bin/sstablemetadata na-4-big-Data.db | > grep repair > SSTable: > /Users/adejanovski/.ccm/inc-repair-4.0/node2/data0/tlp_stress/sensor_data-b7375660ad3111e8a0e59357ff9c9bda/na-4-big > Pending repair: 3844a400-ad33-11e8-b5a7-6b8dd8f31b62 > {noformat} > Restarting these nodes
[jira] [Updated] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2
[ https://issues.apache.org/jira/browse/CASSANDRA-1?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated CASSANDRA-1: --- Labels: pull-request-available (was: ) > Got NPE when querying Cassandra 3.11.2 > -- > > Key: CASSANDRA-1 > URL: https://issues.apache.org/jira/browse/CASSANDRA-1 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Ubuntu 14.04, JDK 1.8.0_171. > Cassandra 3.11.2 >Reporter: Xiaodong Xie >Priority: Blocker > Labels: pull-request-available > > We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2 > After upgrading, we immediately got exceptions in Cassandra like this one: > > {code} > ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 > QueryMessage.java:129 - Unexpected error during query > java.lang.NullPointerException: null > at > org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517) > [apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410) > [apache-cassandra-3.11.2.jar:3.11.2] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_171] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > [apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [apache-cassandra-3.11.2.jar:3.11.2] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171] > {code} > > The table schema is like: > {code} > CREATE TABLE example.example_table ( > id bigint, > hash text, > json text, > PRIMARY KEY (id, hash) > ) WITH COMPACT STORAGE > {code} > > The query is something like: > {code} > "select * from example.example_table;" // (We do know this is bad practise, > and we are trying to fix that right now) > {code} > with fetch-size as 200, using DataStax Java driver. > This table contains about 20k rows. > > Actually, the fix is quite simple, > > {code} > --- a/src/java/org/apache/cassandra/service/pager/PagingState.java > +++ b/src/java/org/apache/cassandra/service/pager/PagingState.java > @@ -46,7 +46,7 @@ public class PagingState > public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, > int remainingInPartition) > { > - this.partitionKey = partitionKey; > + this.partitionKey = partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER > : partitionKey; > this.rowMark = rowMark; > this.remaining = remaining; > this.remainingInPartition = remainingInPartition; > {code} > >
[jira] [Created] (CASSANDRA-14760) CVE-2018-10237 Security vulnerability in 3.11.3
John F. Gbruoski created CASSANDRA-14760: Summary: CVE-2018-10237 Security vulnerability in 3.11.3 Key: CASSANDRA-14760 URL: https://issues.apache.org/jira/browse/CASSANDRA-14760 Project: Cassandra Issue Type: Bug Reporter: John F. Gbruoski As described in the CVE, Guava 11.0 through 24.x before 24.1.1 have a security exposure. Cassandra 3.11.3 uses Guava 18.0. Can Cassandra 3.11 be patched to support Guava 24.1.1 or later? -- 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
[jira] [Commented] (CASSANDRA-14726) ReplicaCollection follow-up
[ https://issues.apache.org/jira/browse/CASSANDRA-14726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619343#comment-16619343 ] Benedict commented on CASSANDRA-14726: -- Initial patch available [here|https://github.com/apache/cassandra/pull/271] This introduces two internal classes: {{ReplicaList}} and {{ReplicaMap}} that permit more efficient implementation of {{AbstractReplicaCollection}}. It also introduces a {{filterLazily}} method, and cleans up the caching (and use thereof) of transient/full sub collections in {{RangesAtEndpoint}}. I'm also open to introducing any {{anyMatch}} and {{allMatch}} if people think it's a good idea. It may be that this change is seen as unnecessary, but since these classes are all new, introducing these optimisations now seems like probably the best course of action. I will follow up with some extra tests, though the new classes are *mostly* covered already through the existing tests. > ReplicaCollection follow-up > --- > > Key: CASSANDRA-14726 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14726 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Benedict >Priority: Major > > We introduced \{{ReplicaCollection}} as part of CASSANDRA-14404, but while it > improves readability, we could do more to ensure it minimises extra garbage, > and does not otherwise unnecessarily waste cycles. -- 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
[jira] [Comment Edited] (CASSANDRA-14759) Transient->Full movements mishandle consistency level upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-14759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619334#comment-16619334 ] Benedict edited comment on CASSANDRA-14759 at 9/18/18 4:01 PM: --- bq. use of an enum instead of true/false might be a hair easier to follow for that API -Agreed, I had that thought before too. I'll update the API.- Actually, the method is no longer used, so I'll remove it (for now) was (Author: benedict): bq. use of an enum instead of true/false might be a hair easier to follow for that API Agreed, I had that thought before too. I'll update the API. > Transient->Full movements mishandle consistency level upgrade > - > > Key: CASSANDRA-14759 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14759 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Benedict >Assignee: Benedict >Priority: Major > Labels: Availability, correctness, transient-replication > Fix For: 4.0 > > > While we need treat a transitioning node as ‘full’ for writes, so that it can > safely begin serving full data requests once it has finished, we cannot > maintain it in the ‘pending’ collection else we will also increase our > consistency requirements by a node that doesn’t exist. -- 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
[jira] [Commented] (CASSANDRA-14759) Transient->Full movements mishandle consistency level upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-14759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619334#comment-16619334 ] Benedict commented on CASSANDRA-14759: -- bq. use of an enum instead of true/false might be a hair easier to follow for that API Agreed, I had that thought before too. I'll update the API. > Transient->Full movements mishandle consistency level upgrade > - > > Key: CASSANDRA-14759 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14759 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Benedict >Assignee: Benedict >Priority: Major > Labels: Availability, correctness, transient-replication > Fix For: 4.0 > > > While we need treat a transitioning node as ‘full’ for writes, so that it can > safely begin serving full data requests once it has finished, we cannot > maintain it in the ‘pending’ collection else we will also increase our > consistency requirements by a node that doesn’t exist. -- 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
[jira] [Commented] (CASSANDRA-14759) Transient->Full movements mishandle consistency level upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-14759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619311#comment-16619311 ] Ariel Weisberg commented on CASSANDRA-14759: I think I understand this. It's fixing a deficiency in write availability where transient -> full would cause that replica to be unavailable for meeting write quorum requirements. Since it would remain present in pending it would also increase group size in blockFor. {code} return natural.filter(r -> !r.isTransient() || !pending.contains(r.endpoint(), true)); {code} Specifically the {{!pending.contains(r.endpoint(), true)}} (use of an enum instead of true/false might be a hair easier to follow for that API) causes it to no longer appear in natural and in pending instead. Once you have the tests I'll do a final review. > Transient->Full movements mishandle consistency level upgrade > - > > Key: CASSANDRA-14759 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14759 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Benedict >Assignee: Benedict >Priority: Major > Labels: Availability, correctness, transient-replication > Fix For: 4.0 > > > While we need treat a transitioning node as ‘full’ for writes, so that it can > safely begin serving full data requests once it has finished, we cannot > maintain it in the ‘pending’ collection else we will also increase our > consistency requirements by a node that doesn’t exist. -- 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
[jira] [Updated] (CASSANDRA-14759) Transient->Full movements mishandle consistency level upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-14759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-14759: - Status: Patch Available (was: Open) > Transient->Full movements mishandle consistency level upgrade > - > > Key: CASSANDRA-14759 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14759 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Benedict >Assignee: Benedict >Priority: Major > Labels: Availability, correctness, transient-replication > Fix For: 4.0 > > > While we need treat a transitioning node as ‘full’ for writes, so that it can > safely begin serving full data requests once it has finished, we cannot > maintain it in the ‘pending’ collection else we will also increase our > consistency requirements by a node that doesn’t exist. -- 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
[jira] [Commented] (CASSANDRA-14759) Transient->Full movements mishandle consistency level upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-14759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619272#comment-16619272 ] Benedict commented on CASSANDRA-14759: -- I thought I had already committed this as one of my other patches, but it seems not. I hope you don't mind both being tagged reviewers, since you've been reviewing my other TR follow ups, but shis bug is a proper Blocker for 4.0. [PR|https://github.com/apache/cassandra/pull/270] [CI|https://circleci.com/workflow-run/636b2eae-2d9e-4245-ad6b-462eab583f13] I will follow up with some tests. > Transient->Full movements mishandle consistency level upgrade > - > > Key: CASSANDRA-14759 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14759 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Benedict >Assignee: Benedict >Priority: Major > Labels: Availability, correctness, transient-replication > Fix For: 4.0 > > > While we need treat a transitioning node as ‘full’ for writes, so that it can > safely begin serving full data requests once it has finished, we cannot > maintain it in the ‘pending’ collection else we will also increase our > consistency requirements by a node that doesn’t exist. -- 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
[jira] [Commented] (CASSANDRA-14685) Incremental repair 4.0 : SSTables remain locked forever if the coordinator dies during streaming
[ https://issues.apache.org/jira/browse/CASSANDRA-14685?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619273#comment-16619273 ] Alexander Dejanovski commented on CASSANDRA-14685: -- Hi [~bdeggleston], Fair enough, I reckon I didn't wait that long for the SSTables to be released. if the SSTables get released eventually and you can't detect all types of failures to release them, I guess it would be worth failing a repair if some SSTables with overlapping token ranges are still part of another repair session. Otherwise, your left with the impression that running a repair would work correctly although some SSTables were skipped (and will be rolled back later). Wdyt? Advising to use "nodetool repair_admin" in the error message would help discover this new command. Stopping the session using it did the trick and the SSTables were released as expected. One weird behavior of streaming is that when the coordinator goes down, "nodetool netstats" still shows progress on the replicas until it reaches 100% and it stays like this. It even starts streaming new files although the target node is still down. > Incremental repair 4.0 : SSTables remain locked forever if the coordinator > dies during streaming > - > > Key: CASSANDRA-14685 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14685 > Project: Cassandra > Issue Type: Bug > Components: Repair >Reporter: Alexander Dejanovski >Assignee: Jason Brown >Priority: Critical > > The changes in CASSANDRA-9143 modified the way incremental repair performs by > applying the following sequence of events : > * Anticompaction is executed on all replicas for all SSTables overlapping > the repaired ranges > * Anticompacted SSTables are then marked as "Pending repair" and cannot be > compacted anymore, nor part of another repair session > * Merkle trees are generated and compared > * Streaming takes place if needed > * Anticompaction is committed and "pending repair" table are marked as > repaired if it succeeded, or they are released if the repair session failed. > If the repair coordinator dies during the streaming phase, *the SSTables on > the replicas will remain in "pending repair" state and will never be eligible > for repair or compaction*, even after all the nodes in the cluster are > restarted. > Steps to reproduce (I've used Jason's 13938 branch that fixes streaming > errors) : > {noformat} > ccm create inc-repair-issue -v github:jasobrown/13938 -n 3 > # Allow jmx access and remove all rpc_ settings in yaml > for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra-env.sh; > do > sed -i'' -e > 's/com.sun.management.jmxremote.authenticate=true/com.sun.management.jmxremote.authenticate=false/g' > $f > done > for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra.yaml; > do > grep -v "rpc_" $f > ${f}.tmp > cat ${f}.tmp > $f > done > ccm start > {noformat} > I used [tlp-stress|https://github.com/thelastpickle/tlp-stress] to generate a > few 10s of MBs of data (killed it after some time). Obviously > cassandra-stress works as well : > {noformat} > bin/tlp-stress run BasicTimeSeries -i 1M -p 1M -t 2 --rate 5000 > --replication "{'class':'SimpleStrategy', 'replication_factor':2}" > --compaction "{'class': 'SizeTieredCompactionStrategy'}" --host > 127.0.0.1 > {noformat} > Flush and delete all SSTables in node1 : > {noformat} > ccm node1 nodetool flush > ccm node1 stop > rm -f ~/.ccm/inc-repair-issue/node1/data0/tlp_stress/sensor*/*.* > ccm node1 start{noformat} > Then throttle streaming throughput to 1MB/s so we have time to take node1 > down during the streaming phase and run repair: > {noformat} > ccm node1 nodetool setstreamthroughput 1 > ccm node2 nodetool setstreamthroughput 1 > ccm node3 nodetool setstreamthroughput 1 > ccm node1 nodetool repair tlp_stress > {noformat} > Once streaming starts, shut down node1 and start it again : > {noformat} > ccm node1 stop > ccm node1 start > {noformat} > Run repair again : > {noformat} > ccm node1 nodetool repair tlp_stress > {noformat} > The command will return very quickly, showing that it skipped all sstables : > {noformat} > [2018-08-31 19:05:16,292] Repair completed successfully > [2018-08-31 19:05:16,292] Repair command #1 finished in 2 seconds > $ ccm node1 nodetool status > Datacenter: datacenter1 > === > Status=Up/Down > |/ State=Normal/Leaving/Joining/Moving > -- AddressLoad Tokens OwnsHost ID >Rack > UN 127.0.0.1 228,64 KiB 256 ? > 437dc9cd-b1a1-41a5-961e-cfc99763e29f rack1 > UN 127.0.0.2 60,09 MiB 256 ? > fbcbbdbb-e32a-4716-8230-8ca59aa93e62 rack1 > UN 127.0.0.3 57,59 MiB 256
[jira] [Created] (CASSANDRA-14759) Transient->Full movements mishandle consistency level upgrade
Benedict created CASSANDRA-14759: Summary: Transient->Full movements mishandle consistency level upgrade Key: CASSANDRA-14759 URL: https://issues.apache.org/jira/browse/CASSANDRA-14759 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Fix For: 4.0 While we need treat a transitioning node as ‘full’ for writes, so that it can safely begin serving full data requests once it has finished, we cannot maintain it in the ‘pending’ collection else we will also increase our consistency requirements by a node that doesn’t exist. -- 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
[jira] [Updated] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-14758: Resolution: Fixed Fix Version/s: 4.0 Status: Resolved (was: Patch Available) committed, thanks! > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Minor > Fix For: 4.0 > > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, it makes it kind of hard to work with files in the > {{org.apache.cassandra.audit}} package -- 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
cassandra git commit: Remove audit from .gitignore
Repository: cassandra Updated Branches: refs/heads/trunk 7a34477a9 -> 27c937477 Remove audit from .gitignore Patch by marcuse; reviewed by Jason Brown for CASSANDRA-14758 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/27c93747 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/27c93747 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/27c93747 Branch: refs/heads/trunk Commit: 27c9374773c4d28cd5c79f78ad0770cd11933818 Parents: 7a34477 Author: Marcus Eriksson Authored: Tue Sep 18 14:53:45 2018 +0200 Committer: Marcus Eriksson Committed: Tue Sep 18 15:23:26 2018 +0200 -- .gitignore | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/27c93747/.gitignore -- diff --git a/.gitignore b/.gitignore index 71b3087..d08471c 100644 --- a/.gitignore +++ b/.gitignore @@ -77,4 +77,3 @@ lib/jsr223/scala/*.jar doc/source/configuration/cassandra_config_file.rst doc/source/tools/nodetool -audit - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-14758: Description: Seems there was a "audit" entry added to the .gitignore file in CASSANDRA-9608, it makes it kind of hard to work with files in the {{org.apache.cassandra.audit}} package (was: Seems there was a "audit" entry added to the .gitignore file in CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files in the {{org.apache.cassandra.audit}} package) > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Minor > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, it makes it kind of hard to work with files in the > {{org.apache.cassandra.audit}} package -- 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
[jira] [Updated] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-14758: Reviewer: Jason Brown (was: Robert Stupp) > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Minor > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files > in the {{org.apache.cassandra.audit}} package -- 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
[jira] [Commented] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619075#comment-16619075 ] Jason Brown commented on CASSANDRA-14758: - +1 > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Minor > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files > in the {{org.apache.cassandra.audit}} package -- 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
[jira] [Updated] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-14758: Priority: Minor (was: Major) > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Minor > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files > in the {{org.apache.cassandra.audit}} package -- 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
[jira] [Updated] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-14758: Issue Type: Bug (was: Test) > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Major > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files > in the {{org.apache.cassandra.audit}} package -- 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
[jira] [Updated] (CASSANDRA-14758) Remove "audit" entry from .gitignore
[ https://issues.apache.org/jira/browse/CASSANDRA-14758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-14758: Assignee: Marcus Eriksson Reviewer: Robert Stupp Status: Patch Available (was: Open) https://github.com/krummas/cassandra/commits/marcuse/14758 > Remove "audit" entry from .gitignore > > > Key: CASSANDRA-14758 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 > Project: Cassandra > Issue Type: Test >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Major > > Seems there was a "audit" entry added to the .gitignore file in > CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files > in the {{org.apache.cassandra.audit}} package -- 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
[jira] [Created] (CASSANDRA-14758) Remove "audit" entry from .gitignore
Marcus Eriksson created CASSANDRA-14758: --- Summary: Remove "audit" entry from .gitignore Key: CASSANDRA-14758 URL: https://issues.apache.org/jira/browse/CASSANDRA-14758 Project: Cassandra Issue Type: Test Reporter: Marcus Eriksson Seems there was a "audit" entry added to the .gitignore file in CASSANDRA-9608, not sure why, but it makes it kind of hard to work with files in the {{org.apache.cassandra.audit}} package -- 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
[jira] [Updated] (CASSANDRA-14757) GCInspector "Error accessing field of java.nio.Bits" under java11
[ https://issues.apache.org/jira/browse/CASSANDRA-14757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Brown updated CASSANDRA-14757: Description: Running under java11, {{GCInspector}} throws the following exception: {noformat} DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing field of java.nio.Bits java.lang.NoSuchFieldException: totalCapacity at java.base/java.lang.Class.getDeclaredField(Class.java:2412) at org.apache.cassandra.service.GCInspector.(GCInspector.java:72) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) {noformat} This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} somewhere between java8 and java11. Note: this is a rather harmless error, as we only look at {{Bits.totalCapacity}} for metrics collection on how much direct memory is being used by {{ByteBuffer}}s. If we fail to read the field, we simply return -1 for the metric value. was: Running under java11, {{GCInspector}} throws the following exception: {noformat} DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing field of java.nio.Bits java.lang.NoSuchFieldException: totalCapacity at java.base/java.lang.Class.getDeclaredField(Class.java:2412) at org.apache.cassandra.service.GCInspector.(GCInspector.java:72) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) {noformat} This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} somewhere between java8 and java11. > GCInspector "Error accessing field of java.nio.Bits" under java11 > - > > Key: CASSANDRA-14757 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14757 > Project: Cassandra > Issue Type: Bug > Components: Metrics >Reporter: Jason Brown >Priority: Trivial > Labels: Java11 > Fix For: 4.0 > > > Running under java11, {{GCInspector}} throws the following exception: > {noformat} > DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing > field of java.nio.Bits > java.lang.NoSuchFieldException: totalCapacity > at java.base/java.lang.Class.getDeclaredField(Class.java:2412) > at > org.apache.cassandra.service.GCInspector.(GCInspector.java:72) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) > {noformat} > This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} > from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} > somewhere between java8 and java11. > Note: this is a rather harmless error, as we only look at > {{Bits.totalCapacity}} for metrics collection on how much direct memory is > being used by {{ByteBuffer}}s. If we fail to read the field, we simply return > -1 for the metric value. -- 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
[jira] [Created] (CASSANDRA-14757) GCInspector "Error accessing field of java.nio.Bits" under java11
Jason Brown created CASSANDRA-14757: --- Summary: GCInspector "Error accessing field of java.nio.Bits" under java11 Key: CASSANDRA-14757 URL: https://issues.apache.org/jira/browse/CASSANDRA-14757 Project: Cassandra Issue Type: Bug Components: Metrics Reporter: Jason Brown Fix For: 4.0 Running under java11, {{GCInspector}} throws the following exception: {noformat} DEBUG [main] 2018-09-18 05:18:25,905 GCInspector.java:78 - Error accessing field of java.nio.Bits java.lang.NoSuchFieldException: totalCapacity at java.base/java.lang.Class.getDeclaredField(Class.java:2412) at org.apache.cassandra.service.GCInspector.(GCInspector.java:72) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:308) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:590) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) {noformat} This is because {{GCInspector}} uses reflection to read the {{totalCapacity}} from {{java.nio.Bits}}. This field was renamed to {{TOTAL_CAPACITY}} somewhere between java8 and java11. -- 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
[jira] [Commented] (CASSANDRA-14672) After deleting data in 3.11.3, reads fail with "open marker and close marker have different deletion times"
[ https://issues.apache.org/jira/browse/CASSANDRA-14672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619051#comment-16619051 ] Aleksey Yeschenko commented on CASSANDRA-14672: --- [~mitsis] JIRA no longer displays email addresses (presumably due to GDPR). If you could maybe share those sstables alongside schema with me (privately and confidentially) please contact me at alek...@apache.org. Having the sstables will go a long way with helping to reproduce locally and determine what the problem is. > After deleting data in 3.11.3, reads fail with "open marker and close marker > have different deletion times" > --- > > Key: CASSANDRA-14672 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14672 > Project: Cassandra > Issue Type: Bug > Environment: CentOS 7, GCE, 9 nodes, 4TB disk/~2TB full each, level > compaction, timeseries data >Reporter: Spiros Ioannou >Priority: Blocker > > We had 3.11.0, then we upgraded to 3.11.3 last week. We routinely perform > deletions as the one described below. After upgrading we run the following > deletion query: > > {code:java} > DELETE FROM measurement_events_dbl WHERE measurement_source_id IN ( > 9df798a2-6337-11e8-b52b-42010afa015a, 9df7717e-6337-11e8-b52b-42010afa015a, > a08b8042-6337-11e8-b52b-42010afa015a, a08e52cc-6337-11e8-b52b-42010afa015a, > a08e6654-6337-11e8-b52b-42010afa015a, a08e6104-6337-11e8-b52b-42010afa015a, > a08e6c76-6337-11e8-b52b-42010afa015a, a08e5a9c-6337-11e8-b52b-42010afa015a, > a08bcc50-6337-11e8-b52b-42010afa015a) AND year IN (2018) AND measurement_time > >= '2018-07-19 04:00:00'{code} > > Immediately after that, trying to read the last value produces an error: > {code:java} > select * FROM measurement_events_dbl WHERE measurement_source_id = > a08b8042-6337-11e8-b52b-42010afa015a AND year IN (2018) order by > measurement_time desc limit 1; > ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] > message="Operation failed - received 0 responses and 2 failures" > info={'failures': 2, 'received_responses': 0, 'required_responses': 1, > 'consistency': 'ONE'}{code} > > And the following exception: > {noformat} > WARN [ReadStage-4] 2018-08-29 06:59:53,505 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-4,5,main]: {} > java.lang.RuntimeException: java.lang.IllegalStateException: > UnfilteredRowIterator for pvpms_mevents.measurement_events_dbl has an illegal > RT bounds sequence: open marker and close marker have different deletion times > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2601) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_181] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134) > [apache-cassandra-3.11.3.jar:3.11.3] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [apache-cassandra-3.11.3.jar:3.11.3] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_181] > Caused by: java.lang.IllegalStateException: UnfilteredRowIterator for > pvpms_mevents.measurement_events_dbl has an illegal RT bounds sequence: open > marker and close marker have different deletion times > at > org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.ise(RTBoundValidator.java:103) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.applyToMarker(RTBoundValidator.java:81) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:148) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:136) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:92) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:79) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:308) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187) >
[jira] [Commented] (CASSANDRA-14672) After deleting data in 3.11.3, reads fail with "open marker and close marker have different deletion times"
[ https://issues.apache.org/jira/browse/CASSANDRA-14672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619035#comment-16619035 ] Aleksey Yeschenko commented on CASSANDRA-14672: --- Thanks. At first glance, individually these dumps seem normal. So this could be a bug in reading/merging, or even a bug in RT validation logic. I'll do my best to investigate further this week. > After deleting data in 3.11.3, reads fail with "open marker and close marker > have different deletion times" > --- > > Key: CASSANDRA-14672 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14672 > Project: Cassandra > Issue Type: Bug > Environment: CentOS 7, GCE, 9 nodes, 4TB disk/~2TB full each, level > compaction, timeseries data >Reporter: Spiros Ioannou >Priority: Blocker > > We had 3.11.0, then we upgraded to 3.11.3 last week. We routinely perform > deletions as the one described below. After upgrading we run the following > deletion query: > > {code:java} > DELETE FROM measurement_events_dbl WHERE measurement_source_id IN ( > 9df798a2-6337-11e8-b52b-42010afa015a, 9df7717e-6337-11e8-b52b-42010afa015a, > a08b8042-6337-11e8-b52b-42010afa015a, a08e52cc-6337-11e8-b52b-42010afa015a, > a08e6654-6337-11e8-b52b-42010afa015a, a08e6104-6337-11e8-b52b-42010afa015a, > a08e6c76-6337-11e8-b52b-42010afa015a, a08e5a9c-6337-11e8-b52b-42010afa015a, > a08bcc50-6337-11e8-b52b-42010afa015a) AND year IN (2018) AND measurement_time > >= '2018-07-19 04:00:00'{code} > > Immediately after that, trying to read the last value produces an error: > {code:java} > select * FROM measurement_events_dbl WHERE measurement_source_id = > a08b8042-6337-11e8-b52b-42010afa015a AND year IN (2018) order by > measurement_time desc limit 1; > ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] > message="Operation failed - received 0 responses and 2 failures" > info={'failures': 2, 'received_responses': 0, 'required_responses': 1, > 'consistency': 'ONE'}{code} > > And the following exception: > {noformat} > WARN [ReadStage-4] 2018-08-29 06:59:53,505 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-4,5,main]: {} > java.lang.RuntimeException: java.lang.IllegalStateException: > UnfilteredRowIterator for pvpms_mevents.measurement_events_dbl has an illegal > RT bounds sequence: open marker and close marker have different deletion times > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2601) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_181] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134) > [apache-cassandra-3.11.3.jar:3.11.3] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [apache-cassandra-3.11.3.jar:3.11.3] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_181] > Caused by: java.lang.IllegalStateException: UnfilteredRowIterator for > pvpms_mevents.measurement_events_dbl has an illegal RT bounds sequence: open > marker and close marker have different deletion times > at > org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.ise(RTBoundValidator.java:103) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.applyToMarker(RTBoundValidator.java:81) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:148) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:136) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:92) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:79) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:308) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:180) >
[jira] [Commented] (CASSANDRA-14672) After deleting data in 3.11.3, reads fail with "open marker and close marker have different deletion times"
[ https://issues.apache.org/jira/browse/CASSANDRA-14672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16619001#comment-16619001 ] Nikos Mitsis commented on CASSANDRA-14672: -- [~iamaleksey] The dump is from the "bad" node 9 and truncated heavily. I'm presenting the dumps that contain the specific partition key below. These dumps are after DELETE (see in ticket the full command) was run on Sep 16 on a replica cluster created with snapshots from Aug 29. >From node 9 there are 3 SSTables with that partition key: * SSTable 1 below (I've truncated a lot of rows): {noformat} mc-2228-big-Data.json { "partition" : { "key" : [ "a08b8042-6337-11e8-b52b-42010afa015a", "2018" ], "position" : 83756793 }, "rows" : [ { "type" : "row", "position" : 83756879, "clustering" : [ "2018-06-03 18:45:00.000Z" ], "liveness_info" : { "tstamp" : "2018-06-03T19:48:27.570903Z" }, "cells" : [ { "name" : "event_reception_time", "value" : "2018-06-03 19:48:27.570Z" }, { "name" : "quality", "value" : 100.0 }, { "name" : "value", "value" : 185532.77 } ] }, * * many more rows * { "type" : "row", "position" : 83860241, "clustering" : [ "2018-06-27 04:45:00.000Z" ], "liveness_info" : { "tstamp" : "2018-06-27T04:50:08.969007Z" }, "cells" : [ { "name" : "event_reception_time", "value" : "2018-06-27 04:50:08.968Z" }, { "name" : "quality", "value" : 100.0 }, { "name" : "value", "value" : 408307.66 } ] }, { "type" : "range_tombstone_bound", "start" : { "type" : "exclusive", "clustering" : [ "2018-06-27 04:55:00.000Z" ], "deletion_info" : { "marked_deleted" : "2018-06-28T17:03:08.555918Z", "local_delete_time" : "2018-06-28T17:03:08Z" } } }, { "type" : "row", "position" : 83860313, "clustering" : [ "2018-06-27 05:00:00.000Z" ], "liveness_info" : { "tstamp" : "2018-06-28T19:45:30.803293Z" }, "cells" : [ { "name" : "event_reception_time", "value" : "2018-06-28 19:45:30.784Z" }, { "name" : "quality", "value" : 100.0 }, { "name" : "value", "value" : 408307.66 } ] }, * * many more rows * { "type" : "row", "position" : 84054985, "clustering" : [ "2018-08-11 04:00:00.000Z" ], "liveness_info" : { "tstamp" : "2018-08-11T04:01:15.708470Z" }, "cells" : [ { "name" : "event_reception_time", "value" : "2018-08-11 04:01:15.703Z" }, { "name" : "quality", "value" : 100.0 }, { "name" : "value", "value" : 372654.53 } ] }, { "type" : "range_tombstone_bound", "end" : { "type" : "inclusive", "deletion_info" : { "marked_deleted" : "2018-06-28T17:03:08.555918Z", "local_delete_time" : "2018-06-28T17:03:08Z" } } } ] } {noformat} * SSTable 2 (I've truncated NOTHING) {noformat} mc-35045-big-Data.json { "partition" : { "key" : [ "a08b8042-6337-11e8-b52b-42010afa015a", "2018" ], "position" : 19414 }, "rows" : [ { "type" : "range_tombstone_bound", "start" : { "type" : "inclusive", "clustering" : [ "2018-07-19 04:00:00.000Z" ], "deletion_info" : { "marked_deleted" : "2018-09-16T13:14:14.634113Z", "local_delete_time" : "2018-09-16T13:14:14Z" } } }, { "type" : "range_tombstone_bound", "end" : { "type" : "inclusive", "deletion_info" : { "marked_deleted" : "2018-09-16T13:14:14.634113Z", "local_delete_time" : "2018-09-16T13:14:14Z" } } } ] } {noformat} * SSTable 3 contains rows with dates *before "2018-06-03 18:45:00.000Z"* with no tombstone information so not presented here. >From node 6 (the "good" node) there are also 3 SSTables with that partition >key: * SSTable 1 below (I've truncated NOTHING): {noformat} mc-2763209-big-Data.json { "partition" : { "key" : [ "a08b8042-6337-11e8-b52b-42010afa015a", "2018" ], "position" : 101207516 }, "rows" : [ { "type" : "range_tombstone_bound", "start" : { "type" : "inclusive", "clustering" : [ "2018-07-19 04:00:00.000Z" ], "deletion_info" : { "marked_deleted" : "2018-09-16T08:37:18.429276Z", "local_delete_time" : "2018-09-16T08:37:18Z" } } }, { "type" : "range_tombstone_bound", "end" : { "type" : "inclusive", "deletion_info" : { "marked_deleted" : "2018-09-16T08:37:18.429276Z", "local_delete_time" : "2018-09-16T08:37:18Z" } } } ] } {noformat} * SSTable 2 below (I've truncated NOTHING): {noformat} mc-2763214-big-Data.json {
[jira] [Updated] (CASSANDRA-5953) Replication validation should reject RF > nodes in cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-5953?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated CASSANDRA-5953: -- Labels: pull-request-available (was: ) > Replication validation should reject RF > nodes in cluster > -- > > Key: CASSANDRA-5953 > URL: https://issues.apache.org/jira/browse/CASSANDRA-5953 > Project: Cassandra > Issue Type: Bug >Reporter: Jonathan Ellis >Priority: Minor > Labels: pull-request-available > > On my local, single node cluster, RF=3 inserts should not succeed: > {noformat} > cqlsh> CREATE KEYSPACE mykeyspace WITH REPLICATION = { 'class' : > 'SimpleStrategy', 'replication_factor' : 3 }; > cqlsh> use mykeyspace ; > cqlsh:mykeyspace> CREATE TABLE users ( > ... user_id int PRIMARY KEY, > ... fname text, > ... lname text > ... ); > cqlsh:mykeyspace> INSERT INTO users (user_id, fname, lname) > ... VALUES (1745, 'john', 'smith'); > cqlsh:mykeyspace> select * from users; > user_id | fname | lname > -+---+--- > 1745 | john | smith > {noformat} -- 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
[jira] [Commented] (CASSANDRA-14755) Reinstate repaired data tracking when ReadRepair == NONE
[ https://issues.apache.org/jira/browse/CASSANDRA-14755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618908#comment-16618908 ] Alex Petrov commented on CASSANDRA-14755: - [~beobal] sorry: I was looking closely at all the {{on*}} methods to ensure they do not have any effects, but overlooked the {{close}}. Thank you for fixing it. +1, LGTM! > Reinstate repaired data tracking when ReadRepair == NONE > > > Key: CASSANDRA-14755 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14755 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe >Priority: Major > Fix For: 4.0 > > > Some of the refactoring in CASSANDRA-14698 breaks repaired data tracking when > read repair is disabled as it skips wrapping the {{MergeIterator}} in > {{DataResolver::wrapMergeListener}}. If repaired tracking is enabled, the > iterator still needs to be extended so that it calls > {{RepairedDataTracker::verify}} on close. This wasn't easy to spot as the new > dtests for CASSANDRA-14145 haven't yet been merged. -- 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
[jira] [Updated] (CASSANDRA-14755) Reinstate repaired data tracking when ReadRepair == NONE
[ https://issues.apache.org/jira/browse/CASSANDRA-14755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Petrov updated CASSANDRA-14755: Reviewer: Alex Petrov > Reinstate repaired data tracking when ReadRepair == NONE > > > Key: CASSANDRA-14755 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14755 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe >Priority: Major > Fix For: 4.0 > > > Some of the refactoring in CASSANDRA-14698 breaks repaired data tracking when > read repair is disabled as it skips wrapping the {{MergeIterator}} in > {{DataResolver::wrapMergeListener}}. If repaired tracking is enabled, the > iterator still needs to be extended so that it calls > {{RepairedDataTracker::verify}} on close. This wasn't easy to spot as the new > dtests for CASSANDRA-14145 haven't yet been merged. -- 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
[jira] [Commented] (CASSANDRA-14672) After deleting data in 3.11.3, reads fail with "open marker and close marker have different deletion times"
[ https://issues.apache.org/jira/browse/CASSANDRA-14672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618804#comment-16618804 ] Aleksey Yeschenko commented on CASSANDRA-14672: --- [~jjirsa] I don't think we can say that this was CASSANDRA-14515 corruption specifically. [~mitsis] Where is this dump from? I'm interested in seeing both mismatched bounds in a dump from an affected node, to prove conclusively that there is corrupt data on disk - and I'm yet to see the half with deletion time in September. > After deleting data in 3.11.3, reads fail with "open marker and close marker > have different deletion times" > --- > > Key: CASSANDRA-14672 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14672 > Project: Cassandra > Issue Type: Bug > Environment: CentOS 7, GCE, 9 nodes, 4TB disk/~2TB full each, level > compaction, timeseries data >Reporter: Spiros Ioannou >Priority: Blocker > > We had 3.11.0, then we upgraded to 3.11.3 last week. We routinely perform > deletions as the one described below. After upgrading we run the following > deletion query: > > {code:java} > DELETE FROM measurement_events_dbl WHERE measurement_source_id IN ( > 9df798a2-6337-11e8-b52b-42010afa015a, 9df7717e-6337-11e8-b52b-42010afa015a, > a08b8042-6337-11e8-b52b-42010afa015a, a08e52cc-6337-11e8-b52b-42010afa015a, > a08e6654-6337-11e8-b52b-42010afa015a, a08e6104-6337-11e8-b52b-42010afa015a, > a08e6c76-6337-11e8-b52b-42010afa015a, a08e5a9c-6337-11e8-b52b-42010afa015a, > a08bcc50-6337-11e8-b52b-42010afa015a) AND year IN (2018) AND measurement_time > >= '2018-07-19 04:00:00'{code} > > Immediately after that, trying to read the last value produces an error: > {code:java} > select * FROM measurement_events_dbl WHERE measurement_source_id = > a08b8042-6337-11e8-b52b-42010afa015a AND year IN (2018) order by > measurement_time desc limit 1; > ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] > message="Operation failed - received 0 responses and 2 failures" > info={'failures': 2, 'received_responses': 0, 'required_responses': 1, > 'consistency': 'ONE'}{code} > > And the following exception: > {noformat} > WARN [ReadStage-4] 2018-08-29 06:59:53,505 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-4,5,main]: {} > java.lang.RuntimeException: java.lang.IllegalStateException: > UnfilteredRowIterator for pvpms_mevents.measurement_events_dbl has an illegal > RT bounds sequence: open marker and close marker have different deletion times > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2601) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_181] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134) > [apache-cassandra-3.11.3.jar:3.11.3] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [apache-cassandra-3.11.3.jar:3.11.3] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_181] > Caused by: java.lang.IllegalStateException: UnfilteredRowIterator for > pvpms_mevents.measurement_events_dbl has an illegal RT bounds sequence: open > marker and close marker have different deletion times > at > org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.ise(RTBoundValidator.java:103) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.applyToMarker(RTBoundValidator.java:81) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:148) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:136) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:92) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:79) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:308) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187) >
[jira] [Commented] (CASSANDRA-14145) Detecting data resurrection during read
[ https://issues.apache.org/jira/browse/CASSANDRA-14145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618668#comment-16618668 ] ASF GitHub Bot commented on CASSANDRA-14145: Github user beobal commented on a diff in the pull request: https://github.com/apache/cassandra-dtest/pull/37#discussion_r218336047 --- Diff: repair_tests/incremental_repair_test.py --- @@ -918,3 +931,196 @@ def test_subrange(self): self.assertRepairedAndUnrepaired(node1, 'ks') self.assertRepairedAndUnrepaired(node2, 'ks') self.assertRepairedAndUnrepaired(node3, 'ks') + +@since('4.0') +def test_repaired_tracking_with_partition_deletes(self): +""" +check that when an tracking repaired data status following a digest mismatch, +repaired data mismatches are marked as unconfirmed as we may skip sstables +after the partition delete are encountered. +@jira_ticket CASSANDRA-14145 +""" +session, node1, node2 = self.setup_for_repaired_data_tracking() +stmt = SimpleStatement("INSERT INTO ks.tbl (k, c, v) VALUES (%s, %s, %s)") +stmt.consistency_level = ConsistencyLevel.ALL +for i in range(10): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() +self.assertNoRepairedSSTables(node, 'ks') + +node1.repair(options=['ks']) +node2.stop(wait_other_notice=True) + +session.execute("delete from ks.tbl where k = 5") + +node1.flush() +node2.start(wait_other_notice=True) + +# expect unconfirmed inconsistencies as the partition deletes cause some sstables to be skipped +with JolokiaAgent(node1) as jmx: +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5", + expect_unconfirmed_inconsistencies=True) +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5 AND c = 5", + expect_unconfirmed_inconsistencies=True) +# no digest reads for range queries so blocking read repair metric isn't incremented +# *all* sstables are read for partition ranges too, and as the repaired set is still in sync there should +# be no inconsistencies +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl", expect_read_repair=False) + +@since('4.0') +def test_repaired_tracking_with_varying_sstable_sets(self): +""" +verify that repaired data digests are computed over the merged data for each replica +and that the particular number of sstables on each doesn't affect the comparisons +both replicas start with the same repaired set, comprising 2 sstables. node1's is +then compacted and additional unrepaired data added (which overwrites some in the +repaired set). We expect the repaired digests to still match as the tracking will +force all sstables containing the partitions to be read +there are two variants of this, for single partition slice & names reads and range reads +@jira_ticket CASSANDRA-14145 +""" +session, node1, node2 = self.setup_for_repaired_data_tracking() +stmt = SimpleStatement("INSERT INTO ks.tbl (k, c, v) VALUES (%s, %s, %s)") +stmt.consistency_level = ConsistencyLevel.ALL +for i in range(10): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() + +for i in range(10,20): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() +self.assertNoRepairedSSTables(node, 'ks') + +node1.repair(options=['ks']) +node2.stop(wait_other_notice=True) + +session.execute("insert into ks.tbl (k, c, v) values (5, 5, 55)") +session.execute("insert into ks.tbl (k, c, v) values (15, 15, 155)") +node1.flush() +node1.compact() +node1.compact() +node2.start(wait_other_notice=True) + +# we don't expect any inconsistencies as all repaired data is read on both replicas +with JolokiaAgent(node1) as jmx: +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5") +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5 AND
[jira] [Commented] (CASSANDRA-14145) Detecting data resurrection during read
[ https://issues.apache.org/jira/browse/CASSANDRA-14145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618669#comment-16618669 ] ASF GitHub Bot commented on CASSANDRA-14145: Github user beobal commented on a diff in the pull request: https://github.com/apache/cassandra-dtest/pull/37#discussion_r218336055 --- Diff: repair_tests/incremental_repair_test.py --- @@ -918,3 +931,196 @@ def test_subrange(self): self.assertRepairedAndUnrepaired(node1, 'ks') self.assertRepairedAndUnrepaired(node2, 'ks') self.assertRepairedAndUnrepaired(node3, 'ks') + +@since('4.0') +def test_repaired_tracking_with_partition_deletes(self): +""" +check that when an tracking repaired data status following a digest mismatch, +repaired data mismatches are marked as unconfirmed as we may skip sstables +after the partition delete are encountered. +@jira_ticket CASSANDRA-14145 +""" +session, node1, node2 = self.setup_for_repaired_data_tracking() +stmt = SimpleStatement("INSERT INTO ks.tbl (k, c, v) VALUES (%s, %s, %s)") +stmt.consistency_level = ConsistencyLevel.ALL +for i in range(10): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() +self.assertNoRepairedSSTables(node, 'ks') + +node1.repair(options=['ks']) +node2.stop(wait_other_notice=True) + +session.execute("delete from ks.tbl where k = 5") + +node1.flush() +node2.start(wait_other_notice=True) + +# expect unconfirmed inconsistencies as the partition deletes cause some sstables to be skipped +with JolokiaAgent(node1) as jmx: +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5", + expect_unconfirmed_inconsistencies=True) +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5 AND c = 5", + expect_unconfirmed_inconsistencies=True) +# no digest reads for range queries so blocking read repair metric isn't incremented +# *all* sstables are read for partition ranges too, and as the repaired set is still in sync there should +# be no inconsistencies +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl", expect_read_repair=False) + +@since('4.0') +def test_repaired_tracking_with_varying_sstable_sets(self): +""" +verify that repaired data digests are computed over the merged data for each replica +and that the particular number of sstables on each doesn't affect the comparisons +both replicas start with the same repaired set, comprising 2 sstables. node1's is +then compacted and additional unrepaired data added (which overwrites some in the +repaired set). We expect the repaired digests to still match as the tracking will +force all sstables containing the partitions to be read +there are two variants of this, for single partition slice & names reads and range reads +@jira_ticket CASSANDRA-14145 +""" +session, node1, node2 = self.setup_for_repaired_data_tracking() +stmt = SimpleStatement("INSERT INTO ks.tbl (k, c, v) VALUES (%s, %s, %s)") +stmt.consistency_level = ConsistencyLevel.ALL +for i in range(10): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() + +for i in range(10,20): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() +self.assertNoRepairedSSTables(node, 'ks') + +node1.repair(options=['ks']) +node2.stop(wait_other_notice=True) + +session.execute("insert into ks.tbl (k, c, v) values (5, 5, 55)") +session.execute("insert into ks.tbl (k, c, v) values (15, 15, 155)") +node1.flush() +node1.compact() +node1.compact() +node2.start(wait_other_notice=True) + +# we don't expect any inconsistencies as all repaired data is read on both replicas +with JolokiaAgent(node1) as jmx: +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5") +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5 AND
[jira] [Commented] (CASSANDRA-14145) Detecting data resurrection during read
[ https://issues.apache.org/jira/browse/CASSANDRA-14145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618670#comment-16618670 ] ASF GitHub Bot commented on CASSANDRA-14145: Github user beobal commented on a diff in the pull request: https://github.com/apache/cassandra-dtest/pull/37#discussion_r218336062 --- Diff: repair_tests/incremental_repair_test.py --- @@ -918,3 +931,196 @@ def test_subrange(self): self.assertRepairedAndUnrepaired(node1, 'ks') self.assertRepairedAndUnrepaired(node2, 'ks') self.assertRepairedAndUnrepaired(node3, 'ks') + +@since('4.0') +def test_repaired_tracking_with_partition_deletes(self): +""" +check that when an tracking repaired data status following a digest mismatch, +repaired data mismatches are marked as unconfirmed as we may skip sstables +after the partition delete are encountered. +@jira_ticket CASSANDRA-14145 +""" +session, node1, node2 = self.setup_for_repaired_data_tracking() +stmt = SimpleStatement("INSERT INTO ks.tbl (k, c, v) VALUES (%s, %s, %s)") +stmt.consistency_level = ConsistencyLevel.ALL +for i in range(10): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() +self.assertNoRepairedSSTables(node, 'ks') + +node1.repair(options=['ks']) +node2.stop(wait_other_notice=True) + +session.execute("delete from ks.tbl where k = 5") + +node1.flush() +node2.start(wait_other_notice=True) + +# expect unconfirmed inconsistencies as the partition deletes cause some sstables to be skipped +with JolokiaAgent(node1) as jmx: +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5", + expect_unconfirmed_inconsistencies=True) +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5 AND c = 5", + expect_unconfirmed_inconsistencies=True) +# no digest reads for range queries so blocking read repair metric isn't incremented +# *all* sstables are read for partition ranges too, and as the repaired set is still in sync there should +# be no inconsistencies +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl", expect_read_repair=False) + +@since('4.0') +def test_repaired_tracking_with_varying_sstable_sets(self): +""" +verify that repaired data digests are computed over the merged data for each replica +and that the particular number of sstables on each doesn't affect the comparisons +both replicas start with the same repaired set, comprising 2 sstables. node1's is +then compacted and additional unrepaired data added (which overwrites some in the +repaired set). We expect the repaired digests to still match as the tracking will +force all sstables containing the partitions to be read +there are two variants of this, for single partition slice & names reads and range reads +@jira_ticket CASSANDRA-14145 +""" +session, node1, node2 = self.setup_for_repaired_data_tracking() +stmt = SimpleStatement("INSERT INTO ks.tbl (k, c, v) VALUES (%s, %s, %s)") +stmt.consistency_level = ConsistencyLevel.ALL +for i in range(10): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() + +for i in range(10,20): +session.execute(stmt, (i, i, i)) + +for node in self.cluster.nodelist(): +node.flush() +self.assertNoRepairedSSTables(node, 'ks') + +node1.repair(options=['ks']) +node2.stop(wait_other_notice=True) + +session.execute("insert into ks.tbl (k, c, v) values (5, 5, 55)") +session.execute("insert into ks.tbl (k, c, v) values (15, 15, 155)") +node1.flush() +node1.compact() +node1.compact() +node2.start(wait_other_notice=True) + +# we don't expect any inconsistencies as all repaired data is read on both replicas +with JolokiaAgent(node1) as jmx: +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5") +self.query_and_check_repaired_mismatches(jmx, session, "SELECT * FROM ks.tbl WHERE k = 5 AND
[jira] [Commented] (CASSANDRA-14145) Detecting data resurrection during read
[ https://issues.apache.org/jira/browse/CASSANDRA-14145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618671#comment-16618671 ] ASF GitHub Bot commented on CASSANDRA-14145: Github user beobal commented on a diff in the pull request: https://github.com/apache/cassandra-dtest/pull/37#discussion_r218336071 --- Diff: repair_tests/incremental_repair_test.py --- @@ -207,6 +208,7 @@ def test_manual_session_fail(self): self.fixture_dtest_setup.setup_overrides.cluster_options = ImmutableMapping({'hinted_handoff_enabled': 'false', 'num_tokens': 1, 'commitlog_sync_period_in_ms': 500}) +self.fixture_dtest_setup.init_default_config() --- End diff -- > I believe `self.init_default_config()` and `self.fixture_dtest_setup.init_default_config()` are synonymous Hmm yeah, so it is. I find that a little non-obvious and easy to miss, but ¯\_(ツ)_/¯ > Is it because the config changes in the preceding lines weren't actually being picked up prior? Exactly, the fixture initialization order means that the cluster config has already been set by this point and so none of these options were being used in the existing tests. > Detecting data resurrection during read > > > Key: CASSANDRA-14145 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14145 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Sam Tunnicliffe >Priority: Minor > Labels: pull-request-available > Fix For: 4.0 > > > We have seen several bugs in which deleted data gets resurrected. We should > try to see if we can detect this on the read path and possibly fix it. Here > are a few examples which brought back data > A replica lost an sstable on startup which caused one replica to lose the > tombstone and not the data. This tombstone was past gc grace which means this > could resurrect data. We can detect such invalid states by looking at other > replicas. > If we are running incremental repair, Cassandra will keep repaired and > non-repaired data separate. Every-time incremental repair will run, it will > move the data from non-repaired to repaired. Repaired data across all > replicas should be 100% consistent. > Here is an example of how we can detect and mitigate the issue in most cases. > Say we have 3 machines, A,B and C. All these machines will have data split > b/w repaired and non-repaired. > 1. Machine A due to some bug bring backs data D. This data D is in repaired > dataset. All other replicas will have data D and tombstone T > 2. Read for data D comes from application which involve replicas A and B. The > data being read involves data which is in repaired state. A will respond > back to co-ordinator with data D and B will send nothing as tombstone is past > gc grace. This will cause digest mismatch. > 3. This patch will only kick in when there is a digest mismatch. Co-ordinator > will ask both replicas to send back all data like we do today but with this > patch, replicas will respond back what data it is returning is coming from > repaired vs non-repaired. If data coming from repaired does not match, we > know there is a something wrong!! At this time, co-ordinator cannot determine > if replica A has resurrected some data or replica B has lost some data. We > can still log error in the logs saying we hit an invalid state. > 4. Besides the log, we can take this further and even correct the response to > the query. After logging an invalid state, we can ask replica A and B (and > also C if alive) to send back all data for this including gcable tombstones. > If any machine returns a tombstone which is after this data, we know we > cannot return this data. This way we can avoid returning data which has been > deleted. > Some Challenges with this > 1. When data will be moved from non-repaired to repaired, there could be a > race here. We can look at which incremental repairs have promoted things on > which replica to avoid false positives. > 2. If the third replica is down and live replica does not have any tombstone, > we wont be able to break the tie in deciding whether data was actually > deleted or resurrected. > 3. If the read is for latest data only, we wont be able to detect it as the > read will be served from non-repaired data. > 4. If the replica where we lose a tombstone is the last replica to compact > the tombstone, we wont be able to decide if data is coming back or rest of > the replicas has lost that data. But we will still detect something is wrong. > 5. We wont
[jira] [Commented] (CASSANDRA-14145) Detecting data resurrection during read
[ https://issues.apache.org/jira/browse/CASSANDRA-14145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16618672#comment-16618672 ] ASF GitHub Bot commented on CASSANDRA-14145: Github user beobal commented on the issue: https://github.com/apache/cassandra-dtest/pull/37 Thanks @jrwest, pushed a followup commit and replied to most comments inline. > Two of the new tests are failing for me. Yep, some of the refactoring in CASSANDRA-14698 broke repaired tracking where the read repair strategy for the table is `NONE`, which is the case in these tests. CASSANDRA-14755 has a fix for this and is patch available. > Is there a build with this branch showing how the dtests fare w/ tracking enabled for all of them? The dtest runs on CASSANDRA-14145 were all done with this branch, so they have tracking enabled (also the CI runs linked from CASSANDRA-14755). > Detecting data resurrection during read > > > Key: CASSANDRA-14145 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14145 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Sam Tunnicliffe >Priority: Minor > Labels: pull-request-available > Fix For: 4.0 > > > We have seen several bugs in which deleted data gets resurrected. We should > try to see if we can detect this on the read path and possibly fix it. Here > are a few examples which brought back data > A replica lost an sstable on startup which caused one replica to lose the > tombstone and not the data. This tombstone was past gc grace which means this > could resurrect data. We can detect such invalid states by looking at other > replicas. > If we are running incremental repair, Cassandra will keep repaired and > non-repaired data separate. Every-time incremental repair will run, it will > move the data from non-repaired to repaired. Repaired data across all > replicas should be 100% consistent. > Here is an example of how we can detect and mitigate the issue in most cases. > Say we have 3 machines, A,B and C. All these machines will have data split > b/w repaired and non-repaired. > 1. Machine A due to some bug bring backs data D. This data D is in repaired > dataset. All other replicas will have data D and tombstone T > 2. Read for data D comes from application which involve replicas A and B. The > data being read involves data which is in repaired state. A will respond > back to co-ordinator with data D and B will send nothing as tombstone is past > gc grace. This will cause digest mismatch. > 3. This patch will only kick in when there is a digest mismatch. Co-ordinator > will ask both replicas to send back all data like we do today but with this > patch, replicas will respond back what data it is returning is coming from > repaired vs non-repaired. If data coming from repaired does not match, we > know there is a something wrong!! At this time, co-ordinator cannot determine > if replica A has resurrected some data or replica B has lost some data. We > can still log error in the logs saying we hit an invalid state. > 4. Besides the log, we can take this further and even correct the response to > the query. After logging an invalid state, we can ask replica A and B (and > also C if alive) to send back all data for this including gcable tombstones. > If any machine returns a tombstone which is after this data, we know we > cannot return this data. This way we can avoid returning data which has been > deleted. > Some Challenges with this > 1. When data will be moved from non-repaired to repaired, there could be a > race here. We can look at which incremental repairs have promoted things on > which replica to avoid false positives. > 2. If the third replica is down and live replica does not have any tombstone, > we wont be able to break the tie in deciding whether data was actually > deleted or resurrected. > 3. If the read is for latest data only, we wont be able to detect it as the > read will be served from non-repaired data. > 4. If the replica where we lose a tombstone is the last replica to compact > the tombstone, we wont be able to decide if data is coming back or rest of > the replicas has lost that data. But we will still detect something is wrong. > 5. We wont affect 99.9% of the read queries as we only do extra work during > digest mismatch. > 6. CL.ONE reads will not be able to detect this. -- 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