[jira] [Commented] (CASSANDRA-8180) Optimize disk seek using min/max column name meta data when the LIMIT clause is used
[ https://issues.apache.org/jira/browse/CASSANDRA-8180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109971#comment-15109971 ] Stefania commented on CASSANDRA-8180: - Thank you for the review. bq. updateClusteringValues() doesn't produce what you need, as it is not just TOP and BOTTOM that are mishandled, any incomplete prefix will also be. What you need is an actual min/maxClusteringPrefix which is different from the min/max of each component separately. I agree that we need a full clustering prefix. What I don't understand is how things like {{shouldInclude()}} in {{ClusteringIndexNamesFilter}} or {{ClusteringIndexSliceFilter}} work. What is an example of any other incomplete prefix and do we have a gap in the tests then? bq. I don't know if it is possible to append the data you need at the end of the stats component and have earlier versions happily ignore that data. There are 4 components that we write in the same file: VALIDATION, STATS, COMPACTION and HEADER so we can't simply keep on reading till the end of the file. However we write a TOC with the position of each component. So it should be possible but it would require changes to {{MetadataSerializer.deserialize()}} and the signature of {{IMetadataComponentSerializer.deserialize()}}, which should receive the total size to work out if there is more stuff to read at the end. I guess we can go for it. bq. I personally would prefer to not modify the behaviour of MergeIterator and keep it doing one simple thing, but this approach does have its charm. The changes to MergeIterator are mostly in the candidate and really minimal, the actual algorithm is unchanged. However if you do have a less invasive approach in mind I'm eager to hear it. bq. An empty row will not work correctly as a lower bound. It does not sort as needed with respect to tombstone bounds, which should also be included in the test (more specifically, one that adds a row, flushes, deletes same row, flushes again, then checks if it resurfaces-- I believe this would break with the current code). Thanks, I'll add this test. bq. Use a RangeTombstoneBound with DeletionTime.LIVE as the deletion time and a bound obtained by RangeTombstone.Bound.inclusiveOpen, which should do the right thing in both directions. I'm not sure what you mean, do this for the test or the fix? I'm sure I'll work it out when I write the test though. bq. IMergeIterator.LowerBound is cryptic, rename it to IteratorWithLowerBound to be explicit about its purpose. OK bq. The choice to set rowIndexLowerBound in partitionLevelDeletion() appears very arbitrary and fragile. What is the reason to do it separately from globalLowerBound? In fact, why have two separate bounds instead of one, set from the most precise information that is available at construction time? The global lower bound is free, since it is available in the metadata. The index lower bound is more accurate but it requires seeking the index file. Calling {{super.partitionLevelDeletion()}} also involves initializing the iterator and accessing the data file (AbstractSSTableIterator constructor). So we decided to use this more accurate bound only when we really have to access the index anyway, that is when partitionLevelDeletion() is called and there are tombstones. See [this comment|https://issues.apache.org/jira/browse/CASSANDRA-8180?focusedCommentId=14388301=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14388301] above. I hope to resume this in the next few days. > Optimize disk seek using min/max column name meta data when the LIMIT clause > is used > > > Key: CASSANDRA-8180 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8180 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths > Environment: Cassandra 2.0.10 >Reporter: DOAN DuyHai >Assignee: Stefania >Priority: Minor > Fix For: 3.x > > Attachments: 8180_001.yaml, 8180_002.yaml > > > I was working on an example of sensor data table (timeseries) and face a use > case where C* does not optimize read on disk. > {code} > cqlsh:test> CREATE TABLE test(id int, col int, val text, PRIMARY KEY(id,col)) > WITH CLUSTERING ORDER BY (col DESC); > cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 10, '10'); > ... > >nodetool flush test test > ... > cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 20, '20'); > ... > >nodetool flush test test > ... > cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 30, '30'); > ... > >nodetool flush test test > {code} > After that, I activate request tracing: > {code} > cqlsh:test> SELECT * FROM test WHERE id=1 LIMIT 1; > activity
[jira] [Commented] (CASSANDRA-9666) Provide an alternative to DTCS
[ https://issues.apache.org/jira/browse/CASSANDRA-9666?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15110055#comment-15110055 ] Christian Winther commented on CASSANDRA-9666: -- I'm sad by not getting it into C* I've also done the DTCS -> TWCS migration on my data-set, and the number of hours i spend on cassandra maintenance and monitoring per week has dropped from 5-10 to 1.. it just works, no crazy compactions, no 100GB SSTables, stable performance in read/write, not running amok with sstables during 2-3d compactions it's just a way better experience as a C* newbie than DTCS - and quite easier to understand and tweak as well. C* would be less awesome in my mind if TWCS was not included > Provide an alternative to DTCS > -- > > Key: CASSANDRA-9666 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9666 > Project: Cassandra > Issue Type: Improvement >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa > Fix For: 2.1.x, 2.2.x > > Attachments: dtcs-twcs-io.png, dtcs-twcs-load.png > > > DTCS is great for time series data, but it comes with caveats that make it > difficult to use in production (typical operator behaviors such as bootstrap, > removenode, and repair have MAJOR caveats as they relate to > max_sstable_age_days, and hints/read repair break the selection algorithm). > I'm proposing an alternative, TimeWindowCompactionStrategy, that sacrifices > the tiered nature of DTCS in order to address some of DTCS' operational > shortcomings. I believe it is necessary to propose an alternative rather than > simply adjusting DTCS, because it fundamentally removes the tiered nature in > order to remove the parameter max_sstable_age_days - the result is very very > different, even if it is heavily inspired by DTCS. > Specifically, rather than creating a number of windows of ever increasing > sizes, this strategy allows an operator to choose the window size, compact > with STCS within the first window of that size, and aggressive compact down > to a single sstable once that window is no longer current. The window size is > a combination of unit (minutes, hours, days) and size (1, etc), such that an > operator can expect all data using a block of that size to be compacted > together (that is, if your unit is hours, and size is 6, you will create > roughly 4 sstables per day, each one containing roughly 6 hours of data). > The result addresses a number of the problems with > DateTieredCompactionStrategy: > - At the present time, DTCS’s first window is compacted using an unusual > selection criteria, which prefers files with earlier timestamps, but ignores > sizes. In TimeWindowCompactionStrategy, the first window data will be > compacted with the well tested, fast, reliable STCS. All STCS options can be > passed to TimeWindowCompactionStrategy to configure the first window’s > compaction behavior. > - HintedHandoff may put old data in new sstables, but it will have little > impact other than slightly reduced efficiency (sstables will cover a wider > range, but the old timestamps will not impact sstable selection criteria > during compaction) > - ReadRepair may put old data in new sstables, but it will have little impact > other than slightly reduced efficiency (sstables will cover a wider range, > but the old timestamps will not impact sstable selection criteria during > compaction) > - Small, old sstables resulting from streams of any kind will be swiftly and > aggressively compacted with the other sstables matching their similar > maxTimestamp, without causing sstables in neighboring windows to grow in size. > - The configuration options are explicit and straightforward - the tuning > parameters leave little room for error. The window is set in common, easily > understandable terms such as “12 hours”, “1 Day”, “30 days”. The > minute/hour/day options are granular enough for users keeping data for hours, > and users keeping data for years. > - There is no explicitly configurable max sstable age, though sstables will > naturally stop compacting once new data is written in that window. > - Streaming operations can create sstables with old timestamps, and they'll > naturally be joined together with sstables in the same time bucket. This is > true for bootstrap/repair/sstableloader/removenode. > - It remains true that if old data and new data is written into the memtable > at the same time, the resulting sstables will be treated as if they were new > sstables, however, that no longer negatively impacts the compaction > strategy’s selection criteria for older windows. > Patch provided for : > - 2.1: https://github.com/jeffjirsa/cassandra/commits/twcs-2.1 > - 2.2: https://github.com/jeffjirsa/cassandra/commits/twcs-2.2 > - trunk (post-8099):
[jira] [Commented] (CASSANDRA-10870) pushed_notifications_test.py:TestPushedNotifications.restart_node_test flapping on C* 2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-10870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109870#comment-15109870 ] Stefania commented on CASSANDRA-10870: -- The test checks that each time node 2 restarts, node 1 sends us exactly 3 notifications in this order: DOWN, UP and NEW_NODE, with the correct IP address. The failures under JDK 8 and JDK 7 are different, and they are both not limited to 2.1, they also happen on 2.2 or 3.0. I suspect 3.2+ too. On JDK 8, example [here|http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/174/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/], the order of {{UP}} and {{NEW_NODE}} notifications is swapped. {code} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'DOWN', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'NEW_NODE', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'UP', 'address': ('127.0.0.2', 9042)} {code} An example for 2.2 is [here|http://cassci.datastax.com/job/cassandra-2.2_dtest_jdk8/156/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/]: {code} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'UP', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'NEW_NODE', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'DOWN', 'address': ('127.0.0.2', 9042)} {code} On JDK 7, example [here|http://cassci.datastax.com/job/cassandra-2.1_dtest/385/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/], we received an extra {{NEW_NODE}} notification: {code} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'NEW_NODE', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'DOWN', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'UP', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'NEW_NODE', 'address': ('127.0.0.2', 9042)} {code} This happened also on 3.0, except the duplicated notifications is {{UP}}, example [here|http://cassci.datastax.com/job/cassandra-3.0_dtest/508/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/]: {code} dtest: DEBUG: Restarting second node... dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'UP', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'DOWN', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Waiting for notifications from 127.0.0.1 dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'UP', 'address': ('127.0.0.2', 9042)} dtest: DEBUG: Source 127.0.0.1 sent {'change_type': u'NEW_NODE', 'address': ('127.0.0.2', 9042)} {code} I'd say there is a chance we might be seeing the previous notifications of when the node starts for the first time during the cluster start-up. If this is the case, it might be enough to add a pause before creating the waiter or - better - only start node1, then start node2 and wait for the 3 notifications, then enter the loop. If this does not fix it, then we really have an issue in production code and you can assign the ticket to me. In fact, I can try to fix the test as well if you want me to, just assign the ticket to me if that's the case. > pushed_notifications_test.py:TestPushedNotifications.restart_node_test > flapping on C* 2.1 > - > > Key: CASSANDRA-10870 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10870 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: DS Test Eng > Fix For: 2.1.x > > > This test flaps on CassCI on 2.1. [~aboudreault] Do I remember correctly that > you did some work on these tests in the past few months? If so, could you > have a look and see if there's some assumption the test makes that don't hold > for 2.1? > Oddly, it fails frequently under JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ > but less frequently on JDK7: > http://cassci.datastax.com/job/cassandra-2.1_dtest/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10661) Integrate SASI to Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-10661?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109984#comment-15109984 ] Pavel Yaskevich commented on CASSANDRA-10661: - I've pushed initial version of the clustering support for SASI into sasi-3.2-integration branch, it's not as well performing yet as it can be but is still an improvement over ClusteringColumnIndex because index intersection is used instead of filtering (although intersection currently only done on the partition key instead of partition key + clustering), so if anybody wants they can start testing it. > Integrate SASI to Cassandra > --- > > Key: CASSANDRA-10661 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10661 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths >Reporter: Pavel Yaskevich >Assignee: Pavel Yaskevich > Labels: sasi > Fix For: 3.x > > > We have recently released new secondary index engine > (https://github.com/xedin/sasi) build using SecondaryIndex API, there are > still couple of things to work out regarding 3.x since it's currently > targeted on 2.0 released. I want to make this an umbrella issue to all of the > things related to integration of SASI, which are also tracked in > [sasi_issues|https://github.com/xedin/sasi/issues], into mainline Cassandra > 3.x release. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11053) COPY FROM on large datasets: fix progress report and debug performance
Stefania created CASSANDRA-11053: Summary: COPY FROM on large datasets: fix progress report and debug performance Key: CASSANDRA-11053 URL: https://issues.apache.org/jira/browse/CASSANDRA-11053 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Stefania Assignee: Stefania Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x Attachments: copy_from_large_benchmark.txt Running COPY from on a large dataset (20G divided in 20M records) revealed two issues: * The progress report is incorrect, it is very slow until almost the end of the test at which point it catches up extremely quickly. * The performance in rows per second is similar to running smaller tests with a smaller cluster locally (approx 35,000 rows per second). As a comparison, cassandra-stress manages 50,000 rows per second under the same set-up, therefore resulting 1.5 times faster. See attached file _copy_from_large_benchmark.txt_ for the benchmark details. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11030) utf-8 characters incorrectly displayed/inserted on cqlsh on Windows
[ https://issues.apache.org/jira/browse/CASSANDRA-11030?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109933#comment-15109933 ] Stefania commented on CASSANDRA-11030: -- Thank you so much for checking! > utf-8 characters incorrectly displayed/inserted on cqlsh on Windows > --- > > Key: CASSANDRA-11030 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11030 > Project: Cassandra > Issue Type: Bug >Reporter: Paulo Motta >Assignee: Paulo Motta >Priority: Minor > Labels: cqlsh, windows > > {noformat} > C:\Users\Paulo\Repositories\cassandra [2.2-10948 +6 ~1 -0 !]> .\bin\cqlsh.bat > --encoding utf-8 > Connected to test at 127.0.0.1:9042. > [cqlsh 5.0.1 | Cassandra 2.2.4-SNAPSHOT | CQL spec 3.3.1 | Native protocol v4] > Use HELP for help. > cqlsh> INSERT INTO bla.test (bla ) VALUES ('não') ; > cqlsh> select * from bla.test; > bla > - > n?o > (1 rows) > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9666) Provide an alternative to DTCS
[ https://issues.apache.org/jira/browse/CASSANDRA-9666?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15110125#comment-15110125 ] Marcus Eriksson commented on CASSANDRA-9666: Well, with CASSANDRA-10280 in, DTCS should perform equally good, if not, we should of course merge TWCS. > Provide an alternative to DTCS > -- > > Key: CASSANDRA-9666 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9666 > Project: Cassandra > Issue Type: Improvement >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa > Fix For: 2.1.x, 2.2.x > > Attachments: dtcs-twcs-io.png, dtcs-twcs-load.png > > > DTCS is great for time series data, but it comes with caveats that make it > difficult to use in production (typical operator behaviors such as bootstrap, > removenode, and repair have MAJOR caveats as they relate to > max_sstable_age_days, and hints/read repair break the selection algorithm). > I'm proposing an alternative, TimeWindowCompactionStrategy, that sacrifices > the tiered nature of DTCS in order to address some of DTCS' operational > shortcomings. I believe it is necessary to propose an alternative rather than > simply adjusting DTCS, because it fundamentally removes the tiered nature in > order to remove the parameter max_sstable_age_days - the result is very very > different, even if it is heavily inspired by DTCS. > Specifically, rather than creating a number of windows of ever increasing > sizes, this strategy allows an operator to choose the window size, compact > with STCS within the first window of that size, and aggressive compact down > to a single sstable once that window is no longer current. The window size is > a combination of unit (minutes, hours, days) and size (1, etc), such that an > operator can expect all data using a block of that size to be compacted > together (that is, if your unit is hours, and size is 6, you will create > roughly 4 sstables per day, each one containing roughly 6 hours of data). > The result addresses a number of the problems with > DateTieredCompactionStrategy: > - At the present time, DTCS’s first window is compacted using an unusual > selection criteria, which prefers files with earlier timestamps, but ignores > sizes. In TimeWindowCompactionStrategy, the first window data will be > compacted with the well tested, fast, reliable STCS. All STCS options can be > passed to TimeWindowCompactionStrategy to configure the first window’s > compaction behavior. > - HintedHandoff may put old data in new sstables, but it will have little > impact other than slightly reduced efficiency (sstables will cover a wider > range, but the old timestamps will not impact sstable selection criteria > during compaction) > - ReadRepair may put old data in new sstables, but it will have little impact > other than slightly reduced efficiency (sstables will cover a wider range, > but the old timestamps will not impact sstable selection criteria during > compaction) > - Small, old sstables resulting from streams of any kind will be swiftly and > aggressively compacted with the other sstables matching their similar > maxTimestamp, without causing sstables in neighboring windows to grow in size. > - The configuration options are explicit and straightforward - the tuning > parameters leave little room for error. The window is set in common, easily > understandable terms such as “12 hours”, “1 Day”, “30 days”. The > minute/hour/day options are granular enough for users keeping data for hours, > and users keeping data for years. > - There is no explicitly configurable max sstable age, though sstables will > naturally stop compacting once new data is written in that window. > - Streaming operations can create sstables with old timestamps, and they'll > naturally be joined together with sstables in the same time bucket. This is > true for bootstrap/repair/sstableloader/removenode. > - It remains true that if old data and new data is written into the memtable > at the same time, the resulting sstables will be treated as if they were new > sstables, however, that no longer negatively impacts the compaction > strategy’s selection criteria for older windows. > Patch provided for : > - 2.1: https://github.com/jeffjirsa/cassandra/commits/twcs-2.1 > - 2.2: https://github.com/jeffjirsa/cassandra/commits/twcs-2.2 > - trunk (post-8099): https://github.com/jeffjirsa/cassandra/commits/twcs > Rebased, force-pushed July 18, with bug fixes for estimated pending > compactions and potential starvation if more than min_threshold tables > existed in current window but STCS did not consider them viable candidates > Rebased, force-pushed Aug 20 to bring in relevant logic from CASSANDRA-9882 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9666) Provide an alternative to DTCS
[ https://issues.apache.org/jira/browse/CASSANDRA-9666?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15110147#comment-15110147 ] Christian Winther commented on CASSANDRA-9666: -- I'm running 2.2.4 and did not see the same stability and performance stability from DTCS as TWCS prior to changing - especially in compaction > Provide an alternative to DTCS > -- > > Key: CASSANDRA-9666 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9666 > Project: Cassandra > Issue Type: Improvement >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa > Fix For: 2.1.x, 2.2.x > > Attachments: dtcs-twcs-io.png, dtcs-twcs-load.png > > > DTCS is great for time series data, but it comes with caveats that make it > difficult to use in production (typical operator behaviors such as bootstrap, > removenode, and repair have MAJOR caveats as they relate to > max_sstable_age_days, and hints/read repair break the selection algorithm). > I'm proposing an alternative, TimeWindowCompactionStrategy, that sacrifices > the tiered nature of DTCS in order to address some of DTCS' operational > shortcomings. I believe it is necessary to propose an alternative rather than > simply adjusting DTCS, because it fundamentally removes the tiered nature in > order to remove the parameter max_sstable_age_days - the result is very very > different, even if it is heavily inspired by DTCS. > Specifically, rather than creating a number of windows of ever increasing > sizes, this strategy allows an operator to choose the window size, compact > with STCS within the first window of that size, and aggressive compact down > to a single sstable once that window is no longer current. The window size is > a combination of unit (minutes, hours, days) and size (1, etc), such that an > operator can expect all data using a block of that size to be compacted > together (that is, if your unit is hours, and size is 6, you will create > roughly 4 sstables per day, each one containing roughly 6 hours of data). > The result addresses a number of the problems with > DateTieredCompactionStrategy: > - At the present time, DTCS’s first window is compacted using an unusual > selection criteria, which prefers files with earlier timestamps, but ignores > sizes. In TimeWindowCompactionStrategy, the first window data will be > compacted with the well tested, fast, reliable STCS. All STCS options can be > passed to TimeWindowCompactionStrategy to configure the first window’s > compaction behavior. > - HintedHandoff may put old data in new sstables, but it will have little > impact other than slightly reduced efficiency (sstables will cover a wider > range, but the old timestamps will not impact sstable selection criteria > during compaction) > - ReadRepair may put old data in new sstables, but it will have little impact > other than slightly reduced efficiency (sstables will cover a wider range, > but the old timestamps will not impact sstable selection criteria during > compaction) > - Small, old sstables resulting from streams of any kind will be swiftly and > aggressively compacted with the other sstables matching their similar > maxTimestamp, without causing sstables in neighboring windows to grow in size. > - The configuration options are explicit and straightforward - the tuning > parameters leave little room for error. The window is set in common, easily > understandable terms such as “12 hours”, “1 Day”, “30 days”. The > minute/hour/day options are granular enough for users keeping data for hours, > and users keeping data for years. > - There is no explicitly configurable max sstable age, though sstables will > naturally stop compacting once new data is written in that window. > - Streaming operations can create sstables with old timestamps, and they'll > naturally be joined together with sstables in the same time bucket. This is > true for bootstrap/repair/sstableloader/removenode. > - It remains true that if old data and new data is written into the memtable > at the same time, the resulting sstables will be treated as if they were new > sstables, however, that no longer negatively impacts the compaction > strategy’s selection criteria for older windows. > Patch provided for : > - 2.1: https://github.com/jeffjirsa/cassandra/commits/twcs-2.1 > - 2.2: https://github.com/jeffjirsa/cassandra/commits/twcs-2.2 > - trunk (post-8099): https://github.com/jeffjirsa/cassandra/commits/twcs > Rebased, force-pushed July 18, with bug fixes for estimated pending > compactions and potential starvation if more than min_threshold tables > existed in current window but STCS did not consider them viable candidates > Rebased, force-pushed Aug 20 to bring in relevant logic from CASSANDRA-9882 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11053) COPY FROM on large datasets: fix progress report and debug performance
[ https://issues.apache.org/jira/browse/CASSANDRA-11053?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefania updated CASSANDRA-11053: - Attachment: (was: copy_from_large_benchmark.txt) > COPY FROM on large datasets: fix progress report and debug performance > -- > > Key: CASSANDRA-11053 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11053 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Stefania >Assignee: Stefania > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > Attachments: copy_from_large_benchmark.txt > > > Running COPY from on a large dataset (20G divided in 20M records) revealed > two issues: > * The progress report is incorrect, it is very slow until almost the end of > the test at which point it catches up extremely quickly. > * The performance in rows per second is similar to running smaller tests with > a smaller cluster locally (approx 35,000 rows per second). As a comparison, > cassandra-stress manages 50,000 rows per second under the same set-up, > therefore resulting 1.5 times faster. > See attached file _copy_from_large_benchmark.txt_ for the benchmark details. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11053) COPY FROM on large datasets: fix progress report and debug performance
[ https://issues.apache.org/jira/browse/CASSANDRA-11053?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefania updated CASSANDRA-11053: - Attachment: copy_from_large_benchmark.txt > COPY FROM on large datasets: fix progress report and debug performance > -- > > Key: CASSANDRA-11053 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11053 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Stefania >Assignee: Stefania > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > Attachments: copy_from_large_benchmark.txt > > > Running COPY from on a large dataset (20G divided in 20M records) revealed > two issues: > * The progress report is incorrect, it is very slow until almost the end of > the test at which point it catches up extremely quickly. > * The performance in rows per second is similar to running smaller tests with > a smaller cluster locally (approx 35,000 rows per second). As a comparison, > cassandra-stress manages 50,000 rows per second under the same set-up, > therefore resulting 1.5 times faster. > See attached file _copy_from_large_benchmark.txt_ for the benchmark details. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-11052) Cannot use Java 8 lambda expression inside UDF code body
[ https://issues.apache.org/jira/browse/CASSANDRA-11052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp reassigned CASSANDRA-11052: Assignee: Robert Stupp > Cannot use Java 8 lambda expression inside UDF code body > > > Key: CASSANDRA-11052 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11052 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: DOAN DuyHai >Assignee: Robert Stupp > > When creating the following **UDF** using Java 8 lambda syntax > {code:sql} > CREATE FUNCTION IF NOT EXISTS music.udf(state map, styles > list) > RETURNS NULL ON NULL INPUT > RETURNS map > LANGUAGE java > AS $$ >styles.forEach((Object o) -> { >String style = (String)o; >if(state.containsKey(style)) { > state.put(style, (Long)state.get(style)+1); >} else { > state.put(style, 1L); >} >}); > >return state; > $$; > {code} > I got the following exception: > {code:java} > Caused by: com.datastax.driver.core.exceptions.InvalidQueryException: Could > not compile function 'music.udf' from Java source: > org.apache.cassandra.exceptions.InvalidRequestException: Java source > compilation failed: > Line 2: The type java.util.function.Consumer cannot be resolved. It is > indirectly referenced from required .class files > Line 2: The method forEach(Consumer) from the type Iterable refers to the > missing type Consumer > Line 2: The target type of this expression must be a functional interface > at > com.datastax.driver.core.Responses$Error.asException(Responses.java:136) > at > com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:179) > at > com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:184) > at > com.datastax.driver.core.RequestHandler.access$2500(RequestHandler.java:43) > at > com.datastax.driver.core.RequestHandler$SpeculativeExecution.setFinalResult(RequestHandler.java:798) > at > com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:617) > at > com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1005) > at > com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:928) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) > at > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) > at > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) > at > io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:276) > at > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:263) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) > at > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) > at > io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112) > ... 1 more > {code} > It looks like the compiler requires importing java.util.Consumer but I have > checked the source code and compiler options already support Java 8 source > code so I'm pretty puzzled here ... > /cc [~snazy] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9666) Provide an alternative to DTCS
[ https://issues.apache.org/jira/browse/CASSANDRA-9666?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15110194#comment-15110194 ] Marcus Eriksson commented on CASSANDRA-9666: [~jippignu] how did your DTCS config look? Datamodel? If it is that bad we should be able to reproduce. > Provide an alternative to DTCS > -- > > Key: CASSANDRA-9666 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9666 > Project: Cassandra > Issue Type: Improvement >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa > Fix For: 2.1.x, 2.2.x > > Attachments: dtcs-twcs-io.png, dtcs-twcs-load.png > > > DTCS is great for time series data, but it comes with caveats that make it > difficult to use in production (typical operator behaviors such as bootstrap, > removenode, and repair have MAJOR caveats as they relate to > max_sstable_age_days, and hints/read repair break the selection algorithm). > I'm proposing an alternative, TimeWindowCompactionStrategy, that sacrifices > the tiered nature of DTCS in order to address some of DTCS' operational > shortcomings. I believe it is necessary to propose an alternative rather than > simply adjusting DTCS, because it fundamentally removes the tiered nature in > order to remove the parameter max_sstable_age_days - the result is very very > different, even if it is heavily inspired by DTCS. > Specifically, rather than creating a number of windows of ever increasing > sizes, this strategy allows an operator to choose the window size, compact > with STCS within the first window of that size, and aggressive compact down > to a single sstable once that window is no longer current. The window size is > a combination of unit (minutes, hours, days) and size (1, etc), such that an > operator can expect all data using a block of that size to be compacted > together (that is, if your unit is hours, and size is 6, you will create > roughly 4 sstables per day, each one containing roughly 6 hours of data). > The result addresses a number of the problems with > DateTieredCompactionStrategy: > - At the present time, DTCS’s first window is compacted using an unusual > selection criteria, which prefers files with earlier timestamps, but ignores > sizes. In TimeWindowCompactionStrategy, the first window data will be > compacted with the well tested, fast, reliable STCS. All STCS options can be > passed to TimeWindowCompactionStrategy to configure the first window’s > compaction behavior. > - HintedHandoff may put old data in new sstables, but it will have little > impact other than slightly reduced efficiency (sstables will cover a wider > range, but the old timestamps will not impact sstable selection criteria > during compaction) > - ReadRepair may put old data in new sstables, but it will have little impact > other than slightly reduced efficiency (sstables will cover a wider range, > but the old timestamps will not impact sstable selection criteria during > compaction) > - Small, old sstables resulting from streams of any kind will be swiftly and > aggressively compacted with the other sstables matching their similar > maxTimestamp, without causing sstables in neighboring windows to grow in size. > - The configuration options are explicit and straightforward - the tuning > parameters leave little room for error. The window is set in common, easily > understandable terms such as “12 hours”, “1 Day”, “30 days”. The > minute/hour/day options are granular enough for users keeping data for hours, > and users keeping data for years. > - There is no explicitly configurable max sstable age, though sstables will > naturally stop compacting once new data is written in that window. > - Streaming operations can create sstables with old timestamps, and they'll > naturally be joined together with sstables in the same time bucket. This is > true for bootstrap/repair/sstableloader/removenode. > - It remains true that if old data and new data is written into the memtable > at the same time, the resulting sstables will be treated as if they were new > sstables, however, that no longer negatively impacts the compaction > strategy’s selection criteria for older windows. > Patch provided for : > - 2.1: https://github.com/jeffjirsa/cassandra/commits/twcs-2.1 > - 2.2: https://github.com/jeffjirsa/cassandra/commits/twcs-2.2 > - trunk (post-8099): https://github.com/jeffjirsa/cassandra/commits/twcs > Rebased, force-pushed July 18, with bug fixes for estimated pending > compactions and potential starvation if more than min_threshold tables > existed in current window but STCS did not consider them viable candidates > Rebased, force-pushed Aug 20 to bring in relevant logic from CASSANDRA-9882 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11054) Added support for IBM zSystems architecture (s390x)
Meerabo Shah created CASSANDRA-11054: Summary: Added support for IBM zSystems architecture (s390x) Key: CASSANDRA-11054 URL: https://issues.apache.org/jira/browse/CASSANDRA-11054 Project: Cassandra Issue Type: Improvement Components: Observability, Testing Environment: s390x Reporter: Meerabo Shah Priority: Minor Fix For: 3.x Added support for IBM zSystems architecture (s390x). These code changes are required to make few test cases 'pass' for zSystems. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10389) Repair session exception Validation failed
[ https://issues.apache.org/jira/browse/CASSANDRA-10389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108643#comment-15108643 ] Jeff Gardner commented on CASSANDRA-10389: -- We are also experiencing this issue in 2.2.3; and yes we have restarted all nodes. Our config: Cassandra 2.2.3 AWS west-2 and east-1 regions with 6 nodes per region [2/AZ] > Repair session exception Validation failed > -- > > Key: CASSANDRA-10389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10389 > Project: Cassandra > Issue Type: Bug > Environment: Debian 8, Java 1.8.0_60, Cassandra 2.2.1 (datastax > compilation) >Reporter: Jędrzej Sieracki > Fix For: 2.2.x > > > I'm running a repair on a ring of nodes, that was recently extented from 3 to > 13 nodes. The extension was done two days ago, the repair was attempted > yesterday. > {quote} > [2015-09-22 11:55:55,266] Starting repair command #9, repairing keyspace > perspectiv with repair options (parallelism: parallel, primary range: false, > incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [], > hosts: [], # of ranges: 517) > [2015-09-22 11:55:58,043] Repair session 1f7c50c0-6110-11e5-b992-9f13fa8664c8 > for range (-5927186132136652665,-5917344746039874798] failed with error > [repair #1f7c50c0-6110-11e5-b992-9f13fa8664c8 on > perspectiv/stock_increment_agg, (-5927186132136652665,-5917344746039874798]] > Validation failed in cblade1.XXX/XXX (progress: 0%) > {quote} > BTW, I am ignoring the LEAK errors for now, that's outside of the scope of > the main issue: > {quote} > ERROR [Reference-Reaper:1] 2015-09-22 11:58:27,843 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@4d25ad8f) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@896826067:/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-73-big > was not released before the reference was garbage collected > {quote} > I scrubbed the sstable with failed validation on cblade1 with nodetool scrub > perspectiv stock_increment_agg: > {quote} > INFO [CompactionExecutor:1704] 2015-09-22 12:05:31,615 OutputHandler.java:42 > - Scrubbing > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big-Data.db') > (345466609 bytes) > INFO [CompactionExecutor:1703] 2015-09-22 12:05:31,615 OutputHandler.java:42 > - Scrubbing > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-82-big-Data.db') > (60496378 bytes) > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@4ca8951e) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@114161559:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-48-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@eeb6383) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1612685364:/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@1de90543) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@2058626950:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-49-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@15616385) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1386628428:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-47-big > was not released before the reference was garbage collected > INFO [CompactionExecutor:1703] 2015-09-22 12:05:35,098 OutputHandler.java:42 > - Scrub of > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-82-big-Data.db') > complete: 51397 rows in new sstable and 0 empty (tombstoned) rows dropped > INFO [CompactionExecutor:1704] 2015-09-22 12:05:47,605 OutputHandler.java:42 > - Scrub of > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big-Data.db') > complete: 292600 rows in new
svn commit: r11974 - /release/cassandra/3.1.1/
Author: jake Date: Wed Jan 20 14:53:48 2016 New Revision: 11974 Log: add back 3.1.1 release Added: release/cassandra/3.1.1/ release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz (with props) release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.md5 release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.sha1 release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.md5 release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.sha1 release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz (with props) release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.asc release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.asc.md5 release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.asc.sha1 release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.md5 release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.sha1 Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz == Binary file - no diff available. Propchange: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz -- svn:mime-type = application/octet-stream Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc == --- release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc (added) +++ release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc Wed Jan 20 14:53:48 2016 @@ -0,0 +1,17 @@ +-BEGIN PGP SIGNATURE- +Version: GnuPG v1 + +iQIcBAABAgAGBQJWcW/rAAoJEHSdbuwDU7EsS8EP/182FM9tbxPGS/S7inqZ6NtJ +Zh1nEkZXsDhB7mCFl/B5D8q5z14Z+YE9Zh/wIXs9tssOJxtNd+hQb9paQNe4q0m0 +g3/aUWZgCH0u48gD4jT6eRAa7d3ioxBHzA+2RgOTgAW+WEicNGjcZ7timHI3uzD3 +fvWXl3v8BkmYN+eoMyE69DbqeiRwAAOiQ4JTuQX5K2msANraIJW52ZR5Oz1X8Ooq +f7oVpt+FCt+HnR2uFzBR0sn+Pa2KjqWW9N/1QnkCrXVkVP9kZ+JUTe8fBW00dn6n +LFPtUds93zQSo+QMJWmV08jpAyIG/Rb43e7iJ+fTdQtkx5vTlT9NlvpWLAvLMrFj +rOP79iEODqpHssS5lKBzaCAbdjo3kbAUdHrxZ3cWS3SAFhLoESrhsK3DrlA1leQO +8ahAvrqGKIS8tWbPhTcU1ZOdivcP4T6Xr4cuagFkB3ujgrbl7Cou+1EO57cDw8fm +bttAzoAp8hsVjzpBTcp6t15Oy9fkyvQxO7/XvpPzcVAIQR5BKoeAzLY52rUwzIe9 +8kqL5BUJrJVKTvg/ydnimVaLi8JVuPiQmqTufiaSCO2rRYzKxoY+Ycac0Q5/KUJG +RMyd4SrNMT1qg/AgV4wxIfKiyc6WL5YJiIWzg8mi3WSvtAu+PEnZGpqwUJ5FmmTH +C+9tpYY2BLWi8pd/f/Sm +=2+Tl +-END PGP SIGNATURE- Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.md5 == --- release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.md5 (added) +++ release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.md5 Wed Jan 20 14:53:48 2016 @@ -0,0 +1 @@ +4325f035cebcd551079685a81a00738f \ No newline at end of file Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.sha1 == --- release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.sha1 (added) +++ release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.asc.sha1 Wed Jan 20 14:53:48 2016 @@ -0,0 +1 @@ +d54a71b5bdbbc884e23151ad725a917f69392eb3 \ No newline at end of file Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.md5 == --- release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.md5 (added) +++ release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.md5 Wed Jan 20 14:53:48 2016 @@ -0,0 +1 @@ +7cc668cb5343e21dd8bbb4264fc93ca5 \ No newline at end of file Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.sha1 == --- release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.sha1 (added) +++ release/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz.sha1 Wed Jan 20 14:53:48 2016 @@ -0,0 +1 @@ +87d51b6289af2b15de3a92df5ed050531e75112d \ No newline at end of file Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz == Binary file - no diff available. Propchange: release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz -- svn:mime-type = application/octet-stream Added: release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.asc == --- release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.asc (added) +++ release/cassandra/3.1.1/apache-cassandra-3.1.1-src.tar.gz.asc Wed Jan 20 14:53:48 2016 @@ -0,0 +1,17 @@ +-BEGIN PGP SIGNATURE- +Version: GnuPG v1 + +iQIcBAABAgAGBQJWcXAHAAoJEHSdbuwDU7EsCpgP/RgW61JUlms5qpJKqHKIfQ9A
[jira] [Commented] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108647#comment-15108647 ] Benjamin Lerer commented on CASSANDRA-10707: ||patch||utests||dtests|| |[trunk|https://github.com/apache/cassandra/compare/trunk...blerer:10707-trunk]|[trunk|http://cassci.datastax.com/view/Dev/view/blerer/job/blerer-10707-trunk-testall/]|[trunk|http://cassci.datastax.com/view/Dev/view/blerer/job/blerer-10707-trunk-dtest/]| The DTest branch is [here|https://github.com/riptano/cassandra-dtest/pull/753/files] The classes {{GroupBySpecification}} and {{GroupSelector}} are used to create {{GroupMaker}} instances. A {{GroupMaker}} is used, on a sorted set of rows, to determine if a row belongs to the same group as the previous row or not. For the moment, only one type of {{GroupSelector}} exists for a primary key columns. Its serialization mechanism has, nevertheless, been implemented in such a way that it will be possible to add new implementations (to allow the use of functions in the {{GROUP BY}} clause, for example) without breaking backward compatibility. {{SelectStatement}} and {{Selection}} have been modified in order to use {{GroupBySpecification}} and {{GroupMaker}} when building the result set. Group by queries are always paged internally to avoid {{OOMExceptions}}. Two new {{DataLimits}} have been added to manage the group by paging {{CQLGroupByLimits}} and {{CQLGroupByPagingLimits}}. They keep track of the group count and of the row count to make sure that the processing is stopped as soon as one of the limits is reached. A group is only counted once the next one is reached, as a group can be spread over multiple pages. The problem with this approach is that a counter can only know if it has reach the group limit when it has reached a row that should not be added to the resultset. As multiple counters are used when a request is processed the extra row is not filtered out until it reachs the counter of the {{QueryPager}}. To do that a special factory method has been added to {{DataLimits}}: {{forPagingByQueryPager(int pageSize)}}. This approach was not working properly in the case of the {{MultiPartitionPager}} as an extra counter was added on top of the one of the {{SinglePartitionPager}}. To solve that problem the use of the counter in {{MultiPartitionPager}} has been replaced by another mechanism. The internal paging is performed by the {{GroupByQueryPager}} which automatically fetch new pages of data when needed. As the {{DataLimits}} needs to be updated for each new internal query and the {{ReadQuery}} classes are immutable a new {{withUpdatedLimit}} method as been a added to all the {{ReadQuery}} classes. In order to simplify the {{SelectStatement}} code, the patch also modify slightly the way queries with aggregates but no {{GROUP BY}} is working. I implemented it initially on top of the Group by paging but realized afterward that it was breaking backward compatibility. We will anyway be able in the future to switch back to it. Once we are sure that the group by paging is supported by the previous versions. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10613) Upgrade test on 2.1->3.0 path fails with NPE in getExistingFiles (likely known bug)
[ https://issues.apache.org/jira/browse/CASSANDRA-10613?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108996#comment-15108996 ] Jim Witschey commented on CASSANDRA-10613: -- Ping [~rhatch] -- You've been looking at these tests a lot lately, yeah? Have you seen this NPE failure lately? > Upgrade test on 2.1->3.0 path fails with NPE in getExistingFiles (likely > known bug) > --- > > Key: CASSANDRA-10613 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10613 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: Russ Hatch > Fix For: 3.0.x > > > In this job: > http://cassci.datastax.com/view/Upgrades/job/cassandra_upgrade_2.1_to_3.0_proto_v3/10/ > The following tests fail due to an NPE in > {{org.apache.cassandra.db.lifecycle.LogRecord.getExistingFiles}}: > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.bootstrap_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.rolling_upgrade_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.parallel_upgrade_with_internode_ssl_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.rolling_upgrade_with_internode_ssl_test > upgrade_through_versions_test.py:TestUpgrade_from_cassandra_2_1_HEAD_to_cassandra_3_0_HEAD.rolling_upgrade_with_internode_ssl_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.parallel_upgrade_test > I believe this is likely happening because of CASSANDRA-10602, so let's hold > off on messing with this until that's merged. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-10905) secondary_indexes_test.py:TestSecondaryIndexes.test_only_coordinator_chooses_index_for_query flaps on 3.0+
[ https://issues.apache.org/jira/browse/CASSANDRA-10905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe reassigned CASSANDRA-10905: --- Assignee: Sam Tunnicliffe (was: DS Test Eng) > secondary_indexes_test.py:TestSecondaryIndexes.test_only_coordinator_chooses_index_for_query > flaps on 3.0+ > -- > > Key: CASSANDRA-10905 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10905 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: Sam Tunnicliffe > Fix For: 3.0.x > > > This test flaps when it expects to find 1 of each of several trace events, > but doesn't find some of them. I noticed it on 3.0: > http://cassci.datastax.com/job/cassandra-3.0_dtest/438/testReport/junit/secondary_indexes_test/TestSecondaryIndexes/test_only_coordinator_chooses_index_for_query/history/ > But I've also seen it on trunk, so it would appear it hasn't been fixed: > http://cassci.datastax.com/job/trunk_dtest/831/testReport/junit/secondary_indexes_test/TestSecondaryIndexes/test_only_coordinator_chooses_index_for_query/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian updated CASSANDRA-11037: --- Fix Version/s: (was: 2.2.0) > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian updated CASSANDRA-11037: --- Labels: lhf (was: easyfix newbie) > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10743) Failed upgradesstables (upgrade from 2.2.2 to 3.0.0)
[ https://issues.apache.org/jira/browse/CASSANDRA-10743?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108818#comment-15108818 ] Carl Yeksigian commented on CASSANDRA-10743: {quote} I saw that 3.2.0 was released earlier, but also yesterday 3.2.1 got released. Do you know if the fix is in 3.2.1? {quote} [~tomas0413] 3.2.1 was a bugfix release which only included a fix for CASSANDRA-11010. The fix version on this ticket is still correct. > Failed upgradesstables (upgrade from 2.2.2 to 3.0.0) > > > Key: CASSANDRA-10743 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10743 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths > Environment: CentOS Linux release 7.1.1503, OpenJDK Runtime > Environment (build 1.8.0_65-b17), DSC Cassandra 3.0.0 (tar.gz) >Reporter: Gábor Auth >Assignee: Sylvain Lebresne > Fix For: 3.0.3, 3.3 > > Attachments: faulty-tables.tar.gz, schema.ddl > > > {code} > [cassandra@dc01-rack01-cass01 ~]$ > /home/cassandra/dsc-cassandra-3.0.0/bin/nodetool upgradesstables > error: null > -- StackTrace -- > java.lang.UnsupportedOperationException > at > org.apache.cassandra.db.rows.CellPath$EmptyCellPath.get(CellPath.java:143) > at > org.apache.cassandra.db.marshal.CollectionType$CollectionPathSerializer.serializedSize(CollectionType.java:226) > at > org.apache.cassandra.db.rows.BufferCell$Serializer.serializedSize(BufferCell.java:325) > at > org.apache.cassandra.db.rows.UnfilteredSerializer.sizeOfComplexColumn(UnfilteredSerializer.java:297) > at > org.apache.cassandra.db.rows.UnfilteredSerializer.serializedRowBodySize(UnfilteredSerializer.java:282) > at > org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:163) > at > org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:108) > at > org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:144) > at > org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:112) > at > org.apache.cassandra.db.ColumnIndex.writeAndBuildIndex(ColumnIndex.java:52) > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:149) > at > org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:121) > at > org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter.realAppend(DefaultCompactionWriter.java:57) > at > org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:110) > at > org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:182) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:78) > at > org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60) > at > org.apache.cassandra.db.compaction.CompactionManager$5.execute(CompactionManager.java:397) > at > org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:292) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11033) Prevent logging in sandboxed state
[ https://issues.apache.org/jira/browse/CASSANDRA-11033?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-11033: - Fix Version/s: (was: 2.2.x) > Prevent logging in sandboxed state > -- > > Key: CASSANDRA-11033 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11033 > Project: Cassandra > Issue Type: Bug >Reporter: Robert Stupp >Assignee: Robert Stupp >Priority: Minor > Fix For: 3.0.x > > > logback will re-read its configuration file regularly. So it is possible that > logback tries to reload the configuration while we log from a sandboxed UDF, > which will fail due to the restricted access privileges for UDFs. UDAs are > also affected as these use UDFs. > /cc [~doanduyhai] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7281) SELECT on tuple relations are broken for mixed ASC/DESC clustering order
[ https://issues.apache.org/jira/browse/CASSANDRA-7281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108830#comment-15108830 ] Benjamin Lerer commented on CASSANDRA-7281: --- It would be perfect. :-) > SELECT on tuple relations are broken for mixed ASC/DESC clustering order > > > Key: CASSANDRA-7281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7281 > Project: Cassandra > Issue Type: Bug >Reporter: Sylvain Lebresne >Assignee: Marcin Szymaniuk > Fix For: 3.3, 2.2.x, 3.0.x > > Attachments: > 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-.patch, > 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v2.patch, > 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v3.patch, > 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v4.patch, > 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v5.patch, > 7281_unit_tests.txt > > > As noted on > [CASSANDRA-6875|https://issues.apache.org/jira/browse/CASSANDRA-6875?focusedCommentId=13992153=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13992153], > the tuple notation is broken when the clustering order mixes ASC and DESC > directives because the range of data they describe don't correspond to a > single continuous slice internally. To copy the example from CASSANDRA-6875: > {noformat} > cqlsh:ks> create table foo (a int, b int, c int, PRIMARY KEY (a, b, c)) WITH > CLUSTERING ORDER BY (b DESC, c ASC); > cqlsh:ks> INSERT INTO foo (a, b, c) VALUES (0, 2, 0); > cqlsh:ks> INSERT INTO foo (a, b, c) VALUES (0, 1, 0); > cqlsh:ks> INSERT INTO foo (a, b, c) VALUES (0, 1, 1); > cqlsh:ks> INSERT INTO foo (a, b, c) VALUES (0, 0, 0); > cqlsh:ks> SELECT * FROM foo WHERE a=0; > a | b | c > ---+---+--- > 0 | 2 | 0 > 0 | 1 | 0 > 0 | 1 | 1 > 0 | 0 | 0 > (4 rows) > cqlsh:ks> SELECT * FROM foo WHERE a=0 AND (b, c) > (1, 0); > a | b | c > ---+---+--- > 0 | 2 | 0 > (1 rows) > {noformat} > The last query should really return {{(0, 2, 0)}} and {{(0, 1, 1)}}. > For that specific example we should generate 2 internal slices, but I believe > that with more clustering columns we may have more slices. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11047) native protocol will not bind ipv6
[ https://issues.apache.org/jira/browse/CASSANDRA-11047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109015#comment-15109015 ] Brandon Williams commented on CASSANDRA-11047: -- Ping [~norman] since this looks like a netty thing. > native protocol will not bind ipv6 > -- > > Key: CASSANDRA-11047 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11047 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Brandon Williams > > When you set rpc_address to 0.0.0.0 it should bind every interface. Of > course for ipv6 you have to comment out -Djava.net.preferIPv4Stack=true from > cassandra-env.sh, however this will not make the native protocol bind on > ipv6, only thrift: > {noformat} > tcp6 0 0 :::9160 :::*LISTEN > 13488/java > tcp6 0 0 0.0.0.0:9042:::*LISTEN > 13488/java > # telnet ::1 9160 > Trying ::1... > Connected to ::1. > Escape character is '^]'. > ^] > telnet> quit > Connection closed. > # telnet ::1 9042 > Trying ::1... > telnet: Unable to connect to remote host: Connection refused > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109146#comment-15109146 ] Benjamin Zarzycki commented on CASSANDRA-11037: --- [~carlyeks] Thanks for the JIRA info and raising the Mac OS compatibility issue. Would be okay to have this script depend on perl? Then we could resolve the cross platform compatibility and symlink issue. Something like: python "$(dirname $(perl -e 'use Cwd "abs_path";print abs_path(shift)' $0))/cqlsh.py" $@" Thoughts? > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10934) upgrade_test map_keys_indexing_test is failing
[ https://issues.apache.org/jira/browse/CASSANDRA-10934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108678#comment-15108678 ] Sam Tunnicliffe commented on CASSANDRA-10934: - The failing dtest is incorrect, creating a values index on a map column which already has a keys index is permitted since 3.0. dtest PR [here|https://github.com/riptano/cassandra-dtest/pull/754] > upgrade_test map_keys_indexing_test is failing > -- > > Key: CASSANDRA-10934 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10934 > Project: Cassandra > Issue Type: Sub-task >Reporter: Philip Thompson >Assignee: Sam Tunnicliffe > Fix For: 3.0.x, 3.x > > Attachments: node1.log, node1_debug.log, node2.log, node2_debug.log, > node3.log, node3_debug.log > > > {{upgrade_tests.cql_tests.TestCQLNodes3RF3.map_keys_indexing_test}} and > {{upgrade_tests.cql_tests.TestCQLNodes2RF1.map_keys_indexing_test}} are both > failing. See > http://cassci.datastax.com/job/cassandra-3.0_dtest/456/testReport/upgrade_tests.cql_tests/TestCQLNodes3RF3/map_keys_indexing_test/ > for an example. > The tests create a secondary index on the keys of a map, then perform an > upgrade, then attempt to create secondary index on the values of that map. > This is expected to fail, as both indices cannot exist at the same time. If > the index creation happens before the upgrade, it does fail. After the > upgrade, however, it works. When I check the cluster schema metadata with the > python driver, I can only see one index after creating the second, rather > than both. > Node logs are attached. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11043) Secondary indexes doesn't properly validate custom expressions
[ https://issues.apache.org/jira/browse/CASSANDRA-11043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrés de la Peña updated CASSANDRA-11043: -- Component/s: Local Write-Read Paths > Secondary indexes doesn't properly validate custom expressions > -- > > Key: CASSANDRA-11043 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11043 > Project: Cassandra > Issue Type: Bug > Components: CQL, Local Write-Read Paths >Reporter: Andrés de la Peña > Labels: 21, index, validation > Attachments: test-index.zip > > > It seems that > [CASSANDRA-7575|https://issues.apache.org/jira/browse/CASSANDRA-7575] is > broken in Cassandra 3.x. As stated in the secondary indexes' API > documentation, custom index implementations should perform any validation of > query expressions at {{Index#searcherFor(ReadCommand)}}, throwing an > {{InvalidRequestException}} if the expressions are not valid. I assume these > validation errors should produce an {{InvalidRequest}} error on cqlsh, or > raise an {{InvalidQueryException}} on Java driver. However, when > {{Index#searcherFor(ReadCommand)}} throws its {{InvalidRequestException}}, I > get this cqlsh output: > {noformat} > Traceback (most recent call last): > File "bin/cqlsh.py", line 1246, in perform_simple_statement > result = future.result() > File > "/Users/adelapena/stratio/platform/src/cassandra-3.2.1/bin/../lib/cassandra-driver-internal-only-3.0.0-6af642d.zip/cassandra-driver-3.0.0-6af642d/cassandra/cluster.py", > line 3122, in result > raise self._final_exception > ReadFailure: code=1300 [Replica(s) failed to execute read] message="Operation > failed - received 0 responses and 1 failures" info={'failures': 1, > 'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'} > {noformat} > I attach a dummy index implementation to reproduce the error: > {noformat} > CREATE KEYSPACE test with replication = {'class' : 'SimpleStrategy', > 'replication_factor' : '1' }; > CREATE TABLE test.test (id int PRIMARY KEY, value varchar); > CREATE CUSTOM INDEX test_index ON test.test() USING 'com.stratio.TestIndex'; > SELECT * FROM test.test WHERE expr(test_index,'ok'); > SELECT * FROM test.test WHERE expr(test_index,'error'); > {noformat} > This is specially problematic when using Cassandra Java Driver, because one > of these server exceptions can produce subsequent queries fail (even if they > are valid) with a no host available exception. > Maybe the validation method added with > [CASSANDRA-7575|https://issues.apache.org/jira/browse/CASSANDRA-7575] should > be restored, unless there is a way to properly manage the exception. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108848#comment-15108848 ] Aleksey Yeschenko commented on CASSANDRA-10707: --- bq. In the future, we might manage to push the aggregate computation to the replicas but we are not there yet. I think what Brian meant here wasn't pushing computation to replicas, but splitting it between sub-coordinators. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11047) native protocol will not bind ipv6
Brandon Williams created CASSANDRA-11047: Summary: native protocol will not bind ipv6 Key: CASSANDRA-11047 URL: https://issues.apache.org/jira/browse/CASSANDRA-11047 Project: Cassandra Issue Type: Bug Components: CQL Reporter: Brandon Williams When you set rpc_address to 0.0.0.0 it should bind every interface. Of course for ipv6 you have to comment out -Djava.net.preferIPv4Stack=true from cassandra-env.sh, however this will not make the native protocol bind on ipv6, only thrift: {noformat} tcp6 0 0 :::9160 :::*LISTEN 13488/java tcp6 0 0 0.0.0.0:9042:::*LISTEN 13488/java # telnet ::1 9160 Trying ::1... Connected to ::1. Escape character is '^]'. ^] telnet> quit Connection closed. # telnet ::1 9042 Trying ::1... telnet: Unable to connect to remote host: Connection refused {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11047) native protocol will not bind ipv6
[ https://issues.apache.org/jira/browse/CASSANDRA-11047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-11047: - Reproduced In: 2.1.x > native protocol will not bind ipv6 > -- > > Key: CASSANDRA-11047 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11047 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Brandon Williams > > When you set rpc_address to 0.0.0.0 it should bind every interface. Of > course for ipv6 you have to comment out -Djava.net.preferIPv4Stack=true from > cassandra-env.sh, however this will not make the native protocol bind on > ipv6, only thrift: > {noformat} > tcp6 0 0 :::9160 :::*LISTEN > 13488/java > tcp6 0 0 0.0.0.0:9042:::*LISTEN > 13488/java > # telnet ::1 9160 > Trying ::1... > Connected to ::1. > Escape character is '^]'. > ^] > telnet> quit > Connection closed. > # telnet ::1 9042 > Trying ::1... > telnet: Unable to connect to remote host: Connection refused > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11048) JSON queries are not thread safe
Sergio Bossa created CASSANDRA-11048: Summary: JSON queries are not thread safe Key: CASSANDRA-11048 URL: https://issues.apache.org/jira/browse/CASSANDRA-11048 Project: Cassandra Issue Type: Bug Reporter: Sergio Bossa Priority: Critical {{org.apache.cassandra.cql3.Json}} uses a shared instance of {{JsonStringEncoder}} which is not thread safe (see 1), while {{JsonStringEncoder#getInstance()}} should be used (see 2). As a consequence, concurrent {{select JSON}} queries often produce wrong (sometimes unreadable) results. 1. http://grepcode.com/file/repo1.maven.org/maven2/org.codehaus.jackson/jackson-core-asl/1.9.2/org/codehaus/jackson/io/JsonStringEncoder.java 2. http://grepcode.com/file/repo1.maven.org/maven2/org.codehaus.jackson/jackson-core-asl/1.9.2/org/codehaus/jackson/io/JsonStringEncoder.java#JsonStringEncoder.getInstance%28%29 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10308) Investigate test_scrub_collections_table success
[ https://issues.apache.org/jira/browse/CASSANDRA-10308?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jim Witschey updated CASSANDRA-10308: - Assignee: DS Test Eng (was: Jim Witschey) > Investigate test_scrub_collections_table success > > > Key: CASSANDRA-10308 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10308 > Project: Cassandra > Issue Type: Test >Reporter: Jim Witschey >Assignee: DS Test Eng >Priority: Minor > > As discussed [in dtest PR > 431|https://github.com/riptano/cassandra-dtest/pull/431#issuecomment-126802307], > {{scrub_test.py:TestScrubIndexes.test_scrub_collections_table}} started > succeeding without any real explanation after [this commit to > C*|https://github.com/apache/cassandra/commit/b70f7ea0ce27b5defa0a7773d448732364e7aee0]. > I hate to be suspicious of a good thing, but I think it's worth > investigating this to make sure that the test's success reflects reality and > that no changes to Cassandra or the dtests had unintended consequences. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-10934) upgrade_test map_keys_indexing_test is failing
[ https://issues.apache.org/jira/browse/CASSANDRA-10934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe resolved CASSANDRA-10934. - Resolution: Fixed Fix Version/s: (was: 3.0.x) (was: 3.x) Resolving as after the dtest PR was merged, the test passes (after failing for the previous 75 runs) > upgrade_test map_keys_indexing_test is failing > -- > > Key: CASSANDRA-10934 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10934 > Project: Cassandra > Issue Type: Sub-task >Reporter: Philip Thompson >Assignee: Sam Tunnicliffe > Attachments: node1.log, node1_debug.log, node2.log, node2_debug.log, > node3.log, node3_debug.log > > > {{upgrade_tests.cql_tests.TestCQLNodes3RF3.map_keys_indexing_test}} and > {{upgrade_tests.cql_tests.TestCQLNodes2RF1.map_keys_indexing_test}} are both > failing. See > http://cassci.datastax.com/job/cassandra-3.0_dtest/456/testReport/upgrade_tests.cql_tests/TestCQLNodes3RF3/map_keys_indexing_test/ > for an example. > The tests create a secondary index on the keys of a map, then perform an > upgrade, then attempt to create secondary index on the values of that map. > This is expected to fail, as both indices cannot exist at the same time. If > the index creation happens before the upgrade, it does fail. After the > upgrade, however, it works. When I check the cluster schema metadata with the > python driver, I can only see one index after creating the second, rather > than both. > Node logs are attached. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8180) Optimize disk seek using min/max column name meta data when the LIMIT clause is used
[ https://issues.apache.org/jira/browse/CASSANDRA-8180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109052#comment-15109052 ] Branimir Lambov commented on CASSANDRA-8180: I personally would prefer to not modify the behaviour of {{MergeIterator}} and keep it doing one simple thing, but this approach does have its charm. An empty row will not work correctly as a lower bound. It does not sort as needed with respect to tombstone bounds, which should also be included in the test (more specifically, one that adds a row, flushes, deletes same row, flushes again, then checks if it resurfaces-- I believe this would break with the current code). Use a {{RangeTombstoneBound}} with {{DeletionTime.LIVE}} as the deletion time and a bound obtained by {{RangeTombstone.Bound.inclusiveOpen}}, which should do the right thing in both directions. {{IMergeIterator.LowerBound}} is cryptic, rename it to {{IteratorWithLowerBound}} to be explicit about its purpose. The choice to set {{rowIndexLowerBound}} in {{partitionLevelDeletion()}} appears very arbitrary and fragile. What is the reason to do it separately from {{globalLowerBound}}? In fact, why have two separate bounds instead of one, set from the most precise information that is available at construction time? > Optimize disk seek using min/max column name meta data when the LIMIT clause > is used > > > Key: CASSANDRA-8180 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8180 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths > Environment: Cassandra 2.0.10 >Reporter: DOAN DuyHai >Assignee: Stefania >Priority: Minor > Fix For: 3.x > > Attachments: 8180_001.yaml, 8180_002.yaml > > > I was working on an example of sensor data table (timeseries) and face a use > case where C* does not optimize read on disk. > {code} > cqlsh:test> CREATE TABLE test(id int, col int, val text, PRIMARY KEY(id,col)) > WITH CLUSTERING ORDER BY (col DESC); > cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 10, '10'); > ... > >nodetool flush test test > ... > cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 20, '20'); > ... > >nodetool flush test test > ... > cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 30, '30'); > ... > >nodetool flush test test > {code} > After that, I activate request tracing: > {code} > cqlsh:test> SELECT * FROM test WHERE id=1 LIMIT 1; > activity | > timestamp| source| source_elapsed > ---+--+---+ > execute_cql3_query | > 23:48:46,498 | 127.0.0.1 | 0 > Parsing SELECT * FROM test WHERE id=1 LIMIT 1; | > 23:48:46,498 | 127.0.0.1 | 74 >Preparing statement | > 23:48:46,499 | 127.0.0.1 |253 > Executing single-partition query on test | > 23:48:46,499 | 127.0.0.1 |930 > Acquiring sstable references | > 23:48:46,499 | 127.0.0.1 |943 >Merging memtable tombstones | > 23:48:46,499 | 127.0.0.1 | 1032 >Key cache hit for sstable 3 | > 23:48:46,500 | 127.0.0.1 | 1160 >Seeking to partition beginning in data file | > 23:48:46,500 | 127.0.0.1 | 1173 >Key cache hit for sstable 2 | > 23:48:46,500 | 127.0.0.1 | 1889 >Seeking to partition beginning in data file | > 23:48:46,500 | 127.0.0.1 | 1901 >Key cache hit for sstable 1 | > 23:48:46,501 | 127.0.0.1 | 2373 >Seeking to partition beginning in data file | > 23:48:46,501 | 127.0.0.1 | 2384 > Skipped 0/3 non-slice-intersecting sstables, included 0 due to tombstones | > 23:48:46,501 | 127.0.0.1 | 2768 > Merging data from memtables and 3 sstables | > 23:48:46,501 | 127.0.0.1 | 2784 > Read 2 live and 0 tombstoned cells | > 23:48:46,501 | 127.0.0.1 | 2976 > Request complete | > 23:48:46,501 | 127.0.0.1 | 3551 > {code} > We can clearly see that C* hits 3 SSTables on disk instead of just one, >
[jira] [Updated] (CASSANDRA-11047) native protocol will not bind ipv6
[ https://issues.apache.org/jira/browse/CASSANDRA-11047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-11047: - Fix Version/s: 3.x 2.2.x 2.1.x > native protocol will not bind ipv6 > -- > > Key: CASSANDRA-11047 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11047 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Brandon Williams > Fix For: 2.1.x, 2.2.x, 3.x > > > When you set rpc_address to 0.0.0.0 it should bind every interface. Of > course for ipv6 you have to comment out -Djava.net.preferIPv4Stack=true from > cassandra-env.sh, however this will not make the native protocol bind on > ipv6, only thrift: > {noformat} > tcp6 0 0 :::9160 :::*LISTEN > 13488/java > tcp6 0 0 0.0.0.0:9042:::*LISTEN > 13488/java > # telnet ::1 9160 > Trying ::1... > Connected to ::1. > Escape character is '^]'. > ^] > telnet> quit > Connection closed. > # telnet ::1 9042 > Trying ::1... > telnet: Unable to connect to remote host: Connection refused > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10868) Skip supercolumns upgrade tests on jdk8
[ https://issues.apache.org/jira/browse/CASSANDRA-10868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109028#comment-15109028 ] Russ Hatch commented on CASSANDRA-10868: I'm not certain what guarantees can be made about the binaries running on jdk8, but seems reasonable to me if that's not a problem. Alternatively I guess we could to jdk switching similar to how it's done in upgrade_through_versions_test.py if the first approach doesn't pan out. Not sure how implementing this will look, but we should take care to not let binaries be used as the upgraded-to (only uprgaded-from) version, unless we're specifically looking to vet those binaries. > Skip supercolumns upgrade tests on jdk8 > --- > > Key: CASSANDRA-10868 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10868 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey >Assignee: Jim Witschey > > The tests in the {{upgrade_supercolumns_test}} dtest module fail when we test > on JDK8 because they attempt to upgrade from 2.0, which will not compile on > JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/160/testReport/upgrade_supercolumns_test/ > [~rhatch] As we look at how we want to run upgrade tests in the future, we > should consider this. In the meantime, I think the best way to deal with this > might be to add something to the exclude files in {{conf/}}. That sound > reasonable, or is there a better way to do this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10446) Run repair with down replicas
[ https://issues.apache.org/jira/browse/CASSANDRA-10446?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109045#comment-15109045 ] Anuj Wadehra commented on CASSANDRA-10446: -- I think this option won't do the job. Referring to scenario, when a node failed in 20 node cluster, what nodes will you set in -hosts and how will you ensure that the entire ring is repaired? Suppose host20 failed, you would run "full repair with -hosts hosts1,host2...host19 option" on all 19 healthy nodes.This option is unrealistic. Clusters generally use repair -pr option to repair the cluster. With RF=5, Repair time would be 5 times more for 19 nodes. Moreover, it requires special planning and manual intervention with just one node failure which should be undesirable in a distributed fault tolerant system. Another option would be to run repair -pr on 19 nodes and run repair separately on the ranges for which the failed node was responsible. But that wont work because -pr and -hosts options don't work together. Can you provide a better way to use -hosts option for addressing the issue? > Run repair with down replicas > - > > Key: CASSANDRA-10446 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10446 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Priority: Minor > Fix For: 3.x > > > We should have an option of running repair when replicas are down. We can > call it -force. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10955) Multi-partitions queries with ORDER BY can result in a NPE
[ https://issues.apache.org/jira/browse/CASSANDRA-10955?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-10955: Reviewer: Sam Tunnicliffe > Multi-partitions queries with ORDER BY can result in a NPE > -- > > Key: CASSANDRA-10955 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10955 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > Attachments: 10955-2.2.txt > > > In the case of a table with static columns, if only the static columns have > been set for some partitions, a multi-partitions query with an {{ORDER BY}} > can cause a {{NPE}}. > The following unit test can be used to reproduce the problem: > {code} > @Test > public void testOrderByForInClauseWithNullValue() throws Throwable > { > createTable("CREATE TABLE %s (a int, b int, c int, s int static, d > int, PRIMARY KEY (a, b, c))"); > execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 1, 1)"); > execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 2, 1)"); > execute("INSERT INTO %s (a, b, c, d) VALUES (2, 2, 1, 1)"); > execute("INSERT INTO %s (a, b, c, d) VALUES (2, 2, 2, 1)"); > execute("UPDATE %s SET s = 1 WHERE a = 1"); > execute("UPDATE %s SET s = 2 WHERE a = 2"); > execute("UPDATE %s SET s = 3 WHERE a = 3"); > assertRows(execute("SELECT a, b, c, d, s FROM %s WHERE a IN (1, 2, 3) > ORDER BY b DESC"), >row(2, 2, 2, 1, 2), >row(2, 2, 1, 1, 2), >row(1, 1, 2, 1, 1), >row(1, 1, 1, 1, 1), >row(3, null, null, null, 3)); > } > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10389) Repair session exception Validation failed
[ https://issues.apache.org/jira/browse/CASSANDRA-10389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108763#comment-15108763 ] Sylvain Lebresne commented on CASSANDRA-10389: -- bq. is there a known work around? yes, stop starting multiple competing repair simultaneously. > Repair session exception Validation failed > -- > > Key: CASSANDRA-10389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10389 > Project: Cassandra > Issue Type: Bug > Environment: Debian 8, Java 1.8.0_60, Cassandra 2.2.1 (datastax > compilation) >Reporter: Jędrzej Sieracki > Fix For: 2.2.x > > > I'm running a repair on a ring of nodes, that was recently extented from 3 to > 13 nodes. The extension was done two days ago, the repair was attempted > yesterday. > {quote} > [2015-09-22 11:55:55,266] Starting repair command #9, repairing keyspace > perspectiv with repair options (parallelism: parallel, primary range: false, > incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [], > hosts: [], # of ranges: 517) > [2015-09-22 11:55:58,043] Repair session 1f7c50c0-6110-11e5-b992-9f13fa8664c8 > for range (-5927186132136652665,-5917344746039874798] failed with error > [repair #1f7c50c0-6110-11e5-b992-9f13fa8664c8 on > perspectiv/stock_increment_agg, (-5927186132136652665,-5917344746039874798]] > Validation failed in cblade1.XXX/XXX (progress: 0%) > {quote} > BTW, I am ignoring the LEAK errors for now, that's outside of the scope of > the main issue: > {quote} > ERROR [Reference-Reaper:1] 2015-09-22 11:58:27,843 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@4d25ad8f) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@896826067:/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-73-big > was not released before the reference was garbage collected > {quote} > I scrubbed the sstable with failed validation on cblade1 with nodetool scrub > perspectiv stock_increment_agg: > {quote} > INFO [CompactionExecutor:1704] 2015-09-22 12:05:31,615 OutputHandler.java:42 > - Scrubbing > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big-Data.db') > (345466609 bytes) > INFO [CompactionExecutor:1703] 2015-09-22 12:05:31,615 OutputHandler.java:42 > - Scrubbing > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-82-big-Data.db') > (60496378 bytes) > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@4ca8951e) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@114161559:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-48-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@eeb6383) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1612685364:/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@1de90543) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@2058626950:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-49-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@15616385) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1386628428:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-47-big > was not released before the reference was garbage collected > INFO [CompactionExecutor:1703] 2015-09-22 12:05:35,098 OutputHandler.java:42 > - Scrub of > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-82-big-Data.db') > complete: 51397 rows in new sstable and 0 empty (tombstoned) rows dropped > INFO [CompactionExecutor:1704] 2015-09-22 12:05:47,605 OutputHandler.java:42 > - Scrub of > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big-Data.db') > complete: 292600 rows in new sstable and 0 empty (tombstoned) rows dropped > {quote} > Now, after
[jira] [Commented] (CASSANDRA-10848) Upgrade paging dtests involving deletion flap on CassCI
[ https://issues.apache.org/jira/browse/CASSANDRA-10848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108981#comment-15108981 ] Jim Witschey commented on CASSANDRA-10848: -- This seems to not be fixed despite CASSANDRA-10730 being resolved. It could be a manifestation of CASSANDRA-11016, but we should keep an eye on it even once that ticket is fixed. > Upgrade paging dtests involving deletion flap on CassCI > --- > > Key: CASSANDRA-10848 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10848 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey > Fix For: 3.0.x, 3.x > > > A number of dtests in the {{upgrade_tests.paging_tests}} that involve > deletion flap with the following error: > {code} > Requested pages were not delivered before timeout. > {code} > This may just be an effect of CASSANDRA-10730, but it's worth having a look > at separately. Here are some examples of tests flapping in this way: > http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/422/testReport/junit/upgrade_tests.paging_test/TestPagingWithDeletionsNodes2RF1/test_multiple_partition_deletions/ > http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/422/testReport/junit/upgrade_tests.paging_test/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11007) Exception when running nodetool info during bootstrap
[ https://issues.apache.org/jira/browse/CASSANDRA-11007?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108767#comment-15108767 ] Yuki Morishita commented on CASSANDRA-11007: ||branch||testall||dtest|| |[11007-2.1|https://github.com/yukim/cassandra/tree/11007-2.1]|[testall|http://cassci.datastax.com/view/Dev/view/yukim/job/yukim-11007-2.1-testall/lastCompletedBuild/testReport/]|[dtest|http://cassci.datastax.com/view/Dev/view/yukim/job/yukim-11007-2.1-dtest/lastCompletedBuild/testReport/]| |[11007-2.2|https://github.com/yukim/cassandra/tree/11007-2.2]|[testall|http://cassci.datastax.com/view/Dev/view/yukim/job/yukim-11007-2.2-testall/lastCompletedBuild/testReport/]|[dtest|http://cassci.datastax.com/view/Dev/view/yukim/job/yukim-11007-2.2-dtest/lastCompletedBuild/testReport/]| Backported to 2.1 and 2.2 as well. If tests are good, I will commit. > Exception when running nodetool info during bootstrap > - > > Key: CASSANDRA-11007 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11007 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: T Jake Luciani >Assignee: Yuki Morishita >Priority: Minor > Labels: fallout > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > > {code} > automaton@ip-172-31-26-150:~$ nodetool info > ID : 7d9aa832-de94-43ab-9548-b2e710418301 > Gossip active : true > Thrift active : false > Native Transport active: false > Load : 114.52 KB > Generation No : 1452721596 > Uptime (seconds) : 1396 > Heap Memory (MB) : 71.80 / 1842.00 > Off Heap Memory (MB) : 0.00 > Data Center: datacenter1 > Rack : rack1 > Exceptions : 0 > Key Cache : entries 11, size 888 bytes, capacity 92 MB, 180 > hits, 212 requests, 0.849 recent hit rate, 14400 save period in seconds > Row Cache : entries 0, size 0 bytes, capacity 0 bytes, 0 hits, 0 > requests, NaN recent hit rate, 0 save period in seconds > Counter Cache : entries 0, size 0 bytes, capacity 46 MB, 0 hits, 0 > requests, NaN recent hit rate, 7200 save period in seconds > error: null > -- StackTrace -- > java.lang.AssertionError > at > org.apache.cassandra.locator.TokenMetadata.getTokens(TokenMetadata.java:488) > at > org.apache.cassandra.service.StorageService.getTokens(StorageService.java:2561) > at > org.apache.cassandra.service.StorageService.getTokens(StorageService.java:2550) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71) > at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275) > at > com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) > at > com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) > at > com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) > at > com.sun.jmx.mbeanserver.PerInterface.getAttribute(PerInterface.java:83) > at > com.sun.jmx.mbeanserver.MBeanSupport.getAttribute(MBeanSupport.java:206) > at > com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttribute(DefaultMBeanServerInterceptor.java:647) > at > com.sun.jmx.mbeanserver.JmxMBeanServer.getAttribute(JmxMBeanServer.java:678) > at > javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1443) > at > javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:76) > at > javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1307) > at > javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1399) > at > javax.management.remote.rmi.RMIConnectionImpl.getAttribute(RMIConnectionImpl.java:637) > at sun.reflect.GeneratedMethodAccessor6.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:323) > at sun.rmi.transport.Transport$1.run(Transport.java:200) >
[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling
[ https://issues.apache.org/jira/browse/CASSANDRA-10070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108704#comment-15108704 ] Marcus Olsson commented on CASSANDRA-10070: --- I completely agree, I should create a document describing these things. I've also thought about making a high level document for the whole proposal, so as to see if everyone agrees that this is the way to go about the distributed scheduling. Then we can take it from there and revise the proposal and hopefully later on break the JIRA into several tasks to make it easier to review and develop this feature. I think this document should contain: * High level description of proposal (flow charts, etc.) * Problems that could occur and possible solutions Any thoughts or ideas on this? > Automatic repair scheduling > --- > > Key: CASSANDRA-10070 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10070 > Project: Cassandra > Issue Type: Improvement >Reporter: Marcus Olsson >Assignee: Marcus Olsson >Priority: Minor > Fix For: 3.x > > > Scheduling and running repairs in a Cassandra cluster is most often a > required task, but this can both be hard for new users and it also requires a > bit of manual configuration. There are good tools out there that can be used > to simplify things, but wouldn't this be a good feature to have inside of > Cassandra? To automatically schedule and run repairs, so that when you start > up your cluster it basically maintains itself in terms of normal > anti-entropy, with the possibility for manual configuration. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10446) Run repair with down replicas
[ https://issues.apache.org/jira/browse/CASSANDRA-10446?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108701#comment-15108701 ] Yuki Morishita commented on CASSANDRA-10446: You can still use '-hosts' repair option to specify which hosts to repair. You can just give live nodes like 'nodetool repair -hosts node1 -hosts node2 -hosts node3', and cassandra will repair among those nodes. > Run repair with down replicas > - > > Key: CASSANDRA-10446 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10446 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Priority: Minor > Fix For: 3.x > > > We should have an option of running repair when replicas are down. We can > call it -force. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11043) Secondary indexes doesn't properly validate custom expressions
[ https://issues.apache.org/jira/browse/CASSANDRA-11043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrés de la Peña updated CASSANDRA-11043: -- Component/s: CQL > Secondary indexes doesn't properly validate custom expressions > -- > > Key: CASSANDRA-11043 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11043 > Project: Cassandra > Issue Type: Bug > Components: CQL, Local Write-Read Paths >Reporter: Andrés de la Peña > Labels: 21, index, validation > Attachments: test-index.zip > > > It seems that > [CASSANDRA-7575|https://issues.apache.org/jira/browse/CASSANDRA-7575] is > broken in Cassandra 3.x. As stated in the secondary indexes' API > documentation, custom index implementations should perform any validation of > query expressions at {{Index#searcherFor(ReadCommand)}}, throwing an > {{InvalidRequestException}} if the expressions are not valid. I assume these > validation errors should produce an {{InvalidRequest}} error on cqlsh, or > raise an {{InvalidQueryException}} on Java driver. However, when > {{Index#searcherFor(ReadCommand)}} throws its {{InvalidRequestException}}, I > get this cqlsh output: > {noformat} > Traceback (most recent call last): > File "bin/cqlsh.py", line 1246, in perform_simple_statement > result = future.result() > File > "/Users/adelapena/stratio/platform/src/cassandra-3.2.1/bin/../lib/cassandra-driver-internal-only-3.0.0-6af642d.zip/cassandra-driver-3.0.0-6af642d/cassandra/cluster.py", > line 3122, in result > raise self._final_exception > ReadFailure: code=1300 [Replica(s) failed to execute read] message="Operation > failed - received 0 responses and 1 failures" info={'failures': 1, > 'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'} > {noformat} > I attach a dummy index implementation to reproduce the error: > {noformat} > CREATE KEYSPACE test with replication = {'class' : 'SimpleStrategy', > 'replication_factor' : '1' }; > CREATE TABLE test.test (id int PRIMARY KEY, value varchar); > CREATE CUSTOM INDEX test_index ON test.test() USING 'com.stratio.TestIndex'; > SELECT * FROM test.test WHERE expr(test_index,'ok'); > SELECT * FROM test.test WHERE expr(test_index,'error'); > {noformat} > This is specially problematic when using Cassandra Java Driver, because one > of these server exceptions can produce subsequent queries fail (even if they > are valid) with a no host available exception. > Maybe the validation method added with > [CASSANDRA-7575|https://issues.apache.org/jira/browse/CASSANDRA-7575] should > be restored, unless there is a way to properly manage the exception. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-10913) netstats_test dtest flaps
[ https://issues.apache.org/jira/browse/CASSANDRA-10913?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jim Witschey resolved CASSANDRA-10913. -- Resolution: Fixed We will keep an eye on this test in weekly review, but it looks like these tests are running clean now. Closing. > netstats_test dtest flaps > - > > Key: CASSANDRA-10913 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10913 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey > Fix For: 3.0.x > > > {{jmx_test.py:TestJMX.netstats_test}} flaps on 2.2: > http://cassci.datastax.com/job/cassandra-2.2_dtest/lastSuccessfulBuild/testReport/jmx_test/TestJMX/netstats_test/history/ > 3.0: > http://cassci.datastax.com/job/cassandra-3.0_dtest/lastSuccessfulBuild/testReport/jmx_test/TestJMX/netstats_test/history/ > and trunk: > http://cassci.datastax.com/job/trunk_dtest/lastSuccessfulBuild/testReport/jmx_test/TestJMX/netstats_test/history/ > The connection over JMX times out after 30 seconds. We may be increasing the > size of the instances we run on CassCI, in which case these timeouts may go > away, so I don't think there's anything we should do just yet; we should just > keep an eye on this going forward. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10868) Skip supercolumns upgrade tests on jdk8
[ https://issues.apache.org/jira/browse/CASSANDRA-10868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108943#comment-15108943 ] Jim Witschey commented on CASSANDRA-10868: -- [~rhatch] does Philip's proposal make sense? I just want to make sure we aren't missing something obvious. > Skip supercolumns upgrade tests on jdk8 > --- > > Key: CASSANDRA-10868 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10868 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey >Assignee: Jim Witschey > > The tests in the {{upgrade_supercolumns_test}} dtest module fail when we test > on JDK8 because they attempt to upgrade from 2.0, which will not compile on > JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/160/testReport/upgrade_supercolumns_test/ > [~rhatch] As we look at how we want to run upgrade tests in the future, we > should consider this. In the meantime, I think the best way to deal with this > might be to add something to the exclude files in {{conf/}}. That sound > reasonable, or is there a better way to do this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11030) utf-8 characters incorrectly displayed/inserted on cqlsh on Windows
[ https://issues.apache.org/jira/browse/CASSANDRA-11030?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108798#comment-15108798 ] Joshua McKenzie commented on CASSANDRA-11030: - Works on Win10. > utf-8 characters incorrectly displayed/inserted on cqlsh on Windows > --- > > Key: CASSANDRA-11030 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11030 > Project: Cassandra > Issue Type: Bug >Reporter: Paulo Motta >Assignee: Paulo Motta >Priority: Minor > Labels: cqlsh, windows > > {noformat} > C:\Users\Paulo\Repositories\cassandra [2.2-10948 +6 ~1 -0 !]> .\bin\cqlsh.bat > --encoding utf-8 > Connected to test at 127.0.0.1:9042. > [cqlsh 5.0.1 | Cassandra 2.2.4-SNAPSHOT | CQL spec 3.3.1 | Native protocol v4] > Use HELP for help. > cqlsh> INSERT INTO bla.test (bla ) VALUES ('não') ; > cqlsh> select * from bla.test; > bla > - > n?o > (1 rows) > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10870) pushed_notifications_test.py:TestPushedNotifications.restart_node_test flapping on C* 2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-10870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108896#comment-15108896 ] Jim Witschey commented on CASSANDRA-10870: -- Also, ping [~aboudreault]; I think you know more about this test than anyone. Do you see any obvious possible problems? > pushed_notifications_test.py:TestPushedNotifications.restart_node_test > flapping on C* 2.1 > - > > Key: CASSANDRA-10870 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10870 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: DS Test Eng > Fix For: 2.1.x > > > This test flaps on CassCI on 2.1. [~aboudreault] Do I remember correctly that > you did some work on these tests in the past few months? If so, could you > have a look and see if there's some assumption the test makes that don't hold > for 2.1? > Oddly, it fails frequently under JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ > but less frequently on JDK7: > http://cassci.datastax.com/job/cassandra-2.1_dtest/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-4430) optional pluggable o.a.c.metrics reporters
[ https://issues.apache.org/jira/browse/CASSANDRA-4430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108842#comment-15108842 ] Chris Burroughs commented on CASSANDRA-4430: * Ease of use, fewer moving parts with polling intervals to reason about * Ease of configuration, you don't have to configure another thing to know all about your cassandra clusters and all about ganglia. Please take further 'how to use it' discussion to the users list. > optional pluggable o.a.c.metrics reporters > -- > > Key: CASSANDRA-4430 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4430 > Project: Cassandra > Issue Type: Improvement >Reporter: Chris Burroughs >Assignee: Chris Burroughs >Priority: Minor > Fix For: 2.0.2 > > Attachments: 4430-2.0.txt, 4430-trunk.txt, > cassandra-ganglia-example.png > > > CASSANDRA-4009 expanded the use of the metrics library which has a set of > reporter modules http://metrics.codahale.com/manual/core/#reporters You can > report to flat files, ganglia, spit everything over http, etc. The next step > is a mechanism for using those reporters with o.a.c.metrics. To avoid > bundling everything I suggest following the mx4j approach of "enable only if > on classpath coupled with a reporter configuration file. > Strawman file: > {noformat} > console: > time: 1 > timeunit: "seconds" > csv: > - time: 1 >timeunit: minutes >file: foo.csv > - time: 10 >timeunit: seconds > file: bar.csv > ganglia: > - time: 30 >timunit: seconds >host: server-1 >port: 8649 > - time: 30 >timunit: seconds >host: server-2 >port: 8649 > {noformat} > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10870) pushed_notifications_test.py:TestPushedNotifications.restart_node_test flapping on C* 2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-10870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108879#comment-15108879 ] Jim Witschey commented on CASSANDRA-10870: -- Assigning to the DataStax C* TE team for debugging. > pushed_notifications_test.py:TestPushedNotifications.restart_node_test > flapping on C* 2.1 > - > > Key: CASSANDRA-10870 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10870 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey > Fix For: 2.1.x > > > This test flaps on CassCI on 2.1. [~aboudreault] Do I remember correctly that > you did some work on these tests in the past few months? If so, could you > have a look and see if there's some assumption the test makes that don't hold > for 2.1? > Oddly, it fails frequently under JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ > but less frequently on JDK7: > http://cassci.datastax.com/job/cassandra-2.1_dtest/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10870) pushed_notifications_test.py:TestPushedNotifications.restart_node_test flapping on C* 2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-10870?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jim Witschey updated CASSANDRA-10870: - Assignee: DS Test Eng > pushed_notifications_test.py:TestPushedNotifications.restart_node_test > flapping on C* 2.1 > - > > Key: CASSANDRA-10870 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10870 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: DS Test Eng > Fix For: 2.1.x > > > This test flaps on CassCI on 2.1. [~aboudreault] Do I remember correctly that > you did some work on these tests in the past few months? If so, could you > have a look and see if there's some assumption the test makes that don't hold > for 2.1? > Oddly, it fails frequently under JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ > but less frequently on JDK7: > http://cassci.datastax.com/job/cassandra-2.1_dtest/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109206#comment-15109206 ] Benjamin Zarzycki commented on CASSANDRA-11037: --- [~JoshuaMcKenzie] Wouldn't a windows environment be running the cqlsh.bat file and not the cqlsh bash script? > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10843) compaction data_size_tests are failing
[ https://issues.apache.org/jira/browse/CASSANDRA-10843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109216#comment-15109216 ] Philip Thompson commented on CASSANDRA-10843: - +1 > compaction data_size_tests are failing > -- > > Key: CASSANDRA-10843 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10843 > Project: Cassandra > Issue Type: Sub-task > Components: Compaction, Testing >Reporter: Philip Thompson >Assignee: Marcus Eriksson > Fix For: 2.1.x, 2.2.x > > > {{compaction_test.TestCompaction_with_LeveledCompactionStrategy.data_size_test}} > as well as the data_size tests for STCS and DTCS are failing. See > http://cassci.datastax.com/job/cassandra-2.1_dtest/373/testReport/ > These tests write out some data, check live size, compact, then check live > size again. On 2.1 and 2.2, we are seeing an increase in live size after the > compaction. This is not occurring on 3.0. I cannot reproduce locally on OSX. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9752) incremental repair dtest flaps on 2.2
[ https://issues.apache.org/jira/browse/CASSANDRA-9752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109245#comment-15109245 ] Jim Witschey commented on CASSANDRA-9752: - [~philipthompson] Sounds right, we've got 30G memory to play with. Do we know how many nodes the dtests top out at? I doubt it's above 10. > incremental repair dtest flaps on 2.2 > -- > > Key: CASSANDRA-9752 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9752 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey > > {{incremental_repair_test.py:TestIncRepair.multiple_subsequent_repair_test}} > flaps on 2.2. It's hard to tell what failures are repair-specific, but there > are a few distinct failures I've seen recently: > - [an NPE in > StorageService|http://cassci.datastax.com/view/cassandra-2.2/job/cassandra-2.2_dtest/143/testReport/junit/incremental_repair_test/TestIncRepair/multiple_subsequent_repair_test/] > - [an NPE in > SSTableRewriter|http://cassci.datastax.com/view/cassandra-2.2/job/cassandra-2.2_dtest/135/testReport/junit/incremental_repair_test/TestIncRepair/multiple_subsequent_repair_test/]. > I believe this is related to CASSANDRA-9730, but someone should confirm this. > - [an on-disk data size that is too > large|http://cassci.datastax.com/view/cassandra-2.2/job/cassandra-2.2_dtest/133/testReport/junit/incremental_repair_test/TestIncRepair/multiple_subsequent_repair_test/] > You can find the test itself [here on > GitHub|https://github.com/riptano/cassandra-dtest/blob/master/incremental_repair_test.py#L206] > and run it with the command > {code} > CASSANDRA_VERSION=git:trunk nosetests > incremental_repair_test.py:TestIncRepair.multiple_subsequent_repair_test > {code} > Assigning [~yukim], since you're the repair person, but feel free to reassign > to whoever's appropriate. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10428) cqlsh: Include sub-second precision in timestamps by default
[ https://issues.apache.org/jira/browse/CASSANDRA-10428?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109277#comment-15109277 ] Tyler Hobbs commented on CASSANDRA-10428: - I tested {{%f}} on pypy, and it works as expected. I think it will be extremely rare for anybody to use cqlsh with Jython, so detecting Jython and omitting the {{%f}} in that case seems like a good approach. You can detect Jython with something like the following: {code} import platform is_jython = platform.python_implementation().startswith('Jython') {code} > cqlsh: Include sub-second precision in timestamps by default > > > Key: CASSANDRA-10428 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10428 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: OSX 10.10.2 >Reporter: Chandran Anjur Narasimhan >Assignee: Stefania > Labels: cqlsh > Fix For: 3.x > > > Query with >= timestamp works. But the exact timestamp value is not working. > {noformat} > NCHAN-M-D0LZ:bin nchan$ ./cqlsh > Connected to CCC Multi-Region Cassandra Cluster at :. > [cqlsh 5.0.1 | Cassandra 2.1.7 | CQL spec 3.2.0 | Native protocol v3] > Use HELP for help. > cqlsh> > {noformat} > {panel:title=Schema|borderStyle=dashed|borderColor=#ccc|titleBGColor=#F7D6C1|bgColor=#CE} > cqlsh:ccc> desc COLUMNFAMILY ez_task_result ; > CREATE TABLE ccc.ez_task_result ( > submissionid text, > ezid text, > name text, > time timestamp, > analyzed_index_root text, > ... > ... > PRIMARY KEY (submissionid, ezid, name, time) > {panel} > {panel:title=Working|borderStyle=dashed|borderColor=#ccc|titleBGColor=#F7D6C1|bgColor=#CE} > cqlsh:ccc> select submissionid, ezid, name, time, state, status, > translated_criteria_status from ez_task_result where > submissionid='760dd154670811e58c04005056bb6ff0' and > ezid='760dd6de670811e594fc005056bb6ff0' and name='run-sanities' and > time>='2015-09-29 20:54:23-0700'; > submissionid | ezid | name > | time | state | status | > translated_criteria_status > --+--+--+--+---+-+ > 760dd154670811e58c04005056bb6ff0 | 760dd6de670811e594fc005056bb6ff0 | > run-sanities | 2015-09-29 20:54:23-0700 | EXECUTING | IN_PROGRESS | > run-sanities started > (1 rows) > cqlsh:ccc> > {panel} > {panel:title=Not > working|borderStyle=dashed|borderColor=#ccc|titleBGColor=#F7D6C1|bgColor=#CE} > cqlsh:ccc> select submissionid, ezid, name, time, state, status, > translated_criteria_status from ez_task_result where > submissionid='760dd154670811e58c04005056bb6ff0' and > ezid='760dd6de670811e594fc005056bb6ff0' and name='run-sanities' and > time='2015-09-29 20:54:23-0700'; > submissionid | ezid | name | time | analyzed_index_root | analyzed_log_path > | clientid | end_time | jenkins_path | log_file_path | path_available | > path_to_task | required_for_overall_status | start_time | state | status | > translated_criteria_status | type > --+--+--+--+-+---+--+--+--+---++--+-++---+++-- > (0 rows) > cqlsh:ccc> > {panel} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109181#comment-15109181 ] Brian Hess commented on CASSANDRA-10707: - Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max(x) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, x INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum(x) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109180#comment-15109180 ] Joshua McKenzie commented on CASSANDRA-11037: - It's not a safe assumption that perl is going to be installed on a Windows environment, unfortunately. > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109181#comment-15109181 ] Brian Hess edited comment on CASSANDRA-10707 at 1/20/16 7:04 PM: -- Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max(x) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, x INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum(x) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. So, the assumption that the data will be sorted when it arrives to the coordinator might not be true in all cases. was (Author: brianmhess): Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max(x) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, x INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum(x) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeremiah Jordan updated CASSANDRA-10979: Fix Version/s: 3.0.x 2.2.x > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland >Assignee: Carl Yeksigian > Labels: compaction, leveled > Fix For: 2.2.x, 3.0.x, 3.x > > Attachments: 10979-2.1.txt > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109177#comment-15109177 ] Jeff Ferland commented on CASSANDRA-10979: -- Applied the patch to a node that had just come up from streaming and was 700+ tables in L0. After restart, observed that as L0 was shifted into L1, L0 continued to compact new tables to prevent the extreme growth of tables. Thank you. Again still requesting a merge into 2.1 since streaming and repair with LCS are practically broken for us without this patch. > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland >Assignee: Carl Yeksigian > Labels: compaction, leveled > Fix For: 3.x > > Attachments: 10979-2.1.txt > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109181#comment-15109181 ] Brian Hess edited comment on CASSANDRA-10707 at 1/20/16 7:07 PM: -- Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max( x ) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, x INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum( x ) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. So, the assumption that the data will be sorted when it arrives to the coordinator might not be true in all cases. was (Author: brianmhess): Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max(y) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, y INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum(y) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. So, the assumption that the data will be sorted when it arrives to the coordinator might not be true in all cases. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10907) Nodetool snapshot should provide an option to skip flushing
[ https://issues.apache.org/jira/browse/CASSANDRA-10907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paulo Motta updated CASSANDRA-10907: Assignee: Anubhav Kale > Nodetool snapshot should provide an option to skip flushing > --- > > Key: CASSANDRA-10907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10907 > Project: Cassandra > Issue Type: Improvement > Components: Configuration > Environment: PROD >Reporter: Anubhav Kale >Assignee: Anubhav Kale >Priority: Minor > Labels: lhf > Attachments: 0001-Skip-Flush-for-snapshots.patch, > 0001-Skip-Flush-option-for-Snapshot.patch, > 0001-Skip-Flush-option-for-Snapshot.patch, 0001-flush.patch > > > For some practical scenarios, it doesn't matter if the data is flushed to > disk before taking a snapshot. However, it's better to save some flushing > time to make snapshot process quick. > As such, it will be a good idea to provide this option to snapshot command. > The wiring from nodetool to MBean to VerbHandler should be easy. > I can provide a patch if this makes sense. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11048) JSON queries are not thread safe
[ https://issues.apache.org/jira/browse/CASSANDRA-11048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ivan Ryndin updated CASSANDRA-11048: Attachment: 0001-Fix-thread-unsafe-usage-of-JsonStringEncoder-see-CAS.patch patch file submitted > JSON queries are not thread safe > > > Key: CASSANDRA-11048 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11048 > Project: Cassandra > Issue Type: Bug >Reporter: Sergio Bossa >Priority: Critical > Labels: easyfix, newbie, patch > Attachments: > 0001-Fix-thread-unsafe-usage-of-JsonStringEncoder-see-CAS.patch > > > {{org.apache.cassandra.cql3.Json}} uses a shared instance of > {{JsonStringEncoder}} which is not thread safe (see 1), while > {{JsonStringEncoder#getInstance()}} should be used (see 2). > As a consequence, concurrent {{select JSON}} queries often produce wrong > (sometimes unreadable) results. > 1. > http://grepcode.com/file/repo1.maven.org/maven2/org.codehaus.jackson/jackson-core-asl/1.9.2/org/codehaus/jackson/io/JsonStringEncoder.java > 2. > http://grepcode.com/file/repo1.maven.org/maven2/org.codehaus.jackson/jackson-core-asl/1.9.2/org/codehaus/jackson/io/JsonStringEncoder.java#JsonStringEncoder.getInstance%28%29 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109206#comment-15109206 ] Benjamin Zarzycki edited comment on CASSANDRA-11037 at 1/20/16 7:12 PM: [~JoshuaMcKenzie] Wouldn't a windows environment be executing the cqlsh.bat file and not the cqlsh bash script? was (Author: kf6nux): [~JoshuaMcKenzie] Wouldn't a windows environment be running the cqlsh.bat file and not the cqlsh bash script? > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10907) Nodetool snapshot should provide an option to skip flushing
[ https://issues.apache.org/jira/browse/CASSANDRA-10907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109209#comment-15109209 ] Paulo Motta commented on CASSANDRA-10907: - LGTM, thanks! > Nodetool snapshot should provide an option to skip flushing > --- > > Key: CASSANDRA-10907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10907 > Project: Cassandra > Issue Type: Improvement > Components: Configuration > Environment: PROD >Reporter: Anubhav Kale >Priority: Minor > Labels: lhf > Attachments: 0001-Skip-Flush-for-snapshots.patch, > 0001-Skip-Flush-option-for-Snapshot.patch, > 0001-Skip-Flush-option-for-Snapshot.patch, 0001-flush.patch > > > For some practical scenarios, it doesn't matter if the data is flushed to > disk before taking a snapshot. However, it's better to save some flushing > time to make snapshot process quick. > As such, it will be a good idea to provide this option to snapshot command. > The wiring from nodetool to MBean to VerbHandler should be easy. > I can provide a patch if this makes sense. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9752) incremental repair dtest flaps on 2.2
[ https://issues.apache.org/jira/browse/CASSANDRA-9752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109221#comment-15109221 ] Philip Thompson commented on CASSANDRA-9752: I believe we have the RAM available to bump those values up. I will try that and see if it resolves the test. > incremental repair dtest flaps on 2.2 > -- > > Key: CASSANDRA-9752 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9752 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey > > {{incremental_repair_test.py:TestIncRepair.multiple_subsequent_repair_test}} > flaps on 2.2. It's hard to tell what failures are repair-specific, but there > are a few distinct failures I've seen recently: > - [an NPE in > StorageService|http://cassci.datastax.com/view/cassandra-2.2/job/cassandra-2.2_dtest/143/testReport/junit/incremental_repair_test/TestIncRepair/multiple_subsequent_repair_test/] > - [an NPE in > SSTableRewriter|http://cassci.datastax.com/view/cassandra-2.2/job/cassandra-2.2_dtest/135/testReport/junit/incremental_repair_test/TestIncRepair/multiple_subsequent_repair_test/]. > I believe this is related to CASSANDRA-9730, but someone should confirm this. > - [an on-disk data size that is too > large|http://cassci.datastax.com/view/cassandra-2.2/job/cassandra-2.2_dtest/133/testReport/junit/incremental_repair_test/TestIncRepair/multiple_subsequent_repair_test/] > You can find the test itself [here on > GitHub|https://github.com/riptano/cassandra-dtest/blob/master/incremental_repair_test.py#L206] > and run it with the command > {code} > CASSANDRA_VERSION=git:trunk nosetests > incremental_repair_test.py:TestIncRepair.multiple_subsequent_repair_test > {code} > Assigning [~yukim], since you're the repair person, but feel free to reassign > to whoever's appropriate. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10974) disk_balance_bootstrap_test is failing on trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-10974?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109239#comment-15109239 ] Philip Thompson commented on CASSANDRA-10974: - [~krummas], why the change to the datadir count? https://github.com/riptano/cassandra-dtest/commit/18f2f684e82c0e807a8068f73d53645991c568a8#diff-1ef92939c7765f8c4041bada71208eebL61 > disk_balance_bootstrap_test is failing on trunk > --- > > Key: CASSANDRA-10974 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10974 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Alan Boudreault >Assignee: Marcus Eriksson > Labels: dtest > > http://cassci.datastax.com/job/trunk_dtest/891/testReport/junit/disk_balance_test/TestDiskBalance/disk_balance_bootstrap_test/ > {code} > == > FAIL: disk_balance_bootstrap_test (disk_balance_test.TestDiskBalance) > -- > Traceback (most recent call last): > File "/home/aboudreault/git/cstar/cassandra-dtest/disk_balance_test.py", > line 51, in disk_balance_bootstrap_test > self.assert_balanced(node) > File "/home/aboudreault/git/cstar/cassandra-dtest/disk_balance_test.py", > line 127, in assert_balanced > assert_almost_equal(*sums, error=0.2, error_message=node.name) > File "/home/aboudreault/git/cstar/cassandra-dtest/assertions.py", line 65, > in assert_almost_equal > assert vmin > vmax * (1.0 - error) or vmin == vmax, "values not within > %.2f%% of the max: %s (%s)" % (error * 100, args, kwargs['error_message']) > AssertionError: values not within 20.00% of the max: (529955, 386060, 473640) > (node4) > >> begin captured logging << > dtest: DEBUG: cluster ccm directory: /tmp/dtest-nNoQzp > - >> end captured logging << - > -- > Ran 1 test in 114.862s > FAILED (failures=1) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109181#comment-15109181 ] Brian Hess edited comment on CASSANDRA-10707 at 1/20/16 7:06 PM: -- Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max(y) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, y INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum(y) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. So, the assumption that the data will be sorted when it arrives to the coordinator might not be true in all cases. was (Author: brianmhess): Correct, what [~iamaleksey] said. In fact, pushing the aggregate computation to the replicas is troublesome at an RF>1. Quick follow up - will this ticket also cover: SELECT clusterCol, Max(x) FROM myData GROUP BY clusterCol; That is, you group on a clustering column, but not on a partition key? Second question - consider a table with schema myData(partitionKey INT, clusteringCol1 INT, clusteringCol2 INT, x INT, PRIMARY KEY ((partitionKey), clusteringCol1, clusteringCol2). Now, will the following query be supported: SELECT partitionKey, clusteringCol2, Sum(x) FROM myData GROUP BY partitionKey, clusteringCol2; The reason I ask is that the following is not supported: SELECT partitionKey, clusteringCol2, x FROM myData WHERE partitionKey=5 ORDER BY clusteringCol2; Because you cannot order by clusteringCol2, only clusteringCol1. So, the assumption that the data will be sorted when it arrives to the coordinator might not be true in all cases. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-10613) Upgrade test on 2.1->3.0 path fails with NPE in getExistingFiles (likely known bug)
[ https://issues.apache.org/jira/browse/CASSANDRA-10613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russ Hatch resolved CASSANDRA-10613. Resolution: Not A Problem looks like no longer occurring. > Upgrade test on 2.1->3.0 path fails with NPE in getExistingFiles (likely > known bug) > --- > > Key: CASSANDRA-10613 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10613 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: Russ Hatch > Fix For: 3.0.x > > > In this job: > http://cassci.datastax.com/view/Upgrades/job/cassandra_upgrade_2.1_to_3.0_proto_v3/10/ > The following tests fail due to an NPE in > {{org.apache.cassandra.db.lifecycle.LogRecord.getExistingFiles}}: > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.bootstrap_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.rolling_upgrade_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.parallel_upgrade_with_internode_ssl_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.rolling_upgrade_with_internode_ssl_test > upgrade_through_versions_test.py:TestUpgrade_from_cassandra_2_1_HEAD_to_cassandra_3_0_HEAD.rolling_upgrade_with_internode_ssl_test > upgrade_through_versions_test.py:TestUpgrade_from_3_0_latest_tag_to_3_0_HEAD.parallel_upgrade_test > I believe this is likely happening because of CASSANDRA-10602, so let's hold > off on messing with this until that's merged. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10909) NPE in ActiveRepairService
[ https://issues.apache.org/jira/browse/CASSANDRA-10909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109319#comment-15109319 ] Carl Yeksigian commented on CASSANDRA-10909: +1 I think the tests look good modulo whatever failures are expected. > NPE in ActiveRepairService > --- > > Key: CASSANDRA-10909 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10909 > Project: Cassandra > Issue Type: Bug > Environment: cassandra-3.0.1.777 >Reporter: Eduard Tudenhoefner >Assignee: Marcus Eriksson > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > > NPE after one started multiple incremental repairs > {code} > INFO [Thread-62] 2015-12-21 11:40:53,742 RepairRunnable.java:125 - Starting > repair command #1, repairing keyspace keyspace1 with repair options > (parallelism: parallel, primary range: false, incremental: true, job threads: > 1, ColumnFamilies: [], dataCenters: [], hosts: [], # of ranges: 2) > INFO [Thread-62] 2015-12-21 11:40:53,813 RepairSession.java:237 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] new session: will sync /10.200.177.32, > /10.200.177.33 on range [(10,-9223372036854775808]] for keyspace1.[counter1, > standard1] > INFO [Repair#1:1] 2015-12-21 11:40:53,853 RepairJob.java:100 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] requesting merkle trees for counter1 > (to [/10.200.177.33, /10.200.177.32]) > INFO [Repair#1:1] 2015-12-21 11:40:53,853 RepairJob.java:174 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] Requesting merkle trees for counter1 > (to [/10.200.177.33, /10.200.177.32]) > INFO [Thread-62] 2015-12-21 11:40:53,854 RepairSession.java:237 - [repair > #b1449fe0-a7d7-11e5-b568-f565b837eb0d] new session: will sync /10.200.177.32, > /10.200.177.31 on range [(0,10]] for keyspace1.[counter1, standard1] > INFO [AntiEntropyStage:1] 2015-12-21 11:40:53,896 RepairSession.java:181 - > [repair #b13e3740-a7d7-11e5-b568-f565b837eb0d] Received merkle tree for > counter1 from /10.200.177.32 > INFO [AntiEntropyStage:1] 2015-12-21 11:40:53,906 RepairSession.java:181 - > [repair #b13e3740-a7d7-11e5-b568-f565b837eb0d] Received merkle tree for > counter1 from /10.200.177.33 > INFO [Repair#1:1] 2015-12-21 11:40:53,906 RepairJob.java:100 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] requesting merkle trees for standard1 > (to [/10.200.177.33, /10.200.177.32]) > INFO [Repair#1:1] 2015-12-21 11:40:53,906 RepairJob.java:174 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] Requesting merkle trees for standard1 > (to [/10.200.177.33, /10.200.177.32]) > INFO [RepairJobTask:2] 2015-12-21 11:40:53,910 SyncTask.java:66 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] Endpoints /10.200.177.33 and > /10.200.177.32 are consistent for counter1 > INFO [RepairJobTask:1] 2015-12-21 11:40:53,910 RepairJob.java:145 - [repair > #b13e3740-a7d7-11e5-b568-f565b837eb0d] counter1 is fully synced > INFO [AntiEntropyStage:1] 2015-12-21 11:40:54,823 Validator.java:272 - > [repair #b17a2ed0-a7d7-11e5-ada8-8304f5629908] Sending completed merkle tree > to /10.200.177.33 for keyspace1.counter1 > ERROR [ValidationExecutor:3] 2015-12-21 11:40:55,104 > CompactionManager.java:1065 - Cannot start multiple repair sessions over the > same sstables > ERROR [ValidationExecutor:3] 2015-12-21 11:40:55,105 Validator.java:259 - > Failed creating a merkle tree for [repair > #b17a2ed0-a7d7-11e5-ada8-8304f5629908 on keyspace1/standard1, > [(10,-9223372036854775808]]], /10.200.177.33 (see log for details) > ERROR [ValidationExecutor:3] 2015-12-21 11:40:55,110 > CassandraDaemon.java:195 - Exception in thread > Thread[ValidationExecutor:3,1,main] > java.lang.RuntimeException: Cannot start multiple repair sessions over the > same sstables > at > org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1066) > ~[cassandra-all-3.0.1.777.jar:3.0.1.777] > at > org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:80) > ~[cassandra-all-3.0.1.777.jar:3.0.1.777] > at > org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:679) > ~[cassandra-all-3.0.1.777.jar:3.0.1.777] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ~[na:1.8.0_40] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > ~[na:1.8.0_40] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_40] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_40] > ERROR [AntiEntropyStage:1] 2015-12-21 11:40:55,174 > RepairMessageVerbHandler.java:161 - Got error, removing parent repair session > INFO [CompactionExecutor:3] 2015-12-21 11:40:55,175 > CompactionManager.java:489 - Starting
[jira] [Commented] (CASSANDRA-10707) Add support for Group By to Select statement
[ https://issues.apache.org/jira/browse/CASSANDRA-10707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109333#comment-15109333 ] Benjamin Lerer commented on CASSANDRA-10707: Those queries will not be supported by this ticket. If such a query is required, it is possible to use a materialized view to reorganized the data in order to allow the group by query. > Add support for Group By to Select statement > > > Key: CASSANDRA-10707 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10707 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer > > Now that Cassandra support aggregate functions, it makes sense to support > {{GROUP BY}} on the {{SELECT}} statements. > It should be possible to group either at the partition level or at the > clustering column level. > {code} > SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey; > SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP > BY partitionKey, clustering0, clustering1; > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10974) disk_balance_bootstrap_test is failing on trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-10974?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109345#comment-15109345 ] Marcus Eriksson commented on CASSANDRA-10974: - [~philipthompson] we default to 3 datadirs now, we didn't when I wrote the test > disk_balance_bootstrap_test is failing on trunk > --- > > Key: CASSANDRA-10974 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10974 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Alan Boudreault >Assignee: Marcus Eriksson > Labels: dtest > > http://cassci.datastax.com/job/trunk_dtest/891/testReport/junit/disk_balance_test/TestDiskBalance/disk_balance_bootstrap_test/ > {code} > == > FAIL: disk_balance_bootstrap_test (disk_balance_test.TestDiskBalance) > -- > Traceback (most recent call last): > File "/home/aboudreault/git/cstar/cassandra-dtest/disk_balance_test.py", > line 51, in disk_balance_bootstrap_test > self.assert_balanced(node) > File "/home/aboudreault/git/cstar/cassandra-dtest/disk_balance_test.py", > line 127, in assert_balanced > assert_almost_equal(*sums, error=0.2, error_message=node.name) > File "/home/aboudreault/git/cstar/cassandra-dtest/assertions.py", line 65, > in assert_almost_equal > assert vmin > vmax * (1.0 - error) or vmin == vmax, "values not within > %.2f%% of the max: %s (%s)" % (error * 100, args, kwargs['error_message']) > AssertionError: values not within 20.00% of the max: (529955, 386060, 473640) > (node4) > >> begin captured logging << > dtest: DEBUG: cluster ccm directory: /tmp/dtest-nNoQzp > - >> end captured logging << - > -- > Ran 1 test in 114.862s > FAILED (failures=1) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11049) Conflicting definitions of VERSION string when compiling cassandra_constants
Peter Butler created CASSANDRA-11049: Summary: Conflicting definitions of VERSION string when compiling cassandra_constants Key: CASSANDRA-11049 URL: https://issues.apache.org/jira/browse/CASSANDRA-11049 Project: Cassandra Issue Type: Bug Components: Configuration Reporter: Peter Butler When compiling c++ code, as generated via thrift (latest version of thrift from git repository as of January 19 2016), there are conflicting versions of the VERSION constant in cassandra_constant.* The specific error is: cassandra_constants.h:18:15: error: expected unqualified-id before string constant cassandra_constants.cpp: In constructor ‘org::apache::cassandra::cassandraConstants::cassandraConstants()’: cassandra_constants.cpp:14:17: error: assignment of read-only location ‘"1.0.0-dev"’ cassandra_constants.cpp:14:17: error: incompatible types in assignment of ‘const char [8]’ to ‘const char [10]’ The location of the original definition is in the thrift include files in config.h: /* Version number of package */ #define VERSION "1.0.0-dev" and the 2nd definition is in cassandra_constants.cpp cassandraConstants::cassandraConstants() { VERSION = "19.36.0"; Note that while the thrift version is up-to-date, the cassandra version we are using is older. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11044) query under certain partition key takes much more time than expected
[ https://issues.apache.org/jira/browse/CASSANDRA-11044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109430#comment-15109430 ] Ivan Ryndin commented on CASSANDRA-11044: - >From log it looks like most time is spent while merging memtables and >sstables. I see that after messages like {{Merging data from memtables and 3 >sstables}} there is a big time gap before next message in log shows. > query under certain partition key takes much more time than expected > - > > Key: CASSANDRA-11044 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11044 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: xiaost > Attachments: tracing.log > > > In my cluster, all the nodes is under low workload, > but query under certain partition key (we found one) takes much more time > than expected. > we write & updates about 3 times per row in one day, > reads are much more than writes. > HARDWARD: > 6*nodes(E5-2630, 1*ssd with 5GB data) > TABLE DESCRIBE: > {noformat} > CREATE TABLE album.user_updates ( > user_id bigint, > time_uuid bigint, > key ascii, > PRIMARY KEY (user_id, time_uuid) > ) WITH CLUSTERING ORDER BY (time_uuid ASC) > AND bloom_filter_fp_chance = 0.01 > AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}' > AND comment = '' > AND compaction = {'class': > 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy'} > AND compression = {'sstable_compression': > 'org.apache.cassandra.io.compress.LZ4Compressor'} > AND dclocal_read_repair_chance = 0.1 > AND default_time_to_live = 0 > AND gc_grace_seconds = 864000 > AND max_index_interval = 2048 > AND memtable_flush_period_in_ms = 0 > AND min_index_interval = 128 > AND read_repair_chance = 0.0 > AND speculative_retry = '99.0PERCENTILE'; > {noformat} > QUERYs: > {noformat} > select * from user_updates where user_id = 1432138730701829 limit 100; > select count(1) from user_updates where user_id = 1432138730701829; > {noformat} > RESULT: (takes about 3.5 minutes) > {noformat} > count > --- > 1058 > (1 rows) > {noformat} > check attachments for the tracing log -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Zarzycki updated CASSANDRA-11037: -- Attachment: (was: 0001-Allows-bash-script-to-be-executed-through-symlinks.patch) > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: lhf > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10025) Allow compaction throttle to be real time
[ https://issues.apache.org/jira/browse/CASSANDRA-10025?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109508#comment-15109508 ] sankalp kohli commented on CASSANDRA-10025: --- The patch is identical in all versions. +1 on all the version. > Allow compaction throttle to be real time > - > > Key: CASSANDRA-10025 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10025 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: Soumava Ghosh >Priority: Minor > Labels: compaction > Fix For: 2.1.x > > Attachments: 10025-2.1.txt, 10025-2.2.txt, 10025-3.0.txt, > 10025-trunk.txt > > > We should allow compaction throttle to be set while compaction is going on. > Currently, it takes effect on the next compaction. This is bad for large > compactions. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-11049) Conflicting definitions of VERSION string when compiling cassandra_constants
[ https://issues.apache.org/jira/browse/CASSANDRA-11049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian resolved CASSANDRA-11049. Resolution: Won't Fix I'm not sure what is the root cause of this, but it is probably a new addition from the Thrift project. As a workaround, you can rename the {{VERSION}} constant in the cassandra.thrift file before you run it through the Thrift compiler to avoid this conflict. > Conflicting definitions of VERSION string when compiling cassandra_constants > > > Key: CASSANDRA-11049 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11049 > Project: Cassandra > Issue Type: Bug > Components: Configuration >Reporter: Peter Butler > > When compiling c++ code, as generated via thrift (latest version of thrift > from git repository as of January 19 2016), there are conflicting versions of > the VERSION constant in cassandra_constant.* The specific error is: > cassandra_constants.h:18:15: error: expected unqualified-id before string > constant > cassandra_constants.cpp: In constructor > ‘org::apache::cassandra::cassandraConstants::cassandraConstants()’: > cassandra_constants.cpp:14:17: error: assignment of read-only location > ‘"1.0.0-dev"’ > cassandra_constants.cpp:14:17: error: incompatible types in assignment of > ‘const char [8]’ to ‘const char [10]’ > The location of the original definition is in the thrift include files in > config.h: > /* Version number of package */ > #define VERSION "1.0.0-dev" > and the 2nd definition is in cassandra_constants.cpp > cassandraConstants::cassandraConstants() { > VERSION = "19.36.0"; > Note that while the thrift version is up-to-date, the cassandra version we > are using is older. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11049) Conflicting definitions of VERSION string when compiling cassandra_constants
[ https://issues.apache.org/jira/browse/CASSANDRA-11049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109590#comment-15109590 ] Peter Butler commented on CASSANDRA-11049: -- P.S. I should probably also point out that I also had to follow the hack shown at https://github.com/Orange-OpenSource/YACassandraPDO/issues/75 to resolve other discrepancies. The hack in question is this, and is performed after running "thrift --gen cpp cassandra.thrift": sed -i "s/apache::thrift::protocol::TInputRecursionTracker/::apache::thrift::protocol::TInputRecursionTracker/" gen-cpp/Cassandra.cpp gen-cpp/cassandra_types.cpp sed -i "s/apache::thrift::protocol::TOutputRecursionTracker/::apache::thrift::protocol::TOutputRecursionTracker/" gen-cpp/Cassandra.cpp gen-cpp/cassandra_types.cpp > Conflicting definitions of VERSION string when compiling cassandra_constants > > > Key: CASSANDRA-11049 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11049 > Project: Cassandra > Issue Type: Bug > Components: Configuration >Reporter: Peter Butler > > When compiling c++ code, as generated via thrift (latest version of thrift > from git repository as of January 19 2016), there are conflicting versions of > the VERSION constant in cassandra_constant.* The specific error is: > cassandra_constants.h:18:15: error: expected unqualified-id before string > constant > cassandra_constants.cpp: In constructor > ‘org::apache::cassandra::cassandraConstants::cassandraConstants()’: > cassandra_constants.cpp:14:17: error: assignment of read-only location > ‘"1.0.0-dev"’ > cassandra_constants.cpp:14:17: error: incompatible types in assignment of > ‘const char [8]’ to ‘const char [10]’ > The location of the original definition is in the thrift include files in > config.h: > /* Version number of package */ > #define VERSION "1.0.0-dev" > and the 2nd definition is in cassandra_constants.cpp > cassandraConstants::cassandraConstants() { > VERSION = "19.36.0"; > Note that while the thrift version is up-to-date, the cassandra version we > are using is older. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[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=15109710#comment-15109710 ] sankalp kohli commented on CASSANDRA-10023: --- Your patch has a bug which will always mark things as replica non aware. In StorageProxy, there will always be endpoints which are not the machine itself. Say endpoints in SP.syncWriteToBatchlog are A,B and C and the code is running in A, it will mark replicaAware once and replicaNotAware twice. > 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: Tushar Agrawal >Priority: Minor > Labels: lhf > Fix For: 3.x > > Attachments: 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 (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10819) Generic Java UDF types
[ https://issues.apache.org/jira/browse/CASSANDRA-10819?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109735#comment-15109735 ] DOAN DuyHai commented on CASSANDRA-10819: - +1 Robert, it will help having cleaner code (without dirty casting) inside UDF Hope it can make it into C* 3.4 > Generic Java UDF types > -- > > Key: CASSANDRA-10819 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10819 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Robert Stupp >Assignee: Robert Stupp >Priority: Minor > Labels: UDF, doc-impacting > Fix For: 3.x > > > At the moment we only generate raw type signatures for Java UDF methods. E.g. > a CQL argument type {{map}} is just mapped to {{java.util.Map}} > but could be mapped to {{java.util.Map }}. > It's a probably simple but nice improvement and feels to be a LHF. > Depending on the complexity it might be doable for 3.0.x, too. > Thanks for the heads-up, [~doanduyhai]! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11052) Cannot use Java 8 lambda expression inside UDF code body
DOAN DuyHai created CASSANDRA-11052: --- Summary: Cannot use Java 8 lambda expression inside UDF code body Key: CASSANDRA-11052 URL: https://issues.apache.org/jira/browse/CASSANDRA-11052 Project: Cassandra Issue Type: Bug Components: CQL Reporter: DOAN DuyHai When creating the following **UDF** using Java 8 lambda syntax {code:sql} CREATE FUNCTION IF NOT EXISTS music.udf(state map, styles list) RETURNS NULL ON NULL INPUT RETURNS map LANGUAGE java AS $$ styles.forEach((Object o) -> { String style = (String)o; if(state.containsKey(style)) { state.put(style, (Long)state.get(style)+1); } else { state.put(style, 1L); } }); return state; $$; {code} I got the following exception: {code:java} Caused by: com.datastax.driver.core.exceptions.InvalidQueryException: Could not compile function 'music.udf' from Java source: org.apache.cassandra.exceptions.InvalidRequestException: Java source compilation failed: Line 2: The type java.util.function.Consumer cannot be resolved. It is indirectly referenced from required .class files Line 2: The method forEach(Consumer) from the type Iterable refers to the missing type Consumer Line 2: The target type of this expression must be a functional interface at com.datastax.driver.core.Responses$Error.asException(Responses.java:136) at com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:179) at com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:184) at com.datastax.driver.core.RequestHandler.access$2500(RequestHandler.java:43) at com.datastax.driver.core.RequestHandler$SpeculativeExecution.setFinalResult(RequestHandler.java:798) at com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:617) at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1005) at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:928) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:276) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:263) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:318) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:304) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112) ... 1 more {code} It looks like the compiler requires importing java.util.Consumer but I have checked the source code and compiler options already support Java 8 source code so I'm pretty puzzled here ... /cc [~snazy] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11037) cqlsh bash script cannot be called through symlink
[ https://issues.apache.org/jira/browse/CASSANDRA-11037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108688#comment-15108688 ] Carl Yeksigian commented on CASSANDRA-11037: [~kf6nux]: Thanks for the patch! Unfortunately, this won't work on OS X as {{readlink}} does not take {{-f}} as an option. In the Cassandra JIRA, the "Ready to Commit" state is for when the reviewer has finished looking at the patch and determined that it is ready to be committed; the state this should have been in was "Patch Available". > cqlsh bash script cannot be called through symlink > -- > > Key: CASSANDRA-11037 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11037 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: BASH >Reporter: Benjamin Zarzycki >Priority: Trivial > Labels: easyfix, newbie > Fix For: 2.2.0 > > Attachments: > 0001-Allows-bash-script-to-be-executed-through-symlinks.patch > > Original Estimate: 0h > Remaining Estimate: 0h > > cqlsh bash script cannot be called through a symlink -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10389) Repair session exception Validation failed
[ https://issues.apache.org/jira/browse/CASSANDRA-10389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108758#comment-15108758 ] Jeff Gardner commented on CASSANDRA-10389: -- this is becoming a significant problem in our production environment... can this be escalated or is there a known work around? > Repair session exception Validation failed > -- > > Key: CASSANDRA-10389 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10389 > Project: Cassandra > Issue Type: Bug > Environment: Debian 8, Java 1.8.0_60, Cassandra 2.2.1 (datastax > compilation) >Reporter: Jędrzej Sieracki > Fix For: 2.2.x > > > I'm running a repair on a ring of nodes, that was recently extented from 3 to > 13 nodes. The extension was done two days ago, the repair was attempted > yesterday. > {quote} > [2015-09-22 11:55:55,266] Starting repair command #9, repairing keyspace > perspectiv with repair options (parallelism: parallel, primary range: false, > incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [], > hosts: [], # of ranges: 517) > [2015-09-22 11:55:58,043] Repair session 1f7c50c0-6110-11e5-b992-9f13fa8664c8 > for range (-5927186132136652665,-5917344746039874798] failed with error > [repair #1f7c50c0-6110-11e5-b992-9f13fa8664c8 on > perspectiv/stock_increment_agg, (-5927186132136652665,-5917344746039874798]] > Validation failed in cblade1.XXX/XXX (progress: 0%) > {quote} > BTW, I am ignoring the LEAK errors for now, that's outside of the scope of > the main issue: > {quote} > ERROR [Reference-Reaper:1] 2015-09-22 11:58:27,843 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@4d25ad8f) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@896826067:/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-73-big > was not released before the reference was garbage collected > {quote} > I scrubbed the sstable with failed validation on cblade1 with nodetool scrub > perspectiv stock_increment_agg: > {quote} > INFO [CompactionExecutor:1704] 2015-09-22 12:05:31,615 OutputHandler.java:42 > - Scrubbing > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big-Data.db') > (345466609 bytes) > INFO [CompactionExecutor:1703] 2015-09-22 12:05:31,615 OutputHandler.java:42 > - Scrubbing > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-82-big-Data.db') > (60496378 bytes) > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@4ca8951e) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@114161559:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-48-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@eeb6383) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1612685364:/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@1de90543) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@2058626950:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-49-big > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2015-09-22 12:05:31,676 Ref.java:187 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@15616385) to class > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1386628428:/var/lib/cassandra/data/perspectiv/receipt_agg_total-76abb0625de711e59f6e0b7d98a25b6e/la-47-big > was not released before the reference was garbage collected > INFO [CompactionExecutor:1703] 2015-09-22 12:05:35,098 OutputHandler.java:42 > - Scrub of > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-82-big-Data.db') > complete: 51397 rows in new sstable and 0 empty (tombstoned) rows dropped > INFO [CompactionExecutor:1704] 2015-09-22 12:05:47,605 OutputHandler.java:42 > - Scrub of > BigTableReader(path='/var/lib/cassandra/data/perspectiv/stock_increment_agg-840cad405de711e5b9929f13fa8664c8/la-83-big-Data.db') > complete: 292600 rows in new sstable and 0 empty (tombstoned) rows dropped >
[jira] [Updated] (CASSANDRA-11007) Exception when running nodetool info during bootstrap
[ https://issues.apache.org/jira/browse/CASSANDRA-11007?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-11007: --- Fix Version/s: 2.2.x 2.1.x > Exception when running nodetool info during bootstrap > - > > Key: CASSANDRA-11007 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11007 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: T Jake Luciani >Assignee: Yuki Morishita >Priority: Minor > Labels: fallout > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > > {code} > automaton@ip-172-31-26-150:~$ nodetool info > ID : 7d9aa832-de94-43ab-9548-b2e710418301 > Gossip active : true > Thrift active : false > Native Transport active: false > Load : 114.52 KB > Generation No : 1452721596 > Uptime (seconds) : 1396 > Heap Memory (MB) : 71.80 / 1842.00 > Off Heap Memory (MB) : 0.00 > Data Center: datacenter1 > Rack : rack1 > Exceptions : 0 > Key Cache : entries 11, size 888 bytes, capacity 92 MB, 180 > hits, 212 requests, 0.849 recent hit rate, 14400 save period in seconds > Row Cache : entries 0, size 0 bytes, capacity 0 bytes, 0 hits, 0 > requests, NaN recent hit rate, 0 save period in seconds > Counter Cache : entries 0, size 0 bytes, capacity 46 MB, 0 hits, 0 > requests, NaN recent hit rate, 7200 save period in seconds > error: null > -- StackTrace -- > java.lang.AssertionError > at > org.apache.cassandra.locator.TokenMetadata.getTokens(TokenMetadata.java:488) > at > org.apache.cassandra.service.StorageService.getTokens(StorageService.java:2561) > at > org.apache.cassandra.service.StorageService.getTokens(StorageService.java:2550) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71) > at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275) > at > com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) > at > com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) > at > com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) > at > com.sun.jmx.mbeanserver.PerInterface.getAttribute(PerInterface.java:83) > at > com.sun.jmx.mbeanserver.MBeanSupport.getAttribute(MBeanSupport.java:206) > at > com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttribute(DefaultMBeanServerInterceptor.java:647) > at > com.sun.jmx.mbeanserver.JmxMBeanServer.getAttribute(JmxMBeanServer.java:678) > at > javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1443) > at > javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:76) > at > javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1307) > at > javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1399) > at > javax.management.remote.rmi.RMIConnectionImpl.getAttribute(RMIConnectionImpl.java:637) > at sun.reflect.GeneratedMethodAccessor6.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:323) > at sun.rmi.transport.Transport$1.run(Transport.java:200) > at sun.rmi.transport.Transport$1.run(Transport.java:197) > at java.security.AccessController.doPrivileged(Native Method) > at sun.rmi.transport.Transport.serviceCall(Transport.java:196) > at > sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:568) > at > sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:826) > at > sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$94(TCPTransport.java:683) > at > sun.rmi.transport.tcp.TCPTransport$ConnectionHandler$$Lambda$179/2070569218.run(Unknown > Source) > at java.security.AccessController.doPrivileged(Native Method) >
[jira] [Commented] (CASSANDRA-10870) pushed_notifications_test.py:TestPushedNotifications.restart_node_test flapping on C* 2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-10870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108904#comment-15108904 ] Alan Boudreault commented on CASSANDRA-10870: - [~Stefania] I think you worked on this test. Do you have any idea what could be the problem? > pushed_notifications_test.py:TestPushedNotifications.restart_node_test > flapping on C* 2.1 > - > > Key: CASSANDRA-10870 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10870 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: DS Test Eng > Fix For: 2.1.x > > > This test flaps on CassCI on 2.1. [~aboudreault] Do I remember correctly that > you did some work on these tests in the past few months? If so, could you > have a look and see if there's some assumption the test makes that don't hold > for 2.1? > Oddly, it fails frequently under JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ > but less frequently on JDK7: > http://cassci.datastax.com/job/cassandra-2.1_dtest/lastCompletedBuild/testReport/pushed_notifications_test/TestPushedNotifications/restart_node_test/history/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10868) Skip supercolumns upgrade tests on jdk8
[ https://issues.apache.org/jira/browse/CASSANDRA-10868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15108932#comment-15108932 ] Philip Thompson commented on CASSANDRA-10868: - If we don't compile the 2.0 version, and just use the binaries, those should run on jdk8 just fine. > Skip supercolumns upgrade tests on jdk8 > --- > > Key: CASSANDRA-10868 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10868 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey >Assignee: Jim Witschey > > The tests in the {{upgrade_supercolumns_test}} dtest module fail when we test > on JDK8 because they attempt to upgrade from 2.0, which will not compile on > JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/160/testReport/upgrade_supercolumns_test/ > [~rhatch] As we look at how we want to run upgrade tests in the future, we > should consider this. In the meantime, I think the best way to deal with this > might be to add something to the exclude files in {{conf/}}. That sound > reasonable, or is there a better way to do this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11050) SSTables not loaded after dropping column
[ https://issues.apache.org/jira/browse/CASSANDRA-11050?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] amorton updated CASSANDRA-11050: Attachment: 11050-3.0.patch > SSTables not loaded after dropping column > - > > Key: CASSANDRA-11050 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11050 > Project: Cassandra > Issue Type: Bug >Reporter: amorton >Assignee: amorton > Attachments: 11050-3.0.patch > > > The {{system_schema.dropped_tables}} table is not flushed when schema is > updated, this can result in SSTables not being loaded at startup and failure > to start if the commit log contains mutations with the column. > Reproduce on cassandra-3.0 branch by starting a node and running following in > cqlsh: > {code} > create keyspace dev WITH replication = {'class':'SimpleStrategy', > 'replication_factor':1}; > use dev; > create table foo ( > foo text primary key, > bar text, > baz text > ); > insert into foo (foo, bar, baz) values ('foo','this is bar', 'this is baz'); > alter table foo > drop baz; > {code} > Stop the node and restart, the following errors are raised and the node does > not start: > {code} > ERROR 16:38:19 Exception in thread Thread[SSTableBatchOpen:1,5,main] > java.lang.RuntimeException: Unknown column baz during deserialization > at > org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:331) > ~[main/:na] > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:485) > ~[main/:na] > at > org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:374) > ~[main/:na] > at > org.apache.cassandra.io.sstable.format.SSTableReader$4.run(SSTableReader.java:533) > ~[main/:na] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_60] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ~[na:1.8.0_60] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > ~[na:1.8.0_60] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_60] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60] > ... > ERROR 16:38:19 Exiting due to error while processing commit log during > initialization. > org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: > Unexpected error deserializing mutation; saved to > /var/folders/r2/rkv1jz3j0j74r9s1zm5xx9wcgn/T/mutation5408885979635225676dat. > This may be caused by replaying a mutation against a table with the same > name but incompatible schema. Exception follows: java.lang.RuntimeException: > Unknown column baz during deserialization > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:633) > [main/:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:556) > [main/:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:509) > [main/:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:404) > [main/:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:151) > [main/:na] > at > org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:189) > [main/:na] > at > org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:169) > [main/:na] > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:283) > [main/:na] > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:551) > [main/:na] > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) > [main/:na] > {code} > {{dropped_columns}} is not in the list of tables to flush, > {{SchemaKeyspace.ALL}}. > It's a simple patch to add it, attached. The fix will need to go to 3.0, 3.1 > and trunk AFAIK > however this will change the way the schema hash is calculated in > {{SchemaKeyspace.calculateSchemaDigest()}} It looks like this would cause the > nodes to announce a new version of the schema on (first) restart. > I currently donit understand all the implications of changing the schema > hash, thoughts ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11049) Conflicting definitions of VERSION string when compiling cassandra_constants
[ https://issues.apache.org/jira/browse/CASSANDRA-11049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109611#comment-15109611 ] Carl Yeksigian commented on CASSANDRA-11049: Cassandra only uses the {{VERSION}} constant to return to the client in case the version is used in the client library; if it isn't used, it isn't needed. It looks like this library has been written against a different version of Thrift; the wire protocol is stable between versions, but the APIs do change around a bit, so if the compiler version doesn't match the library version, there will be some differences. If you are using the same library and compiler, it's probably a Thrift bug and should be addressed on their Jira. Another alternative would be to use one of the [native protocol drivers|http://wiki.apache.org/cassandra/ClientOptions], especially if you are only using CQL. Support for Thrift was officially deprecated in 3.x and will be removed in 4.x. > Conflicting definitions of VERSION string when compiling cassandra_constants > > > Key: CASSANDRA-11049 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11049 > Project: Cassandra > Issue Type: Bug > Components: Configuration >Reporter: Peter Butler > > When compiling c++ code, as generated via thrift (latest version of thrift > from git repository as of January 19 2016), there are conflicting versions of > the VERSION constant in cassandra_constant.* The specific error is: > cassandra_constants.h:18:15: error: expected unqualified-id before string > constant > cassandra_constants.cpp: In constructor > ‘org::apache::cassandra::cassandraConstants::cassandraConstants()’: > cassandra_constants.cpp:14:17: error: assignment of read-only location > ‘"1.0.0-dev"’ > cassandra_constants.cpp:14:17: error: incompatible types in assignment of > ‘const char [8]’ to ‘const char [10]’ > The location of the original definition is in the thrift include files in > config.h: > /* Version number of package */ > #define VERSION "1.0.0-dev" > and the 2nd definition is in cassandra_constants.cpp > cassandraConstants::cassandraConstants() { > VERSION = "19.36.0"; > Note that while the thrift version is up-to-date, the cassandra version we > are using is older. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11051) Make LZ4 Compression Level Configurable
Michael Kjellman created CASSANDRA-11051: Summary: Make LZ4 Compression Level Configurable Key: CASSANDRA-11051 URL: https://issues.apache.org/jira/browse/CASSANDRA-11051 Project: Cassandra Issue Type: Improvement Components: Compaction Reporter: Michael Kjellman Attachments: lz4_2.2.patch We'd like to make the LZ4 Compressor implementation configurable on a per column family basis. Testing has shown a ~4% reduction in file size with the higher compression LZ4 implementation vs the standard compressor we currently use instantiated by the default constructor. The attached patch adds the following optional parameters 'lz4_compressor_type' and 'lz4_high_compressor_level' to the LZ4Compressor. If none of the new optional parameters are specified, the Compressor will use the same defaults Cassandra has always had for LZ4. New LZ4Compressor Optional Parameters: * lz4_compressor_type can currently be either 'high' (uses LZ4HCCompressor) or 'fast' (uses LZ4Compressor) * lz4_high_compressor_level can be set between 1 and 17. Not specifying a compressor level while specifying lz4_compressor_type as 'high' will use a default level of 9 (as picked by the LZ4 library as the "default"). Currently, we use the default LZ4 compressor constructor. This change would just expose the level (and implementation to use) to the user via the schema. There are many potential cases where users may find that the tradeoff in additional CPU and memory usage is worth the on-disk space savings. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-11051) Make LZ4 Compression Level Configurable
[ https://issues.apache.org/jira/browse/CASSANDRA-11051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] sankalp kohli updated CASSANDRA-11051: -- Assignee: Michael Kjellman > Make LZ4 Compression Level Configurable > > > Key: CASSANDRA-11051 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11051 > Project: Cassandra > Issue Type: Improvement > Components: Compaction >Reporter: Michael Kjellman >Assignee: Michael Kjellman > Attachments: lz4_2.2.patch > > > We'd like to make the LZ4 Compressor implementation configurable on a per > column family basis. Testing has shown a ~4% reduction in file size with the > higher compression LZ4 implementation vs the standard compressor we currently > use instantiated by the default constructor. The attached patch adds the > following optional parameters 'lz4_compressor_type' and > 'lz4_high_compressor_level' to the LZ4Compressor. If none of the new optional > parameters are specified, the Compressor will use the same defaults Cassandra > has always had for LZ4. > New LZ4Compressor Optional Parameters: > * lz4_compressor_type can currently be either 'high' (uses LZ4HCCompressor) > or 'fast' (uses LZ4Compressor) > * lz4_high_compressor_level can be set between 1 and 17. Not specifying a > compressor level while specifying lz4_compressor_type as 'high' will use a > default level of 9 (as picked by the LZ4 library as the "default"). > Currently, we use the default LZ4 compressor constructor. This change would > just expose the level (and implementation to use) to the user via the schema. > There are many potential cases where users may find that the tradeoff in > additional CPU and memory usage is worth the on-disk space savings. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11049) Conflicting definitions of VERSION string when compiling cassandra_constants
[ https://issues.apache.org/jira/browse/CASSANDRA-11049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109614#comment-15109614 ] Peter Butler commented on CASSANDRA-11049: -- Thanks for all the info. Much appreciated. > Conflicting definitions of VERSION string when compiling cassandra_constants > > > Key: CASSANDRA-11049 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11049 > Project: Cassandra > Issue Type: Bug > Components: Configuration >Reporter: Peter Butler > > When compiling c++ code, as generated via thrift (latest version of thrift > from git repository as of January 19 2016), there are conflicting versions of > the VERSION constant in cassandra_constant.* The specific error is: > cassandra_constants.h:18:15: error: expected unqualified-id before string > constant > cassandra_constants.cpp: In constructor > ‘org::apache::cassandra::cassandraConstants::cassandraConstants()’: > cassandra_constants.cpp:14:17: error: assignment of read-only location > ‘"1.0.0-dev"’ > cassandra_constants.cpp:14:17: error: incompatible types in assignment of > ‘const char [8]’ to ‘const char [10]’ > The location of the original definition is in the thrift include files in > config.h: > /* Version number of package */ > #define VERSION "1.0.0-dev" > and the 2nd definition is in cassandra_constants.cpp > cassandraConstants::cassandraConstants() { > VERSION = "19.36.0"; > Note that while the thrift version is up-to-date, the cassandra version we > are using is older. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109648#comment-15109648 ] Carl Yeksigian commented on CASSANDRA-10979: I've rebased and pushed up new 2.2-trunk branches. ||2.2||3.0||3.3||trunk|| |[branch|https://github.com/carlyeks/cassandra/tree/ticket/10979/2.2]|[branch|https://github.com/carlyeks/cassandra/tree/ticket/10979/3.0]|[branch|https://github.com/carlyeks/cassandra/tree/ticket/10979/3.3]|[branch|https://github.com/carlyeks/cassandra/tree/ticket/10979/trunk]| |[utest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-2.2-testall/]|[utest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-3.0-testall/]|[utest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-3.3-testall/]|[utest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-trunk-testall/]| |[dtest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-2.2-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-3.0-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-3.3-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/carlyeks/job/carlyeks-ticket-10979-trunk-dtest/]| > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland >Assignee: Carl Yeksigian > Labels: compaction, leveled > Fix For: 2.2.x, 3.0.x, 3.x > > Attachments: 10979-2.1.txt > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11049) Conflicting definitions of VERSION string when compiling cassandra_constants
[ https://issues.apache.org/jira/browse/CASSANDRA-11049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109586#comment-15109586 ] Peter Butler commented on CASSANDRA-11049: -- OK thanks Carl I will try that. And renaming the cassandra.thrift-specific VERSION string (as opposed to touching the one in thrift) will not adversely affect anything else? > Conflicting definitions of VERSION string when compiling cassandra_constants > > > Key: CASSANDRA-11049 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11049 > Project: Cassandra > Issue Type: Bug > Components: Configuration >Reporter: Peter Butler > > When compiling c++ code, as generated via thrift (latest version of thrift > from git repository as of January 19 2016), there are conflicting versions of > the VERSION constant in cassandra_constant.* The specific error is: > cassandra_constants.h:18:15: error: expected unqualified-id before string > constant > cassandra_constants.cpp: In constructor > ‘org::apache::cassandra::cassandraConstants::cassandraConstants()’: > cassandra_constants.cpp:14:17: error: assignment of read-only location > ‘"1.0.0-dev"’ > cassandra_constants.cpp:14:17: error: incompatible types in assignment of > ‘const char [8]’ to ‘const char [10]’ > The location of the original definition is in the thrift include files in > config.h: > /* Version number of package */ > #define VERSION "1.0.0-dev" > and the 2nd definition is in cassandra_constants.cpp > cassandraConstants::cassandraConstants() { > VERSION = "19.36.0"; > Note that while the thrift version is up-to-date, the cassandra version we > are using is older. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9666) Provide an alternative to DTCS
[ https://issues.apache.org/jira/browse/CASSANDRA-9666?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-9666: -- Attachment: dtcs-twcs-io.png dtcs-twcs-load.png One last data point from a real world cluster. The two screenshots attached (dtcs-twcs-load and dtcs-twcs-io) show difference in IO and CPU on a real world cluster as it transitioned from DTCS -> TWCS with no other changes/tuning. This cluster is running a stable version of DSE/Cassandra, so it does not have changes to DTCS that were not backported. It seems that there's little desire to integrate this work upstream, given that DTCS already exists and compaction is pluggable. Rather than try to keep rebasing for no reason, I'm fine with this going to won't-fix, and users who prefer twcs (because it's easier to reason about, or because it doesn't constantly recompact, or for whatever reason) can just build it on their own from my github repo. > Provide an alternative to DTCS > -- > > Key: CASSANDRA-9666 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9666 > Project: Cassandra > Issue Type: Improvement >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa > Fix For: 2.1.x, 2.2.x > > Attachments: dtcs-twcs-io.png, dtcs-twcs-load.png > > > DTCS is great for time series data, but it comes with caveats that make it > difficult to use in production (typical operator behaviors such as bootstrap, > removenode, and repair have MAJOR caveats as they relate to > max_sstable_age_days, and hints/read repair break the selection algorithm). > I'm proposing an alternative, TimeWindowCompactionStrategy, that sacrifices > the tiered nature of DTCS in order to address some of DTCS' operational > shortcomings. I believe it is necessary to propose an alternative rather than > simply adjusting DTCS, because it fundamentally removes the tiered nature in > order to remove the parameter max_sstable_age_days - the result is very very > different, even if it is heavily inspired by DTCS. > Specifically, rather than creating a number of windows of ever increasing > sizes, this strategy allows an operator to choose the window size, compact > with STCS within the first window of that size, and aggressive compact down > to a single sstable once that window is no longer current. The window size is > a combination of unit (minutes, hours, days) and size (1, etc), such that an > operator can expect all data using a block of that size to be compacted > together (that is, if your unit is hours, and size is 6, you will create > roughly 4 sstables per day, each one containing roughly 6 hours of data). > The result addresses a number of the problems with > DateTieredCompactionStrategy: > - At the present time, DTCS’s first window is compacted using an unusual > selection criteria, which prefers files with earlier timestamps, but ignores > sizes. In TimeWindowCompactionStrategy, the first window data will be > compacted with the well tested, fast, reliable STCS. All STCS options can be > passed to TimeWindowCompactionStrategy to configure the first window’s > compaction behavior. > - HintedHandoff may put old data in new sstables, but it will have little > impact other than slightly reduced efficiency (sstables will cover a wider > range, but the old timestamps will not impact sstable selection criteria > during compaction) > - ReadRepair may put old data in new sstables, but it will have little impact > other than slightly reduced efficiency (sstables will cover a wider range, > but the old timestamps will not impact sstable selection criteria during > compaction) > - Small, old sstables resulting from streams of any kind will be swiftly and > aggressively compacted with the other sstables matching their similar > maxTimestamp, without causing sstables in neighboring windows to grow in size. > - The configuration options are explicit and straightforward - the tuning > parameters leave little room for error. The window is set in common, easily > understandable terms such as “12 hours”, “1 Day”, “30 days”. The > minute/hour/day options are granular enough for users keeping data for hours, > and users keeping data for years. > - There is no explicitly configurable max sstable age, though sstables will > naturally stop compacting once new data is written in that window. > - Streaming operations can create sstables with old timestamps, and they'll > naturally be joined together with sstables in the same time bucket. This is > true for bootstrap/repair/sstableloader/removenode. > - It remains true that if old data and new data is written into the memtable > at the same time, the resulting sstables will be treated as if they were new > sstables, however, that no longer negatively impacts the compaction >
[jira] [Resolved] (CASSANDRA-10865) commitlog_test.py:TestCommitLog.test_bad_crc fails on C* 2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-10865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jim Witschey resolved CASSANDRA-10865. -- Resolution: Fixed Fix Version/s: (was: 2.1.x) Addressed with this dtest PR: https://github.com/riptano/cassandra-dtest/pull/755 Closing. > commitlog_test.py:TestCommitLog.test_bad_crc fails on C* 2.1 > > > Key: CASSANDRA-10865 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10865 > Project: Cassandra > Issue Type: Sub-task >Reporter: Jim Witschey >Assignee: Jim Witschey > > This test is failing hard on 2.1: > http://cassci.datastax.com/job/cassandra-2.1_dtest/376/testReport/commitlog_test/TestCommitLog/test_bad_crc/history/ > In spite of having a bad commitlog, the node successfully starts. Is this a > C* bug, or just something that wasn't implemented in 2.1? It seems to succeed > on 2.2+. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10868) Skip supercolumns upgrade tests on jdk8
[ https://issues.apache.org/jira/browse/CASSANDRA-10868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109488#comment-15109488 ] Jim Witschey commented on CASSANDRA-10868: -- This should be addressed by this dtest PR: https://github.com/riptano/cassandra-dtest/pull/756 > Skip supercolumns upgrade tests on jdk8 > --- > > Key: CASSANDRA-10868 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10868 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey >Assignee: Jim Witschey > > The tests in the {{upgrade_supercolumns_test}} dtest module fail when we test > on JDK8 because they attempt to upgrade from 2.0, which will not compile on > JDK8: > http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/160/testReport/upgrade_supercolumns_test/ > [~rhatch] As we look at how we want to run upgrade tests in the future, we > should consider this. In the meantime, I think the best way to deal with this > might be to add something to the exclude files in {{conf/}}. That sound > reasonable, or is there a better way to do this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10211) SliceQueryFilter warnings should print the partition key
[ https://issues.apache.org/jira/browse/CASSANDRA-10211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109549#comment-15109549 ] sankalp kohli commented on CASSANDRA-10211: --- [~soumava] Please provide patches for later branches if this patch does not apply to them. > SliceQueryFilter warnings should print the partition key > > > Key: CASSANDRA-10211 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10211 > Project: Cassandra > Issue Type: Improvement >Reporter: Soumava Ghosh >Assignee: Soumava Ghosh >Priority: Minor > Fix For: 2.1.x > > Attachments: 10211-2.1.txt > > > Currently the SliceQueryFilter does not log the partition key while logging > the warning/error when the number of tombstones exceed the limits set by the > tombstone_warn_threshold and the tombstone_failure_threshold. Logging the > partition key would be really helpful for debugging. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-11050) SSTables not loaded after dropping column
amorton created CASSANDRA-11050: --- Summary: SSTables not loaded after dropping column Key: CASSANDRA-11050 URL: https://issues.apache.org/jira/browse/CASSANDRA-11050 Project: Cassandra Issue Type: Bug Reporter: amorton Assignee: amorton The {{system_schema.dropped_tables}} table is not flushed when schema is updated, this can result in SSTables not being loaded at startup and failure to start if the commit log contains mutations with the column. Reproduce on cassandra-3.0 branch by starting a node and running following in cqlsh: {code} create keyspace dev WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; use dev; create table foo ( foo text primary key, bar text, baz text ); insert into foo (foo, bar, baz) values ('foo','this is bar', 'this is baz'); alter table foo drop baz; {code} Stop the node and restart, the following errors are raised and the node does not start: {code} ERROR 16:38:19 Exception in thread Thread[SSTableBatchOpen:1,5,main] java.lang.RuntimeException: Unknown column baz during deserialization at org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:331) ~[main/:na] at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:485) ~[main/:na] at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:374) ~[main/:na] at org.apache.cassandra.io.sstable.format.SSTableReader$4.run(SSTableReader.java:533) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_60] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_60] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_60] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_60] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60] ... ERROR 16:38:19 Exiting due to error while processing commit log during initialization. org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: Unexpected error deserializing mutation; saved to /var/folders/r2/rkv1jz3j0j74r9s1zm5xx9wcgn/T/mutation5408885979635225676dat. This may be caused by replaying a mutation against a table with the same name but incompatible schema. Exception follows: java.lang.RuntimeException: Unknown column baz during deserialization at org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:633) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:556) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:509) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:404) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:151) [main/:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:189) [main/:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:169) [main/:na] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:283) [main/:na] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:551) [main/:na] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) [main/:na] {code} {{dropped_columns}} is not in the list of tables to flush, {{SchemaKeyspace.ALL}}. It's a simple patch to add it, attached. The fix will need to go to 3.0, 3.1 and trunk AFAIK however this will change the way the schema hash is calculated in {{SchemaKeyspace.calculateSchemaDigest()}} It looks like this would cause the nodes to announce a new version of the schema on (first) restart. I currently donit understand all the implications of changing the schema hash, thoughts ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10965) Shadowable tombstones can continue to shadow view results when timestamps match
[ https://issues.apache.org/jira/browse/CASSANDRA-10965?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15109829#comment-15109829 ] Taiyuan Zhang commented on CASSANDRA-10965: --- It's been a while and it seems the output of the given cql is changed: When I execute {code} INSERT INTO base (k, c, val) VALUES (1, 1, 1) USING TIMESTAMP 0; SELECT c, k, val FROM base; SELECT c, k, val FROM mv_reuse; UPDATE base USING TIMESTAMP 1 SET c = 0 WHERE k = 1; SELECT c, k, val FROM base; SELECT c, k, val FROM mv_reuse; UPDATE base USING TIMESTAMP 1 SET c = 1 WHERE k = 1; {code} The output is {code} cqlsh:mykeyspace> INSERT INTO base (k, c, val) VALUES (1, 1, 1) USING TIMESTAMP 0; cqlsh:mykeyspace> select * FROM mv_reuse ; SELECT * FROM mv_reuse ; k | c | val ---+---+- 1 | 1 | 1 (1 rows) k | c | val ---+---+- 1 | 1 | 1 (1 rows) cqlsh:mykeyspace> UPDATE base USING TIMESTAMP 1 SET c = 0 WHERE k = 1; cqlsh:mykeyspace> select * FROM mv_reuse ; SELECT * FROM mv_reuse ; k | c | val ---+---+- 1 | 0 | 1 (1 rows) k | c | val ---+---+- 1 | 0 | 1 (1 rows) cqlsh:mykeyspace> UPDATE base USING TIMESTAMP 1 SET c = 1 WHERE k = 1; cqlsh:mykeyspace> select * FROM mv_reuse ; SELECT * FROM mv_reuse ; k | c | val ---+---+- (0 rows) k | c | val ---+---+- (0 rows) {code} So previously, the updated row will appear when selecting from the base, but disappears when selecting from the view; now, it shows from neither select query. So, just to confirm, the correct, expected behavior should be: the row k=1, c=1, val=1 should appears in BOTH select query (from base and from the view), right? > Shadowable tombstones can continue to shadow view results when timestamps > match > --- > > Key: CASSANDRA-10965 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10965 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Carl Yeksigian >Assignee: Carl Yeksigian > Fix For: 3.0.x > > Attachments: shadow-ts.cql > > > I've attached a script which reproduces the issue. The first time we insert > with {{TIMESTAMP 2}}, we are inserting a new row which has the same timestamp > as the previous shadow tombstone, and it continues to be shadowed by that > tombstone because we shadow values with the same timestamp. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (CASSANDRA-10965) Shadowable tombstones can continue to shadow view results when timestamps match
[ https://issues.apache.org/jira/browse/CASSANDRA-10965?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Taiyuan Zhang updated CASSANDRA-10965: -- Comment: was deleted (was: It's been a while and it seems the output of the given cql is changed: When I execute {code} INSERT INTO base (k, c, val) VALUES (1, 1, 1) USING TIMESTAMP 0; SELECT c, k, val FROM base; SELECT c, k, val FROM mv_reuse; UPDATE base USING TIMESTAMP 1 SET c = 0 WHERE k = 1; SELECT c, k, val FROM base; SELECT c, k, val FROM mv_reuse; UPDATE base USING TIMESTAMP 1 SET c = 1 WHERE k = 1; {code} The output is {code} cqlsh:mykeyspace> INSERT INTO base (k, c, val) VALUES (1, 1, 1) USING TIMESTAMP 0; cqlsh:mykeyspace> select * FROM mv_reuse ; SELECT * FROM mv_reuse ; k | c | val ---+---+- 1 | 1 | 1 (1 rows) k | c | val ---+---+- 1 | 1 | 1 (1 rows) cqlsh:mykeyspace> UPDATE base USING TIMESTAMP 1 SET c = 0 WHERE k = 1; cqlsh:mykeyspace> select * FROM mv_reuse ; SELECT * FROM mv_reuse ; k | c | val ---+---+- 1 | 0 | 1 (1 rows) k | c | val ---+---+- 1 | 0 | 1 (1 rows) cqlsh:mykeyspace> UPDATE base USING TIMESTAMP 1 SET c = 1 WHERE k = 1; cqlsh:mykeyspace> select * FROM mv_reuse ; SELECT * FROM mv_reuse ; k | c | val ---+---+- (0 rows) k | c | val ---+---+- (0 rows) {code} So previously, the updated row will appear when selecting from the base, but disappears when selecting from the view; now, it shows from neither select query. So, just to confirm, the correct, expected behavior should be: the row k=1, c=1, val=1 should appears in BOTH select query (from base and from the view), right?) > Shadowable tombstones can continue to shadow view results when timestamps > match > --- > > Key: CASSANDRA-10965 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10965 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Carl Yeksigian >Assignee: Carl Yeksigian > Fix For: 3.0.x > > Attachments: shadow-ts.cql > > > I've attached a script which reproduces the issue. The first time we insert > with {{TIMESTAMP 2}}, we are inserting a new row which has the same timestamp > as the previous shadow tombstone, and it continues to be shadowed by that > tombstone because we shadow values with the same timestamp. -- This message was sent by Atlassian JIRA (v6.3.4#6332)