You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2013/12/03 16:44:21 UTC

[jira] [Updated] (CASSANDRA-6428) Use 4 bytes to encode collection size in next native protocol version

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

Sylvain Lebresne updated CASSANDRA-6428:
----------------------------------------

    Summary: Use 4 bytes to encode collection size in next native protocol version  (was: Inconsistency in CQL native protocol)

> Use 4 bytes to encode collection size in next native protocol version
> ---------------------------------------------------------------------
>
>                 Key: CASSANDRA-6428
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6428
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Jan Chochol
>
> We are trying to use Cassandra CQL3 collections (sets and maps) for denormalizing data.
> Problem is, when size of these collections go above some limit. We found that current limitation is 64k - 1 (65535) items in collection.
> We found that there is inconsistency in CQL binary protocol (all current available versions). 
> In protocol (for set) there are these fields:
> {noformat}
> [value size: int] [items count: short] [items] ...
> {noformat}
> One example in our case (collection with 65536 elements):
> {noformat}
> 00 21 ff ee 00 00 00 20 30 30 30 30 35 63 38 69 65 33 67 37 73 61 ...
> {noformat}
> So decode {{value size}} is 1245166 bytes and {{items count}} is 0.
> This is wrong - you can not have collection with 0 items occupying more than 1MB.
> I understand that in unsigned short you can not have more than 65535, but I do not understand why there is such limitation in protocol, when all data are currently sent.
> In this case we have several possibilities:
> * ignore {{items count}} field and read all bytes specified in {{value size}}
> ** there is problem that we can not be sure, that this behaviour will be kept over for future versions of Cassandra, as it is quite strange
> * refactor our code to use only small collections (this seems quite odd, as Cassandra has no problems with wide rows)
> * do not use collections, and fall-back to net wide rows
> * wait for change in protocol for removing unnecessary limitation



--
This message was sent by Atlassian JIRA
(v6.1#6144)