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-10130) Node failure during 2i update after streaming can have incomplete 2i when restarted
Date Fri, 19 May 2017 22:27:04 GMT

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

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

Overall I like the new approach and the idea of keeping {{markIndex*}} usage restricted to
{{SecondaryIndexManager}}, since it will keep things more self-contained and prevent bad usages.

While inspecting usages of {{buildAllIndexesBlocking}} with the {{preBuildTask}} parameter,
I noticed that rebuilding indexes is a natural consequence of adding new SSTables to the tracker
- I don't see a situation where we want to add SSTables to the tracker and NOT rebuild the
indexes, so instead of requiring users of {{Tracker.addSSTables}} to figure out they need
to rebuild indexes and create a dependency with the {{SecondaryIndexManager}} (such as {{OnCompletionRunnable}}
or {{ColumnFamilyStore.loadNewSSTables}}), or even creating a dependency between {{Tracker.addSSTables}}
and the secondary index manager, we could leverage the tracker notification support and make
the secondary index automatically rebuild indexes when receiving an {{SSTableAddedNotification}}
from the tracker.

However this notification is only triggered *after* the SSTables are added to the tracker,
but there is a possibility that there is a failure after some SSTables were already added
and we would need to rebuild indexes in that case, so we could maybe add a new {{SSTableBeforeAddedNotification}}
(or better name) that is triggered at the start of {{Tracker.addSSTables}}, mark the index
as building when receiving that notification and actually trigger {{buildAllIndexesBlocking}}
when receiving the {{SSTableAddedNotification}}.

WDYT of this suggestion?

> 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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message