[jira] [Comment Edited] (CASSANDRA-16710) Read repairs can break row isolation

2021-06-04 Thread Samuel Klock (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-16710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17357465#comment-17357465
 ] 

Samuel Klock edited comment on CASSANDRA-16710 at 6/4/21, 4:08 PM:
---

The repro steps in the description yield ('b', 2) for the final read for 
{{VERSION=2.1.22}} and {{VERSION=2.2.19}}.  So 2.x versions don't appear to be 
affected (unless the full row was synced to node3 via a channel the steps don't 
account for).


was (Author: sklock):
The repro steps in the description yield ('b', 2) for the final read for 
{{VERSION=2.1.22}} and {{VERSION=2.2.19}}.  So 2.x versions don't appear to be 
affected.

> Read repairs can break row isolation
> 
>
> Key: CASSANDRA-16710
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16710
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Samuel Klock
>Priority: Urgent
> Fix For: 3.0.x, 3.11.x, 4.0.x
>
>
> This issue essentially revives CASSANDRA-8287, was resolved "Later" in 2015.  
> While it was possible in principle at that time for read repair to break row 
> isolation, that couldn't happen in practice because Cassandra always pulled 
> all of the columns for each row in response to regular reads, so read repairs 
> would never partially resolve a row.  CASSANDRA-10657 modified Cassandra to 
> only pull the requested columns for reads, which enabled read repair to break 
> row isolation in practice.
> Note also that this is distinct from CASSANDRA-14593 (for read repair 
> breaking partition-level isolation): that issue (as we understand it) 
> captures isolation being broken across multiple rows within an update to a 
> partition, while this issue covers broken isolation across multiple columns 
> within an update to a single row.
> This behavior is easy to reproduce under affected versions using {{ccm}}:
> {code:bash}
> ccm create -n 3 -v $VERSION rrtest
> ccm updateconf -y 'hinted_handoff_enabled: false
> max_hint_window_in_ms: 0'
> ccm start
> (cat < CREATE KEYSPACE IF NOT EXISTS rrtest WITH REPLICATION = {'class': 
> 'SimpleStrategy', 'replication_factor': '3'};
> CREATE TABLE IF NOT EXISTS rrtest.kv (key TEXT PRIMARY KEY, col1 TEXT, col2 
> INT);
> CONSISTENCY ALL;
> INSERT INTO rrtest.kv (key, col1, col2) VALUES ('key', 'a', 1);
> EOF
> ) | ccm node1 cqlsh
> ccm node3 stop
> (cat < CONSISTENCY QUORUM;
> INSERT INTO rrtest.kv (key, col1, col2) VALUES ('key', 'b', 2);
> EOF
> ) | ccm node1 cqlsh
> ccm node3 start
> ccm node2 stop
> (cat < CONSISTENCY QUORUM;
> SELECT key, col1 FROM rrtest.kv WHERE key = 'key';
> EOF
> ) | ccm node1 cqlsh
> ccm node1 stop
> (cat < CONSISTENCY ONE;
> SELECT * FROM rrtest.kv WHERE key = 'key';
> EOF
> ) | ccm node3 cqlsh
> {code}
> This snippet creates a three-node cluster with an RF=3 keyspace containing a 
> table with three columns: a partition key and two value columns.  (Hinted 
> handoff can mask the problem if the repro steps are executed in quick 
> succession, so the snippet disables it for this exercise.)  Then:
> # It adds a full row to the table with values ('a', 1), ensuring it's 
> replicated to all three nodes.
> # It stops a node, then replaces the initial row with new values ('b', 2) in 
> a single update, ensuring that it's replicated to both available nodes.
> # It starts the node that was down, then stops one of the other nodes and 
> performs a quorum read just for the letter column.  The read observes 'b'.
> # Finally, it stops the other node that observed the second update, then 
> performs a CL=ONE read of the entire row on the node that was down for that 
> update.
> If read repair respects row isolation, then the final read should observe 
> ('b', 2).  (('a', 1) is also acceptable if we're willing to sacrifice 
> monotonicity.)
> * With {{VERSION=3.0.24}}, the final read observes ('b', 2), as expected.
> * With {{VERSION=3.11.10}} and {{VERSION=4.0-rc1}}, the final read instead 
> observes ('b', 1).  The same is true for 3.0.24 if CASSANDRA-10657 is 
> backported to it.
> The scenario above is somewhat contrived in that it supposes multiple read 
> workflows consulting different sets of columns with different consistency 
> levels.  Under 3.11, asynchronous read repair makes this scenario possible 
> even using just CL=ONE -- and with speculative retry, even if 
> {{read_repair_chance}}/{{dclocal_read_repair_chance}} are both zeroed.  We 
> haven't looked closely at 4.0, but even though (as we understand it) it lacks 
> async read repair, scenarios like CL=ONE writes or failed, 
> partially-committed CL>ONE writes create some surface area for this behavior, 
> even without mixed consistency/column reads.
> Given the importance of paging to reads from wide partitions, it makes some 
> intuit

[jira] [Commented] (CASSANDRA-16710) Read repairs can break row isolation

2021-06-04 Thread Samuel Klock (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-16710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17357465#comment-17357465
 ] 

Samuel Klock commented on CASSANDRA-16710:
--

The repro steps in the description yield ('b', 2) for the final read for 
{{VERSION=2.1.22}} and {{VERSION=2.2.19}}.  So 2.x versions don't appear to be 
affected.

> Read repairs can break row isolation
> 
>
> Key: CASSANDRA-16710
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16710
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Samuel Klock
>Priority: Urgent
> Fix For: 3.0.x, 3.11.x, 4.0.x
>
>
> This issue essentially revives CASSANDRA-8287, was resolved "Later" in 2015.  
> While it was possible in principle at that time for read repair to break row 
> isolation, that couldn't happen in practice because Cassandra always pulled 
> all of the columns for each row in response to regular reads, so read repairs 
> would never partially resolve a row.  CASSANDRA-10657 modified Cassandra to 
> only pull the requested columns for reads, which enabled read repair to break 
> row isolation in practice.
> Note also that this is distinct from CASSANDRA-14593 (for read repair 
> breaking partition-level isolation): that issue (as we understand it) 
> captures isolation being broken across multiple rows within an update to a 
> partition, while this issue covers broken isolation across multiple columns 
> within an update to a single row.
> This behavior is easy to reproduce under affected versions using {{ccm}}:
> {code:bash}
> ccm create -n 3 -v $VERSION rrtest
> ccm updateconf -y 'hinted_handoff_enabled: false
> max_hint_window_in_ms: 0'
> ccm start
> (cat < CREATE KEYSPACE IF NOT EXISTS rrtest WITH REPLICATION = {'class': 
> 'SimpleStrategy', 'replication_factor': '3'};
> CREATE TABLE IF NOT EXISTS rrtest.kv (key TEXT PRIMARY KEY, col1 TEXT, col2 
> INT);
> CONSISTENCY ALL;
> INSERT INTO rrtest.kv (key, col1, col2) VALUES ('key', 'a', 1);
> EOF
> ) | ccm node1 cqlsh
> ccm node3 stop
> (cat < CONSISTENCY QUORUM;
> INSERT INTO rrtest.kv (key, col1, col2) VALUES ('key', 'b', 2);
> EOF
> ) | ccm node1 cqlsh
> ccm node3 start
> ccm node2 stop
> (cat < CONSISTENCY QUORUM;
> SELECT key, col1 FROM rrtest.kv WHERE key = 'key';
> EOF
> ) | ccm node1 cqlsh
> ccm node1 stop
> (cat < CONSISTENCY ONE;
> SELECT * FROM rrtest.kv WHERE key = 'key';
> EOF
> ) | ccm node3 cqlsh
> {code}
> This snippet creates a three-node cluster with an RF=3 keyspace containing a 
> table with three columns: a partition key and two value columns.  (Hinted 
> handoff can mask the problem if the repro steps are executed in quick 
> succession, so the snippet disables it for this exercise.)  Then:
> # It adds a full row to the table with values ('a', 1), ensuring it's 
> replicated to all three nodes.
> # It stops a node, then replaces the initial row with new values ('b', 2) in 
> a single update, ensuring that it's replicated to both available nodes.
> # It starts the node that was down, then stops one of the other nodes and 
> performs a quorum read just for the letter column.  The read observes 'b'.
> # Finally, it stops the other node that observed the second update, then 
> performs a CL=ONE read of the entire row on the node that was down for that 
> update.
> If read repair respects row isolation, then the final read should observe 
> ('b', 2).  (('a', 1) is also acceptable if we're willing to sacrifice 
> monotonicity.)
> * With {{VERSION=3.0.24}}, the final read observes ('b', 2), as expected.
> * With {{VERSION=3.11.10}} and {{VERSION=4.0-rc1}}, the final read instead 
> observes ('b', 1).  The same is true for 3.0.24 if CASSANDRA-10657 is 
> backported to it.
> The scenario above is somewhat contrived in that it supposes multiple read 
> workflows consulting different sets of columns with different consistency 
> levels.  Under 3.11, asynchronous read repair makes this scenario possible 
> even using just CL=ONE -- and with speculative retry, even if 
> {{read_repair_chance}}/{{dclocal_read_repair_chance}} are both zeroed.  We 
> haven't looked closely at 4.0, but even though (as we understand it) it lacks 
> async read repair, scenarios like CL=ONE writes or failed, 
> partially-committed CL>ONE writes create some surface area for this behavior, 
> even without mixed consistency/column reads.
> Given the importance of paging to reads from wide partitions, it makes some 
> intuitive sense that applications shouldn't rely on isolation at the 
> partition level.  Being unable to rely on row isolation is much more 
> surprising, especially given that (modulo the possibility of other atomicity 
> bugs) Cassandra did preserve it before 3.11.  Cassandra should either find a 
> solution for this in cod

[jira] [Created] (CASSANDRA-16710) Read repairs can break row isolation

2021-06-03 Thread Samuel Klock (Jira)
Samuel Klock created CASSANDRA-16710:


 Summary: Read repairs can break row isolation
 Key: CASSANDRA-16710
 URL: https://issues.apache.org/jira/browse/CASSANDRA-16710
 Project: Cassandra
  Issue Type: Bug
Reporter: Samuel Klock


This issue essentially revives CASSANDRA-8287, was resolved "Later" in 2015.  
While it was possible in principle at that time for read repair to break row 
isolation, that couldn't happen in practice because Cassandra always pulled all 
of the columns for each row in response to regular reads, so read repairs would 
never partially resolve a row.  CASSANDRA-10657 modified Cassandra to only pull 
the requested columns for reads, which enabled read repair to break row 
isolation in practice.

Note also that this is distinct from CASSANDRA-14593 (for read repair breaking 
partition-level isolation): that issue (as we understand it) captures isolation 
being broken across multiple rows within an update to a partition, while this 
issue covers broken isolation across multiple columns within an update to a 
single row.

This behavior is easy to reproduce under affected versions using {{ccm}}:

{code:bash}
ccm create -n 3 -v $VERSION rrtest
ccm updateconf -y 'hinted_handoff_enabled: false
max_hint_window_in_ms: 0'
ccm start
(cat 

[jira] [Commented] (CASSANDRA-14941) Expired secondary index sstables are not promptly discarded under TWCS

2020-01-30 Thread Samuel Klock (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14941?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17026736#comment-17026736
 ] 

Samuel Klock commented on CASSANDRA-14941:
--

Pinging again.  If nothing else, it'd be helpful to have confirmation that the 
tweak in the description is safe or, if not, whether it'd be better to make a 
modification to {{Memtable}} instead.  Thanks!

> Expired secondary index sstables are not promptly discarded under TWCS
> --
>
> Key: CASSANDRA-14941
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14941
> Project: Cassandra
>  Issue Type: Bug
>  Components: Feature/2i Index
>Reporter: Samuel Klock
>Assignee: Marcus Eriksson
>Priority: Normal
>
> We have a table in a cluster running 3.0.17 storing roughly time-series data 
> using TWCS with a secondary index. We've noticed that while expired sstables 
> for the table are discarded mostly when we expect them to be, the expired 
> sstables for the secondary index would linger for weeks longer than expected 
> – essentially indefinitely. Eventually the sstables would fill disks, which 
> would require manual steps (deleting ancient index sstables) to address. We 
> verified with {{sstableexpiredblockers}} that there wasn't anything on disk 
> blocking the expired sstables from being dropped, so this looks like a bug.
> Through some debugging, we traced the problem to the index's memtables, which 
> were consistently (except _just_ after node restarts) reporting a minimum 
> timestamp from September 2015 – much older than any of our live data – which 
> causes {{CompactionController.getFullyExpiredSSTables()}} to consistently 
> return an empty set. The reason that the index sstables report this minimum 
> timestamp is because of how index updates are created, using 
> {{PartitionUpdate.singleRowUpdate()}}:
> {code:java}
> public static PartitionUpdate singleRowUpdate(CFMetaData metadata, 
> DecoratedKey key, Row row, Row staticRow)
> {
> MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
> Holder holder = new Holder(
> new PartitionColumns(
> staticRow == null ? Columns.NONE : 
> Columns.from(staticRow.columns()),
> row == null ? Columns.NONE : Columns.from(row.columns())
> ),
> row == null ? BTree.empty() : BTree.singleton(row),
> deletionInfo,
> staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow,
> EncodingStats.NO_STATS
> );
> return new PartitionUpdate(metadata, key, holder, deletionInfo, 
> false);
> }
> {code}
> The use of {{EncodingStats.NO_STATS}} makes it appear as though the earliest 
> timestamp in the resulting {{PartitionUpdate}} is from September 2015. That 
> timestamp becomes the minimum for the memtable.
> Modifying this version of {{PartitionUpdate.singleRowUpdate()}} to:
> {code:java}
> public static PartitionUpdate singleRowUpdate(CFMetaData metadata, 
> DecoratedKey key, Row row, Row staticRow)
> {
> MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
> staticRow = (staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow);
> EncodingStats stats = EncodingStats.Collector.collect(staticRow,
>   (row == null ?
>
> Collections.emptyIterator() :
>
> Iterators.singletonIterator(row)),
>   deletionInfo);
> Holder holder = new Holder(
> new PartitionColumns(
> staticRow == Rows.EMPTY_STATIC_ROW ? Columns.NONE : 
> Columns.from(staticRow.columns()),
> row == null ? Columns.NONE : Columns.from(row.columns())
> ),
> row == null ? BTree.empty() : BTree.singleton(row),
> deletionInfo,
> staticRow,
> stats
> );
> return new PartitionUpdate(metadata, key, holder, deletionInfo, 
> false);
> }
> {code}
> (i.e., computing an {{EncodingStats}} from the contents of the update) seems 
> to fix the problem. However, we're not certain whether A) there's a 
> functional reason the method was using {{EncodingStats.NO_STATS}} previously 
> or B) whether the {{EncodingStats}} the revised version creates is correct 
> (in particular, the use of {{deletionInfo}} feels a little suspect). We're 
> also not sure whether there's a more appropriate fix (e.g., changing how the 
> memtables compute the minimum timestamp, particularly in the {{NO_STATS}} 
> case).



--
This message was sent by Atlassian Jira
(v8.3

[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2020-01-30 Thread Samuel Klock (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17026732#comment-17026732
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

Pinging again.  Are there any remaining blockers?

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Local Write-Read Paths
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Normal
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting {{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
>  * One is that Cassandra was reading more data from disk than was necessary 
> to satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
>  * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in {{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of {{RebufferingInputStream}} in use for our 
> queries, {{RandomAccessReader}} (over compressed sstables), implements a 
> {{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
>  * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 
> 10,000 entries), and much larger values (1 MB, 10,000 entries);
>  * compressible data (a single byte repeated) and uncompressible data (output 
> from {{openssl rand $bytes}}); and
>  * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a {{SELECT DISTINCT key FROM ...}} query with a page size of 
> 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and with the fix:
> {noformat}
> working on compressible
> 2.86733293533
> 1.24895811081
> 1.108907938
> 1.12742400169
> 1.04647302628
> working on uncompressible_uncompressed
> 56.4146180153
> 0.895509958267
> 0.922824144363
> 0.772884130478
> 0.731923818588
> working on uncompressible
> 64.4587619305
> 1.81325793266
> 1.52577018738
> 1.41769099236
> 1.60442209244
> {noformat}
> The long initial runs for the uncompressible data presumably come from 
> repeatedly hitting the disk.  In contrast to the runs without the fix, the 
> initial runs seem to be effective at warming the page cache (as lots of data 
> is skipped, so the data that's read can fit in memory), so subsequent r

[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2019-08-16 Thread Samuel Klock (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16909228#comment-16909228
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

Thanks for the feedback.  I've tweaked the 3.11 patch accordingly.  (Minor 
wrinkle: we don't end up deferring to {{seek()}} in the {{null}} buffer case as 
{{current()}}, which uses the buffer, is called first.)

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Local Write-Read Paths
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Normal
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting {{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
>  * One is that Cassandra was reading more data from disk than was necessary 
> to satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
>  * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in {{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of {{RebufferingInputStream}} in use for our 
> queries, {{RandomAccessReader}} (over compressed sstables), implements a 
> {{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
>  * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 
> 10,000 entries), and much larger values (1 MB, 10,000 entries);
>  * compressible data (a single byte repeated) and uncompressible data (output 
> from {{openssl rand $bytes}}); and
>  * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a {{SELECT DISTINCT key FROM ...}} query with a page size of 
> 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and with the fix:
> {noformat}
> working on compressible
> 2.86733293533
> 1.24895811081
> 1.108907938
> 1.12742400169
> 1.04647302628
> working on uncompressible_uncompressed
> 56.4146180153
> 0.895509958267
> 0.922824144363
> 0.772884130478
> 0.731923818588
> working on uncompressible
> 64.4587619305
> 1.81325793266
> 1.52577018738
> 1.41769099236
> 1.60442209244
> {noformat}
> The long initial runs for the uncompressible data presumably come from 
> repeatedly hitting the disk.  In contrast to the runs without the 

[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2019-08-15 Thread Samuel Klock (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16908203#comment-16908203
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

Ping.  Are there any blockers to merging this?  We've been using this software 
fix in our local Cassandra distribution for some time without any problems, but 
we're happy to make additional changes to make this ready for the community.

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Local Write-Read Paths
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Normal
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting {{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
>  * One is that Cassandra was reading more data from disk than was necessary 
> to satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
>  * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in {{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of {{RebufferingInputStream}} in use for our 
> queries, {{RandomAccessReader}} (over compressed sstables), implements a 
> {{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
>  * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 
> 10,000 entries), and much larger values (1 MB, 10,000 entries);
>  * compressible data (a single byte repeated) and uncompressible data (output 
> from {{openssl rand $bytes}}); and
>  * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a {{SELECT DISTINCT key FROM ...}} query with a page size of 
> 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and with the fix:
> {noformat}
> working on compressible
> 2.86733293533
> 1.24895811081
> 1.108907938
> 1.12742400169
> 1.04647302628
> working on uncompressible_uncompressed
> 56.4146180153
> 0.895509958267
> 0.922824144363
> 0.772884130478
> 0.731923818588
> working on uncompressible
> 64.4587619305
> 1.81325793266
> 1.52577018738
> 1.41769099236
> 1.60442209244
> {noformat}
> The long initial runs for the uncompressible data presumably come from 
> repeatedly hitting the disk.  In contrast

[jira] [Commented] (CASSANDRA-14941) Expired secondary index sstables are not promptly discarded under TWCS

2019-08-15 Thread Samuel Klock (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14941?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16908200#comment-16908200
 ] 

Samuel Klock commented on CASSANDRA-14941:
--

Pinging this issue.  Any chance someone knowledgeable about this part of the 
code could take a look?  We've had positive experience testing this fix in a 
cluster exclusively handling time-series data, but we'd like confirmation that 
it's safe in general before deploying it in other clusters.  It'd also be great 
of course to have this fixed in 3.0.x (or at least 3.11.x/4.0) as well.

> Expired secondary index sstables are not promptly discarded under TWCS
> --
>
> Key: CASSANDRA-14941
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14941
> Project: Cassandra
>  Issue Type: Bug
>  Components: Feature/2i Index
>Reporter: Samuel Klock
>Priority: Normal
>
> We have a table in a cluster running 3.0.17 storing roughly time-series data 
> using TWCS with a secondary index. We've noticed that while expired sstables 
> for the table are discarded mostly when we expect them to be, the expired 
> sstables for the secondary index would linger for weeks longer than expected 
> – essentially indefinitely. Eventually the sstables would fill disks, which 
> would require manual steps (deleting ancient index sstables) to address. We 
> verified with {{sstableexpiredblockers}} that there wasn't anything on disk 
> blocking the expired sstables from being dropped, so this looks like a bug.
> Through some debugging, we traced the problem to the index's memtables, which 
> were consistently (except _just_ after node restarts) reporting a minimum 
> timestamp from September 2015 – much older than any of our live data – which 
> causes {{CompactionController.getFullyExpiredSSTables()}} to consistently 
> return an empty set. The reason that the index sstables report this minimum 
> timestamp is because of how index updates are created, using 
> {{PartitionUpdate.singleRowUpdate()}}:
> {code:java}
> public static PartitionUpdate singleRowUpdate(CFMetaData metadata, 
> DecoratedKey key, Row row, Row staticRow)
> {
> MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
> Holder holder = new Holder(
> new PartitionColumns(
> staticRow == null ? Columns.NONE : 
> Columns.from(staticRow.columns()),
> row == null ? Columns.NONE : Columns.from(row.columns())
> ),
> row == null ? BTree.empty() : BTree.singleton(row),
> deletionInfo,
> staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow,
> EncodingStats.NO_STATS
> );
> return new PartitionUpdate(metadata, key, holder, deletionInfo, 
> false);
> }
> {code}
> The use of {{EncodingStats.NO_STATS}} makes it appear as though the earliest 
> timestamp in the resulting {{PartitionUpdate}} is from September 2015. That 
> timestamp becomes the minimum for the memtable.
> Modifying this version of {{PartitionUpdate.singleRowUpdate()}} to:
> {code:java}
> public static PartitionUpdate singleRowUpdate(CFMetaData metadata, 
> DecoratedKey key, Row row, Row staticRow)
> {
> MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
> staticRow = (staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow);
> EncodingStats stats = EncodingStats.Collector.collect(staticRow,
>   (row == null ?
>
> Collections.emptyIterator() :
>
> Iterators.singletonIterator(row)),
>   deletionInfo);
> Holder holder = new Holder(
> new PartitionColumns(
> staticRow == Rows.EMPTY_STATIC_ROW ? Columns.NONE : 
> Columns.from(staticRow.columns()),
> row == null ? Columns.NONE : Columns.from(row.columns())
> ),
> row == null ? BTree.empty() : BTree.singleton(row),
> deletionInfo,
> staticRow,
> stats
> );
> return new PartitionUpdate(metadata, key, holder, deletionInfo, 
> false);
> }
> {code}
> (i.e., computing an {{EncodingStats}} from the contents of the update) seems 
> to fix the problem. However, we're not certain whether A) there's a 
> functional reason the method was using {{EncodingStats.NO_STATS}} previously 
> or B) whether the {{EncodingStats}} the revised version creates is correct 
> (in particular, the use of {{deletionInfo}} feels a little suspect). We're 
> also not sure whether there's a more appropriate fix (e.g., changing ho

[jira] [Created] (CASSANDRA-14941) Expired secondary index sstables are not promptly discarded under TWCS

2018-12-20 Thread Samuel Klock (JIRA)
Samuel Klock created CASSANDRA-14941:


 Summary: Expired secondary index sstables are not promptly 
discarded under TWCS
 Key: CASSANDRA-14941
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14941
 Project: Cassandra
  Issue Type: Bug
  Components: Secondary Indexes
Reporter: Samuel Klock


We have a table in a cluster running 3.0.17 storing roughly time-series data 
using TWCS with a secondary index. We've noticed that while expired sstables 
for the table are discarded mostly when we expect them to be, the expired 
sstables for the secondary index would linger for weeks longer than expected – 
essentially indefinitely. Eventually the sstables would fill disks, which would 
require manual steps (deleting ancient index sstables) to address. We verified 
with {{sstableexpiredblockers}} that there wasn't anything on disk blocking the 
expired sstables from being dropped, so this looks like a bug.

Through some debugging, we traced the problem to the index's memtables, which 
were consistently (except _just_ after node restarts) reporting a minimum 
timestamp from September 2015 – much older than any of our live data – which 
causes {{CompactionController.getFullyExpiredSSTables()}} to consistently 
return an empty set. The reason that the index sstables report this minimum 
timestamp is because of how index updates are created, using 
{{PartitionUpdate.singleRowUpdate()}}:
{code:java}
public static PartitionUpdate singleRowUpdate(CFMetaData metadata, 
DecoratedKey key, Row row, Row staticRow)
{
MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
Holder holder = new Holder(
new PartitionColumns(
staticRow == null ? Columns.NONE : 
Columns.from(staticRow.columns()),
row == null ? Columns.NONE : Columns.from(row.columns())
),
row == null ? BTree.empty() : BTree.singleton(row),
deletionInfo,
staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow,
EncodingStats.NO_STATS
);
return new PartitionUpdate(metadata, key, holder, deletionInfo, false);
}
{code}
The use of {{EncodingStats.NO_STATS}} makes it appear as though the earliest 
timestamp in the resulting {{PartitionUpdate}} is from September 2015. That 
timestamp becomes the minimum for the memtable.

Modifying this version of {{PartitionUpdate.singleRowUpdate()}} to:
{code:java}
public static PartitionUpdate singleRowUpdate(CFMetaData metadata, 
DecoratedKey key, Row row, Row staticRow)
{
MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
staticRow = (staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow);
EncodingStats stats = EncodingStats.Collector.collect(staticRow,
  (row == null ?
   
Collections.emptyIterator() :
   
Iterators.singletonIterator(row)),
  deletionInfo);
Holder holder = new Holder(
new PartitionColumns(
staticRow == Rows.EMPTY_STATIC_ROW ? Columns.NONE : 
Columns.from(staticRow.columns()),
row == null ? Columns.NONE : Columns.from(row.columns())
),
row == null ? BTree.empty() : BTree.singleton(row),
deletionInfo,
staticRow,
stats
);
return new PartitionUpdate(metadata, key, holder, deletionInfo, false);
}
{code}
(i.e., computing an {{EncodingStats}} from the contents of the update) seems to 
fix the problem. However, we're not certain whether A) there's a functional 
reason the method was using {{EncodingStats.NO_STATS}} previously or B) whether 
the {{EncodingStats}} the revised version creates is correct (in particular, 
the use of {{deletionInfo}} feels a little suspect). We're also not sure 
whether there's a more appropriate fix (e.g., changing how the memtables 
compute the minimum timestamp, particularly in the {{NO_STATS}} case).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-05-15 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16476009#comment-16476009
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

Thanks.  NPE in 3.11 and {{IOException}} in trunk sounds very reasonable.  The 
patches now reflect that feedback.

||Patch||Tests||
|[3.11|https://github.com/apache/cassandra/compare/cassandra-3.11...akasklock:CASSANDRA-14415-Use-seek-for-skipBytes-3.11.2]
 |[link|https://circleci.com/gh/akasklock/cassandra/12] |
|[trunk|https://github.com/apache/cassandra/compare/trunk...akasklock:CASSANDRA-14415-Use-seek-for-skipBytes-trunk]
 |[link|https://circleci.com/gh/akasklock/cassandra/13] |

* All of the tests for trunk passed in this run.
* The patch for 3.11 should also apply to 3.0, but as noted above, we're not 
confident it would be useful without CASSANDRA-10657, at least for the workflow 
we're concerned about in this issue.

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Major
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting {{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
>  * One is that Cassandra was reading more data from disk than was necessary 
> to satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
>  * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in {{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of {{RebufferingInputStream}} in use for our 
> queries, {{RandomAccessReader}} (over compressed sstables), implements a 
> {{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
>  * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 
> 10,000 entries), and much larger values (1 MB, 10,000 entries);
>  * compressible data (a single byte repeated) and uncompressible data (output 
> from {{openssl rand $bytes}}); and
>  * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a {{SELECT DISTINCT key FROM ...}} query with a page size of 
> 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and with the fix:
>

[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-05-14 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16474717#comment-16474717
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

[The 
branch|https://github.com/apache/cassandra/compare/trunk...akasklock:CASSANDRA-14415-Use-seek-for-skipBytes-3.11.2]
 now includes test coverage.  CI runs: 
[3.11|https://circleci.com/gh/akasklock/cassandra/6] and 
[trunk|https://circleci.com/gh/akasklock/cassandra/9].

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Major
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting {{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
>  * One is that Cassandra was reading more data from disk than was necessary 
> to satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
>  * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in {{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of {{RebufferingInputStream}} in use for our 
> queries, {{RandomAccessReader}} (over compressed sstables), implements a 
> {{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
>  * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 
> 10,000 entries), and much larger values (1 MB, 10,000 entries);
>  * compressible data (a single byte repeated) and uncompressible data (output 
> from {{openssl rand $bytes}}); and
>  * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a {{SELECT DISTINCT key FROM ...}} query with a page size of 
> 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and with the fix:
> {noformat}
> working on compressible
> 2.86733293533
> 1.24895811081
> 1.108907938
> 1.12742400169
> 1.04647302628
> working on uncompressible_uncompressed
> 56.4146180153
> 0.895509958267
> 0.922824144363
> 0.772884130478
> 0.731923818588
> working on uncompressible
> 64.4587619305
> 1.81325793266
> 1.52577018738
> 1.41769099236
> 1.60442209244
> {noformat}
> The long initial runs for the uncompressible data presumably come from 
> repeatedly hitting the 

[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-05-14 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16474190#comment-16474190
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

Thanks for taking a look, [~KurtG]. Adding a test is certainly reasonable 
feedback; we'll make that change soon.
{quote}In the case that n < 0, delegating back to {{RIS.skipBytes()}} will 
return the same thing, and in the case that buffer == null, {{RIS.skipBytes()}} 
will NPE. If the extra check is necessary seems to me it should apply to both 
methods, but all you can do is return 0. Might be better to just leave the 
extra check out...?
{quote}
The intent of this check is to conservatively preserve the existing behavior 
for {{n}} < 0 or a null {{buffer}}. Regarding the NPE behavior in particular: 
having a {{skipBytes()}} implementation NPE is at least surprising (and 
possibly a bug), but it's not straightforward to tell if there's any logic in 
Cassandra that depends on it. So we elected not to change that behavior. 
Delegating to the superclass seemed like a good way to ensure that the behavior 
of {{skipBytes()}} is kept consistent in case the superclass implementation 
does ever end up changing.

That said: happy not to delegate to the superclass (or add an explanatory 
comment) if that'd be preferable. For an implementation here:
 * If {{n}} < 0, the obvious choice is to return 0 without mutating the 
reader's state.
 * If {{buffer}} is null, if we just fall through to {{seek()}}, we'll throw an 
{{IllegalStateException}}, which seems undesirable. The contract for 
{{skipBytes()}} suggests returning 0 would be a reasonable choice, although it 
might be better to signal a problem to the caller by throwing an 
{{IOException}}. That would be a pretty abrasive change in behavior, however.

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Major
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting {{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
>  * One is that Cassandra was reading more data from disk than was necessary 
> to satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
>  * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in {{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of {{RebufferingInputStream}} in use for our 
> queries, {{RandomAccessReader}} (over compressed sstables), implements a 
> {{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
>  * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 
> 10,000 entries), and much larger values (1 MB, 10,000 entries);
>  * compressible data (a single byte repeated) and uncompressible data (output 
> from {{openssl rand $bytes}}); and
>  * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompressio

[jira] [Updated] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-05-09 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-14415:
-
Description: 
Running Cassandra 3.0.16, we observed a major performance regression affecting 
{{SELECT DISTINCT keys}}-style queries against certain tables.  Based on some 
investigation (guided by some helpful feedback from Benjamin on the dev list), 
we tracked the regression down to two problems.
 * One is that Cassandra was reading more data from disk than was necessary to 
satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x release.
 * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
snippet in {{RebufferingInputStream}}:
{code:java}
    @Override
    public int skipBytes(int n) throws IOException
    {
    if (n < 0)
    return 0;
    int requested = n;
    int position = buffer.position(), limit = buffer.limit(), remaining;
    while ((remaining = limit - position) < n)
    {
    n -= remaining;
    buffer.position(limit);
    reBuffer();
    position = buffer.position();
    limit = buffer.limit();
    if (position == limit)
    return requested - n;
    }
    buffer.position(position + n);
    return requested;
    }
{code}
The gist of it is that to skip bytes, the stream needs to read those bytes into 
memory then throw them away.  In our tests, we were spending a lot of time in 
this method, so it looked like the chief drag on performance.

We noticed that the subclass of {{RebufferingInputStream}} in use for our 
queries, {{RandomAccessReader}} (over compressed sstables), implements a 
{{seek()}} method.  Overriding {{skipBytes()}} in it to use {{seek()}} instead 
was sufficient to fix the performance regression.

The performance difference is significant for tables with large values.  It's 
straightforward to evaluate with very simple key-value tables, e.g.:

{{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}

We did some basic experimentation with the following variations (all in a 
single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
workstation):
 * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 10,000 
entries), and much larger values (1 MB, 10,000 entries);
 * compressible data (a single byte repeated) and uncompressible data (output 
from {{openssl rand $bytes}}); and
 * with and without sstable compression.  (With compression, we use Cassandra's 
defaults.)

The difference is most conspicuous for tables with large, uncompressible data 
and sstable decompression (which happens to describe the use case that 
triggered our investigation).  It is smaller but still readily apparent for 
tables with effective compression.  For uncompressible data without compression 
enabled, there is no appreciable difference.

Here's what the performance looks like without our patch for the 1-MB entries 
(times in seconds, five consecutive runs for each data set, all exhausting the 
results from a {{SELECT DISTINCT key FROM ...}} query with a page size of 24):
{noformat}
working on compressible
5.21180510521
5.10270500183
5.22311806679
4.6732840538
4.84219098091
working on uncompressible_uncompressed
55.0423607826
0.769015073776
0.850513935089
0.713396072388
0.62596988678
working on uncompressible
413.292617083
231.345913887
449.524993896
425.135111094
243.469946861
{noformat}
and with the fix:
{noformat}
working on compressible
2.86733293533
1.24895811081
1.108907938
1.12742400169
1.04647302628
working on uncompressible_uncompressed
56.4146180153
0.895509958267
0.922824144363
0.772884130478
0.731923818588
working on uncompressible
64.4587619305
1.81325793266
1.52577018738
1.41769099236
1.60442209244
{noformat}
The long initial runs for the uncompressible data presumably come from 
repeatedly hitting the disk.  In contrast to the runs without the fix, the 
initial runs seem to be effective at warming the page cache (as lots of data is 
skipped, so the data that's read can fit in memory), so subsequent runs are 
faster.

For smaller data sets, {{RandomAccessReader.seek()}} and 
{{RebufferingInputStream.skipBytes()}} are approximately equivalent in their 
behavior (reducing to changing the position pointer of an in-memory buffer most 
of the time), so there isn't much difference.  Here's before the fix for the 
1-KB entries:
{noformat}
working on small_compressible
8.34115099907
8.57280993462
8.3534219265
8.55130696297
8.17362189293
working on small_uncompressible_uncompressed
7.85155582428
7.54075288773
7.50106596947
7.39202189445
7.95735621452
working on small_uncompressible
7.89256501198
7.88875198364
7.9013261795
7.76551413536
7.84927678108
{noformat}
and after:
{noformat}
working on small_compressible
8.29225707054
7.57822394371
8.10092878342
8.21332192421
8.19347810745
working on small_uncompressible_

[jira] [Commented] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-05-09 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16468868#comment-16468868
 ] 

Samuel Klock commented on CASSANDRA-14415:
--

We haven't tested, but I doubt the patch would be much help for _this_ workflow 
without CASSANDRA-10657.  If there are other contexts where Cassandra wants to 
skip large chunks of a (compressed) file it's modeling as a stream, then the 
patch might provide some meaningful benefit for 3.0.x.  I don't know if there 
are any though.

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Major
>  Labels: performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting \{{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
> * One is that Cassandra was reading more data from disk than was necessary to 
> satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
> * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in \{{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of \{{RebufferingInputStream}} in use for our 
> queries, \{{RandomAccessReader}} (over compressed sstables), implements a 
> \{{seek()}} method.  Overriding \{{skipBytes()}} in it to use \{{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {\{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
> * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 10,000 
> entries), and much larger values (1 MB, 10,000 entries);
> * compressible data (a single byte repeated) and uncompressible data (output 
> from \{{openssl rand $bytes}}); and
> * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a \{{SELECT DISTINCT key FROM ...}} query with a page size 
> of 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and without the fix:
> {noformat}
> working on compressible
> 2.86733293533
> 1.24895811081
> 1.108907938
> 1.12742400169
> 1.04647302628
> working on uncompressible_uncompressed
> 56.4146180153
> 0.895509958267
> 0.922824144363
> 0.772884130478
> 0.731923818588
> working on uncompressible
> 64.4587619305
> 1.81325793266
> 1.52577018738
> 1.41769099236
> 1.60442209244
> {noformat}
> The long initial runs for the 

[jira] [Updated] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-04-24 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14415?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-14415:
-
Reproduced In: 3.11.2, 3.0.16  (was: 3.0.16, 3.11.2)
   Status: Patch Available  (was: Open)

Patch available 
[here|https://github.com/akasklock/cassandra/tree/CASSANDRA-14415-Use-seek-for-skipBytes-3.11.2].

> Performance regression in queries for distinct keys
> ---
>
> Key: CASSANDRA-14415
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Samuel Klock
>Assignee: Samuel Klock
>Priority: Major
>
> Running Cassandra 3.0.16, we observed a major performance regression 
> affecting \{{SELECT DISTINCT keys}}-style queries against certain tables.  
> Based on some investigation (guided by some helpful feedback from Benjamin on 
> the dev list), we tracked the regression down to two problems.
> * One is that Cassandra was reading more data from disk than was necessary to 
> satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x 
> release.
> * If the fix for CASSANDRA-10657 is incorporated, the other is this code 
> snippet in \{{RebufferingInputStream}}:
> {code:java}
>     @Override
>     public int skipBytes(int n) throws IOException
>     {
>     if (n < 0)
>     return 0;
>     int requested = n;
>     int position = buffer.position(), limit = buffer.limit(), remaining;
>     while ((remaining = limit - position) < n)
>     {
>     n -= remaining;
>     buffer.position(limit);
>     reBuffer();
>     position = buffer.position();
>     limit = buffer.limit();
>     if (position == limit)
>     return requested - n;
>     }
>     buffer.position(position + n);
>     return requested;
>     }
> {code}
> The gist of it is that to skip bytes, the stream needs to read those bytes 
> into memory then throw them away.  In our tests, we were spending a lot of 
> time in this method, so it looked like the chief drag on performance.
> We noticed that the subclass of \{{RebufferingInputStream}} in use for our 
> queries, \{{RandomAccessReader}} (over compressed sstables), implements a 
> \{{seek()}} method.  Overriding \{{skipBytes()}} in it to use \{{seek()}} 
> instead was sufficient to fix the performance regression.
> The performance difference is significant for tables with large values.  It's 
> straightforward to evaluate with very simple key-value tables, e.g.:
> {\{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}
> We did some basic experimentation with the following variations (all in a 
> single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
> workstation):
> * small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 10,000 
> entries), and much larger values (1 MB, 10,000 entries);
> * compressible data (a single byte repeated) and uncompressible data (output 
> from \{{openssl rand $bytes}}); and
> * with and without sstable compression.  (With compression, we use 
> Cassandra's defaults.)
> The difference is most conspicuous for tables with large, uncompressible data 
> and sstable decompression (which happens to describe the use case that 
> triggered our investigation).  It is smaller but still readily apparent for 
> tables with effective compression.  For uncompressible data without 
> compression enabled, there is no appreciable difference.
> Here's what the performance looks like without our patch for the 1-MB entries 
> (times in seconds, five consecutive runs for each data set, all exhausting 
> the results from a \{{SELECT DISTINCT key FROM ...}} query with a page size 
> of 24):
> {noformat}
> working on compressible
> 5.21180510521
> 5.10270500183
> 5.22311806679
> 4.6732840538
> 4.84219098091
> working on uncompressible_uncompressed
> 55.0423607826
> 0.769015073776
> 0.850513935089
> 0.713396072388
> 0.62596988678
> working on uncompressible
> 413.292617083
> 231.345913887
> 449.524993896
> 425.135111094
> 243.469946861
> {noformat}
> and without the fix:
> {noformat}
> working on compressible
> 2.86733293533
> 1.24895811081
> 1.108907938
> 1.12742400169
> 1.04647302628
> working on uncompressible_uncompressed
> 56.4146180153
> 0.895509958267
> 0.922824144363
> 0.772884130478
> 0.731923818588
> working on uncompressible
> 64.4587619305
> 1.81325793266
> 1.52577018738
> 1.41769099236
> 1.60442209244
> {noformat}
> The long initial runs for the uncompressible data presumably come from 
> repeatedly hitting the disk.  In contrast to the runs without the fix, the 
> initial runs seem to be effective at warming the page cache (as lots of data 
> is skipped, so the data that's read can fit in memory), so subsequent runs 

[jira] [Created] (CASSANDRA-14415) Performance regression in queries for distinct keys

2018-04-24 Thread Samuel Klock (JIRA)
Samuel Klock created CASSANDRA-14415:


 Summary: Performance regression in queries for distinct keys
 Key: CASSANDRA-14415
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14415
 Project: Cassandra
  Issue Type: Improvement
Reporter: Samuel Klock
Assignee: Samuel Klock


Running Cassandra 3.0.16, we observed a major performance regression affecting 
\{{SELECT DISTINCT keys}}-style queries against certain tables.  Based on some 
investigation (guided by some helpful feedback from Benjamin on the dev list), 
we tracked the regression down to two problems.

* One is that Cassandra was reading more data from disk than was necessary to 
satisfy the query.  This was fixed under CASSANDRA-10657 in a later 3.x release.
* If the fix for CASSANDRA-10657 is incorporated, the other is this code 
snippet in \{{RebufferingInputStream}}:
{code:java}
    @Override
    public int skipBytes(int n) throws IOException
    {
    if (n < 0)
    return 0;
    int requested = n;
    int position = buffer.position(), limit = buffer.limit(), remaining;
    while ((remaining = limit - position) < n)
    {
    n -= remaining;
    buffer.position(limit);
    reBuffer();
    position = buffer.position();
    limit = buffer.limit();
    if (position == limit)
    return requested - n;
    }
    buffer.position(position + n);
    return requested;
    }
{code}
The gist of it is that to skip bytes, the stream needs to read those bytes into 
memory then throw them away.  In our tests, we were spending a lot of time in 
this method, so it looked like the chief drag on performance.

We noticed that the subclass of \{{RebufferingInputStream}} in use for our 
queries, \{{RandomAccessReader}} (over compressed sstables), implements a 
\{{seek()}} method.  Overriding \{{skipBytes()}} in it to use \{{seek()}} 
instead was sufficient to fix the performance regression.

The performance difference is significant for tables with large values.  It's 
straightforward to evaluate with very simple key-value tables, e.g.:

{\{CREATE TABLE testtable (key TEXT PRIMARY KEY, value BLOB);}}

We did some basic experimentation with the following variations (all in a 
single-node 3.11.2 cluster with off-the-shelf settings running on a dev 
workstation):

* small values (1 KB, 100,000 entries), somewhat larger values (25 KB, 10,000 
entries), and much larger values (1 MB, 10,000 entries);
* compressible data (a single byte repeated) and uncompressible data (output 
from \{{openssl rand $bytes}}); and
* with and without sstable compression.  (With compression, we use Cassandra's 
defaults.)

The difference is most conspicuous for tables with large, uncompressible data 
and sstable decompression (which happens to describe the use case that 
triggered our investigation).  It is smaller but still readily apparent for 
tables with effective compression.  For uncompressible data without compression 
enabled, there is no appreciable difference.

Here's what the performance looks like without our patch for the 1-MB entries 
(times in seconds, five consecutive runs for each data set, all exhausting the 
results from a \{{SELECT DISTINCT key FROM ...}} query with a page size of 24):

{noformat}
working on compressible
5.21180510521
5.10270500183
5.22311806679
4.6732840538
4.84219098091
working on uncompressible_uncompressed
55.0423607826
0.769015073776
0.850513935089
0.713396072388
0.62596988678
working on uncompressible
413.292617083
231.345913887
449.524993896
425.135111094
243.469946861
{noformat}

and without the fix:

{noformat}
working on compressible
2.86733293533
1.24895811081
1.108907938
1.12742400169
1.04647302628
working on uncompressible_uncompressed
56.4146180153
0.895509958267
0.922824144363
0.772884130478
0.731923818588
working on uncompressible
64.4587619305
1.81325793266
1.52577018738
1.41769099236
1.60442209244
{noformat}

The long initial runs for the uncompressible data presumably come from 
repeatedly hitting the disk.  In contrast to the runs without the fix, the 
initial runs seem to be effective at warming the page cache (as lots of data is 
skipped, so the data that's read can fit in memory), so subsequent runs are 
faster.

For smaller data sets, \{{RandomAccessReader.seek()}} and 
\{{RebufferingInputStream.skipBytes()}} are approximately equivalent in their 
behavior (reducing to changing the position pointer of an in-memory buffer most 
of the time), so there isn't much difference.  Here's before the fix for the 
1-KB entries:

{noformat}
working on small_compressible
8.34115099907
8.57280993462
8.3534219265
8.55130696297
8.17362189293
working on small_uncompressible_uncompressed
7.85155582428
7.54075288773
7.50106596947
7.39202189445
7.95735621452
working on small_uncompressible
7.89256501198
7.88875198364
7.90132617

[jira] [Updated] (CASSANDRA-13109) Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0

2017-01-06 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13109?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-13109:
-
Attachment: 13109-3.0.txt

Attaching the patch.

> Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0
> ---
>
> Key: CASSANDRA-13109
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13109
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Samuel Klock
> Attachments: 13109-3.0.txt
>
>
> We've observed this upgrading from 2.1.15 to 3.0.8 and from 2.1.16 to 3.0.10: 
> some lightweight transactions executed on upgraded nodes fail with a read 
> failure.  The following conditions seem relevant to this occurring:
> * The transaction must be conditioned on the current value of at least one 
> column, e.g., {{IF NOT EXISTS}} transactions don't seem to be affected.
> * There should be a collection column (in our case, a map) defined on the 
> table on which the transaction is executed.
> * The transaction should be executed before sstables on the node are 
> upgraded.  The failure does not occur after the sstables have been upgraded 
> (whether via {{nodetool upgradesstables}} or effectively via compaction).
> * Upgraded nodes seem to be able to participate in lightweight transactions 
> as long as they're not the coordinator.
> * The values in the row being manipulated by the transaction must have been 
> consistently manipulated by lightweight transactions (perhaps the existence 
> of Paxos state for the partition is somehow relevant?).
> * In 3.0.10, it _seems_ to be necessary to have the partition split across 
> multiple legacy sstables.  This was not necessary to reproduce the bug in 
> 3.0.8 or .9.
> For applications affected by this bug, a possible workaround is to prevent 
> nodes being upgraded from coordinating requests until sstables have been 
> upgraded.
> We're able to reproduce this when upgrading from 2.1.16 to 3.0.10 with the 
> following steps on a single-node cluster using a mostly pristine 
> {{cassandra.yaml}} from the source distribution.
> # Start Cassandra-2.1.16 on the node.
> # Create a table with a collection column and insert some data into it.
> {code:sql}
> CREATE KEYSPACE test WITH REPLICATION = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> CREATE TABLE test.test (key TEXT PRIMARY KEY, cas_target TEXT, 
> some_collection MAP);
> INSERT INTO test.test (key, cas_target, some_collection) VALUES ('key', 
> 'value', {}) IF NOT EXISTS;
> {code}
> # Flush the row to an sstable: {{nodetool flush}}.
> # Update the row:
> {code:sql}
> UPDATE test.test SET cas_target = 'newvalue', some_collection = {} WHERE key 
> = 'key' IF cas_target = 'value';
> {code}
> # Drain the node: {{nodetool drain}}
> # Stop the node, upgrade to 3.0.10, and start the node.
> # Attempt to update the row again:
> {code:sql}
> UPDATE test.test SET cas_target = 'lastvalue' WHERE key = 'key' IF cas_target 
> = 'newvalue';
> {code}
> Using {{cqlsh}}, if the error is reproduced, the following output will be 
> returned:
> {code:sql}
> $ ./cqlsh <<< "UPDATE test.test SET cas_target = 'newvalue', some_collection 
> = {} WHERE key = 'key' IF cas_target = 'value';"
> (start: 2016-12-22 10:14:27 EST)
> :2:ReadFailure: Error from server: code=1300 [Replica(s) failed to 
> execute read] message="Operation failed - received 0 responses and 1 
> failures" info={'failures': 1, 'received_responses': 0, 'required_responses': 
> 1, 'consistency': 'QUORUM'}
> {code}
> and the following stack trace will be present in the system log:
> {noformat}
> WARN  15:14:28 Uncaught exception on thread 
> Thread[SharedPool-Worker-10,10,main]: {}
> java.lang.RuntimeException: java.lang.NullPointerException
>   at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2476)
>  ~[main/:na]
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_101]
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
>  ~[main/:na]
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
>  [main/:na]
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [main/:na]
>   at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> Caused by: java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:617)
>  ~[main/:na]
>   at 
> org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:569)
>  ~[main/:na]
>   at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:220)
>  ~[main/:na

[jira] [Updated] (CASSANDRA-13109) Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0

2017-01-06 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13109?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-13109:
-
Reproduced In: 3.0.10, 3.0.9, 3.0.8  (was: 3.0.8, 3.0.9, 3.0.10)
   Status: Patch Available  (was: Open)

> Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0
> ---
>
> Key: CASSANDRA-13109
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13109
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Samuel Klock
> Attachments: 13109-3.0.txt
>
>
> We've observed this upgrading from 2.1.15 to 3.0.8 and from 2.1.16 to 3.0.10: 
> some lightweight transactions executed on upgraded nodes fail with a read 
> failure.  The following conditions seem relevant to this occurring:
> * The transaction must be conditioned on the current value of at least one 
> column, e.g., {{IF NOT EXISTS}} transactions don't seem to be affected.
> * There should be a collection column (in our case, a map) defined on the 
> table on which the transaction is executed.
> * The transaction should be executed before sstables on the node are 
> upgraded.  The failure does not occur after the sstables have been upgraded 
> (whether via {{nodetool upgradesstables}} or effectively via compaction).
> * Upgraded nodes seem to be able to participate in lightweight transactions 
> as long as they're not the coordinator.
> * The values in the row being manipulated by the transaction must have been 
> consistently manipulated by lightweight transactions (perhaps the existence 
> of Paxos state for the partition is somehow relevant?).
> * In 3.0.10, it _seems_ to be necessary to have the partition split across 
> multiple legacy sstables.  This was not necessary to reproduce the bug in 
> 3.0.8 or .9.
> For applications affected by this bug, a possible workaround is to prevent 
> nodes being upgraded from coordinating requests until sstables have been 
> upgraded.
> We're able to reproduce this when upgrading from 2.1.16 to 3.0.10 with the 
> following steps on a single-node cluster using a mostly pristine 
> {{cassandra.yaml}} from the source distribution.
> # Start Cassandra-2.1.16 on the node.
> # Create a table with a collection column and insert some data into it.
> {code:sql}
> CREATE KEYSPACE test WITH REPLICATION = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> CREATE TABLE test.test (key TEXT PRIMARY KEY, cas_target TEXT, 
> some_collection MAP);
> INSERT INTO test.test (key, cas_target, some_collection) VALUES ('key', 
> 'value', {}) IF NOT EXISTS;
> {code}
> # Flush the row to an sstable: {{nodetool flush}}.
> # Update the row:
> {code:sql}
> UPDATE test.test SET cas_target = 'newvalue', some_collection = {} WHERE key 
> = 'key' IF cas_target = 'value';
> {code}
> # Drain the node: {{nodetool drain}}
> # Stop the node, upgrade to 3.0.10, and start the node.
> # Attempt to update the row again:
> {code:sql}
> UPDATE test.test SET cas_target = 'lastvalue' WHERE key = 'key' IF cas_target 
> = 'newvalue';
> {code}
> Using {{cqlsh}}, if the error is reproduced, the following output will be 
> returned:
> {code:sql}
> $ ./cqlsh <<< "UPDATE test.test SET cas_target = 'newvalue', some_collection 
> = {} WHERE key = 'key' IF cas_target = 'value';"
> (start: 2016-12-22 10:14:27 EST)
> :2:ReadFailure: Error from server: code=1300 [Replica(s) failed to 
> execute read] message="Operation failed - received 0 responses and 1 
> failures" info={'failures': 1, 'received_responses': 0, 'required_responses': 
> 1, 'consistency': 'QUORUM'}
> {code}
> and the following stack trace will be present in the system log:
> {noformat}
> WARN  15:14:28 Uncaught exception on thread 
> Thread[SharedPool-Worker-10,10,main]: {}
> java.lang.RuntimeException: java.lang.NullPointerException
>   at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2476)
>  ~[main/:na]
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_101]
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
>  ~[main/:na]
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
>  [main/:na]
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [main/:na]
>   at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> Caused by: java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:617)
>  ~[main/:na]
>   at 
> org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:569)
>  ~[main/:na]
>   at 
> org.apache.cassandra.utils.Merg

[jira] [Created] (CASSANDRA-13109) Lightweight transactions temporarily fail after upgrade from 2.1 to 3.0

2017-01-06 Thread Samuel Klock (JIRA)
Samuel Klock created CASSANDRA-13109:


 Summary: Lightweight transactions temporarily fail after upgrade 
from 2.1 to 3.0
 Key: CASSANDRA-13109
 URL: https://issues.apache.org/jira/browse/CASSANDRA-13109
 Project: Cassandra
  Issue Type: Bug
Reporter: Samuel Klock


We've observed this upgrading from 2.1.15 to 3.0.8 and from 2.1.16 to 3.0.10: 
some lightweight transactions executed on upgraded nodes fail with a read 
failure.  The following conditions seem relevant to this occurring:

* The transaction must be conditioned on the current value of at least one 
column, e.g., {{IF NOT EXISTS}} transactions don't seem to be affected.
* There should be a collection column (in our case, a map) defined on the table 
on which the transaction is executed.
* The transaction should be executed before sstables on the node are upgraded.  
The failure does not occur after the sstables have been upgraded (whether via 
{{nodetool upgradesstables}} or effectively via compaction).
* Upgraded nodes seem to be able to participate in lightweight transactions as 
long as they're not the coordinator.
* The values in the row being manipulated by the transaction must have been 
consistently manipulated by lightweight transactions (perhaps the existence of 
Paxos state for the partition is somehow relevant?).
* In 3.0.10, it _seems_ to be necessary to have the partition split across 
multiple legacy sstables.  This was not necessary to reproduce the bug in 3.0.8 
or .9.

For applications affected by this bug, a possible workaround is to prevent 
nodes being upgraded from coordinating requests until sstables have been 
upgraded.

We're able to reproduce this when upgrading from 2.1.16 to 3.0.10 with the 
following steps on a single-node cluster using a mostly pristine 
{{cassandra.yaml}} from the source distribution.

# Start Cassandra-2.1.16 on the node.
# Create a table with a collection column and insert some data into it.
{code:sql}
CREATE KEYSPACE test WITH REPLICATION = {'class': 'SimpleStrategy', 
'replication_factor': 1};
CREATE TABLE test.test (key TEXT PRIMARY KEY, cas_target TEXT, some_collection 
MAP);
INSERT INTO test.test (key, cas_target, some_collection) VALUES ('key', 
'value', {}) IF NOT EXISTS;
{code}
# Flush the row to an sstable: {{nodetool flush}}.
# Update the row:
{code:sql}
UPDATE test.test SET cas_target = 'newvalue', some_collection = {} WHERE key = 
'key' IF cas_target = 'value';
{code}
# Drain the node: {{nodetool drain}}
# Stop the node, upgrade to 3.0.10, and start the node.
# Attempt to update the row again:
{code:sql}
UPDATE test.test SET cas_target = 'lastvalue' WHERE key = 'key' IF cas_target = 
'newvalue';
{code}
Using {{cqlsh}}, if the error is reproduced, the following output will be 
returned:
{code:sql}
$ ./cqlsh <<< "UPDATE test.test SET cas_target = 'newvalue', some_collection = 
{} WHERE key = 'key' IF cas_target = 'value';"
(start: 2016-12-22 10:14:27 EST)
:2:ReadFailure: Error from server: code=1300 [Replica(s) failed to 
execute read] message="Operation failed - received 0 responses and 1 failures" 
info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 
'consistency': 'QUORUM'}
{code}
and the following stack trace will be present in the system log:
{noformat}
WARN  15:14:28 Uncaught exception on thread 
Thread[SharedPool-Worker-10,10,main]: {}
java.lang.RuntimeException: java.lang.NullPointerException
at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2476)
 ~[main/:na]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_101]
at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
 ~[main/:na]
at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
 [main/:na]
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
[main/:na]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
Caused by: java.lang.NullPointerException: null
at 
org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:617)
 ~[main/:na]
at 
org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:569)
 ~[main/:na]
at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:220)
 ~[main/:na]
at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:159)
 ~[main/:na]
at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
at org.apache.cassandra.db.rows.Row$Merger.merge(Row.java:546) 
~[main/:na]
at 
org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterat

[jira] [Commented] (CASSANDRA-10956) Enable authentication of native protocol users via client certificates

2016-03-30 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-10956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15218031#comment-15218031
 ] 

Samuel Klock commented on CASSANDRA-10956:
--

Revisiting the native protocol's SASL implementation definitely sounds like the 
right solution, at least in the abstract.  If that can be done in 4.0 in a way 
that supports this authentication strategy, then I think we're comfortable 
closing this ticket.  (We could also have the ticket wait until the new SASL 
implementation is available, at which point we could consider contributing a 
new solution that works with it.)

It's worth noting that the EXTERNAL mechanism is potentially very open-ended, 
though; it's not clear to me what system state is potentially relevant or how 
Cassandra would expose it.  If it turns out that it's impractical to implement 
SASL in such a way that this strategy is supported, it may be wise to revisit 
the decision to close this ticket.

> Enable authentication of native protocol users via client certificates
> --
>
> Key: CASSANDRA-10956
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10956
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Attachments: 10956.patch
>
>
> Currently, the native protocol only supports user authentication via SASL.  
> While this is adequate for many use cases, it may be superfluous in scenarios 
> where clients are required to present an SSL certificate to connect to the 
> server.  If the certificate presented by a client is sufficient by itself to 
> specify a user, then an additional (series of) authentication step(s) via 
> SASL merely add overhead.  Worse, for uses wherein it's desirable to obtain 
> the identity from the client's certificate, it's necessary to implement a 
> custom SASL mechanism to do so, which increases the effort required to 
> maintain both client and server and which also duplicates functionality 
> already provided via SSL/TLS.
> Cassandra should provide a means of using certificates for user 
> authentication in the native protocol without any effort above configuring 
> SSL on the client and server.  Here's a possible strategy:
> * Add a new authenticator interface that returns {{AuthenticatedUser}} 
> objects based on the certificate chain presented by the client.
> * If this interface is in use, the user is authenticated immediately after 
> the server receives the {{STARTUP}} message.  It then responds with a 
> {{READY}} message.
> * Otherwise, the existing flow of control is used (i.e., if the authenticator 
> requires authentication, then an {{AUTHENTICATE}} message is sent to the 
> client).
> One advantage of this strategy is that it is backwards-compatible with 
> existing schemes; current users of SASL/{{IAuthenticator}} are not impacted.  
> Moreover, it can function as a drop-in replacement for SASL schemes without 
> requiring code changes (or even config changes) on the client side.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10956) Enable authentication of native protocol users via client certificates

2016-03-15 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-10956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15195354#comment-15195354
 ] 

Samuel Klock commented on CASSANDRA-10956:
--

Thanks Sam and Stefan.

bq. The reason for presenting this option in the yaml config is not really 
clear to me. It’s contrary to the idea of using the certificate authenticator.

To obtain the same behavior as {{NOT_REQUIRED}}, would it make sense to allow 
leaving {{authenticator}} unset?  Then it could default to {{null}}.  If we 
don't want to special-case {{null}} in the code, then 
{{NoOpCertificateAuthenticator}} could simply use {{OPTIONAL}} and always throw 
an {{AuthenticationException}} from {{authenticate()}}.

bq. I’d assume that authentication should be handled by providing a 
IAuthenticator implementation, but I can see how this is not a good fit here as 
we can’t provide any SASL support.

I think we'd be comfortable with Sam's scheme.  The main risk is that it can 
make authentication somewhat more complex: now there could be more than one way 
to authenticate to a role, and one way could take priority over other ways.  On 
the other hand, based on Stefan's comments, it sounds like there are use cases 
for schemes like this.

It's also worth noting that RFC 4422 specifies a mechanism that could support 
certificate authentication ([the EXTERNAL 
mechanism|https://tools.ietf.org/html/rfc4422#appendix-A]).  The obstacle to 
using EXTERNAL is that AFAICT Cassandra doesn't expose an interface to SASL 
authenticators for obtaining data about the context (e.g., whether TLS is in 
use and, if so, what certificates the client presented).  I think exposing such 
an interface would be a more general solution, but (at least at first glance) 
it could also be a significantly more complicated change.

> Enable authentication of native protocol users via client certificates
> --
>
> Key: CASSANDRA-10956
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10956
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Attachments: 10956.patch
>
>
> Currently, the native protocol only supports user authentication via SASL.  
> While this is adequate for many use cases, it may be superfluous in scenarios 
> where clients are required to present an SSL certificate to connect to the 
> server.  If the certificate presented by a client is sufficient by itself to 
> specify a user, then an additional (series of) authentication step(s) via 
> SASL merely add overhead.  Worse, for uses wherein it's desirable to obtain 
> the identity from the client's certificate, it's necessary to implement a 
> custom SASL mechanism to do so, which increases the effort required to 
> maintain both client and server and which also duplicates functionality 
> already provided via SSL/TLS.
> Cassandra should provide a means of using certificates for user 
> authentication in the native protocol without any effort above configuring 
> SSL on the client and server.  Here's a possible strategy:
> * Add a new authenticator interface that returns {{AuthenticatedUser}} 
> objects based on the certificate chain presented by the client.
> * If this interface is in use, the user is authenticated immediately after 
> the server receives the {{STARTUP}} message.  It then responds with a 
> {{READY}} message.
> * Otherwise, the existing flow of control is used (i.e., if the authenticator 
> requires authentication, then an {{AUTHENTICATE}} message is sent to the 
> client).
> One advantage of this strategy is that it is backwards-compatible with 
> existing schemes; current users of SASL/{{IAuthenticator}} are not impacted.  
> Moreover, it can function as a drop-in replacement for SASL schemes without 
> requiring code changes (or even config changes) on the client side.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10956) Enable authentication of native protocol users via client certificates

2016-02-19 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-10956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15154488#comment-15154488
 ] 

Samuel Klock commented on CASSANDRA-10956:
--

Thank you for the feedback.  I've left some replies to your comments on GitHub, 
and we'll plan to incorporate your feedback in a new version of the patch in 
the next few days.

Regarding anonymous authentication: would it be reasonable to make this 
behavior configurable? The intent is to enable operators to provide some level 
of access (perhaps read-only) to users who are not capable of authenticating. I 
do agree that it hardcoding this behavior in 
{{CommonNameCertificateAuthenticator}} probably isn't correct.

(It's also worth noting that the native protocol doesn't appear to support 
authentication at all for existing {{IAuthenticators}} that don't require 
authentication, so maybe {{ICertificateAuthenticator}} shouldn't support it 
either.)

> Enable authentication of native protocol users via client certificates
> --
>
> Key: CASSANDRA-10956
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10956
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Attachments: 10956.patch
>
>
> Currently, the native protocol only supports user authentication via SASL.  
> While this is adequate for many use cases, it may be superfluous in scenarios 
> where clients are required to present an SSL certificate to connect to the 
> server.  If the certificate presented by a client is sufficient by itself to 
> specify a user, then an additional (series of) authentication step(s) via 
> SASL merely add overhead.  Worse, for uses wherein it's desirable to obtain 
> the identity from the client's certificate, it's necessary to implement a 
> custom SASL mechanism to do so, which increases the effort required to 
> maintain both client and server and which also duplicates functionality 
> already provided via SSL/TLS.
> Cassandra should provide a means of using certificates for user 
> authentication in the native protocol without any effort above configuring 
> SSL on the client and server.  Here's a possible strategy:
> * Add a new authenticator interface that returns {{AuthenticatedUser}} 
> objects based on the certificate chain presented by the client.
> * If this interface is in use, the user is authenticated immediately after 
> the server receives the {{STARTUP}} message.  It then responds with a 
> {{READY}} message.
> * Otherwise, the existing flow of control is used (i.e., if the authenticator 
> requires authentication, then an {{AUTHENTICATE}} message is sent to the 
> client).
> One advantage of this strategy is that it is backwards-compatible with 
> existing schemes; current users of SASL/{{IAuthenticator}} are not impacted.  
> Moreover, it can function as a drop-in replacement for SASL schemes without 
> requiring code changes (or even config changes) on the client side.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10956) Enable authentication of native protocol users via client certificates

2015-12-30 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-10956:
-
Attachment: 10956.patch

Attaching a patch with a first-pass implementation of the proposal, along with 
a new concrete authenticator {{CommonNameCertificateAuthenticator}} that uses 
the CN field in the client certificate's subject as the Cassandra username.

> Enable authentication of native protocol users via client certificates
> --
>
> Key: CASSANDRA-10956
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10956
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Attachments: 10956.patch
>
>
> Currently, the native protocol only supports user authentication via SASL.  
> While this is adequate for many use cases, it may be superfluous in scenarios 
> where clients are required to present an SSL certificate to connect to the 
> server.  If the certificate presented by a client is sufficient by itself to 
> specify a user, then an additional (series of) authentication step(s) via 
> SASL merely add overhead.  Worse, for uses wherein it's desirable to obtain 
> the identity from the client's certificate, it's necessary to implement a 
> custom SASL mechanism to do so, which increases the effort required to 
> maintain both client and server and which also duplicates functionality 
> already provided via SSL/TLS.
> Cassandra should provide a means of using certificates for user 
> authentication in the native protocol without any effort above configuring 
> SSL on the client and server.  Here's a possible strategy:
> * Add a new authenticator interface that returns {{AuthenticatedUser}} 
> objects based on the certificate chain presented by the client.
> * If this interface is in use, the user is authenticated immediately after 
> the server receives the {{STARTUP}} message.  It then responds with a 
> {{READY}} message.
> * Otherwise, the existing flow of control is used (i.e., if the authenticator 
> requires authentication, then an {{AUTHENTICATE}} message is sent to the 
> client).
> One advantage of this strategy is that it is backwards-compatible with 
> existing schemes; current users of SASL/{{IAuthenticator}} are not impacted.  
> Moreover, it can function as a drop-in replacement for SASL schemes without 
> requiring code changes (or even config changes) on the client side.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-10956) Enable authentication of native protocol users via client certificates

2015-12-30 Thread Samuel Klock (JIRA)
Samuel Klock created CASSANDRA-10956:


 Summary: Enable authentication of native protocol users via client 
certificates
 Key: CASSANDRA-10956
 URL: https://issues.apache.org/jira/browse/CASSANDRA-10956
 Project: Cassandra
  Issue Type: New Feature
Reporter: Samuel Klock
Assignee: Samuel Klock


Currently, the native protocol only supports user authentication via SASL.  
While this is adequate for many use cases, it may be superfluous in scenarios 
where clients are required to present an SSL certificate to connect to the 
server.  If the certificate presented by a client is sufficient by itself to 
specify a user, then an additional (series of) authentication step(s) via SASL 
merely add overhead.  Worse, for uses wherein it's desirable to obtain the 
identity from the client's certificate, it's necessary to implement a custom 
SASL mechanism to do so, which increases the effort required to maintain both 
client and server and which also duplicates functionality already provided via 
SSL/TLS.

Cassandra should provide a means of using certificates for user authentication 
in the native protocol without any effort above configuring SSL on the client 
and server.  Here's a possible strategy:

* Add a new authenticator interface that returns {{AuthenticatedUser}} objects 
based on the certificate chain presented by the client.
* If this interface is in use, the user is authenticated immediately after the 
server receives the {{STARTUP}} message.  It then responds with a {{READY}} 
message.
* Otherwise, the existing flow of control is used (i.e., if the authenticator 
requires authentication, then an {{AUTHENTICATE}} message is sent to the 
client).

One advantage of this strategy is that it is backwards-compatible with existing 
schemes; current users of SASL/{{IAuthenticator}} are not impacted.  Moreover, 
it can function as a drop-in replacement for SASL schemes without requiring 
code changes (or even config changes) on the client side.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8550) Internal pagination in CQL3 index queries creating substantial overhead

2014-12-31 Thread Samuel Klock (JIRA)
Samuel Klock created CASSANDRA-8550:
---

 Summary: Internal pagination in CQL3 index queries creating 
substantial overhead
 Key: CASSANDRA-8550
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8550
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Samuel Klock


While benchmarking CQL3 secondary indexes in 2.1.2, we've noticed substantial 
performance degradation as the volume of indexed data increases.  In trying to 
figure out what's going on, we found that a major factor contributing to this 
degradation appears to be logic in 
{{o.a.c.db.index.composites.CompositesSearcher}} used to paginate scans of 
index tables.  In particular, in the use cases we've explored, this short 
algorithm used to select a page size appears to be the culprit:

{code:java}
private int meanColumns = 
Math.max(index.getIndexCfs().getMeanColumns(), 1);
// We shouldn't fetch only 1 row as this provides buggy paging in 
case the first row doesn't satisfy all clauses
private int rowsPerQuery = Math.max(Math.min(filter.maxRows(), 
filter.maxColumns() / meanColumns), 2);
{code}

In indexes where the cardinality doesn't scale linearly with the volume of data 
indexed, it seems likely that the value of {{meanColumns}} will steadily rise 
in write-heavy workloads.  In the cases we've explored, {{filter.maxColumns()}} 
returns a small enough number (related to the lesser of the native-protocol 
page size or the user-specified limit for the query) that, after 
{{meanColumns}} reaches a few thousand, {{rowsPerQuery}} (the page size) is 
consistently set to 2.

The resulting overhead is severe.  In our environment, if we fix 
{{rowsPerQuery}} to some reasonably large constant (e.g., 5,000), queries that 
with the existing logic would require over two minutes to complete can run in 
under ten seconds.

Using a constant clearly seems like the wrong answer.  But the overhead the 
existing algorithm seems to introduce suggests that it isn't the right answer 
either.  An intuitive solution might be to use the minimum of 
{{filter.maxRows()}} and {{filter.maxColumns()}} (or 2 if both of those are 1), 
but it's not immediately clear that there aren't safety considerations the 
algorithm is attempting to account for that this strategy does not.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8473) Secondary index support for key-value pairs in CQL3 maps

2014-12-19 Thread Samuel Klock (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14253589#comment-14253589
 ] 

Samuel Klock commented on CASSANDRA-8473:
-

Understood; your caution and your policy certainly make sense to us.  For the 
record, if you folks should change your mind at some point down the line, I 
think we'd be happy to assist with rebasing.

Thanks for the help!

> Secondary index support for key-value pairs in CQL3 maps
> 
>
> Key: CASSANDRA-8473
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8473
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Fix For: 3.0
>
> Attachments: cassandra-2.1-8473-actual-v1.txt, 
> cassandra-2.1-8473-v2.txt, cassandra-2.1-8473-v3.txt, cassandra-2.1-8473.txt, 
> trunk-8473-v2.txt, trunk-8473-v3.txt
>
>
> CASSANDRA-4511 and CASSANDRA-6383 made substantial progress on secondary 
> indexes on CQL3 maps, but support for a natural use case is still missing: 
> queries to find rows with map columns containing some key-value pair.  For 
> example (from a comment on CASSANDRA-4511):
> {code:sql}
> SELECT * FROM main.users WHERE notify['email'] = true;
> {code}
> Cassandra should add support for this kind of index.  One option is to expose 
> a CQL interface like the following:
> * Creating an index:
> {code:sql}
> cqlsh:mykeyspace> CREATE TABLE mytable (key TEXT PRIMARY KEY, value MAP TEXT>);
> cqlsh:mykeyspace> CREATE INDEX ON mytable(ENTRIES(value));
> {code}
> * Querying the index:
> {code:sql}
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('foo', {'a': '1', 
> 'b': '2', 'c': '3'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('bar', {'a': '1', 
> 'b': '4'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('baz', {'b': '4', 
> 'c': '3'});
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1';
>  key | value
> -+
>  bar |   {'a': '1', 'b': '4'}
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (2 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1' AND value['b'] 
> = '2' ALLOW FILTERING;
>  key | value
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '2' ALLOW 
> FILTERING;
>  key | value 
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows) 
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '4';
>  key | value
> -+--
>  bar | {'a': '1', 'b': '4'}
>  baz | {'b': '4', 'c': '3'}
> (2 rows)
> {code}
> A patch against the Cassandra-2.1 branch that implements this interface will 
> be attached to this issue shortly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8473) Secondary index support for key-value pairs in CQL3 maps

2014-12-18 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-8473:

Attachment: trunk-8473-v3.txt
cassandra-2.1-8473-v3.txt

Attaching new versions of the patch.  Details:

bq. Hmm, that error message indicates that the logic for frozen collections in 
{{toReceivers()}} still isn't quite correct. (...)

Added a test targeting the error message.  Adjusted the logic in the trunk 
patch to make it pass.

bq. (...) Frozen collections aren't tested with this method, and lists and sets 
only support CONTAINS. So you could go ahead and simplify this whole method to 
reflect that. (...)

Done.  All of the {{switch}} logic has been removed except for the {{MAP}} case.

bq. I think this could replace the switch statement (although I haven't tested 
it): (...)

This worked with some minor adjustments to the logic.  I attempted to refactor 
it for clarity at the expense of a little extra verbosity.  I also reverted the 
error message for unsupported indexes on frozen collections to breaking 
{{FrozenCollectionsTest}}.

bq. I would be okay with a common abstract superclass. I agree that the 
abstraction doesn't line up 100%.

Done via new class {{CompositesIndexIncludingCollectionKey}}.  Also: tweaked 
the code slightly to address a compiler warning.

bq. My preference is still to stick with a 3.0 target. (...)

Before we close this question, can you give us a sense of what your concerns 
are and how we might address them?  It's worth noting that we do have 
experience operating a couple of clusters running 2.1.x releases that include 
variants of this patch, so we're confident that it doesn't impact existing 
functionality.  Of course, our usage doesn't cover every possible case, but we 
might be able to take some additional steps to improve your confidence in the 
proposed implementation.

> Secondary index support for key-value pairs in CQL3 maps
> 
>
> Key: CASSANDRA-8473
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8473
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Fix For: 3.0
>
> Attachments: cassandra-2.1-8473-actual-v1.txt, 
> cassandra-2.1-8473-v2.txt, cassandra-2.1-8473-v3.txt, cassandra-2.1-8473.txt, 
> trunk-8473-v2.txt, trunk-8473-v3.txt
>
>
> CASSANDRA-4511 and CASSANDRA-6383 made substantial progress on secondary 
> indexes on CQL3 maps, but support for a natural use case is still missing: 
> queries to find rows with map columns containing some key-value pair.  For 
> example (from a comment on CASSANDRA-4511):
> {code:sql}
> SELECT * FROM main.users WHERE notify['email'] = true;
> {code}
> Cassandra should add support for this kind of index.  One option is to expose 
> a CQL interface like the following:
> * Creating an index:
> {code:sql}
> cqlsh:mykeyspace> CREATE TABLE mytable (key TEXT PRIMARY KEY, value MAP TEXT>);
> cqlsh:mykeyspace> CREATE INDEX ON mytable(ENTRIES(value));
> {code}
> * Querying the index:
> {code:sql}
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('foo', {'a': '1', 
> 'b': '2', 'c': '3'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('bar', {'a': '1', 
> 'b': '4'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('baz', {'b': '4', 
> 'c': '3'});
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1';
>  key | value
> -+
>  bar |   {'a': '1', 'b': '4'}
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (2 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1' AND value['b'] 
> = '2' ALLOW FILTERING;
>  key | value
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '2' ALLOW 
> FILTERING;
>  key | value 
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows) 
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '4';
>  key | value
> -+--
>  bar | {'a': '1', 'b': '4'}
>  baz | {'b': '4', 'c': '3'}
> (2 rows)
> {code}
> A patch against the Cassandra-2.1 branch that implements this interface will 
> be attached to this issue shortly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8473) Secondary index support for key-value pairs in CQL3 maps

2014-12-14 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-8473:

Attachment: trunk-8473-v2.txt
cassandra-2.1-8473-v2.txt

Attaching new versions of the patch (for 2.1 and trunk) that are intended to 
address your feedback.  Details:

* {{SingleColumnRelation}}
** bq. For lists, this error message may be confusing: 
{{checkTrue(receiver.type instanceof MapType, "Column \"%s\" cannot be used as 
a map", receiver.name);}}. For example, if you do {{WHERE mylist\[0\] = 
'foo'}}, you're not really trying to use it as a map. You may want to handle 
lists specially.
Done: lists get a different error message now.  (In the 2.1 patch, this change 
is made in {{SelectStatement}}.)
** bq. The {{checkFalse()}} statement below this is starting to get confusing, 
I would break it up
Agreed.  Refactored the condition into a couple of extracted methods.  (In the 
2.1 patch, a similar change is made in {{SelectStatement}}.)
** bq. Use curly braces on the "if" clause when they're used on the "else"
Done in trunk patch.  No equivalent issue in 2.1 patch.
** bq. I'm not sure that frozen maps are handled correctly here (e.g. WHERE 
myfrozenmap\['foo'\] = 'bar'). May want to double-check that.
Added a test.  I think trunk is okay (although the error message is imperfect: 
"Invalid STRING constant (foo) for "myfrozenmap" of type frozen>").  The 2.1 patch incorrectly gave no error for queries with this 
condition; that's been fixed.
* {{SingleColumnRelation.Contains()}}
** bq. Update class-level comment (should be a javadoc) to include map entry 
restrictions
Updated the comment in the trunk patch (it was already updated in the 2.1 
version).  Making it a Javadoc would be inconsistent with the other nested 
classes in {{SingleColumnRestriction}} (which have no Javadoc), so I'm 
reluctant to do so.  But I wouldn't forcefully object.
** bq. entries(): no need for curly braces with a single-line for-loop
Moot because the for-loop is now multiline.  No equivalent issue in the 2.1 
patch.
* {{CreateIndexStatement}}
** bq. switch on target.type could be clearer if re-organized; also, the error 
message about 'keys' is slightly misleading for 'entries' indexes
Corrected both error messages.  It's not obvious to me how to reorganize the 
{{switch}} to make it clearer (very likely because I wrote it) -- did you have 
something specific in mind?
* {{IndexTarget.TargetType.fromIndexOptions()}}
** bq. Should this return FULL if index_values isn't present? Also, no curlies 
needed for single-line clauses.
Removed the braces.  On the return value: no index options will be present if 
the column isn't a non-frozen collection.  Even so, this is a correctness 
issue: the v1 patch would yield the wrong error message if a user attempted to 
create a {{FULL}} index on a frozen map that already had one.  Fixed in v2.
* {{ExtendedFilter}}
** bq. {{else if (expr.isContains())}} will always be false (due to the 
{{isContains()}} check above).
The block has been removed.  Note that it was an attempt to preserve the logic 
that existed in the {{MAP}} case previously if {{expr.isContainsKey()}} were 
false; since the only kind of expressions apart from {{CONTAINS KEY}} that were 
valid for map columns were {{CONTAINS}} relations, this seemed like the right 
choice.  But your analysis appears to be correct.  Is there some other way that 
code would have been reachable?  (It _looks_ like the code may have been 
intended to check {{map\[key\] = value}} conditions, but AFAIK there would have 
been no way to trigger its execution.)
* {{CompositesIndex}}
** bq. No nested ternaries, please
Rewritten as {{if}} statements.
* {{CompositesIndexOnCollectionKeyAndValue}}
** bq. makeIndexColumnPrefix(): need to use {{min(count - 1, cellName.size())}} 
for loop end (see CASSANDRA-8053 for why)
Fixed by extending {{CompositesIndexOnCollectionKey}}.  (Also, did you mean 
CASSANDRA-8073?)
** bq. by extending CompositesIndexOnCollectionKey, you could eliminate about 
half of the methods
Done, although I'm not positive the abstractions are quite right (is 
{{CompositesIndexOnCollectionKeyAndValue}} really a kind of 
{{CompositesIndexOnCollectionKey}}?  Would it be better to use a common 
superclass?).  But the reduced duplication is very nice.
** bq. isStale(): instead of building a new composite to compare with the index 
entry key, why not compare the cell value with the second item in the index 
entry composite? This method could also use a comment or two
Good point -- the implementation no longer builds a new composite for the 
comparison.  I've also refactored the code for clarity; if you still think it 
needs comments, I'll certainly add them.  (Also: fixed the return value for 
when the cell in the indexed table is dead.)
* {{SecondaryIndexOnMapEntriesTest}}
** bq. Unusued and commented out imports
Removed the co

[jira] [Updated] (CASSANDRA-8473) Secondary index support for key-value pairs in CQL3 maps

2014-12-12 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-8473:

Attachment: cassandra-2.1-8473-actual-v1.txt

bq. Also, I noticed that your patch applies to trunk (despite the name). I do 
feel like it would be better to target 3.0 than 2.1 for this, so I'm going to 
change the fixVersion to 3.0 unless there are strong objections.

Sorry about that!  Bookkeeping error on my part.  I'm attaching a version of 
the patch that should be based against 2.1 (as intended).  I haven't yet 
addressed your (very good) feedback, but I will do so in the next day or two.  
The logic in both patch versions is very similar, so most of your feedback 
should apply to them both.  I'll update both patches to reflect your 
observations.

Regarding the fixVersion: the folks in my organization would definitely vote 
for a 2.1.x target if that's feasible.  We have a use case for this 
functionality that we're planning to deploy to production in the next few 
months, and we plan to do so using a 2.1.x release.  We certainly have the 
option of using the 2.1 version of patch against our internal Cassandra 
project, but of course we would prefer not to do so.

> Secondary index support for key-value pairs in CQL3 maps
> 
>
> Key: CASSANDRA-8473
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8473
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Samuel Klock
>Assignee: Samuel Klock
> Fix For: 3.0
>
> Attachments: cassandra-2.1-8473-actual-v1.txt, cassandra-2.1-8473.txt
>
>
> CASSANDRA-4511 and CASSANDRA-6383 made substantial progress on secondary 
> indexes on CQL3 maps, but support for a natural use case is still missing: 
> queries to find rows with map columns containing some key-value pair.  For 
> example (from a comment on CASSANDRA-4511):
> {code:sql}
> SELECT * FROM main.users WHERE notify['email'] = true;
> {code}
> Cassandra should add support for this kind of index.  One option is to expose 
> a CQL interface like the following:
> * Creating an index:
> {code:sql}
> cqlsh:mykeyspace> CREATE TABLE mytable (key TEXT PRIMARY KEY, value MAP TEXT>);
> cqlsh:mykeyspace> CREATE INDEX ON mytable(ENTRIES(value));
> {code}
> * Querying the index:
> {code:sql}
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('foo', {'a': '1', 
> 'b': '2', 'c': '3'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('bar', {'a': '1', 
> 'b': '4'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('baz', {'b': '4', 
> 'c': '3'});
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1';
>  key | value
> -+
>  bar |   {'a': '1', 'b': '4'}
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (2 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1' AND value['b'] 
> = '2' ALLOW FILTERING;
>  key | value
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '2' ALLOW 
> FILTERING;
>  key | value 
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows) 
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '4';
>  key | value
> -+--
>  bar | {'a': '1', 'b': '4'}
>  baz | {'b': '4', 'c': '3'}
> (2 rows)
> {code}
> A patch against the Cassandra-2.1 branch that implements this interface will 
> be attached to this issue shortly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8473) Secondary index support for key-value pairs in CQL3 maps

2014-12-12 Thread Samuel Klock (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samuel Klock updated CASSANDRA-8473:

Attachment: cassandra-2.1-8473.txt

Attaching proposed patch.

> Secondary index support for key-value pairs in CQL3 maps
> 
>
> Key: CASSANDRA-8473
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8473
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Samuel Klock
> Attachments: cassandra-2.1-8473.txt
>
>
> CASSANDRA-4511 and CASSANDRA-6383 made substantial progress on secondary 
> indexes on CQL3 maps, but support for a natural use case is still missing: 
> queries to find rows with map columns containing some key-value pair.  For 
> example (from a comment on CASSANDRA-4511):
> {code:sql}
> SELECT * FROM main.users WHERE notify['email'] = true;
> {code}
> Cassandra should add support for this kind of index.  One option is to expose 
> a CQL interface like the following:
> * Creating an index:
> {code:sql}
> cqlsh:mykeyspace> CREATE TABLE mytable (key TEXT PRIMARY KEY, value MAP TEXT>);
> cqlsh:mykeyspace> CREATE INDEX ON mytable(ENTRIES(value));
> {code}
> * Querying the index:
> {code:sql}
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('foo', {'a': '1', 
> 'b': '2', 'c': '3'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('bar', {'a': '1', 
> 'b': '4'});
> cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('baz', {'b': '4', 
> 'c': '3'});
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1';
>  key | value
> -+
>  bar |   {'a': '1', 'b': '4'}
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (2 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1' AND value['b'] 
> = '2' ALLOW FILTERING;
>  key | value
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows)
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '2' ALLOW 
> FILTERING;
>  key | value 
> -+
>  foo | {'a': '1', 'b': '2', 'c': '3'}
> (1 rows) 
> cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '4';
>  key | value
> -+--
>  bar | {'a': '1', 'b': '4'}
>  baz | {'b': '4', 'c': '3'}
> (2 rows)
> {code}
> A patch against the Cassandra-2.1 branch that implements this interface will 
> be attached to this issue shortly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8473) Secondary index support for key-value pairs in CQL3 maps

2014-12-12 Thread Samuel Klock (JIRA)
Samuel Klock created CASSANDRA-8473:
---

 Summary: Secondary index support for key-value pairs in CQL3 maps
 Key: CASSANDRA-8473
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8473
 Project: Cassandra
  Issue Type: Improvement
Reporter: Samuel Klock


CASSANDRA-4511 and CASSANDRA-6383 made substantial progress on secondary 
indexes on CQL3 maps, but support for a natural use case is still missing: 
queries to find rows with map columns containing some key-value pair.  For 
example (from a comment on CASSANDRA-4511):

{code:sql}
SELECT * FROM main.users WHERE notify['email'] = true;
{code}

Cassandra should add support for this kind of index.  One option is to expose a 
CQL interface like the following:

* Creating an index:
{code:sql}
cqlsh:mykeyspace> CREATE TABLE mytable (key TEXT PRIMARY KEY, value MAP);
cqlsh:mykeyspace> CREATE INDEX ON mytable(ENTRIES(value));
{code}
* Querying the index:
{code:sql}
cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('foo', {'a': '1', 
'b': '2', 'c': '3'});
cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('bar', {'a': '1', 
'b': '4'});
cqlsh:mykeyspace> INSERT INTO mytable (key, value) VALUES ('baz', {'b': '4', 
'c': '3'});
cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1';

 key | value
-+
 bar |   {'a': '1', 'b': '4'}
 foo | {'a': '1', 'b': '2', 'c': '3'}

(2 rows)
cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['a'] = '1' AND value['b'] = 
'2' ALLOW FILTERING;

 key | value
-+
 foo | {'a': '1', 'b': '2', 'c': '3'}

(1 rows)
cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '2' ALLOW FILTERING;

 key | value 
-+
 foo | {'a': '1', 'b': '2', 'c': '3'}

(1 rows) 
cqlsh:mykeyspace> SELECT * FROM mytable WHERE value['b'] = '4';

 key | value
-+--
 bar | {'a': '1', 'b': '4'}
 baz | {'b': '4', 'c': '3'}

(2 rows)
{code}

A patch against the Cassandra-2.1 branch that implements this interface will be 
attached to this issue shortly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)