[jira] [Comment Edited] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088444#comment-15088444 ] DOAN DuyHai edited comment on CASSANDRA-8844 at 1/8/16 12:15 AM: - I've read the updated design doc and I have a concern with the following proposal: - _.yaml configurable limit of on-disk space allowed to be take up by cdc directory. If at or above limit, throw UnavailableException on CDC-enabled mutations_ I certainly understand the need to raise a warning if the on-disk space limit for CDC overflows, but raising an UnavailableException will basically block the server for any future write (until the disk space is released). This situation occurs when CDC client does not "consume" CDC log as fast as C* flush incoming data. So we have basically a sizing/throughput issue with the consumer. Throwing UnavailableException is rather radical, and I certainly understand the need to prevent any desync between base data and consumer, but raising a WARNING or at least, proposing different failure strategy (similar to **disk_failure_policy**) like EXCEPTION_ON_OVERFLOW, WARN_ON_OVERFLOW, DISCARD_OLD_ON_OVERFLOW would offers some flexibility. Not sure how much complexity it would add to the actual impl. WDYT ? was (Author: doanduyhai): I've read the updated design doc and I have a concern with the following proposal: - _.yaml configurable limit of on-disk space allowed to be take up by cdc directory. If at or above limit, throw UnavailableException on CDC-enabled mutations_ I certainly understand the need to raise a warning if the on-disk space limit for CDC overflows, but raising an UnavailableException will basically blocks the server for any future write (until the disk space is released). This situation occurs when CDC client does not "consume" CDC log as fast as C* flush incoming data. So we have basically a sizing/throughput issue with the consumer. Throwing UnavailableException is rather radical, and I certainly understand the need to prevent any desync between base data and consumer, but raising a WARNING or at least, proposing different failure strategy (similar to **disk_failure_policy**) like EXCEPTION_ON_OVERFLOW, WARN_ON_OVERFLOW, DISCARD_OLD_ON_OVERFLOW would offers some flexibility. Not sure how much complexity it would add to the actual impl. WDYT ? > 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
[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=15088444#comment-15088444 ] DOAN DuyHai commented on CASSANDRA-8844: I've read the updated design doc and I have a concern with the following proposal: - _.yaml configurable limit of on-disk space allowed to be take up by cdc directory. If at or above limit, throw UnavailableException on CDC-enabled mutations_ I certainly understand the need to raise a warning if the on-disk space limit for CDC overflows, but raising an UnavailableException will basically blocks the server for any future write (until the disk space is released). This situation occurs when CDC client does not "consume" CDC log as fast as C* flush incoming data. So we have basically a sizing/throughput issue with the consumer. Throwing UnavailableException is rather radical, and I certainly understand the need to prevent any desync between base data and consumer, but raising a WARNING or at least, proposing different failure strategy (similar to **disk_failure_policy**) like EXCEPTION_ON_OVERFLOW, WARN_ON_OVERFLOW, DISCARD_OLD_ON_OVERFLOW would offers some flexibility. Not sure how much complexity it would add to the actual impl. WDYT ? > 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
[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=15088485#comment-15088485 ] Joshua McKenzie commented on CASSANDRA-8844: bq. similar to *disk_failure_policy* I'm +1 on that. Should be easy to implement and provide more flexibility for people to determine how to treat failures of CDC logging. Could also make it a property per keyspace along with CDC being enabled or not rather than a system-wide, but I'm not sure the benefits of that flexibility outweigh the costs since that would be considerably more work to implement. > 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
[jira] [Created] (CASSANDRA-10983) Metrics for tracking offending queries
Sharvanath Pathak created CASSANDRA-10983: - Summary: Metrics for tracking offending queries Key: CASSANDRA-10983 URL: https://issues.apache.org/jira/browse/CASSANDRA-10983 Project: Cassandra Issue Type: Improvement Reporter: Sharvanath Pathak I have seen big GC pauses leading to nodes being marked DOWN in our cluster. The most common issue is someone, would add a large range scan and it would be difficult to pin-point the specific query. I have added a mechanism to account the memory allocation for a specific query. In order to allow aggregates over a period I added a metric as well. Attached is the diff. I was wondering if something like this would be interesting for more general audience. There are some things which need to be fixed for proper release. For instance, Cleaning up existing metrics on server restart. However, just wanted to check before that if something like this would be useful for others. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9747) JNA makes Cassandra slower
[ https://issues.apache.org/jira/browse/CASSANDRA-9747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087103#comment-15087103 ] Roman Bielik commented on CASSANDRA-9747: - Hi, I have performed testing on HP Gen9 servers (2 node installation) under heavy load with various HW and Cassandra configuration. For some reason the combination of Cassandra and JNA performed worse. I could see high IO waits (using 'top') and also my test application got worse results (in matter of response times and overall performance). I have no idea how this is possible. The test performed roughly 2*6500 updates/s (writes+deletes) and 1500 reads/s over 5 million records using only primary key. > JNA makes Cassandra slower > -- > > Key: CASSANDRA-9747 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9747 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra 2.0.14; JNA 4.x; 32-core CPU; 64GB RAM (8GB > heap); 2 physical HDDs used as commit and data; Java 1.7 >Reporter: Roman Bielik >Priority: Minor > > When JNA is loaded (as recommended in the production settings), it has > negative effect on performance, especially when used with HDD => it > significantly increases the disk IO wait. > It looks like JNA works fine only when used in combination with SSD disk. > Otherwise it is better not to use JNA at all. This is very unexpected > behavior, can you please analyse? > Test description: > Roughly 50% reads and 50% writes of about 1.5kB records using Thrift > interface. > Test results (sorted by performance): > *HDD + JNA* > Worst performance. High IO wait. > *HDD, no JNA* > Good performance. Low IO wait, but the Cassandra CPU usage is quite high. > *SSD + JNA* > Seems to provide the best performance. Low IO wait. Low Cassandra CPU usage. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[13/15] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3
Merge branch 'cassandra-3.0' into cassandra-3.3 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/15cfeb77 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/15cfeb77 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/15cfeb77 Branch: refs/heads/cassandra-3.3 Commit: 15cfeb779e6dd079da02fa4175b6898069b3318f Parents: 5bf69ab 601203c Author: Sylvain LebresneAuthored: Thu Jan 7 12:03:52 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 12:03:52 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 conf/cassandra.yaml | 3 +- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 +++ .../cassandra/service/StorageServiceMBean.java | 3 ++ .../org/apache/cassandra/tools/BulkLoader.java | 12 ++- .../org/apache/cassandra/tools/NodeProbe.java | 15 +++- .../org/apache/cassandra/tools/NodeTool.java| 2 ++ .../nodetool/GetInterDCStreamThroughput.java| 33 + .../nodetool/SetInterDCStreamThroughput.java| 37 11 files changed, 123 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/CHANGES.txt -- diff --cc CHANGES.txt index db336e5,1533619..f8933fb --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -47,7 -20,12 +47,8 @@@ Merged from 2.2 * (cqlsh) show correct column names for empty result sets (CASSANDRA-9813) * Add new types to Stress (CASSANDRA-9556) * Add property to allow listening on broadcast interface (CASSANDRA-9748) - * Fix regression in split size on CqlInputFormat (CASSANDRA-10835) - * Better handling of SSL connection errors inter-node (CASSANDRA-10816) - * Disable reloading of GossipingPropertyFileSnitch (CASSANDRA-9474) - * Verify tables in pseudo-system keyspaces at startup (CASSANDRA-10761) Merged from 2.1: + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) * cqlsh fails to decode utf-8 characters for text typed columns (CASSANDRA-10875) http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/NEWS.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/conf/cassandra.yaml -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/config/Config.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/service/StorageService.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/service/StorageServiceMBean.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/tools/NodeProbe.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/tools/NodeTool.java --
[14/15] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3
Merge branch 'cassandra-3.0' into cassandra-3.3 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/15cfeb77 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/15cfeb77 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/15cfeb77 Branch: refs/heads/trunk Commit: 15cfeb779e6dd079da02fa4175b6898069b3318f Parents: 5bf69ab 601203c Author: Sylvain LebresneAuthored: Thu Jan 7 12:03:52 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 12:03:52 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 conf/cassandra.yaml | 3 +- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 +++ .../cassandra/service/StorageServiceMBean.java | 3 ++ .../org/apache/cassandra/tools/BulkLoader.java | 12 ++- .../org/apache/cassandra/tools/NodeProbe.java | 15 +++- .../org/apache/cassandra/tools/NodeTool.java| 2 ++ .../nodetool/GetInterDCStreamThroughput.java| 33 + .../nodetool/SetInterDCStreamThroughput.java| 37 11 files changed, 123 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/CHANGES.txt -- diff --cc CHANGES.txt index db336e5,1533619..f8933fb --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -47,7 -20,12 +47,8 @@@ Merged from 2.2 * (cqlsh) show correct column names for empty result sets (CASSANDRA-9813) * Add new types to Stress (CASSANDRA-9556) * Add property to allow listening on broadcast interface (CASSANDRA-9748) - * Fix regression in split size on CqlInputFormat (CASSANDRA-10835) - * Better handling of SSL connection errors inter-node (CASSANDRA-10816) - * Disable reloading of GossipingPropertyFileSnitch (CASSANDRA-9474) - * Verify tables in pseudo-system keyspaces at startup (CASSANDRA-10761) Merged from 2.1: + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) * cqlsh fails to decode utf-8 characters for text typed columns (CASSANDRA-10875) http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/NEWS.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/conf/cassandra.yaml -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/config/Config.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/service/StorageService.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/service/StorageServiceMBean.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/tools/NodeProbe.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/15cfeb77/src/java/org/apache/cassandra/tools/NodeTool.java --
[09/15] 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/e41fa341 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e41fa341 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e41fa341 Branch: refs/heads/cassandra-3.3 Commit: e41fa3419195a1b512efa478135d82d3295de5cd Parents: 202cf9b 6d6d189 Author: Sylvain LebresneAuthored: Thu Jan 7 12:03:26 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 12:03:26 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 conf/cassandra.yaml | 3 +- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 +++ .../cassandra/service/StorageServiceMBean.java | 3 ++ .../org/apache/cassandra/tools/BulkLoader.java | 12 ++- .../org/apache/cassandra/tools/NodeProbe.java | 15 +++- .../org/apache/cassandra/tools/NodeTool.java| 2 ++ .../nodetool/GetInterDCStreamThroughput.java| 33 + .../nodetool/SetInterDCStreamThroughput.java| 37 11 files changed, 123 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e41fa341/CHANGES.txt -- diff --cc CHANGES.txt index b12f593,14c5ee6..90d990e --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,20 -1,8 +1,21 @@@ -2.1.13 +2.2.5 + * Enable GC logging by default (CASSANDRA-10140) + * Optimize pending range computation (CASSANDRA-9258) + * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902) + * drop/alter user should be case sensitive (CASSANDRA-10817) + * jemalloc detection fails due to quoting issues in regexv (CASSANDRA-10946) + * Support counter-columns for native aggregates (sum,avg,max,min) (CASSANDRA-9977) + * (cqlsh) show correct column names for empty result sets (CASSANDRA-9813) + * Add new types to Stress (CASSANDRA-9556) + * Add property to allow listening on broadcast interface (CASSANDRA-9748) + * Fix regression in split size on CqlInputFormat (CASSANDRA-10835) + * Better handling of SSL connection errors inter-node (CASSANDRA-10816) + * Disable reloading of GossipingPropertyFileSnitch (CASSANDRA-9474) + * Verify tables in pseudo-system keyspaces at startup (CASSANDRA-10761) +Merged from 2.1: + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) - * Fix the way we replace sstables after anticompaction (CASSANDRA-10831) * cqlsh fails to decode utf-8 characters for text typed columns (CASSANDRA-10875) * Log error when stream session fails (CASSANDRA-9294) * Fix bugs in commit log archiving startup behavior (CASSANDRA-10593) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e41fa341/NEWS.txt -- diff --cc NEWS.txt index 8cbe4f7,845801d..f26fe3b --- a/NEWS.txt +++ b/NEWS.txt @@@ -13,29 -13,23 +13,36 @@@ restore snapshots created with the prev 'sstableloader' tool. You can upgrade the file format of your snapshots using the provided 'sstableupgrade' tool. -2.1.13 -== +2.2.4 += -New features - -- New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details. +Deprecation +--- +- Pig support has been deprecated, and will be removed in 3.0. + Please see CASSANDRA-10542 for more details. +- Configuration parameter memory_allocator in cassandra.yaml has been deprecated + and will be removed in 3.0.0. As mentioned below for 2.2.0, jemalloc is + automatically preloaded on Unix platforms. + +Operations +-- +- Switching data center or racks is no longer an allowed operation on a node + which has data. Instead, the node will need to be decommissioned and + rebootstrapped. If moving from the SimpleSnitch, make sure that the data + center and rack containing all current nodes is named "datacenter1" and + "rack1". To override this behaviour use -Dcassandra.ignore_rack=true and/or + -Dcassandra.ignore_dc=true. +- Reloading the configuration file of GossipingPropertyFileSnitch has been disabled. +- GC logging is now enabled by default (but you can disable it if you want by + commenting the relevant lines of the cassandra-env file). + Upgrading + - + - The default for the inter-DC stream throughput setting +
[02/15] cassandra git commit: Fix inter datacenter streaming default
Fix inter datacenter streaming default patch by jeromatron; reviewed by aweisberg for CASSANDRA-9708 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6d6d1890 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6d6d1890 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6d6d1890 Branch: refs/heads/cassandra-2.2 Commit: 6d6d1890425e0372debc3acc21f86dce7c08cf37 Parents: 1171654 Author: Jeremy HannaAuthored: Wed Dec 30 13:56:32 2015 -0800 Committer: Sylvain Lebresne Committed: Thu Jan 7 11:54:51 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 ++ conf/cassandra.yaml | 3 ++- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 ++ .../cassandra/service/StorageServiceMBean.java | 3 +++ .../org/apache/cassandra/tools/BulkLoader.java | 12 +- .../org/apache/cassandra/tools/NodeProbe.java | 10 .../org/apache/cassandra/tools/NodeTool.java| 25 9 files changed, 72 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 844a28f..14c5ee6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.13 + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) * Fix the way we replace sstables after anticompaction (CASSANDRA-10831) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 088efae..845801d 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -20,6 +20,13 @@ New features - New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details. +Upgrading +- +- The default for the inter-DC stream throughput setting + (inter_dc_stream_throughput_outbound_megabits_per_sec in cassandra.yaml) is + the same than the one for intra-DC one (200Mbps) instead of being unlimited. + Having it unlimited was never intended and was a bug. + 2.1.12 == http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 0d0282b..1fa04e6 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -599,7 +599,8 @@ sstable_preemptive_open_interval_in_mb: 50 # this setting allows users to throttle inter dc stream throughput in addition # to throttling all network stream traffic as configured with # stream_throughput_outbound_megabits_per_sec -# inter_dc_stream_throughput_outbound_megabits_per_sec: +# When unset, the default is 200 Mbps or 25 MB/s +# inter_dc_stream_throughput_outbound_megabits_per_sec: 200 # How long the coordinator should wait for read operations to complete read_request_timeout_in_ms: 5000 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/config/Config.java -- diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 911dd73..63bbf96 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -151,7 +151,7 @@ public class Config public Integer max_streaming_retries = 3; public volatile Integer stream_throughput_outbound_megabits_per_sec = 200; -public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 0; +public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 200; public String[] data_file_directories; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 6e38b92..f134e8a 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1088,6 +1088,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE return DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec();
[04/15] cassandra git commit: Fix inter datacenter streaming default
Fix inter datacenter streaming default patch by jeromatron; reviewed by aweisberg for CASSANDRA-9708 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6d6d1890 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6d6d1890 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6d6d1890 Branch: refs/heads/cassandra-3.0 Commit: 6d6d1890425e0372debc3acc21f86dce7c08cf37 Parents: 1171654 Author: Jeremy HannaAuthored: Wed Dec 30 13:56:32 2015 -0800 Committer: Sylvain Lebresne Committed: Thu Jan 7 11:54:51 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 ++ conf/cassandra.yaml | 3 ++- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 ++ .../cassandra/service/StorageServiceMBean.java | 3 +++ .../org/apache/cassandra/tools/BulkLoader.java | 12 +- .../org/apache/cassandra/tools/NodeProbe.java | 10 .../org/apache/cassandra/tools/NodeTool.java| 25 9 files changed, 72 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 844a28f..14c5ee6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.13 + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) * Fix the way we replace sstables after anticompaction (CASSANDRA-10831) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 088efae..845801d 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -20,6 +20,13 @@ New features - New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details. +Upgrading +- +- The default for the inter-DC stream throughput setting + (inter_dc_stream_throughput_outbound_megabits_per_sec in cassandra.yaml) is + the same than the one for intra-DC one (200Mbps) instead of being unlimited. + Having it unlimited was never intended and was a bug. + 2.1.12 == http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 0d0282b..1fa04e6 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -599,7 +599,8 @@ sstable_preemptive_open_interval_in_mb: 50 # this setting allows users to throttle inter dc stream throughput in addition # to throttling all network stream traffic as configured with # stream_throughput_outbound_megabits_per_sec -# inter_dc_stream_throughput_outbound_megabits_per_sec: +# When unset, the default is 200 Mbps or 25 MB/s +# inter_dc_stream_throughput_outbound_megabits_per_sec: 200 # How long the coordinator should wait for read operations to complete read_request_timeout_in_ms: 5000 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/config/Config.java -- diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 911dd73..63bbf96 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -151,7 +151,7 @@ public class Config public Integer max_streaming_retries = 3; public volatile Integer stream_throughput_outbound_megabits_per_sec = 200; -public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 0; +public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 200; public String[] data_file_directories; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 6e38b92..f134e8a 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1088,6 +1088,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE return DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec();
[05/15] cassandra git commit: Fix inter datacenter streaming default
Fix inter datacenter streaming default patch by jeromatron; reviewed by aweisberg for CASSANDRA-9708 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6d6d1890 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6d6d1890 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6d6d1890 Branch: refs/heads/cassandra-3.3 Commit: 6d6d1890425e0372debc3acc21f86dce7c08cf37 Parents: 1171654 Author: Jeremy HannaAuthored: Wed Dec 30 13:56:32 2015 -0800 Committer: Sylvain Lebresne Committed: Thu Jan 7 11:54:51 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 ++ conf/cassandra.yaml | 3 ++- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 ++ .../cassandra/service/StorageServiceMBean.java | 3 +++ .../org/apache/cassandra/tools/BulkLoader.java | 12 +- .../org/apache/cassandra/tools/NodeProbe.java | 10 .../org/apache/cassandra/tools/NodeTool.java| 25 9 files changed, 72 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 844a28f..14c5ee6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.13 + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) * Fix the way we replace sstables after anticompaction (CASSANDRA-10831) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 088efae..845801d 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -20,6 +20,13 @@ New features - New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details. +Upgrading +- +- The default for the inter-DC stream throughput setting + (inter_dc_stream_throughput_outbound_megabits_per_sec in cassandra.yaml) is + the same than the one for intra-DC one (200Mbps) instead of being unlimited. + Having it unlimited was never intended and was a bug. + 2.1.12 == http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 0d0282b..1fa04e6 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -599,7 +599,8 @@ sstable_preemptive_open_interval_in_mb: 50 # this setting allows users to throttle inter dc stream throughput in addition # to throttling all network stream traffic as configured with # stream_throughput_outbound_megabits_per_sec -# inter_dc_stream_throughput_outbound_megabits_per_sec: +# When unset, the default is 200 Mbps or 25 MB/s +# inter_dc_stream_throughput_outbound_megabits_per_sec: 200 # How long the coordinator should wait for read operations to complete read_request_timeout_in_ms: 5000 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/config/Config.java -- diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 911dd73..63bbf96 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -151,7 +151,7 @@ public class Config public Integer max_streaming_retries = 3; public volatile Integer stream_throughput_outbound_megabits_per_sec = 200; -public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 0; +public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 200; public String[] data_file_directories; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 6e38b92..f134e8a 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1088,6 +1088,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE return DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec();
[01/15] cassandra git commit: Fix inter datacenter streaming default
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 11716547f -> 6d6d18904 refs/heads/cassandra-2.2 202cf9b0b -> e41fa3419 refs/heads/cassandra-3.0 f28838791 -> 601203cbc refs/heads/cassandra-3.3 5bf69abe1 -> 15cfeb779 refs/heads/trunk 83aeeca30 -> 6c6c7067a Fix inter datacenter streaming default patch by jeromatron; reviewed by aweisberg for CASSANDRA-9708 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6d6d1890 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6d6d1890 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6d6d1890 Branch: refs/heads/cassandra-2.1 Commit: 6d6d1890425e0372debc3acc21f86dce7c08cf37 Parents: 1171654 Author: Jeremy HannaAuthored: Wed Dec 30 13:56:32 2015 -0800 Committer: Sylvain Lebresne Committed: Thu Jan 7 11:54:51 2016 +0100 -- CHANGES.txt | 1 + NEWS.txt| 7 ++ conf/cassandra.yaml | 3 ++- .../org/apache/cassandra/config/Config.java | 2 +- .../cassandra/service/StorageService.java | 12 ++ .../cassandra/service/StorageServiceMBean.java | 3 +++ .../org/apache/cassandra/tools/BulkLoader.java | 12 +- .../org/apache/cassandra/tools/NodeProbe.java | 10 .../org/apache/cassandra/tools/NodeTool.java| 25 9 files changed, 72 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 844a28f..14c5ee6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.13 + * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-9708) * Match cassandra-loader options in COPY FROM (CASSANDRA-9303) * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309) * Fix the way we replace sstables after anticompaction (CASSANDRA-10831) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 088efae..845801d 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -20,6 +20,13 @@ New features - New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details. +Upgrading +- +- The default for the inter-DC stream throughput setting + (inter_dc_stream_throughput_outbound_megabits_per_sec in cassandra.yaml) is + the same than the one for intra-DC one (200Mbps) instead of being unlimited. + Having it unlimited was never intended and was a bug. + 2.1.12 == http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 0d0282b..1fa04e6 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -599,7 +599,8 @@ sstable_preemptive_open_interval_in_mb: 50 # this setting allows users to throttle inter dc stream throughput in addition # to throttling all network stream traffic as configured with # stream_throughput_outbound_megabits_per_sec -# inter_dc_stream_throughput_outbound_megabits_per_sec: +# When unset, the default is 200 Mbps or 25 MB/s +# inter_dc_stream_throughput_outbound_megabits_per_sec: 200 # How long the coordinator should wait for read operations to complete read_request_timeout_in_ms: 5000 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/config/Config.java -- diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 911dd73..63bbf96 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -151,7 +151,7 @@ public class Config public Integer max_streaming_retries = 3; public volatile Integer stream_throughput_outbound_megabits_per_sec = 200; -public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 0; +public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 200; public String[] data_file_directories; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d6d1890/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 6e38b92..f134e8a 100644 ---
[jira] [Comment Edited] (CASSANDRA-10963) Can join cluster java.lang.InterruptedException
[ https://issues.apache.org/jira/browse/CASSANDRA-10963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087211#comment-15087211 ] Jack Money edited comment on CASSANDRA-10963 at 1/7/16 11:26 AM: - Hello I don't run scrubbing, because I think that sstable are correct and scrub can take lot of time. Meantime i declared joining node joiningDC1 as DC3 and force to load data from DC2 with nodetool rebuild. The same error occured on nodeDC2. Now i started nodetool scrub on existing nodes. I will let you know when finished. was (Author: jack doo): Hello I don't run scrubbing, because I think that sstable are correct and scrub can take lot of time. Meantime i declared joining node joiningDC1 as DC3 and force to load data from DC2 with nodetool rebuild. The same error occured. Now i started nodetool scrub on existing nodes. I will let you know when finished. > Can join cluster java.lang.InterruptedException > > > Key: CASSANDRA-10963 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10963 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging > Environment: [cqlsh 5.0.1 | Cassandra 2.2.4 | CQL spec 3.3.1 | Native > protocol v4] > java version "1.8.0_65" >Reporter: Jack Money >Assignee: Paulo Motta > > hello > I got 2 nodes in 2 DC. > Each node own 100% data of keyspace hugespace. > Keyspace have 21 tables with 2TB data > Biggest table have 1.6 TB of data. > Biggest sstable 1,3 TB. > Schemats: > {noformat} > KEYSPACE hugespace WITH replication = {'class': 'NetworkTopologyStrategy', > 'DC1': '3', 'DC2': '1'}; > CREATE TABLE hugespace.content ( > y int, > m int, > d int, > ts bigint, > ha text, > co text, > he text, > ids bigint, > ifr text, > js text, > PRIMARY KEY ((y, m, d), ts, ha) > ) WITH CLUSTERING ORDER BY (ts ASC, ha ASC) > AND bloom_filter_fp_chance = 0.01 > AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}' > AND comment = '' > AND compaction = {'class': > 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy'} > AND compression = {'sstable_compression': > 'org.apache.cassandra.io.compress.LZ4Compressor'} > AND dclocal_read_repair_chance = 0.1 > AND default_time_to_live = 0 > AND gc_grace_seconds = 864000 > AND max_index_interval = 2048 > AND memtable_flush_period_in_ms = 0 > AND min_index_interval = 128 > AND read_repair_chance = 0.0 > AND speculative_retry = '99.0PERCENTILE'; > CREATE INDEX content_ids_idx ON hugespace.content (ids); > {noformat} > I tried to add one node (target 6 node in DC1) to DC1. > Names: > Existing node in DC1 = nodeDC1 > Existing node in DC2 = nodeDC2 > New node joining DC1 = joiningDC1 > joiningDC1 > {noformat} > INFO [main] 2016-01-04 12:17:55,535 StorageService.java:1176 - JOINING: > Starting to bootstrap... > INFO [main] 2016-01-04 12:17:55,802 StreamResultFuture.java:86 - [Stream > #2f473320-b2dd-11e5-8353-b5506ad414a4] Executing streaming plan for Bootstrap > INFO [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,803 > StreamSession.java:232 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Starting streaming to /nodeDC1 > INFO [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,803 > StreamSession.java:232 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Starting streaming to /nodeDC2 > DEBUG [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,803 > ConnectionHandler.java:82 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for incoming stream > DEBUG [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,803 > ConnectionHandler.java:82 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for incoming stream > DEBUG [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,806 > ConnectionHandler.java:87 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for outgoing stream > DEBUG [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,806 > ConnectionHandler.java:87 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for outgoing stream > DEBUG [STREAM-OUT-/nodeDC1] 2016-01-04 12:17:55,810 > ConnectionHandler.java:334 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending Prepare (5 requests, 0 files} > DEBUG [STREAM-OUT-/nodeDC2] 2016-01-04 12:17:55,810 > ConnectionHandler.java:334 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending Prepare (2 requests, 0 files} > INFO [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,810 > StreamCoordinator.java:213 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4, > ID#0] Beginning stream session with /nodeDC2 > INFO [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,810 > StreamCoordinator.java:213 - [Stream
[jira] [Commented] (CASSANDRA-10963) Can join cluster java.lang.InterruptedException
[ https://issues.apache.org/jira/browse/CASSANDRA-10963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087211#comment-15087211 ] Jack Money commented on CASSANDRA-10963: Hello I don't run scrubbing, because I think that sstable are correct and scrub can take lot of time. Meantime i declared joining node joiningDC1 as DC3 and force to load data from DC2 with nodetool rebuild. The same error occured. Now i started nodetool scrub on existing nodes. I will let you know when finished. > Can join cluster java.lang.InterruptedException > > > Key: CASSANDRA-10963 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10963 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging > Environment: [cqlsh 5.0.1 | Cassandra 2.2.4 | CQL spec 3.3.1 | Native > protocol v4] > java version "1.8.0_65" >Reporter: Jack Money >Assignee: Paulo Motta > > hello > I got 2 nodes in 2 DC. > Each node own 100% data of keyspace hugespace. > Keyspace have 21 tables with 2TB data > Biggest table have 1.6 TB of data. > Biggest sstable 1,3 TB. > Schemats: > {noformat} > KEYSPACE hugespace WITH replication = {'class': 'NetworkTopologyStrategy', > 'DC1': '3', 'DC2': '1'}; > CREATE TABLE hugespace.content ( > y int, > m int, > d int, > ts bigint, > ha text, > co text, > he text, > ids bigint, > ifr text, > js text, > PRIMARY KEY ((y, m, d), ts, ha) > ) WITH CLUSTERING ORDER BY (ts ASC, ha ASC) > AND bloom_filter_fp_chance = 0.01 > AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}' > AND comment = '' > AND compaction = {'class': > 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy'} > AND compression = {'sstable_compression': > 'org.apache.cassandra.io.compress.LZ4Compressor'} > AND dclocal_read_repair_chance = 0.1 > AND default_time_to_live = 0 > AND gc_grace_seconds = 864000 > AND max_index_interval = 2048 > AND memtable_flush_period_in_ms = 0 > AND min_index_interval = 128 > AND read_repair_chance = 0.0 > AND speculative_retry = '99.0PERCENTILE'; > CREATE INDEX content_ids_idx ON hugespace.content (ids); > {noformat} > I tried to add one node (target 6 node in DC1) to DC1. > Names: > Existing node in DC1 = nodeDC1 > Existing node in DC2 = nodeDC2 > New node joining DC1 = joiningDC1 > joiningDC1 > {noformat} > INFO [main] 2016-01-04 12:17:55,535 StorageService.java:1176 - JOINING: > Starting to bootstrap... > INFO [main] 2016-01-04 12:17:55,802 StreamResultFuture.java:86 - [Stream > #2f473320-b2dd-11e5-8353-b5506ad414a4] Executing streaming plan for Bootstrap > INFO [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,803 > StreamSession.java:232 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Starting streaming to /nodeDC1 > INFO [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,803 > StreamSession.java:232 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Starting streaming to /nodeDC2 > DEBUG [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,803 > ConnectionHandler.java:82 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for incoming stream > DEBUG [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,803 > ConnectionHandler.java:82 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for incoming stream > DEBUG [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,806 > ConnectionHandler.java:87 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for outgoing stream > DEBUG [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,806 > ConnectionHandler.java:87 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending stream init for outgoing stream > DEBUG [STREAM-OUT-/nodeDC1] 2016-01-04 12:17:55,810 > ConnectionHandler.java:334 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending Prepare (5 requests, 0 files} > DEBUG [STREAM-OUT-/nodeDC2] 2016-01-04 12:17:55,810 > ConnectionHandler.java:334 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] > Sending Prepare (2 requests, 0 files} > INFO [StreamConnectionEstablisher:2] 2016-01-04 12:17:55,810 > StreamCoordinator.java:213 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4, > ID#0] Beginning stream session with /nodeDC2 > INFO [StreamConnectionEstablisher:1] 2016-01-04 12:17:55,810 > StreamCoordinator.java:213 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4, > ID#0] Beginning stream session with /nodeDC1 > DEBUG [STREAM-IN-/nodeDC2] 2016-01-04 12:17:55,821 ConnectionHandler.java:266 > - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4] Received Prepare (0 > requests, 1 files} > INFO [STREAM-IN-/nodeDC2] 2016-01-04 12:17:55,822 > StreamResultFuture.java:168 - [Stream #2f473320-b2dd-11e5-8353-b5506ad414a4 > ID#0] Prepare completed.
[jira] [Commented] (CASSANDRA-10855) Use Caffeine (W-TinyLFU) for on-heap caches
[ https://issues.apache.org/jira/browse/CASSANDRA-10855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087252#comment-15087252 ] Ben Manes commented on CASSANDRA-10855: --- The latest [snapshot jar|https://oss.sonatype.org/content/repositories/snapshots/com/github/ben-manes/caffeine/caffeine] includes a two optimizations. Insertions now avoid an unnecessary lambda. I suspect that will have a negligible benefit, but its always good to be more GC hygienic. A cache below 50% capacity will skip read policy work. That means it won't record the access in ring buffers which reduces contention. That also reduces the how often policy the maintenance work is scheduled, as the buffers don't need to be drained. A write will still trigger a maintenance cycle, but that should be shorter by doing less. This result in throughput close to a raw ConcurrentHashMap and then incurring the penalty when the threshold is crossed. That should improve _trades-fwd-lcs-nolz4_ and anyone else's usage where the cache is merely a safety threshold but isn't likely to grow close to the maximum. > Use Caffeine (W-TinyLFU) for on-heap caches > --- > > Key: CASSANDRA-10855 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10855 > Project: Cassandra > Issue Type: Improvement >Reporter: Ben Manes > Labels: performance > > Cassandra currently uses > [ConcurrentLinkedHashMap|https://code.google.com/p/concurrentlinkedhashmap] > for performance critical caches (key, counter) and Guava's cache for > non-critical (auth, metrics, security). All of these usages have been > replaced by [Caffeine|https://github.com/ben-manes/caffeine], written by the > author of the previously mentioned libraries. > The primary incentive is to switch from LRU policy to W-TinyLFU, which > provides [near optimal|https://github.com/ben-manes/caffeine/wiki/Efficiency] > hit rates. It performs particularly well in database and search traces, is > scan resistant, and as adds a very small time/space overhead to LRU. > Secondarily, Guava's caches never obtained similar > [performance|https://github.com/ben-manes/caffeine/wiki/Benchmarks] to CLHM > due to some optimizations not being ported over. This change results in > faster reads and not creating garbage as a side-effect. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/3] cassandra git commit: Commit CASSANDRA-9258 to 3.3+
Commit CASSANDRA-9258 to 3.3+ For some reason I don't fully understand, the patch for CASSANRA-9258 didn't made it from 3.0 to the 3.3 and trunk branches so this commit it. Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ffc0840d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ffc0840d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ffc0840d Branch: refs/heads/trunk Commit: ffc0840d03014331035f24be3c76aa812973089f Parents: 65b1f2d Author: Sylvain LebresneAuthored: Thu Jan 7 11:23:05 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 11:23:05 2016 +0100 -- .../apache/cassandra/locator/TokenMetadata.java | 61 ++-- .../cassandra/service/StorageService.java | 2 +- 2 files changed, 32 insertions(+), 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ffc0840d/src/java/org/apache/cassandra/locator/TokenMetadata.java -- diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java index a3be9de..f1348c1 100644 --- a/src/java/org/apache/cassandra/locator/TokenMetadata.java +++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java @@ -84,7 +84,7 @@ public class TokenMetadata // (don't need to record Token here since it's still part of tokenToEndpointMap until it's done leaving) private final Set leavingEndpoints = new HashSet<>(); // this is a cache of the calculation from {tokenToEndpointMap, bootstrapTokens, leavingEndpoints} -private final ConcurrentMap > pendingRanges = new ConcurrentHashMap<>(); +private final ConcurrentMap pendingRanges = new ConcurrentHashMap (); // nodes which are migrating to the new tokens in the ring private final Set > movingEndpoints = new HashSet<>(); @@ -688,23 +688,30 @@ public class TokenMetadata return sortedTokens; } -private Multimap getPendingRangesMM(String keyspaceName) +public Multimap getPendingRangesMM(String keyspaceName) { -Multimap map = pendingRanges.get(keyspaceName); -if (map == null) +Multimap map = HashMultimap.create(); +PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName); + +if (pendingRangeMaps != null) { -map = HashMultimap.create(); -Multimap priorMap = pendingRanges.putIfAbsent(keyspaceName, map); -if (priorMap != null) -map = priorMap; +for (Map.Entry entry : pendingRangeMaps) +{ +Range range = entry.getKey(); +for (InetAddress address : entry.getValue()) +{ +map.put(range, address); +} +} } + return map; } /** a mutable map may be returned but caller should not modify it */ -public Map getPendingRanges(String keyspaceName) +public PendingRangeMaps getPendingRanges(String keyspaceName) { -return getPendingRangesMM(keyspaceName).asMap(); +return this.pendingRanges.get(keyspaceName); } public List getPendingRanges(String keyspaceName, InetAddress endpoint) @@ -748,7 +755,7 @@ public class TokenMetadata lock.readLock().lock(); try { -Multimap newPendingRanges = HashMultimap.create(); +PendingRangeMaps newPendingRanges = new PendingRangeMaps(); if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty()) { @@ -776,7 +783,10 @@ public class TokenMetadata { Set currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata)); Set newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata)); -newPendingRanges.putAll(range, Sets.difference(newEndpoints, currentEndpoints)); +for (InetAddress address : Sets.difference(newEndpoints, currentEndpoints)) +{ +newPendingRanges.addPendingRange(range, address); +} } // At this stage newPendingRanges has been updated according to leave operations. We can @@ -791,7 +801,9 @@ public class TokenMetadata
[3/3] cassandra git commit: Merge branch 'cassandra-3.3' into trunk
Merge branch 'cassandra-3.3' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/83aeeca3 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/83aeeca3 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/83aeeca3 Branch: refs/heads/trunk Commit: 83aeeca3053b15272713388601f9d7907286a0d4 Parents: 6ec5d55 5bf69ab Author: Sylvain LebresneAuthored: Thu Jan 7 11:37:03 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 11:37:03 2016 +0100 -- .../cassandra/locator/PendingRangeMaps.java | 209 +++ .../test/microbench/PendingRangesBench.java | 89 .../cassandra/locator/PendingRangeMapsTest.java | 78 +++ 3 files changed, 376 insertions(+) --
[2/9] cassandra git commit: Add notice for CASSANDRA-10880 in NEWS file
Add notice for CASSANDRA-10880 in NEWS file Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b477c0f2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b477c0f2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b477c0f2 Branch: refs/heads/cassandra-3.3 Commit: b477c0f27dc6d76ae806508efc669079fa24bf35 Parents: 601203c Author: Sylvain LebresneAuthored: Thu Jan 7 16:22:34 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 16:22:34 2016 +0100 -- NEWS.txt | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b477c0f2/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index c88f47b..89fc4a7 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -21,6 +21,11 @@ New features - Hinted handoff now supports compression. Reference cassandra.yaml:hints_compression. Note: hints compression is currently disabled by default. +Upgrading +- +- Nothing specific to 3.0.3 but please see previous versions upgrading section, + especially if you are upgrading from 2.2. + 3.0.1 = @@ -68,6 +73,9 @@ New features Upgrading - + - Clients must use the native protocol version 3 when upgrading from 2.2.X as + the native protocol version 4 is not compatible between 2.2.X and 3.Y. See + https://www.mail-archive.com/user@cassandra.apache.org/msg45381.html for details. - A new argument of type InetAdress has been added to IAuthenticator::newSaslNegotiator, representing the IP address of the client attempting authentication. It will be a breaking change for any custom implementations.
[3/9] cassandra git commit: Add notice for CASSANDRA-10880 in NEWS file
Add notice for CASSANDRA-10880 in NEWS file Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b477c0f2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b477c0f2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b477c0f2 Branch: refs/heads/trunk Commit: b477c0f27dc6d76ae806508efc669079fa24bf35 Parents: 601203c Author: Sylvain LebresneAuthored: Thu Jan 7 16:22:34 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 16:22:34 2016 +0100 -- NEWS.txt | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b477c0f2/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index c88f47b..89fc4a7 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -21,6 +21,11 @@ New features - Hinted handoff now supports compression. Reference cassandra.yaml:hints_compression. Note: hints compression is currently disabled by default. +Upgrading +- +- Nothing specific to 3.0.3 but please see previous versions upgrading section, + especially if you are upgrading from 2.2. + 3.0.1 = @@ -68,6 +73,9 @@ New features Upgrading - + - Clients must use the native protocol version 3 when upgrading from 2.2.X as + the native protocol version 4 is not compatible between 2.2.X and 3.Y. See + https://www.mail-archive.com/user@cassandra.apache.org/msg45381.html for details. - A new argument of type InetAdress has been added to IAuthenticator::newSaslNegotiator, representing the IP address of the client attempting authentication. It will be a breaking change for any custom implementations.
[jira] [Commented] (CASSANDRA-10969) long-running cluster sees bad gossip generation when a node restarts
[ https://issues.apache.org/jira/browse/CASSANDRA-10969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087314#comment-15087314 ] T. David Hudson commented on CASSANDRA-10969: - A single-pass rolling restart proved insufficient; there's probably an additional problem with gossip in this area. Node 1's gossip generation had been being rejected by nodes 2, 3, and 4. N2 was the first to be restarted. Nodetool status on N2 showed N1 up, at least for a little while (until N3 got restarted?). Then nodetool status on N2 started reporting N1 down, and in its log, it was rejecting N1's generation based on an old generation, despite that its system.local had a new generation. Nodetool gossipinfo on N2 was reporting an old generation for N1. After N3 and N4 had been restarted, nodetool status commands on N2 and N3 were still reporting N1 down, but N4 was reporting N1 up. Restarting N1 made no difference. Restarting N2 and then N3 again was required for the cluster to become fully up. > long-running cluster sees bad gossip generation when a node restarts > > > Key: CASSANDRA-10969 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10969 > Project: Cassandra > Issue Type: Bug > Components: Coordination > Environment: 4-node Cassandra 2.1.1 cluster, each node running on a > Linux 2.6.32-431.20.3.dl6.x86_64 VM >Reporter: T. David Hudson >Assignee: Joel Knighton >Priority: Minor > > One of the nodes in a long-running Cassandra 2.1.1 cluster (not under my > control) restarted. The remaining nodes are logging errors like this: > "received an invalid gossip generation for peer xxx.xxx.xxx.xxx; local > generation = 1414613355, received generation = 1450978722" > The gap between the local and received generation numbers exceeds the > one-year threshold added for CASSANDRA-8113. The system clocks are > up-to-date for all nodes. > If this is a bug, the latest released Gossiper.java code in 2.1.x, 2.2.x, and > 3.0.x seems not to have changed the behavior that I'm seeing. > I presume that restarting the remaining nodes will clear up the problem, > whence the minor priority. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10975) Histogram buckets exposed in jmx are sorted by count
[ https://issues.apache.org/jira/browse/CASSANDRA-10975?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] T Jake Luciani updated CASSANDRA-10975: --- Fix Version/s: 3.2 3.0.3 > Histogram buckets exposed in jmx are sorted by count > > > Key: CASSANDRA-10975 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10975 > Project: Cassandra > Issue Type: Bug > Components: Observability >Reporter: Chris Lohfink > Fix For: 3.0.3, 3.2 > > Attachments: 10975-2.2.patch > > > The estimated histogram snapshot lets its parent provide the getValues > implementation which sorts the bucket array: > https://github.com/dropwizard/metrics/blob/3.1-maintenance/metrics-core/src/main/java/com/codahale/metrics/UniformSnapshot.java#L25 > making it hard to determine what count belonged to what bucket. Along with > removal of the pre 2.2 deprecated metrics this makes it nearly impossible to > track latencies over time. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-10880) Paging state between 2.2 and 3.0 are incompatible on protocol v4
[ https://issues.apache.org/jira/browse/CASSANDRA-10880?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-10880. -- Resolution: Fixed Assignee: Sylvain Lebresne Fix Version/s: (was: 3.x) 3.2 3.0.3 As said above, I've now pushed an email to the mailing list about this and added instructions (to use the native protocol v3 during upgrades) to the NEWS file, so closing. > Paging state between 2.2 and 3.0 are incompatible on protocol v4 > > > Key: CASSANDRA-10880 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10880 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Sylvain Lebresne >Assignee: Sylvain Lebresne >Priority: Critical > Labels: client-impacting > Fix For: 3.0.3, 3.2 > > > In CASSANDRA-10254, the paging states generated by 3.0 for the native > protocol v4 were made 3.0 specific. This was done because the paging state in > pre-3.0 versions contains a serialized cell name, but 3.0 doesn't talk in > term of cells internally (at least not the pre-3.0 ones) and so using an > old-format cell name when we only have 3.0 nodes is inefficient and inelegant. > Unfortunately that change was made on the assumption than the protocol v4 was > 3.0 only but it's not, it ended up being released with 2.2 and that > completely slipped my mind. So in practice, you can't properly have a mixed > 2.2/3.0 cluster if your driver is using the protocol v4. > And unfortunately, I don't think there is an easy way to fix that without > breaking something. Concretely, I can see 3 choices: > # we change 3.0 so that it generates old-format paging states on the v4 > protocol. The 2 main downsides are that 1) this breaks 3.0 upgrades if the > driver is using the v4 protocol, and at least on the java side the only > driver versions that support 3.0 will use v4 by default and 2) we're signing > off on having sub-optimal paging state until the protocol v5 ships (probably > not too soon). > # we remove the v4 protocol from 2.2. This means 2.2 will have to use v3 > before upgrade at the risk of breaking upgrade. This is also bad, but I'm not > sure the driver version using the v4 protocol are quite ready yet (at least > the java driver is not GA yet) so if we work with the drivers teams to make > sure the v3 protocol gets prefered by default on 2.2 in the GA versions of > these driver, this might be somewhat transparent to users. > # we don't change anything code-wise, but we document clearly that you can't > upgrade from 2.2 to 3.0 if your clients use protocol v4 (so we leave upgrade > broken if the v4 protocol is used as it is currently). This is not great, but > we can work with the drivers teams here again to make sure drivers prefer the > v3 version for 2.2 nodes so most people don't notice in practice. > I think I'm leaning towards solution 3). It's not great but at least we break > no minor upgrades (neither on 2.2, nor on 3.0) which is probably the most > important. We'd basically be just adding a new condition on 2.2->3.0 > upgrades. We could additionally make 3.0 node completely refuse v4 > connections if they know a 2.2 nodes is in the cluster for extra safety. > Ping [~omichallat], [~adutra] and [~aholmber] as you might want to be aware > of that ticket. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10970) SSL/TLS: Certificate Domain is ignored
[ https://issues.apache.org/jira/browse/CASSANDRA-10970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087364#comment-15087364 ] Stefan Podkowinski commented on CASSANDRA-10970: This has been addressed in CASSANDRA-9220 (intra-node) and [PYTHON-296|https://datastax-oss.atlassian.net/browse/PYTHON-296] (cqlsh). > SSL/TLS: Certificate Domain is ignored > -- > > Key: CASSANDRA-10970 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10970 > Project: Cassandra > Issue Type: Bug >Reporter: Matthias Brandt > > I've set up server_encryption_options as well as client_encryption_options. > In both settings, I use the same keystore with an wild-card SSL certificate > in it. It is signed by our own CA, which root certificate is in the > configured truststore: > {code} > server_encryption_options: > internode_encryption: all > keystore: /etc/cassandra/conf/wildcard-cert.keystore > keystore_password: "" > truststore: /etc/cassandra/conf/my-cacerts > truststore_password: changeit > require_client_auth: true > client_encryption_options: > enabled: true > keystore: /etc/cassandra/conf/wildcard-cert.keystore > keystore_password: "" > require_client_auth: false > {code} > The certifcate's subject is: > {code}CN=*.my.domain.com,OU=my unit,O=my org{code} > When I deploy this setting on a server which domain is > node1.my.*other-domain*.com a connection via cqlsh wrongly works. > Additionally, the inter-node connection between other nodes in this wrong > domain also works. > I would expect that the connection would fail with a meaningful error message. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Add 3.0 metadata to sstablemetadata output
Repository: cassandra Updated Branches: refs/heads/trunk 8047d5a76 -> f13bebe52 Add 3.0 metadata to sstablemetadata output Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f13bebe5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f13bebe5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f13bebe5 Branch: refs/heads/trunk Commit: f13bebe52e5a1f11635e9eb4f300b2a36d936d24 Parents: 8047d5a Author: Shogo HoshiiAuthored: Wed Jan 6 14:45:17 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 09:29:42 2016 -0600 -- CHANGES.txt | 1 + .../cassandra/db/SerializationHeader.java | 25 + .../cassandra/tools/SSTableMetadataViewer.java | 55 3 files changed, 81 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f13bebe5/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 8276a9a..8667edd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 3.4 * Add nodetool gettimeout and settimeout commands (CASSANDRA-10953) + * Add 3.0 metadata to sstablemetadata output (CASSANDRA-10838) 3.3 http://git-wip-us.apache.org/repos/asf/cassandra/blob/f13bebe5/src/java/org/apache/cassandra/db/SerializationHeader.java -- diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java index 0706d06..6aee0b6 100644 --- a/src/java/org/apache/cassandra/db/SerializationHeader.java +++ b/src/java/org/apache/cassandra/db/SerializationHeader.java @@ -361,6 +361,31 @@ public class SerializationHeader return String.format("SerializationHeader.Component[key=%s, cks=%s, statics=%s, regulars=%s, stats=%s]", keyType, clusteringTypes, staticColumns, regularColumns, stats); } + +public AbstractType getKetType() +{ +return keyType; +} + +public List getClusteringTypes() +{ +return clusteringTypes; +} + +public Map getStaticColumns() +{ +return staticColumns; +} + +public Map getRegularColumns() +{ +return regularColumns; +} + +public EncodingStats getEncodingStats() +{ +return stats; +} } public static class Serializer implements IMetadataComponentSerializer http://git-wip-us.apache.org/repos/asf/cassandra/blob/f13bebe5/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java -- diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java index 5b47004..25e6a8f 100644 --- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java +++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java @@ -20,13 +20,22 @@ package org.apache.cassandra.tools; import java.io.File; import java.io.IOException; import java.io.PrintStream; +import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.EnumSet; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.rows.EncodingStats; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.metadata.*; +import org.apache.cassandra.utils.FBUtilities; /** * Shows the contents of sstable metadata @@ -60,6 +69,7 @@ public class SSTableMetadataViewer File compressionFile = new File(descriptor.filenameFor(Component.COMPRESSION_INFO)); if (compressionFile.exists()) compression = CompressionMetadata.create(fname); +SerializationHeader.Component header = (SerializationHeader.Component) metadata.get(MetadataType.HEADER); out.printf("SSTable: %s%n", descriptor); if (validation != null) @@ -71,15 +81,36 @@ public class SSTableMetadataViewer { out.printf("Minimum timestamp: %s%n", stats.minTimestamp); out.printf("Maximum timestamp: %s%n", stats.maxTimestamp); +out.printf("SSTable min local
[jira] [Commented] (CASSANDRA-10961) Not enough bytes error when add nodes to cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-10961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088654#comment-15088654 ] xiaost commented on CASSANDRA-10961: wonderful! I will test it ASAP. > Not enough bytes error when add nodes to cluster > > > Key: CASSANDRA-10961 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10961 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: xiaost >Assignee: Paulo Motta > Attachments: apache-cassandra-2.2.4-SNAPSHOT.jar, debug.1.log, > debug.logs.zip, netstats.1.log > > > we got the same problem all the time when we add nodes to cluster. > netstats: > on HostA > {noformat} > /la-38395-big-Data.db 14792091851/14792091851 bytes(100%) sent to idx:0/HostB > {noformat} > on HostB > {noformat} > tmp-la-4-big-Data.db 2667087450/14792091851 bytes(18%) received from > idx:0/HostA > {noformat} > After a while, Error on HostB > {noformat} > WARN [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:644 - > [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Retrying for following error > java.lang.IllegalArgumentException: Not enough bytes > at > org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) > ~[guava-16.0.jar:na] > at > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:173) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:58) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261) > [apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal] > ERROR [Thread-28] 2016-01-02 12:08:14,737 CassandraDaemon.java:185 - > Exception in thread Thread[Thread-28,5,main] > java.lang.RuntimeException: java.lang.InterruptedException > at com.google.common.base.Throwables.propagate(Throwables.java:160) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_66-internal] > Caused by: java.lang.InterruptedException: null > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220) > ~[na:1.8.0_66-internal] > at > java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335) > ~[na:1.8.0_66-internal] > at > java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:350) > ~[na:1.8.0_66-internal] > at > org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:176) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > ~[apache-cassandra-2.2.4.jar:2.2.4] > ... 1 common
cassandra git commit: push down allocations to where they are needed
Repository: cassandra Updated Branches: refs/heads/trunk 15da29f0f -> 67330bec1 push down allocations to where they are needed Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/67330bec Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/67330bec Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/67330bec Branch: refs/heads/trunk Commit: 67330bec1d26e378ac47ffc65b50201f620cbe0d Parents: 15da29f Author: Dave BrosiusAuthored: Thu Jan 7 23:07:11 2016 -0500 Committer: Dave Brosius Committed: Thu Jan 7 23:07:11 2016 -0500 -- .../cassandra/db/compaction/CompactionStrategyManager.java | 6 ++ 1 file changed, 2 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/67330bec/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java index 067a0c1..a9d42eb 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java @@ -178,11 +178,11 @@ public class CompactionStrategyManager implements INotificationConsumer if (!cfs.getPartitioner().splitter().isPresent()) return 0; -Directories.DataDirectory[] directories = locations.getWriteableLocations(); - List boundaries = StorageService.getDiskBoundaries(cfs, locations.getWriteableLocations()); if (boundaries == null) { +Directories.DataDirectory[] directories = locations.getWriteableLocations(); + // try to figure out location based on sstable directory: for (int i = 0; i < directories.length; i++) { @@ -196,8 +196,6 @@ public class CompactionStrategyManager implements INotificationConsumer int pos = Collections.binarySearch(boundaries, sstable.first); assert pos < 0; // boundaries are .minkeybound and .maxkeybound so they should never be equal return -pos - 1; - - } public void shutdown()
cassandra git commit: fix logging context
Repository: cassandra Updated Branches: refs/heads/trunk 0ad1adb3b -> 3fc02dfcc fix logging context Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3fc02dfc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3fc02dfc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3fc02dfc Branch: refs/heads/trunk Commit: 3fc02dfcc775f979b9e47c20814487dd5a05f733 Parents: 0ad1adb Author: Dave BrosiusAuthored: Thu Jan 7 23:48:37 2016 -0500 Committer: Dave Brosius Committed: Thu Jan 7 23:48:37 2016 -0500 -- .../cassandra/db/compaction/writers/CompactionAwareWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3fc02dfc/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java index 46023ce..89c834f 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java @@ -46,7 +46,7 @@ import org.apache.cassandra.service.StorageService; */ public abstract class CompactionAwareWriter extends Transactional.AbstractTransactional implements Transactional { -protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class); +protected static final Logger logger = LoggerFactory.getLogger(CompactionAwareWriter.class); protected final ColumnFamilyStore cfs; protected final Directories directories;
cassandra git commit: loggers should be static
Repository: cassandra Updated Branches: refs/heads/trunk 67330bec1 -> 0ad1adb3b loggers should be static Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0ad1adb3 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0ad1adb3 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0ad1adb3 Branch: refs/heads/trunk Commit: 0ad1adb3b327320c362b25be9d43cd7231fe096e Parents: 67330be Author: Dave BrosiusAuthored: Thu Jan 7 23:42:56 2016 -0500 Committer: Dave Brosius Committed: Thu Jan 7 23:42:56 2016 -0500 -- src/java/org/apache/cassandra/security/JKSKeyProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ad1adb3/src/java/org/apache/cassandra/security/JKSKeyProvider.java -- diff --git a/src/java/org/apache/cassandra/security/JKSKeyProvider.java b/src/java/org/apache/cassandra/security/JKSKeyProvider.java index af9d515..db7a2b9 100644 --- a/src/java/org/apache/cassandra/security/JKSKeyProvider.java +++ b/src/java/org/apache/cassandra/security/JKSKeyProvider.java @@ -32,7 +32,7 @@ import org.apache.cassandra.config.TransparentDataEncryptionOptions; */ public class JKSKeyProvider implements KeyProvider { -private final Logger logger = LoggerFactory.getLogger(JKSKeyProvider.class); +private static final Logger logger = LoggerFactory.getLogger(JKSKeyProvider.class); static final String PROP_KEYSTORE = "keystore"; static final String PROP_KEYSTORE_PW = "keystore_password"; static final String PROP_KEYSTORE_TYPE = "store_type";
[jira] [Created] (CASSANDRA-10984) Cassandra should not depend on netty-all
James Roper created CASSANDRA-10984: --- Summary: Cassandra should not depend on netty-all Key: CASSANDRA-10984 URL: https://issues.apache.org/jira/browse/CASSANDRA-10984 Project: Cassandra Issue Type: Bug Reporter: James Roper Priority: Minor netty-all is a jar that bundles all the individual netty dependencies for convenience together for people trying out netty to get started quickly. Serious projects like Cassandra should never ever ever use it, since it's a recipe for classpath disasters. To illustrate, I'm running Cassandra embedded in an app, and I get this error: {noformat} [JVM-1] java.lang.NoSuchMethodError: io.netty.util.internal.PlatformDependent.newLongCounter()Lio/netty/util/internal/LongCounter; [JVM-1] at io.netty.buffer.PoolArena.(PoolArena.java:64) ~[netty-buffer-4.0.33.Final.jar:4.0.33.Final] [JVM-1] at io.netty.buffer.PoolArena$HeapArena.(PoolArena.java:593) ~[netty-buffer-4.0.33.Final.jar:4.0.33.Final] [JVM-1] at io.netty.buffer.PooledByteBufAllocator.(PooledByteBufAllocator.java:179) ~[netty-buffer-4.0.33.Final.jar:4.0.33.Final] [JVM-1] at io.netty.buffer.PooledByteBufAllocator.(PooledByteBufAllocator.java:153) ~[netty-buffer-4.0.33.Final.jar:4.0.33.Final] [JVM-1] at io.netty.buffer.PooledByteBufAllocator.(PooledByteBufAllocator.java:145) ~[netty-buffer-4.0.33.Final.jar:4.0.33.Final] [JVM-1] at io.netty.buffer.PooledByteBufAllocator.(PooledByteBufAllocator.java:128) ~[netty-buffer-4.0.33.Final.jar:4.0.33.Final] [JVM-1] at org.apache.cassandra.transport.CBUtil.(CBUtil.java:56) ~[cassandra-all-3.0.0.jar:3.0.0] [JVM-1] at org.apache.cassandra.transport.Server.start(Server.java:134) ~[cassandra-all-3.0.0.jar:3.0.0] {noformat} {{PlatformDependent}} comes from netty-common, of which version 4.0.33 is on the classpath, but it's also provided by netty-all, which has version 4.0.23 brought in by cassandra. By a fluke of classpath ordering, the classloader has loaded the netty buffer classes from netty-buffer 4.0.33, but the PlatformDependent class from netty-all 4.0.23, and these two versions are not binary compatible, hence the linkage error. Essentially to avoid these problems in serious projects, anyone that ever brings in cassandra is going to have to exclude the netty dependency from it, which is error prone, and when you get it wrong, due to the nature of classpath ordering bugs, it might not be till you deploy to production that you actually find out there's a problem. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10838) print 3.0 statistics in sstablemetadata command output
[ https://issues.apache.org/jira/browse/CASSANDRA-10838?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-10838: - Assignee: Shogo Hoshii > print 3.0 statistics in sstablemetadata command output > -- > > Key: CASSANDRA-10838 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10838 > Project: Cassandra > Issue Type: Improvement > Components: Tools >Reporter: Shogo Hoshii >Assignee: Shogo Hoshii >Priority: Minor > Fix For: 3.4 > > Attachments: CASSANDRA-10838.txt, CASSANDRA-10838.txt, > sample_result.txt > > > In CASSANDRA-7159, some statistics were added in 2.1.x, and in version 3.0, > we can print additional statistics. > So I would like to print them in sstablemetadata output. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7464) Replace sstable2json and json2sstable
[ https://issues.apache.org/jira/browse/CASSANDRA-7464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087603#comment-15087603 ] Jonathan Ellis commented on CASSANDRA-7464: --- Chris, when would the "type" of a row not be "row?" Is that how you'd support static columns? Maybe that would be better as its own sub-object rather than a different type of row. > Replace sstable2json and json2sstable > - > > Key: CASSANDRA-7464 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7464 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Assignee: Chris Lohfink >Priority: Minor > Fix For: 3.x > > Attachments: sstable-only.patch > > > Both tools are pretty awful. They are primarily meant for debugging (there is > much more efficient and convenient ways to do import/export data), but their > output manage to be hard to handle both for humans and for tools (especially > as soon as you have modern stuff like composites). > There is value to having tools to export sstable contents into a format that > is easy to manipulate by human and tools for debugging, small hacks and > general tinkering, but sstable2json and json2sstable are not that. > So I propose that we deprecate those tools and consider writing better > replacements. It shouldn't be too hard to come up with an output format that > is more aware of modern concepts like composites, UDTs, -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/2] 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/24630b4a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/24630b4a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/24630b4a Branch: refs/heads/cassandra-3.0 Commit: 24630b4a1fcfe8fc7fdb228a3f87f153905b519f Parents: b477c0f 52d8197 Author: T Jake LucianiAuthored: Thu Jan 7 12:28:00 2016 -0500 Committer: T Jake Luciani Committed: Thu Jan 7 12:28:00 2016 -0500 -- CHANGES.txt | 1 + .../apache/cassandra/metrics/EstimatedHistogramReservoir.java | 5 + 2 files changed, 6 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/24630b4a/CHANGES.txt -- diff --cc CHANGES.txt index 1533619,a26f9e0..7f6b761 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,17 -1,5 +1,18 @@@ -2.2.5 +3.0.3 + * MV should use the maximum timestamp of the primary key (CASSANDRA-10910) + * Fix potential assertion error during compaction (CASSANDRA-10944) + * Fix counting of received sstables in streaming (CASSANDRA-10949) + * Implement hints compression (CASSANDRA-9428) + * Fix potential assertion error when reading static columns (CASSANDRA-10903) + * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711) + * Avoid building PartitionUpdate in toString (CASSANDRA-10897) + * Reduce heap spent when receiving many SSTables (CASSANDRA-10797) + * Add back support for 3rd party auth providers to bulk loader (CASSANDRA-10873) + * Eliminate the dependency on jgrapht for UDT resolution (CASSANDRA-10653) + * (Hadoop) Close Clusters and Sessions in Hadoop Input/Output classes (CASSANDRA-10837) + * Fix sstableloader not working with upper case keyspace name (CASSANDRA-10806) +Merged from 2.2: + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975) * Enable GC logging by default (CASSANDRA-10140) * Optimize pending range computation (CASSANDRA-9258) * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
cassandra git commit: override histogram reservoir getValues to return buckets in correct order
Repository: cassandra Updated Branches: refs/heads/cassandra-2.2 e41fa3419 -> 52d8197d1 override histogram reservoir getValues to return buckets in correct order Patch by Chris Lohfink; reviewed by tjake for CASSANDRA-10975 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/52d8197d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/52d8197d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/52d8197d Branch: refs/heads/cassandra-2.2 Commit: 52d8197d1bba774b2165fc73f9279f237e27d3d4 Parents: e41fa34 Author: Chris LohfinkAuthored: Wed Jan 6 13:43:07 2016 -0600 Committer: T Jake Luciani Committed: Thu Jan 7 12:25:35 2016 -0500 -- CHANGES.txt | 1 + .../apache/cassandra/metrics/EstimatedHistogramReservoir.java | 5 + 2 files changed, 6 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/52d8197d/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 90d990e..a26f9e0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.5 + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975) * Enable GC logging by default (CASSANDRA-10140) * Optimize pending range computation (CASSANDRA-9258) * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902) http://git-wip-us.apache.org/repos/asf/cassandra/blob/52d8197d/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java -- diff --git a/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java index 3658f3a..29baad8 100644 --- a/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java +++ b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java @@ -102,5 +102,10 @@ public class EstimatedHistogramReservoir implements Reservoir { return histogram.rawMean(); } + +@Override +public long[] getValues() { +return histogram.getBuckets(false); +} } }
[4/4] cassandra git commit: Merge branch 'cassandra-3.3' into trunk
Merge branch 'cassandra-3.3' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/31f67c28 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/31f67c28 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/31f67c28 Branch: refs/heads/trunk Commit: 31f67c2898a9d4b310de513737f703dc2514b84c Parents: c9d209b 17e79b7 Author: T Jake LucianiAuthored: Thu Jan 7 12:43:13 2016 -0500 Committer: T Jake Luciani Committed: Thu Jan 7 12:43:13 2016 -0500 -- CHANGES.txt | 5 + .../apache/cassandra/metrics/EstimatedHistogramReservoir.java | 5 + 2 files changed, 10 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/31f67c28/CHANGES.txt --
[1/4] cassandra git commit: override histogram reservoir getValues to return buckets in correct order
Repository: cassandra Updated Branches: refs/heads/trunk c9d209b02 -> 31f67c289 override histogram reservoir getValues to return buckets in correct order Patch by Chris Lohfink; reviewed by tjake for CASSANDRA-10975 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/52d8197d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/52d8197d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/52d8197d Branch: refs/heads/trunk Commit: 52d8197d1bba774b2165fc73f9279f237e27d3d4 Parents: e41fa34 Author: Chris LohfinkAuthored: Wed Jan 6 13:43:07 2016 -0600 Committer: T Jake Luciani Committed: Thu Jan 7 12:25:35 2016 -0500 -- CHANGES.txt | 1 + .../apache/cassandra/metrics/EstimatedHistogramReservoir.java | 5 + 2 files changed, 6 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/52d8197d/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 90d990e..a26f9e0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.5 + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975) * Enable GC logging by default (CASSANDRA-10140) * Optimize pending range computation (CASSANDRA-9258) * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902) http://git-wip-us.apache.org/repos/asf/cassandra/blob/52d8197d/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java -- diff --git a/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java index 3658f3a..29baad8 100644 --- a/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java +++ b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java @@ -102,5 +102,10 @@ public class EstimatedHistogramReservoir implements Reservoir { return histogram.rawMean(); } + +@Override +public long[] getValues() { +return histogram.getBuckets(false); +} } }
[2/4] 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/24630b4a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/24630b4a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/24630b4a Branch: refs/heads/trunk Commit: 24630b4a1fcfe8fc7fdb228a3f87f153905b519f Parents: b477c0f 52d8197 Author: T Jake LucianiAuthored: Thu Jan 7 12:28:00 2016 -0500 Committer: T Jake Luciani Committed: Thu Jan 7 12:28:00 2016 -0500 -- CHANGES.txt | 1 + .../apache/cassandra/metrics/EstimatedHistogramReservoir.java | 5 + 2 files changed, 6 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/24630b4a/CHANGES.txt -- diff --cc CHANGES.txt index 1533619,a26f9e0..7f6b761 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,17 -1,5 +1,18 @@@ -2.2.5 +3.0.3 + * MV should use the maximum timestamp of the primary key (CASSANDRA-10910) + * Fix potential assertion error during compaction (CASSANDRA-10944) + * Fix counting of received sstables in streaming (CASSANDRA-10949) + * Implement hints compression (CASSANDRA-9428) + * Fix potential assertion error when reading static columns (CASSANDRA-10903) + * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711) + * Avoid building PartitionUpdate in toString (CASSANDRA-10897) + * Reduce heap spent when receiving many SSTables (CASSANDRA-10797) + * Add back support for 3rd party auth providers to bulk loader (CASSANDRA-10873) + * Eliminate the dependency on jgrapht for UDT resolution (CASSANDRA-10653) + * (Hadoop) Close Clusters and Sessions in Hadoop Input/Output classes (CASSANDRA-10837) + * Fix sstableloader not working with upper case keyspace name (CASSANDRA-10806) +Merged from 2.2: + * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975) * Enable GC logging by default (CASSANDRA-10140) * Optimize pending range computation (CASSANDRA-9258) * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
[jira] [Updated] (CASSANDRA-10786) Include hash of result set metadata in prepared statement id
[ https://issues.apache.org/jira/browse/CASSANDRA-10786?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-10786: -- Fix Version/s: (was: 3.2) 3.x > Include hash of result set metadata in prepared statement id > > > Key: CASSANDRA-10786 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10786 > Project: Cassandra > Issue Type: Improvement > Components: CQL >Reporter: Olivier Michallat >Priority: Minor > Fix For: 3.x > > > This is a follow-up to CASSANDRA-7910, which was about invalidating a > prepared statement when the table is altered, to force clients to update > their local copy of the metadata. > There's still an issue if multiple clients are connected to the same host. > The first client to execute the query after the cache was invalidated will > receive an UNPREPARED response, re-prepare, and update its local metadata. > But other clients might miss it entirely (the MD5 hasn't changed), and they > will keep using their old metadata. > One solution that was suggested is to include a hash of the result set > metadata in the prepared statement id. The old id would become invalid, > forcing all clients to reprepare. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10975) Histogram buckets exposed in jmx are sorted by count
[ https://issues.apache.org/jira/browse/CASSANDRA-10975?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] T Jake Luciani updated CASSANDRA-10975: --- Fix Version/s: (was: 3.2) 3.3 > Histogram buckets exposed in jmx are sorted by count > > > Key: CASSANDRA-10975 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10975 > Project: Cassandra > Issue Type: Bug > Components: Observability >Reporter: Chris Lohfink >Assignee: Chris Lohfink > Fix For: 2.2.5, 3.0.3, 3.3 > > Attachments: 10975-2.2.patch > > > The estimated histogram snapshot lets its parent provide the getValues > implementation which sorts the bucket array: > https://github.com/dropwizard/metrics/blob/3.1-maintenance/metrics-core/src/main/java/com/codahale/metrics/UniformSnapshot.java#L25 > making it hard to determine what count belonged to what bucket. Along with > removal of the pre 2.2 deprecated metrics this makes it nearly impossible to > track latencies over time. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10540) RangeAwareCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-10540: -- Fix Version/s: (was: 3.2) 3.x > RangeAwareCompaction > > > Key: CASSANDRA-10540 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10540 > Project: Cassandra > Issue Type: New Feature >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson > Fix For: 3.x > > > Broken out from CASSANDRA-6696, we should split sstables based on ranges > during compaction. > Requirements; > * dont create tiny sstables - keep them bunched together until a single vnode > is big enough (configurable how big that is) > * make it possible to run existing compaction strategies on the per-range > sstables > We should probably add a global compaction strategy parameter that states > whether this should be enabled or not. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10855) Use Caffeine (W-TinyLFU) for on-heap caches
[ https://issues.apache.org/jira/browse/CASSANDRA-10855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087617#comment-15087617 ] Jeremy Hanna commented on CASSANDRA-10855: -- As part of testing, should we also fix cstar perf relating to cache to no longer be uniform access distribution? It seems like for this and future tests, that would give a more real-world result. > Use Caffeine (W-TinyLFU) for on-heap caches > --- > > Key: CASSANDRA-10855 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10855 > Project: Cassandra > Issue Type: Improvement >Reporter: Ben Manes > Labels: performance > > Cassandra currently uses > [ConcurrentLinkedHashMap|https://code.google.com/p/concurrentlinkedhashmap] > for performance critical caches (key, counter) and Guava's cache for > non-critical (auth, metrics, security). All of these usages have been > replaced by [Caffeine|https://github.com/ben-manes/caffeine], written by the > author of the previously mentioned libraries. > The primary incentive is to switch from LRU policy to W-TinyLFU, which > provides [near optimal|https://github.com/ben-manes/caffeine/wiki/Efficiency] > hit rates. It performs particularly well in database and search traces, is > scan resistant, and as adds a very small time/space overhead to LRU. > Secondarily, Guava's caches never obtained similar > [performance|https://github.com/ben-manes/caffeine/wiki/Benchmarks] to CLHM > due to some optimizations not being ported over. This change results in > faster reads and not creating garbage as a side-effect. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10978) MV add_write_survey_node_after_mv_test is failing on trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-10978?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian updated CASSANDRA-10978: --- Assignee: Joel Knighton Reviewer: Carl Yeksigian Fix Version/s: (was: 3.2) 3.0.x Looks related to CASSANDRA-10621; [~jkni] can you take a look at this failure? > MV add_write_survey_node_after_mv_test is failing on trunk > -- > > Key: CASSANDRA-10978 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10978 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Alan Boudreault >Assignee: Joel Knighton > Labels: dtest > Fix For: 3.0.x > > > http://cassci.datastax.com/job/trunk_dtest/893/testReport/materialized_views_test/TestMaterializedViews/add_write_survey_node_after_mv_test/ > {code} > == > ERROR: add_write_survey_node_after_mv_test > (materialized_views_test.TestMaterializedViews) > -- > Traceback (most recent call last): > File "/home/aboudreault/git/cstar/cassandra-dtest/dtest.py", line 558, in > tearDown > raise AssertionError('Unexpected error in %s node log: %s' % (node.name, > errors)) > AssertionError: Unexpected error in node4 node log: ['ERROR [main] 2016-01-06 > 17:03:41,614 MigrationManager.java:164 - Migration task failed to > complete\nERROR [main] 2016-01-06 17:03:42,614 MigrationManager.java:164 - > Migration task failed to complete'] > >> begin captured logging << > dtest: DEBUG: cluster ccm directory: /tmp/dtest-14GpwW > dtest: DEBUG: removing ccm cluster test at: /tmp/dtest-14GpwW > dtest: DEBUG: clearing ssl stores from [/tmp/dtest-14GpwW] directory > - >> end captured logging << - > -- > Ran 1 test in 85.369s > FAILED (errors=1) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[3/3] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3
Merge branch 'cassandra-3.0' into cassandra-3.3 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/17e79b7a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/17e79b7a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/17e79b7a Branch: refs/heads/cassandra-3.3 Commit: 17e79b7acde24b159fdf820c6b8b5f5c45743e59 Parents: caedc8f 24630b4 Author: T Jake LucianiAuthored: Thu Jan 7 12:42:37 2016 -0500 Committer: T Jake Luciani Committed: Thu Jan 7 12:42:37 2016 -0500 -- CHANGES.txt | 5 + .../apache/cassandra/metrics/EstimatedHistogramReservoir.java | 5 + 2 files changed, 10 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/17e79b7a/CHANGES.txt --
[jira] [Updated] (CASSANDRA-10924) Pass base table's metadata to Index.validateOptions
[ https://issues.apache.org/jira/browse/CASSANDRA-10924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] T Jake Luciani updated CASSANDRA-10924: --- Fix Version/s: (was: 3.0.3) (was: 3.2) > Pass base table's metadata to Index.validateOptions > --- > > Key: CASSANDRA-10924 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10924 > Project: Cassandra > Issue Type: Improvement > Components: CQL, Local Write-Read Paths >Reporter: Andrés de la Peña >Assignee: Andrés de la Peña >Priority: Minor > Labels: 2i, index, validation > Fix For: 3.0.x, 3.x > > Attachments: CASSANDRA-10924-v0.diff > > > Some custom index implementations require the base table's metadata to > validate their creation options. For example, the options of these > implementations can contain information about which base table's columns are > going to be indexed and how, so the implementation needs to know the > existence and the type of the columns to be indexed to properly validate. > The attached patch proposes to add base table's {{CFMetaData}} to Index' > optional static method to validate the custom index options: > {{public static MapvalidateOptions(CFMetaData cfm, > Map options);}} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10961) Not enough bytes error when add nodes to cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-10961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088801#comment-15088801 ] xiaost commented on CASSANDRA-10961: It works! over 500GB streaming data is received without any error. hope release in 2.2.5 soon thanks [~pauloricardomg] > Not enough bytes error when add nodes to cluster > > > Key: CASSANDRA-10961 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10961 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: xiaost >Assignee: Paulo Motta > Attachments: apache-cassandra-2.2.4-SNAPSHOT.jar, debug.1.log, > debug.logs.zip, netstats.1.log > > > we got the same problem all the time when we add nodes to cluster. > netstats: > on HostA > {noformat} > /la-38395-big-Data.db 14792091851/14792091851 bytes(100%) sent to idx:0/HostB > {noformat} > on HostB > {noformat} > tmp-la-4-big-Data.db 2667087450/14792091851 bytes(18%) received from > idx:0/HostA > {noformat} > After a while, Error on HostB > {noformat} > WARN [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:644 - > [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Retrying for following error > java.lang.IllegalArgumentException: Not enough bytes > at > org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) > ~[guava-16.0.jar:na] > at > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:173) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:58) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261) > [apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal] > ERROR [Thread-28] 2016-01-02 12:08:14,737 CassandraDaemon.java:185 - > Exception in thread Thread[Thread-28,5,main] > java.lang.RuntimeException: java.lang.InterruptedException > at com.google.common.base.Throwables.propagate(Throwables.java:160) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_66-internal] > Caused by: java.lang.InterruptedException: null > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220) > ~[na:1.8.0_66-internal] > at > java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335) > ~[na:1.8.0_66-internal] > at > java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:350) > ~[na:1.8.0_66-internal] > at > org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:176) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at >
[jira] [Commented] (CASSANDRA-10961) Not enough bytes error when add nodes to cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-10961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088843#comment-15088843 ] Juliano Vidal commented on CASSANDRA-10961: --- Shall we use this snapshot jar while waiting.for 2.2.5? When is it going to be released? > Not enough bytes error when add nodes to cluster > > > Key: CASSANDRA-10961 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10961 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: xiaost >Assignee: Paulo Motta > Attachments: apache-cassandra-2.2.4-SNAPSHOT.jar, debug.1.log, > debug.logs.zip, netstats.1.log > > > we got the same problem all the time when we add nodes to cluster. > netstats: > on HostA > {noformat} > /la-38395-big-Data.db 14792091851/14792091851 bytes(100%) sent to idx:0/HostB > {noformat} > on HostB > {noformat} > tmp-la-4-big-Data.db 2667087450/14792091851 bytes(18%) received from > idx:0/HostA > {noformat} > After a while, Error on HostB > {noformat} > WARN [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:644 - > [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Retrying for following error > java.lang.IllegalArgumentException: Not enough bytes > at > org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) > ~[guava-16.0.jar:na] > at > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:173) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:58) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261) > [apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal] > ERROR [Thread-28] 2016-01-02 12:08:14,737 CassandraDaemon.java:185 - > Exception in thread Thread[Thread-28,5,main] > java.lang.RuntimeException: java.lang.InterruptedException > at com.google.common.base.Throwables.propagate(Throwables.java:160) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_66-internal] > Caused by: java.lang.InterruptedException: null > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220) > ~[na:1.8.0_66-internal] > at > java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335) > ~[na:1.8.0_66-internal] > at > java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:350) > ~[na:1.8.0_66-internal] > at > org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:176) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at >
[5/9] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.3
Merge branch 'cassandra-3.0' into cassandra-3.3 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/53969d52 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/53969d52 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/53969d52 Branch: refs/heads/trunk Commit: 53969d52fc3ceb7952b6fc6d3b7e004033fa3f9b Parents: 15cfeb7 b477c0f Author: Sylvain LebresneAuthored: Thu Jan 7 16:24:48 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 16:24:48 2016 +0100 -- NEWS.txt | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/53969d52/NEWS.txt -- diff --cc NEWS.txt index 20ab202,89fc4a7..48b7996 --- a/NEWS.txt +++ b/NEWS.txt @@@ -29,19 -21,13 +29,24 @@@ New feature - Hinted handoff now supports compression. Reference cassandra.yaml:hints_compression. Note: hints compression is currently disabled by default. + Upgrading + - + - Nothing specific to 3.0.3 but please see previous versions upgrading section, + especially if you are upgrading from 2.2. + -3.0.1 +Upgrading +- + - The compression ratio metrics computation has been modified to be more accurate. + - Running Cassandra as root is prevented by default. + - JVM options are moved from cassandra-env.(sh|ps1) to jvm.options file + +Deprecation +--- + - The Thrift API is deprecated and will be removed in Cassandra 4.0. + + +3.1 = Upgrading
[9/9] cassandra git commit: Add notice for CASSANDRA-10880 in NEWS file
Add notice for CASSANDRA-10880 in NEWS file Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/aeaa5016 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/aeaa5016 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/aeaa5016 Branch: refs/heads/cassandra-3.2 Commit: aeaa5016e0a71d2ad9b0f4b4b07463820f652a87 Parents: 85b8d02 Author: Sylvain LebresneAuthored: Thu Jan 7 16:22:34 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 16:26:22 2016 +0100 -- NEWS.txt | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/aeaa5016/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 3d468b6..c995d89 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -29,6 +29,11 @@ New features - Hinted handoff now supports compression. Reference cassandra.yaml:hints_compression. Note: hints compression is currently disabled by default. +Upgrading +- +- Nothing specific to 3.2 but please see previous versions upgrading section, + especially if you are upgrading from 2.2. + Upgrading - @@ -85,6 +90,9 @@ New features Upgrading - + - Clients must use the native protocol version 3 when upgrading from 2.2.X as + the native protocol version 4 is not compatible between 2.2.X and 3.Y. See + https://www.mail-archive.com/user@cassandra.apache.org/msg45381.html for details. - A new argument of type InetAdress has been added to IAuthenticator::newSaslNegotiator, representing the IP address of the client attempting authentication. It will be a breaking change for any custom implementations.
[1/3] cassandra git commit: Fix bad merge
Repository: cassandra Updated Branches: refs/heads/cassandra-3.3 f7843d26d -> caedc8fdc refs/heads/trunk f13bebe52 -> c9d209b02 Fix bad merge Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/caedc8fd Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/caedc8fd Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/caedc8fd Branch: refs/heads/cassandra-3.3 Commit: caedc8fdcf1a449531e03c96b82db629809e10ab Parents: f7843d2 Author: Carl YeksigianAuthored: Thu Jan 7 10:37:21 2016 -0500 Committer: Carl Yeksigian Committed: Thu Jan 7 10:37:21 2016 -0500 -- .../apache/cassandra/db/view/TemporalRow.java | 65 ++-- .../org/apache/cassandra/cql3/ViewTest.java | 30 + 2 files changed, 76 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/caedc8fd/src/java/org/apache/cassandra/db/view/TemporalRow.java -- diff --git a/src/java/org/apache/cassandra/db/view/TemporalRow.java b/src/java/org/apache/cassandra/db/view/TemporalRow.java index 8898857..8ee310d 100644 --- a/src/java/org/apache/cassandra/db/view/TemporalRow.java +++ b/src/java/org/apache/cassandra/db/view/TemporalRow.java @@ -22,6 +22,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -279,9 +280,7 @@ public class TemporalRow this.nowInSec = nowInSec; LivenessInfo liveness = row.primaryKeyLivenessInfo(); -this.viewClusteringLocalDeletionTime = minValueIfSet(viewClusteringLocalDeletionTime, row.deletion().time().localDeletionTime(), NO_DELETION_TIME); -this.viewClusteringTimestamp = minValueIfSet(viewClusteringTimestamp, liveness.timestamp(), NO_TIMESTAMP); -this.viewClusteringTtl = minValueIfSet(viewClusteringTtl, liveness.ttl(), NO_TTL); +updateLiveness(liveness.ttl(), liveness.timestamp(), row.deletion().time().localDeletionTime()); List clusteringDefs = baseCfs.metadata.clusteringColumns(); clusteringColumns = new HashMap<>(); @@ -295,6 +294,31 @@ public class TemporalRow } } +/* + * PK ts:5, ttl:1, deletion: 2 + * Col ts:4, ttl:2, deletion: 3 + * + * TTL use min, since it expires at the lowest time which we are expiring. If we have the above values, we + * would want to return 1, since the base row expires in 1 second. + * + * Timestamp uses max, as this is the time that the row has been written to the view. See CASSANDRA-10910. + * + * Local Deletion Time should use max, as this deletion will cover all previous values written. + */ +@SuppressWarnings("unchecked") +private void updateLiveness(int ttl, long timestamp, int localDeletionTime) +{ +// We are returning whichever is higher from valueIfSet +// Natural order will return the max: 1.compareTo(2) < 0, so 2 is returned +// Reverse order will return the min: 1.compareTo(2) > 0, so 1 is returned +final Comparator max = Comparator.naturalOrder(); +final Comparator min = Comparator.reverseOrder(); + +this.viewClusteringTtl = valueIfSet(viewClusteringTtl, ttl, NO_TTL, min); +this.viewClusteringTimestamp = valueIfSet(viewClusteringTimestamp, timestamp, NO_TIMESTAMP, max); +this.viewClusteringLocalDeletionTime = valueIfSet(viewClusteringLocalDeletionTime, localDeletionTime, NO_DELETION_TIME, max); +} + @Override public String toString() { @@ -351,30 +375,33 @@ public class TemporalRow // If this column is part of the view's primary keys if (viewPrimaryKey.contains(identifier)) { -this.viewClusteringTtl = minValueIfSet(this.viewClusteringTtl, ttl, NO_TTL); -this.viewClusteringTimestamp = minValueIfSet(this.viewClusteringTimestamp, timestamp, NO_TIMESTAMP); -this.viewClusteringLocalDeletionTime = minValueIfSet(this.viewClusteringLocalDeletionTime, localDeletionTime, NO_DELETION_TIME); +updateLiveness(ttl, timestamp, localDeletionTime); } innerMap.get(cellPath).setVersion(new TemporalCell(value, timestamp, ttl, localDeletionTime, isNew)); } -private static int minValueIfSet(int existing, int update, int defaultValue) -{ -if (existing == defaultValue) -return update; -if (update == defaultValue) -return existing; -return Math.min(existing, update); -} - -private static long minValueIfSet(long existing, long update, long
[3/3] cassandra git commit: Merge branch 'cassandra-3.3' into trunk
Merge branch 'cassandra-3.3' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c9d209b0 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c9d209b0 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c9d209b0 Branch: refs/heads/trunk Commit: c9d209b02bb47a3ca2b7c13b110700997c01f8d8 Parents: f13bebe caedc8f Author: Carl YeksigianAuthored: Thu Jan 7 10:37:36 2016 -0500 Committer: Carl Yeksigian Committed: Thu Jan 7 10:37:36 2016 -0500 -- .../apache/cassandra/db/view/TemporalRow.java | 65 ++-- .../org/apache/cassandra/cql3/ViewTest.java | 30 + 2 files changed, 76 insertions(+), 19 deletions(-) --
[jira] [Commented] (CASSANDRA-10855) Use Caffeine (W-TinyLFU) for on-heap caches
[ https://issues.apache.org/jira/browse/CASSANDRA-10855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087621#comment-15087621 ] Jonathan Ellis commented on CASSANDRA-10855: (cstar is back.) > Use Caffeine (W-TinyLFU) for on-heap caches > --- > > Key: CASSANDRA-10855 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10855 > Project: Cassandra > Issue Type: Improvement >Reporter: Ben Manes > Labels: performance > > Cassandra currently uses > [ConcurrentLinkedHashMap|https://code.google.com/p/concurrentlinkedhashmap] > for performance critical caches (key, counter) and Guava's cache for > non-critical (auth, metrics, security). All of these usages have been > replaced by [Caffeine|https://github.com/ben-manes/caffeine], written by the > author of the previously mentioned libraries. > The primary incentive is to switch from LRU policy to W-TinyLFU, which > provides [near optimal|https://github.com/ben-manes/caffeine/wiki/Efficiency] > hit rates. It performs particularly well in database and search traces, is > scan resistant, and as adds a very small time/space overhead to LRU. > Secondarily, Guava's caches never obtained similar > [performance|https://github.com/ben-manes/caffeine/wiki/Benchmarks] to CLHM > due to some optimizations not being ported over. This change results in > faster reads and not creating garbage as a side-effect. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10961) Not enough bytes error when add nodes to cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-10961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088039#comment-15088039 ] Paulo Motta commented on CASSANDRA-10961: - I might have found the culprit on {{CompressedStreamReader}}, in this snippet: {noformat} int sectionLength = (int) (section.right - section.left); {noformat} This might cause sections larger than 2.14GB to have negative length and be skipped, what may break deserealization of next sections. I also reverted CASSANDRA-10005, since interrupting a thread waiting on a {{SocketChannel}} will cause the channel to be closed what will make retry fail. Furthermore I added more debugging statements, so in case this is not the problem we will have more elements to investigate further. If that's the case, please attach new debug logs in the next run. [~xiaost] I attached a new [jar|https://issues.apache.org/jira/secure/attachment/12781053/apache-cassandra-2.2.4-SNAPSHOT.jar] that you can test, and in case you want to build your own you can do it from this [branch|https://github.com/pauloricardomg/cassandra/tree/2.2-10961] > Not enough bytes error when add nodes to cluster > > > Key: CASSANDRA-10961 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10961 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: xiaost >Assignee: Paulo Motta > Attachments: apache-cassandra-2.2.4-SNAPSHOT.jar, debug.1.log, > debug.logs.zip, netstats.1.log > > > we got the same problem all the time when we add nodes to cluster. > netstats: > on HostA > {noformat} > /la-38395-big-Data.db 14792091851/14792091851 bytes(100%) sent to idx:0/HostB > {noformat} > on HostB > {noformat} > tmp-la-4-big-Data.db 2667087450/14792091851 bytes(18%) received from > idx:0/HostA > {noformat} > After a while, Error on HostB > {noformat} > WARN [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:644 - > [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Retrying for following error > java.lang.IllegalArgumentException: Not enough bytes > at > org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) > ~[guava-16.0.jar:na] > at > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:173) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:58) > [apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261) > [apache-cassandra-2.2.4.jar:2.2.4] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal] > ERROR [Thread-28] 2016-01-02 12:08:14,737 CassandraDaemon.java:185 - > Exception in thread Thread[Thread-28,5,main] > java.lang.RuntimeException: java.lang.InterruptedException > at com.google.common.base.Throwables.propagate(Throwables.java:160) > ~[guava-16.0.jar:na] > at >
[jira] [Created] (CASSANDRA-10982) Put gc.log in -Dcassandra.logdir location by default
Philip Thompson created CASSANDRA-10982: --- Summary: Put gc.log in -Dcassandra.logdir location by default Key: CASSANDRA-10982 URL: https://issues.apache.org/jira/browse/CASSANDRA-10982 Project: Cassandra Issue Type: Bug Components: Configuration Reporter: Philip Thompson Fix For: 2.2.x, 3.0.x, 3.x CASSANDRA-10140 turned on gc.log by default, and set it's location to CASSANDRA_HOME/logs. It would be much better UX if when -Dcassandra.logdir was set, that it was used instead. This way users don't have to separately configure gc.log from the other log files. Additionally, I don't think 10140 made it to trunk, as grepping for `loggc` there shows me nothing in cassandra-env.sh as of 31f67c289. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10961) Not enough bytes error when add nodes to cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-10961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088039#comment-15088039 ] Paulo Motta edited comment on CASSANDRA-10961 at 1/7/16 8:25 PM: - I might have found the culprit on {{CompressedStreamReader}}, in this snippet: {noformat} int sectionLength = (int) (section.right - section.left); {noformat} This might cause sections larger than 2.14GB to have negative length due to {{int}} overflow and be skipped, what may break deserealization of next sections. I changed to use {{long}} type instead. I also reverted CASSANDRA-10005, since interrupting a thread waiting on a {{SocketChannel}} will cause the channel to be closed what will make retry fail. Furthermore I added more debugging statements, so in case this is not the problem we will have more elements to investigate further. If that's the case, please attach new debug logs in the next run. [~xiaost] I attached a new [jar|https://issues.apache.org/jira/secure/attachment/12781053/apache-cassandra-2.2.4-SNAPSHOT.jar] that you can test, and in case you want to build your own you can do it from this [branch|https://github.com/pauloricardomg/cassandra/tree/2.2-10961] was (Author: pauloricardomg): I might have found the culprit on {{CompressedStreamReader}}, in this snippet: {noformat} int sectionLength = (int) (section.right - section.left); {noformat} This might cause sections larger than 2.14GB to have negative length and be skipped, what may break deserealization of next sections. I also reverted CASSANDRA-10005, since interrupting a thread waiting on a {{SocketChannel}} will cause the channel to be closed what will make retry fail. Furthermore I added more debugging statements, so in case this is not the problem we will have more elements to investigate further. If that's the case, please attach new debug logs in the next run. [~xiaost] I attached a new [jar|https://issues.apache.org/jira/secure/attachment/12781053/apache-cassandra-2.2.4-SNAPSHOT.jar] that you can test, and in case you want to build your own you can do it from this [branch|https://github.com/pauloricardomg/cassandra/tree/2.2-10961] > Not enough bytes error when add nodes to cluster > > > Key: CASSANDRA-10961 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10961 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: xiaost >Assignee: Paulo Motta > Attachments: apache-cassandra-2.2.4-SNAPSHOT.jar, debug.1.log, > debug.logs.zip, netstats.1.log > > > we got the same problem all the time when we add nodes to cluster. > netstats: > on HostA > {noformat} > /la-38395-big-Data.db 14792091851/14792091851 bytes(100%) sent to idx:0/HostB > {noformat} > on HostB > {noformat} > tmp-la-4-big-Data.db 2667087450/14792091851 bytes(18%) received from > idx:0/HostA > {noformat} > After a while, Error on HostB > {noformat} > WARN [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:644 - > [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Retrying for following error > java.lang.IllegalArgumentException: Not enough bytes > at > org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) > ~[guava-16.0.jar:na] > at > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) > ~[guava-16.0.jar:na] > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at > org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:173) > ~[apache-cassandra-2.2.4.jar:2.2.4] > at >
[jira] [Commented] (CASSANDRA-10688) Stack overflow from SSTableReader$InstanceTidier.runOnClose in Leak Detector
[ https://issues.apache.org/jira/browse/CASSANDRA-10688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088089#comment-15088089 ] Ariel Weisberg commented on CASSANDRA-10688: Updated, tests running now. > Stack overflow from SSTableReader$InstanceTidier.runOnClose in Leak Detector > > > Key: CASSANDRA-10688 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10688 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths, Testing >Reporter: Jeremiah Jordan >Assignee: Ariel Weisberg > Fix For: 3.0.x > > > Running some tests against cassandra-3.0 > 9fc957cf3097e54ccd72e51b2d0650dc3e83eae0 > The tests are just running cassandra-stress write and read while adding and > removing nodes from the cluster. After the test runs when I go back through > logs I find the following Stackoverflow fairly often: > ERROR [Strong-Reference-Leak-Detector:1] 2015-11-11 00:04:10,638 > Ref.java:413 - Stackoverflow [private java.lang.Runnable > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier.runOnClose, > final java.lang.Runnable > org.apache.cassandra.io.sstable.format.SSTableReader$DropPageCache.andThen, > final org.apache.cassandra.cache.InstrumentingCache > org.apache.cassandra.io.sstable.SSTableRewriter$InvalidateKeys.cache, private > final org.apache.cassandra.cache.ICache > org.apache.cassandra.cache.InstrumentingCache.map, private final > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap > org.apache.cassandra.cache.ConcurrentLinkedHashCache.map, final > com.googlecode.concurrentlinkedhashmap.LinkedDeque > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.evictionDeque, > com.googlecode.concurrentlinkedhashmap.Linked > com.googlecode.concurrentlinkedhashmap.LinkedDeque.first, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > ... (repeated a whole bunch more) > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > final java.lang.Object > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.key, > public final byte[] org.apache.cassandra.cache.KeyCacheKey.key -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8072) Exception during startup: Unable to gossip with any seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-8072?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087755#comment-15087755 ] Russ Hatch commented on CASSANDRA-8072: --- [~Stefania] It's been a long while since I tested, but I vaguely remember that a long enough sleep would prevent the issue from happening when I was working to repro it consistently. > Exception during startup: Unable to gossip with any seeds > - > > Key: CASSANDRA-8072 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8072 > Project: Cassandra > Issue Type: Bug > Components: Lifecycle >Reporter: Ryan Springer >Assignee: Stefania > Fix For: 2.1.x > > Attachments: cas-dev-dt-01-uw1-cassandra-seed01_logs.tar.bz2, > cas-dev-dt-01-uw1-cassandra-seed02_logs.tar.bz2, > cas-dev-dt-01-uw1-cassandra02_logs.tar.bz2, > casandra-system-log-with-assert-patch.log, screenshot-1.png, > trace_logs.tar.bz2 > > > When Opscenter 4.1.4 or 5.0.1 tries to provision a 2-node DSC 2.0.10 cluster > in either ec2 or locally, an error occurs sometimes with one of the nodes > refusing to start C*. The error in the /var/log/cassandra/system.log is: > ERROR [main] 2014-10-06 15:54:52,292 CassandraDaemon.java (line 513) > Exception encountered during startup > java.lang.RuntimeException: Unable to gossip with any seeds > at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1200) > at > org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:444) > at > org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:655) > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:609) > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:502) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585) > INFO [StorageServiceShutdownHook] 2014-10-06 15:54:52,326 Gossiper.java > (line 1279) Announcing shutdown > INFO [StorageServiceShutdownHook] 2014-10-06 15:54:54,326 > MessagingService.java (line 701) Waiting for messaging service to quiesce > INFO [ACCEPT-localhost/127.0.0.1] 2014-10-06 15:54:54,327 > MessagingService.java (line 941) MessagingService has terminated the accept() > thread > This errors does not always occur when provisioning a 2-node cluster, but > probably around half of the time on only one of the nodes. I haven't been > able to reproduce this error with DSC 2.0.9, and there have been no code or > definition file changes in Opscenter. > I can reproduce locally with the above steps.  I'm happy to test any proposed > fixes since I'm the only person able to reproduce reliably so far. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-10977) MV view_tombstone_test is failing on trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-10977?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian resolved CASSANDRA-10977. Resolution: Fixed This was caused by a merge wiping out the changes in CASSANDRA-10910. The [latest run|http://cassci.datastax.com/job/trunk_dtest/896/testReport/materialized_views_test/TestMaterializedViews/view_tombstone_test/] shows this has been fixed. > MV view_tombstone_test is failing on trunk > -- > > Key: CASSANDRA-10977 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10977 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Alan Boudreault > Labels: dtest > Fix For: 3.x > > > http://cassci.datastax.com/job/trunk_dtest/893/testReport/materialized_views_test/TestMaterializedViews/view_tombstone_test/ > {code} > == > FAIL: view_tombstone_test (materialized_views_test.TestMaterializedViews) > -- > Traceback (most recent call last): > File > "/home/aboudreault/git/cstar/cassandra-dtest/materialized_views_test.py", > line 735, in view_tombstone_test > assert_none(session, "SELECT * FROM t_by_v WHERE v = 1") > File "/home/aboudreault/git/cstar/cassandra-dtest/assertions.py", line 44, > in assert_none > assert list_res == [], "Expected nothing from %s, but got %s" % (query, > list_res) > AssertionError: Expected nothing from SELECT * FROM t_by_v WHERE v = 1, but > got [[1, 1, u'b', None]] > >> begin captured logging << > dtest: DEBUG: cluster ccm directory: /tmp/dtest-MFSCKQ > - >> end captured logging << - > -- > Ran 1 test in 27.986s > FAILED (failures=1) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10550) NPE on null 'highestSelectivityIndex()'
[ https://issues.apache.org/jira/browse/CASSANDRA-10550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-10550: Fix Version/s: (was: 2.1.x) 2.1.12 > NPE on null 'highestSelectivityIndex()' > --- > > Key: CASSANDRA-10550 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10550 > Project: Cassandra > Issue Type: Bug >Reporter: Berenguer Blasi >Assignee: Berenguer Blasi > Fix For: 2.1.12 > > Attachments: cassandra-2.1-10550.txt > > > {{org.apache.cassandra.db.index.SecondaryIndexSearcher.highestSelectivityIndex()}} > might return 'null' which makes > {{org.apache.cassandra.service.StorageProxy.estimateResultRowsPerRange()}} > NPE on some custom index implementations. -- 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=15087843#comment-15087843 ] Joshua McKenzie commented on CASSANDRA-8844: Updated design doc w/a refinement on the "separate CDC log vs. CommitLog". General and details on page 2 and 3. > 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 > latency, the following could supplement the approach outlined above > - Instead
[jira] [Commented] (CASSANDRA-10951) Fix ReadCommandTest
[ https://issues.apache.org/jira/browse/CASSANDRA-10951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087861#comment-15087861 ] Yuki Morishita commented on CASSANDRA-10951: +1 for Branimir's patch. > Fix ReadCommandTest > --- > > Key: CASSANDRA-10951 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10951 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths, Testing >Reporter: Yuki Morishita >Assignee: Branimir Lambov >Priority: Blocker > Fix For: 3.2 > > > [ReadCommandTest is > failing|http://cassci.datastax.com/view/trunk/job/trunk_testall/641/testReport/org.apache.cassandra.db/ReadCommandTest/history/] > since CASSANDRA-9975 was merged to trunk. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10688) Stack overflow from SSTableReader$InstanceTidier.runOnClose in Leak Detector
[ https://issues.apache.org/jira/browse/CASSANDRA-10688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087944#comment-15087944 ] Ariel Weisberg commented on CASSANDRA-10688: That's not what I am seeing when I run this test case? Am I missing something? {code} @Test public void testArray() { int entryCount = 500; final Object objects[] = new Object[entryCount]; for (int i = 0; i < entryCount; i++) objects[i] = new Object(); RefCounted.Tidy tidier = new RefCounted.Tidy() { Object ref = objects; @Override public void tidy() throws Exception { } @Override public String name() { return "42"; } }; Ref ref = new Ref(new AtomicReference
[jira] [Commented] (CASSANDRA-7464) Replace sstable2json and json2sstable
[ https://issues.apache.org/jira/browse/CASSANDRA-7464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087789#comment-15087789 ] Chris Lohfink commented on CASSANDRA-7464: -- I am good with no json2sstable, its a little non trivial to write. I could change to something like: ./bin/sstableexport json and add support for few other formats. > Replace sstable2json and json2sstable > - > > Key: CASSANDRA-7464 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7464 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Assignee: Chris Lohfink >Priority: Minor > Fix For: 3.x > > Attachments: sstable-only.patch > > > Both tools are pretty awful. They are primarily meant for debugging (there is > much more efficient and convenient ways to do import/export data), but their > output manage to be hard to handle both for humans and for tools (especially > as soon as you have modern stuff like composites). > There is value to having tools to export sstable contents into a format that > is easy to manipulate by human and tools for debugging, small hacks and > general tinkering, but sstable2json and json2sstable are not that. > So I propose that we deprecate those tools and consider writing better > replacements. It shouldn't be too hard to come up with an output format that > is more aware of modern concepts like composites, UDTs, -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10761) Possible regression of CASSANDRA-9201
[ https://issues.apache.org/jira/browse/CASSANDRA-10761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-10761: Fix Version/s: (was: 2.2.x) 2.2.5 > Possible regression of CASSANDRA-9201 > - > > Key: CASSANDRA-10761 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10761 > Project: Cassandra > Issue Type: Sub-task > Components: Distributed Metadata >Reporter: Philip Thompson >Assignee: Sam Tunnicliffe > Fix For: 2.2.5, 3.0.1, 3.1 > > Attachments: 10761-logs.tar.gz > > > Some dtests like > {{consistency_test.TestAccuracy.test_network_topology_strategy_each_quorum_counters}} > are failing with the follow auth related assertion exception > {code} > [node6 ERROR] java.lang.AssertionError: > org.apache.cassandra.exceptions.InvalidRequestException: unconfigured table > roles > at > org.apache.cassandra.auth.CassandraRoleManager.prepare(CassandraRoleManager.java:450) > at > org.apache.cassandra.auth.CassandraRoleManager.setup(CassandraRoleManager.java:144) > at > org.apache.cassandra.service.StorageService.doAuthSetup(StorageService.java:1036) > at > org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:984) > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:708) > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:579) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:345) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:561) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:689) > Caused by: org.apache.cassandra.exceptions.InvalidRequestException: > unconfigured table roles > at > org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily(ThriftValidation.java:114) > at > org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:757) > at > org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:752) > at > org.apache.cassandra.auth.CassandraRoleManager.prepare(CassandraRoleManager.java:446) > ... 8 more > {code} > This looks very similar to CASSANDRA-9201. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10977) MV view_tombstone_test is failing on trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-10977?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-10977: -- Fix Version/s: (was: 3.2) 3.x > MV view_tombstone_test is failing on trunk > -- > > Key: CASSANDRA-10977 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10977 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Alan Boudreault > Labels: dtest > Fix For: 3.x > > > http://cassci.datastax.com/job/trunk_dtest/893/testReport/materialized_views_test/TestMaterializedViews/view_tombstone_test/ > {code} > == > FAIL: view_tombstone_test (materialized_views_test.TestMaterializedViews) > -- > Traceback (most recent call last): > File > "/home/aboudreault/git/cstar/cassandra-dtest/materialized_views_test.py", > line 735, in view_tombstone_test > assert_none(session, "SELECT * FROM t_by_v WHERE v = 1") > File "/home/aboudreault/git/cstar/cassandra-dtest/assertions.py", line 44, > in assert_none > assert list_res == [], "Expected nothing from %s, but got %s" % (query, > list_res) > AssertionError: Expected nothing from SELECT * FROM t_by_v WHERE v = 1, but > got [[1, 1, u'b', None]] > >> begin captured logging << > dtest: DEBUG: cluster ccm directory: /tmp/dtest-MFSCKQ > - >> end captured logging << - > -- > Ran 1 test in 27.986s > FAILED (failures=1) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7464) Replace sstable2json and json2sstable
[ https://issues.apache.org/jira/browse/CASSANDRA-7464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087785#comment-15087785 ] Chris Lohfink commented on CASSANDRA-7464: -- A row can be a static row, also a range tombstone. > Replace sstable2json and json2sstable > - > > Key: CASSANDRA-7464 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7464 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Assignee: Chris Lohfink >Priority: Minor > Fix For: 3.x > > Attachments: sstable-only.patch > > > Both tools are pretty awful. They are primarily meant for debugging (there is > much more efficient and convenient ways to do import/export data), but their > output manage to be hard to handle both for humans and for tools (especially > as soon as you have modern stuff like composites). > There is value to having tools to export sstable contents into a format that > is easy to manipulate by human and tools for debugging, small hacks and > general tinkering, but sstable2json and json2sstable are not that. > So I propose that we deprecate those tools and consider writing better > replacements. It shouldn't be too hard to come up with an output format that > is more aware of modern concepts like composites, UDTs, -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7464) Replace sstable2json and json2sstable
[ https://issues.apache.org/jira/browse/CASSANDRA-7464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087796#comment-15087796 ] Robert Stupp commented on CASSANDRA-7464: - How about {{sstabledump}}? Export implies to me that there's also something that can do the import. (bike shedding, i know) > Replace sstable2json and json2sstable > - > > Key: CASSANDRA-7464 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7464 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Assignee: Chris Lohfink >Priority: Minor > Fix For: 3.x > > Attachments: sstable-only.patch > > > Both tools are pretty awful. They are primarily meant for debugging (there is > much more efficient and convenient ways to do import/export data), but their > output manage to be hard to handle both for humans and for tools (especially > as soon as you have modern stuff like composites). > There is value to having tools to export sstable contents into a format that > is easy to manipulate by human and tools for debugging, small hacks and > general tinkering, but sstable2json and json2sstable are not that. > So I propose that we deprecate those tools and consider writing better > replacements. It shouldn't be too hard to come up with an output format that > is more aware of modern concepts like composites, UDTs, -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10688) Stack overflow from SSTableReader$InstanceTidier.runOnClose in Leak Detector
[ https://issues.apache.org/jira/browse/CASSANDRA-10688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15087872#comment-15087872 ] Benedict commented on CASSANDRA-10688: -- Yes, it does > Stack overflow from SSTableReader$InstanceTidier.runOnClose in Leak Detector > > > Key: CASSANDRA-10688 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10688 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths, Testing >Reporter: Jeremiah Jordan >Assignee: Ariel Weisberg > Fix For: 3.0.x > > > Running some tests against cassandra-3.0 > 9fc957cf3097e54ccd72e51b2d0650dc3e83eae0 > The tests are just running cassandra-stress write and read while adding and > removing nodes from the cluster. After the test runs when I go back through > logs I find the following Stackoverflow fairly often: > ERROR [Strong-Reference-Leak-Detector:1] 2015-11-11 00:04:10,638 > Ref.java:413 - Stackoverflow [private java.lang.Runnable > org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier.runOnClose, > final java.lang.Runnable > org.apache.cassandra.io.sstable.format.SSTableReader$DropPageCache.andThen, > final org.apache.cassandra.cache.InstrumentingCache > org.apache.cassandra.io.sstable.SSTableRewriter$InvalidateKeys.cache, private > final org.apache.cassandra.cache.ICache > org.apache.cassandra.cache.InstrumentingCache.map, private final > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap > org.apache.cassandra.cache.ConcurrentLinkedHashCache.map, final > com.googlecode.concurrentlinkedhashmap.LinkedDeque > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.evictionDeque, > com.googlecode.concurrentlinkedhashmap.Linked > com.googlecode.concurrentlinkedhashmap.LinkedDeque.first, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > ... (repeated a whole bunch more) > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.next, > final java.lang.Object > com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node.key, > public final byte[] org.apache.cassandra.cache.KeyCacheKey.key -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-10331) Establish and implement canonical bulk reading workload(s)
[ https://issues.apache.org/jira/browse/CASSANDRA-10331?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg reassigned CASSANDRA-10331: -- Assignee: Ariel Weisberg > Establish and implement canonical bulk reading workload(s) > -- > > Key: CASSANDRA-10331 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10331 > Project: Cassandra > Issue Type: Sub-task >Reporter: Ariel Weisberg >Assignee: Ariel Weisberg > Fix For: 3.x > > > Implement a client, use stress, or extend stress to a bulk reading workload > that is indicative of the performance we are trying to improve. -- 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=15088182#comment-15088182 ] Carl Yeksigian commented on CASSANDRA-8844: --- # Yes, it was moved to keyspace to handle atomic mutations. Atomicity is provided at the keyspace level, not the table level; if we were to split apart an atomic mutation into CDC and non-CDC commit logs, we would be breaking that atomicity # Yes, the daemon will require using the {{CommitLogReplayer}}, and thus the full Cassandra jars. We don't have any document describing the mutation format or the commit log format, and it is subject to change with versions. Changes to the mutation format or the commit log format will also require deploying a new version of the daemon > 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
[jira] [Commented] (CASSANDRA-7715) Add a credentials cache to the PasswordAuthenticator
[ https://issues.apache.org/jira/browse/CASSANDRA-7715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088188#comment-15088188 ] sankalp kohli commented on CASSANDRA-7715: -- Any updates here? > Add a credentials cache to the PasswordAuthenticator > > > Key: CASSANDRA-7715 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7715 > Project: Cassandra > Issue Type: New Feature > Components: CQL >Reporter: Mike Adamson >Assignee: Sam Tunnicliffe >Priority: Minor > Fix For: 3.x > > > If the PasswordAuthenticator cached credentials for a short time it would > reduce the overhead of user journeys when they need to do multiple > authentications in quick succession. > This cache should work in the same way as the cache in CassandraAuthorizer in > that if it's TTL is set to 0 the cache will be disabled. -- 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=15088208#comment-15088208 ] Brian Hess commented on CASSANDRA-8844: With respect to the atomicity, saying that you break into CDC and non-CDC commit logs would not change existing behavior, as essentially prior to this ticket all tables have CDC disabled. If I have a table in a non-CDC keyspace and I choose enable CDC on that table, how would I do that here? Or vice versa. I expect that turning on CDC after the fact is likely to come up (possibly often). > 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
[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=15088210#comment-15088210 ] Brian Hess commented on CASSANDRA-8844: The only supported language for CDC consumer is Java, then? Will there be an example consumer leveraging the CommitLogReplayer? Will that program need to have Cassandra running on that machine to process the logs? As in, will I be able to copy off the CDC logs to be processed elsewhere? > 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.
[jira] [Commented] (CASSANDRA-10981) Consider striping view locks by key and cfid
[ https://issues.apache.org/jira/browse/CASSANDRA-10981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088223#comment-15088223 ] Tyler Hobbs commented on CASSANDRA-10981: - Patch and (pending) test runs: |[CASSANDRA-10981|https://github.com/thobbs/cassandra/tree/CASSANDRA-10981]|[testall|http://cassci.datastax.com/view/Dev/view/thobbs/job/thobbs-CASSANDRA-10981-testall]|[dtest|http://cassci.datastax.com/view/Dev/view/thobbs/job/thobbs-CASSANDRA-10981-dtest]| I've also created [a new dtest|https://github.com/riptano/cassandra-dtest/pull/736] to focus on single-partition contention. > Consider striping view locks by key and cfid > > > Key: CASSANDRA-10981 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10981 > Project: Cassandra > Issue Type: Improvement > Components: Coordination >Reporter: Tyler Hobbs >Assignee: Tyler Hobbs > Fix For: 3.x > > > We use a striped lock to protect updates to tables with materialized views, > and the lock is currently striped by the partition key of the {{Mutation}}. > This causes concurrent updates to separate tables with the same partition key > to contend for the same lock, resulting in one or more of the mutations being > rescheduled on the {{MUTATION}} threadpool (potentially becoming an > asynchronous operation instead a synchronous operations, from the perspective > of local internal modifications). > Since it's probably fairly common to use the same partition key across > multiple tables, I suggest that we add the cfid of the affected table to the > lock striping, and acquire one lock per affected table (with the same > rescheduling-under-contention behavior). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10981) Consider striping view locks by key and cfid
[ https://issues.apache.org/jira/browse/CASSANDRA-10981?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-10981: Priority: Minor (was: Major) > Consider striping view locks by key and cfid > > > Key: CASSANDRA-10981 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10981 > Project: Cassandra > Issue Type: Improvement > Components: Coordination >Reporter: Tyler Hobbs >Assignee: Tyler Hobbs >Priority: Minor > Fix For: 3.x > > > We use a striped lock to protect updates to tables with materialized views, > and the lock is currently striped by the partition key of the {{Mutation}}. > This causes concurrent updates to separate tables with the same partition key > to contend for the same lock, resulting in one or more of the mutations being > rescheduled on the {{MUTATION}} threadpool (potentially becoming an > asynchronous operation instead a synchronous operations, from the perspective > of local internal modifications). > Since it's probably fairly common to use the same partition key across > multiple tables, I suggest that we add the cfid of the affected table to the > lock striping, and acquire one lock per affected table (with the same > rescheduling-under-contention behavior). -- 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=15088221#comment-15088221 ] Tupshin Harper commented on CASSANDRA-8844: --- Relying on the Cassandra libs doesn't prevent you from copying the logs elsewhere and processing there, and doesn't require cassandra to be running on those machines. It does require the Java consumer to be implemented in a JVM language, however. I'm not fond of that last part, and would love it if we formalized the format, but I suppose I'll start by reverse engineering it. :) > 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
[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=15088231#comment-15088231 ] Carl Yeksigian commented on CASSANDRA-8844: --- {quote} With respect to the atomicity, saying that you break into CDC and non-CDC commit logs would not change existing behavior, as essentially prior to this ticket all tables have CDC disabled. {quote} This would represent a change as we wouldn't be writing all of the mutation to the same commit log and flushing it out to disk. If we were to enable CDC at the table level, they could be flushed to different devices and one could fail while the other was synced, which would mean the whole mutation was not atomically applied. {quote} If I have a table in a non-CDC keyspace and I choose enable CDC on that table, how would I do that here? Or vice versa. I expect that turning on CDC after the fact is likely to come up (possibly often). {quote} It will require altering the keyspace. All of the tables in that keyspace will now be persisting to the CDC log. On the daemon side, it would be possible to filter the CF so that it only processes one. > 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
[jira] [Commented] (CASSANDRA-10055) High CPU load for Cassandra 2.1.8
[ https://issues.apache.org/jira/browse/CASSANDRA-10055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088149#comment-15088149 ] Praveen Peddi commented on CASSANDRA-10055: --- Hi Vijay, Can you elaborate on the changes you did with coreConnPerHostLocal and maxConnPerHostLocal that helped with CPU load? We are in the process of migrating from 2.0.9 to 2.1.11 and with 2.0.9 we are seeing around 5 to 15% CPu and with 2.1.11 we are seeing 25 to 40% CPU for the same exact load test. Latencies with 2.1.11 are also 2 to 3 times higher. What values did you before and after? Were your latencies also improve after you made those config changes? > High CPU load for Cassandra 2.1.8 > - > > Key: CASSANDRA-10055 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10055 > Project: Cassandra > Issue Type: Bug > Components: Configuration > Environment: Prod >Reporter: vijay > Labels: triaged > Attachments: dstst-lcdn.log, dstst-lcdn2.log, dstst-lcdn3.log, > dstst-lcdn4.log, dstst-lcdn5.log, dstst-lcdn6.log, js.log, js2.log, js3.log, > js4.log, js5.log, js6.log, top-bHn1-2.log, top-bHn1-3.log, top-bHn1-4.log, > top-bHn1-5.log, top-bHn1-6.log, top-bHn1.log > > > We are seeing High CPU Load about 80% to 100% in Cassandra 2.1.8 when doing > Data ingest, we did not had this issue in 2.0.x version of Cassandra > we tested this in different Cloud platforms and results are same. > CPU: Tested with M3 2xlarge AWS instances > Ingest rate: Injecting 1 million Inserts each insert is of 1000bytes > no other Operations are happening except inserts in Cassandra > let me know if more info is needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-10334) Generate flame graphs from canonical bulk reading workload running in CI
[ https://issues.apache.org/jira/browse/CASSANDRA-10334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg resolved CASSANDRA-10334. Resolution: Fixed This was done as part of cstar_perf. Thanks! > Generate flame graphs from canonical bulk reading workload running in CI > > > Key: CASSANDRA-10334 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10334 > Project: Cassandra > Issue Type: Sub-task >Reporter: Ariel Weisberg >Assignee: Alan Boudreault > Fix For: 3.x > > > Flame graphs for CPU utilization. Bonus points if we can get source code > annotated with cache misses or at least total counts of cache misses for the > entire run. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[6/8] cassandra git commit: Merge branch 'cassandra-3.2' into cassandra-3.3
Merge branch 'cassandra-3.2' into cassandra-3.3 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1717e10a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1717e10a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1717e10a Branch: refs/heads/trunk Commit: 1717e10aad2d6526c9b51f5597de97cda70758da Parents: 17e79b7 3c6dfa4 Author: Yuki MorishitaAuthored: Thu Jan 7 15:17:55 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 15:17:55 2016 -0600 -- .../org/apache/cassandra/db/ReadCommand.java| 21 ++-- 1 file changed, 15 insertions(+), 6 deletions(-) --
[7/8] cassandra git commit: Merge branch 'cassandra-3.2' into cassandra-3.3
Merge branch 'cassandra-3.2' into cassandra-3.3 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1717e10a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1717e10a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1717e10a Branch: refs/heads/cassandra-3.3 Commit: 1717e10aad2d6526c9b51f5597de97cda70758da Parents: 17e79b7 3c6dfa4 Author: Yuki MorishitaAuthored: Thu Jan 7 15:17:55 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 15:17:55 2016 -0600 -- .../org/apache/cassandra/db/ReadCommand.java| 21 ++-- 1 file changed, 15 insertions(+), 6 deletions(-) --
[4/8] cassandra git commit: Fix regression of CASSANDRA-7392
Fix regression of CASSANDRA-7392 patch by blambov; reviewed by yukim for CASSANDRA-10951 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3c6dfa4a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3c6dfa4a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3c6dfa4a Branch: refs/heads/trunk Commit: 3c6dfa4aa0b9ffb0a48a02b949bff2a8406764e6 Parents: aeaa501 Author: Branimir LambovAuthored: Thu Jan 7 15:16:26 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 15:16:26 2016 -0600 -- .../org/apache/cassandra/db/ReadCommand.java| 21 ++-- 1 file changed, 15 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3c6dfa4a/src/java/org/apache/cassandra/db/ReadCommand.java -- diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 959085b..90372dd 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -479,23 +479,32 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery { protected BaseRowIterator applyToPartition(BaseRowIterator partition) { -maybeAbort(); +if (maybeAbort()) +{ +partition.close(); +return null; +} + return partition; } protected Row applyToRow(Row row) { -maybeAbort(); -return row; +return maybeAbort() ? null : row; } -private void maybeAbort() +private boolean maybeAbort() { +if (TEST_ITERATION_DELAY_MILLIS > 0) +maybeDelayForTesting(); + if (isAborted()) +{ stop(); +return true; +} -if (TEST_ITERATION_DELAY_MILLIS > 0) -maybeDelayForTesting(); +return false; } }
[2/8] cassandra git commit: Add notice for CASSANDRA-10880 in NEWS file
Add notice for CASSANDRA-10880 in NEWS file Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/aeaa5016 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/aeaa5016 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/aeaa5016 Branch: refs/heads/trunk Commit: aeaa5016e0a71d2ad9b0f4b4b07463820f652a87 Parents: 85b8d02 Author: Sylvain LebresneAuthored: Thu Jan 7 16:22:34 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 16:26:22 2016 +0100 -- NEWS.txt | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/aeaa5016/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 3d468b6..c995d89 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -29,6 +29,11 @@ New features - Hinted handoff now supports compression. Reference cassandra.yaml:hints_compression. Note: hints compression is currently disabled by default. +Upgrading +- +- Nothing specific to 3.2 but please see previous versions upgrading section, + especially if you are upgrading from 2.2. + Upgrading - @@ -85,6 +90,9 @@ New features Upgrading - + - Clients must use the native protocol version 3 when upgrading from 2.2.X as + the native protocol version 4 is not compatible between 2.2.X and 3.Y. See + https://www.mail-archive.com/user@cassandra.apache.org/msg45381.html for details. - A new argument of type InetAdress has been added to IAuthenticator::newSaslNegotiator, representing the IP address of the client attempting authentication. It will be a breaking change for any custom implementations.
[8/8] cassandra git commit: Merge branch 'cassandra-3.3' into trunk
Merge branch 'cassandra-3.3' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/15da29f0 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/15da29f0 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/15da29f0 Branch: refs/heads/trunk Commit: 15da29f0fe2dff1216421620fe7ba36e8c4b6d3d Parents: 31f67c2 1717e10 Author: Yuki MorishitaAuthored: Thu Jan 7 15:18:04 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 15:18:04 2016 -0600 -- .../org/apache/cassandra/db/ReadCommand.java| 21 ++-- 1 file changed, 15 insertions(+), 6 deletions(-) --
[1/8] cassandra git commit: Add notice for CASSANDRA-10880 in NEWS file
Repository: cassandra Updated Branches: refs/heads/cassandra-3.2 aeaa5016e -> 3c6dfa4aa refs/heads/cassandra-3.3 17e79b7ac -> 1717e10aa refs/heads/trunk 31f67c289 -> 15da29f0f Add notice for CASSANDRA-10880 in NEWS file Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/aeaa5016 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/aeaa5016 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/aeaa5016 Branch: refs/heads/cassandra-3.3 Commit: aeaa5016e0a71d2ad9b0f4b4b07463820f652a87 Parents: 85b8d02 Author: Sylvain LebresneAuthored: Thu Jan 7 16:22:34 2016 +0100 Committer: Sylvain Lebresne Committed: Thu Jan 7 16:26:22 2016 +0100 -- NEWS.txt | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/aeaa5016/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 3d468b6..c995d89 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -29,6 +29,11 @@ New features - Hinted handoff now supports compression. Reference cassandra.yaml:hints_compression. Note: hints compression is currently disabled by default. +Upgrading +- +- Nothing specific to 3.2 but please see previous versions upgrading section, + especially if you are upgrading from 2.2. + Upgrading - @@ -85,6 +90,9 @@ New features Upgrading - + - Clients must use the native protocol version 3 when upgrading from 2.2.X as + the native protocol version 4 is not compatible between 2.2.X and 3.Y. See + https://www.mail-archive.com/user@cassandra.apache.org/msg45381.html for details. - A new argument of type InetAdress has been added to IAuthenticator::newSaslNegotiator, representing the IP address of the client attempting authentication. It will be a breaking change for any custom implementations.
[5/8] cassandra git commit: Fix regression of CASSANDRA-7392
Fix regression of CASSANDRA-7392 patch by blambov; reviewed by yukim for CASSANDRA-10951 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3c6dfa4a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3c6dfa4a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3c6dfa4a Branch: refs/heads/cassandra-3.2 Commit: 3c6dfa4aa0b9ffb0a48a02b949bff2a8406764e6 Parents: aeaa501 Author: Branimir LambovAuthored: Thu Jan 7 15:16:26 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 15:16:26 2016 -0600 -- .../org/apache/cassandra/db/ReadCommand.java| 21 ++-- 1 file changed, 15 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3c6dfa4a/src/java/org/apache/cassandra/db/ReadCommand.java -- diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 959085b..90372dd 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -479,23 +479,32 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery { protected BaseRowIterator applyToPartition(BaseRowIterator partition) { -maybeAbort(); +if (maybeAbort()) +{ +partition.close(); +return null; +} + return partition; } protected Row applyToRow(Row row) { -maybeAbort(); -return row; +return maybeAbort() ? null : row; } -private void maybeAbort() +private boolean maybeAbort() { +if (TEST_ITERATION_DELAY_MILLIS > 0) +maybeDelayForTesting(); + if (isAborted()) +{ stop(); +return true; +} -if (TEST_ITERATION_DELAY_MILLIS > 0) -maybeDelayForTesting(); +return false; } }
[3/8] cassandra git commit: Fix regression of CASSANDRA-7392
Fix regression of CASSANDRA-7392 patch by blambov; reviewed by yukim for CASSANDRA-10951 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3c6dfa4a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3c6dfa4a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3c6dfa4a Branch: refs/heads/cassandra-3.3 Commit: 3c6dfa4aa0b9ffb0a48a02b949bff2a8406764e6 Parents: aeaa501 Author: Branimir LambovAuthored: Thu Jan 7 15:16:26 2016 -0600 Committer: Yuki Morishita Committed: Thu Jan 7 15:16:26 2016 -0600 -- .../org/apache/cassandra/db/ReadCommand.java| 21 ++-- 1 file changed, 15 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3c6dfa4a/src/java/org/apache/cassandra/db/ReadCommand.java -- diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 959085b..90372dd 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -479,23 +479,32 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery { protected BaseRowIterator applyToPartition(BaseRowIterator partition) { -maybeAbort(); +if (maybeAbort()) +{ +partition.close(); +return null; +} + return partition; } protected Row applyToRow(Row row) { -maybeAbort(); -return row; +return maybeAbort() ? null : row; } -private void maybeAbort() +private boolean maybeAbort() { +if (TEST_ITERATION_DELAY_MILLIS > 0) +maybeDelayForTesting(); + if (isAborted()) +{ stop(); +return true; +} -if (TEST_ITERATION_DELAY_MILLIS > 0) -maybeDelayForTesting(); +return false; } }
[cassandra] Git Push Summary
Repository: cassandra Updated Tags: refs/tags/3.2-tentative [created] 3c6dfa4aa
[jira] [Commented] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088139#comment-15088139 ] Jeff Ferland commented on CASSANDRA-10979: -- Debug logging using the current code: https://gist.github.com/autocracy/346afa253175af475770 > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland > Labels: compaction, leveled > Fix For: 2.1.x > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088139#comment-15088139 ] Jeff Ferland edited comment on CASSANDRA-10979 at 1/7/16 9:24 PM: -- Debug logging using the current (unpatched) code: https://gist.github.com/autocracy/346afa253175af475770 was (Author: autocracy): Debug logging using the current code: https://gist.github.com/autocracy/346afa253175af475770 > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland > Labels: compaction, leveled > Fix For: 2.1.x > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088155#comment-15088155 ] Brian Hess commented on CASSANDRA-8844: A couple comments/question 1. You seem to have changed from specifying CDC at the table level to the Keyspace level. Is that correct? That sort of contradicts one of the requirements in your document. 2. The CDC consumer needs to process a serialized mutation. How will a developer write that? Is there a commitlog serialized mutation format document that someone can use? Do they need to write that in Java and link to the full Cassandra jars (and dependencies)? > 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
[jira] [Resolved] (CASSANDRA-10335) Collect flight recordings of canonical bulk read workload in CI
[ https://issues.apache.org/jira/browse/CASSANDRA-10335?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg resolved CASSANDRA-10335. Resolution: Fixed This was done using Yourkit in cstar_perf. Thanks! > Collect flight recordings of canonical bulk read workload in CI > --- > > Key: CASSANDRA-10335 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10335 > Project: Cassandra > Issue Type: Sub-task >Reporter: Ariel Weisberg >Assignee: Ryan McGuire > Fix For: 3.x > > > Flight recorder to track GC, IO stalls, lock contention, idle threads. Don't > need CPU profiling since that will be covered by flame graphs. -- 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=15088257#comment-15088257 ] Brian Hess commented on CASSANDRA-8844: The semantics don't change. There are additional semantics - mutations have to be in the same keyspace, have the same partition, have the same CDC-enabled flag, and can be in different tables. That is the same as today, as all tables have the CDC-enabled flag set to false. Changing the whole keyspace would mean my CDC consumer is now processing a lot of unneeded data, just to get to the one table of interest. > 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
[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=15088261#comment-15088261 ] Carl Yeksigian commented on CASSANDRA-8844: --- {quote} Will there be an example consumer leveraging the CommitLogReplayer? {quote} I would expect we will provide a very simple example which can be used to test CDC and make sure that future changes don't break the interface, as well as demonstrate how to use the {{CommitLogReplayer}}. {quote} Will that program need to have Cassandra running on that machine to process the logs? As in, will I be able to copy off the CDC logs to be processed elsewhere? {quote} It won't need to have Cassandra running, but it will require access to the schema in order to be able to understand the mutations. > 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 >
[jira] [Commented] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088274#comment-15088274 ] Sebastian Estevez commented on CASSANDRA-10979: --- [~krummas] I mentioned this briefly this morning over chat but now we have more information specifically we observe a large L0 to L1 compaction that blocks L0 STCS compactions, probably because the remaining L0 sstables overlap compactingL0. You can see in the log above that {quote}L0 is too far behind, performing size-tiering there first{quote} does not appear. What do you think? > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland > Labels: compaction, leveled > Fix For: 2.1.x > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088275#comment-15088275 ] Carl Yeksigian commented on CASSANDRA-8844: --- {quote} The semantics don't change. There are additional semantics - mutations have to be in the same keyspace, have the same partition, have the same CDC-enabled flag, and can be in different tables. That is the same as today, as all tables have the CDC-enabled flag set to false. {quote} I disagree. It would be surprising that altering the CDC flag of a previously created table changes the semantics regarding batch queries against that keyspace. Also, from a user point of view, I think it makes sense that you are dealing with which DCs are going to process the data at the keyspace level, as that is also where we deal with the replication strategy. > 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.
[jira] [Comment Edited] (CASSANDRA-10979) LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress
[ https://issues.apache.org/jira/browse/CASSANDRA-10979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088274#comment-15088274 ] Sebastian Estevez edited comment on CASSANDRA-10979 at 1/7/16 10:34 PM: [~krummas] I mentioned this briefly this morning over chat but now we have more information. Specifically we observe a large L0 to L1 compaction that blocks additional L0 STCS compactions, probably because the remaining L0 sstables overlap compactingL0. You can see in the log above that {quote}L0 is too far behind, performing size-tiering there first{quote} does not appear. What do you think? was (Author: sebastian.este...@datastax.com): [~krummas] I mentioned this briefly this morning over chat but now we have more information specifically we observe a large L0 to L1 compaction that blocks L0 STCS compactions, probably because the remaining L0 sstables overlap compactingL0. You can see in the log above that {quote}L0 is too far behind, performing size-tiering there first{quote} does not appear. What do you think? > LCS doesn't do L0 STC on new tables while an L0->L1 compaction is in progress > - > > Key: CASSANDRA-10979 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10979 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: 2.1.11 / 4.8.3 DSE. >Reporter: Jeff Ferland > Labels: compaction, leveled > Fix For: 2.1.x > > > Reading code from > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java > and comparing with behavior shown in > https://gist.github.com/autocracy/c95aca6b00e42215daaf, the following happens: > Score for L1,L2,and L3 is all < 1 (paste shows 20/10 and 200/100, due to > incremental repair). > Relevant code from here is > if (Sets.intersection(l1overlapping, compacting).size() > 0) > return Collections.emptyList(); > Since there will be overlap between what is compacting and L1 (in my case, > pushing over 1,000 tables in to L1 from L0 SCTS), I get a pile up of 1,000 > smaller tables in L0 while awaiting the transition from L0 to L1 and destroy > my performance. > Requested outcome is to continue to perform SCTS on non-compacting L0 tables. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088287#comment-15088287 ] Joshua McKenzie commented on CASSANDRA-8844: bq. You seem to have changed from specifying CDC at the table level to the Keyspace level. Is that correct? That sort of contradicts one of the requirements in your document. Definitely changed, and that's due to an implementation detail of our existing infrastructure and atomicity guarantees surrounding a Mutation. Either *all* of a Mutation or *none* of a Mutation will be applied. Mutations are grouped at the Keyspace level. Given that restriction, we can either: # Allow CDC flagging on a per-CF basis. We would need to write all mutations to the existing CommitLog infrastructure, mixing CDC w/non, and have a consumer be responsible for filtering out non-CDC mutations on consumption. This pushes the responsibility for being schema-aware into the CDC consumption daemon scope. Since all or none of a Mutation must succeed, we cannot write that data to separate buffers and fsync to separate files and still give that guarantee. # make the CDC vs. non-CDC a per-keyspace flag (which fits nicely w/keyspace level replication options). This allows us to write a Mutation as an atomic unit to either the CDC CommitLog or the non-CDC CommitLog and prevents the added scope creep of schema-awareness on consumption daemon. Along with that, it lightens the filtering burden on consumption and allows us to discard CommitLog segments for non-CDC related data much quicker than if the data were interleaved. To me, option #2 is the clear winner and why we settled on that for this phase of the design. As to the statement that it contradicts a requirement in the doc: that depends on how you read it and what we actually need. Since CF is encoded in the mutation in the CDC-log, you have the ability to track, per table, what data has changed. bq. Changing the whole keyspace would mean my CDC consumer is now processing a lot of unneeded data, just to get to the one table of interest. See above for reasons from a technical perspective why per-keyspace is a better fit for our existing architecture. Either you have an entire keyspace with a select amount of data you need to read and thus filter out, or you have to filter out *all mutations in the entire system*. I vote the lesser of two evils. > 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
[jira] [Commented] (CASSANDRA-10977) MV view_tombstone_test is failing on trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-10977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088310#comment-15088310 ] Carl Yeksigian commented on CASSANDRA-10977: Just wanted to point out that the fix in CASSANDRA-10910 was not committed to 3.2, so this dtest is expected to fail there. > MV view_tombstone_test is failing on trunk > -- > > Key: CASSANDRA-10977 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10977 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Alan Boudreault > Labels: dtest > Fix For: 3.x > > > http://cassci.datastax.com/job/trunk_dtest/893/testReport/materialized_views_test/TestMaterializedViews/view_tombstone_test/ > {code} > == > FAIL: view_tombstone_test (materialized_views_test.TestMaterializedViews) > -- > Traceback (most recent call last): > File > "/home/aboudreault/git/cstar/cassandra-dtest/materialized_views_test.py", > line 735, in view_tombstone_test > assert_none(session, "SELECT * FROM t_by_v WHERE v = 1") > File "/home/aboudreault/git/cstar/cassandra-dtest/assertions.py", line 44, > in assert_none > assert list_res == [], "Expected nothing from %s, but got %s" % (query, > list_res) > AssertionError: Expected nothing from SELECT * FROM t_by_v WHERE v = 1, but > got [[1, 1, u'b', None]] > >> begin captured logging << > dtest: DEBUG: cluster ccm directory: /tmp/dtest-MFSCKQ > - >> end captured logging << - > -- > Ran 1 test in 27.986s > FAILED (failures=1) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)