cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "sankalp kohli (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-13987) Multithreaded commitlog subtly changed durability
Date Mon, 13 Nov 2017 22:08:00 GMT

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

sankalp kohli commented on CASSANDRA-13987:
-------------------------------------------

+1 for doing this in 3.0+ 

> Multithreaded commitlog subtly changed durability
> -------------------------------------------------
>
>                 Key: CASSANDRA-13987
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13987
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Jason Brown
>            Assignee: Jason Brown
>             Fix For: 4.x
>
>
> When multithreaded commitlog was introduced in CASSANDRA-3578, we subtly changed the
way that commitlog durability worked. Everything still gets written to an mmap file. However,
not everything is replayable from the mmaped file after a process crash, in periodic mode.
> In brief, the reason this changesd is due to the chained markers that are required for
the multithreaded commit log. At each msync, we wait for outstanding mutations to serialize
into the commitlog, and update a marker before and after the commits that have accumluated
since the last sync. With those markers, we can safely replay that section of the commitlog.
Without the markers, we have no guarantee that the commits in that section were successfully
written, thus we abandon those commits on replay.
> If you have correlated process failures of multiple nodes at "nearly" the same time (see
["There Is No Now"|http://queue.acm.org/detail.cfm?id=2745385]), it is possible to have data
loss if none of the nodes msync the commitlog. For example, with RF=3, if quorum write succeeds
on two nodes (and we acknowledge the write back to the client), and then the process on both
nodes OOMs (say, due to reading the index for a 100GB partition), the write will be lost if
neither process msync'ed the commitlog. More exactly, the commitlog cannot be fully replayed.
The reason why this data is silently lost is due to the chained markers that were introduced
with CASSANDRA-3578.
> The problem we are addressing with this ticket is incrementally improving 'durability'
due to process crash, not host crash. (Note: operators should use batch mode to ensure greater
durability, but batch mode in it's current implementation is a) borked, and b) will burn through,
*very* rapidly, SSDs that don't have a non-volatile write cache sitting in front.) 
> The current default for {{commitlog_sync_period_in_ms}} is 10 seconds, which means that
a node could lose up to ten seconds of data due to process crash. The unfortunate thing is
that the data is still avaialble, in the mmap file, but we can't replay it due to incomplete
chained markers.
> ftr, I don't believe we've ever had a stated policy about commitlog durability wrt process
crash. Pre-2.0 we naturally piggy-backed off the memory mapped file and the fact that every
mutation was acquired a lock and wrote into the mmap buffer, and the ability to replay everything
out of it came for free. With CASSANDRA-3578, that was subtly changed. 
> Something [~jjirsa] pointed out to me is that [MySQL provides a way to adjust the durability
guarantees|https://dev.mysql.com/doc/refman/5.6/en/innodb-parameters.html#sysvar_innodb_flush_log_at_trx_commit]
of each commit in innodb via the {{innodb_flush_log_at_trx_commit}}. I'm using that idea as
a loose springboard for what to do here.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message