[jira] [Commented] (CASSANDRA-9540) Cql query doesn't return right information when using IN on columns for some keys

2015-06-04 Thread Mathijs Vogelzang (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-9540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14572973#comment-14572973
 ] 

Mathijs Vogelzang commented on CASSANDRA-9540:
--

Another observation: when testing this bug from our java code with the datastax 
cql driver, we notice that while the written data is still in the memtable, 
everything works correctly. Once its flushed to an SSTable on disk, the 
unpredictable behavior starts. Maybe this bug is another argument for 
https://issues.apache.org/jira/browse/CASSANDRA-9161 ?

 Cql query doesn't return right information when using IN on columns for some 
 keys
 -

 Key: CASSANDRA-9540
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9540
 Project: Cassandra
  Issue Type: Bug
  Components: API
 Environment: Cassandra 2.1.5
Reporter: Mathijs Vogelzang
Assignee: Carl Yeksigian
 Fix For: 2.1.x


 We are investigating a weird issue where one of our clients doesn't get data 
 on his dashboard. It seems Cassandra is not returning data for a particular 
 key (brokenkey from now on).
 Some background:
 We have a row where we store a metadata column and data in columns 
 bucket/0, bucket/1, bucket/2, etc. Depending on the date selection of 
 the UI, we know that we only need to retrieve bucket/0, bucket/0 and bucket/1 
 etc. (we always need to retrieve metadata).
 A typical query may look like this (using SELECT column1 to just show what is 
 returned, normally we would of course do SELECT value):
 {noformat}
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/workingkey');
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/brokenkey');
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 {noformat}
 These two queries work as expected, and return the information that we 
 actually stored.
 However, when we filter for certain columns, the brokenkey starts behaving 
 very weird:
 {noformat}
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/workingkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'));
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/workingkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'),textAsBlob('asdfasdfasdf'));
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 ***  As expected, querying for more information doesn't really matter for the 
 working key ***
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/brokenkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'));
  blobAsText(column1)
 -
 bucket/0
 (1 rows)
 *** Cassandra stops giving us the metadata column when asking for a few more 
 columns! ***
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/brokenkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'),textAsBlob('asdfasdfasdf'));
  key | column1 | value
 -+-+---
 (0 rows)
 *** Adding the bogus column name even makes it return nothing from this row 
 anymore! ***
 {noformat}
 There are at least two rows that malfunction like this in our table (which is 
 quite old already and has gone through a bunch of Cassandra upgrades). I've 
 upgraded our whole cluster to 2.1.5 (we were on 2.1.2 when I discovered this 
 problem) and compacted, repaired and scrubbed this column family, which 
 hasn't helped.
 Our table structure is:
 {noformat}
 cqlsh:AppBrain describe table GroupedSeries;
 CREATE TABLE AppBrain.GroupedSeries (
 key blob,
 column1 blob,
 value blob,
 PRIMARY KEY (key, column1)
 ) WITH COMPACT STORAGE
 AND CLUSTERING ORDER BY (column1 ASC)
 AND caching = '{keys:ALL, rows_per_partition:NONE}'
 AND comment = ''
 AND compaction = {'min_threshold': '4', 'class': 
 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 
 'max_threshold': '32'}
 AND compression = {'sstable_compression': 
 'org.apache.cassandra.io.compress.LZ4Compressor'}
 AND dclocal_read_repair_chance = 0.1
 AND default_time_to_live = 0
 AND gc_grace_seconds = 864000
 AND max_index_interval 

[jira] [Commented] (CASSANDRA-9540) Cql query doesn't return right information when using IN on columns for some keys

2015-06-04 Thread Mathijs Vogelzang (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-9540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14573093#comment-14573093
 ] 

Mathijs Vogelzang commented on CASSANDRA-9540:
--

And one final observation from our testing: it seems the bug happens as soon as 
a cell value exceeds 64kb in size. (My earlier comment was wrong, there were 
144,000 2-byte hex chars in the json file, so the cell size is roughly 70kb 
instead of 144kb).

This bug is very difficult for us to work around (we're using the IN query as a 
replacement for the former thrift slice query to get specific column values), 
as we currently can not trust cassandra to return values that we wrote into it 
earlier, depending on whether our queried columns are present or not. If 
there's any workaround how we can query a set of columns using CQL that works, 
please let me know.

 Cql query doesn't return right information when using IN on columns for some 
 keys
 -

 Key: CASSANDRA-9540
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9540
 Project: Cassandra
  Issue Type: Bug
  Components: API
 Environment: Cassandra 2.1.5
Reporter: Mathijs Vogelzang
Assignee: Carl Yeksigian
 Fix For: 2.1.x


 We are investigating a weird issue where one of our clients doesn't get data 
 on his dashboard. It seems Cassandra is not returning data for a particular 
 key (brokenkey from now on).
 Some background:
 We have a row where we store a metadata column and data in columns 
 bucket/0, bucket/1, bucket/2, etc. Depending on the date selection of 
 the UI, we know that we only need to retrieve bucket/0, bucket/0 and bucket/1 
 etc. (we always need to retrieve metadata).
 A typical query may look like this (using SELECT column1 to just show what is 
 returned, normally we would of course do SELECT value):
 {noformat}
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/workingkey');
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/brokenkey');
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 {noformat}
 These two queries work as expected, and return the information that we 
 actually stored.
 However, when we filter for certain columns, the brokenkey starts behaving 
 very weird:
 {noformat}
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/workingkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'));
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/workingkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'),textAsBlob('asdfasdfasdf'));
  blobAsText(column1)
 -
 bucket/0
 metadata
 (2 rows)
 ***  As expected, querying for more information doesn't really matter for the 
 working key ***
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/brokenkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'));
  blobAsText(column1)
 -
 bucket/0
 (1 rows)
 *** Cassandra stops giving us the metadata column when asking for a few more 
 columns! ***
 cqlsh:AppBrain select blobAsText(column1) from GroupedSeries where 
 key=textAsBlob('install/brokenkey') and column1 IN 
 (textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'),textAsBlob('asdfasdfasdf'));
  key | column1 | value
 -+-+---
 (0 rows)
 *** Adding the bogus column name even makes it return nothing from this row 
 anymore! ***
 {noformat}
 There are at least two rows that malfunction like this in our table (which is 
 quite old already and has gone through a bunch of Cassandra upgrades). I've 
 upgraded our whole cluster to 2.1.5 (we were on 2.1.2 when I discovered this 
 problem) and compacted, repaired and scrubbed this column family, which 
 hasn't helped.
 Our table structure is:
 {noformat}
 cqlsh:AppBrain describe table GroupedSeries;
 CREATE TABLE AppBrain.GroupedSeries (
 key blob,
 column1 blob,
 value blob,
 PRIMARY KEY (key, column1)
 ) WITH COMPACT STORAGE
 AND CLUSTERING ORDER BY (column1 ASC)
 AND caching = '{keys:ALL, rows_per_partition:NONE}'
 AND comment = ''
 AND compaction = {'min_threshold': '4', 'class': 
 

[jira] [Commented] (CASSANDRA-9540) Cql query doesn't return right information when using IN on columns for some keys

2015-06-04 Thread Mathijs Vogelzang (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-9540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14572659#comment-14572659
 ] 

Mathijs Vogelzang commented on CASSANDRA-9540:
--

I was able to make this case reproducible by getting the key from our 
production database and saving it with sstable2json.
It seems that the broken keys have one somewhat bigger cell (144 kB in this 
case).

A testcase json file is available at 
https://www.dropbox.com/s/kzu2jmqmwz788k8/testcase.json?dl=0 (the SSTable that 
it creates is available at 
https://www.dropbox.com/s/ilwpjka5r70n2os/test-testbug-ka-2-Data.db?dl=0 )

The commands that I executed are in cqlsh:
{noformat}
create keyspace test with 
replication={'class':'SimpleStrategy','replication_factor':1};
use test;
CREATE TABLE test.testbug ( key blob, column1 blob, value blob, 
PRIMARY KEY (key, column1) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY 
(column1 ASC) AND bloom_filter_fp_chance = 0.01 AND caching = 
'{keys:ALL, rows_per_partition:NONE}' AND comment = '' AND 
compaction = {'min_threshold': '4', 'class': 
'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 
'max_threshold': '32'} AND compression = {'sstable_compression': 
'org.apache.cassandra.io.compress.LZ4Compressor'} AND 
dclocal_read_repair_chance = 0.1 AND default_time_to_live = 0 AND 
gc_grace_seconds = 864000 AND max_index_interval = 2048 AND 
memtable_flush_period_in_ms = 0 AND min_index_interval = 128 AND 
read_repair_chance = 1.0 AND speculative_retry = 'NONE';
{noformat}
Then I injected the json as follows on the commandline:
{noformat}
json2sstable -K test -c testbug testcase.json 
DATADIR/test/testbug/test-testbug-ka-1-Data.db
nodetool refresh test testbug
{noformat}

The cqlsh then behaves as reported earlier, where querying for row test 
behaves as expected, but for row broken we don't get any data when we ask for 
columns that don't exist in that row (depending on the exact set of asked for 
columns, either 'metadata' is dropped, or no data is returned at all):
{noformat}
cqlsh:test select blobAsText(key),blobAsText(column1) from testbug where 
key=textAsBlob('test') and column1 in 
(textAsBlob('metadata'),textAsBlob('bucket/0'));

 blobAsText(key) | blobAsText(column1)
-+-
test |bucket/0
test |metadata

(2 rows)
cqlsh:test select blobAsText(key),blobAsText(column1) from testbug where 
key=textAsBlob('broken') and column1 in 
(textAsBlob('metadata'),textAsBlob('bucket/0'));

 blobAsText(key) | blobAsText(column1)
-+-
  broken |bucket/0
  broken |metadata

(2 rows)
cqlsh:test select blobAsText(key),blobAsText(column1) from testbug where 
key=textAsBlob('test') and column1 in 
(textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'),textAsBlob('asdfasdf'));

 blobAsText(key) | blobAsText(column1)
-+-
test |bucket/0
test |metadata

(2 rows)
cqlsh:test select blobAsText(key),blobAsText(column1) from testbug where 
key=textAsBlob('broken') and column1 in 
(textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'),textAsBlob('bucket/2'),textAsBlob('asdfasdf'));

 key | column1 | value
-+-+---

(0 rows)
cqlsh:test select blobAsText(key),blobAsText(column1) from testbug where 
key=textAsBlob('broken') and column1 in 
(textAsBlob('metadata'),textAsBlob('bucket/0'),textAsBlob('bucket/1'));

 blobAsText(key) | blobAsText(column1)
-+-
  broken |bucket/0

(1 rows)
{noformat}

 Cql query doesn't return right information when using IN on columns for some 
 keys
 -

 Key: CASSANDRA-9540
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9540
 Project: Cassandra
  Issue Type: Bug
  Components: API
 Environment: Cassandra 2.1.5
Reporter: Mathijs Vogelzang

 We are investigating a weird issue where one of our clients doesn't get data 
 on his dashboard. It seems Cassandra is not returning data for a particular 
 key (brokenkey from now on).
 Some background:
 We have a row where we store a metadata column and data in columns 
 bucket/0, bucket/1, bucket/2, etc. Depending on the date selection of 
 the UI, we know that we only need to retrieve bucket/0, bucket/0 and bucket/1 
 etc. (we always need to retrieve metadata).
 A typical query may look like this (using SELECT column1 to just show what is 
 returned, normally we would of course do SELECT value):
 {noformat}
 cqlsh:AppBrain select blobAsText(column1) from