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

Failed to resume DB after "no space left on device" error. #11643

Open
assafka opened this issue Jul 26, 2023 · 4 comments · May be fixed by #12767 or #12772
Open

Failed to resume DB after "no space left on device" error. #11643

assafka opened this issue Jul 26, 2023 · 4 comments · May be fixed by #12767 or #12772
Labels
bug Confirmed RocksDB bugs up-for-grabs Up for grabs

Comments

@assafka
Copy link

assafka commented Jul 26, 2023

It looks like RocksDB failed to resume after hitting "no space left on device".
"no space left on device" error is marked as an HardError, which according to Background-Error-Handling should be recoverable back to read-write mode after the issue has been addressed.

Unfortunately, @siying has introduced 10489 which adds a flag in WritableFileWriter that remembers an error has occurred.
Resuming the DB after no space error doesn't clear the flag, causing successive write failures, even though the recovery was completed successfully.

Expected behavior

RocksDB should be able to recover after not enough disk space error, either by calling Resume manually or using auto recovery.

Actual behavior

Both auto recovery and calling resume manually succeed but any further writes fails.

Steps to reproduce the behavior

Consider the following code:

#include <iostream>
#include "rocksdb/db.h"
#include <unistd.h>
class StorageExtender : public rocksdb::EventListener
{
public:
    bool isRecovered = false;
    StorageExtender() = default;
    void OnErrorRecoveryBegin(
        rocksdb::BackgroundErrorReason reason,
        rocksdb::Status bg_error,
        bool* /*auto_recovery*/
    ) {
        std::cout << "Got error:" << bg_error.ToString() << std::endl;
        if (bg_error.IsNoSpace()) {
            system("mount -o remount,size=256M /mnt/mytmpfs");
        }
    }
    void OnErrorRecoveryCompleted(rocksdb::Status old_bg_error) {
        std::cout << "Recovered from error:" << old_bg_error.ToString() << std::endl;
        isRecovered = true;
    }
};
int main()
{
    auto storageExtender = std::make_shared<StorageExtender>();
    rocksdb::DB* db{};
    rocksdb::Options options;
    options.listeners.push_back(storageExtender);
    options.create_if_missing = true;
    system("umount /mnt/mytmpfs");
    system("mount -t tmpfs -o size=1024K tmpfs /mnt/mytmpfs");
    rocksdb::Status s = rocksdb::DB::Open(options, "/mnt/mytmpfs", &db);
    if (!s.ok()) {
        std::cout << s.ToString() << std::endl;
        return 1;
    }
    rocksdb::WriteBatch wb;
    for (int i = 0; i < 1024 * 1024 * 5; ++i) {
        auto kv = std::to_string(i);
        s = wb.Put(kv, kv);
    }
    rocksdb::WriteOptions wo;
    s = db->Write(wo, &wb);
    std::cout << s.ToString() << std::endl;
    // system("mount -o remount,size=32768K /mnt/mytmpfs");
    std::cout << "Waiting for recovery to complete" << std::endl;
    while (!storageExtender->isRecovered) {
        std::cout << "."  << std::endl;
        sleep(1);
    }
    std::cout << "Done" << std::endl;
    s = db->Write(wo, &wb);
    std::cout << s.ToString() << std::endl;
    return 0;
}

In the above code example, I expect the DB to be writeable after a disk space has been freed, but it keeps failing due to the seen_error_ flag in WritableFileWriter.

Notes:

  • It looks similar to 9762, but has a wider effect as it happens not only with PessimisticTransactionDB.
  • A possible solution is to call logs_.back().writer->file()->reset_seen_error(); in ResumeImpl, but I'm not familiar enough with rocks error handling, and I'm not sure what are the consequences of such change.
@ajkr ajkr added bug Confirmed RocksDB bugs up-for-grabs Up for grabs labels Jan 8, 2024
@liyichao
Copy link

we hit this too, any update?

@cbi42
Copy link
Member

cbi42 commented Jun 22, 2024

I don't think we can simply reset error on the last WAL. In this repro, nothing is written to memtable and WAL has a partial write batch record. After error is cleared, new writes will append to this WAL after the corrupted record, and may not be recovered after reopening.

@liyichao
Copy link

liyichao commented Jun 25, 2024

I don't think we can simply reset error on the last WAL. In this repro, nothing is written to memtable and WAL has a partial write batch record. After error is cleared, new writes will append to this WAL after the corrupted record, and may not be recovered after reopening.

where in the code do we insert the partial record? maybe we should undo the inserting when encountering no space error?

@cbi42
Copy link
Member

cbi42 commented Jul 7, 2024

where in the code do we insert the partial record? maybe we should undo the inserting when encountering no space error?

Sorry for the late reply. When writing a write batch to WAL, the write batch can be broken down into several WAL records and several writes. This happens in

IOStatus Writer::AddRecord(const WriteOptions& write_options,
. WAL record formats are explained in log_writer.h.

I think for auto-recovery, we should make sure not to write to this WAL anymore and make sure this WAL is not needed for this DB anymore. For the latter, we do flush during auto-recovery, but in this case memtable could be empty and flush is not done.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Confirmed RocksDB bugs up-for-grabs Up for grabs
Projects
None yet
4 participants