facebook / rocksdb

A library that provides an embeddable, persistent key-value store for fast storage.
http://rocksdb.org
GNU General Public License v2.0
28.52k stars 6.3k forks source link

RocksDB crash during ingest external file #5878

Closed koldat closed 5 years ago

koldat commented 5 years ago

There is an issue that under some conditions ingest file tries to erase uninitialized iterator:

  std::list<uint64_t>::iterator pending_output_elem;
  size_t total = 0;
  for (const auto& arg : args) {
    total += arg.external_files.size();
  }
  uint64_t next_file_number = 0;
  Status status = ReserveFileNumbersBeforeIngestion(
      static_cast<ColumnFamilyHandleImpl*>(args[0].column_family)->cfd(), total,
      &pending_output_elem, &next_file_number);
  if (!status.ok()) {
    !!!HERE WE CAN END UP WITH NOT INITIALIZED ITERATOR!!!
    InstrumentedMutexLock l(&mutex_);
    ReleaseFileNumberFromPendingOutputs(pending_output_elem);
    return status;
  }

Why it happens? On some NFS databases we are hitting some OS race condition where ingest move file succeeds, but compaction job that is started immediately does not see that file yet. It looks like this:

2019/09/19-08:07:13.785002 7fc921d56700 [/external_sst_file_ingestion_job.cc:282] [AddFile] External SST file ./data/nass_archivedb/1.spool_1568879400.0.ingest.tmp was ingested in L1 with path ./data/nass_archivedb/000084.sst (global_seqno=182) {"table_properties": {"data_size": 48158985, "index_size": 730761, "top_level_index_size": 8630, "index_partitions": 178, "filter_size": 0, "raw_key_size": 100522972, "raw_average_key_size": 44, "raw_value_size": 431207811, "raw_average_value_size": 188, "num_data_blocks": 14856, "num_entries": 2284613, "filter_policy_name": ""}}
2019/09/19-08:07:13.785177 7fc93d803700 [/compaction_job.cc:1645] [default] [JOB 380] Compacting 1@1 + 1@2 files to L2, score 1.09
2019/09/19-08:07:13.785189 7fc93d803700 [/compaction_job.cc:1649] [default] Compaction start summary: Base version 137 Base level 1, inputs: [84(46MB)], [83(45MB)]
2019/09/19-08:07:13.785219 7fc93d803700 EVENT_LOG_v1 {"time_micros": 1568880433785200, "job": 380, "event": "compaction_started", "compaction_reason": "LevelMaxLevelSize", "files_L1": [84], "files_L2": [83], "score": 1.0863, "input_data_size": 95582278}
2019/09/19-08:07:13.799839 7fc93d803700 [WARN] [/db_impl_compaction_flush.cc:2660] Compaction error: IO error: No such file or directoryWhile open a file for random read: ./data/nass_archivedb/000084.sst: No such file or directory
2019/09/19-08:07:13.799872 7fc93d803700 (Original Log Time 2019/09/19-08:07:13.799792) [/compaction_job.cc:768] [default] compacted to: files[0 5 8 8 8 8 8] max score 0.91, MB/sec: 6573.3 rd, 0.0 wr, level 2, files in(1, 1) out(0) MB in(46.2, 45.0) out(0.0), read-write-amplify(2.0) write-amplify(0.0) IO error: No such file or directoryWhile open a file for random read: ./data/nass_archivedb/000084.sst: No such file or directory, records in: 4543932, records dropped: 0 output_compression: ZSTD
2019/09/19-08:07:13.799881 7fc93d803700 (Original Log Time 2019/09/19-08:07:13.799824) EVENT_LOG_v1 {"time_micros": 1568880433799812, "job": 380, "event": "compaction_finished", "compaction_time_micros": 14541, "compaction_time_cpu_micros": 7774, "output_level": 2, "num_output_files": 0, "total_output_size": 0, "num_input_records": 0, "num_output_records": 0, "num_subcompactions": 1, "output_compression": "ZSTD", "num_single_delete_mismatches": 0, "num_single_delete_fallthrough": 0, "lsm_state": [0, 5, 8, 8, 8, 8, 8]}
2019/09/19-08:07:13.799886 7fc93d803700 [ERROR] [/db_impl_compaction_flush.cc:2212] Waiting after background compaction error: IO error: No such file or directoryWhile open a file for random read: ./data/nass_archivedb/000084.sst: No such file or directory, Accumulated background error counts: 1

You can see file was ingested, but then it is not readable. Compaction fails with IO_ERROR which is mapped as kFatalError with paranoid checks. Error handler then stops DB which is still expected. But next call to ingest does:

Status DBImpl::ReserveFileNumbersBeforeIngestion(
    ColumnFamilyData* cfd, uint64_t num,
    std::list<uint64_t>::iterator* pending_output_elem,
    std::auto_ptr<std::list<uint64_t>::iterator>& pending_output_elem,
    uint64_t* next_file_number) {
  Status s;
  SuperVersionContext dummy_sv_ctx(true /* create_superversion */);
  assert(nullptr != pending_output_elem);
  assert(nullptr != next_file_number);
  InstrumentedMutexLock l(&mutex_);
  if (error_handler_.IsDBStopped()) {
    // Do not ingest files when there is a bg_error
    !!!HERE WE CAN END UP WITH NOT INITIALIZED ITERATOR!!!
    return error_handler_.GetBGError();
  }
  *pending_output_elem = CaptureCurrentFileNumberInPendingOutputs();

Thus not initialized iterator thus crash. It should not crash even if we switch to read only database.

Expected behavior

Process should not crash.

Actual behavior

Crash callstack:

#1561 <signal handler called>
#1562 0x00007f404a09ed70 in std::__detail::_List_node_base::_M_unhook() () from /tmp/librocksdbjni12653991741000616685.so
#1563 0x00007f4049d649a4 in _M_erase (this=<optimized out>, __position=...) at /opt/rh/devtoolset-2/root/usr/include/c++/4.8.2/bits/stl_list.h:1570
#1564 erase (this=<optimized out>, __position=...) at /opt/rh/devtoolset-2/root/usr/include/c++/4.8.2/bits/list.tcc:112
#1565 rocksdb::DBImpl::ReleaseFileNumberFromPendingOutputs (this=<optimized out>, v=...) at db/db_impl.cc:2537
#1566 0x00007f4049d74817 in rocksdb::DBImpl::IngestExternalFiles (this=this@entry=0x7f40498a1400, args=...) at db/db_impl.cc:3347
#1567 0x00007f4049d66668 in rocksdb::DBImpl::IngestExternalFile (this=0x7f40498a1400, column_family=<optimized out>, external_files=..., ingestion_options=...) at db/db_impl.cc:3296
---Type <return> to continue, or q <return> to quit---
#1568 0x00007f4049cda59d in Java_org_rocksdb_RocksDB_ingestExternalFile (env=<optimized out>, jdb_handle=139914088420352, jcf_handle=139913919581312, jfile_path_list=0x7f402c77c930, jfile_path_list_len=1,
    jingest_external_file_options_handle=139914127802376) at java/rocksjni/rocksjni.cc:2924

Steps to reproduce the behavior

I was not able to reproduce on local drive. It happens once a day on NFS. We are heavily using ingest file. When it causes compaction there is high chance to make this happen.

  1. Use NFS
  2. Use Ingest that cause compaction (LevelMaxLevelSize, LevelL0FilesNum, ...)

I have a fix and will create pull request.

yiwu-arbug commented 5 years ago

Just curious, will fsync before ingestion fix the issue for NFS? i.e. https://github.com/facebook/rocksdb/commit/2730fe693edf306aad11a48491cfe3be4c178a47

koldat commented 5 years ago

Thanks @yiwu-arbug it is very nice change! I have merged it into our production build. Hopefully will help. We are using SSTFileWriters which are flushing at and. To be sure I have added use_fsync=true for writer. I will keep you posted.

Your change is maybe fixing root cause, but the crash is happening because iterators are not properly used in subsequent parts of RocksDB. It took me a while to retrieve crash dump from production (we are running in k8s) and compile RocksJNI with separated debug symbols. I think it will help many people. Crash is the worst that can happen.