[jira] [Commented] (CASSANDRA-10903) AssertionError while reading sstable when querying static column
[ https://issues.apache.org/jira/browse/CASSANDRA-10903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069451#comment-15069451 ] Branimir Lambov commented on CASSANDRA-10903: - Patch looks good overall. It is a bit unclear why the reverse iteration if fine, could you extend the test a little to also exercise the reversed iterator? Also, I think {{IndexState.updateBlock}} needs an {{assert !reversed}} or at least a comment to state that it is only used when going forwards. > AssertionError while reading sstable when querying static column > > > Key: CASSANDRA-10903 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10903 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Tyler Hobbs >Assignee: Sylvain Lebresne > Fix For: 3.0.x, 3.x > > > I'm able to consistently reproduce the following error: > {noformat} > 1 WARN [SharedPool-Worker-1] 2015-12-17 17:37:00,196 > AbstractTracingAwareExecutorService.java (line 169) Uncaught exception on > thread Thread[SharedPool-Worker-1,5,main]: {} > java.lang.AssertionError: null > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator$IndexState.updateBlock(AbstractSSTableIterator.java:463) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardIndexedReader.computeNext(SSTableIterator.java:268) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:158) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:352) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:219) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:32) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:369) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:426) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:286) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:108) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:298) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:136) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:128) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:123) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:288) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1692) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at >
[jira] [Commented] (CASSANDRA-8103) Secondary Indices for Static Columns
[ https://issues.apache.org/jira/browse/CASSANDRA-8103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069458#comment-15069458 ] Sam Tunnicliffe commented on CASSANDRA-8103: While I was working on CASSANDRA-9459, I had a version of this which was pretty much fully working. I haven't had chance to go back and rebase it yet, but you might find it a useful starting point. My branch is https://github.com/beobal/cassandra/tree/8103 Only the last commit is really specfic to indexes on static columns, but there was a fair bit of renaming & refactoring towards the end of #9459, so a few things will have shuffled about by now. There's also no unit test coverage in that branch, but I've attached a cql script I've used to do a quick smoke test. It should be trivial to add tests to {{CassandraIndexTest}} for this now though. > Secondary Indices for Static Columns > > > Key: CASSANDRA-8103 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8103 > Project: Cassandra > Issue Type: New Feature > Components: CQL >Reporter: Ron Cohen >Assignee: Sam Tunnicliffe > Fix For: 3.x > > Attachments: in_progress.patch, smoke-test.cql > > > We should add secondary index support for static columns. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8103) Secondary Indices for Static Columns
[ https://issues.apache.org/jira/browse/CASSANDRA-8103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-8103: --- Attachment: smoke-test.cql > Secondary Indices for Static Columns > > > Key: CASSANDRA-8103 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8103 > Project: Cassandra > Issue Type: New Feature > Components: CQL >Reporter: Ron Cohen >Assignee: Sam Tunnicliffe > Fix For: 3.x > > Attachments: in_progress.patch, smoke-test.cql > > > We should add secondary index support for static columns. -- 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=15069465#comment-15069465 ] Sam Tunnicliffe commented on CASSANDRA-10661: - [~xedin] that sounds to me like a reasonable approach. Does that basically make this ticket Patch Available based on your integration branch (if so, I'll be sure and wrap up review asap after the holidays)? > 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] [Commented] (CASSANDRA-7653) Add role based access control to Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-7653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069466#comment-15069466 ] Sam Tunnicliffe commented on CASSANDRA-7653: bq. Is there any reason why we don't do this automatically? Because, as you say, this is a potentially dangerous operation it's preferable to have its execution controlled by an operator. Also, the schema change provides a handy synchronisation point so we can have all nodes cut over to using the new tables for live auth requests simultaneously. Coordinating that otherwise is tricky. > Add role based access control to Cassandra > -- > > Key: CASSANDRA-7653 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7653 > Project: Cassandra > Issue Type: Sub-task > Components: CQL, Distributed Metadata >Reporter: Mike Adamson >Assignee: Sam Tunnicliffe > Labels: docs-impacting, security > Fix For: 2.2.0 beta 1 > > Attachments: 7653.patch, CQLSmokeTest.java, cql_smoke_test.py > > > The current authentication model supports granting permissions to individual > users. While this is OK for small or medium organizations wanting to > implement authorization, it does not work well in large organizations because > of the overhead of having to maintain the permissions for each user. > Introducing roles into the authentication model would allow sets of > permissions to be controlled in one place as a role and then the role granted > to users. Roles should also be able to be granted to other roles to allow > hierarchical sets of permissions to be built up. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[1/2] cassandra git commit: Fix assertion error when reading static on an indexed sstable
Repository: cassandra Updated Branches: refs/heads/10903 [created] e7c6e14fd Fix assertion error when reading static on an indexed sstable Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/bc887acf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bc887acf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bc887acf Branch: refs/heads/10903 Commit: bc887acfaec939efb5b9abeb3b7e7bcf4c1c3870 Parents: 11165f4 Author: Sylvain LebresneAuthored: Mon Dec 21 12:14:46 2015 +0100 Committer: Sylvain Lebresne Committed: Mon Dec 21 13:41:11 2015 +0100 -- .../columniterator/AbstractSSTableIterator.java | 33 .../db/columniterator/SSTableIterator.java | 38 +++--- .../columniterator/SSTableReversedIterator.java | 28 ++- .../cql3/QueryWithIndexedSSTableTest.java | 80 4 files changed, 112 insertions(+), 67 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc887acf/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java -- diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java index 5f280d7..f103ee2 100644 --- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java +++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java @@ -99,14 +99,14 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator // Note that this needs to be called after file != null and after the partitionDeletion has been set, but before readStaticRow // (since it uses it) so we can't move that up (but we'll be able to simplify as soon as we drop support for the old file format). -this.reader = needsReader ? createReader(indexEntry, file, true, shouldCloseFile) : null; +this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null; this.staticRow = readStaticRow(sstable, file, helper, columns.fetchedColumns().statics, isForThrift, reader == null ? null : reader.deserializer); } else { this.partitionLevelDeletion = indexEntry.deletionTime(); this.staticRow = Rows.EMPTY_STATIC_ROW; -this.reader = needsReader ? createReader(indexEntry, file, false, shouldCloseFile) : null; +this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null; } if (reader == null && file != null && shouldCloseFile) @@ -180,7 +180,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator } } -protected abstract Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile); +protected abstract Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile); public CFMetaData metadata() { @@ -291,19 +291,13 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator // Records the currently open range tombstone (if any) protected DeletionTime openMarker = null; -// !isInit means we have never seeked in the file and thus should seek before reading anything -protected boolean isInit; - -protected Reader(FileDataInput file, boolean isInit, boolean shouldCloseFile) +protected Reader(FileDataInput file, boolean shouldCloseFile) { this.file = file; -this.isInit = isInit; this.shouldCloseFile = shouldCloseFile; if (file != null) createDeserializer(); -else -assert !isInit; } private void createDeserializer() @@ -343,12 +337,6 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator { try { -if (!isInit) -{ -init(); -isInit = true; -} - return hasNextInternal(); } catch (IOException e) @@ -387,9 +375,6 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator } } -// Called is hasNext() is called but we haven't been yet initialized -protected abstract void init() throws IOException; - // Set the reader so its hasNext/next methods
[2/2] cassandra git commit: Assert !reversed in updateBlock and improve test
Assert !reversed in updateBlock and improve test Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e7c6e14f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e7c6e14f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e7c6e14f Branch: refs/heads/10903 Commit: e7c6e14fd2662c35310d1b27017aed75640ce002 Parents: bc887ac Author: Sylvain LebresneAuthored: Wed Dec 23 12:13:54 2015 +0100 Committer: Sylvain Lebresne Committed: Wed Dec 23 12:18:38 2015 +0100 -- .../cassandra/db/columniterator/AbstractSSTableIterator.java | 4 .../org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java | 4 2 files changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7c6e14f/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java -- diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java index f103ee2..8ac3dcb 100644 --- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java +++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java @@ -443,8 +443,12 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator } // Update the block idx based on the current reader position if we're past the current block. +// This only makes sense for forward iteration (for reverse ones, when we reach the end of a block we +// should seek to the previous one, not update the index state and continue). public void updateBlock() throws IOException { +assert !reversed; + // If we get here with currentBlockIdx < 0, it means setToBlock() has never been called, so it means // we're about to read from the beginning of the partition, but haven't "prepared" the IndexState yet. // Do so by setting us on the first block. http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7c6e14f/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java -- diff --git a/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java b/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java index 00f0ca2..4838392 100644 --- a/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java +++ b/test/unit/org/apache/cassandra/cql3/QueryWithIndexedSSTableTest.java @@ -65,6 +65,10 @@ public class QueryWithIndexedSSTableTest extends CQLTester assert hasIndexed; assertRowCount(execute("SELECT s FROM %s WHERE k = ?", 0), ROWS); +assertRowCount(execute("SELECT s FROM %s WHERE k = ? ORDER BY t DESC", 0), ROWS); + +assertRowCount(execute("SELECT DISTINCT s FROM %s WHERE k = ?", 0), 1); +assertRowCount(execute("SELECT DISTINCT s FROM %s WHERE k = ? ORDER BY t DESC", 0), 1); } // Creates a random string
[jira] [Commented] (CASSANDRA-10903) AssertionError while reading sstable when querying static column
[ https://issues.apache.org/jira/browse/CASSANDRA-10903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069524#comment-15069524 ] Sylvain Lebresne commented on CASSANDRA-10903: -- I've pushed a simple followup that adds assertion and comment for {{udpateBlock}} and adds the reverse case in the test (but again, indirect testing through CQL is not ideal and CASSANDRA-10911 will be a better to get decently exhausting testing of this). For the records, the reverse case was not affected because we're unconditionally seeking to the last block which wasn't done as part of the {{init}} method that was at fault here. > AssertionError while reading sstable when querying static column > > > Key: CASSANDRA-10903 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10903 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Tyler Hobbs >Assignee: Sylvain Lebresne > Fix For: 3.0.x, 3.x > > > I'm able to consistently reproduce the following error: > {noformat} > 1 WARN [SharedPool-Worker-1] 2015-12-17 17:37:00,196 > AbstractTracingAwareExecutorService.java (line 169) Uncaught exception on > thread Thread[SharedPool-Worker-1,5,main]: {} > java.lang.AssertionError: null > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator$IndexState.updateBlock(AbstractSSTableIterator.java:463) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardIndexedReader.computeNext(SSTableIterator.java:268) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:158) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:352) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:219) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:32) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:369) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:426) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:286) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:108) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:298) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:136) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:128) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:123) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:288) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at >
[jira] [Commented] (CASSANDRA-10903) AssertionError while reading sstable when querying static column
[ https://issues.apache.org/jira/browse/CASSANDRA-10903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069546#comment-15069546 ] Branimir Lambov commented on CASSANDRA-10903: - LGTM > AssertionError while reading sstable when querying static column > > > Key: CASSANDRA-10903 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10903 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Tyler Hobbs >Assignee: Sylvain Lebresne > Fix For: 3.0.x, 3.x > > > I'm able to consistently reproduce the following error: > {noformat} > 1 WARN [SharedPool-Worker-1] 2015-12-17 17:37:00,196 > AbstractTracingAwareExecutorService.java (line 169) Uncaught exception on > thread Thread[SharedPool-Worker-1,5,main]: {} > java.lang.AssertionError: null > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator$IndexState.updateBlock(AbstractSSTableIterator.java:463) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardIndexedReader.computeNext(SSTableIterator.java:268) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:158) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:352) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:219) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:32) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:369) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:426) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:286) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:108) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:298) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:136) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:128) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:123) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:288) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1692) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2346) > ~[cassandra-all-3.0.0.740.jar:3.0.0.740] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_45] > at >
[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=15069610#comment-15069610 ] Stefania commented on CASSANDRA-10428: -- Here is the patch for trunk and for the dtests since changing the default timestamp format breaks a few of them (both patches must be committed at the same time to avoid failures): * [trunk patch|https://github.com/stef1927/cassandra/commits/10428] * [dtest patch|https://github.com/stef1927/cassandra-dtest/commits/10428] * [dtest results|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10428-dtest/] CI is still pending. I could not reproduce the {{%f}} observed by [~reach.nchan]. > 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-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069834#comment-15069834 ] Tupshin Harper commented on CASSANDRA-8844: --- While I haven't really followed how MVs are doing mutation-based repair, your idea to go down that path mirrors my own thinking. to clarify, I believe there are two separate issues: 1) Currently, nothing, including repair, is able to cause a partially replicated CDC table to converge towards fully CDC-replicated, even when only worrying about delivering the latest copy and not caring about intermediate mutations 2) intermediate mutations aren't retained, and therefore any plausible fixes to #1, short of mutation-based repair, will still not recover all mutations that were applied to mutable-state columns. So +1 to [~JoshuaMcKenzie]'s suggestion. > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be built into the > daemons. > h2. Format and Consumption > - Cassandra would only write to the CDC log, and never delete from it. > - Cleaning up consumed logfiles would be the client daemon's responibility > - Logfile size should probably be configurable. > - Logfiles should be named with a predictable naming schema, making it > triivial to process them in order. > - Daemons should be able to checkpoint their work, and resume from where they > left off. This
[jira] [Updated] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-9303: - Fix Version/s: 2.2.x 2.1.x > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10854) cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter is throwing 'list index out of range'
[ https://issues.apache.org/jira/browse/CASSANDRA-10854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069637#comment-15069637 ] Stefania commented on CASSANDRA-10854: -- We need to keep the current approach in case users want to insert an empty string in one of the primary key components, for example a clustering key. It works as follows: # if the value is not equal to the NULL option value then we parse it and in this case we get {{invalid literal for int()}} because the key is an integer but for a text key it would have been a valid value # If the value is equal to the NULL option value (by default an empty string) then we insert {{None}} for any column except for primary keys in which case we complain with {{Cannot insert null value for primary key column ...}}. # The reason the default NULL option is an empty string is because the csv parser returns an empty string for missing values. > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > > > Key: CASSANDRA-10854 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10854 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: cqlsh 5.0.1 | Cassandra 2.1.11.969 | DSE 4.8.3 | CQL > spec 3.2.1 >Reporter: Puspendu Banerjee >Assignee: Stefania >Priority: Minor > > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > Steps to re-produce: > {code} > CREATE TABLE tracks_by_album ( > album_title TEXT, > album_year INT, > performer TEXT STATIC, > album_genre TEXT STATIC, > track_number INT, > track_title TEXT, > PRIMARY KEY ((album_title, album_year), track_number) > ); > {code} > Create a file: tracks_by_album.csv having following 2 lines : > {code} > album,year,performer,genre,number,title > a,2015,b c d,e f g,, > {code} > {code} > cqlsh> COPY music.tracks_by_album > (album_title, album_year, performer, album_genre, track_number, > track_title) > FROM '~/tracks_by_album.csv' > WITH HEADER = 'true'; > Error : > Starting copy of music.tracks_by_album with columns ['album_title', > 'album_year', 'performer', 'album_genre', 'track_number', 'track_title']. > list index out of range > Aborting import at record #1. Previously inserted records are still present, > and some records after that may be present as well. > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7396) Allow selecting Map key, List index
[ https://issues.apache.org/jira/browse/CASSANDRA-7396?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069646#comment-15069646 ] Robert Stupp commented on CASSANDRA-7396: - I still have [a branch|https://github.com/snazy/cassandra/tree/7396-coll-slice] regularly rebased on trunk. Not sure, if it still works but it definitely needs some more functionality to read only those "cells" that are actually requested - i.e. only read the collection slice and not the whole collection. The current impl reads the whole collection and just returns the slice. However, we could split this into a separate ticket. > Allow selecting Map key, List index > --- > > Key: CASSANDRA-7396 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7396 > Project: Cassandra > Issue Type: New Feature > Components: CQL >Reporter: Jonathan Ellis >Assignee: Robert Stupp > Labels: cql > Fix For: 3.x > > Attachments: 7396_unit_tests.txt > > > Allow "SELECT map['key]" and "SELECT list[index]." (Selecting a UDT subfield > is already supported.) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9977) Support counter-columns for native aggregates (sum,avg,max,min)
[ https://issues.apache.org/jira/browse/CASSANDRA-9977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069657#comment-15069657 ] Robert Stupp commented on CASSANDRA-9977: - Updated the tests with the nits and scheduled a testall + dtest runs for all three branches. > Support counter-columns for native aggregates (sum,avg,max,min) > --- > > Key: CASSANDRA-9977 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9977 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Noam Liran >Assignee: Robert Stupp > Fix For: 2.2.x > > > When trying to SUM a column of type COUNTER, this error is returned: > {noformat} > InvalidRequest: code=2200 [Invalid query] message="Invalid call to function > sum, none of its type signatures match (known type signatures: system.sum : > (tinyint) -> tinyint, system.sum : (smallint) -> smallint, system.sum : (int) > -> int, system.sum : (bigint) -> bigint, system.sum : (float) -> float, > system.sum : (double) -> double, system.sum : (decimal) -> decimal, > system.sum : (varint) -> varint)" > {noformat} > This might be relevant for other agg. functions. > CQL for reproduction: > {noformat} > CREATE TABLE test ( > key INT, > ctr COUNTER, > PRIMARY KEY ( > key > ) > ); > UPDATE test SET ctr = ctr + 1 WHERE key = 1; > SELECT SUM(ctr) FROM test; > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069832#comment-15069832 ] Jonathan Ellis commented on CASSANDRA-9303: --- I'm reluctant to pull 9302 out, so I'd prefer adding this to 2.1 as well. > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/3] cassandra git commit: Fix assertion error when reading static on an indexed sstable
Fix assertion error when reading static on an indexed sstable patch by slebresne; reviewed by blambov for CASSANDRA-10903 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8bc567b3 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8bc567b3 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8bc567b3 Branch: refs/heads/trunk Commit: 8bc567b3ca40199f6d4a7b6f32971f2fea56a6b9 Parents: 796db6e Author: Sylvain LebresneAuthored: Mon Dec 21 12:14:46 2015 +0100 Committer: Sylvain Lebresne Committed: Wed Dec 23 15:15:14 2015 +0100 -- CHANGES.txt | 1 + .../columniterator/AbstractSSTableIterator.java | 37 - .../db/columniterator/SSTableIterator.java | 38 +++-- .../columniterator/SSTableReversedIterator.java | 28 ++- .../cql3/QueryWithIndexedSSTableTest.java | 84 5 files changed, 121 insertions(+), 67 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bc567b3/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7794c96..a669b17 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.3 + * Fix potential assertion error when reading static columns (CASSANDRA-0903) * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711) * Avoid building PartitionUpdate in toString (CASSANDRA-10897) * Reduce heap spent when receiving many SSTables (CASSANDRA-10797) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bc567b3/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java -- diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java index 5f280d7..8ac3dcb 100644 --- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java +++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java @@ -99,14 +99,14 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator // Note that this needs to be called after file != null and after the partitionDeletion has been set, but before readStaticRow // (since it uses it) so we can't move that up (but we'll be able to simplify as soon as we drop support for the old file format). -this.reader = needsReader ? createReader(indexEntry, file, true, shouldCloseFile) : null; +this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null; this.staticRow = readStaticRow(sstable, file, helper, columns.fetchedColumns().statics, isForThrift, reader == null ? null : reader.deserializer); } else { this.partitionLevelDeletion = indexEntry.deletionTime(); this.staticRow = Rows.EMPTY_STATIC_ROW; -this.reader = needsReader ? createReader(indexEntry, file, false, shouldCloseFile) : null; +this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null; } if (reader == null && file != null && shouldCloseFile) @@ -180,7 +180,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator } } -protected abstract Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile); +protected abstract Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile); public CFMetaData metadata() { @@ -291,19 +291,13 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator // Records the currently open range tombstone (if any) protected DeletionTime openMarker = null; -// !isInit means we have never seeked in the file and thus should seek before reading anything -protected boolean isInit; - -protected Reader(FileDataInput file, boolean isInit, boolean shouldCloseFile) +protected Reader(FileDataInput file, boolean shouldCloseFile) { this.file = file; -this.isInit = isInit; this.shouldCloseFile = shouldCloseFile; if (file != null) createDeserializer(); -else -assert !isInit; } private void createDeserializer() @@ -343,12 +337,6 @@ abstract class AbstractSSTableIterator implements
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069790#comment-15069790 ] Joshua McKenzie commented on CASSANDRA-8844: bq. I do a CL_ONE write and only one replica is available CL_ONE implications for data consistency are the same between regular mutations and CDC, however from a consistency perspective (CL vs. CDC)... bq. a repair operation happens to bring those nodes up to date. This raises a good point. My initial reaction / thought is that an approach similar to the mutation-based repair we're using for MV would allow replicas to process CDC-records for mutations as they're written to the local CL during that repair, meaning the CDC records would then be available and CL / CDC consistency would be retained. I'd like to see more data on the impact on repair performance this has vs. the traditional approach so I'll take a look at that while working on this. > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be built into the > daemons. > h2. Format and Consumption > - Cassandra would only write to the CDC log, and never delete from it. > - Cleaning up consumed logfiles would be the client daemon's responibility > - Logfile size should probably be configurable. > - Logfiles should be named with a predictable naming schema, making it > triivial to process them in
[jira] [Comment Edited] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069743#comment-15069743 ] Stefania edited comment on CASSANDRA-9303 at 12/23/15 3:30 PM: --- bq. I'd agree with Aleksey Yeschenko that this should go only on 3.0+, however, since this is a follow-up/complement to CASSANDRA-9302, which is a new feature and went into an unreleased 2.1 version, I'd advocate for this to go into 2.1 as well, unless CASSANDRA-9302 is removed from 2.1, otherwise the new copy from/to feature would ship half-complete on 2.1 what wouldn't make much sense IMO. I tend to agree that CASSANDRA-9302 is somewhat incomplete without these options. So either we roll it back from 2.1 and 2.2 or we commit this as well. Further, it would be a pain to fix 9302 bugs without this patch since the code changed significantly enough to cause merge conflicts. -[~iamaleksey] WDYT?- Already replied above, comments crossed. was (Author: stefania): bq. I'd agree with Aleksey Yeschenko that this should go only on 3.0+, however, since this is a follow-up/complement to CASSANDRA-9302, which is a new feature and went into an unreleased 2.1 version, I'd advocate for this to go into 2.1 as well, unless CASSANDRA-9302 is removed from 2.1, otherwise the new copy from/to feature would ship half-complete on 2.1 what wouldn't make much sense IMO. I tend to agree that CASSANDRA-9302 is somewhat incomplete without these options. So either we roll it back from 2.1 and 2.2 or we commit this as well. Further, it would be a pain to fix 9302 bugs without this patch since the code changed significantly enough to cause merge conflicts. [~iamaleksey] WDYT? > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[3/3] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/04a99ab8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/04a99ab8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/04a99ab8 Branch: refs/heads/trunk Commit: 04a99ab84cd24c10b2220667667651afb3dd29bf Parents: e5b6a9c 8bc567b Author: Sylvain LebresneAuthored: Wed Dec 23 15:16:05 2015 +0100 Committer: Sylvain Lebresne Committed: Wed Dec 23 15:16:05 2015 +0100 -- CHANGES.txt | 1 + .../columniterator/AbstractSSTableIterator.java | 37 - .../db/columniterator/SSTableIterator.java | 38 +++-- .../columniterator/SSTableReversedIterator.java | 28 ++- .../cql3/QueryWithIndexedSSTableTest.java | 84 5 files changed, 121 insertions(+), 67 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/04a99ab8/CHANGES.txt -- diff --cc CHANGES.txt index e36e4c2,a669b17..ab6cb92 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,28 -1,5 +1,29 @@@ -3.0.3 +3.2 + * Add forceUserDefinedCleanup to allow more flexible cleanup (CASSANDRA-10708) + * (cqlsh) allow setting TTL with COPY (CASSANDRA-9494) + * Fix EstimatedHistogram creation in nodetool tablehistograms (CASSANDRA-10859) + * Establish bootstrap stream sessions sequentially (CASSANDRA-6992) + * Sort compactionhistory output by timestamp (CASSANDRA-10464) + * More efficient BTree removal (CASSANDRA-9991) + * Make tablehistograms accept the same syntax as tablestats (CASSANDRA-10149) + * Group pending compactions based on table (CASSANDRA-10718) + * Add compressor name in sstablemetadata output (CASSANDRA-9879) + * Fix type casting for counter columns (CASSANDRA-10824) + * Prevent running Cassandra as root (CASSANDRA-8142) + * bound maximum in-flight commit log replay mutation bytes to 64 megabytes (CASSANDRA-8639) + * Normalize all scripts (CASSANDRA-10679) + * Make compression ratio much more accurate (CASSANDRA-10225) + * Optimize building of Clustering object when only one is created (CASSANDRA-10409) + * Make index building pluggable (CASSANDRA-10681) + * Add sstable flush observer (CASSANDRA-10678) + * Improve NTS endpoints calculation (CASSANDRA-10200) + * Improve performance of the folderSize function (CASSANDRA-10677) + * Add support for type casting in selection clause (CASSANDRA-10310) + * Added graphing option to cassandra-stress (CASSANDRA-7918) + * Abort in-progress queries that time out (CASSANDRA-7392) + * Add transparent data encryption core classes (CASSANDRA-9945) +Merged from 3.0: + * Fix potential assertion error when reading static columns (CASSANDRA-0903) * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711) * Avoid building PartitionUpdate in toString (CASSANDRA-10897) * Reduce heap spent when receiving many SSTables (CASSANDRA-10797)
[1/3] cassandra git commit: Fix assertion error when reading static on an indexed sstable
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 796db6e09 -> 8bc567b3c refs/heads/trunk e5b6a9c8b -> 04a99ab84 Fix assertion error when reading static on an indexed sstable patch by slebresne; reviewed by blambov for CASSANDRA-10903 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8bc567b3 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8bc567b3 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8bc567b3 Branch: refs/heads/cassandra-3.0 Commit: 8bc567b3ca40199f6d4a7b6f32971f2fea56a6b9 Parents: 796db6e Author: Sylvain LebresneAuthored: Mon Dec 21 12:14:46 2015 +0100 Committer: Sylvain Lebresne Committed: Wed Dec 23 15:15:14 2015 +0100 -- CHANGES.txt | 1 + .../columniterator/AbstractSSTableIterator.java | 37 - .../db/columniterator/SSTableIterator.java | 38 +++-- .../columniterator/SSTableReversedIterator.java | 28 ++- .../cql3/QueryWithIndexedSSTableTest.java | 84 5 files changed, 121 insertions(+), 67 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bc567b3/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7794c96..a669b17 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.3 + * Fix potential assertion error when reading static columns (CASSANDRA-0903) * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711) * Avoid building PartitionUpdate in toString (CASSANDRA-10897) * Reduce heap spent when receiving many SSTables (CASSANDRA-10797) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bc567b3/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java -- diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java index 5f280d7..8ac3dcb 100644 --- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java +++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java @@ -99,14 +99,14 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator // Note that this needs to be called after file != null and after the partitionDeletion has been set, but before readStaticRow // (since it uses it) so we can't move that up (but we'll be able to simplify as soon as we drop support for the old file format). -this.reader = needsReader ? createReader(indexEntry, file, true, shouldCloseFile) : null; +this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null; this.staticRow = readStaticRow(sstable, file, helper, columns.fetchedColumns().statics, isForThrift, reader == null ? null : reader.deserializer); } else { this.partitionLevelDeletion = indexEntry.deletionTime(); this.staticRow = Rows.EMPTY_STATIC_ROW; -this.reader = needsReader ? createReader(indexEntry, file, false, shouldCloseFile) : null; +this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null; } if (reader == null && file != null && shouldCloseFile) @@ -180,7 +180,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator } } -protected abstract Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile); +protected abstract Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile); public CFMetaData metadata() { @@ -291,19 +291,13 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator // Records the currently open range tombstone (if any) protected DeletionTime openMarker = null; -// !isInit means we have never seeked in the file and thus should seek before reading anything -protected boolean isInit; - -protected Reader(FileDataInput file, boolean isInit, boolean shouldCloseFile) +protected Reader(FileDataInput file, boolean shouldCloseFile) { this.file = file; -this.isInit = isInit; this.shouldCloseFile = shouldCloseFile; if (file != null) createDeserializer(); -else -assert !isInit; }
[jira] [Commented] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069701#comment-15069701 ] Aleksey Yeschenko commented on CASSANDRA-9303: -- This is tricky. This *should* only go to 3.x. 2.1 is close to EOL and at this stage should only include critical bug fixes. That said, for pragmatic reasons, committing to 3.0.x as well should outweigh rues breakage, so I'm fine if we do it (this one time). > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069739#comment-15069739 ] Aleksey Yeschenko commented on CASSANDRA-9303: -- This situation is unfortunate, but you are right. Technically we would revert CASSANDRA-9302 and CASSANDRA-9304 from 2.1, but it does seem easier to just go ahead and commit this to 2.1. Actually, I'm fine with either option. Revert the previous commits or commit this patch to 2.1 as well. [~jbellis], as the reporter of this JIRA, what'd be your preference? > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069743#comment-15069743 ] Stefania commented on CASSANDRA-9303: - bq. I'd agree with Aleksey Yeschenko that this should go only on 3.0+, however, since this is a follow-up/complement to CASSANDRA-9302, which is a new feature and went into an unreleased 2.1 version, I'd advocate for this to go into 2.1 as well, unless CASSANDRA-9302 is removed from 2.1, otherwise the new copy from/to feature would ship half-complete on 2.1 what wouldn't make much sense IMO. I tend to agree that CASSANDRA-9302 is somewhat incomplete without these options. So either we roll it back from 2.1 and 2.2 or we commit this as well. Further, it would be a pain to fix 9302 bugs without this patch since the code changed significantly enough to cause merge conflicts. [~iamaleksey] WDYT? > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (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:all-tabpanel ] Joshua McKenzie updated CASSANDRA-10428: Reviewer: Paulo Motta > 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-10854) cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter is throwing 'list index out of range'
[ https://issues.apache.org/jira/browse/CASSANDRA-10854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069642#comment-15069642 ] Paulo Motta commented on CASSANDRA-10854: - Sounds good. Thanks for the clarification and sorry for the confusion! Marking as "Ready to commit". Thanks! > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > > > Key: CASSANDRA-10854 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10854 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: cqlsh 5.0.1 | Cassandra 2.1.11.969 | DSE 4.8.3 | CQL > spec 3.2.1 >Reporter: Puspendu Banerjee >Assignee: Stefania >Priority: Minor > > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > Steps to re-produce: > {code} > CREATE TABLE tracks_by_album ( > album_title TEXT, > album_year INT, > performer TEXT STATIC, > album_genre TEXT STATIC, > track_number INT, > track_title TEXT, > PRIMARY KEY ((album_title, album_year), track_number) > ); > {code} > Create a file: tracks_by_album.csv having following 2 lines : > {code} > album,year,performer,genre,number,title > a,2015,b c d,e f g,, > {code} > {code} > cqlsh> COPY music.tracks_by_album > (album_title, album_year, performer, album_genre, track_number, > track_title) > FROM '~/tracks_by_album.csv' > WITH HEADER = 'true'; > Error : > Starting copy of music.tracks_by_album with columns ['album_title', > 'album_year', 'performer', 'album_genre', 'track_number', 'track_title']. > list index out of range > Aborting import at record #1. Previously inserted records are still present, > and some records after that may be present as well. > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10854) cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter is throwing 'list index out of range'
[ https://issues.apache.org/jira/browse/CASSANDRA-10854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069617#comment-15069617 ] Paulo Motta commented on CASSANDRA-10854: - With the current approach if you try to insert an empty {{int}} value, the message will vary: * {{Failed to import 4 rows: ValueError - invalid literal for int() with base 10: '' - given up after 1 attempts}} will be print if {{WITH NULL}} is set. * {{Failed to import 4 rows: ValueError - Cannot insert null value for primary key column 'bla'. If you want to insert empty strings, consider using the WITH NULL= option for COPY. - given up after 1 attempts}} will be print if {{WITH NULL}} is not set. So it would be nice to unify those. And I think we can also get rid of the {{If you want to insert empty strings, consider using the WITH NULL= option for COPY}} message, since even if it's a text, we cannot have nulls in the primary keys so this might confuse users. I'd vote for just having the message {{Failed to import 1 rows: ValueError - Cannot insert null value for primary key column 'bla2'}}, regardless of type and if nullval is set. > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > > > Key: CASSANDRA-10854 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10854 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: cqlsh 5.0.1 | Cassandra 2.1.11.969 | DSE 4.8.3 | CQL > spec 3.2.1 >Reporter: Puspendu Banerjee >Assignee: Stefania >Priority: Minor > > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > Steps to re-produce: > {code} > CREATE TABLE tracks_by_album ( > album_title TEXT, > album_year INT, > performer TEXT STATIC, > album_genre TEXT STATIC, > track_number INT, > track_title TEXT, > PRIMARY KEY ((album_title, album_year), track_number) > ); > {code} > Create a file: tracks_by_album.csv having following 2 lines : > {code} > album,year,performer,genre,number,title > a,2015,b c d,e f g,, > {code} > {code} > cqlsh> COPY music.tracks_by_album > (album_title, album_year, performer, album_genre, track_number, > track_title) > FROM '~/tracks_by_album.csv' > WITH HEADER = 'true'; > Error : > Starting copy of music.tracks_by_album with columns ['album_title', > 'album_year', 'performer', 'album_genre', 'track_number', 'track_title']. > list index out of range > Aborting import at record #1. Previously inserted records are still present, > and some records after that may be present as well. > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-9303: - Fix Version/s: (was: 2.1.x) 3.x 3.0.x > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069736#comment-15069736 ] Paulo Motta commented on CASSANDRA-9303: Looks good now. Tested locally and all options look good. Dtests are also passing. Only minor nit is to use {{os.linesep}} instead of {{'\n'}} on {{_printmsg(msg, eol='\n')}}. bq. It doesn't work because stdin is actually set to the file specified with the -f option. Since this is not an issue with COPY but with the way -f is implemented, I would prefer deferring to another ticket if this functionality is required. +1 bq. I've also rebased on the 2.1 branch (since CASSANDRA-9494 will only be on trunk) and applied the fix for CASSANDRA-10854 since it requires extra work on this branch. +1, can you just check the failing dtest {{cqlsh_copy_tests.py:CqlshCopyTest.test_read_missing_partition_key}} from CASSANDRA-10854? bq. I would like to squash the dtest commits as well, let me know if you still need to review some individual commits first. Feel free to squash and up-merge. bq. I'm still waiting to hear about which branches we need to apply this patch to; plus I would like to squash the commits before up-merging. I'd agree with [~iamaleksey] that this should go only on 3.0+, however, since this is a follow-up/complement to CASSANDRA-9302, which is a new feature and went into an unreleased 2.1 version, I'd advocate for this to go into 2.1 as well, unless CASSANDRA-9302 is removed from 2.1, otherwise the *new copy from/to* feature would ship half-complete on 2.1 what wouldn't make much sense. > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9303) Match cassandra-loader options in COPY FROM
[ https://issues.apache.org/jira/browse/CASSANDRA-9303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069736#comment-15069736 ] Paulo Motta edited comment on CASSANDRA-9303 at 12/23/15 3:21 PM: -- Looks good now. Tested locally and all options look good. Dtests are also passing. Only minor nit is to use {{os.linesep}} instead of {{'\n'}} on {{_printmsg(msg, eol='\n')}}. bq. It doesn't work because stdin is actually set to the file specified with the -f option. Since this is not an issue with COPY but with the way -f is implemented, I would prefer deferring to another ticket if this functionality is required. +1 bq. I've also rebased on the 2.1 branch (since CASSANDRA-9494 will only be on trunk) and applied the fix for CASSANDRA-10854 since it requires extra work on this branch. +1, can you just check the failing dtest {{cqlsh_copy_tests.py:CqlshCopyTest.test_read_missing_partition_key}} from CASSANDRA-10854? bq. I would like to squash the dtest commits as well, let me know if you still need to review some individual commits first. Feel free to squash and up-merge. bq. I'm still waiting to hear about which branches we need to apply this patch to; plus I would like to squash the commits before up-merging. I'd agree with [~iamaleksey] that this should go only on 3.0+, however, since this is a follow-up/complement to CASSANDRA-9302, which is a new feature and went into an unreleased 2.1 version, I'd advocate for this to go into 2.1 as well, unless CASSANDRA-9302 is removed from 2.1, otherwise the *new copy from/to* feature would ship half-complete on 2.1 what wouldn't make much sense IMO. was (Author: pauloricardomg): Looks good now. Tested locally and all options look good. Dtests are also passing. Only minor nit is to use {{os.linesep}} instead of {{'\n'}} on {{_printmsg(msg, eol='\n')}}. bq. It doesn't work because stdin is actually set to the file specified with the -f option. Since this is not an issue with COPY but with the way -f is implemented, I would prefer deferring to another ticket if this functionality is required. +1 bq. I've also rebased on the 2.1 branch (since CASSANDRA-9494 will only be on trunk) and applied the fix for CASSANDRA-10854 since it requires extra work on this branch. +1, can you just check the failing dtest {{cqlsh_copy_tests.py:CqlshCopyTest.test_read_missing_partition_key}} from CASSANDRA-10854? bq. I would like to squash the dtest commits as well, let me know if you still need to review some individual commits first. Feel free to squash and up-merge. bq. I'm still waiting to hear about which branches we need to apply this patch to; plus I would like to squash the commits before up-merging. I'd agree with [~iamaleksey] that this should go only on 3.0+, however, since this is a follow-up/complement to CASSANDRA-9302, which is a new feature and went into an unreleased 2.1 version, I'd advocate for this to go into 2.1 as well, unless CASSANDRA-9302 is removed from 2.1, otherwise the *new copy from/to* feature would ship half-complete on 2.1 what wouldn't make much sense. > Match cassandra-loader options in COPY FROM > --- > > Key: CASSANDRA-9303 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9303 > Project: Cassandra > Issue Type: New Feature > Components: Tools >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.0.x, 3.x > > > https://github.com/brianmhess/cassandra-loader added a bunch of options to > handle real world requirements, we should match those. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10730) periodic timeout errors in dtest
[ https://issues.apache.org/jira/browse/CASSANDRA-10730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1506#comment-1506 ] Greg Bestland commented on CASSANDRA-10730: --- Jim, One thing that comes to mind. We had quite a few timeout issues at one point in our jenkins CI environment. The cause was changes made under the covers which caused disk performance to suffer. It might be worth investigating what your disk performance looks like on those nodes. Disk performance can cause all sorts of timeout, and schema agreement problems especially when that disk is shared across multiple c* nodes. Might be worth checking out. > periodic timeout errors in dtest > > > Key: CASSANDRA-10730 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10730 > Project: Cassandra > Issue Type: Bug >Reporter: Jim Witschey >Assignee: Jim Witschey > > Dtests often fail with connection timeout errors. For example: > http://cassci.datastax.com/job/cassandra-3.1_dtest/lastCompletedBuild/testReport/upgrade_tests.cql_tests/TestCQLNodes3RF3/deletion_test/ > {code} > ('Unable to connect to any servers', {'127.0.0.1': > OperationTimedOut('errors=Timed out creating connection (10 seconds), > last_host=None',)}) > {code} > We've merged a PR to increase timeouts: > https://github.com/riptano/cassandra-dtest/pull/663 > It doesn't look like this has improved things: > http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/363/testReport/ > Next steps here are > * to scrape Jenkins history to see if and how the number of tests failing > this way has increased (it feels like it has). From there we can bisect over > the dtests, ccm, or C*, depending on what looks like the source of the > problem. > * to better instrument the dtest/ccm/C* startup process to see why the nodes > start but don't successfully make the CQL port available. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[09/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/382d9a94 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/382d9a94 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/382d9a94 Branch: refs/heads/trunk Commit: 382d9a943bdf33e528fa6509e3bd12c2bc9ecc70 Parents: 8bc567b ee36f14 Author: Joshua McKenzieAuthored: Wed Dec 23 13:21:19 2015 -0500 Committer: Joshua McKenzie Committed: Wed Dec 23 13:21:19 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) --
[05/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ee36f14c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ee36f14c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ee36f14c Branch: refs/heads/cassandra-3.0 Commit: ee36f14c1427db8e3b91b7eda6c0e6f48ecb4f24 Parents: ae8baa4 3ccffc9 Author: Joshua McKenzieAuthored: Wed Dec 23 13:21:12 2015 -0500 Committer: Joshua McKenzie Committed: Wed Dec 23 13:21:12 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee36f14c/pylib/cqlshlib/copyutil.py --
[04/10] cassandra git commit: Fix multiple consecutive delimiters on cqlsh COPY FROM
Fix multiple consecutive delimiters on cqlsh COPY FROM Patch by stefania; reviewed by pmotta for CASSANDRA-10854 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3ccffc94 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3ccffc94 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3ccffc94 Branch: refs/heads/trunk Commit: 3ccffc94b8a6127f6a452a3fb93306ad3bbc8c97 Parents: 7c3966b Author: Stefania AlborghettiAuthored: Tue Dec 22 09:16:54 2015 +0100 Committer: Joshua McKenzie Committed: Wed Dec 23 13:20:27 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3ccffc94/pylib/cqlshlib/copyutil.py -- diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py index f699e64..65f5997 100644 --- a/pylib/cqlshlib/copyutil.py +++ b/pylib/cqlshlib/copyutil.py @@ -1020,16 +1020,19 @@ class ImportConversion(object): ret[i] = self.converters[self.columns[i]](val) else: if i in self.primary_key_indexes: -message = "Cannot insert null value for primary key column '%s'." % (self.columns[i],) -if self.nullval == '': -message += " If you want to insert empty strings, consider using" \ - " the WITH NULL= option for COPY." -raise Exception(message=message) +raise ValueError(self.get_null_primary_key_message(i)) ret[i] = None return ret +def get_null_primary_key_message(self, idx): +message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],) +if self.nullval == '': +message += " If you want to insert empty strings, consider using" \ + " the WITH NULL= option for COPY." +return message + def get_row_partition_key_values(self, row): """ Return a string composed of the partition key values, serialized and binary packed - @@ -1037,6 +1040,8 @@ class ImportConversion(object): """ def serialize(n): c, v = self.columns[n], row[n] +if v == self.nullval: +raise ValueError(self.get_null_primary_key_message(n)) return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version) partition_key_indexes = self.partition_key_indexes
[07/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ee36f14c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ee36f14c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ee36f14c Branch: refs/heads/trunk Commit: ee36f14c1427db8e3b91b7eda6c0e6f48ecb4f24 Parents: ae8baa4 3ccffc9 Author: Joshua McKenzieAuthored: Wed Dec 23 13:21:12 2015 -0500 Committer: Joshua McKenzie Committed: Wed Dec 23 13:21:12 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee36f14c/pylib/cqlshlib/copyutil.py --
[02/10] cassandra git commit: Fix multiple consecutive delimiters on cqlsh COPY FROM
Fix multiple consecutive delimiters on cqlsh COPY FROM Patch by stefania; reviewed by pmotta for CASSANDRA-10854 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3ccffc94 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3ccffc94 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3ccffc94 Branch: refs/heads/cassandra-2.2 Commit: 3ccffc94b8a6127f6a452a3fb93306ad3bbc8c97 Parents: 7c3966b Author: Stefania AlborghettiAuthored: Tue Dec 22 09:16:54 2015 +0100 Committer: Joshua McKenzie Committed: Wed Dec 23 13:20:27 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3ccffc94/pylib/cqlshlib/copyutil.py -- diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py index f699e64..65f5997 100644 --- a/pylib/cqlshlib/copyutil.py +++ b/pylib/cqlshlib/copyutil.py @@ -1020,16 +1020,19 @@ class ImportConversion(object): ret[i] = self.converters[self.columns[i]](val) else: if i in self.primary_key_indexes: -message = "Cannot insert null value for primary key column '%s'." % (self.columns[i],) -if self.nullval == '': -message += " If you want to insert empty strings, consider using" \ - " the WITH NULL= option for COPY." -raise Exception(message=message) +raise ValueError(self.get_null_primary_key_message(i)) ret[i] = None return ret +def get_null_primary_key_message(self, idx): +message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],) +if self.nullval == '': +message += " If you want to insert empty strings, consider using" \ + " the WITH NULL= option for COPY." +return message + def get_row_partition_key_values(self, row): """ Return a string composed of the partition key values, serialized and binary packed - @@ -1037,6 +1040,8 @@ class ImportConversion(object): """ def serialize(n): c, v = self.columns[n], row[n] +if v == self.nullval: +raise ValueError(self.get_null_primary_key_message(n)) return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version) partition_key_indexes = self.partition_key_indexes
[jira] [Commented] (CASSANDRA-10876) Alter behavior of batch WARN and fail on single partition batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070056#comment-15070056 ] Taiyuan Zhang commented on CASSANDRA-10876: --- To make sure I understand this correctly - single partition means single partition key, not single column family(table), right? > Alter behavior of batch WARN and fail on single partition batches > - > > Key: CASSANDRA-10876 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10876 > Project: Cassandra > Issue Type: Improvement >Reporter: Patrick McFadin >Priority: Minor > > In an attempt to give operator insight into potentially harmful batch usage, > Jiras were created to log WARN or fail on certain batch sizes. This ignores > the single partition batch, which doesn't create the same issues as a > multi-partition batch. > The proposal is to ignore size on single partition batch statements. > Reference: > [CASSANDRA-6487|https://issues.apache.org/jira/browse/CASSANDRA-6487] > [CASSANDRA-8011|https://issues.apache.org/jira/browse/CASSANDRA-8011] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070068#comment-15070068 ] Joshua McKenzie commented on CASSANDRA-8844: This is about getting the CDC-log to reflect changes in the local CL or SSTables for any given replica. With respect to that context, the usage of mutations for repair should be equivalent. > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be built into the > daemons. > h2. Format and Consumption > - Cassandra would only write to the CDC log, and never delete from it. > - Cleaning up consumed logfiles would be the client daemon's responibility > - Logfile size should probably be configurable. > - Logfiles should be named with a predictable naming schema, making it > triivial to process them in order. > - Daemons should be able to checkpoint their work, and resume from where they > left off. This means they would have to leave some file artifact in the CDC > log's directory. > - A sophisticated daemon should be able to be written that could > -- Catch up, in written-order, even when it is multiple logfiles behind in > processing > -- Be able to continuously "tail" the most recent logfile and get > low-latency(ms?) access to the data as it is written. > h2. Alternate approach > In order to make consuming a change log easy and efficient to do with low >
[jira] [Updated] (CASSANDRA-10929) cql_tests.py:AbortedQueriesTester fails
[ https://issues.apache.org/jira/browse/CASSANDRA-10929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-10929: Summary: cql_tests.py:AbortedQueriesTester fails (was: cql_tests.py:AbortedQueriesTester.remote_query_test fails) > cql_tests.py:AbortedQueriesTester fails > --- > > Key: CASSANDRA-10929 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10929 > Project: Cassandra > Issue Type: Sub-task > Components: CQL >Reporter: Philip Thompson > Fix For: 3.0.x, 3.x > > Attachments: node1.log, node1_debug.log, node2.log, node2_debug.log > > > All four tests in the {{cql_tests.AbortedQueriesTester}} dtest suite are > failing on HEAD of cassandra-3.0, here is an example link from cassci: > http://cassci.datastax.com/job/cassandra-3.0_dtest/455/testReport/cql_tests/AbortedQueriesTester/remote_query_test/ > The tests set {{'read_request_timeout_in_ms': 1000}} and > {{"-Dcassandra.test.read_iteration_delay_ms=1500"}}, then issues read queries > and expects them to timeout. However, they are succeeding. I can reproduce > this locally. > Looking at remote_query_test, from the logs, it appears that the query is > being sent from the driver to node1, which forwards it to node2 > appropriately. I've tried also setting {{range_request_timeout_in_ms}} lower, > but that has had no effect. Trace logs from remote_query_test are attached. > The same issue is happening on local_query_test, remote_query_test, > materialized_view_test, and index_query_test. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10929) cql_tests.py:AbortedQueriesTester.remote_query_test fails
[ https://issues.apache.org/jira/browse/CASSANDRA-10929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-10929: Description: All four tests in the {{cql_tests.AbortedQueriesTester}} dtest suite are failing on HEAD of cassandra-3.0, here is an example link from cassci: http://cassci.datastax.com/job/cassandra-3.0_dtest/455/testReport/cql_tests/AbortedQueriesTester/remote_query_test/ The tests set {{'read_request_timeout_in_ms': 1000}} and {{"-Dcassandra.test.read_iteration_delay_ms=1500"}}, then issues read queries and expects them to timeout. However, they are succeeding. I can reproduce this locally. Looking at remote_query_test, from the logs, it appears that the query is being sent from the driver to node1, which forwards it to node2 appropriately. I've tried also setting {{range_request_timeout_in_ms}} lower, but that has had no effect. Trace logs from remote_query_test are attached. The same issue is happening on local_query_test, remote_query_test, materialized_view_test, and index_query_test. was: {{cql_tests.AbortedQueriesTester.remote_query_test}} is failing on HEAD of cassandra-3.0, see this link from cassci: http://cassci.datastax.com/job/cassandra-3.0_dtest/455/testReport/cql_tests/AbortedQueriesTester/remote_query_test/ The test sets {{'read_request_timeout_in_ms': 1000}} and {{"-Dcassandra.test.read_iteration_delay_ms=1500"}}, then issues read queries and expects them to timeout. However, they are succeeding. I can reproduce this locally. From the logs, it appears that the query is being sent from the driver to node1, which forwards it to node2 appropriately. I've tried also setting {{range_request_timeout_in_ms}} lower, but that has had no effect. Trace logs are attached. > cql_tests.py:AbortedQueriesTester.remote_query_test fails > - > > Key: CASSANDRA-10929 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10929 > Project: Cassandra > Issue Type: Sub-task > Components: CQL >Reporter: Philip Thompson > Fix For: 3.0.x, 3.x > > Attachments: node1.log, node1_debug.log, node2.log, node2_debug.log > > > All four tests in the {{cql_tests.AbortedQueriesTester}} dtest suite are > failing on HEAD of cassandra-3.0, here is an example link from cassci: > http://cassci.datastax.com/job/cassandra-3.0_dtest/455/testReport/cql_tests/AbortedQueriesTester/remote_query_test/ > The tests set {{'read_request_timeout_in_ms': 1000}} and > {{"-Dcassandra.test.read_iteration_delay_ms=1500"}}, then issues read queries > and expects them to timeout. However, they are succeeding. I can reproduce > this locally. > Looking at remote_query_test, from the logs, it appears that the query is > being sent from the driver to node1, which forwards it to node2 > appropriately. I've tried also setting {{range_request_timeout_in_ms}} lower, > but that has had no effect. Trace logs from remote_query_test are attached. > The same issue is happening on local_query_test, remote_query_test, > materialized_view_test, and index_query_test. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10580) Add latency metrics for dropped messages
[ https://issues.apache.org/jira/browse/CASSANDRA-10580?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paulo Motta updated CASSANDRA-10580: Fix Version/s: (was: 2.2.x) > Add latency metrics for dropped messages > > > Key: CASSANDRA-10580 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10580 > Project: Cassandra > Issue Type: Improvement > Components: Coordination, Observability > Environment: Production >Reporter: Anubhav Kale >Assignee: Anubhav Kale >Priority: Minor > Fix For: 3.2 > > Attachments: 0001-Metrics.patch, 10580-Metrics.patch, 10580.patch, > 2.2-All-Comments.patch, CASSANDRA-10580-Head.patch, Trunk-All-Comments.patch, > Trunk.patch > > > In our production cluster, we are seeing a large number of dropped mutations. > At a minimum, we should print the time the thread took to get scheduled > thereby dropping the mutation (We should also print the Message / Mutation so > it helps in figuring out which column family was affected). This will help > find the right tuning parameter for write_timeout_in_ms. > The change is small and is in StorageProxy.java and MessagingTask.java. I > will submit a patch shortly. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10854) cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter is throwing 'list index out of range'
[ https://issues.apache.org/jira/browse/CASSANDRA-10854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069889#comment-15069889 ] Stefania commented on CASSANDRA-10854: -- Thanks Paulo, I've created the dtest pull request [here|https://github.com/riptano/cassandra-dtest/pull/721]. > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > > > Key: CASSANDRA-10854 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10854 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: cqlsh 5.0.1 | Cassandra 2.1.11.969 | DSE 4.8.3 | CQL > spec 3.2.1 >Reporter: Puspendu Banerjee >Assignee: Stefania >Priority: Minor > > cqlsh COPY FROM csv having line with more than one consecutive ',' delimiter > is throwing 'list index out of range' > Steps to re-produce: > {code} > CREATE TABLE tracks_by_album ( > album_title TEXT, > album_year INT, > performer TEXT STATIC, > album_genre TEXT STATIC, > track_number INT, > track_title TEXT, > PRIMARY KEY ((album_title, album_year), track_number) > ); > {code} > Create a file: tracks_by_album.csv having following 2 lines : > {code} > album,year,performer,genre,number,title > a,2015,b c d,e f g,, > {code} > {code} > cqlsh> COPY music.tracks_by_album > (album_title, album_year, performer, album_genre, track_number, > track_title) > FROM '~/tracks_by_album.csv' > WITH HEADER = 'true'; > Error : > Starting copy of music.tracks_by_album with columns ['album_title', > 'album_year', 'performer', 'album_genre', 'track_number', 'track_title']. > list index out of range > Aborting import at record #1. Previously inserted records are still present, > and some records after that may be present as well. > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10931) CassandraVersion complains about 3.x version strings
Robert Stupp created CASSANDRA-10931: Summary: CassandraVersion complains about 3.x version strings Key: CASSANDRA-10931 URL: https://issues.apache.org/jira/browse/CASSANDRA-10931 Project: Cassandra Issue Type: Bug Reporter: Robert Stupp The utest {{SystemKeyspaceTest.snapshotSystemKeyspaceIfUpgrading}} fails with {{java.lang.IllegalArgumentException: Invalid version value: 3.2}} (e.g. [here|http://cassci.datastax.com/view/trunk/job/trunk_testall/629/testReport/org.apache.cassandra.db/SystemKeyspaceTest/snapshotSystemKeyspaceIfUpgrading/]). The question is just: # change the regex pattern in {{CassandraVersion}} and silently assume {{.0}} as the patch version or # go with {{x.y.0}} version strings instead of {{x.y}} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10932) pushed_notifications_test.py schema_changes_test is failing
Philip Thompson created CASSANDRA-10932: --- Summary: pushed_notifications_test.py schema_changes_test is failing Key: CASSANDRA-10932 URL: https://issues.apache.org/jira/browse/CASSANDRA-10932 Project: Cassandra Issue Type: Sub-task Reporter: Philip Thompson Fix For: 3.0.x, 3.x {{pushed_notifications_test.py:TestPushedNotifications.schema_changes_test}} is failing on HEAD of cassandra-3.0. It may be simply a problem with the test assertions, so someone just needs to double check if the schema change notifications pushed to the driver are correct. In actuality, the driver gets 8 notifications, listed in the debug output of the test failure: {code} == FAIL: schema_changes_test (pushed_notifications_test.TestPushedNotifications) -- Traceback (most recent call last): File "/Users/philipthompson/cstar/cassandra-dtest/tools.py", line 253, in wrapped f(obj) File "/Users/philipthompson/cstar/cassandra-dtest/pushed_notifications_test.py", line 244, in schema_changes_test self.assertEquals(14, len(notifications)) AssertionError: 14 != 8 >> begin captured logging << dtest: DEBUG: cluster ccm directory: /var/folders/v3/z4wf_34n1q506_xjdy49gb78gn/T/dtest-93xMe2 dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'CREATED', 'target_type': uCorrect typo in MV creation query 'KEYSPACE'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'CREATED', 'target_type': u'TABLE', u'table': u't'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'UPDATED', 'target_type': u'TABLE', u'table': u't'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'CREATED', 'target_type': u'TABLE', u'table': u'mv'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'UPDATED', 'target_type': u'TABLE', u'table': u'mv'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'DROPPED', 'target_type': u'TABLE', u'table': u'mv'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'DROPPED', 'target_type': u'TABLE', u'table': u't'} dtest: DEBUG: Source 127.0.0.2 sent {'keyspace': u'ks', 'change_type': u'DROPPED', 'target_type': u'KEYSPACE'} dtest: DEBUG: Waiting for notifications from 127.0.0.2 - >> end captured logging << - {code} The test has been expecting the following 14, though: {code} self.assertDictContainsSubset({'change_type': u'CREATED', 'target_type': u'KEYSPACE'}, notifications[0]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'KEYSPACE'}, notifications[1]) self.assertDictContainsSubset({'change_type': u'CREATED', 'target_type': u'TABLE', u'table': u't'}, notifications[2]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'KEYSPACE'}, notifications[3]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'TABLE', u'table': u't'}, notifications[4]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'KEYSPACE'}, notifications[5]) self.assertDictContainsSubset({'change_type': u'CREATED', 'target_type': u'TABLE', u'table': u'mv'}, notifications[6]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'KEYSPACE'}, notifications[7]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'TABLE', u'table': u'mv'}, notifications[8]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'KEYSPACE'}, notifications[9]) self.assertDictContainsSubset({'change_type': u'DROPPED', 'target_type': u'TABLE', u'table': u'mv'}, notifications[10]) self.assertDictContainsSubset({'change_type': u'UPDATED', 'target_type': u'KEYSPACE'}, notifications[11]) self.assertDictContainsSubset({'change_type': u'DROPPED', 'target_type': u'TABLE', u'table': u't'}, notifications[12]) self.assertDictContainsSubset({'change_type': u'DROPPED', 'target_type': u'KEYSPACE'}, notifications[13]) {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9244) replace_address is not topology-aware
[ https://issues.apache.org/jira/browse/CASSANDRA-9244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069940#comment-15069940 ] Paulo Motta commented on CASSANDRA-9244: Created [dtests|https://github.com/pauloricardomg/cassandra-dtest/commit/6f08c82c9a13df4f909c85d6aadf44da43409226] to reproduce this by using a replacement node from a rack that already replicates the replaced node primary range. In this case, the other node from the same rack is removed from the replication group but its data is not streamed to the next node in a different rack that becomes a new replica of the range. The issue does not happen when replacing the node with a node from the same rack, or with a node from a "non-overlapping" rack. The issue is reproducible from 2.0 all the way up to trunk. I suspect the issue does not happen when bootstrapping with {{tokens-1}} because the failed node is removed afterwards with {{nodetool removenode}} so that will trigger the data movement to the correct racks. Solution that comes to mind is to detect that a range is already replicated in the same rack, and move the data of the previous replicating node to the next non-overlapping rack. I will investigate the feasibility of this approach. > replace_address is not topology-aware > - > > Key: CASSANDRA-9244 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9244 > Project: Cassandra > Issue Type: Bug > Components: Distributed Metadata, Streaming and Messaging > Environment: 2.0.12 >Reporter: Rick Branson >Assignee: Paulo Motta > > Replaced a node with one in another rack (using replace_address) and it > caused improper distribution after the bootstrap was finished. It looks like > the ranges for the streams are not created in a way that is topology-aware. > This should probably either be prevented, or ideally, would work properly. > The use case is migrating several nodes from one rack to another. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9244) replace_address is not topology-aware
[ https://issues.apache.org/jira/browse/CASSANDRA-9244?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paulo Motta updated CASSANDRA-9244: --- Reproduced In: 3.1, 3.0.2, 2.1.12, 2.0.17 Fix Version/s: 3.x 3.0.x 2.2.x 2.1.x > replace_address is not topology-aware > - > > Key: CASSANDRA-9244 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9244 > Project: Cassandra > Issue Type: Bug > Components: Distributed Metadata, Streaming and Messaging > Environment: 2.0.12 >Reporter: Rick Branson >Assignee: Paulo Motta > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > > Replaced a node with one in another rack (using replace_address) and it > caused improper distribution after the bootstrap was finished. It looks like > the ranges for the streams are not created in a way that is topology-aware. > This should probably either be prevented, or ideally, would work properly. > The use case is migrating several nodes from one rack to another. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10580) Add latency metrics for dropped messages
[ https://issues.apache.org/jira/browse/CASSANDRA-10580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069937#comment-15069937 ] Anubhav Kale commented on CASSANDRA-10580: -- Thanks for your help in working through this. > Add latency metrics for dropped messages > > > Key: CASSANDRA-10580 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10580 > Project: Cassandra > Issue Type: Improvement > Components: Coordination, Observability > Environment: Production >Reporter: Anubhav Kale >Assignee: Anubhav Kale >Priority: Minor > Fix For: 3.2 > > Attachments: 0001-Metrics.patch, 10580-Metrics.patch, 10580.patch, > 2.2-All-Comments.patch, CASSANDRA-10580-Head.patch, Trunk-All-Comments.patch, > Trunk.patch > > > In our production cluster, we are seeing a large number of dropped mutations. > At a minimum, we should print the time the thread took to get scheduled > thereby dropping the mutation (We should also print the Message / Mutation so > it helps in figuring out which column family was affected). This will help > find the right tuning parameter for write_timeout_in_ms. > The change is small and is in StorageProxy.java and MessagingTask.java. I > will submit a patch shortly. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10934) upgrade_test map_keys_indexing_test
Philip Thompson created CASSANDRA-10934: --- Summary: upgrade_test map_keys_indexing_test Key: CASSANDRA-10934 URL: https://issues.apache.org/jira/browse/CASSANDRA-10934 Project: Cassandra Issue Type: Sub-task Reporter: Philip Thompson 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-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 ] Philip Thompson updated CASSANDRA-10934: Summary: upgrade_test map_keys_indexing_test is failing (was: upgrade_test map_keys_indexing_test) > 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 > 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] [Created] (CASSANDRA-10929) cql_tests.py:AbortedQueriesTester.remote_query_test fails
Philip Thompson created CASSANDRA-10929: --- Summary: cql_tests.py:AbortedQueriesTester.remote_query_test fails Key: CASSANDRA-10929 URL: https://issues.apache.org/jira/browse/CASSANDRA-10929 Project: Cassandra Issue Type: Sub-task Components: CQL Reporter: Philip Thompson Fix For: 3.0.x, 3.x Attachments: node1.log, node1_debug.log, node2.log, node2_debug.log {{cql_tests.AbortedQueriesTester.remote_query_test}} is failing on HEAD of cassandra-3.0, see this link from cassci: http://cassci.datastax.com/job/cassandra-3.0_dtest/455/testReport/cql_tests/AbortedQueriesTester/remote_query_test/ The test sets {{'read_request_timeout_in_ms': 1000}} and {{"-Dcassandra.test.read_iteration_delay_ms=1500"}}, then issues read queries and expects them to timeout. However, they are succeeding. I can reproduce this locally. From the logs, it appears that the query is being sent from the driver to node1, which forwards it to node2 appropriately. I've tried also setting {{range_request_timeout_in_ms}} lower, but that has had no effect. Trace logs are attached. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10875) cqlsh fails to decode utf-8 characters for text typed columns.
[ https://issues.apache.org/jira/browse/CASSANDRA-10875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-10875: -- Fix Version/s: (was: 2.1.13) (was: 3.1) 3.x 3.0.x 2.2.x 2.1.x > cqlsh fails to decode utf-8 characters for text typed columns. > -- > > Key: CASSANDRA-10875 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10875 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Yasuharu Goto >Assignee: Yasuharu Goto >Priority: Minor > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > Attachments: 10875-2.1.12.txt, 10875-3.1.txt > > > Hi, we've found a bug that cqlsh can't handle unicode text in select > conditions even if it were text type. > {noformat} > $ ./bin/cqlsh > Connected to Test Cluster at 127.0.0.1:9042. > [cqlsh 5.0.1 | Cassandra 3.2-SNAPSHOT | CQL spec 3.3.1 | Native protocol v4] > Use HELP for help. > cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', > 'replication_factor': 1}; > cqlsh> create table test.test(txt text primary key); > cqlsh> insert into test.test (txt) values('日本語'); > cqlsh> select * from test.test where txt='日本語'; > 'ascii' codec can't decode byte 0xe6 in position 35: ordinal not in range(128) > cqlsh> > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069876#comment-15069876 ] Joshua McKenzie commented on CASSANDRA-8844: bq. I haven't really followed how MVs are doing mutation-based repair [3.0 mutation-based MV repair|https://github.com/apache/cassandra/blob/cassandra-3.0.2/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java#L158-L178] > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be built into the > daemons. > h2. Format and Consumption > - Cassandra would only write to the CDC log, and never delete from it. > - Cleaning up consumed logfiles would be the client daemon's responibility > - Logfile size should probably be configurable. > - Logfiles should be named with a predictable naming schema, making it > triivial to process them in order. > - Daemons should be able to checkpoint their work, and resume from where they > left off. This means they would have to leave some file artifact in the CDC > log's directory. > - A sophisticated daemon should be able to be written that could > -- Catch up, in written-order, even when it is multiple logfiles behind in > processing > -- Be able to continuously "tail" the most recent logfile and get > low-latency(ms?) access to the data as it is written. > h2. Alternate approach > In order to make consuming a change
[jira] [Commented] (CASSANDRA-10918) remove leftover code from refactor
[ https://issues.apache.org/jira/browse/CASSANDRA-10918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069931#comment-15069931 ] Robert Stupp commented on CASSANDRA-10918: -- +1 > remove leftover code from refactor > -- > > Key: CASSANDRA-10918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10918 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths >Reporter: Dave Brosius >Assignee: Dave Brosius >Priority: Trivial > Fix For: 3.x > > Attachments: 10918.txt > > > code seems to have been left over from refactor from 2.2 to 3.0. removed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10918) remove leftover code from refactor
[ https://issues.apache.org/jira/browse/CASSANDRA-10918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-10918: - Reviewer: Robert Stupp > remove leftover code from refactor > -- > > Key: CASSANDRA-10918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10918 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths >Reporter: Dave Brosius >Assignee: Dave Brosius >Priority: Trivial > Fix For: 3.x > > Attachments: 10918.txt > > > code seems to have been left over from refactor from 2.2 to 3.0. removed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9954) Improve Java-UDF timeout detection
[ https://issues.apache.org/jira/browse/CASSANDRA-9954?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069868#comment-15069868 ] Robert Stupp commented on CASSANDRA-9954: - Updated the branch with the following changes: * rename "timeout" to "cpu time" to make clear that the values relate to CPU time and not wall clock * let Java UDFs run in the calling thread and updated security-manager accordingly * byte code inspection now checks class names against general UDF black/white lists * refactorings to (hopefully) simplify code I *think* the changes to Java UDFs are fine WRT class loading and security manager. Still unsure about timeouts related wall-clock and (remaining) request-time. In order to not issue false-positive warnings and errors caused by "badly timed GCs" or an overloaded system, the timeouts need to be quite high. Triggered cassci runs, too. PS: while fixing the branches for another ticket, I ... up the branch for this ticket (both local and github) and the only backup was in IntelliJ's history. Sorry, that the branch now appears squashed. > Improve Java-UDF timeout detection > -- > > Key: CASSANDRA-9954 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9954 > Project: Cassandra > Issue Type: Improvement >Reporter: Robert Stupp >Assignee: Robert Stupp > Fix For: 3.x > > > CASSANDRA-9402 introduced a sandbox using a thread-pool to enforce security > constraints and to detect "amok UDFs" - i.e. UDFs that essentially never > return (e.g. {{while (true)}}. > Currently the safest way to react on such an "amok UDF" is to _fail-fast_ - > to stop the C* daemon since stopping a thread (in Java) is just no solution. > CASSANDRA-9890 introduced further protection by inspecting the byte-code. The > same mechanism can also be used to manipulate the Java-UDF byte-code. > By manipulating the byte-code I mean to add regular "is-amok-UDF" checks in > the compiled code. > EDIT: These "is-amok-UDF" checks would also work for _UNFENCED_ Java-UDFs. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[01/10] cassandra git commit: Fix multiple consecutive delimiters on cqlsh COPY FROM
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 7c3966bfe -> 3ccffc94b refs/heads/cassandra-2.2 ae8baa499 -> ee36f14c1 refs/heads/cassandra-3.0 8bc567b3c -> 382d9a943 refs/heads/trunk 3c8d87f43 -> ce6ad5bf8 Fix multiple consecutive delimiters on cqlsh COPY FROM Patch by stefania; reviewed by pmotta for CASSANDRA-10854 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3ccffc94 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3ccffc94 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3ccffc94 Branch: refs/heads/cassandra-2.1 Commit: 3ccffc94b8a6127f6a452a3fb93306ad3bbc8c97 Parents: 7c3966b Author: Stefania AlborghettiAuthored: Tue Dec 22 09:16:54 2015 +0100 Committer: Joshua McKenzie Committed: Wed Dec 23 13:20:27 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3ccffc94/pylib/cqlshlib/copyutil.py -- diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py index f699e64..65f5997 100644 --- a/pylib/cqlshlib/copyutil.py +++ b/pylib/cqlshlib/copyutil.py @@ -1020,16 +1020,19 @@ class ImportConversion(object): ret[i] = self.converters[self.columns[i]](val) else: if i in self.primary_key_indexes: -message = "Cannot insert null value for primary key column '%s'." % (self.columns[i],) -if self.nullval == '': -message += " If you want to insert empty strings, consider using" \ - " the WITH NULL= option for COPY." -raise Exception(message=message) +raise ValueError(self.get_null_primary_key_message(i)) ret[i] = None return ret +def get_null_primary_key_message(self, idx): +message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],) +if self.nullval == '': +message += " If you want to insert empty strings, consider using" \ + " the WITH NULL= option for COPY." +return message + def get_row_partition_key_values(self, row): """ Return a string composed of the partition key values, serialized and binary packed - @@ -1037,6 +1040,8 @@ class ImportConversion(object): """ def serialize(n): c, v = self.columns[n], row[n] +if v == self.nullval: +raise ValueError(self.get_null_primary_key_message(n)) return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version) partition_key_indexes = self.partition_key_indexes
[06/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ee36f14c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ee36f14c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ee36f14c Branch: refs/heads/cassandra-2.2 Commit: ee36f14c1427db8e3b91b7eda6c0e6f48ecb4f24 Parents: ae8baa4 3ccffc9 Author: Joshua McKenzieAuthored: Wed Dec 23 13:21:12 2015 -0500 Committer: Joshua McKenzie Committed: Wed Dec 23 13:21:12 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee36f14c/pylib/cqlshlib/copyutil.py --
[08/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/382d9a94 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/382d9a94 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/382d9a94 Branch: refs/heads/cassandra-3.0 Commit: 382d9a943bdf33e528fa6509e3bd12c2bc9ecc70 Parents: 8bc567b ee36f14 Author: Joshua McKenzieAuthored: Wed Dec 23 13:21:19 2015 -0500 Committer: Joshua McKenzie Committed: Wed Dec 23 13:21:19 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) --
[03/10] cassandra git commit: Fix multiple consecutive delimiters on cqlsh COPY FROM
Fix multiple consecutive delimiters on cqlsh COPY FROM Patch by stefania; reviewed by pmotta for CASSANDRA-10854 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3ccffc94 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3ccffc94 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3ccffc94 Branch: refs/heads/cassandra-3.0 Commit: 3ccffc94b8a6127f6a452a3fb93306ad3bbc8c97 Parents: 7c3966b Author: Stefania AlborghettiAuthored: Tue Dec 22 09:16:54 2015 +0100 Committer: Joshua McKenzie Committed: Wed Dec 23 13:20:27 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3ccffc94/pylib/cqlshlib/copyutil.py -- diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py index f699e64..65f5997 100644 --- a/pylib/cqlshlib/copyutil.py +++ b/pylib/cqlshlib/copyutil.py @@ -1020,16 +1020,19 @@ class ImportConversion(object): ret[i] = self.converters[self.columns[i]](val) else: if i in self.primary_key_indexes: -message = "Cannot insert null value for primary key column '%s'." % (self.columns[i],) -if self.nullval == '': -message += " If you want to insert empty strings, consider using" \ - " the WITH NULL= option for COPY." -raise Exception(message=message) +raise ValueError(self.get_null_primary_key_message(i)) ret[i] = None return ret +def get_null_primary_key_message(self, idx): +message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],) +if self.nullval == '': +message += " If you want to insert empty strings, consider using" \ + " the WITH NULL= option for COPY." +return message + def get_row_partition_key_values(self, row): """ Return a string composed of the partition key values, serialized and binary packed - @@ -1037,6 +1040,8 @@ class ImportConversion(object): """ def serialize(n): c, v = self.columns[n], row[n] +if v == self.nullval: +raise ValueError(self.get_null_primary_key_message(n)) return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version) partition_key_indexes = self.partition_key_indexes
[10/10] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ce6ad5bf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ce6ad5bf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ce6ad5bf Branch: refs/heads/trunk Commit: ce6ad5bf835aab747a1dfc6165e7033a3ea04e6e Parents: 3c8d87f 382d9a9 Author: Joshua McKenzieAuthored: Wed Dec 23 13:21:27 2015 -0500 Committer: Joshua McKenzie Committed: Wed Dec 23 13:21:27 2015 -0500 -- pylib/cqlshlib/copyutil.py | 15 ++- 1 file changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce6ad5bf/pylib/cqlshlib/copyutil.py --
[jira] [Commented] (CASSANDRA-8113) Gossip should ignore generation numbers too far in the future
[ https://issues.apache.org/jira/browse/CASSANDRA-8113?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069845#comment-15069845 ] T. David Hudson commented on CASSANDRA-8113: I'm seeing 3 nodes of a 4-node Cassandra 2.1.1 (upgraded from 2.Xs a while back) cluster reporting ancient generations and refusing to accept a modern generation from the fourth. Could the generation check allow a timestamp far past the local generation but nevertheless reasonable w.r.t. the current system clock? > Gossip should ignore generation numbers too far in the future > - > > Key: CASSANDRA-8113 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8113 > Project: Cassandra > Issue Type: Improvement >Reporter: Richard Low >Assignee: Jason Brown > Fix For: 2.1.1 > > Attachments: 8113-v1.txt, 8113-v2.txt, 8113-v3.txt, 8113-v4.txt, > 8133-fix.txt > > > If a node sends corrupted gossip, it could set the generation numbers for > other nodes to arbitrarily large values. This is dangerous since one bad node > (e.g. with bad memory) could in theory bring down the cluster. Nodes should > refuse to accept generation numbers that are too far in the future. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10930) Limit off-heap memory
Paulo Motta created CASSANDRA-10930: --- Summary: Limit off-heap memory Key: CASSANDRA-10930 URL: https://issues.apache.org/jira/browse/CASSANDRA-10930 Project: Cassandra Issue Type: Improvement Components: Configuration Reporter: Paulo Motta Priority: Minor Fix For: 3.x It seems the default maximum off-heap memory varies according to the JVM version. For oracle hotspot, this defaults to the heap size, according to [this|https://terracotta.org/generated/4.3.1/html/bmg-all/index.html#page/bigmemory-go-webhelp/co-tiers_configuring_offheap_store.html] and [this|http://hg.openjdk.java.net/jdk7u/jdk7u/jdk/file/55f6804b4352/src/share/classes/sun/misc/VM.java#l279]. This may cause oomkiller to kick in if max_mem <= 2*heap. Opening this ticket to discuss potentially limiting off-heap usage by default on {{cassandra-env.sh}} with the {{-XX:MaxDirectMemorySize}}, but I wouldn't know to which value. Or at least checking if max_mem > 2*heap. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070063#comment-15070063 ] Ariel Weisberg commented on CASSANDRA-8844: --- Is that actually equivalent? MV is materializing an end state. CDC is a change log with a record of the intermediate states and all those state changes need to be replicated. HT to Brian Hess. > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be built into the > daemons. > h2. Format and Consumption > - Cassandra would only write to the CDC log, and never delete from it. > - Cleaning up consumed logfiles would be the client daemon's responibility > - Logfile size should probably be configurable. > - Logfiles should be named with a predictable naming schema, making it > triivial to process them in order. > - Daemons should be able to checkpoint their work, and resume from where they > left off. This means they would have to leave some file artifact in the CDC > log's directory. > - A sophisticated daemon should be able to be written that could > -- Catch up, in written-order, even when it is multiple logfiles behind in > processing > -- Be able to continuously "tail" the most recent logfile and get > low-latency(ms?) access to the data as it is written. > h2. Alternate approach > In order to make consuming a change log easy and efficient to do with low >
[jira] [Created] (CASSANDRA-10933) index_summary_upgrade_test.py failing
Philip Thompson created CASSANDRA-10933: --- Summary: index_summary_upgrade_test.py failing Key: CASSANDRA-10933 URL: https://issues.apache.org/jira/browse/CASSANDRA-10933 Project: Cassandra Issue Type: Sub-task Reporter: Philip Thompson Fix For: 3.0.x, 3.x Attachments: node1.log index_summary_upgrade_test.py is failing on HEAD for cassandra-3.0. The test upgrades from 2.1.3 to 3.0, and checks the following behavior {code} # on startup, it should detect that the old-format sstable had its # index summary downsampled (forcing it to be rebuilt) node.watch_log_for("Detected erroneously downsampled index summary") {code} However, this is not happening. The node's log is attached. It is quite possible that the "fix" here is that we only want to test upgrading from 2.1.3 to 2.1.LATEST, but that's not for me to decide. I can easily implement that, if that happens to be the case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Add latency logging for dropped messages
Repository: cassandra Updated Branches: refs/heads/trunk 04a99ab84 -> 3c8d87f43 Add latency logging for dropped messages Patch by akale; reviewed by pmotta for CASSANDRA-10580 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3c8d87f4 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3c8d87f4 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3c8d87f4 Branch: refs/heads/trunk Commit: 3c8d87f4324e5ff8bf6b1c3652e9c5eacf03bc20 Parents: 04a99ab Author: anubhavkaleAuthored: Thu Dec 10 12:28:45 2015 -0800 Committer: Joshua McKenzie Committed: Wed Dec 23 13:15:10 2015 -0500 -- .../cassandra/net/MessageDeliveryTask.java | 42 +-- .../apache/cassandra/service/StorageProxy.java | 44 ++-- 2 files changed, 79 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3c8d87f4/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 818cfc6..bede3d8 100644 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@ -18,11 +18,13 @@ package org.apache.cassandra.net; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.EnumSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import org.apache.cassandra.db.IMutation; import org.apache.cassandra.db.filter.TombstoneOverwhelmingException; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.index.IndexNotAvailableException; @@ -43,10 +45,11 @@ public class MessageDeliveryTask implements Runnable public void run() { +long timeTaken = System.currentTimeMillis() - message.constructionTime.timestamp; MessagingService.Verb verb = message.verb; -if (MessagingService.DROPPABLE_VERBS.contains(verb) -&& System.currentTimeMillis() > message.constructionTime.timestamp + message.getTimeout()) +if (MessagingService.DROPPABLE_VERBS.contains(verb)&& message.getTimeout() > timeTaken) { +LogDroppedMessageDetails(timeTaken); MessagingService.instance().incrementDroppedMessages(message); return; } @@ -82,6 +85,37 @@ public class MessageDeliveryTask implements Runnable Gossiper.instance.setLastProcessedMessageAt(message.constructionTime.timestamp); } +private void LogDroppedMessageDetails(long timeTaken) +{ +logger.debug("MessageDeliveryTask ran after {} ms, allowed time was {} ms. Dropping message {}", +timeTaken, message.getTimeout(), message.toString()); +// Print KS and CF if Payload is mutation or a list of mutations (sent due to schema announcements) +IMutation mutation; +if (message.payload instanceof IMutation) +{ +mutation = (IMutation)message.payload; +if (mutation != null) +{ +logger.debug("MessageDeliveryTask dropped mutation of KS {}, CF {}", mutation.getKeyspaceName(), Arrays.toString(mutation.getColumnFamilyIds().toArray())); +} +} +else if (message.payload instanceof Collection) +{ +Collection payloadItems = (Collection)message.payload; +for (Object payloadItem : payloadItems) +{ +if (payloadItem instanceof IMutation) +{ +mutation = (IMutation)payloadItem; +if (mutation != null) +{ +logger.debug("MessageDeliveryTask dropped mutation of KS {}, CF {}", mutation.getKeyspaceName(), Arrays.toString(mutation.getColumnFamilyIds().toArray())); +} +} +} +} +} + private void handleFailure(Throwable t) { if (message.doCallbackOnFailure()) @@ -95,4 +129,4 @@ public class MessageDeliveryTask implements Runnable private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, MessagingService.Verb.GOSSIP_DIGEST_ACK2, MessagingService.Verb.GOSSIP_DIGEST_SYN); -} +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/cassandra/blob/3c8d87f4/src/java/org/apache/cassandra/service/StorageProxy.java
[jira] [Commented] (CASSANDRA-10580) Add latency metrics for dropped messages
[ https://issues.apache.org/jira/browse/CASSANDRA-10580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069881#comment-15069881 ] Paulo Motta commented on CASSANDRA-10580: - New features are only going on 3.2, so only trunk patch is needed. Tests look good. Since there are no objections, I will mark this as ready to commit. > Add latency metrics for dropped messages > > > Key: CASSANDRA-10580 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10580 > Project: Cassandra > Issue Type: Improvement > Components: Coordination, Observability > Environment: Production >Reporter: Anubhav Kale >Assignee: Anubhav Kale >Priority: Minor > Fix For: 3.2 > > Attachments: 0001-Metrics.patch, 10580-Metrics.patch, 10580.patch, > 2.2-All-Comments.patch, CASSANDRA-10580-Head.patch, Trunk-All-Comments.patch, > Trunk.patch > > > In our production cluster, we are seeing a large number of dropped mutations. > At a minimum, we should print the time the thread took to get scheduled > thereby dropping the mutation (We should also print the Message / Mutation so > it helps in figuring out which column family was affected). This will help > find the right tuning parameter for write_timeout_in_ms. > The change is small and is in StorageProxy.java and MessagingTask.java. I > will submit a patch shortly. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10686) cqlsh schema refresh on timeout dtest is flaky
[ https://issues.apache.org/jira/browse/CASSANDRA-10686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069895#comment-15069895 ] Paulo Motta commented on CASSANDRA-10686: - Tests fixed. > cqlsh schema refresh on timeout dtest is flaky > --- > > Key: CASSANDRA-10686 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10686 > Project: Cassandra > Issue Type: Bug > Components: Testing, Tools >Reporter: Joel Knighton >Assignee: Paulo Motta >Priority: Minor > > [flaky 3.0 > runs|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/lastCompletedBuild/testReport/cqlsh_tests.cqlsh_tests/TestCqlsh/test_refresh_schema_on_timeout_error/history/] > [flaky 2.2 > runs|http://cassci.datastax.com/job/cassandra-2.2_dtest/381/testReport/cqlsh_tests.cqlsh_tests/TestCqlsh/test_refresh_schema_on_timeout_error/history/] > [flaky 2.1 > runs|http://cassci.datastax.com/job/cassandra-2.1_dtest/324/testReport/cqlsh_tests.cqlsh_tests/TestCqlsh/test_refresh_schema_on_timeout_error/history/] > As far as I can tell, the issue could be with the test or the original issue. > Pinging [~pauloricardomg] since he knows this best. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10866) Column Family should expose count metrics for dropped mutations.
[ https://issues.apache.org/jira/browse/CASSANDRA-10866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069936#comment-15069936 ] Anubhav Kale commented on CASSANDRA-10866: -- Attached 10866-Trunk.patch. > Column Family should expose count metrics for dropped mutations. > > > Key: CASSANDRA-10866 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10866 > Project: Cassandra > Issue Type: Improvement > Environment: PROD >Reporter: Anubhav Kale >Assignee: Anubhav Kale >Priority: Minor > Attachments: 0001-CFCount.patch, 10866-Trunk.patch > > > Please take a look at the discussion in CASSANDRA-10580. This is opened so > that the latency on dropped mutations is exposed as a metric on column > families. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10866) Column Family should expose count metrics for dropped mutations.
[ https://issues.apache.org/jira/browse/CASSANDRA-10866?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anubhav Kale updated CASSANDRA-10866: - Attachment: 10866-Trunk.patch > Column Family should expose count metrics for dropped mutations. > > > Key: CASSANDRA-10866 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10866 > Project: Cassandra > Issue Type: Improvement > Environment: PROD >Reporter: Anubhav Kale >Assignee: Anubhav Kale >Priority: Minor > Attachments: 0001-CFCount.patch, 10866-Trunk.patch > > > Please take a look at the discussion in CASSANDRA-10580. This is opened so > that the latency on dropped mutations is exposed as a metric on column > families. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10917) better validator randomness
[ https://issues.apache.org/jira/browse/CASSANDRA-10917?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069966#comment-15069966 ] Robert Stupp commented on CASSANDRA-10917: -- +1 on the change. One nit: {{keys.clear()}} could be moved up into the else branch > better validator randomness > --- > > Key: CASSANDRA-10917 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10917 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths >Reporter: Dave Brosius >Assignee: Dave Brosius >Priority: Trivial > Fix For: 3.x > > Attachments: 10917.txt > > > get better randomness by reusing a Random object rather than recreating it. > Also reuse keys list to avoid reallocations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10917) better validator randomness
[ https://issues.apache.org/jira/browse/CASSANDRA-10917?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-10917: - Reviewer: Robert Stupp > better validator randomness > --- > > Key: CASSANDRA-10917 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10917 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths >Reporter: Dave Brosius >Assignee: Dave Brosius >Priority: Trivial > Fix For: 3.x > > Attachments: 10917.txt > > > get better randomness by reusing a Random object rather than recreating it. > Also reuse keys list to avoid reallocations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10935) sstableloader_uppercase_keyspace_name_test is failing on 2.2
Philip Thompson created CASSANDRA-10935: --- Summary: sstableloader_uppercase_keyspace_name_test is failing on 2.2 Key: CASSANDRA-10935 URL: https://issues.apache.org/jira/browse/CASSANDRA-10935 Project: Cassandra Issue Type: Sub-task Components: Tools Reporter: Philip Thompson Fix For: 2.2.x {{sstable_generation_loading_test.py:TestSSTableGenerationAndLoading.sstableloader_uppercase_keyspace_name_test}} is failing on HEAD on cassandra-2.2. It passes on 2.1 and 3.0. The error looks like: {code} == ERROR: sstableloader_uppercase_keyspace_name_test (sstable_generation_loading_test.TestSSTableGenerationAndLoading) -- Traceback (most recent call last): File "/Users/philipthompson/cstar/cassandra-dtest/sstable_generation_loading_test.py", line 122, in sstableloader_uppercase_keyspace_name_test self.load_sstable_with_configuration(ks='"Keyspace1"') File "/Users/philipthompson/cstar/cassandra-dtest/sstable_generation_loading_test.py", line 210, in load_sstable_with_configuration read_and_validate_data(session) File "/Users/philipthompson/cstar/cassandra-dtest/sstable_generation_loading_test.py", line 203, in read_and_validate_data self.assertEquals([str(i), 'col', str(i)], list(rows[0])) IndexError: list index out of range >> begin captured logging << dtest: DEBUG: cluster ccm directory: /var/folders/v3/z4wf_34n1q506_xjdy49gb78gn/T/dtest-PY1RTe dtest: DEBUG: Testing sstableloader with pre_compression=None and post_compression=None dtest: DEBUG: creating keyspace and inserting dtest: DEBUG: Making a copy of the sstables dtest: DEBUG: Wiping out the data and restarting cluster cassandra.cluster: WARNING: Host 127.0.0.2 has been marked down cassandra.cluster: WARNING: Host 127.0.0.1 has been marked down cassandra.cluster: WARNING: Failed to create connection pool for new host 127.0.0.1: Traceback (most recent call last): File "build/bdist.macosx-10.11-x86_64/egg/cassandra/cluster.py", line 1787, in run_add_or_renew_pool new_pool = HostConnection(host, distance, self) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/pool.py", line 297, in __init__ self._connection.set_keyspace_blocking(session.keyspace) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/connection.py", line 773, in set_keyspace_blocking raise conn_exc ConnectionException: Problem while setting keyspace: InvalidRequest(u'code=2200 [Invalid query] message="Keyspace \'Keyspace1\' does not exist"',) cassandra.cluster: WARNING: Failed to create connection pool for new host 127.0.0.2: Traceback (most recent call last): File "build/bdist.macosx-10.11-x86_64/egg/cassandra/cluster.py", line 1787, in run_add_or_renew_pool new_pool = HostConnection(host, distance, self) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/pool.py", line 297, in __init__ self._connection.set_keyspace_blocking(session.keyspace) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/connection.py", line 773, in set_keyspace_blocking raise conn_exc ConnectionException: Problem while setting keyspace: InvalidRequest(u'code=2200 [Invalid query] message="Keyspace \'Keyspace1\' does not exist"',) cassandra.cluster: WARNING: Failed to create connection pool for new host 127.0.0.2: Traceback (most recent call last): File "build/bdist.macosx-10.11-x86_64/egg/cassandra/cluster.py", line 1787, in run_add_or_renew_pool new_pool = HostConnection(host, distance, self) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/pool.py", line 297, in __init__ self._connection.set_keyspace_blocking(session.keyspace) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/connection.py", line 773, in set_keyspace_blocking raise conn_exc ConnectionException: Problem while setting keyspace: InvalidRequest(u'code=2200 [Invalid query] message="Keyspace \'Keyspace1\' does not exist"',) cassandra.cluster: WARNING: Failed to create connection pool for new host 127.0.0.1: Traceback (most recent call last): File "build/bdist.macosx-10.11-x86_64/egg/cassandra/cluster.py", line 1787, in run_add_or_renew_pool new_pool = HostConnection(host, distance, self) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/pool.py", line 297, in __init__ self._connection.set_keyspace_blocking(session.keyspace) File "build/bdist.macosx-10.11-x86_64/egg/cassandra/connection.py", line 773, in set_keyspace_blocking raise conn_exc ConnectionException: Problem while setting keyspace: InvalidRequest(u'code=2200 [Invalid query] message="Keyspace \'Keyspace1\' does not exist"',) cassandra.cluster: WARNING: Failed to create connection pool for new host 127.0.0.2: Traceback (most recent call last):
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070075#comment-15070075 ] Ariel Weisberg commented on CASSANDRA-8844: --- How do the mutations get generated? There is no state in the base table reflecting that the mutations need to propagate. So when repair runs and generates a merkle tree is that going to include the CDC log containing the mutations that are a side effect of state transitions no longer visible in the base table? > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be built into the > daemons. > h2. Format and Consumption > - Cassandra would only write to the CDC log, and never delete from it. > - Cleaning up consumed logfiles would be the client daemon's responibility > - Logfile size should probably be configurable. > - Logfiles should be named with a predictable naming schema, making it > triivial to process them in order. > - Daemons should be able to checkpoint their work, and resume from where they > left off. This means they would have to leave some file artifact in the CDC > log's directory. > - A sophisticated daemon should be able to be written that could > -- Catch up, in written-order, even when it is multiple logfiles behind in > processing > -- Be able to continuously "tail" the most recent logfile and get > low-latency(ms?) access to the data
[2/2] cassandra git commit: remove leftover code from refactor patch by dbrosius, reviewed by rstupp for CASSANDRA-10918
remove leftover code from refactor patch by dbrosius, reviewed by rstupp for CASSANDRA-10918 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8bafc180 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8bafc180 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8bafc180 Branch: refs/heads/cassandra-3.0 Commit: 8bafc180b62f9bb60ac50a1d87e510bfa9952fcf Parents: 867fd8e Author: Dave BrosiusAuthored: Wed Dec 23 19:06:34 2015 -0500 Committer: Dave Brosius Committed: Wed Dec 23 19:06:34 2015 -0500 -- .../cassandra/db/commitlog/CommitLogSegmentManager.java | 7 --- 1 file changed, 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bafc180/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java -- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java index a514028..564652f 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java @@ -293,13 +293,6 @@ public class CommitLogSegmentManager // make sure the writes have materialized inside of the memtables by waiting for all outstanding writes // on the relevant keyspaces to complete -Set keyspaces = new HashSet<>(); -for (UUID cfId : last.getDirtyCFIDs()) -{ -ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(cfId); -if (cfs != null) -keyspaces.add(cfs.keyspace); -} Keyspace.writeOrder.awaitNewBarrier(); // flush and wait for all CFs that are dirty in segments up-to and including 'last'
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070494#comment-15070494 ] Brian Hess commented on CASSANDRA-8844: This makes the consistency level of the CDC log completely defined by the CL of the mutation as it is written. That impacts anyone using CDC all the way to the application. The CDC log should transparently be additive to the table. That is, the user should not have to write to the table in any special or prescribed way in order to get the mutations captured. Requiring the client to write at CL_QUORUM of CDC is enabled is not okay. The thing that concerns me is that the consistency/replication of the CDC log now has a completely different guarantee than the base table (system, actually). The fact that a user can use Cassandra with any CL on writes and the system will eventually get to a consistent state is a good thing. The CDC log should be similarly eventually consistent, via any CL for the mutations. The situation described above is a situation that Cassandra covers just fine and results in an eventually consistent state. However, it will result in CDC loss which is not acceptable. The goals of CDC is not to get to snapshot differences, but to capture every mutation (including the intermediate mutations). The solution should tolerate nodes being offline and not lose any mutations, without requiring the application producing the mutations to be modified. > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters,
[jira] [Updated] (CASSANDRA-10876) Alter behavior of batch WARN and fail on single partition batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10876?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Taiyuan Zhang updated CASSANDRA-10876: -- Attachment: 10876.txt > Alter behavior of batch WARN and fail on single partition batches > - > > Key: CASSANDRA-10876 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10876 > Project: Cassandra > Issue Type: Improvement >Reporter: Patrick McFadin >Priority: Minor > Attachments: 10876.txt > > > In an attempt to give operator insight into potentially harmful batch usage, > Jiras were created to log WARN or fail on certain batch sizes. This ignores > the single partition batch, which doesn't create the same issues as a > multi-partition batch. > The proposal is to ignore size on single partition batch statements. > Reference: > [CASSANDRA-6487|https://issues.apache.org/jira/browse/CASSANDRA-6487] > [CASSANDRA-8011|https://issues.apache.org/jira/browse/CASSANDRA-8011] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/3] cassandra git commit: remove leftover code from refactor patch by dbrosius, reviewed by rstupp for CASSANDRA-10918
remove leftover code from refactor patch by dbrosius, reviewed by rstupp for CASSANDRA-10918 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8bafc180 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8bafc180 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8bafc180 Branch: refs/heads/trunk Commit: 8bafc180b62f9bb60ac50a1d87e510bfa9952fcf Parents: 867fd8e Author: Dave BrosiusAuthored: Wed Dec 23 19:06:34 2015 -0500 Committer: Dave Brosius Committed: Wed Dec 23 19:06:34 2015 -0500 -- .../cassandra/db/commitlog/CommitLogSegmentManager.java | 7 --- 1 file changed, 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8bafc180/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java -- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java index a514028..564652f 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java @@ -293,13 +293,6 @@ public class CommitLogSegmentManager // make sure the writes have materialized inside of the memtables by waiting for all outstanding writes // on the relevant keyspaces to complete -Set keyspaces = new HashSet<>(); -for (UUID cfId : last.getDirtyCFIDs()) -{ -ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(cfId); -if (cfs != null) -keyspaces.add(cfs.keyspace); -} Keyspace.writeOrder.awaitNewBarrier(); // flush and wait for all CFs that are dirty in segments up-to and including 'last'
[1/2] cassandra git commit: better validator randomness patch by dbrosius, reviewed by rstupp for CASSANDRA-10917
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 382d9a943 -> 8bafc180b better validator randomness patch by dbrosius, reviewed by rstupp for CASSANDRA-10917 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/867fd8e2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/867fd8e2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/867fd8e2 Branch: refs/heads/cassandra-3.0 Commit: 867fd8e2e9c33e8f2ee1d13c3b7dd177434ec645 Parents: 382d9a9 Author: Dave BrosiusAuthored: Wed Dec 23 19:04:32 2015 -0500 Committer: Dave Brosius Committed: Wed Dec 23 19:04:32 2015 -0500 -- src/java/org/apache/cassandra/repair/Validator.java | 6 -- 1 file changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/867fd8e2/src/java/org/apache/cassandra/repair/Validator.java -- diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java index d206305..3db9761 100644 --- a/src/java/org/apache/cassandra/repair/Validator.java +++ b/src/java/org/apache/cassandra/repair/Validator.java @@ -90,9 +90,11 @@ public class Validator implements Runnable } else { +List keys = new ArrayList<>(); +Random random = new Random(); + for (Range range : tree.ranges()) { -List keys = new ArrayList<>(); for (DecoratedKey sample : cfs.keySamples(range)) { assert range.contains(sample.getToken()) : "Token " + sample.getToken() + " is not within range " + desc.ranges; @@ -107,7 +109,6 @@ public class Validator implements Runnable else { int numKeys = keys.size(); -Random random = new Random(); // sample the column family using random keys from the index while (true) { @@ -115,6 +116,7 @@ public class Validator implements Runnable if (!tree.split(dk.getToken())) break; } +keys.clear(); } } }
[1/3] cassandra git commit: better validator randomness patch by dbrosius, reviewed by rstupp for CASSANDRA-10917
Repository: cassandra Updated Branches: refs/heads/trunk ce6ad5bf8 -> 02c92dfce better validator randomness patch by dbrosius, reviewed by rstupp for CASSANDRA-10917 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/867fd8e2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/867fd8e2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/867fd8e2 Branch: refs/heads/trunk Commit: 867fd8e2e9c33e8f2ee1d13c3b7dd177434ec645 Parents: 382d9a9 Author: Dave BrosiusAuthored: Wed Dec 23 19:04:32 2015 -0500 Committer: Dave Brosius Committed: Wed Dec 23 19:04:32 2015 -0500 -- src/java/org/apache/cassandra/repair/Validator.java | 6 -- 1 file changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/867fd8e2/src/java/org/apache/cassandra/repair/Validator.java -- diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java index d206305..3db9761 100644 --- a/src/java/org/apache/cassandra/repair/Validator.java +++ b/src/java/org/apache/cassandra/repair/Validator.java @@ -90,9 +90,11 @@ public class Validator implements Runnable } else { +List keys = new ArrayList<>(); +Random random = new Random(); + for (Range range : tree.ranges()) { -List keys = new ArrayList<>(); for (DecoratedKey sample : cfs.keySamples(range)) { assert range.contains(sample.getToken()) : "Token " + sample.getToken() + " is not within range " + desc.ranges; @@ -107,7 +109,6 @@ public class Validator implements Runnable else { int numKeys = keys.size(); -Random random = new Random(); // sample the column family using random keys from the index while (true) { @@ -115,6 +116,7 @@ public class Validator implements Runnable if (!tree.split(dk.getToken())) break; } +keys.clear(); } } }
[3/3] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/02c92dfc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/02c92dfc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/02c92dfc Branch: refs/heads/trunk Commit: 02c92dfce322086532462d9a13454f3ba52b1954 Parents: ce6ad5b 8bafc18 Author: Dave BrosiusAuthored: Wed Dec 23 19:11:53 2015 -0500 Committer: Dave Brosius Committed: Wed Dec 23 19:11:53 2015 -0500 -- .../cassandra/db/commitlog/CommitLogSegmentManager.java | 7 --- src/java/org/apache/cassandra/repair/Validator.java | 6 -- 2 files changed, 4 insertions(+), 9 deletions(-) --
[jira] [Updated] (CASSANDRA-10875) cqlsh fails to decode utf-8 characters for text typed columns.
[ https://issues.apache.org/jira/browse/CASSANDRA-10875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yasuharu Goto updated CASSANDRA-10875: -- Attachment: 10875-2.1-2.txt > cqlsh fails to decode utf-8 characters for text typed columns. > -- > > Key: CASSANDRA-10875 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10875 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Yasuharu Goto >Assignee: Yasuharu Goto >Priority: Minor > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > Attachments: 10875-2.1-2.txt, 10875-2.1.12.txt, 10875-2.2.txt, > 10875-3.1.txt > > > Hi, we've found a bug that cqlsh can't handle unicode text in select > conditions even if it were text type. > {noformat} > $ ./bin/cqlsh > Connected to Test Cluster at 127.0.0.1:9042. > [cqlsh 5.0.1 | Cassandra 3.2-SNAPSHOT | CQL spec 3.3.1 | Native protocol v4] > Use HELP for help. > cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', > 'replication_factor': 1}; > cqlsh> create table test.test(txt text primary key); > cqlsh> insert into test.test (txt) values('日本語'); > cqlsh> select * from test.test where txt='日本語'; > 'ascii' codec can't decode byte 0xe6 in position 35: ordinal not in range(128) > cqlsh> > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10875) cqlsh fails to decode utf-8 characters for text typed columns.
[ https://issues.apache.org/jira/browse/CASSANDRA-10875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yasuharu Goto updated CASSANDRA-10875: -- Attachment: 10875-2.2.txt > cqlsh fails to decode utf-8 characters for text typed columns. > -- > > Key: CASSANDRA-10875 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10875 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Yasuharu Goto >Assignee: Yasuharu Goto >Priority: Minor > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > Attachments: 10875-2.1-2.txt, 10875-2.1.12.txt, 10875-2.2.txt, > 10875-3.1.txt > > > Hi, we've found a bug that cqlsh can't handle unicode text in select > conditions even if it were text type. > {noformat} > $ ./bin/cqlsh > Connected to Test Cluster at 127.0.0.1:9042. > [cqlsh 5.0.1 | Cassandra 3.2-SNAPSHOT | CQL spec 3.3.1 | Native protocol v4] > Use HELP for help. > cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', > 'replication_factor': 1}; > cqlsh> create table test.test(txt text primary key); > cqlsh> insert into test.test (txt) values('日本語'); > cqlsh> select * from test.test where txt='日本語'; > 'ascii' codec can't decode byte 0xe6 in position 35: ordinal not in range(128) > cqlsh> > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10875) cqlsh fails to decode utf-8 characters for text typed columns.
[ https://issues.apache.org/jira/browse/CASSANDRA-10875?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070499#comment-15070499 ] Yasuharu Goto commented on CASSANDRA-10875: --- [~pauloricardomg] Thank you for your great review! (And sorry to be late) I've updated my patches to 10875-2.1-2.txt and 10875-2.2.txt. I could merge 10875-2.2 to 2.2,3.0, and trunk. > cqlsh fails to decode utf-8 characters for text typed columns. > -- > > Key: CASSANDRA-10875 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10875 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Yasuharu Goto >Assignee: Yasuharu Goto >Priority: Minor > Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x > > Attachments: 10875-2.1-2.txt, 10875-2.1.12.txt, 10875-2.2.txt, > 10875-3.1.txt > > > Hi, we've found a bug that cqlsh can't handle unicode text in select > conditions even if it were text type. > {noformat} > $ ./bin/cqlsh > Connected to Test Cluster at 127.0.0.1:9042. > [cqlsh 5.0.1 | Cassandra 3.2-SNAPSHOT | CQL spec 3.3.1 | Native protocol v4] > Use HELP for help. > cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', > 'replication_factor': 1}; > cqlsh> create table test.test(txt text primary key); > cqlsh> insert into test.test (txt) values('日本語'); > cqlsh> select * from test.test where txt='日本語'; > 'ascii' codec can't decode byte 0xe6 in position 35: ordinal not in range(128) > cqlsh> > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10661) Integrate SASI to Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-10661?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070176#comment-15070176 ] Pavel Yaskevich edited comment on CASSANDRA-10661 at 12/24/15 12:00 AM: [~beobal] Yes, I will remove OR/parenthesis features from the branch (since we have a separate repo we can port it back from), do some more cleanup, and mark this as Patch Available. I forgot to mention that there is no clustering columns support yet in the branch (only Regular), but we are working on it. was (Author: xedin): [~beobal] Yes, I will remove OR/parenthesis features from the branch (since we have a separate repo we can port it back from), do some more cleanup, and mark this as Patch Available. > 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] [Commented] (CASSANDRA-10876) Alter behavior of batch WARN and fail on single partition batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070578#comment-15070578 ] Taiyuan Zhang commented on CASSANDRA-10876: --- Just attached the file. Sorry. > Alter behavior of batch WARN and fail on single partition batches > - > > Key: CASSANDRA-10876 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10876 > Project: Cassandra > Issue Type: Improvement >Reporter: Patrick McFadin >Priority: Minor > Attachments: 10876.txt > > > In an attempt to give operator insight into potentially harmful batch usage, > Jiras were created to log WARN or fail on certain batch sizes. This ignores > the single partition batch, which doesn't create the same issues as a > multi-partition batch. > The proposal is to ignore size on single partition batch statements. > Reference: > [CASSANDRA-6487|https://issues.apache.org/jira/browse/CASSANDRA-6487] > [CASSANDRA-8011|https://issues.apache.org/jira/browse/CASSANDRA-8011] -- 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=15070095#comment-15070095 ] Anubhav Kale commented on CASSANDRA-10907: -- For point in time backups, its always somewhat unpredictable what data is backed up especially with replication on. The concern here is the unnecessary time and resources spent in a blocking flush when its not really required. I have provided a patch. Its possible to provide overrides at other places, I took a stab at providing those on KS and CF and did the wiring. If you prefer some other approach, let me know. > 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 > > 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-10661) Integrate SASI to Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-10661?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070176#comment-15070176 ] Pavel Yaskevich commented on CASSANDRA-10661: - [~beobal] Yes, I will remove OR/parenthesis features from the branch (since we have a separate repo we can port it back from), do some more cleanup, and mark this as Patch Available. > 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] [Commented] (CASSANDRA-10876) Alter behavior of batch WARN and fail on single partition batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070201#comment-15070201 ] Taiyuan Zhang commented on CASSANDRA-10876: --- Attached a patch - it seems to be a really simple change, unless I misunderstood the issue... > Alter behavior of batch WARN and fail on single partition batches > - > > Key: CASSANDRA-10876 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10876 > Project: Cassandra > Issue Type: Improvement >Reporter: Patrick McFadin >Priority: Minor > > In an attempt to give operator insight into potentially harmful batch usage, > Jiras were created to log WARN or fail on certain batch sizes. This ignores > the single partition batch, which doesn't create the same issues as a > multi-partition batch. > The proposal is to ignore size on single partition batch statements. > Reference: > [CASSANDRA-6487|https://issues.apache.org/jira/browse/CASSANDRA-6487] > [CASSANDRA-8011|https://issues.apache.org/jira/browse/CASSANDRA-8011] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10876) Alter behavior of batch WARN and fail on single partition batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070223#comment-15070223 ] Patrick McFadin commented on CASSANDRA-10876: - You are correct. A single partition key on one keyspace.table. No matter how many statements are in the batch. I don't see the patch file attached. You may want to try again. > Alter behavior of batch WARN and fail on single partition batches > - > > Key: CASSANDRA-10876 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10876 > Project: Cassandra > Issue Type: Improvement >Reporter: Patrick McFadin >Priority: Minor > > In an attempt to give operator insight into potentially harmful batch usage, > Jiras were created to log WARN or fail on certain batch sizes. This ignores > the single partition batch, which doesn't create the same issues as a > multi-partition batch. > The proposal is to ignore size on single partition batch statements. > Reference: > [CASSANDRA-6487|https://issues.apache.org/jira/browse/CASSANDRA-6487] > [CASSANDRA-8011|https://issues.apache.org/jira/browse/CASSANDRA-8011] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8708) inter_dc_stream_throughput_outbound_megabits_per_sec to defaults to unlimited
[ https://issues.apache.org/jira/browse/CASSANDRA-8708?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070134#comment-15070134 ] Ariel Weisberg commented on CASSANDRA-8708: --- Jeremy can you make a 2.2, 3.0, and trunk branch of this? > inter_dc_stream_throughput_outbound_megabits_per_sec to defaults to unlimited > - > > Key: CASSANDRA-8708 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8708 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: Adam Hattrell >Assignee: Jeremy Hanna > Fix For: 2.1.x, 2.2.x > > Attachments: 8708-2.1-jmx-nodetool-bulkloader-v2.txt, > 8708-2.1-jmx-nodetool-bulkloader.txt, 8708-2.1-with-jmx-nodetool.txt, > 8708-2.1.txt > > > inter_dc_stream_throughput_outbound_megabits_per_sec was introduced in > CASSANDRA-6596. > There's some discussion in the ticket of the intention to link the default to > whatever stream_throughput_outbound_megabits_per_sec is set to. > However, it looks like it's just set to 0 - from > /src/java/org/apache/cassandra/config/Config.java > This is a bit of a pain - usually folks want to set the inter dc limits lower > than the base streaming figure. -- 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 ] Anubhav Kale updated CASSANDRA-10907: - Attachment: 0001-flush.patch > 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-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-10765) add RangeIterator interface and QueryPlan for SI
[ https://issues.apache.org/jira/browse/CASSANDRA-10765?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-10765: -- Fix Version/s: (was: 3.2) 3.x > add RangeIterator interface and QueryPlan for SI > > > Key: CASSANDRA-10765 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10765 > Project: Cassandra > Issue Type: Sub-task > Components: Local Write-Read Paths >Reporter: Pavel Yaskevich >Assignee: Pavel Yaskevich > Fix For: 3.x > > > Currently built-in indexes have only one way of handling > intersections/unions: pick the highest selectivity predicate and filter on > other index expressions. This is not always the most efficient approach. > Dynamic query planning based on the different index characteristics would be > more optimal. Query Plan should be able to choose how to do intersections, > unions based on the metadata provided by indexes (returned by RangeIterator) > and RangeIterator would became a base for cross index interactions and should > have information such as min/max token, estimate number of wrapped tokens etc. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10936) Provide option to repair from a data center in "nodetool repair"
Anubhav Kale created CASSANDRA-10936: Summary: Provide option to repair from a data center in "nodetool repair" Key: CASSANDRA-10936 URL: https://issues.apache.org/jira/browse/CASSANDRA-10936 Project: Cassandra Issue Type: Improvement Components: Tools Environment: PROD Reporter: Anubhav Kale Priority: Minor Sometimes, its known that the correct / latest data resides in a Data Center. It would be useful if nodetool repair can provide a "Source DC" option to source the data from. This will save a ton of traffic on the network. There are some discussions around this in CASSANDRA-6552. A case in point where this is handy: People may want to backup data from a designated data center (so that only one copy of data is backed up) to remote storage (azure / AWS). At restore time once the data is restored to this DC, other data centers can "source" data from this through "nodetool repair -- source Foo". -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070087#comment-15070087 ] Joshua McKenzie commented on CASSANDRA-8844: I think I understand what you're getting at: mutations from a mutation-based repair would represent the delta on the current state so interim states that data passed through that a replica missed would not be reflected in the CDC-log on the replica that was offline. Mutation-based repair would only give us a point-in-time snapshot of the CDC record for that key so, if the original node failed, you'd be able to have an accurate view of the state of the data if not the full history. My original proposal of our consistency guarantee is "At the consistency-level of the query that generated the CDC-record", so if you CL ONE, have 2 nodes down with RF=3, and then repair, you still only have the guarantee of the full, accurate state-history of the CDC-transaction reflected on one node. If you CL QUORUM, you should have quorum copies, etc. On aggregation (assuming no node failure on the CL ONE), since CDC data from the original node will be consumed along with CDC data from replicas that will have this "temporal gap", the combined result of aggregating all the CDC-data together should still be accurate. > Change Data Capture (CDC) > - > > Key: CASSANDRA-8844 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 > Project: Cassandra > Issue Type: New Feature > Components: Coordination, Local Write-Read Paths >Reporter: Tupshin Harper >Assignee: Joshua McKenzie >Priority: Critical > Fix For: 3.x > > > "In databases, change data capture (CDC) is a set of software design patterns > used to determine (and track) the data that has changed so that action can be > taken using the changed data. Also, Change data capture (CDC) is an approach > to data integration that is based on the identification, capture and delivery > of the changes made to enterprise data sources." > -Wikipedia > As Cassandra is increasingly being used as the Source of Record (SoR) for > mission critical data in large enterprises, it is increasingly being called > upon to act as the central hub of traffic and data flow to other systems. In > order to try to address the general need, we (cc [~brianmhess]), propose > implementing a simple data logging mechanism to enable per-table CDC patterns. > h2. The goals: > # Use CQL as the primary ingestion mechanism, in order to leverage its > Consistency Level semantics, and in order to treat it as the single > reliable/durable SoR for the data. > # To provide a mechanism for implementing good and reliable > (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) > continuous semi-realtime feeds of mutations going into a Cassandra cluster. > # To eliminate the developmental and operational burden of users so that they > don't have to do dual writes to other systems. > # For users that are currently doing batch export from a Cassandra system, > give them the opportunity to make that realtime with a minimum of coding. > h2. The mechanism: > We propose a durable logging mechanism that functions similar to a commitlog, > with the following nuances: > - Takes place on every node, not just the coordinator, so RF number of copies > are logged. > - Separate log per table. > - Per-table configuration. Only tables that are specified as CDC_LOG would do > any logging. > - Per DC. We are trying to keep the complexity to a minimum to make this an > easy enhancement, but most likely use cases would prefer to only implement > CDC logging in one (or a subset) of the DCs that are being replicated to > - In the critical path of ConsistencyLevel acknowledgment. Just as with the > commitlog, failure to write to the CDC log should fail that node's write. If > that means the requested consistency level was not met, then clients *should* > experience UnavailableExceptions. > - Be written in a Row-centric manner such that it is easy for consumers to > reconstitute rows atomically. > - Written in a simple format designed to be consumed *directly* by daemons > written in non JVM languages > h2. Nice-to-haves > I strongly suspect that the following features will be asked for, but I also > believe that they can be deferred for a subsequent release, and to guage > actual interest. > - Multiple logs per table. This would make it easy to have multiple > "subscribers" to a single table's changes. A workaround would be to create a > forking daemon listener, but that's not a great answer. > - Log filtering. Being able to apply filters, including UDF-based filters > would make Casandra a much more versatile feeder into other systems, and > again, reduce complexity that would otherwise need to be
[jira] [Created] (CASSANDRA-10937) OOM on multiple nodes during test of insert load
Peter Kovgan created CASSANDRA-10937: Summary: OOM on multiple nodes during test of insert load Key: CASSANDRA-10937 URL: https://issues.apache.org/jira/browse/CASSANDRA-10937 Project: Cassandra Issue Type: Bug Environment: Cassandra : 3.0.0 Installed as open archive, no connection to any OS specific installer. Java: Java(TM) SE Runtime Environment (build 1.8.0_65-b17) OS : Linux version 2.6.32-431.el6.x86_64 (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 We have: 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each physical host keeps 4 guests. Each guest assigned to have: 1 disk 300 Gb for seq. log 1 disk 4T for data 11 CPU cores Disks are local, not shared. (lshw and cpuinfo attached in file) Reporter: Peter Kovgan Fix For: 3.0.3 Attachments: more-logs.rar, test2.rar 8 cassandra nodes. Load test started with 4 clients(different and not equal machines), each running 1000 threads. Each thread assigned in round-robin way to run one of 4 different inserts. Consistency->ONE. I attach the full CQL schema of tables and the query of insert. Replication factor - 2: create keyspace OBLREPOSITORY_NY with replication = {'class':'NetworkTopologyStrategy','NY':2}; Initiall throughput is: 215.000 inserts /sec or 54Mb/sec, considering single insert size a bit larger than 256byte. Data: all fields(5-6) are short strings, except one is BLOB of 256 bytes. After about a 2-3 hours of work, I was forced to increase timout from 2000 to 5000ms, for some requests failed for short timeout. Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. (all failed nodes logs attached) I attach also example java load client. Attachments: test2.rar -contains most of material more-logs.rar - contains additional nodes logs -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10937) OOM on multiple nodes during test of insert load
[ https://issues.apache.org/jira/browse/CASSANDRA-10937?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Kovgan updated CASSANDRA-10937: - Fix Version/s: (was: 3.0.3) > OOM on multiple nodes during test of insert load > > > Key: CASSANDRA-10937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10937 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra : 3.0.0 > Installed as open archive, no connection to any OS specific installer. > Java: > Java(TM) SE Runtime Environment (build 1.8.0_65-b17) > OS : > Linux version 2.6.32-431.el6.x86_64 > (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red > Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 > We have: > 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each > physical host keeps 4 guests. > Each guest assigned to have: > 1 disk 300 Gb for seq. log > 1 disk 4T for data > 11 CPU cores > Disks are local, not shared. > (lshw and cpuinfo attached in file) >Reporter: Peter Kovgan > Attachments: more-logs.rar, test2.rar > > > 8 cassandra nodes. > Load test started with 4 clients(different and not equal machines), each > running 1000 threads. > Each thread assigned in round-robin way to run one of 4 different inserts. > Consistency->ONE. > I attach the full CQL schema of tables and the query of insert. > Replication factor - 2: > create keyspace OBLREPOSITORY_NY with replication = > {'class':'NetworkTopologyStrategy','NY':2}; > Initiall throughput is: > 215.000 inserts /sec > or > 54Mb/sec, considering single insert size a bit larger than 256byte. > Data: > all fields(5-6) are short strings, except one is BLOB of 256 bytes. > After about a 2-3 hours of work, I was forced to increase timeout from 2000 > to 5000ms, for some requests failed for short timeout. > Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. > (all failed nodes logs attached) > I attach also example java load client. > Attachments: > test2.rar -contains most of material > more-logs.rar - contains additional nodes logs -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10937) OOM on multiple nodes during test of insert load
[ https://issues.apache.org/jira/browse/CASSANDRA-10937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070702#comment-15070702 ] Peter Kovgan edited comment on CASSANDRA-10937 at 12/24/15 7:59 AM: Important: Found on one node 224Gb of hints. And this hints directory is on small disk(the save disk for logs). The disk is full. May be this is the part of the problem (other nodes are OK). was (Author: tierhetze): Important: Found on one node 224Gb of hints. And this hints directory is on small disk. The disk is full. May be this is the part of the problem (other nodes are OK). > OOM on multiple nodes during test of insert load > > > Key: CASSANDRA-10937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10937 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra : 3.0.0 > Installed as open archive, no connection to any OS specific installer. > Java: > Java(TM) SE Runtime Environment (build 1.8.0_65-b17) > OS : > Linux version 2.6.32-431.el6.x86_64 > (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red > Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 > We have: > 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each > physical host keeps 4 guests. > Each guest assigned to have: > 1 disk 300 Gb for seq. log (NOT SSD) > 1 disk 4T for data (NOT SSD) > 11 CPU cores > Disks are local, not shared. > Memory on each host - 24 Gb total. > 8 Gb - cassandra heap > (lshw and cpuinfo attached in file) >Reporter: Peter Kovgan > Attachments: more-logs.rar, some-heap-stats.rar, test2.rar > > > 8 cassandra nodes. > Load test started with 4 clients(different and not equal machines), each > running 1000 threads. > Each thread assigned in round-robin way to run one of 4 different inserts. > Consistency->ONE. > I attach the full CQL schema of tables and the query of insert. > Replication factor - 2: > create keyspace OBLREPOSITORY_NY with replication = > {'class':'NetworkTopologyStrategy','NY':2}; > Initiall throughput is: > 215.000 inserts /sec > or > 54Mb/sec, considering single insert size a bit larger than 256byte. > Data: > all fields(5-6) are short strings, except one is BLOB of 256 bytes. > After about a 2-3 hours of work, I was forced to increase timeout from 2000 > to 5000ms, for some requests failed for short timeout. > Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. > (all failed nodes logs attached) > I attach also java load client and instructions how set-up and use it. > The test is important for our strategic project and we hope it is curable. > Attachments: > test2.rar -contains most of material > more-logs.rar - contains additional nodes logs -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10937) OOM on multiple nodes during test of insert load
[ https://issues.apache.org/jira/browse/CASSANDRA-10937?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Kovgan updated CASSANDRA-10937: - Description: 8 cassandra nodes. Load test started with 4 clients(different and not equal machines), each running 1000 threads. Each thread assigned in round-robin way to run one of 4 different inserts. Consistency->ONE. I attach the full CQL schema of tables and the query of insert. Replication factor - 2: create keyspace OBLREPOSITORY_NY with replication = {'class':'NetworkTopologyStrategy','NY':2}; Initiall throughput is: 215.000 inserts /sec or 54Mb/sec, considering single insert size a bit larger than 256byte. Data: all fields(5-6) are short strings, except one is BLOB of 256 bytes. After about a 2-3 hours of work, I was forced to increase timeout from 2000 to 5000ms, for some requests failed for short timeout. Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. (all failed nodes logs attached) I attach also java load client and instructions how set-up and use it. The test is important for our strategic project and we hope it is curable. Attachments: test2.rar -contains most of material more-logs.rar - contains additional nodes logs was: 8 cassandra nodes. Load test started with 4 clients(different and not equal machines), each running 1000 threads. Each thread assigned in round-robin way to run one of 4 different inserts. Consistency->ONE. I attach the full CQL schema of tables and the query of insert. Replication factor - 2: create keyspace OBLREPOSITORY_NY with replication = {'class':'NetworkTopologyStrategy','NY':2}; Initiall throughput is: 215.000 inserts /sec or 54Mb/sec, considering single insert size a bit larger than 256byte. Data: all fields(5-6) are short strings, except one is BLOB of 256 bytes. After about a 2-3 hours of work, I was forced to increase timeout from 2000 to 5000ms, for some requests failed for short timeout. Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. (all failed nodes logs attached) I attach also java load client and instructions how set-up and use it. Attachments: test2.rar -contains most of material more-logs.rar - contains additional nodes logs > OOM on multiple nodes during test of insert load > > > Key: CASSANDRA-10937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10937 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra : 3.0.0 > Installed as open archive, no connection to any OS specific installer. > Java: > Java(TM) SE Runtime Environment (build 1.8.0_65-b17) > OS : > Linux version 2.6.32-431.el6.x86_64 > (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red > Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 > We have: > 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each > physical host keeps 4 guests. > Each guest assigned to have: > 1 disk 300 Gb for seq. log > 1 disk 4T for data > 11 CPU cores > Disks are local, not shared. > (lshw and cpuinfo attached in file) >Reporter: Peter Kovgan > Attachments: more-logs.rar, test2.rar > > > 8 cassandra nodes. > Load test started with 4 clients(different and not equal machines), each > running 1000 threads. > Each thread assigned in round-robin way to run one of 4 different inserts. > Consistency->ONE. > I attach the full CQL schema of tables and the query of insert. > Replication factor - 2: > create keyspace OBLREPOSITORY_NY with replication = > {'class':'NetworkTopologyStrategy','NY':2}; > Initiall throughput is: > 215.000 inserts /sec > or > 54Mb/sec, considering single insert size a bit larger than 256byte. > Data: > all fields(5-6) are short strings, except one is BLOB of 256 bytes. > After about a 2-3 hours of work, I was forced to increase timeout from 2000 > to 5000ms, for some requests failed for short timeout. > Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. > (all failed nodes logs attached) > I attach also java load client and instructions how set-up and use it. > The test is important for our strategic project and we hope it is curable. > Attachments: > test2.rar -contains most of material > more-logs.rar - contains additional nodes logs -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10937) OOM on multiple nodes during test of insert load
[ https://issues.apache.org/jira/browse/CASSANDRA-10937?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Kovgan updated CASSANDRA-10937: - Environment: Cassandra : 3.0.0 Installed as open archive, no connection to any OS specific installer. Java: Java(TM) SE Runtime Environment (build 1.8.0_65-b17) OS : Linux version 2.6.32-431.el6.x86_64 (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 We have: 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each physical host keeps 4 guests. Each guest assigned to have: 1 disk 300 Gb for seq. log (NOT SSD) 1 disk 4T for data (NOT SSD) 11 CPU cores Disks are local, not shared. Memory on each host - 24 Gb total. 8 Gb - cassandra heap (lshw and cpuinfo attached in file) was: Cassandra : 3.0.0 Installed as open archive, no connection to any OS specific installer. Java: Java(TM) SE Runtime Environment (build 1.8.0_65-b17) OS : Linux version 2.6.32-431.el6.x86_64 (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 We have: 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each physical host keeps 4 guests. Each guest assigned to have: 1 disk 300 Gb for seq. log 1 disk 4T for data 11 CPU cores Disks are local, not shared. Memory on each host - 24 Gb total. 8 Gb - cassandra heap (lshw and cpuinfo attached in file) > OOM on multiple nodes during test of insert load > > > Key: CASSANDRA-10937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10937 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra : 3.0.0 > Installed as open archive, no connection to any OS specific installer. > Java: > Java(TM) SE Runtime Environment (build 1.8.0_65-b17) > OS : > Linux version 2.6.32-431.el6.x86_64 > (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red > Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 > We have: > 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each > physical host keeps 4 guests. > Each guest assigned to have: > 1 disk 300 Gb for seq. log (NOT SSD) > 1 disk 4T for data (NOT SSD) > 11 CPU cores > Disks are local, not shared. > Memory on each host - 24 Gb total. > 8 Gb - cassandra heap > (lshw and cpuinfo attached in file) >Reporter: Peter Kovgan > Attachments: more-logs.rar, test2.rar > > > 8 cassandra nodes. > Load test started with 4 clients(different and not equal machines), each > running 1000 threads. > Each thread assigned in round-robin way to run one of 4 different inserts. > Consistency->ONE. > I attach the full CQL schema of tables and the query of insert. > Replication factor - 2: > create keyspace OBLREPOSITORY_NY with replication = > {'class':'NetworkTopologyStrategy','NY':2}; > Initiall throughput is: > 215.000 inserts /sec > or > 54Mb/sec, considering single insert size a bit larger than 256byte. > Data: > all fields(5-6) are short strings, except one is BLOB of 256 bytes. > After about a 2-3 hours of work, I was forced to increase timeout from 2000 > to 5000ms, for some requests failed for short timeout. > Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. > (all failed nodes logs attached) > I attach also java load client and instructions how set-up and use it. > The test is important for our strategic project and we hope it is curable. > Attachments: > test2.rar -contains most of material > more-logs.rar - contains additional nodes logs -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10937) OOM on multiple nodes during test of insert load
[ https://issues.apache.org/jira/browse/CASSANDRA-10937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15070702#comment-15070702 ] Peter Kovgan edited comment on CASSANDRA-10937 at 12/24/15 7:36 AM: Important: Found on one node 224Gb of hints. And this hints directory is on small disk. The disk is full. May be this is the part of the problem (other nodes are OK). was (Author: tierhetze): Important: Found on one node 224Gb of hints. And this hints directory is on small disk. The disk is full. May be this is the problem. > OOM on multiple nodes during test of insert load > > > Key: CASSANDRA-10937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10937 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra : 3.0.0 > Installed as open archive, no connection to any OS specific installer. > Java: > Java(TM) SE Runtime Environment (build 1.8.0_65-b17) > OS : > Linux version 2.6.32-431.el6.x86_64 > (mockbu...@x86-023.build.eng.bos.redhat.com) (gcc version 4.4.7 20120313 (Red > Hat 4.4.7-4) (GCC) ) #1 SMP Sun Nov 10 22:19:54 EST 2013 > We have: > 8 guests ( Linux OS as above) on 2 VMWare managed physical IBM M6 hosts. Each > physical host keeps 4 guests. > Each guest assigned to have: > 1 disk 300 Gb for seq. log (NOT SSD) > 1 disk 4T for data (NOT SSD) > 11 CPU cores > Disks are local, not shared. > Memory on each host - 24 Gb total. > 8 Gb - cassandra heap > (lshw and cpuinfo attached in file) >Reporter: Peter Kovgan > Attachments: more-logs.rar, test2.rar > > > 8 cassandra nodes. > Load test started with 4 clients(different and not equal machines), each > running 1000 threads. > Each thread assigned in round-robin way to run one of 4 different inserts. > Consistency->ONE. > I attach the full CQL schema of tables and the query of insert. > Replication factor - 2: > create keyspace OBLREPOSITORY_NY with replication = > {'class':'NetworkTopologyStrategy','NY':2}; > Initiall throughput is: > 215.000 inserts /sec > or > 54Mb/sec, considering single insert size a bit larger than 256byte. > Data: > all fields(5-6) are short strings, except one is BLOB of 256 bytes. > After about a 2-3 hours of work, I was forced to increase timeout from 2000 > to 5000ms, for some requests failed for short timeout. > Later on(after aprox. 12 hous of work) OOM happens on multiple nodes. > (all failed nodes logs attached) > I attach also java load client and instructions how set-up and use it. > The test is important for our strategic project and we hope it is curable. > Attachments: > test2.rar -contains most of material > more-logs.rar - contains additional nodes logs -- This message was sent by Atlassian JIRA (v6.3.4#6332)