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

Sergio Bossa commented on CASSANDRA-10130:
------------------------------------------

bq.  I have replaced SSTableLoadedNotification by a 
SSTableBeforeAddedNotification

Nit: {{SSTableBeforeAddedNotification}} would probably read better as 
{{SSTableBeforeAddNotification}}.

bq. 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

A comment is a good idea, but I think we shouldn't go further than that: 
{{buildIndexesBlocking}} is private so we can assume callers know what they're 
doing.

bq. 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

I think in such case the new {{Index}} initialization task would take care of 
indexing, so it doesn't really matter if the new index misses the sstable 
notification. So, to cover against the specific race you mentioned, we can just 
filter out in {{SIM#handleNotification()}}, when receiving the 
{{SSTableAddedNotification}}, all indexes not marked as building, as we can 
assume those missed the first notification because not yet registered (and 
being just registered, the initialization task will eventually take care of any 
initial indexing).

bq. we could probably get rid of the pendingBuilds counter complexity since we 
wouldn't race with manual individual index rebuilds on the BUILT_INDEXES table

I think the race above can be solved easily without adding columns to the 
system table. Other than that, let's not forget the {{pendingBuilds}} counter 
was needed to protect us not just against concurrent building of multiple 
indexes, but also against concurrent building of multiple sstables "batches" 
for the same index.

Thoughts? I'll now proceed with reviewing the latest code changes.

> 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