cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Paulo Motta (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-10341) Streaming does not guarantee cache invalidation
Date Wed, 30 Sep 2015 22:34:04 GMT

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

Paulo Motta commented on CASSANDRA-10341:
-----------------------------------------

I thought of 3 approaches to solve this problem:

1. After all sstables of a given table are received and added to CFS, iterate over them and
invalidate the keys.
Upside: ease of implementation.
Drawback: for large streams, this would mean reading all data from disk again at a later point
in time, which could add some overhead, like affecting the page cache.

2. Instead of adding sstables to CFS after all sstables of a given table are transferred,
add each individual sstable to CFS when it is received, and iterate over them to invalidate
keys.
Upside: better than previous approach, since for moderate-sized sstables would probably still
be cached soon after it is received.
Drawback: Failed stream sessions would add data to the live system, even though they did not
complete successfully.

3. Implement a dependency invalidation mechanism on the caching service, so we would pre-invalidate
keys as they arrive, and complete the invalidation after the stream session is complete (or
drop the invalidation).
Upside: we would not need to re-read data to invalidate the keys.
Drawback: add complexity to caching service.

[~benedict] [~yukim] any thoughts on what would be the preferable approach?

> Streaming does not guarantee cache invalidation
> -----------------------------------------------
>
>                 Key: CASSANDRA-10341
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10341
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Benedict
>            Assignee: Paulo Motta
>
> Looking at the code, we attempt to invalidate the row cache for any rows we receive via
streaming, however we invalidate them immediately, before the new data is available. So, if
it is requested (which is likely if it is "hot") in the interval, it will be re-cached and
not invalidated.



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

Mime
View raw message