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

db: pebble internal error #567

Closed
lni opened this issue Mar 11, 2020 · 33 comments · Fixed by #580
Closed

db: pebble internal error #567

lni opened this issue Mar 11, 2020 · 33 comments · Fixed by #580
Assignees

Comments

@lni
Copy link
Contributor

lni commented Mar 11, 2020

Hi @petermattis

I got the following error when trying to open the attached DB using the latest version of Pebble. The attached DB is generated using RocksDB.

pebble: internal error: L0 flushed file 000019 overlaps with the largest seqnum of a preceding flushed file: 8212-11042 vs 8416

pebble_check_failed.zip

Both this one and #566 were observed when testing Pebble to see whether it can operate correctly on RocksDB generated DB and the other way around. RocksDB/Pebble features used are pretty basic, they are -

  • Write to DB via WriteBatch with sync set to true
  • Read from DB via Get and forward iterator
  • Delete a single record using the regular delete operation
  • Range delete
  • Manual compaction
@petermattis
Copy link
Collaborator

This looks like a duplicate of #565. @itsbilal can you take a look?

@itsbilal
Copy link
Member

Yup, different error message but same underlying issue as #565. I’ll keep this issue around anyway so I can confirm the fix for that also fixes this.

@lni
Copy link
Contributor Author

lni commented Mar 11, 2020

just got another error, probably related as well. DB attached below.

pebble: internal error: L0 file 000017 does not have strictly increasing largest seqnum: 12321-12762 vs 12762

not_strictly_increasing.zip

@itsbilal
Copy link
Member

@lni Seems related if rocksdb was the previous engine in that case as well. @sumeerbhola saw the exact same error earlier too.

@lni
Copy link
Contributor Author

lni commented Mar 11, 2020

@itsbilal Yes, the DB was created by RocksDB.

@itsbilal
Copy link
Member

@lni can you confirm if the latest commit on pebble master fixes this issue?

@lni
Copy link
Contributor Author

lni commented Mar 12, 2020

@itsbilal

I am still getting the following error when running Pebble v0.0.0-20200311200940-d2ecbc248dec

pebble: internal error: L0 flushed file 000016 overlaps with the largest seqnum of a preceding flushed file: 8965-16460 vs 11982

DB attached below.

internal_error.zip

@itsbilal
Copy link
Member

On closer inspection this looks like a RocksDB bug; no L0 SSTables should overlap with that many sequence numbers, and overlapping sequence numbers in SSTables (even in L0) are not something Pebble supports.

Can you see if disabling subcompactions (options.max_subcompations = 1 in rocksdb) and then running the same workload produces the same problematic scenario? We've observed in the past that subcompactions frequently result in overlapping SSTs.

@lni
Copy link
Contributor Author

lni commented Mar 13, 2020

@itsbilal I am not using subcompaction, the max_subcompactions field is always set to 1 in my tests.

I am not sure whether it is a rocksdb bug, as it would be pretty easy to pick up such issue. My concern is if this is somehow by design, then RocksDB based projects won't be easily migrated to pebble.

@petermattis
Copy link
Collaborator

@ini We're concerned about the same problem: bidirectional compatibility with RocksDB. Can you share some details on how this DB was created? What RocksDB options were you running with? What RocksDB version? If you can provide reproduction steps for creating a database in this state from scratch, we'll be able to track down the problem very quickly.

There was a bug in RocksDB a few months ago that could create sstables with overlapping seqnum ranges: facebook/rocksdb#5958. Looks like the first RocksDB release with that fix was 6.6.3.

@ini
Copy link

ini commented Mar 13, 2020

FYI: The person you're trying to reply to is "@lni" (L-N-I).

@lni
Copy link
Contributor Author

lni commented Mar 13, 2020

@petermattis

Interestingly, once I have logRecycler.limit set to 1 as we discussed in #566, I can no longer reproduce the error reported here.

For your questions -

This DB was created in my monkey test in which the component that owns the DB gets frequently restarted (open->close->open). As I am trying to test the compatibility of RocksDB/Pebble, I made the test program to keep switching between RocksDB and Pebble during each of such restarts.

When the above error is reported, it is possible that both RocksDB and Pebble have written to the DB in previous runs.

I am using RocksDB v6.4.6, its options should be in the OPTIONS file included in the above attached zip. The OPTIONS file says the version is v6.6.4, that is probably because I opened it on my laptop with v6.6.4 before uploading.

@petermattis
Copy link
Collaborator

@lni I wonder if the WAL recycling interaction with RocksDB could have led to RocksDB replaying entries from an old portion of the log. What RocksDB WAL recovery mode are you using? I'd be interested to hear of the combination of WAL recycling enabled combined with the kPointInTimeRecovery WAL recovery mode.

@sumeerbhola
Copy link
Collaborator

I can produce the same error when changing Pebble to write more than 1 sstable during a flush (since even if the user keys in sst1 and sst2 are non-overlapping, the sequence numbers may overlap) -- does RocksDB support writing more than one sst during a flush or an intra-L0 compaction?

@petermattis
Copy link
Collaborator

I can produce the same error when changing Pebble to write more than 1 sstable during a flush (since even if the user keys in sst1 and sst2 are non-overlapping, the sequence numbers may overlap) -- does RocksDB support writing more than one sst during a flush or an intra-L0 compaction?

RocksDB always writes to a single sstable during flushing. Pebble currently has that same restriction. When we introduce L0-partitioning we're going to have to adjust the L0 consistency checks.

@lni
Copy link
Contributor Author

lni commented Mar 14, 2020

@petermattis kTolerateCorruptedTailRecords is used in RocksDB.

WAL recycling is always disabled in RocksDB.

@lni
Copy link
Contributor Author

lni commented Mar 14, 2020

I have been running my tests overnight with logRecycler.limit set to 1, it didn't reproduce any of the above reported internal error. Switched between RocksDB and Pebble for hundreds of thousands of times, didn't cause any visible issue.

This let me wonder whether it makes sense to add an option to allow WAL recycling to be disabled. From what I experienced, WAL recycling is the major showstopper for bidirectional compatibility with RocksDB and the caused errors are difficult for reasoning.

@petermattis
Copy link
Collaborator

@lni Can you test what happens if you switch to using RocksDB's kPointInTimeRecovery option? I want to have that final bit of evidence that what we're seeing is an incompatibility between WAL recycling and kTolerateCorruptTailRecords.

@lni
Copy link
Contributor Author

lni commented Mar 14, 2020

@petermattis I test it again using RocksDB's (v6.4.6) kPointInTimeRecovery and unmodified Pebble version v0.0.0-20200311200940-d2ecbc248dec. I go the same internal error -

pebble: internal error: L0 flushed file 000016 overlaps with the largest seqnum of a preceding flushed file: 7859-12241 vs 8415

DB is attached below.

internal_error_20200315.zip

@petermattis
Copy link
Collaborator

@lni Huh, that is surprising. Can you share this testing code so I can reproduce myself? I'd like to understand where exactly this problematic L0 setup is coming from. I know it is happening on the RocksDB side because Pebble runs the check that is failing before committing new LSM state. What I don't know is why RocksDB is getting into this situation and exactly how it is related to WAL recycling.

@lni
Copy link
Contributor Author

lni commented Mar 15, 2020

@petermattis sure, will provide you access to the testing code I have. please allow a couple of days

@petermattis
Copy link
Collaborator

@lni Great. Thanks. If you have concerns about sharing the test code, you can email it to be directly. (My email is on github / in commit messages).

@lni
Copy link
Contributor Author

lni commented Mar 15, 2020

sorry for the typo above, logRecycler.limit was set to 0, not 1.

@lni
Copy link
Contributor Author

lni commented Mar 17, 2020

@petermattis I've emailed you details.

@petermattis
Copy link
Collaborator

TIL that kPointInTimeRecovery and kTolerateCorruptedTailRecords are almost identical code paths inside of RocksDB. The only difference is in how they handle truncated and corrupted (checksum mismatch) records. In fact, up until very recently, kPointInTimeRecovery was automatically translated to kTolerateCorruptedTailRecords when WAL recycling was enabled. The upshot of this is that whatever problem is occurring almost certainly has nothing to do with kTolerateCorruptedTailRecords vs kPointInTimeRecovery.

@petermattis
Copy link
Collaborator

I managed to reproduce @lni's problem with some additional instrumentation: before every restart cycle I create a checkpointXXX directory and hard link all of the DB files into that directory. The problem we're seeing is that Pebble is running, then the test restarts into RocksDB and RocksDB creates sstables with overlapping seqnums. What the instrumentation shows is that the problem is somewhere in the WAL file itself. Here is a portion of the WAL file that RocksDB is replaying:

~ pebble wal dump --key=null --value=null checkpoint003/000017.log
...
33577(42) seq=16291 count=1
    SET(,)
33630(42) seq=16292 count=1
    SET(,)
33683(106) seq=10552 count=2
    SET(,)
    SET(,)
33796(42) seq=10554 count=1
    SET(,)

Notice the sequence numbers here. That should never happen in a WAL file. I need to do some more analysis of this WAL to determine how this is possibly happening.

@petermattis
Copy link
Collaborator

Ah, I think I understand what is going on:

recyclable record header
33630(42) seq=16292 count=1
    SET(,)
non-recyclable record header
33683(106) seq=10552 count=2
    SET(,)
    SET(,)

Pebble recycled a log file written by RocksDB. RocksDB is not writing log records with the "recyclable record header" which includes the WAL number. Pebble then merrily recycles the log file writing its new records to the front. Both RocksDB and Pebble will reject any recyclable record which has a mismatched file num, but they both accept non-recyclable records.

I see two possible fixes. We can consider doing both.

  • Pebble can avoid recycling a log file unless it knows the log file was created using only recyclable records. One way to do this is to avoid recycling WALs from a previous incarnation of the database. I need to see if RocksDB is already doing this.
  • Pebble can stop reading a WAL if it sees a non-recyclable record that has been preceded by a recyclable record. This would be different behavior from RocksDB which makes me a little anxious.

@itsbilal, @sumeerbhola thoughts?

@petermattis
Copy link
Collaborator

@lni I'm testing https://github.com/cockroachdb/pebble/compare/pmattis/log-recycle?expand=1 to see if it fixes the problem. I'm feeling reasonably confident as the analysis above completely explains the overlapping seqnum problem. I think it could also explain the other crashes you've seen with this test when WAL recycling is enabled.

@petermattis
Copy link
Collaborator

So far so good. @lni's tests are now on iteration 3 without error. Previously I was hitting errors on the first iteration (each iteration takes ~30min). I'm going to let this run for a while longer.

Concurrently, I'm trying to see how RocksDB handles this situation. So far I don't see anything preventing RocksDB from recycling a log from a previous incarnation, but I also don't see it actually doing so in testing.

@petermattis
Copy link
Collaborator

Ok, I see how this is working in RocksDB. RocksDB has two modes of finding obsolete files. One mode utilizes precise tracking of which files can be deleted, and the other does a scan of the filesystem state, deleting any file that isn't referenced. WAL files can only be recycled via the precise tracking mode. The subtle part is that a WAL that is recovered at startup is deleted by the filesystem scanning mode of finding obsolete files, and is thus never a candidate for recycling. The approach in the pmattis/log-recycle branch will bring Pebble in line with this behavior.

@itsbilal
Copy link
Member

I like the solution you have in your branch; where we maintain a minRecycleLogNum. Seems safer than introducing new checks to trip up WAL replay.

Should we also add a rocksdb assertion/error (at least in our fork) that stops WAL replay if it ever sees non-increasing sequence numbers? Would help us catch things like these quicker, if we ever see them in a cockroach cluster.

@petermattis
Copy link
Collaborator

Should we also add a rocksdb assertion/error (at least in our fork) that stops WAL replay if it ever sees non-increasing sequence numbers? Would help us catch things like these quicker, if we ever see them in a cockroach cluster.

I think there is an argument for doing that, but the last I looked at the RocksDB log reader code it had tons of conditionals. We could easily foul something up. We've also had WAL recycling enabled for a while in CockroachDB so the risk is small. That said, can you take a look at the RocksDB WAL reading code and see what it would take add an assertion?

petermattis added a commit that referenced this issue Mar 18, 2020
WAL recycling depends on the recycled WAL only containing log records
written with the recyclable record format. Pebble always writes WAL
files with the recyclable record format, but RocksDB only uses that
format if WAL recycling is enabled. If Pebble recycles a RocksDB-written
WAL that doesn't use the recyclable record format, it can leave valid
looking log entries in the tail of the log. A subsequent replay of the
WAL by either Pebble or RocksDB could then replay those already
committed log entries. The most common way this badness would be
detected is by Pebble's L0 consistency checks, though the varieties of
badness here are almost unlimited (deleted records reappearing, written
records being deleted, etc). In order to protect against this, Pebble
now only recycles WAL files that it has written in the current
incarnation and thus knows for certain have been written with the
recyclable record format. RocksDB has similar behavior, though the code
which achieves this is somewhat convoluted so the behavior may have been
accidental.

Fixes #567
@petermattis petermattis self-assigned this Mar 18, 2020
@petermattis
Copy link
Collaborator

Got through 9 iterations of @lni's tests with 64x parallelism while running on #580. Zero errors.

itsbilal added a commit to cockroachdb/rocksdb that referenced this issue Mar 18, 2020
This should help trip early on issues such as the one found in:
cockroachdb/pebble#567
itsbilal added a commit to cockroachdb/rocksdb that referenced this issue Mar 19, 2020
This should help trip early on issues such as the one found in:
cockroachdb/pebble#567
itsbilal added a commit to cockroachdb/rocksdb that referenced this issue Mar 19, 2020
This should help trip early on issues such as the one found in:
cockroachdb/pebble#567
@itsbilal itsbilal removed their assignment Mar 20, 2020
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