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

Sylvain Lebresne commented on CASSANDRA-6586:
---------------------------------------------

Yes, all columns of a CQL row are read even if only a subset of them is 
selected. Basically, the reason is that we have to do that because of 
CASSANDRA-5762, and more generally to enforce proper CQL semantic.

bq. This is quite counter intuitive behaviour, as all Cassandra guides said, 
that using wide rows should not affect performance, but in CQL3 they create big 
performance bottleneck

That part is a misunderstanding of CQL3. "wide rows" refers to the thrift 
notion of using a 'thrift row' that has lots of columns added dynamically. CQL3 
does handle the exact same use cases, but the "presentation" is not the exact 
same one. In particular, CQL rows are *not* the equivalent to 'thrift rows' in 
general and 'wide CQL rows' is not a thing. You can read 
http://www.datastax.com/dev/blog/does-cql-support-dynamic-columns-wide-rows for 
instance for more details.

In fact, it could be argued that this is *less* of a problem for "wide rows" 
usage, as when you do use a "wide row", most of your queries will be slice 
queries and in that case, there is really no way around reading it all between 
the slice start and end bound (this is not specific to CQL in any way in 
particular).

CQL rows are not expected to grow very large (that includes collections btw) 
and technically speaking, C* don't read one column at a time from disk anyway.  
So don't get me wrong: it would be nice if we had an easy way to optimize this 
out. But that in practice, I doubt this will "create big performance 
bottleneck" unless you are doing something rather specific. This does mean that 
if you plan on mixing some (CQL) columns with small values and some with large 
ones (blobs), and that you very often need to only query the small values 
alone, it might be a better idea to separate small and large values in separate 
table (or wait for and use CASSANDRA-6588).

In terms of solutions, as said on CASSANDRA-5762, I don't see a way to optimize 
this out in the general case. But CASSANDRA-6588 is probably a good solution 
for when such thing matter.


> Cassandra touches all columns on CQL3 select
> --------------------------------------------
>
>                 Key: CASSANDRA-6586
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6586
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Jan Chochol
>
> It seems that Cassandra is checking (garbage collecting) all columns of all 
> returned rows, despite the fact that not all columns are requested.
> Example:
> * use following script to fill Cassandra with test data:
> {noformat}
> perl -e "print(\"DROP KEYSPACE t;\nCREATE KEYSPACE t WITH replication = 
> {'class': 'SimpleStrategy', 'replication_factor' : 1};\nuse t;\nCREATE TABLE 
> t (a varchar PRIMARY KEY, b varchar, c varchar, d varchar);\nCREATE INDEX t_b 
> ON t (b);\nCREATE INDEX t_c ON t (c);\nCREATE INDEX t_d ON t (d);\n\");\$max 
> = 200; for(\$i = 0; \$i < \$max; \$i++) { \$j = int(\$i * 10 / \$max); \$k = 
> int(\$i * 100 / \$max); print(\"INSERT INTO t (a, b, c, d) VALUES ('a\$i', 
> 'b\$j', 'c\$k', 'd\$i');\n\")}\n" | cqlsh
> {noformat}
> * turn on {{ALL}} logging for Cassandra
> * issue this query:
> {noformat}
> select a from t where c = 'c1';
> {noformat}
> This is result:
> {noformat}
> [root@jch3-devel:~/c4] cqlsh --no-color
> Connected to C4 Cluster Single at localhost:9160.
> [cqlsh 3.1.7 | Cassandra 1.2.11-SNAPSHOT | CQL spec 3.0.0 | Thrift protocol 
> 19.36.1]
> Use HELP for help.
> cqlsh> use t;
> cqlsh:t> select a from t where c = 'c1';
>  a
> ----
>  a3
>  a2
> {noformat}
> From Cassandra log:
> {noformat}
> 2014-01-15 09:14:56.663+0100 [Thrift:1] [TRACE] QueryProcessor.java(125) 
> org.apache.cassandra.cql3.QueryProcessor: component=c4 Process 
> org.apache.cassandra.cql3.statements.SelectStatement@614b3189 @CL.ONE
> 2014-01-15 09:14:56.810+0100 [Thrift:1] [TRACE] ReadCallback.java(67) 
> org.apache.cassandra.service.ReadCallback: component=c4 Blockfor is 1; 
> setting up requests to /127.0.0.1
> 2014-01-15 09:14:56.816+0100 [ReadStage:2] [DEBUG] 
> CompositesSearcher.java(112) 
> org.apache.cassandra.db.index.composites.CompositesSearcher: component=c4 
> Most-selective indexed predicate is 't.c EQ c1'
> 2014-01-15 09:14:56.817+0100 [ReadStage:2] [TRACE] 
> ColumnFamilyStore.java(1493) org.apache.cassandra.db.ColumnFamilyStore: 
> component=c4 Filtering 
> org.apache.cassandra.db.index.composites.CompositesSearcher$1@e15911 for rows 
> matching 
> org.apache.cassandra.db.filter.ExtendedFilter$FilterWithCompositeClauses@4a9e6b8a
> 2014-01-15 09:14:56.817+0100 [ReadStage:2] [TRACE] 
> CompositesSearcher.java(237) 
> org.apache.cassandra.db.index.composites.CompositesSearcher: component=c4 
> Scanning index 't.c EQ c1' starting with 
> 2014-01-15 09:14:56.820+0100 [ReadStage:2] [TRACE] SSTableReader.java(776) 
> org.apache.cassandra.io.sstable.SSTableReader: component=c4 Adding cache 
> entry for KeyCacheKey(/mnt/ebs/cassandra/data/t/t/t-t.t_c-ic-1, 6331) -> 
> org.apache.cassandra.db.RowIndexEntry@66a6574b
> 2014-01-15 09:14:56.821+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 0 of 
> 10000: 6133:false:0@1389773577394000
> 2014-01-15 09:14:56.821+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 10000: 6132:false:0@1389773577391000
> 2014-01-15 09:14:56.822+0100 [ReadStage:2] [TRACE] 
> CompositesSearcher.java(313) 
> org.apache.cassandra.db.index.composites.CompositesSearcher: component=c4 
> Adding index hit to current row for 6133
> 2014-01-15 09:14:56.825+0100 [ReadStage:2] [TRACE] SSTableReader.java(776) 
> org.apache.cassandra.io.sstable.SSTableReader: component=c4 Adding cache 
> entry for KeyCacheKey(/mnt/ebs/cassandra/data/t/t/t-t-ic-1, 6133) -> 
> org.apache.cassandra.db.RowIndexEntry@32ad3193
> 2014-01-15 09:14:56.826+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 0 of 
> 2147483647: :false:0@1389773577394000
> 2014-01-15 09:14:56.826+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 2147483647: b:false:2@1389773577394000
> 2014-01-15 09:14:56.826+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 2147483647: c:false:2@1389773577394000
> 2014-01-15 09:14:56.826+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 2147483647: d:false:2@1389773577394000
> 2014-01-15 09:14:56.828+0100 [ReadStage:2] [TRACE] 
> CompositesSearcher.java(313) 
> org.apache.cassandra.db.index.composites.CompositesSearcher: component=c4 
> Adding index hit to current row for 6132
> 2014-01-15 09:14:56.828+0100 [ReadStage:2] [TRACE] SSTableReader.java(776) 
> org.apache.cassandra.io.sstable.SSTableReader: component=c4 Adding cache 
> entry for KeyCacheKey(/mnt/ebs/cassandra/data/t/t/t-t-ic-1, 6132) -> 
> org.apache.cassandra.db.RowIndexEntry@87d66d5
> 2014-01-15 09:14:56.829+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 0 of 
> 2147483647: :false:0@1389773577391000
> 2014-01-15 09:14:56.829+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 2147483647: b:false:2@1389773577391000
> 2014-01-15 09:14:56.829+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 2147483647: c:false:2@1389773577391000
> 2014-01-15 09:14:56.829+0100 [ReadStage:2] [TRACE] SliceQueryFilter.java(164) 
> org.apache.cassandra.db.filter.SliceQueryFilter: component=c4 collecting 1 of 
> 2147483647: d:false:2@1389773577391000
> 2014-01-15 09:14:56.829+0100 [ReadStage:2] [TRACE] 
> CompositesSearcher.java(232) 
> org.apache.cassandra.db.index.composites.CompositesSearcher: component=c4 
> Read only 2 (< 10000) last page through, must be done
> 2014-01-15 09:14:56.829+0100 [ReadStage:2] [TRACE] 
> CompositesSearcher.java(232) 
> org.apache.cassandra.db.index.composites.CompositesSearcher: component=c4 
> Read only 2 (< 10000) last page through, must be done
> 2014-01-15 09:14:56.830+0100 [Thrift:1] [DEBUG] Tracing.java(169) 
> org.apache.cassandra.tracing.Tracing: component=c4 request complete
> {noformat}
> Note that Cassandra is checking all columns ({{a}}, {{b}}, {{c}} and {{d}}), 
> even we requested only column {{a}}.
> Things became really nasty, when using lots of columns, or bigger collections 
> (yes - each member of collection is checked).
> This is quite counter intuitive behaviour, as all Cassandra guides said, that 
> using wide rows should not affect performance, but in CQL3 they create big 
> performance bottleneck.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to