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 Fri, 11 Apr 2014 10:26:28 GMT

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

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

OK, so this is a whole lot more complex a problem than it first appears. I was being thrown
by the fact that some of the logs were replaying, but it seems those were the empty recycled
logs created at startup. There are two problems to fix:

# To increase safety of log replay, we introduced a checksum of the log id into the header.
Unfortunately it appears that on _restore_ the id in the name is ignored and a new segment
id is allocated. Not only does this mean logs are replayed out of order (so is probably undesirable),
it also means that the new checksum is rejecting these log files as they have the wrong checksum
for their named id. However since we impose no constraints on the archive command, it is possible
that the end user has always been archiving in a way that destroys the original segment name+id,
so relying on it being present may be impossible. I'm reticent to drop the checksum safety
check, as it's tied into safety of the new CL model.
# The test restores a CF, but creates a new CF and streams the old CFs data to it. The CL
holds data against the old CF, and on replay ignores the mutations because the target CF does
not exist

[~jbellis] [~vijay2win@yahoo.com] do you have any opinion on point 1? We could encode the
ID in the header itself, and use the ID on restore to construct the ID of the target file
only, which would probably retain the present guarantees. Or we could throw a hissy fit if
we're provided a non-standard name for the segment. This also brings up a point about CL and
MS version - these are currently encoded in the name as well, so if somebody restores an old
version against a new C* cluster, they'll find things don't go as planned, so we may want
to consider encoding these in the header going forwards also. We can use the presence of the
checksum to confirm that we're operating on a new enough version that supports the scheme.

As to point 2, I'm not sure if this is a problem with the test or with restore procedure:
I'm guessing it's not atypical to restore by creating a new cluster. In which case we have
a whole separate problem to address. [~yukim]: thoughts?

Also, for the record [~enigmacurry], the log files from the run where the inserts and archives
happen are getting trashed, so diagnosing this was trickier than it might otherwise have been.
Would be nice to fix that (I assume it may apply to other tests as well)

> 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