[jira] [Comment Edited] (CASSANDRA-10540) RangeAwareCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445374#comment-16445374 ] Lerh Chuan Low edited comment on CASSANDRA-10540 at 4/20/18 6:38 AM: - Hey Marcus, thanks for getting back so quickly :) The big motivation for us is repairs...because when vnodes are turned on, every SSTable has many vnodes in them...so when a (incremental) repair happens, the range it is interested in gets anticompacted out. After that the next range gets anticompacted out and so on. RACS solves that big pain. Besides making the SSTables per read much nicer, it's like a LCS on steroids. I think there are other benefits of maintaining each SSTable to one token range...but I can't quite remember any more off the top of my head. So I am hoping it doesn't come to grouping the vnodesunless it's a last resort. Currently it looks like you create a RACS for each of the repaired/unrepaired/pending repairs set, and each RACS keeps track of the compaction strategies it is in charge of (which are all of the same class). The CS instances are lazily initiated (so that's a win right there) until needed. It seems to be that the reason why we want so many CS instances is so that each of them can keep track of their own SSTables (which all belong to that single token range). How about if RACS doesn't instantiate the individual CS instances? It keeps track of all the SSTables in the CF like other CS instances - just that the logic on which SSTables to involve in a compaction is done in RACS. So we can make RACS check L0 and if there are none, L1 would involve grouping the SSTables by range and then calling the next background task for the underlying/wrapped CS instances and submitting them. In this way, the downside is calculating the grouping each time we ask for the next background task. We could also store it in memory in the form of a manifest like in LCS? So an array with the SSTables in each of them - beats having 256 instances but we're still going to have a 256 sized array in memory, I guess. It just seems so starkingly similar to a LCS restricted to just L0 and L1. A final thought: Is the memory footprint actually significant enough for us to want to not bite the bullet and further group the vnodes because the gains from having each SSTable as a single range is a lot, simple is a feature and RACS is customizable? Please excuse my ignorance if none of those suggestions made sense/worked, still not very confident with the code base.. (Btw also feel free to let me know if you would like a hand with anything) was (Author: lerh low): Hey Marcus, thanks for getting back so quickly :) The big motivation for us is repairs...because when vnodes are turned on, every SSTable has many vnodes in them...so when a (incremental) repair happens, the range it is interested in gets anticompacted out. After that the next range gets anticompacted out and so on. RACS solves that big pain. Besides making the SSTables per read much nicer, it's like a LCS on steroids. I think there are other benefits of maintaining each SSTable to one token range...but I can't quite remember any more off the top of my head. So I am hoping it doesn't come to grouping the vnodesunless it's a last resort. Currently it looks like you create a RACS for each of the repaired/unrepaired/pending repairs set, and each RACS keeps track of the compaction strategies it is in charge of (which are all of the same class). The CS instances are lazily initiated (so that's a win right there) until needed. It seems to be that the reason why we want so many CS instances is so that each of them can keep track of their own SSTables (which all belong to that single token range). How about if RACS doesn't instantiate the individual CS instances? It keeps track of all the SSTables in the CF like other CS instances - just that the logic on which SSTables to involve in a compaction is done in RACS. So we can make RACS check L0 and if there are none, L1 would involve grouping the SSTables by range and then calling the next background task for the underlying/wrapped CS instances and submitting them. In this way, the downside is calculating the grouping each time we ask for the next background task. We could also store it in memory in the form of a manifest like in LCS? So an array with the SSTables in each of them - beats having 256 instances but we're still going to have a 256 sized array in memory, I guess. It just seems so starkingly similar to a LCS restricted to just L0 and L1. A final thought: Is the memory footprint actually significant enough for us to want to not bite the bullet and further group the vnodes because the gains from having each SSTable as a single range is a lot, simple is a feature and RACS is customizable? Please excuse my
[jira] [Commented] (CASSANDRA-10540) RangeAwareCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445374#comment-16445374 ] Lerh Chuan Low commented on CASSANDRA-10540: Hey Marcus, thanks for getting back so quickly :) The big motivation for us is repairs...because when vnodes are turned on, every SSTable has many vnodes in them...so when a (incremental) repair happens, the range it is interested in gets anticompacted out. After that the next range gets anticompacted out and so on. RACS solves that big pain. Besides making the SSTables per read much nicer, it's like a LCS on steroids. I think there are other benefits of maintaining each SSTable to one token range...but I can't quite remember any more off the top of my head. So I am hoping it doesn't come to grouping the vnodesunless it's a last resort. Currently it looks like you create a RACS for each of the repaired/unrepaired/pending repairs set, and each RACS keeps track of the compaction strategies it is in charge of (which are all of the same class). The CS instances are lazily initiated (so that's a win right there) until needed. It seems to be that the reason why we want so many CS instances is so that each of them can keep track of their own SSTables (which all belong to that single token range). How about if RACS doesn't instantiate the individual CS instances? It keeps track of all the SSTables in the CF like other CS instances - just that the logic on which SSTables to involve in a compaction is done in RACS. So we can make RACS check L0 and if there are none, L1 would involve grouping the SSTables by range and then calling the next background task for the underlying/wrapped CS instances and submitting them. In this way, the downside is calculating the grouping each time we ask for the next background task. We could also store it in memory in the form of a manifest like in LCS? So an array with the SSTables in each of them - beats having 256 instances but we're still going to have a 256 sized array in memory, I guess. It just seems so starkingly similar to a LCS restricted to just L0 and L1. A final thought: Is the memory footprint actually significant enough for us to want to not bite the bullet and further group the vnodes because the gains from having each SSTable as a single range is a lot, simple is a feature and RACS is customizable? Please excuse my ignorance if none of those suggestions made sense/worked, still not very confident with the code base.. > RangeAwareCompaction > > > Key: CASSANDRA-10540 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10540 > Project: Cassandra > Issue Type: New Feature >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Major > Labels: compaction, lcs, vnodes > Fix For: 4.x > > > Broken out from CASSANDRA-6696, we should split sstables based on ranges > during compaction. > Requirements; > * dont create tiny sstables - keep them bunched together until a single vnode > is big enough (configurable how big that is) > * make it possible to run existing compaction strategies on the per-range > sstables > We should probably add a global compaction strategy parameter that states > whether this should be enabled or not. -- 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-14402) Remove StreamCoordinator.streamExecutor thread pool
[ https://issues.apache.org/jira/browse/CASSANDRA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445300#comment-16445300 ] Dinesh Joshi commented on CASSANDRA-14402: -- LGTM +1 Thank you for getting rid of an unneeded thread [~jasobrown] > Remove StreamCoordinator.streamExecutor thread pool > --- > > Key: CASSANDRA-14402 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14402 > Project: Cassandra > Issue Type: Improvement > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 4.x > > > {{StreamCoordinator.streamExecutor}} was previously introduced to initiate > stream connections on a separate thread from the session invocation logic. > With CASSANDRA-12229 streaming now uses non-blocking IO, and connection > establishment is asynchronous via netty. Thus, the thread pool in > {{StreamCoordinator}} is unneeded. -- 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-14402) Remove StreamCoordinator.streamExecutor thread pool
[ https://issues.apache.org/jira/browse/CASSANDRA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dinesh Joshi updated CASSANDRA-14402: - Status: Patch Available (was: Open) > Remove StreamCoordinator.streamExecutor thread pool > --- > > Key: CASSANDRA-14402 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14402 > Project: Cassandra > Issue Type: Improvement > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 4.x > > > {{StreamCoordinator.streamExecutor}} was previously introduced to initiate > stream connections on a separate thread from the session invocation logic. > With CASSANDRA-12229 streaming now uses non-blocking IO, and connection > establishment is asynchronous via netty. Thus, the thread pool in > {{StreamCoordinator}} is unneeded. -- 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-14389) Resolve local address binding in 4.0
[ https://issues.apache.org/jira/browse/CASSANDRA-14389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dinesh Joshi updated CASSANDRA-14389: - Reviewer: Jason Brown Status: Patch Available (was: Open) > Resolve local address binding in 4.0 > > > Key: CASSANDRA-14389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14389 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Dinesh Joshi >Priority: Minor > Fix For: 4.x > > > CASSANDRA-8457/CASSANDRA-12229 introduced a regression against > CASSANDRA-12673. This was discovered with CASSANDRA-14362 and moved here for > resolution independent of that ticket. -- 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-14389) Resolve local address binding in 4.0
[ https://issues.apache.org/jira/browse/CASSANDRA-14389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445297#comment-16445297 ] Dinesh Joshi edited comment on CASSANDRA-14389 at 4/20/18 4:53 AM: --- Here's a fix along with restoring the behavior from CASSANDRA-12673 ||trunk|| |[branch|https://github.com/dineshjoshi/cassandra/tree/CASSANDRA-14389-trunk-fix-streaming]| |[utests & dtests|https://circleci.com/gh/dineshjoshi/workflows/cassandra/tree/CASSANDRA-14389-trunk-fix-streaming]| || was (Author: djoshi3): ||trunk|| |[branch|https://github.com/dineshjoshi/cassandra/tree/CASSANDRA-14389-trunk-fix-streaming]| |[utests & dtests|https://circleci.com/gh/dineshjoshi/workflows/cassandra/tree/CASSANDRA-14389-trunk-fix-streaming]| || > Resolve local address binding in 4.0 > > > Key: CASSANDRA-14389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14389 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Dinesh Joshi >Priority: Minor > Fix For: 4.x > > > CASSANDRA-8457/CASSANDRA-12229 introduced a regression against > CASSANDRA-12673. This was discovered with CASSANDRA-14362 and moved here for > resolution independent of that ticket. -- 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-14389) Resolve local address binding in 4.0
[ https://issues.apache.org/jira/browse/CASSANDRA-14389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445297#comment-16445297 ] Dinesh Joshi commented on CASSANDRA-14389: -- ||trunk|| |[branch|https://github.com/dineshjoshi/cassandra/tree/CASSANDRA-14389-trunk-fix-streaming]| |[utests & dtests|https://circleci.com/gh/dineshjoshi/workflows/cassandra/tree/CASSANDRA-14389-trunk-fix-streaming]| || > Resolve local address binding in 4.0 > > > Key: CASSANDRA-14389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14389 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Dinesh Joshi >Priority: Minor > Fix For: 4.x > > > CASSANDRA-8457/CASSANDRA-12229 introduced a regression against > CASSANDRA-12673. This was discovered with CASSANDRA-14362 and moved here for > resolution independent of that ticket. -- 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-10023) Emit a metric for number of local read and write calls
[ https://issues.apache.org/jira/browse/CASSANDRA-10023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damien Stevenson updated CASSANDRA-10023: - Status: Patch Available (was: Open) > Emit a metric for number of local read and write calls > -- > > Key: CASSANDRA-10023 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10023 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Damien Stevenson >Priority: Minor > Labels: lhf > Fix For: 4.x > > Attachments: 10023-trunk-dtests.txt, 10023-trunk.txt, > CASSANDRA-10023.patch > > > Many C* drivers have feature to be replica aware and chose the co-ordinator > which is a replica. We should add a metric which tells us whether all calls > to the co-ordinator are replica aware. > We have seen issues where client thinks they are replica aware when they > forget to add routing key at various places in the 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-14403) Inconsistent query results for different sort orders (ORDER BY)
[ https://issues.apache.org/jira/browse/CASSANDRA-14403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445282#comment-16445282 ] Chris Lohfink commented on CASSANDRA-14403: --- can you try it with CL.ALL to rule out issues with like unexpected RFs and insert CLs? > Inconsistent query results for different sort orders (ORDER BY) > --- > > Key: CASSANDRA-14403 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14403 > Project: Cassandra > Issue Type: Bug > Components: Core, CQL > Environment: Cassandra 3.5 on Windows Server 2016. Cluster is > geo-replicated with 120 machines in 3 datacenters (40 in each). Replication > factor for the table is 3. >Reporter: Mahmut Arslan >Priority: Major > > Cassandra returns inconsistent (different) results for the following queries > which only have sort orders specified in ORDER BY clause different. Table is > activities_v2 which has PK as (userid, lastmodified, activityid). Clustering > key lastmodified has clustering order DESC. Data types for key columns are > (userId, Text), (lastModified, Timestamp), (activityid, UUID). > Query "select activityid, lastmodified from activity_feed.activities_v2 where > userid = 'userid' and (lastmodified, activityid) >= > ('2018-04-01T07:29:52.611Z', e3afc72e-c41c-3e01-5397-0d972f6ced71) and > (lastmodified, activityid) < ('2018-06-01T07:29:52.611Z', > ----) ORDER BY lastmodified ASC, activityid > DESC;" returns 81 rows as seen below. > This query returns > activityid | lastmodified > --+- > a5358a6d-669d-3acd-221b-43e62e9adae0 | 2018-04-01 07:29:52.83+ > 11f3bc30-4522-dcb1-8dea-a883e2e9514d | 2018-04-01 07:29:53.189000+ > cd430200-de20-a614-7269-e9d7528e9cf9 | 2018-04-01 07:29:53.392000+ > c78a3000-8c9d-c6ad-1df5-c275c6410175 | 2018-04-01 07:29:53.611000+ > e93cde33-e2e0-a2f4-0083-4a1314e83364 | 2018-04-01 07:29:53.845000+ > 6289441d-2485-6345-5c0a-a06f6b571fab | 2018-04-01 07:31:53.524000+ > 8b4e8d1c-91c6-c549-a548-d87a297deb9c | 2018-04-01 07:31:53.758000+ > abe73ae9-0d7a-bcb5-13dc-b61f555c7e8a | 2018-04-01 07:31:53.993000+ > c386e284-c2ee-d511-6122-d6d410e95e34 | 2018-04-01 07:40:26.443000+ > 54b4c120-042d-1e27-ebd9-85824160c9ea | 2018-04-01 07:40:26.443000+ > c386e284-c2ee-d511-6122-d6d410e95e34 | 2018-04-01 07:40:26.662000+ > 544f224e-9ba1-d4a2-5554-c1a34bbf5fa4 | 2018-04-01 07:40:28.914000+ > b90ea728-5fb2-6047-d8ed-f64e6268a45e | 2018-04-01 07:40:28.914000+ > 963b308c-4c37-39df-4a0b-829026f3d054 | 2018-04-01 07:40:28.914000+ > 963b308c-4c37-39df-4a0b-829026f3d054 | 2018-04-01 07:40:29.414000+ > b90ea728-5fb2-6047-d8ed-f64e6268a45e | 2018-04-01 07:40:29.726000+ > 586dac57-fa58-9243-de1e-ceefb297dcd9 | 2018-04-06 21:22:33.50+ > > (81 rows) > When sort order changed in ORDER BY clause as "select activityid, > lastmodified from activity_feed.activities_v2 where userid = 'userid' and > (lastmodified, activityid) >= ('2018-04-01T07:29:52.611Z', > e3afc72e-c41c-3e01-5397-0d972f6ced71) and (lastmodified, activityid) < > ('2018-06-01T07:29:52.611Z', ----) ORDER BY > lastmodified DESC, activityid ASC;", query returns 142 rows as shown below. > > activityid | lastmodified > --+- > 3e2ab8b7-5619-79bf-c992-73d98a1ea9f6 | 2018-04-20 00:54:55.908000+ > 3073f667-5c9f-4bb7-99fb-5e84c219ac91 | 2018-04-20 00:23:49.899000+ > a3e375df-85a5-4e4f-bd74-73743efe1836 | 2018-04-20 00:23:49.805000+ > 3f91d870-34de-434a-9329-909d66cae0b4 | 2018-04-20 00:23:43.915000+ > 24179d47-4637-4c8d-a818-1b2b1752f790 | 2018-04-20 00:23:05.445000+ > 15bf9cb7-4e67-4cf6-818b-3713215dda32 | 2018-04-20 00:23:05.257000+ > 46c0a730-58c9-42f5-adfd-60261446b9e3 | 2018-04-20 00:23:02.788000+ > 24ecbe40-1990-4a43-a503-e6bf1ce537fb | 2018-04-19 23:44:15.339000+ > 833a2351-7fb7-2723-36d9-932f07a9bf1b | 2018-04-19 23:41:57.131000+ > 9158316d-022f-4150-94e4-229cbff777dc | 2018-04-19 23:41:00.129000+ > 848d5c2e-af22-4235-a6b8-4dcab83f44e5 | 2018-04-19 23:40:38.377000+ > 5646edc6-2f91-4e41-96b2-224bd74c7244 | 2018-04-19 23:37:42.294000+ > e81c24ff-8b6f-49b3-bdea-d1e178126716 | 2018-04-19 23:27:27.524000+ > 85d137ce-d88b-4d4a-bace-c85ffdccc137 | 2018-04-19 23:20:45.682000+ > c5c09a5d-4ffa-4a80-91b8-c44af30f0741 | 2018-04-19 23:20:07.587000+ > c3fc0958-5efb-4555-acca-07e7755cf8cf | 2018-04-19 23:19:55.368000+ > ... > (142 rows) > > In both cases, queries ran with Local_Quorum. > > > -- This message was sent by Atlassian JIRA (v7.6.
[jira] [Commented] (CASSANDRA-10023) Emit a metric for number of local read and write calls
[ https://issues.apache.org/jira/browse/CASSANDRA-10023?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445283#comment-16445283 ] Damien Stevenson commented on CASSANDRA-10023: -- I have attached a patch, including unit tests and dtests. Please review and let me know you thoughts. Cheers. > Emit a metric for number of local read and write calls > -- > > Key: CASSANDRA-10023 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10023 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Damien Stevenson >Priority: Minor > Labels: lhf > Fix For: 4.x > > Attachments: 10023-trunk-dtests.txt, 10023-trunk.txt, > CASSANDRA-10023.patch > > > Many C* drivers have feature to be replica aware and chose the co-ordinator > which is a replica. We should add a metric which tells us whether all calls > to the co-ordinator are replica aware. > We have seen issues where client thinks they are replica aware when they > forget to add routing key at various places in the 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-10023) Emit a metric for number of local read and write calls
[ https://issues.apache.org/jira/browse/CASSANDRA-10023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damien Stevenson updated CASSANDRA-10023: - Attachment: 10023-trunk-dtests.txt > Emit a metric for number of local read and write calls > -- > > Key: CASSANDRA-10023 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10023 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Damien Stevenson >Priority: Minor > Labels: lhf > Fix For: 4.x > > Attachments: 10023-trunk-dtests.txt, 10023-trunk.txt, > CASSANDRA-10023.patch > > > Many C* drivers have feature to be replica aware and chose the co-ordinator > which is a replica. We should add a metric which tells us whether all calls > to the co-ordinator are replica aware. > We have seen issues where client thinks they are replica aware when they > forget to add routing key at various places in the 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-10023) Emit a metric for number of local read and write calls
[ https://issues.apache.org/jira/browse/CASSANDRA-10023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damien Stevenson updated CASSANDRA-10023: - Attachment: 10023-trunk.txt > Emit a metric for number of local read and write calls > -- > > Key: CASSANDRA-10023 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10023 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Damien Stevenson >Priority: Minor > Labels: lhf > Fix For: 4.x > > Attachments: 10023-trunk-dtests.txt, 10023-trunk.txt, > CASSANDRA-10023.patch > > > Many C* drivers have feature to be replica aware and chose the co-ordinator > which is a replica. We should add a metric which tells us whether all calls > to the co-ordinator are replica aware. > We have seen issues where client thinks they are replica aware when they > forget to add routing key at various places in the 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-10989) Move away from SEDA to TPC
[ https://issues.apache.org/jira/browse/CASSANDRA-10989?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Murukesh Mohanan updated CASSANDRA-10989: - Description: Since its inception, Cassandra has been utilising [SEDA|http://www.mdw.la/papers/seda-sosp01.pdf] at its core. As originally conceived, it means every request is split into several stages, and each stage is backed by a thread pool. That imposes certain challenges: - thread parking/unparking overheads (partially improved by SEPExecutor in CASSANDRA-4718) - extensive context switching (i-/d- caches thrashing) - less than optimal multiple writer/multiple reader data structures for memtables, partitions, metrics, more - hard to grok concurrent code - large number of GC roots, longer TTSP - increased complexity for moving data structures off java heap - inability to easily balance writes/reads/compaction/flushing Latency implications of SEDA have been acknowledged by the authors themselves - see 2010 [retrospective on SEDA|http://matt-welsh.blogspot.co.uk/2010/07/retrospective-on-seda.html]. To fix these issues (and more), two years ago at NGCC [~benedict] suggested moving Cassandra away from SEDA to the more mechanically sympathetic thread per core architecture (TPC). See the slides from the original presentation [here|https://docs.google.com/presentation/d/19_U8I7mq9JKBjgPmmi6Hri3y308QEx1FmXLt-53QqEw/edit?ts=56265eb4#slide=id.g98ad32b25_1_19]. In a nutshell, each core would become a logical shared nothing micro instance of Cassandra, taking over a portion of the node’s range {{*}}. Client connections will be assigned randomly to one of the cores (sharing a single listen socket). A request that cannot be served by the client’s core will be proxied to the one owning the data, similar to the way we perform remote coordination today. Each thread (pinned to an exclusive core) would have a single event loop, and be responsible for both serving requests and performing maintenance tasks (flushing, compaction, repair), scheduling them intelligently. One notable exception from the original proposal is that we cannot, unfortunately, use linux AIO for file I/O, as it's only properly implemented for xfs. We might, however, have a specialised implementation for xfs and Windows (based on IOCP) later. In the meantime, we have no other choice other than to hand off I/O that cannot be served from cache to a separate threadpool. Transitioning from SEDA to TPC will be done in stages, incrementally and in parallel. This is a high-level overview meta-ticket that will track JIRA issues for each individual stage. {{*}} they’ll share certain things still, like schema, gossip, file I/O threadpool(s), and maybe MessagingService. was: Since its inception, Cassandra has been utilising [SEDA |http://www.eecs.harvard.edu/~mdw/papers/seda-sosp01.pdf] at its core. As originally conceived, it means every request is split into several stages, and each stage is backed by a thread pool. That imposes certain challenges: - thread parking/unparking overheads (partially improved by SEPExecutor in CASSANDRA-4718) - extensive context switching (i-/d- caches thrashing) - less than optimal multiple writer/multiple reader data structures for memtables, partitions, metrics, more - hard to grok concurrent code - large number of GC roots, longer TTSP - increased complexity for moving data structures off java heap - inability to easily balance writes/reads/compaction/flushing Latency implications of SEDA have been acknowledged by the authors themselves - see 2010 [retrospective on SEDA|http://matt-welsh.blogspot.co.uk/2010/07/retrospective-on-seda.html]. To fix these issues (and more), two years ago at NGCC [~benedict] suggested moving Cassandra away from SEDA to the more mechanically sympathetic thread per core architecture (TPC). See the slides from the original presentation [here|https://docs.google.com/presentation/d/19_U8I7mq9JKBjgPmmi6Hri3y308QEx1FmXLt-53QqEw/edit?ts=56265eb4#slide=id.g98ad32b25_1_19]. In a nutshell, each core would become a logical shared nothing micro instance of Cassandra, taking over a portion of the node’s range {{*}}. Client connections will be assigned randomly to one of the cores (sharing a single listen socket). A request that cannot be served by the client’s core will be proxied to the one owning the data, similar to the way we perform remote coordination today. Each thread (pinned to an exclusive core) would have a single event loop, and be responsible for both serving requests and performing maintenance tasks (flushing, compaction, repair), scheduling them intelligently. One notable exception from the original proposal is that we cannot, unfortunately, use linux AIO for file I/O, as it's only properly implemented for xfs. We might, however, have a specialised implementation for xfs and Windows (based on IOCP) later. In the meantime, w
[jira] [Comment Edited] (CASSANDRA-10540) RangeAwareCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445273#comment-16445273 ] Marcus Eriksson edited comment on CASSANDRA-10540 at 4/20/18 4:21 AM: -- [~Lerh Low] yeah its still on my plate, just not very happy with it at the moment, mostly because of the number of compaction strategy instances it runs with vnodes (#tokens * rf * (2 + #pending repairs)), might need to group vnodes to get it down to a reasonable number or something. was (Author: krummas): [~Lerh Low] yeah its still on my plate, just not very happy with it at the moment, mostly because of the number of compaction strategy instances it runs with vnodes (#tokens * rf), might need to group vnodes to get it down to a reasonable number or something. > RangeAwareCompaction > > > Key: CASSANDRA-10540 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10540 > Project: Cassandra > Issue Type: New Feature >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Major > Labels: compaction, lcs, vnodes > Fix For: 4.x > > > Broken out from CASSANDRA-6696, we should split sstables based on ranges > during compaction. > Requirements; > * dont create tiny sstables - keep them bunched together until a single vnode > is big enough (configurable how big that is) > * make it possible to run existing compaction strategies on the per-range > sstables > We should probably add a global compaction strategy parameter that states > whether this should be enabled or not. -- 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-10540) RangeAwareCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445273#comment-16445273 ] Marcus Eriksson commented on CASSANDRA-10540: - [~Lerh Low] yeah its still on my plate, just not very happy with it at the moment, mostly because of the number of compaction strategy instances it runs with vnodes (#tokens * rf), might need to group vnodes to get it down to a reasonable number or something. > RangeAwareCompaction > > > Key: CASSANDRA-10540 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10540 > Project: Cassandra > Issue Type: New Feature >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Major > Labels: compaction, lcs, vnodes > Fix For: 4.x > > > Broken out from CASSANDRA-6696, we should split sstables based on ranges > during compaction. > Requirements; > * dont create tiny sstables - keep them bunched together until a single vnode > is big enough (configurable how big that is) > * make it possible to run existing compaction strategies on the per-range > sstables > We should probably add a global compaction strategy parameter that states > whether this should be enabled or not. -- 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-14403) Inconsistent query results for different sort orders (ORDER BY)
Mahmut Arslan created CASSANDRA-14403: - Summary: Inconsistent query results for different sort orders (ORDER BY) Key: CASSANDRA-14403 URL: https://issues.apache.org/jira/browse/CASSANDRA-14403 Project: Cassandra Issue Type: Bug Components: Core, CQL Environment: Cassandra 3.5 on Windows Server 2016. Cluster is geo-replicated with 120 machines in 3 datacenters (40 in each). Replication factor for the table is 3. Reporter: Mahmut Arslan Cassandra returns inconsistent (different) results for the following queries which only have sort orders specified in ORDER BY clause different. Table is activities_v2 which has PK as (userid, lastmodified, activityid). Clustering key lastmodified has clustering order DESC. Data types for key columns are (userId, Text), (lastModified, Timestamp), (activityid, UUID). Query "select activityid, lastmodified from activity_feed.activities_v2 where userid = 'userid' and (lastmodified, activityid) >= ('2018-04-01T07:29:52.611Z', e3afc72e-c41c-3e01-5397-0d972f6ced71) and (lastmodified, activityid) < ('2018-06-01T07:29:52.611Z', ----) ORDER BY lastmodified ASC, activityid DESC;" returns 81 rows as seen below. This query returns activityid | lastmodified --+- a5358a6d-669d-3acd-221b-43e62e9adae0 | 2018-04-01 07:29:52.83+ 11f3bc30-4522-dcb1-8dea-a883e2e9514d | 2018-04-01 07:29:53.189000+ cd430200-de20-a614-7269-e9d7528e9cf9 | 2018-04-01 07:29:53.392000+ c78a3000-8c9d-c6ad-1df5-c275c6410175 | 2018-04-01 07:29:53.611000+ e93cde33-e2e0-a2f4-0083-4a1314e83364 | 2018-04-01 07:29:53.845000+ 6289441d-2485-6345-5c0a-a06f6b571fab | 2018-04-01 07:31:53.524000+ 8b4e8d1c-91c6-c549-a548-d87a297deb9c | 2018-04-01 07:31:53.758000+ abe73ae9-0d7a-bcb5-13dc-b61f555c7e8a | 2018-04-01 07:31:53.993000+ c386e284-c2ee-d511-6122-d6d410e95e34 | 2018-04-01 07:40:26.443000+ 54b4c120-042d-1e27-ebd9-85824160c9ea | 2018-04-01 07:40:26.443000+ c386e284-c2ee-d511-6122-d6d410e95e34 | 2018-04-01 07:40:26.662000+ 544f224e-9ba1-d4a2-5554-c1a34bbf5fa4 | 2018-04-01 07:40:28.914000+ b90ea728-5fb2-6047-d8ed-f64e6268a45e | 2018-04-01 07:40:28.914000+ 963b308c-4c37-39df-4a0b-829026f3d054 | 2018-04-01 07:40:28.914000+ 963b308c-4c37-39df-4a0b-829026f3d054 | 2018-04-01 07:40:29.414000+ b90ea728-5fb2-6047-d8ed-f64e6268a45e | 2018-04-01 07:40:29.726000+ 586dac57-fa58-9243-de1e-ceefb297dcd9 | 2018-04-06 21:22:33.50+ (81 rows) When sort order changed in ORDER BY clause as "select activityid, lastmodified from activity_feed.activities_v2 where userid = 'userid' and (lastmodified, activityid) >= ('2018-04-01T07:29:52.611Z', e3afc72e-c41c-3e01-5397-0d972f6ced71) and (lastmodified, activityid) < ('2018-06-01T07:29:52.611Z', ----) ORDER BY lastmodified DESC, activityid ASC;", query returns 142 rows as shown below. activityid | lastmodified --+- 3e2ab8b7-5619-79bf-c992-73d98a1ea9f6 | 2018-04-20 00:54:55.908000+ 3073f667-5c9f-4bb7-99fb-5e84c219ac91 | 2018-04-20 00:23:49.899000+ a3e375df-85a5-4e4f-bd74-73743efe1836 | 2018-04-20 00:23:49.805000+ 3f91d870-34de-434a-9329-909d66cae0b4 | 2018-04-20 00:23:43.915000+ 24179d47-4637-4c8d-a818-1b2b1752f790 | 2018-04-20 00:23:05.445000+ 15bf9cb7-4e67-4cf6-818b-3713215dda32 | 2018-04-20 00:23:05.257000+ 46c0a730-58c9-42f5-adfd-60261446b9e3 | 2018-04-20 00:23:02.788000+ 24ecbe40-1990-4a43-a503-e6bf1ce537fb | 2018-04-19 23:44:15.339000+ 833a2351-7fb7-2723-36d9-932f07a9bf1b | 2018-04-19 23:41:57.131000+ 9158316d-022f-4150-94e4-229cbff777dc | 2018-04-19 23:41:00.129000+ 848d5c2e-af22-4235-a6b8-4dcab83f44e5 | 2018-04-19 23:40:38.377000+ 5646edc6-2f91-4e41-96b2-224bd74c7244 | 2018-04-19 23:37:42.294000+ e81c24ff-8b6f-49b3-bdea-d1e178126716 | 2018-04-19 23:27:27.524000+ 85d137ce-d88b-4d4a-bace-c85ffdccc137 | 2018-04-19 23:20:45.682000+ c5c09a5d-4ffa-4a80-91b8-c44af30f0741 | 2018-04-19 23:20:07.587000+ c3fc0958-5efb-4555-acca-07e7755cf8cf | 2018-04-19 23:19:55.368000+ ... (142 rows) In both cases, queries ran with Local_Quorum. -- 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-14298) cqlshlib tests broken on b.a.o
[ https://issues.apache.org/jira/browse/CASSANDRA-14298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445174#comment-16445174 ] Patrick Bannister commented on CASSANDRA-14298: --- For cqlsh_tests/cqlsh_copy_tests.py, It looks like our workaround for importing functions from the cassandra cqlshlib no longer works. Since cqlshlib pointedly does not work in Python 3, this will be an interesting problem to solve. There are 25 tests that depend on functions imported from cqlshlib, which is about a third of the copy tests. > cqlshlib tests broken on b.a.o > -- > > Key: CASSANDRA-14298 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14298 > Project: Cassandra > Issue Type: Bug > Components: Build, Testing >Reporter: Stefan Podkowinski >Assignee: Patrick Bannister >Priority: Major > Attachments: cqlsh_tests_notes.md > > > It appears that cqlsh-tests on builds.apache.org on all branches stopped > working since we removed nosetests from the system environment. See e.g. > [here|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-trunk-cqlsh-tests/458/cython=no,jdk=JDK%201.8%20(latest),label=cassandra/console]. > Looks like we either have to make nosetests available again or migrate to > pytest as we did with dtests. Giving pytest a quick try resulted in many > errors locally, but I haven't inspected them in detail yet. -- 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-10540) RangeAwareCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16445061#comment-16445061 ] Lerh Chuan Low commented on CASSANDRA-10540: [~krummas] any chance you're still working on this? (It's been a while). I recently updated your branch against trunk and fixed a few merge conflicts: [https://github.com/juiceblender/cassandra/tree/10540] I'm also planning to write more unit tests to help get this in...which is in the works at the moment. Feel free to let me know if there's any other assistance you could use/need in getting this committed? > RangeAwareCompaction > > > Key: CASSANDRA-10540 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10540 > Project: Cassandra > Issue Type: New Feature >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Major > Labels: compaction, lcs, vnodes > Fix For: 4.x > > > Broken out from CASSANDRA-6696, we should split sstables based on ranges > during compaction. > Requirements; > * dont create tiny sstables - keep them bunched together until a single vnode > is big enough (configurable how big that is) > * make it possible to run existing compaction strategies on the per-range > sstables > We should probably add a global compaction strategy parameter that states > whether this should be enabled or not. -- 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-14389) Resolve local address binding in 4.0
[ https://issues.apache.org/jira/browse/CASSANDRA-14389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dinesh Joshi reassigned CASSANDRA-14389: Assignee: Dinesh Joshi (was: Jason Brown) > Resolve local address binding in 4.0 > > > Key: CASSANDRA-14389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14389 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Dinesh Joshi >Priority: Minor > Fix For: 4.x > > > CASSANDRA-8457/CASSANDRA-12229 introduced a regression against > CASSANDRA-12673. This was discovered with CASSANDRA-14362 and moved here for > resolution independent of that ticket. -- 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-14346) Scheduled Repair in Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-14346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444942#comment-16444942 ] Blake Eggleston edited comment on CASSANDRA-14346 at 4/19/18 10:56 PM: --- bq. The scheduler right now just uses the ActiveRepairService thread pool to see if Cassandra is running repairs I'm pretty sure that's not going to work in trunk. Repair was made async in 3.0. As of 4.0 it doesn't even block on validation, so the threadpool isn't going give you any valuable information. bq. I'm not sure of the state of incremental + subrange, is it fixed in trunk? Incremental repair and incremental repair + subrange is fixed in trunk, but you really shouldn't do subrange incremental repairs unless you have a really good reason, since you'll do a lot of additional anti-compaction. Anyway, as long as you're running incremental repair regularly, you should be able to repair full token ranges in less than 30 min. was (Author: bdeggleston): bq. The scheduler right now just uses the ActiveRepairService thread pool to see if Cassandra is running repairs I'm pretty sure that's not going to work in trunk. Repair was made async in 3.0. As of 4.0 it doesn't even block on validation, so the threadpool isn't going give you any valuable information. bq. I'm not sure of the state of incremental + subrange, is it fixed in trunk? Incremental repair and incremental repair + subrange is fixed in trunk, but you really shouldn't do subrange repairs unless you have a really good reason, since you'll do a lot of additional anti-compaction. Anyway, as long as you're running incremental repair regularly, you should be able to repair full token ranges in less than 30 min. > Scheduled Repair in Cassandra > - > > Key: CASSANDRA-14346 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14346 > Project: Cassandra > Issue Type: Improvement > Components: Repair >Reporter: Joseph Lynch >Priority: Major > Labels: CommunityFeedbackRequested > Fix For: 4.0 > > Attachments: ScheduledRepairV1_20180327.pdf > > > There have been many attempts to automate repair in Cassandra, which makes > sense given that it is necessary to give our users eventual consistency. Most > recently CASSANDRA-10070, CASSANDRA-8911 and CASSANDRA-13924 have all looked > for ways to solve this problem. > At Netflix we've built a scheduled repair service within Priam (our sidecar), > which we spoke about last year at NGCC. Given the positive feedback at NGCC > we focussed on getting it production ready and have now been using it in > production to repair hundreds of clusters, tens of thousands of nodes, and > petabytes of data for the past six months. Also based on feedback at NGCC we > have invested effort in figuring out how to integrate this natively into > Cassandra rather than open sourcing it as an external service (e.g. in Priam). > As such, [~vinaykumarcse] and I would like to re-work and merge our > implementation into Cassandra, and have created a [design > document|https://docs.google.com/document/d/1RV4rOrG1gwlD5IljmrIq_t45rz7H3xs9GbFSEyGzEtM/edit?usp=sharing] > showing how we plan to make it happen, including the the user interface. > As we work on the code migration from Priam to Cassandra, any feedback would > be greatly appreciated about the interface or v1 implementation features. I > have tried to call out in the document features which we explicitly consider > future work (as well as a path forward to implement them in the future) > because I would very much like to get this done before the 4.0 merge window > closes, and to do that I think aggressively pruning scope is going to be a > necessity. -- 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-14346) Scheduled Repair in Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-14346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444942#comment-16444942 ] Blake Eggleston commented on CASSANDRA-14346: - bq. The scheduler right now just uses the ActiveRepairService thread pool to see if Cassandra is running repairs I'm pretty sure that's not going to work in trunk. Repair was made async in 3.0. As of 4.0 it doesn't even block on validation, so the threadpool isn't going give you any valuable information. bq. I'm not sure of the state of incremental + subrange, is it fixed in trunk? Incremental repair and incremental repair + subrange is fixed in trunk, but you really shouldn't do subrange repairs unless you have a really good reason, since you'll do a lot of additional anti-compaction. Anyway, as long as you're running incremental repair regularly, you should be able to repair full token ranges in less than 30 min. > Scheduled Repair in Cassandra > - > > Key: CASSANDRA-14346 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14346 > Project: Cassandra > Issue Type: Improvement > Components: Repair >Reporter: Joseph Lynch >Priority: Major > Labels: CommunityFeedbackRequested > Fix For: 4.0 > > Attachments: ScheduledRepairV1_20180327.pdf > > > There have been many attempts to automate repair in Cassandra, which makes > sense given that it is necessary to give our users eventual consistency. Most > recently CASSANDRA-10070, CASSANDRA-8911 and CASSANDRA-13924 have all looked > for ways to solve this problem. > At Netflix we've built a scheduled repair service within Priam (our sidecar), > which we spoke about last year at NGCC. Given the positive feedback at NGCC > we focussed on getting it production ready and have now been using it in > production to repair hundreds of clusters, tens of thousands of nodes, and > petabytes of data for the past six months. Also based on feedback at NGCC we > have invested effort in figuring out how to integrate this natively into > Cassandra rather than open sourcing it as an external service (e.g. in Priam). > As such, [~vinaykumarcse] and I would like to re-work and merge our > implementation into Cassandra, and have created a [design > document|https://docs.google.com/document/d/1RV4rOrG1gwlD5IljmrIq_t45rz7H3xs9GbFSEyGzEtM/edit?usp=sharing] > showing how we plan to make it happen, including the the user interface. > As we work on the code migration from Priam to Cassandra, any feedback would > be greatly appreciated about the interface or v1 implementation features. I > have tried to call out in the document features which we explicitly consider > future work (as well as a path forward to implement them in the future) > because I would very much like to get this done before the 4.0 merge window > closes, and to do that I think aggressively pruning scope is going to be a > necessity. -- 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-13985) Support restricting reads and writes to specific datacenters on a per user basis
[ https://issues.apache.org/jira/browse/CASSANDRA-13985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444935#comment-16444935 ] Blake Eggleston commented on CASSANDRA-13985: - pushed an updated branch [here|https://github.com/bdeggleston/cassandra/tree/13985-v2], corresponding dtest branch [here|https://github.com/bdeggleston/cassandra-dtest/tree/13985-v2], and the [tests|https://circleci.com/workflow-run/caecca68-6dcc-4a4a-84fc-5eb6083825c0]. Some notes / responses: * I prefer your version of the syntax. Set literal syntax is familiar, and it's clearer what it is you're doing, although slightly more verbose. * I went with throwing a configuration exception on startup if you have a nonsensical config. * Regarding re-using the cache, I'd prefer sharing config with the RolesCache. Even though they're separate systems under the hood, they're sort of exposed as part of the same system through the DCL. * I've reverted the changes to CREATE/ALTER USER statements > Support restricting reads and writes to specific datacenters on a per user > basis > > > Key: CASSANDRA-13985 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13985 > Project: Cassandra > Issue Type: Improvement >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Minor > Fix For: 4.0 > > > There are a few use cases where it makes sense to restrict the operations a > given user can perform in specific data centers. The obvious use case is the > production/analytics datacenter configuration. You don’t want the production > user to be reading/or writing to the analytics datacenter, and you don’t want > the analytics user to be reading from the production datacenter. > Although we expect users to get this right on that application level, we > should also be able to enforce this at the database level. The first approach > that comes to mind would be to support an optional DC parameter when granting > select and modify permissions to roles. Something like {{GRANT SELECT ON > some_keyspace TO that_user IN DC dc1}}, statements that omit the dc would > implicitly be granting permission to all dcs. However, I’m not married to > this approach. -- 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-14381) nodetool listsnapshots is missing snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-14381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444914#comment-16444914 ] Cyril Scetbon commented on CASSANDRA-14381: --- +1 on [https://github.com/apache/cassandra/commit/a0ceb3] > nodetool listsnapshots is missing snapshots > --- > > Key: CASSANDRA-14381 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14381 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: MacOs 10.12.5 > Java 1.8.0_144 > Cassandra 3.11.2 (brew install) >Reporter: Cyril Scetbon >Assignee: Ariel Weisberg >Priority: Major > > The output of *nodetool listsnapshots* is inconsistent with the snapshots > created : > {code:java} > $ nodetool listsnapshots > Snapshot Details: > There are no snapshots > $ nodetool snapshot -t tag1 --table local system > Requested creating snapshot(s) for [system] with snapshot name [tag1] and > options {skipFlush=false} > Snapshot directory: tag1 > $ nodetool snapshot -t tag2 --table local system > Requested creating snapshot(s) for [system] with snapshot name [tag2] and > options {skipFlush=false} > Snapshot directory: tag2 > $ nodetool listsnapshots > Snapshot Details: > There are no snapshots > $ ls > /usr/local/var/lib/cassandra/data/system/local-7ad54392bcdd35a684174e047860b377/snapshots/ > tag1 tag2{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-14392) Rename nodetool --with-port to --print-port to disambiguate from --port
[ https://issues.apache.org/jira/browse/CASSANDRA-14392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-14392: --- Resolution: Fixed Status: Resolved (was: Ready to Commit) Committed as [a8be43e45424950abaabed4e450d125e91ecaab2|https://github.com/apache/cassandra/commit/a8be43e45424950abaabed4e450d125e91ecaab2]. Thanks! > Rename nodetool --with-port to --print-port to disambiguate from --port > --- > > Key: CASSANDRA-14392 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14392 > Project: Cassandra > Issue Type: Improvement > Components: Tools >Reporter: Ariel Weisberg >Assignee: Ariel Weisberg >Priority: Major > Fix For: 4.0 > > > Right now it looks kind of like a third way to set the port number. -- 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: Rename nodetool --with-port to --print-port to disambiguate from --port
Repository: cassandra Updated Branches: refs/heads/trunk cb67bfc16 -> a8be43e45 Rename nodetool --with-port to --print-port to disambiguate from --port Patch by Ariel Weisberg; Reviewed by Stefan Podkowinski for CASSANDRA-14392 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a8be43e4 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a8be43e4 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a8be43e4 Branch: refs/heads/trunk Commit: a8be43e45424950abaabed4e450d125e91ecaab2 Parents: cb67bfc Author: Ariel Weisberg Authored: Tue Apr 17 12:35:38 2018 -0400 Committer: Ariel Weisberg Committed: Thu Apr 19 18:20:15 2018 -0400 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/tools/NodeTool.java | 4 ++-- .../cassandra/tools/nodetool/DescribeCluster.java | 2 +- .../cassandra/tools/nodetool/DescribeRing.java| 2 +- .../tools/nodetool/FailureDetectorInfo.java | 2 +- .../cassandra/tools/nodetool/GetEndpoints.java| 2 +- .../cassandra/tools/nodetool/GossipInfo.java | 2 +- .../apache/cassandra/tools/nodetool/NetStats.java | 8 .../cassandra/tools/nodetool/RemoveNode.java | 4 ++-- .../org/apache/cassandra/tools/nodetool/Ring.java | 4 ++-- .../apache/cassandra/tools/nodetool/Status.java | 18 +- 11 files changed, 25 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/a8be43e4/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index a8dbbba..7d9769c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Rename nodetool --with-port to --print-port to disambiguate from --port (CASSANDRA-14392) * Client TOPOLOGY_CHANGE messages have wrong port. (CASSANDRA-14398) * Add ability to load new SSTables from a separate directory (CASSANDRA-6719) * Eliminate background repair and probablistic read_repair_chance table options http://git-wip-us.apache.org/repos/asf/cassandra/blob/a8be43e4/src/java/org/apache/cassandra/tools/NodeTool.java -- diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index d4cc291..fb9204c 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -253,8 +253,8 @@ public class NodeTool @Option(type = OptionType.GLOBAL, name = {"-pwf", "--password-file"}, description = "Path to the JMX password file") private String passwordFilePath = EMPTY; -@Option(type = OptionType.GLOBAL, name = { "-wp", "--with-port"}, description = "Operate in 4.0 mode with hosts disambiguated by port number", arity = 0) -protected boolean withPort = false; +@Option(type = OptionType.GLOBAL, name = { "-pp", "--print-port"}, description = "Operate in 4.0 mode with hosts disambiguated by port number", arity = 0) +protected boolean printPort = false; @Override public void run() http://git-wip-us.apache.org/repos/asf/cassandra/blob/a8be43e4/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java -- diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java index da02db6..fe33551 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java +++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java @@ -59,7 +59,7 @@ public class DescribeCluster extends NodeToolCmd // display schema version for each node System.out.println("\tSchema versions:"); -Map> schemaVersions = withPort ? probe.getSpProxy().getSchemaVersionsWithPort() : probe.getSpProxy().getSchemaVersions(); +Map> schemaVersions = printPort ? probe.getSpProxy().getSchemaVersionsWithPort() : probe.getSpProxy().getSchemaVersions(); for (String version : schemaVersions.keySet()) { System.out.println(format("\t\t%s: %s%n", version, schemaVersions.get(version))); http://git-wip-us.apache.org/repos/asf/cassandra/blob/a8be43e4/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java -- diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java index c57e54d..ef8c97e 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java +++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java @@ -39,7 +39,7 @@ public
[jira] [Commented] (CASSANDRA-14381) nodetool listsnapshots is missing snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-14381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444896#comment-16444896 ] Ariel Weisberg commented on CASSANDRA-14381: [~cscetbon] I need your explicit +1 to use you as the reviewer. Thanks. > nodetool listsnapshots is missing snapshots > --- > > Key: CASSANDRA-14381 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14381 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: MacOs 10.12.5 > Java 1.8.0_144 > Cassandra 3.11.2 (brew install) >Reporter: Cyril Scetbon >Assignee: Ariel Weisberg >Priority: Major > > The output of *nodetool listsnapshots* is inconsistent with the snapshots > created : > {code:java} > $ nodetool listsnapshots > Snapshot Details: > There are no snapshots > $ nodetool snapshot -t tag1 --table local system > Requested creating snapshot(s) for [system] with snapshot name [tag1] and > options {skipFlush=false} > Snapshot directory: tag1 > $ nodetool snapshot -t tag2 --table local system > Requested creating snapshot(s) for [system] with snapshot name [tag2] and > options {skipFlush=false} > Snapshot directory: tag2 > $ nodetool listsnapshots > Snapshot Details: > There are no snapshots > $ ls > /usr/local/var/lib/cassandra/data/system/local-7ad54392bcdd35a684174e047860b377/snapshots/ > tag1 tag2{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-14392) Rename nodetool --with-port to --print-port to disambiguate from --port
[ https://issues.apache.org/jira/browse/CASSANDRA-14392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-14392: --- Status: Ready to Commit (was: Patch Available) > Rename nodetool --with-port to --print-port to disambiguate from --port > --- > > Key: CASSANDRA-14392 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14392 > Project: Cassandra > Issue Type: Improvement > Components: Tools >Reporter: Ariel Weisberg >Assignee: Ariel Weisberg >Priority: Major > Fix For: 4.0 > > > Right now it looks kind of like a third way to set the port number. -- 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-14381) nodetool listsnapshots is missing snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-14381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-14381: --- Status: Ready to Commit (was: Patch Available) > nodetool listsnapshots is missing snapshots > --- > > Key: CASSANDRA-14381 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14381 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: MacOs 10.12.5 > Java 1.8.0_144 > Cassandra 3.11.2 (brew install) >Reporter: Cyril Scetbon >Assignee: Ariel Weisberg >Priority: Major > > The output of *nodetool listsnapshots* is inconsistent with the snapshots > created : > {code:java} > $ nodetool listsnapshots > Snapshot Details: > There are no snapshots > $ nodetool snapshot -t tag1 --table local system > Requested creating snapshot(s) for [system] with snapshot name [tag1] and > options {skipFlush=false} > Snapshot directory: tag1 > $ nodetool snapshot -t tag2 --table local system > Requested creating snapshot(s) for [system] with snapshot name [tag2] and > options {skipFlush=false} > Snapshot directory: tag2 > $ nodetool listsnapshots > Snapshot Details: > There are no snapshots > $ ls > /usr/local/var/lib/cassandra/data/system/local-7ad54392bcdd35a684174e047860b377/snapshots/ > tag1 tag2{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] [Assigned] (CASSANDRA-13985) Support restricting reads and writes to specific datacenters on a per user basis
[ https://issues.apache.org/jira/browse/CASSANDRA-13985?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg reassigned CASSANDRA-13985: -- Assignee: Blake Eggleston (was: Ariel Weisberg) > Support restricting reads and writes to specific datacenters on a per user > basis > > > Key: CASSANDRA-13985 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13985 > Project: Cassandra > Issue Type: Improvement >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Minor > Fix For: 4.0 > > > There are a few use cases where it makes sense to restrict the operations a > given user can perform in specific data centers. The obvious use case is the > production/analytics datacenter configuration. You don’t want the production > user to be reading/or writing to the analytics datacenter, and you don’t want > the analytics user to be reading from the production datacenter. > Although we expect users to get this right on that application level, we > should also be able to enforce this at the database level. The first approach > that comes to mind would be to support an optional DC parameter when granting > select and modify permissions to roles. Something like {{GRANT SELECT ON > some_keyspace TO that_user IN DC dc1}}, statements that omit the dc would > implicitly be granting permission to all dcs. However, I’m not married to > this approach. -- 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-13985) Support restricting reads and writes to specific datacenters on a per user basis
[ https://issues.apache.org/jira/browse/CASSANDRA-13985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444772#comment-16444772 ] Ariel Weisberg commented on CASSANDRA-13985: There was a mixup and Blake kept working on it. So at this point I'll just leave it to Blake. > Support restricting reads and writes to specific datacenters on a per user > basis > > > Key: CASSANDRA-13985 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13985 > Project: Cassandra > Issue Type: Improvement >Reporter: Blake Eggleston >Assignee: Ariel Weisberg >Priority: Minor > Fix For: 4.0 > > > There are a few use cases where it makes sense to restrict the operations a > given user can perform in specific data centers. The obvious use case is the > production/analytics datacenter configuration. You don’t want the production > user to be reading/or writing to the analytics datacenter, and you don’t want > the analytics user to be reading from the production datacenter. > Although we expect users to get this right on that application level, we > should also be able to enforce this at the database level. The first approach > that comes to mind would be to support an optional DC parameter when granting > select and modify permissions to roles. Something like {{GRANT SELECT ON > some_keyspace TO that_user IN DC dc1}}, statements that omit the dc would > implicitly be granting permission to all dcs. However, I’m not married to > this approach. -- 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-13985) Support restricting reads and writes to specific datacenters on a per user basis
[ https://issues.apache.org/jira/browse/CASSANDRA-13985?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg reassigned CASSANDRA-13985: -- Assignee: Ariel Weisberg (was: Blake Eggleston) > Support restricting reads and writes to specific datacenters on a per user > basis > > > Key: CASSANDRA-13985 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13985 > Project: Cassandra > Issue Type: Improvement >Reporter: Blake Eggleston >Assignee: Ariel Weisberg >Priority: Minor > Fix For: 4.0 > > > There are a few use cases where it makes sense to restrict the operations a > given user can perform in specific data centers. The obvious use case is the > production/analytics datacenter configuration. You don’t want the production > user to be reading/or writing to the analytics datacenter, and you don’t want > the analytics user to be reading from the production datacenter. > Although we expect users to get this right on that application level, we > should also be able to enforce this at the database level. The first approach > that comes to mind would be to support an optional DC parameter when granting > select and modify permissions to roles. Something like {{GRANT SELECT ON > some_keyspace TO that_user IN DC dc1}}, statements that omit the dc would > implicitly be granting permission to all dcs. However, I’m not married to > this approach. -- 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-13985) Support restricting reads and writes to specific datacenters on a per user basis
[ https://issues.apache.org/jira/browse/CASSANDRA-13985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444694#comment-16444694 ] Ariel Weisberg commented on CASSANDRA-13985: RE: Added syntax Philosophically I want there to be the minimum number of ways to accomplish something. The fact that it's default open to all data centers is a bit sketchy. But the situation we are in is that we probably shouldn't start requiring people to add "ACCESS TO ALL DATACENTERS" so if we only want one way then it's just #4. AND ACCESS TO DATA CENTERS vs WITH DATACENTERS I don't have a strong preference, but more succinct is better since it is easier to remember. I think listing the DCs as a sequence of ORs is a bit awkward so I am on board with that change. bq. I'm also not sure whether we should bother updating CREATE/ALTER USER. They're basically deprecated and just support a subset of the role management statements, i.e. no support for OPTIONS or LOGIN. I won't argue though if we do want to add this to them. [~bdeggleston] if it's deprecated I would like to leave this functionality out to further discourage people from continuing to use it. The less it appears in the wild the better. bq. Should we warn at startup if authentication is not enabled, but network authorization is? Warn or fail to start? This seems like someone is asking for something nonsensical and when it comes to authz fail closed is the best thing to do right? Looking into the rest of the feedback deeper now, but most of it makes sense to me. > Support restricting reads and writes to specific datacenters on a per user > basis > > > Key: CASSANDRA-13985 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13985 > Project: Cassandra > Issue Type: Improvement >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Minor > Fix For: 4.0 > > > There are a few use cases where it makes sense to restrict the operations a > given user can perform in specific data centers. The obvious use case is the > production/analytics datacenter configuration. You don’t want the production > user to be reading/or writing to the analytics datacenter, and you don’t want > the analytics user to be reading from the production datacenter. > Although we expect users to get this right on that application level, we > should also be able to enforce this at the database level. The first approach > that comes to mind would be to support an optional DC parameter when granting > select and modify permissions to roles. Something like {{GRANT SELECT ON > some_keyspace TO that_user IN DC dc1}}, statements that omit the dc would > implicitly be granting permission to all dcs. However, I’m not married to > this approach. -- 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-14346) Scheduled Repair in Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-14346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444675#comment-16444675 ] Joseph Lynch commented on CASSANDRA-14346: -- Ah ok, good to know. The scheduler right now just uses the [{{ActiveRepairService}}|https://github.com/apache/cassandra/blob/34a1d5da58fb8edcad39633084541bb4162f5ede/src/java/org/apache/cassandra/service/ActiveRepairService.java#L98] thread pool to see if Cassandra is running repairs (looks like this is renamed to {{Repair-Task}} in trunk) and [{{forceTerminateAllRepairSessions}}|https://github.com/apache/cassandra/blob/cb67bfc1639ded1b6937e7347ad42177ea3f24e3/src/java/org/apache/cassandra/service/StorageServiceMBean.java#L348] to kill them after a resume + timeout. We can do the same for trunk or enrich the interface to give more granular control (status by repair cmd number would probably be sufficient, although maybe we'd need actual uuids for repairs). Right now we don't support timing out individual parallel subranges (we just kill everything), so that would be a nice improvement to be able to cancel individual repairs (I know this doesn't cancel the streaming in 2.x, not sure about trunk). {quote}Should I interpret this to mean that your scheduler breaks incremental repairs into small subranges? {quote} We're running 2.1 in production so we only do full range since we heard that incremental was very broken in 2.1, but the subrange breaking is at a higher level of abstraction so I don't see why it couldn't apply to incremental if we wanted. I'm not sure of the state of incremental + subrange, is it fixed in trunk? If so we can definitely do the splitting for incremental as well. We like splitting up the token ranges into similarly sized pieces because it makes the timeout logic much easier to reason about (long running repairs are super annoying to tell if they are stuck or not). > Scheduled Repair in Cassandra > - > > Key: CASSANDRA-14346 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14346 > Project: Cassandra > Issue Type: Improvement > Components: Repair >Reporter: Joseph Lynch >Priority: Major > Labels: CommunityFeedbackRequested > Fix For: 4.0 > > Attachments: ScheduledRepairV1_20180327.pdf > > > There have been many attempts to automate repair in Cassandra, which makes > sense given that it is necessary to give our users eventual consistency. Most > recently CASSANDRA-10070, CASSANDRA-8911 and CASSANDRA-13924 have all looked > for ways to solve this problem. > At Netflix we've built a scheduled repair service within Priam (our sidecar), > which we spoke about last year at NGCC. Given the positive feedback at NGCC > we focussed on getting it production ready and have now been using it in > production to repair hundreds of clusters, tens of thousands of nodes, and > petabytes of data for the past six months. Also based on feedback at NGCC we > have invested effort in figuring out how to integrate this natively into > Cassandra rather than open sourcing it as an external service (e.g. in Priam). > As such, [~vinaykumarcse] and I would like to re-work and merge our > implementation into Cassandra, and have created a [design > document|https://docs.google.com/document/d/1RV4rOrG1gwlD5IljmrIq_t45rz7H3xs9GbFSEyGzEtM/edit?usp=sharing] > showing how we plan to make it happen, including the the user interface. > As we work on the code migration from Priam to Cassandra, any feedback would > be greatly appreciated about the interface or v1 implementation features. I > have tried to call out in the document features which we explicitly consider > future work (as well as a path forward to implement them in the future) > because I would very much like to get this done before the 4.0 merge window > closes, and to do that I think aggressively pruning scope is going to be a > necessity. -- 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-14401) Attempted serializing to buffer exceeded maximum of 65535 bytes
[ https://issues.apache.org/jira/browse/CASSANDRA-14401?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444656#comment-16444656 ] Sam Tunnicliffe commented on CASSANDRA-14401: - It seems you're using a custom index expression, I guess that this is a solr/lucene index and you have a 67k querystring? > Attempted serializing to buffer exceeded maximum of 65535 bytes > > > Key: CASSANDRA-14401 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14401 > Project: Cassandra > Issue Type: Bug >Reporter: Artem Rokhin >Priority: Major > > Cassandra version: 3.11.2 > 3 nodes cluster > The following exception appears on all 3 nodes and after awhile cluster > becomes unreposnsive > > {code} > java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of > 65535 bytes: 67661 > at > org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) > [apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) > [apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) > [apache-cassandra-3.11.2.jar:3.11.2] > {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-14346) Scheduled Repair in Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-14346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444654#comment-16444654 ] Blake Eggleston commented on CASSANDRA-14346: - bq. I'm not entirely sure which interfaces need to be spruced up? I think the existing trunk methods are sufficient for the sidecar to rectify the state of repairs that are running in Cassandra with those in the database. [~jolynch], those methods only work with incremental repairs, full repairs can't be controlled through those. Also, they only fail repairs in the sense that they prevent the incremental repair session (which is different from a parent repair session) from moving to it's next state and instead short circuit it to failed. Validations and streams that are in flight are still uncontrollable. What I'm assuming Kurt meant is the fact that we rely on an unbroken jmx connection on the repair client side. bq. Since the repair scheduler keeps work very small (targeting ~30 minute pieces of work) even if we do the calculations wrong we shouldn't lose very much work. Should I interpret this to mean that your scheduler breaks incremental repairs into small subranges? > Scheduled Repair in Cassandra > - > > Key: CASSANDRA-14346 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14346 > Project: Cassandra > Issue Type: Improvement > Components: Repair >Reporter: Joseph Lynch >Priority: Major > Labels: CommunityFeedbackRequested > Fix For: 4.0 > > Attachments: ScheduledRepairV1_20180327.pdf > > > There have been many attempts to automate repair in Cassandra, which makes > sense given that it is necessary to give our users eventual consistency. Most > recently CASSANDRA-10070, CASSANDRA-8911 and CASSANDRA-13924 have all looked > for ways to solve this problem. > At Netflix we've built a scheduled repair service within Priam (our sidecar), > which we spoke about last year at NGCC. Given the positive feedback at NGCC > we focussed on getting it production ready and have now been using it in > production to repair hundreds of clusters, tens of thousands of nodes, and > petabytes of data for the past six months. Also based on feedback at NGCC we > have invested effort in figuring out how to integrate this natively into > Cassandra rather than open sourcing it as an external service (e.g. in Priam). > As such, [~vinaykumarcse] and I would like to re-work and merge our > implementation into Cassandra, and have created a [design > document|https://docs.google.com/document/d/1RV4rOrG1gwlD5IljmrIq_t45rz7H3xs9GbFSEyGzEtM/edit?usp=sharing] > showing how we plan to make it happen, including the the user interface. > As we work on the code migration from Priam to Cassandra, any feedback would > be greatly appreciated about the interface or v1 implementation features. I > have tried to call out in the document features which we explicitly consider > future work (as well as a path forward to implement them in the future) > because I would very much like to get this done before the 4.0 merge window > closes, and to do that I think aggressively pruning scope is going to be a > necessity. -- 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-14346) Scheduled Repair in Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-14346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444598#comment-16444598 ] Joseph Lynch commented on CASSANDRA-14346: -- [~KurtG] {quote}I think sidecar is a better choice purely for isolation from the read/write path, but think that we need to fix up the interface to repair first. As Blake mentioned, most problems come from the fact that JMX sucks and managing repairs over JMX is worse. I think as part of this work (or as a first step) we should be better defining this interface, and making it far more robust. I think we should target the initial work for 4.0 - sprucing up interfaces so that repair is easier to work with and making failure handling fool-proof, as at least we'll probably be able to reach agreement on that front in a somewhat timely fashion. It seems a bit optimistic to target all the scheduling for 4.0 at this stage, but I suppose it depends how much time people want to dedicate to this. {quote} I'm not entirely sure which interfaces need to be spruced up? I think the existing trunk [methods|https://github.com/apache/cassandra/blob/8b3a60b9a7dbefeecc06bace617279612ec7092d/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java#L28-L29] are sufficient for the sidecar to rectify the state of repairs that are running in Cassandra with those in the database. Since the repair scheduler keeps work very small (targeting ~30 minute pieces of work) even if we do the calculations wrong we shouldn't lose very much work. {quote}Also we should keep in mind CASSANDRA-14395 as there's going to be a lot of overlap here if we go down the sidecar route. {quote} Yea, I agree but don't want to block on that. If/when that ends up getting merged we can definitely unify the two tools. One of the nice things about HTTP interfaces over a known port is that you can swap out what provides them pretty easily. {quote}If referring to incremental repair, wouldn't this already be the case in 4.0? Subrange repair works with incremental repair in trunk at the moment, so we should already get some major benefits here. Unless I'm missing something... In other news, for interests sake (slightly off topic) it seems DS is trying to do away with traditional repair, and instead they've gone the query at CL.ALL route (or similar) in their new "repair" system. I don't think this is a good idea, but good to keep in mind how everyone is approaching the problem. {quote} Adaptive subrange is an existing strategy we use for 2.1 and 3.0 (pre-incremental) where the repair scheduler ensures lots of small pieces of work (which can be done in parallel) so that if we lose it we can resume without losing too much work; essentially you never ever ever do full range unless the dataset is small. I think incremental or continuous repair (read repairing only data that is inconsistent) are complementary to this concept in that they provide a way to make the work take less time generally speaking. If/when those techniques are production ready, I believe the design makes it super easy for users to switch (by changing the {{type}}). > Scheduled Repair in Cassandra > - > > Key: CASSANDRA-14346 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14346 > Project: Cassandra > Issue Type: Improvement > Components: Repair >Reporter: Joseph Lynch >Priority: Major > Labels: CommunityFeedbackRequested > Fix For: 4.0 > > Attachments: ScheduledRepairV1_20180327.pdf > > > There have been many attempts to automate repair in Cassandra, which makes > sense given that it is necessary to give our users eventual consistency. Most > recently CASSANDRA-10070, CASSANDRA-8911 and CASSANDRA-13924 have all looked > for ways to solve this problem. > At Netflix we've built a scheduled repair service within Priam (our sidecar), > which we spoke about last year at NGCC. Given the positive feedback at NGCC > we focussed on getting it production ready and have now been using it in > production to repair hundreds of clusters, tens of thousands of nodes, and > petabytes of data for the past six months. Also based on feedback at NGCC we > have invested effort in figuring out how to integrate this natively into > Cassandra rather than open sourcing it as an external service (e.g. in Priam). > As such, [~vinaykumarcse] and I would like to re-work and merge our > implementation into Cassandra, and have created a [design > document|https://docs.google.com/document/d/1RV4rOrG1gwlD5IljmrIq_t45rz7H3xs9GbFSEyGzEtM/edit?usp=sharing] > showing how we plan to make it happen, including the the user interface. > As we work on the code migration from Priam to Cassandra, any feedback would > be greatly appreciated about the interface or v1 implement
[jira] [Commented] (CASSANDRA-14395) C* Management process
[ https://issues.apache.org/jira/browse/CASSANDRA-14395?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444582#comment-16444582 ] Joseph Lynch commented on CASSANDRA-14395: -- I hope we run the management sidecar on all nodes as if it were part of the database, providing a super easy/language agnostic HTTP interface and proxying requests to the proper other sidecars just as Cassandra coordinators do today for CQL requests. This means that users don't have to have a separate discovery system for the sidecar, understand the difference between "special" nodes and not, and assures that we build in fault tolerance by default. This is also especially important if we want to unlock next level automatic operations features like extracting compaction/repair execution from the main process. {quote} so do we pass commands between the management processes and then each node issues its own JMX commands to itself? {quote} I think this is the right approach, use JMX for the last mile and use HTTP between the sidecars for the rest. Since the connections are not persistent it will be much more resilient and easy to program for (dealing with JMX availability over remote long lived connections is ... no fun). > 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 > > 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-14401) Attempted serializing to buffer exceeded maximum of 65535 bytes
[ https://issues.apache.org/jira/browse/CASSANDRA-14401?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1677#comment-1677 ] Jason Brown commented on CASSANDRA-14401: - Can you share the query you issued to get into this state? It's clearly some kind of read, and filter is large (or referencing something large). > Attempted serializing to buffer exceeded maximum of 65535 bytes > > > Key: CASSANDRA-14401 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14401 > Project: Cassandra > Issue Type: Bug >Reporter: Artem Rokhin >Priority: Major > > Cassandra version: 3.11.2 > 3 nodes cluster > The following exception appears on all 3 nodes and after awhile cluster > becomes unreposnsive > > {code} > java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of > 65535 bytes: 67661 > at > org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) > [apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) > [apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) > [apache-cassandra-3.11.2.jar:3.11.2] > {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-14402) Remove StreamCoordinator.streamExecutor thread pool
[ https://issues.apache.org/jira/browse/CASSANDRA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeremy Hanna updated CASSANDRA-14402: - Description: {{StreamCoordinator.streamExecutor}} was previously introduced to initiate stream connections on a separate thread from the session invocation logic. With CASSANDRA-12229 streaming now uses non-blocking IO, and connection establishment is asynchronous via netty. Thus, the thread pool in {{StreamCoordinator}} is unneeded. (was: {{StreamCoordinator.streamExecutor}} was previously introduced to initiate stream connections on a separate thread from the session invocation logic. With CASSANDRA-1229 streaming now uses non-blocking IO, and connection establishment is asynchronous via netty. Thus, the thread pool in {{StreamCoordinator}} is unneeded. ) > Remove StreamCoordinator.streamExecutor thread pool > --- > > Key: CASSANDRA-14402 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14402 > Project: Cassandra > Issue Type: Improvement > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 4.x > > > {{StreamCoordinator.streamExecutor}} was previously introduced to initiate > stream connections on a separate thread from the session invocation logic. > With CASSANDRA-12229 streaming now uses non-blocking IO, and connection > establishment is asynchronous via netty. Thus, the thread pool in > {{StreamCoordinator}} is unneeded. -- 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-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444371#comment-16444371 ] Blake Eggleston commented on CASSANDRA-14400: - Right, unless they stay in pending after a forced compaction, things are working as expected. > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-14398: --- Resolution: Fixed Status: Resolved (was: Ready to Commit) Committed as [cb67bfc1639ded1b6937e7347ad42177ea3f24e3|https://github.com/apache/cassandra/commit/cb67bfc1639ded1b6937e7347ad42177ea3f24e3]. Thanks! > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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: Client TOPOLOGY_CHANGE messages have wrong port.
Repository: cassandra Updated Branches: refs/heads/trunk 127cfff26 -> cb67bfc16 Client TOPOLOGY_CHANGE messages have wrong port. Patch by Ariel Weisberg; Reviewed by Greg Bestland for CASSANDRA-14398 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cb67bfc1 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cb67bfc1 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cb67bfc1 Branch: refs/heads/trunk Commit: cb67bfc1639ded1b6937e7347ad42177ea3f24e3 Parents: 127cfff Author: Ariel Weisberg Authored: Wed Apr 18 18:09:17 2018 -0400 Committer: Ariel Weisberg Committed: Thu Apr 19 12:15:27 2018 -0400 -- CHANGES.txt | 1 + .../cassandra/service/StorageService.java | 2 +- .../service/StorageServiceServerTest.java | 26 3 files changed, 28 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb67bfc1/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 6c9e30a..a8dbbba 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Client TOPOLOGY_CHANGE messages have wrong port. (CASSANDRA-14398) * Add ability to load new SSTables from a separate directory (CASSANDRA-6719) * Eliminate background repair and probablistic read_repair_chance table options (CASSANDRA-13910) http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb67bfc1/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 6cbc49a..4f62dd5 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1699,7 +1699,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { try { -InetAddressAndPort address = InetAddressAndPort.getByName(Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS).value); +InetAddressAndPort address = InetAddressAndPort.getByName(Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.NATIVE_ADDRESS_AND_PORT).value); return address.getHostAddress(withPort); } catch (UnknownHostException e) http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb67bfc1/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java -- diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 3884f5a..8c4f5f6 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; import java.io.PrintWriter; +import java.net.InetAddress; import java.util.*; import com.google.common.collect.HashMultimap; @@ -34,6 +35,9 @@ import org.junit.runner.RunWith; import org.apache.cassandra.OrderedJUnit4ClassRunner; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -592,4 +596,26 @@ public class StorageServiceServerTest repairRangeFrom = StorageService.instance.createRepairRangeFrom("2000", "2000"); assert repairRangeFrom.size() == 0; } + +/** + * Test that StorageService.getNativeAddress returns the correct value based on available yaml and gossip state + * @throws Exception + */ +@Test +public void testGetNativeAddress() throws Exception +{ +String internalAddressString = "127.0.0.2:666"; +InetAddressAndPort internalAddress = InetAddressAndPort.getByName(internalAddressString); +Gossiper.instance.addSavedEndpoint(internalAddress); +//Default to using the provided address with the configured port +assertEquals("127.0.0.2:" + DatabaseDescriptor.getNativeTransportPort(), StorageService.instance.getNativeaddress(internalAddress, true)); + +VersionedValue.VersionedValueFactory valueFactory = new VersionedVa
[jira] [Updated] (CASSANDRA-14402) Remove StreamCoordinator.streamExecutor thread pool
[ https://issues.apache.org/jira/browse/CASSANDRA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dinesh Joshi updated CASSANDRA-14402: - Reviewer: Dinesh Joshi > Remove StreamCoordinator.streamExecutor thread pool > --- > > Key: CASSANDRA-14402 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14402 > Project: Cassandra > Issue Type: Improvement > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 4.x > > > {{StreamCoordinator.streamExecutor}} was previously introduced to initiate > stream connections on a separate thread from the session invocation logic. > With CASSANDRA-1229 streaming now uses non-blocking IO, and connection > establishment is asynchronous via netty. Thus, the thread pool in > {{StreamCoordinator}} is unneeded. -- 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-14402) Remove StreamCoordinator.streamExecutor thread pool
[ https://issues.apache.org/jira/browse/CASSANDRA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444294#comment-16444294 ] Jason Brown commented on CASSANDRA-14402: - Patch to remove the thread pool: ||trunk|| |[branch|https://github.com/jasobrown/cassandra/tree/remove-async-stream-establisher]| |[utests & dtests|https://circleci.com/gh/jasobrown/workflows/cassandra/tree/remove-async-stream-establisher]| || The one failed dtest is unrelated. > Remove StreamCoordinator.streamExecutor thread pool > --- > > Key: CASSANDRA-14402 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14402 > Project: Cassandra > Issue Type: Improvement > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 4.x > > > {{StreamCoordinator.streamExecutor}} was previously introduced to initiate > stream connections on a separate thread from the session invocation logic. > With CASSANDRA-1229 streaming now uses non-blocking IO, and connection > establishment is asynchronous via netty. Thus, the thread pool in > {{StreamCoordinator}} is unneeded. -- 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-14402) Remove StreamCoordinator.streamExecutor thread pool
Jason Brown created CASSANDRA-14402: --- Summary: Remove StreamCoordinator.streamExecutor thread pool Key: CASSANDRA-14402 URL: https://issues.apache.org/jira/browse/CASSANDRA-14402 Project: Cassandra Issue Type: Improvement Components: Streaming and Messaging Reporter: Jason Brown Assignee: Jason Brown Fix For: 4.x {{StreamCoordinator.streamExecutor}} was previously introduced to initiate stream connections on a separate thread from the session invocation logic. With CASSANDRA-1229 streaming now uses non-blocking IO, and connection establishment is asynchronous via netty. Thus, the thread pool in {{StreamCoordinator}} is unneeded. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-14398: --- Status: Ready to Commit (was: Patch Available) > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-14398: --- Status: Patch Available (was: In Progress) > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444270#comment-16444270 ] Greg Bestland commented on CASSANDRA-14398: --- +1 works perfectly. Thanks a ton. > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14389) Resolve local address binding in 4.0
[ https://issues.apache.org/jira/browse/CASSANDRA-14389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444266#comment-16444266 ] Dinesh Joshi commented on CASSANDRA-14389: -- I found the issue. When you leave the local side of the socket unbound, the kernel will prefer the IP address that matches the remote IP. Say node1 with IP {{127.0.0.1}} wants to open a connection to node2 with IP {{127.0.0.2}}, the socket would look like {{<127.0.0.2:61002, 127.0.0.2:7000>}} on node1. This seems to confuse the streaming code. Here's how - Say we have three nodes node1, node2 & node3 with IPs {{127.0.0.1, 127.0.0.2, 127.0.0.3}}. node1 has data and node3 is bootstrapping. It requests a stream from node1. So node3 is the `peer` in this case and node1's code execution is described below - * node1 receives the request ({{StreamingInboundHandler#deriveSession}}) and {{StreamResultFuture#initReceivingSide}} creates a new {{StreamResultFuture}} and calls {{attachConnection()}}. At this point it has two sets of IP & Ports from the peer. They are identified by the variable `{{from}}` & expression `{{channel.remoteAddress()}}` a.k.a `{{connecting}}` ). * {{StreamResultFuture#attachConnection calls StreamCoordinator#getOrCreateSessionById}} passing the from IP & {{InetAddressAndPort.getByAddressOverrideDefaults(connecting, from.port)}} (!!!) * The key observation here is `from` is the IP that the peer sent in the `{{StreamMessageHeader}}` while `connecting` is the remote IP of the peer. * {{StreamCoordinator#getOrCreateSessionById}} subsequently calls {{StreamCoordinator#getOrCreateHostData(peer)}}. So we're indexing the {{peerSessions}} by the `{{peer}}` IP address. We also end up creating a `{{StreamSession}}` in the process. * During `{{StreamSession}}` creation, we end up passing the `{{peer}}` and `{{connecting}}` IPs. We use the `connecting` IP to establish the outbound connection to the peer. ({{NettyStreamingMessageSender}} is now connected to `{{connecting}}` IP on port {{7000}}). In our case, since we leave the local side of the socket unbound, although the `{{peer}}` correctly sets its IP to {{127.0.0.3}} in the {{StreamMessageHeader}}, the {{localAddress}} that the kernel chooses for it is {{127.0.0.1}}. On the inbound node1 seems to think that the `peer` is {{127.0.0.3}} however the connecting IP address should be {{127.0.0.1}}. Therefore, it prefers that IP when trying to establish an outbound session. In fact it establishes a connection to itself leading to the `{{Unknown peer requested: 127.0.0.1:7000}}` exception. Note that along the way it actually drops the ephemeral port and instead uses the port returned by {{MessagingService#portFor}}. Streaming code seems to rely on the perceived remote IP address of the host rather than the one that is set in the message header. I am not sure if preferring the IP address set in the header is the correct approach. > Resolve local address binding in 4.0 > > > Key: CASSANDRA-14389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14389 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 4.x > > > CASSANDRA-8457/CASSANDRA-12229 introduced a regression against > CASSANDRA-12673. This was discovered with CASSANDRA-14362 and moved here for > resolution independent of that ticket. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444264#comment-16444264 ] Greg Bestland commented on CASSANDRA-14398: --- Accidentally marked as ready to commit. Still reviewing and testing will be done shortly. > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anonymous updated CASSANDRA-14398: -- Status: In Progress (was: Ready to Commit) > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anonymous updated CASSANDRA-14398: -- Status: Ready to Commit (was: Patch Available) > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14398) Client TOPOLOGY_CHANGE messages have wrong port.
[ https://issues.apache.org/jira/browse/CASSANDRA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Adam Holmberg updated CASSANDRA-14398: -- Labels: client-impacting (was: ) > Client TOPOLOGY_CHANGE messages have wrong port. > - > > Key: CASSANDRA-14398 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14398 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Greg Bestland >Assignee: Ariel Weisberg >Priority: Blocker > Labels: client-impacting > Fix For: 4.0 > > > Summary: > TOPOLOGY_CHANGE events that are recieved by the client(Driver), with C* 4.0 > (Trunk). Contain the storage port (7000) rather than the native port (9042). > I believe this is due to changes stuck in for CASSANDRA-7544. > I was able to track it down to this specific call here. > > [https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1703] > We need an accurate port number from this Topology change event, otherwise we > won't be able to correlate node up event against the system.peers_v2 table > accurately. > C* version I'm testing against : 4.0.x (trunk) > Steps to reproduce: > 1. create a single node, cluster, in this case I'm using ccm. > {code:java} > ccm create 400-1 --install-dir=/Users/gregbestland/git/cassandra > ccm populate -n 1 > ccm start > {code} > 2. Connect the java driver, and check metadata. > see that there is one node with the correct native port (9042). > 3. Add a brand new node: > {code:java} > ccm add node2 -i 127.0.0.2 > ccm node2 start > {code} > 4. Incoming topology change message to client will have the storage port, > rather then the native port in the message. -- 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-14286) IndexOutOfBoundsException with SELECT JSON using IN and ORDER BY
[ https://issues.apache.org/jira/browse/CASSANDRA-14286?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Lerer updated CASSANDRA-14286: --- Fix Version/s: 3.11.3 3.0.17 2.2.13 4.0 > IndexOutOfBoundsException with SELECT JSON using IN and ORDER BY > > > Key: CASSANDRA-14286 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14286 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: Kubernetes cluster using cassandra:3.11.1 Docker image. >Reporter: Szymon Acedański >Assignee: Francisco Fernandez >Priority: Major > Fix For: 4.0, 2.2.13, 3.0.17, 3.11.3 > > Attachments: orderbug-traceback.txt > > > When running the following code: > {code} > public class CassandraJsonOrderingBug { > public static void main(String[] args) { > Session session = CassandraFactory.getSession(); > session.execute("CREATE TABLE thebug ( PRIMARY KEY (a, b), a INT, b > INT)"); > try { > session.execute("INSERT INTO thebug (a, b) VALUES (20, 30)"); > session.execute("INSERT INTO thebug (a, b) VALUES (100, 200)"); > Statement statement = new SimpleStatement("SELECT JSON a, b FROM > thebug WHERE a IN (20, 100) ORDER BY b"); > statement.setFetchSize(Integer.MAX_VALUE); > for (Row w: session.execute(statement)) { > System.out.println(w.toString()); > } > } finally { > session.execute("DROP TABLE thebug"); > } > } > } > {code} > The following exception is thrown server-side: > {noformat} > java.lang.IndexOutOfBoundsException: Index: 1, Size: 1 > at java.util.Collections$SingletonList.get(Collections.java:4815) > ~[na:1.8.0_151] > at > org.apache.cassandra.cql3.statements.SelectStatement$SingleColumnComparator.compare(SelectStatement.java:1297) > ~[apache-cassandra-3.11.1.jar:3.11.1] > at > org.apache.cassandra.cql3.statements.SelectStatement$SingleColumnComparator.compare(SelectStatement.java:1284) > ~[apache-cassandra-3.11.1.jar:3.11.1] > at java.util.TimSort.countRunAndMakeAscending(TimSort.java:355) > ~[na:1.8.0_151] > at java.util.TimSort.sort(TimSort.java:220) ~[na:1.8.0_151] > at java.util.Arrays.sort(Arrays.java:1512) ~[na:1.8.0_151] > at java.util.ArrayList.sort(ArrayList.java:1460) ~[na:1.8.0_151] > at java.util.Collections.sort(Collections.java:175) ~[na:1.8.0_151] > {noformat} > (full traceback attached) > The accessed index is the index of the sorted column in the SELECT JSON > fields list. > Similarly, if the select clause is changed to > SELECT JSON b, a FROM thebug WHERE a IN (20, 100) ORDER BY b > then the query finishes, but the output is sorted incorrectly (by textual > JSON representation): > {noformat} > Row[{"b": 200, "a": 100}] > Row[{"b": 30, "a": 20}] > {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] [Updated] (CASSANDRA-14401) Attempted serializing to buffer exceeded maximum of 65535 bytes
[ https://issues.apache.org/jira/browse/CASSANDRA-14401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Artem Rokhin updated CASSANDRA-14401: - Description: Cassandra version: 3.11.2 3 nodes cluster The following exception appears on all 3 nodes and after awhile cluster becomes unreposnsive {code} java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] {code} was: Cassandra version: 3.11.2 3 nodes cluster The following exception appears on 3 nodes and after awhile cluster becomes unreposnsive {code} java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] {code} > Attempted serializing to buffer exceeded maximum of 65535 bytes > > > Key: CASSANDRA-14401 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14401 > Project: Cassandra > Issue Type: Bug >Reporter: Artem Rokhin >Priority: Major > > Cassandra version: 3.11.2 > 3 nodes cluster > The following exception appears on all 3 nodes and after awhile cluster > becomes unreposnsive > > {code} > java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of > 65535 bytes: 67661 > at > org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeInternal(Outboun
[jira] [Updated] (CASSANDRA-14401) Attempted serializing to buffer exceeded maximum of 65535 bytes
[ https://issues.apache.org/jira/browse/CASSANDRA-14401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Artem Rokhin updated CASSANDRA-14401: - Description: Cassandra version: 3.11.2 3 nodes cluster The following exception appears on 3 nodes and after awhile cluster becomes unreposnsive {code} java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] {code} was: Cassandra version: 3.11.2 3 nodes cluster The following exception appears on 3 nodes and after awhile cluster becomes unreposnsive *** java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] *** > Attempted serializing to buffer exceeded maximum of 65535 bytes > > > Key: CASSANDRA-14401 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14401 > Project: Cassandra > Issue Type: Bug >Reporter: Artem Rokhin >Priority: Major > > Cassandra version: 3.11.2 > 3 nodes cluster > The following exception appears on 3 nodes and after awhile cluster becomes > unreposnsive > > {code} > java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of > 65535 bytes: 67661 > at > org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnectio
[jira] [Updated] (CASSANDRA-14401) Attempted serializing to buffer exceeded maximum of 65535 bytes
[ https://issues.apache.org/jira/browse/CASSANDRA-14401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Artem Rokhin updated CASSANDRA-14401: - Description: Cassandra version: 3.11.2 3 nodes cluster The following exception appears on 3 nodes and after awhile cluster becomes unreposnsive *** java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] *** was: Cassandra version: 3.11.2 3 nodes cluster The following exception appears on 3 nodes and after awhile cluster becomes unreposnsive ``` java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] ``` > Attempted serializing to buffer exceeded maximum of 65535 bytes > > > Key: CASSANDRA-14401 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14401 > Project: Cassandra > Issue Type: Bug >Reporter: Artem Rokhin >Priority: Major > > Cassandra version: 3.11.2 > 3 nodes cluster > The following exception appears on 3 nodes and after awhile cluster becomes > unreposnsive > > *** > java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of > 65535 bytes: 67661 > at > org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) > ~[apache-cassandra-3.11.2.jar:3.11.2] > at > org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:3
[jira] [Created] (CASSANDRA-14401) Attempted serializing to buffer exceeded maximum of 65535 bytes
Artem Rokhin created CASSANDRA-14401: Summary: Attempted serializing to buffer exceeded maximum of 65535 bytes Key: CASSANDRA-14401 URL: https://issues.apache.org/jira/browse/CASSANDRA-14401 Project: Cassandra Issue Type: Bug Reporter: Artem Rokhin Cassandra version: 3.11.2 3 nodes cluster The following exception appears on 3 nodes and after awhile cluster becomes unreposnsive ``` java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535 bytes: 67661 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Expression$Serializer.serialize(RowFilter.java:547) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.filter.RowFilter$Serializer.serialize(RowFilter.java:1143) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:726) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.db.ReadCommand$Serializer.serialize(ReadCommand.java:683) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.io.ForwardingVersionedSerializer.serialize(ForwardingVersionedSerializer.java:45) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120) ~[apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:385) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:337) [apache-cassandra-3.11.2.jar:3.11.2] at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263) [apache-cassandra-3.11.2.jar:3.11.2] ``` -- 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-14385) Fix Some Potential NPE
[ https://issues.apache.org/jira/browse/CASSANDRA-14385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444078#comment-16444078 ] ASF GitHub Bot commented on CASSANDRA-14385: Github user brettKK commented on the issue: https://github.com/apache/cassandra/pull/219 https://issues.apache.org/jira/browse/CASSANDRA-14385 > Fix Some Potential NPE > --- > > Key: CASSANDRA-14385 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14385 > Project: Cassandra > Issue Type: Bug >Reporter: lujie >Priority: Major > Attachments: CA-14385_1.patch > > > We have developed a static analysis tool > [NPEDetector|https://github.com/lujiefsi/NPEDetector] to find some potential > NPE. Our analysis shows that some callees may return null in corner case(e.g. > node crash , IO exception), some of their callers have _!=null_ check but > some do not have. In this issue we post a patch which can add !=null based > on existed !=null check. For example: > Calle Schema#getView may return null: > {code:java} > public ViewMetadata getView(String keyspaceName, String viewName) > { > assert keyspaceName != null; > KeyspaceMetadata ksm = keyspaces.getNullable(keyspaceName); > return (ksm == null) ? null : ksm.views.getNullable(viewName);//may > return null > } > {code} > it have 4 callers, 3 of them have !=null check, like its caller > MigrationManager#announceViewDrop have !=null check() > {code:java} > public static void announceViewDrop(String ksName, String viewName, boolean > announceLocally) throws ConfigurationException > { >ViewMetadata view = Schema.instance.getView(ksName, viewName); > if (view == null)//null pointer checker > throw new ConfigurationException(String.format("Cannot drop non > existing materialized view '%s' in keyspace '%s'.", viewName, ksName)); >KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName); >logger.info("Drop table '{}/{}'", view.keyspace, view.name); >announce(SchemaKeyspace.makeDropViewMutation(ksm, view, > FBUtilities.timestampMicros()), announceLocally); > } > {code} > but caller MigrationManager#announceMigration does not have > We add !=null check based on MigrationManager#announceViewDrop: > {code:java} > if (current == null) > throw new InvalidRequestException("There is no materialized view in > keyspace " + keyspace()); > {code} > But due to we are not very familiar with CASSANDRA, hope some expert can > review it. > Thanks > -- 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-14385) Fix Some Potential NPE
[ https://issues.apache.org/jira/browse/CASSANDRA-14385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444075#comment-16444075 ] ASF GitHub Bot commented on CASSANDRA-14385: GitHub user brettKK opened a pull request: https://github.com/apache/cassandra/pull/219 CASSANDRA-14385 @LJ1043041006 found a potential NPE in cassandra --- We have developed a static analysis tool [NPEDetector](https://github.com/lujiefsi/NPEDetector) to find some potential NPE. Our analysis shows that some callees may return null in corner case(e.g. node crash , IO exception), some of their callers have !=null check but some do not have. In this issue we post a patch which can add !=null based on existed !=null check. For example: Calle Schema#getView may return null: ``` public ViewMetadata getView(String keyspaceName, String viewName) { assert keyspaceName != null; KeyspaceMetadata ksm = keyspaces.getNullable(keyspaceName); return (ksm == null) ? null : ksm.views.getNullable(viewName);//may return null } ``` it have 4 callers, 3 of them have !=null check, like its caller MigrationManager#announceViewDrop have !=null check() ``` public static void announceViewDrop(String ksName, String viewName, boolean announceLocally) throws ConfigurationException { ViewMetadata view = Schema.instance.getView(ksName, viewName); if (view == null)//null pointer checker throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", viewName, ksName)); KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName); logger.info("Drop table '{}/{}'", view.keyspace, view.name); announce(SchemaKeyspace.makeDropViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally); } ``` but caller MigrationManager#announceMigration does not have You can merge this pull request into a Git repository by running: $ git pull https://github.com/brettKK/cassandra CASSANDRA-14385 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/cassandra/pull/219.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #219 commit 61b4ed4afbd80fbde24bef3cedb9e8d3b06f788c Author: brettkk <1099446733@...> Date: 2018-04-19T13:49:49Z fix CASSANDRA-14385 > Fix Some Potential NPE > --- > > Key: CASSANDRA-14385 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14385 > Project: Cassandra > Issue Type: Bug >Reporter: lujie >Priority: Major > Attachments: CA-14385_1.patch > > > We have developed a static analysis tool > [NPEDetector|https://github.com/lujiefsi/NPEDetector] to find some potential > NPE. Our analysis shows that some callees may return null in corner case(e.g. > node crash , IO exception), some of their callers have _!=null_ check but > some do not have. In this issue we post a patch which can add !=null based > on existed !=null check. For example: > Calle Schema#getView may return null: > {code:java} > public ViewMetadata getView(String keyspaceName, String viewName) > { > assert keyspaceName != null; > KeyspaceMetadata ksm = keyspaces.getNullable(keyspaceName); > return (ksm == null) ? null : ksm.views.getNullable(viewName);//may > return null > } > {code} > it have 4 callers, 3 of them have !=null check, like its caller > MigrationManager#announceViewDrop have !=null check() > {code:java} > public static void announceViewDrop(String ksName, String viewName, boolean > announceLocally) throws ConfigurationException > { >ViewMetadata view = Schema.instance.getView(ksName, viewName); > if (view == null)//null pointer checker > throw new ConfigurationException(String.format("Cannot drop non > existing materialized view '%s' in keyspace '%s'.", viewName, ksName)); >KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName); >logger.info("Drop table '{}/{}'", view.keyspace, view.name); >announce(SchemaKeyspace.makeDropViewMutation(ksm, view, > FBUtilities.timestampMicros()), announceLocally); > } > {code} > but caller MigrationManager#announceMigration does not have > We add !=null check based on MigrationManager#announceViewDrop: > {code:java} > if (current == null) > throw new InvalidRequestException("There is no materialized view in > keyspace " + keyspace()); > {code} > But due to we are not very familiar with CASSANDRA, hope some expert can > review it. > Thanks > -- This message was sent by Atlassian JIRA (v7.6.3#76005) --
[jira] [Updated] (CASSANDRA-13304) Add checksumming to the native protocol
[ https://issues.apache.org/jira/browse/CASSANDRA-13304?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anonymous updated CASSANDRA-13304: -- Status: In Progress (was: Awaiting Feedback) > Add checksumming to the native protocol > --- > > Key: CASSANDRA-13304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Michael Kjellman >Assignee: Michael Kjellman >Priority: Blocker > Labels: client-impacting > Fix For: 4.x > > Attachments: 13304_v1.diff, boxplot-read-throughput.png, > boxplot-write-throughput.png > > > The native binary transport implementation doesn't include checksums. This > makes it highly susceptible to silently inserting corrupted data either due > to hardware issues causing bit flips on the sender/client side, C*/receiver > side, or network in between. > Attaching an implementation that makes checksum'ing mandatory (assuming both > client and server know about a protocol version that supports checksums) -- > and also adds checksumming to clients that request compression. > The serialized format looks something like this: > {noformat} > * 1 1 1 1 1 1 1 1 1 1 2 2 2 2 2 2 2 2 2 2 3 3 > * 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Number of Compressed Chunks | Compressed Length (e1)/ > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * / Compressed Length cont. (e1) |Uncompressed Length (e1) / > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Uncompressed Length cont. (e1)| CRC32 Checksum of Lengths (e1)| > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Checksum of Lengths cont. (e1)|Compressed Bytes (e1)+// > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | CRC32 Checksum (e1) || > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * |Compressed Length (e2) | > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Uncompressed Length (e2)| > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * |CRC32 Checksum of Lengths (e2) | > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Compressed Bytes (e2) +// > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | CRC32 Checksum (e2) || > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * |Compressed Length (en) | > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Uncompressed Length (en)| > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * |CRC32 Checksum of Lengths (en) | > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | Compressed Bytes (en) +// > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > * | CRC32 Checksum (en) || > * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ > {noformat} > The first pass here adds checksums only to the actual contents of the frame > body itself (and doesn't actually checksum lengths and headers). While it > would be great to fully add checksuming across the entire protocol, the > proposed implementation will ensure we at least catch corrupted data and > likely protect ourselves pretty well anyways. > I didn't go to the trouble of implementing a Snappy Checksum'ed Compressor > implementation as it's been deprecated for a while -- is really slow and > crappy compared to LZ4 -- and we should do everything in our power to make > sure no one in the community is still using it. I left it in (for obvious > backwards compatibility aspects) old for clients that don't know about the > new protocol. > The current protocol has a 256MB (max) frame body -- where the serialized > contents are simply written in to the frame body. > If the client sends a compression option in the startup, we will install a > FrameCompressor inline. Unfortunately, we went with a decision to treat the > frame body separately from the header bits etc in a given message. So, > instead we put a compressor implementation in the options and then if it's > not null, we push the serialized bytes for the frame bo
[jira] [Commented] (CASSANDRA-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16443712#comment-16443712 ] Marcus Eriksson commented on CASSANDRA-14400: - [~spo...@gmail.com] yeah that used to be true, but from that ticket: bq. The original reason for doing this was to prevent anti-compaction, but since anti-compaction is preformed at the start of an incremental repair, I don't see any reason to just promote the incrementally repaired subrange when it's finished. > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- 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-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16443711#comment-16443711 ] Marcus Eriksson edited comment on CASSANDRA-14400 at 4/19/18 8:12 AM: -- SSTables get marked as repaired when they move from 'pending' to 'repaired': https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java#L400-L453 and that sstable is marked as pending - it might stay in pending until a compaction has run was (Author: krummas): SSTables get marked as repaired when they move from 'pending' to 'repaired': https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java#L400-L453 and that sstable is marked as pending > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mai
[jira] [Commented] (CASSANDRA-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16443711#comment-16443711 ] Marcus Eriksson commented on CASSANDRA-14400: - SSTables get marked as repaired when they move from 'pending' to 'repaired': https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java#L400-L453 and that sstable is marked as pending > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- 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-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16443660#comment-16443660 ] Stefan Podkowinski edited comment on CASSANDRA-14400 at 4/19/18 7:49 AM: - The main reason for not marking sstables as repaired on sub-range repairs, was to avoid anti-compaction. Creating lots of small tables for small repair ranges will be inefficient and should also not be necessary, as incremental repairs should be run often enough to keep the unrepaired set reasonably small. See also CASSANDRA-13818. was (Author: spo...@gmail.com): The main reason for not marking sstables as repaired on sub-range repairs, was to avoid anti-compaction. Creating lots of small tables for small repair ranges will be inefficient and should also not be necessary, as incremental repairs should be run often enough to keep the unrepaired set reasonably small. > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- This message was sent by A
[jira] [Commented] (CASSANDRA-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16443660#comment-16443660 ] Stefan Podkowinski commented on CASSANDRA-14400: The main reason for not marking sstables as repaired on sub-range repairs, was to avoid anti-compaction. Creating lots of small tables for small repair ranges will be inefficient and should also not be necessary, as incremental repairs should be run often enough to keep the unrepaired set reasonably small. > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- 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-14400) Subrange repair doesn't always mark as repaired
[ https://issues.apache.org/jira/browse/CASSANDRA-14400?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Podkowinski updated CASSANDRA-14400: --- Since Version: 4.0 Fix Version/s: (was: 4.0) > Subrange repair doesn't always mark as repaired > --- > > Key: CASSANDRA-14400 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14400 > Project: Cassandra > Issue Type: Bug >Reporter: Kurt Greaves >Priority: Major > > So was just messing around with subrange repair on trunk and found that if I > generated an SSTable with a single token and then tried to repair that > SSTable using subrange repairs it wouldn't get marked as repaired. > > Before repair: > {code:java} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > Repair command: > {code} > ccm node1 nodetool "repair -st -9223362383595311663 -et -9223362383595311661 > aoeu" > [2018-04-19 05:44:42,806] Starting repair command #7 > (c23f76c0-4394-11e8-8f20-3b8ee110d005), repairing keyspace aoeu with repair > options (parallelism: parallel, primary range: false, incremental: true, job > threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], previewKind: > NONE, # of ranges: 1, pull repair: false, force repair: false, optimise > streams: false) > [2018-04-19 05:44:42,843] Repair session c242d220-4394-11e8-8f20-3b8ee110d005 > for range [(-9223362383595311663,-9223362383595311661]] finished (progress: > 20%) > [2018-04-19 05:44:43,139] Repair completed successfully > [2018-04-19 05:44:43,140] Repair command #7 finished in 0 seconds > {code} > After repair SSTable hasn't changed and sstablemetadata outputs: > {code} > First token: -9223362383595311662 (derphead4471291) > Last token: -9223362383595311662 (derphead4471291) > Repaired at: 0 > Pending repair: 862395e0-4394-11e8-8f20-3b8ee110d005 > {code} > And parent_repair_history states that the repair is complete/range was > successful: > {code} > select * from system_distributed.parent_repair_history where > parent_id=862395e0-4394-11e8-8f20-3b8ee110d005 ; > parent_id| columnfamily_names | > exception_message | exception_stacktrace | finished_at | > keyspace_name | options > > > | requested_ranges > | started_at | successful_ranges > --++---+--+-+---++-+-+- > 862395e0-4394-11e8-8f20-3b8ee110d005 | {'aoeu'} | > null | null | 2018-04-19 05:43:14.578000+ | aoeu > | {'dataCenters': '', 'forceRepair': 'false', 'hosts': '', 'incremental': > 'true', 'jobThreads': '1', 'optimiseStreams': 'false', 'parallelism': > 'parallel', 'previewKind': 'NONE', 'primaryRange': 'false', 'pullRepair': > 'false', 'sub_range_repair': 'true', 'trace': 'false'} | > {'(-9223362383595311663,-9223362383595311661]'} | 2018-04-19 > 05:43:01.952000+ | {'(-9223362383595311663,-9223362383595311661]'} > {code} > Subrange repairs seem to work fine over large ranges and set {{Repaired at}} > as expected, but I haven't figured out why it works for a large range versus > a small range so far. -- 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