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, 07 Apr 2014 20:39:15 GMT

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

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

OK, so looking at this a little more closely, I think your test is broken (not to say there
may not still be some issues with archiving). I've uploaded a new version anyway that improves
upon my previous patch, but will wait until the dtest is corrected to proceed further.

Specifically, it looks like you're simply copying all of the "active" commit logs (i.e. the
ones in the commitlog directory) manually, without making any attempt to ensure they're actually
synced to disk. You either need to wait until the periodic sync interval has passed (*2 probably,
to be sure) or you need to drain() the node. Otherwise the CL data may be sitting in memory.
You should never copy the CL segments straight from under C*, you must let it archive them
for you.

> 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