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

Paulo Motta commented on CASSANDRA-10130:
-----------------------------------------

Thanks for the update! The notification comments look great! See follow-up 
below:

* It's not clear that users of {{buildIndexesBlocking}} should mark the indexes 
as building beforehand, so we should make that explicit via an assertion and 
probably also add a comment - or perhaps provide a default version where it 
marks indexes as building beforehand, since the only case we will not want to 
do that is when receiving an {{SSTableAddedNotification}}, since we have 
already marked them as building when receiving an 
{{SSTableBeforeAddedNotification}}.

* Even though we have unit tests testing the tracker notifications and marking 
indexes as build, we don't have a SecondaryIndexManager unit test that checks 
that it's correctly marking the index as building when receiving an 
{{SSTableBeforeAddedNotification}} and marked as built after a successful 
{{SSTableAddedNotification}}. Could you add that please?

* There's still a slight chance that an index is created between an 
{{SSTableBeforeAddedNotification}} and {{SSTableAddedNotification}} and we 
won't have marked it as building, so we should probably save the indexes we 
have marked as building when receiving the {{SSTableBeforeAddedNotification}} 
and mark any new index as building after receiving an 
{{SSTableBeforeAddedNotification}} but before building all indexes, and maybe 
extend the previous unit test to test this unlikely scenario.

* Is there any particular reason why you moved the 
[order|https://github.com/apache/cassandra/compare/trunk...adelapena:10130-trunk-memtable#diff-a1dda2df07c96dc42ddb58766362703fL357]
 of {{updateSizeTracking}} on {{Tracker.addSSTables}}?

After those are addressed I think this should be mostly done unless 
[~sbtourist] has further comments. Thanks!

> Node failure during 2i update after streaming can have incomplete 2i when 
> restarted
> -----------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-10130
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10130
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Coordination
>            Reporter: Yuki Morishita
>            Assignee: Andrés de la Peña
>            Priority: Minor
>
> Since MV/2i update happens after SSTables are received, node failure during 
> MV/2i update can leave received SSTables live when restarted while MV/2i are 
> partially up to date.
> We can add some kind of tracking mechanism to automatically rebuild at the 
> startup, or at least warn user when the node restarts.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to