Skip to content

RocksDB FAQ

Yueh-Hsuan Chiang edited this page Apr 5, 2016 · 61 revisions

Q: If my process crashes, can it corrupt the database?

A: No, but data in the un-flushed mem-tables might be lost if Write-Ahead-Log (WAL) is disabled.

Q: If my machine crashes and rebooted, will RocksDB preserve the data?

A: Data is synced when you issue a sync write (write with WriteOptions.sync=true), call DB::SyncWAL(), or when memtables are flushed.

Q: Does RocksDB throw exceptions?

A: No, RocksDB returns rocksdb::Status to indicate any error. However, RocksDB does not catch exceptions thrown by STL or other dependencies. For instance, so it's possible that you will see std::bad_malloc when memory allocation fails, or similar exceptions in other situations.

Q: How to know the number of keys stored in a RocksDB database?

A: Use GetIntProperty(cf_handle, “rocksdb.estimate-num-keys") to obtain an estimated number of keys stored in a column family, or use GetAggregatedIntProperty(“rocksdb.estimate-num-keys", &num_keys) to obtain an estimated number of keys stored in the whole RocksDB database.

Q: Why GetIntProperty can only returns an estimated number of keys in a RocksDB database?

A: Obtaining an accurate number of keys in any LSM databases like RocksDB is a challenging problem as they have duplicate keys and deletion entries (i.e., tombstones) that will require a full compaction in order to get an accurate number of keys. In addition, if the RocksDB database contains merge operators, it will also make the estimated number of keys less accurate.

Q: Is basic operations Put(), Write(), Get() and NewIterator() thread safe?

A: Yes.

Q: Can I write to RocksDB using multiple processes?

A: No. However, it can be opened in read-only mode from multiple processes.

Q: Does RocksDB support multi-process read access?

A: RocksDB support multi-process read only process without writing the database. This can be done by opening the database with DB::OpenForReadOnly() call.

Q: Is it safe to close RocksDB while another thread is issuing read, write or manual compaction requests?

A: No. The users of RocksDB need to make sure all functions have finished before they close RocksDB.

Q: What's the maximum key and value sizes supported?

A: In general, RocksDB is not designed for large keys. The maximum recommended sizes for key and value are 8MB and 3GB respectively.

Q: Can I run RocksDB and store the data on HDFS?

A: Yes, by using the Env returned by NewHdfsEnv(), RocksDB will store data on HDFS. However, the file lock is currently not supported in HDFS Env.

Q: Can I preserve a “snapshot” of RocksDB and later roll back the DB state to it?

A: Yes, via the BackupEngine or Checkpoint.

Q: What's the fastest way to load data into RocksDB?

A: A fast way to direct insert data to the DB:

  1. using single writer thread and insert in sorted order
  2. batch hundreds of keys into one write batch
  3. use vector memtable
  4. make sure options.max_background_flushes is at least 4
  5. before inserting the data, disable automatic compaction, set options.level0_file_num_compaction_trigger, options.level0_slowdown_writes_trigger and options.level0_stop_writes_trigger to very large. After inserting all the data, issue a manual compaction.

3-5 will be automatically done if you call Options::PrepareForBulkLoad() to your option

If you can pre-process the data offline before inserting. There is a faster way: you can sort the data, generate SST files with non-overlapping ranges in parallel and bulkload the SST files. See https://github.com/facebook/rocksdb/wiki/Creating-and-Ingesting-SST-files

Q: Does RocksJava support all the features?

A: We are working toward making RocksJava feature compatible. However, you're more than welcomed to submit pull request if you find something is missing

Q: Who are using RocksDB?

A: https://github.com/facebook/rocksdb/blob/master/USERS.md

Q: What is the correct way to delete the DB? Can I simply call DestroyDB() on a live DB?

A: Close the DB then destroy the DB is the correct way. Calling DestroyDB() on a live DB is an undefined behavior.

Q: What is the difference between DestroyDB() and directly deleting the DB directory manually?

A: The major difference is that DestroyDB() will take care of the case where the RocksDB database is stored in multiple directories. For instance, a single DB can be configured to store its data in multiple directories by specifying different paths to DBOptions::db_paths, DBOptions::db_log_dir, and DBOptions::wal_dir.

Q: Does BackupableDB create a point-in-time snapshot of the database?

A: Yes when BackupOptions::backup_log_files = true or flush_before_backup is set to true when calling CreateNewBackup().

Q: Does the backup process affect accesses to the database in the mean while?

A: No, you can keep reading and writing to the database at the same time.

Q: Any better way to dump key-value pairs generated by map-reduce job into RocksDB?

A: A better way is to use SstFileWriter, which allows you to directly create RocksDB SST files and add them to a rocksdb database. However, if you're adding SST files to an existing RocksDB database, then its key-range must not overlap with the database. https://github.com/facebook/rocksdb/wiki/Creating-and-Ingesting-SST-files

Q: Is it safe to configure different prefix extractor for different column family?

A: Yes.

Q: Can I change the prefix extractor?

A: No. Once you've specified a prefix extractor, you cannot change it. However, you can disable it by specifying a null value.

Q: What is the absolute minimum version of gcc that we need to build RocksDB?

A: 4.7, but 4.8 or up is recommended.

Q: How to configure rocksdb to use multiple disks?

A: You can create a single filesystem (ext3, xfs, etc) on multiple disks. Then you can run rocksdb on that single file system. Some tips when using disks:

  • if using RAID then don't use a too small RAID stripe size (64kb is too small, 1MB would be excellent).
  • consider enabling compaction readahead by specifying ColumnFamilyOptions::compaction_readahead_size to at least 2MB.
  • if workload is write-heavy then have enough compaction threads to keep the disks busy
  • consider enabling async write behind for compaction

Q: Is it safe to directly copy an open RocksDB instance?

A: No, unless the RocksDB instance is opened in read-only mode.

Q: Can I open RocksDB with a different compression type and still read old data?

A: Yes, since rocksdb stored the compression information in each SST file and performs decompression accordingly, you can change the compression and the db will still be able to read existing files. In addition, you can also specify different compression for different level by specifying ColumnFamilyOptions::compression_per_level.

Contents

Clone this wiki locally