cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stefania (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-12516) Interned column identifiers can be overridden incorrectly
Date Wed, 31 Aug 2016 09:10:22 GMT

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

Stefania commented on CASSANDRA-12516:
--------------------------------------

I don't think we have a problem in 2.1 and 2.2 because the interned maps are confined to {{CompoundSparseCellNameType}}
and {{SimpleSparseCellNameType}} and in both cases they are not static, so they are limited
to the scope of a column family. We also map the column identifier byte buffers to column
definitions in {{CFMetadata.columnMetadata}} and so we'd have a much bigger issue here but
we don't because we stay within the scope of a column family, with a single comparator, and
the column identifier byte buffers are created consistently.

Interned identifiers became a problem in 3.0 when a static map was added to {{ColumnIdentifier}},
which spans across column families that might have different comparators. 

Do you agree [~iamaleksey]?

> Interned column identifiers can be overridden incorrectly
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-12516
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12516
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Aleksey Yeschenko
>            Assignee: Stefania
>             Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x
>
>
> As discovered by [~Stefania] in CASSANDRA-12423, our use of {{ColumnIdentifier.internedInstances}}
to map byte buffer column names to column identifiers is incorrect. It's possible for two
non-equal {{ColumnIdentifier}} instances to have the same {{ByteBuffer}} as raw name, and
yet different text representations, due to different column name types, leading to hard to
debug correctness issues.
> Should either amend the map to have the type as part of the key, or limit the use of
that map.



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

Mime
View raw message