Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Point-in-time recovery should fail on IO error when reading WAL #6288

Closed
yiwu-arbug opened this issue Jan 13, 2020 · 15 comments
Closed

Point-in-time recovery should fail on IO error when reading WAL #6288

yiwu-arbug opened this issue Jan 13, 2020 · 15 comments
Assignees

Comments

@yiwu-arbug
Copy link
Contributor

Expected behavior

With point-in-time recovery, if it encounter IO error when reading WAL, it could be due to intermittent error (a loose cable?). RocksDB should fail to open in this case. Truncating the WAL could result in data loss (even when the WAL is sync-ed previously).

Actual behavior

rocksdb will treat the error as a corruption in WAL, stop replaying the rest of WAL and open successfully.

Steps to reproduce the behavior

Only by reading the code.

@ajkr
Copy link
Contributor

ajkr commented Feb 14, 2020

To be honest I am a bit lost on what is expected from point-in-time recovery lately. Wouldn't recovering no WAL data at all be a valid recovery to some point-in-time?

The other recent time I'm confused was #6351. Why is that feature incompatible with kPointInTimeRecovery?

@huachaohuang
Copy link
Contributor

IMO, point-in-time recovery only guarantees to recover to a consistent point, any point is acceptable. So the behavior @yiwu-arbug describes is also acceptable.
@ajkr I agree with you that kPointInTimeRecovery is the only mode that is compatible with recyclable log files.
I think for most databases, only the last partial log is tolerable. Implementing such a guarantee needs two syncs: sync the log first and then syn some metadata to indicate the end of the log. On recovery, if the metadata is synced successfully, we know where the log ends. Otherwise, we can ignore the last log safely.

@yiwu-arbug
Copy link
Contributor Author

To be honest I am a bit lost on what is expected from point-in-time recovery lately. Wouldn't recovering no WAL data at all be a valid recovery to some point-in-time?

The other recent time I'm confused was #6351. Why is that feature incompatible with kPointInTimeRecovery?

there's the loose guarantee of point-in-time recovery where it can recover to any consistent point, but there's also the guarantee of sync wal that data is persistent after that point. The latter cannot be guaranteed if there's corruption on disk, but IMO we should make that promise as far as possible.

@ajkr
Copy link
Contributor

ajkr commented Apr 6, 2020

That sounds good to me. kPointInTimeRecovery should be able to recover up to the latest fsync assuming no disk corruption. If disk corruption happens and it's reported, Open() should fail. If disk corruption happens and hardware somehow does not report it, Open() should succeed and synced data can be dropped. Users for whom the last case is intolerable can use a stricter recovery mode.

cc @yhchiang who is our visiting expert in power loss recovery!

@riversand963 riversand963 self-assigned this May 29, 2020
@riversand963
Copy link
Contributor

Assigning to me to track it.

@riversand963
Copy link
Contributor

Thanks @yiwu-arbug for flagging this. I am working on a fix. Not sure if my understanding is accurate. In point-in-time recovery,

  1. if there is disk corruption (assuming no device-level ECC), then there can be data loss. The corruption can occur before the last-synced offset. Dropping content of the WAL can also lead to data loss. If we follow the suggestion here (fail db open only for IOError), then the user will still suffer from data loss in case of disk corruption.
  2. if there is IOError, then following suggestion here will report db open failure, thus preventing data loss since we won't drop WAL.
    If my understanding of 1 and 2 are both correct, does it imply that (per-file, transient) IOError is more severe than disk corruption? Otherwise, we report failure on a less severe case, but let the more severe error pass?

@riversand963
Copy link
Contributor

Or is the reasoning as follows:
Disk corruption is less frequent than the IOError scenario which you mentioned, so we make effort for the common case?

@ajkr
Copy link
Contributor

ajkr commented Jun 10, 2020

Thanks @yiwu-arbug for flagging this. I am working on a fix. Not sure if my understanding is accurate. In point-in-time recovery,

  1. if there is disk corruption (assuming no device-level ECC), then there can be data loss. The corruption can occur before the last-synced offset. Dropping content of the WAL can also lead to data loss. If we follow the suggestion here (fail db open only for IOError), then the user will still suffer from data loss in case of disk corruption.
  2. if there is IOError, then following suggestion here will report db open failure, thus preventing data loss since we won't drop WAL.
    If my understanding of 1 and 2 are both correct, does it imply that (per-file, transient) IOError is more severe than disk corruption? Otherwise, we report failure on a less severe case, but let the more severe error pass?

Good questions. I thought about them for a while today. Here is my analysis which I believe supports the original plan.

  • Recovery has a tradeoff between allowing Open() to succeed, and ensuring ack'd data is recovered. Compared to kAbsoluteConsistency and kTolerateCorruptTailRecord, kPointInTimeRecovery is supposed to be more flexible (but not totally permissive) in dropping ack'd data to enable successful Open.
  • A case where kTolerateCorruptTailRecord annoyingly fails Open() is when unsynced data is dropped. This is because an essentially random subset of the pages that weren't explicitly synced will have been written back at the point of hard crash. Then, the subsequent recovery will encounter a checksum error. In my view, this is the case kPointInTimeRecovery wants to permit.
  • However, checksum error could also mean something else: corruption of synced data. This could happen during the write since it involves steps where data integrity is not protected, or at rest if the hardware/firmware has an issue. Ideally we would not want kPointInTimeRecovery to drop the data and permit Open() in this case; however, given we do not track last synced position, we cannot distinguish this case from the previous case.
  • I believe the I/O error we were talking about above included both transient errors and permanent ones (e.g., device is telling us a sector we're trying to read is corrupted). In either case, denying the Open() seems OK:
    • If it's transient, it is better to deny the Open() and allow the user to retry, then open the DB. That's because to open the DB we'd have to drop potentially synced data. With a retry that data may be savable.
    • If it's permanent, it's reasonably likely that synced data has been lost. Unlike checksum error, which is expectable in the unsynced data portion following a hard reset, a permanent read error is not expected.

@riversand963
Copy link
Contributor

Thanks @yiwu-arbug for flagging this. I am working on a fix. Not sure if my understanding is accurate. In point-in-time recovery,

  1. if there is disk corruption (assuming no device-level ECC), then there can be data loss. The corruption can occur before the last-synced offset. Dropping content of the WAL can also lead to data loss. If we follow the suggestion here (fail db open only for IOError), then the user will still suffer from data loss in case of disk corruption.
  2. if there is IOError, then following suggestion here will report db open failure, thus preventing data loss since we won't drop WAL.
    If my understanding of 1 and 2 are both correct, does it imply that (per-file, transient) IOError is more severe than disk corruption? Otherwise, we report failure on a less severe case, but let the more severe error pass?

Good questions. I thought about them for a while today. Here is my analysis which I believe supports the original plan.

  • Recovery has a tradeoff between allowing Open() to succeed, and ensuring ack'd data is recovered. Compared to kAbsoluteConsistency and kTolerateCorruptTailRecord, kPointInTimeRecovery is supposed to be more flexible (but not totally permissive) in dropping ack'd data to enable successful Open.

  • A case where kTolerateCorruptTailRecord annoyingly fails Open() is when unsynced data is dropped. This is because an essentially random subset of the pages that weren't explicitly synced will have been written back at the point of hard crash. Then, the subsequent recovery will encounter a checksum error. In my view, this is the case kPointInTimeRecovery wants to permit.

  • However, checksum error could also mean something else: corruption of synced data. This could happen during the write since it involves steps where data integrity is not protected, or at rest if the hardware/firmware has an issue. Ideally we would not want kPointInTimeRecovery to drop the data and permit Open() in this case; however, given we do not track last synced position, we cannot distinguish this case from the previous case.

  • I believe the I/O error we were talking about above included both transient errors and permanent ones (e.g., device is telling us a sector we're trying to read is corrupted). In either case, denying the Open() seems OK:

    • If it's transient, it is better to deny the Open() and allow the user to retry, then open the DB. That's because to open the DB we'd have to drop potentially synced data. With a retry that data may be savable.
    • If it's permanent, it's reasonably likely that synced data has been lost. Unlike checksum error, which is expectable in the unsynced data portion following a hard reset, a permanent read error is not expected.

Make sense and thanks for the insights. As you said, the crucial assumptions are:

  1. Checksum error can happen anywhere in the WAL due to unsynced data or corruption of synced data. In this case, we cannot be sure of the cause, thus we choose not to fail the recovery because we want to be more permissive than kAbsoluteConsistency and kTolerateCorruptTailRecord.
  2. IOError very likely means high risk of losing synced data, thus we choose to be more conservative and fail the recovery.

@pdillinger
Copy link
Contributor

Checksum error can happen anywhere in the WAL due to unsynced data

Are we planning to improve this? Should we add an "after sync" marker to the WAL (new record type) along with the next appended data, so that everything before such a marker must checksum (else must be data loss)?

@ajkr
Copy link
Contributor

ajkr commented Jun 12, 2020

Checksum error can happen anywhere in the WAL due to unsynced data

Are we planning to improve this? Should we add an "after sync" marker to the WAL (new record type) along with the next appended data, so that everything before such a marker must checksum (else must be data loss)?

I think the whole sentence needs to be quoted for that claim to be true (i.e., also "or corruption of synced data"). Unsynced data should only cause corruption in the unsynced tail of entries.

It'd be great though if we can improve this to know where the synced portion ends and unsynced portion begins, so we can fail Open() only when the checksum error happens in the synced portion. I am not sure about the marker as proposed as it sounds like a marker referring to synced data might be unsynced.

@ajkr
Copy link
Contributor

ajkr commented Jun 12, 2020

Maybe we can store the expected length of a sync interval at its start? I think in either case of sync -- WriteOptions::sync or FlushWAL(true /* sync */) -- we know the full sync interval before we begin writing it. If the file ends before the sync interval is expected to end, we know that sync didn't complete so can tolerate checksum failures within it.

Actually the logic I'm describing is the same logic kTolerateCorruptedTailRecords uses to permit checksum corruption on individual tail records, but generalized for batches of records...

@pdillinger
Copy link
Contributor

Maybe we can store the expected length of a sync interval at its start?

Better, yes. If that works, anything after should indicate the sync completed.

@ghost
Copy link

ghost commented Nov 11, 2020

Now, when track_and_verify_wals_in_manifest = true, if a WAL is fully synced, the WAL will be tracked in MANIFEST. So during recovery, we can check whether the WAL is synced, if it is, then even in point-in-time recovery, errors should be reported.

Maybe in the future, we can track the last synced size of the WAL, so that if the recovery error happens before the synced size, then we report error.

@riversand963
Copy link
Contributor

I believe this is resolved by #6963 and WAL tracking in MANIFEST feature. Closing. cc @yiwu-arbug feel free to reopen if you see fit.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants