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

Andrés de la Peña commented on CASSANDRA-10130:
-----------------------------------------------

Here is the updated version of the patch:

||[trunk|https://github.com/apache/cassandra/compare/trunk...adelapena:10130-trunk-memtable]|[utests|http://cassci.datastax.com/view/Dev/view/adelapena/job/adelapena-10130-trunk-memtable-testall/]|[dtests|http://cassci.datastax.com/view/Dev/view/adelapena/job/adelapena-10130-trunk-memtable-dtest/]|

bq. I personally like the "memtable" version more, as logically speaking, if an 
sstable is added with a memtable, it means it has been also indexed, while in 
all other cases it means the sstable(s) have been externally loaded and needs 
indexing

The only exception to this seems to be 
[{{CompactionStress}}|https://github.com/apache/cassandra/blob/trunk/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java#L147].
 It calls to {{ColumnFamilyStore#addSSTables}} without memtable and without 
indexing. Fortunately there shouldn't be any associated 2i when the call is 
done, I have added [a 
check|https://github.com/adelapena/cassandra/blob/bb1f80b0b7be5122887621878fd6137627f72558/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java#L148]
 to make sure that it is so.

bq. 1) There are too many overloads of addSSTable(s), and some of them do not 
make much sense and are there just to support self calls (i.e. the version with 
many sstables and a single memtable I believe), so can we clean that up?

Cleaned.

bq. 2) Who is actually calling the addSSTable method with the memtable? I think 
I'm missing where it's actually used...

This is called from 
[{{Tracker#replaceFlushed}}|https://github.com/adelapena/cassandra/blob/bb1f80b0b7be5122887621878fd6137627f72558/src/java/org/apache/cassandra/db/lifecycle/Tracker.java#L340-L371],
 and it is checked in 
[{{TrackerTest#testMemtableReplacement}}|https://github.com/adelapena/cassandra/blob/bb1f80b0b7be5122887621878fd6137627f72558/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java#L312].

bq. 3) Why did you reduce the test coverage in 
{{indexCorrectlyMarkedAsBuildAndRemoved}}?

Because it seemed covered by dtests and simulating an index building failure 
seemed harder without the function passed as parameter. But really there is a 
case that is not covered by dtests and the failure can be simulated passing a 
null sstable collections, so I have added [an equivalent 
check|https://github.com/adelapena/cassandra/blob/bb1f80b0b7be5122887621878fd6137627f72558/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java#L512]
 using rebuilding.

bq. It would be nice if you could add a short comment explaining when the 
memtable is present or not.

I have just added [several 
comments|https://github.com/adelapena/cassandra/blob/bb1f80b0b7be5122887621878fd6137627f72558/src/java/org/apache/cassandra/notifications/SSTableAddedNotification.java],
 not sure about if they are clear enough or too repetitive.

bq. I'm not sure we should restrict the {{SSTableLoadedNotification}} to only 
loaded sstables, since a subscriber may want to do an action before and after 
sstables are added to the tracker, regardless if they are flushed or externally 
loaded, so we should probably trigger regardless if memtable is null or not and 
rename the notification to {{SSTableBeforeAddedNotification}} (or similar) - 
this will also prevent confusion from users thinking the sstables are already 
in the tracker when receiving an {{SSTableLoadedNotification}}.

Good idea. I have replaced {{SSTableLoadedNotification}} by a 
[{{SSTableBeforeAddedNotification}}|https://github.com/adelapena/cassandra/blob/bb1f80b0b7be5122887621878fd6137627f72558/src/java/org/apache/cassandra/notifications/SSTableBeforeAddedNotification.java]
 that is always send before {{SSTableAddedNotification}}. It includes the 
memtable to allow consumers to know if it comes from a flush.

bq. Also you should probably ubsubscribe from the tracker when the index is 
dropped.

It is the {{SecondaryIndexManager}} who is subscribed to the tracker. It 
manages all the indexes on its column family store, so it shouldn't unsubscribe 
when one of its indexes is dropped, unless I'm missing something.

> 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