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

Sylvain Lebresne commented on CASSANDRA-10219:
----------------------------------------------

bq. Should I rebase to bring the test into the code

Is the test testing something that isn't covered otherwise? That is, did 
CASSANDRA-10155 added an equivalent test? If that test is still of value, then 
I'll just ninja-commit it.

bq. the generic type parameter of RowIndexEntry, which wasn't used in any way

Fyi, I believe the reason for that parameter is CASSANDRA-3067. Now, I'm not 
really sure that pluggability still works in 3.0 and I have my doubt about 
trying to maintain such pluggability without making any concrete use of it, but 
let's not have this debate on this ticket :)

> KeyCache deserialization doesn't properly read indexed entries
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-10219
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10219
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sylvain Lebresne
>            Assignee: Branimir Lambov
>             Fix For: 3.x, 2.1.x
>
>
> When we write the key cache, we write the RowIndeEntry entirely, as can be 
> seen in {{CacheService.KeyCacheSerializer.serialize}}:
> {noformat}
> ByteBufferUtil.writeWithLength(key.key, out);
> out.writeInt(key.desc.generation);
> out.writeBoolean(true);
> key.desc.getFormat().getIndexSerializer(cfm, key.desc.version, 
> SerializationHeader.forKeyCache(cfm)).serialize(entry, out);
> {noformat}
> When we deserialize such entry, we have the case where an entry correspon to 
> a sstable that doesn't exist anymore (presumably, it's been deleted between 
> the last cache saving and the restart that triggers the cache loading), which 
> goes this way:
> {noformat}
> ByteBuffer key = ByteBufferUtil.read(input, keyLength);
> int generation = input.readInt();
> SSTableReader reader = findDesc(generation, 
> cfs.getSSTables(SSTableSet.CANONICAL));
> input.readBoolean(); // backwards compatibility for "promoted indexes" boolean
> if (reader == null)
> {
>     RowIndexEntry.Serializer.skipPromotedIndex(input);
>     return null;
> }
> {noformat}
> But the thing is, {{RowIndexEntry.Serializer.skipPromotedIndex}} doesn't skip 
> a full {{RowIndexEntry}}, it doesn't skip the position in particular, and so 
> as far as I can tell, that part is buggy. I believe we should call 
> {{RowIndexEntry.Serializer.skip}}.
> I'll note that I just noticed this while reading the code but I haven't 
> reproduced that problem personally. I also haven't pin-pointed the exact 
> source of that problem, but it's been there for a while as far as I can tell. 
> At the same time, the condition to reach that branch is probably pretty 
> uncommon, and failure to load the key cache does not prevent the node from 
> starting and it doesn't even seem that we log an error (we log at DEBUG, 
> maybe we should log at WARN), which would explain nobody reporting this.



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

Reply via email to