cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Benedict (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6974) Replaying archived commitlogs isn't working
Date Mon, 21 Apr 2014 11:32:15 GMT

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

Benedict commented on CASSANDRA-6974:
-------------------------------------

I've pushed -f a branch [here|https://github.com/belliottsmith/cassandra/tree/6974-badarchiving].

This branch should fix any and all commit log related problems. The ball is now in [~yukim]'s
(?) court to address the replay of mutations that are associated with column families that
have been assigned a different UUID.

This patch does a number of things:
1) It ensures we always wait for the final sync of a segment before archiving
2) It writes a header out to each segment which contains the descriptor (id and version) the
segment was created with
3) This header and any possible descriptor from the name are read during restore, and used
to perform various sanity checks before restoring the segment with the same id as it was first
created with
4) The "base" commit log segment id is now based on the max of current time and the largest
currently extant segment id, to ensure we never reuse an old id
5) It also fiddles with the size and update of the crcs we're writing, as realised we've been
writing out two longs since time immemorial, when the crc is only 32bit, so we're wasting
8bytes per mutation; and also the sync marker crcs were being updated incorrectly, using update(int)
which uses the int as a byte.

> Replaying archived commitlogs isn't working
> -------------------------------------------
>
>                 Key: CASSANDRA-6974
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6974
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Ryan McGuire
>            Assignee: Benedict
>             Fix For: 2.1 beta2
>
>         Attachments: 2.0.system.log, 2.1.system.log
>
>
> I have a test for restoring archived commitlogs, which is not working in 2.1 HEAD.  My
commitlogs consist of 30,000 inserts, but system.log indicates there were only 2 mutations
replayed:
> {code}
> INFO  [main] 2014-04-02 11:49:54,173 CommitLog.java:115 - Log replay complete, 2 replayed
mutations
> {code}
> There are several warnings in the logs about bad headers and invalid CRCs: 
> {code}
> WARN  [main] 2014-04-02 11:49:54,156 CommitLogReplayer.java:138 - Encountered bad header
at position 0 of commit log /tmp/dtest
> -mZIlPE/test/node1/commitlogs/CommitLog-4-1396453793570.log, with invalid CRC. The end
of segment marker should be zero.
> {code}
> compare that to the same test run on 2.0, where it replayed many more mutations:
> {code}
>  INFO [main] 2014-04-02 11:49:04,673 CommitLog.java (line 132) Log replay complete, 35960
replayed mutations
> {code}
> I'll attach the system logs for reference.
> [Here is the dtest to reproduce this|https://github.com/riptano/cassandra-dtest/blob/master/snapshot_test.py#L75]
- (This currently relies on the fix for snapshots available in CASSANDRA-6965.)



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message