-
Notifications
You must be signed in to change notification settings - Fork 466
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
Comments
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. |
just got another error, probably related as well. DB attached below.
|
@lni Seems related if rocksdb was the previous engine in that case as well. @sumeerbhola saw the exact same error earlier too. |
@itsbilal Yes, the DB was created by RocksDB. |
@lni can you confirm if the latest commit on pebble master fixes this issue? |
I am still getting the following error when running Pebble v0.0.0-20200311200940-d2ecbc248dec
DB attached below. |
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 ( |
@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. |
@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. |
FYI: The person you're trying to reply to is "@lni" (L-N-I). |
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. |
@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 |
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. |
@petermattis kTolerateCorruptedTailRecords is used in RocksDB. WAL recycling is always disabled in RocksDB. |
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. |
@lni Can you test what happens if you switch to using RocksDB's |
@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 -
DB is attached below. |
@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. |
@petermattis sure, will provide you access to the testing code I have. please allow a couple of days |
@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). |
sorry for the typo above, logRecycler.limit was set to 0, not 1. |
@petermattis I've emailed you details. |
TIL that |
I managed to reproduce @lni's problem with some additional instrumentation: before every restart cycle I create a
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. |
Ah, I think I understand what is going on:
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.
@itsbilal, @sumeerbhola thoughts? |
@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. |
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. |
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 |
I like the solution you have in your branch; where we maintain a 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? |
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
This should help trip early on issues such as the one found in: cockroachdb/pebble#567
This should help trip early on issues such as the one found in: cockroachdb/pebble#567
This should help trip early on issues such as the one found in: cockroachdb/pebble#567
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_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 -
The text was updated successfully, but these errors were encountered: