Summary: DBImpl::SyncWAL() releases db mutex before calling DBImpl::MarkLogsSynced(), while inside DBImpl::MarkLogsSynced() we assert there is none or one outstanding log file. However, a memtable switch can happen in between and causing two or outstanding logs there, failing the assert. The diff adds a unit test that repros the issue and fix the assert so that the unit test passes.
Test Plan: Run the new tests.
Reviewers: anthony, kolmike, yhchiang, IslamAbdelRahman, kradhakrishnan, andrewkr
Reviewed By: andrewkr
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D54621
Summary:
We started getting two kinds of crashes since we started using `DB::CompactFiles()`:
(1) `CompactFiles()` fails saying something like "/data/logdevice/4440/shard12/012302.sst: No such file or directory", and presumably makes DB read-only,
(2) DB fails to open saying "Corruption: Can't access /267000.sst: IO error: /data/logdevice/4440/shard1/267000.sst: No such file or directory".
AFAICT, both can be explained by background thread deleting compaction output as "obsolete" while it's being written, before it's committed to manifest. If it ends up committed to the manifest, we get (2); if compaction notices the disappearance and fails, we get (1). The internal tasks t10068021 and t10134177 have some details about the investigation that led to this.
Test Plan: `make -j check`; the new test fails to reopen the DB without the fix
Reviewers: yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, sdong
Differential Revision: https://reviews.facebook.net/D54561
Summary:
Introude SstFileManager::SetMaxAllowedSpaceUsage() that can be used to limit the maximum space usage allowed for RocksDB.
When this limit is exceeded WriteImpl() will fail and return Status::Aborted()
Test Plan: unit testing
Reviewers: yhchiang, anthony, andrewkr, sdong
Reviewed By: sdong
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D53763
Summary: There is an issue in DBImpl::WriteImpl where if an empty writebatch comes in and sync=true then the logs will be marked as being synced yet the sync never actually happens because there is no data in the writebatch. This causes the next incoming batch to hang while waiting for the logs to complete syncing. This fix syncs logs even if the writebatch is empty.
Test Plan: DoubleEmptyBatch unit test in transaction_test.
Reviewers: yoshinorim, hermanlee4, sdong, ngbronson, anthony
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D54057
Summary:
Add kSstFileTier to ReadTier, which allows Get and MultiGet to
read only directly from SST files and skip mem-tables.
kSstFileTier = 0x2 // data in SST files.
// Note that this ReadTier currently only supports
// Get and MultiGet and does not support iterators.
Test Plan: add new test in db_test.
Reviewers: anthony, IslamAbdelRahman, rven, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: igor, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D53511
Summary:
copy from task 8196669:
1) Optimistic transactions do not support batching writes from different threads.
2) Pessimistic transactions do not support batching writes if an expiration time is set.
In these 2 cases, we currently do not do any write batching in DBImpl::WriteImpl() because there is a WriteCallback that could decide at the last minute to abort the write. But we could support batching write operations with callbacks if we make sure to process the callbacks correctly.
To do this, we would first need to modify write_thread.cc to stop preventing writes with callbacks from being batched together. Then we would need to change DBImpl::WriteImpl() to call all WriteCallback's in a batch, only write the batches that succeed, and correctly set the state of each batch's WriteThread::Writer.
Test Plan: Added test WriteWithCallbackTest to write_callback_test.cc which creates multiple client threads and verifies that writes are batched and executed properly.
Reviewers: hermanlee4, anthony, ngbronson
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52863
Summary:
Before this diff, there were duplicated constants to refer to properties (user-
facing API had strings and InternalStats had an enum). I noticed these were
inconsistent in terms of which constants are provided, names of constants, and
documentation of constants. Overall it seemed annoying/error-prone to maintain
these duplicated constants.
So, this diff gets rid of InternalStats's constants and replaces them with a map
keyed on the user-facing constant. The value in that map contains a function
pointer to get the property value, so we don't need to do string matching while
holding db->mutex_. This approach has a side benefit of making many small
handler functions rather than a giant switch-statement.
Test Plan: db_properties_test passes, running "make commit-prereq -j32"
Reviewers: sdong, yhchiang, kradhakrishnan, IslamAbdelRahman, rven, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D53253
Summary:
Concurrent memtable adds were incorrectly computing
the last sequence number for a write batch group when the
write batches were not solitary. This is the cause of
https://github.com/facebook/mysql-5.6/issues/155
Test Plan:
1. unit tests
2. new unit test
3. parallel db_bench stress tests with batch size of 10 and asserts enabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: IslamAbdelRahman, MarkCallaghan, dhruba
Differential Revision: https://reviews.facebook.net/D53595
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
Summary:
Add a new class SstFileTracker that will be notified whenever a DB add/delete/move and sst file, it will also replace DeleteScheduler
SstFileTracker can be used later to abort writes when we exceed a specific size
Test Plan: unit tests
Reviewers: rven, anthony, yhchiang, sdong
Reviewed By: sdong
Subscribers: igor, lovro, march, dhruba
Differential Revision: https://reviews.facebook.net/D50469
Summary:
We can avoid the dependency by forward-declaring ColumnFamilyData and
then treating it as a black box. That means callers of ThreadStatusUtil need to
explicitly provide more options, even if they can be derived from the
ColumnFamilyData, since ThreadStatusUtil doesn't include the definition.
This is part of a series of diffs to eliminate circular dependencies between
directories (e.g., db/* files depending on util/* files and vice-versa).
Test Plan:
$ ./db_test --gtest_filter=DBTest.GetThreadStatus
$ make -j32 commit-prereq
Reviewers: sdong, yhchiang, IslamAbdelRahman
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D53361
Summary:
I split the db-specific test points out into a separate file under db/
directory. There were also a few bugs to fix in xfunc.{h,cc} that prevented it
from compiling previously; see https://reviews.facebook.net/D36825.
Test Plan:
compilation works now, below command works, will also run "make xfunc".
$ make check ROCKSDB_XFUNC_TEST='managed_new' tests-regexp='DBTest' -j32
Reviewers: sdong, yhchiang
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D53343
Summary:
SstFileWriter may create an sst file with no entries
Right now this will fail when being ingested using DB::AddFile() saying that the keys are corrupted
Test Plan: make check
Reviewers: yhchiang, rven, anthony, sdong
Reviewed By: sdong
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D52815
Makefile adjust paths for solaris build
Makefile enable _GLIBCXX_USE_C99 so that std::to_string is available
db_compaction_test.cc Initialise a variable to avoid a compilation error
db_impl.cc Include <alloca.h>
db_test.cc Include <alloca.h>
Environment.java recognise solaris envrionment
options_bulder.cc Make log unambiguous
geodb_impl.cc Make log and floor unambiguous
Summary:
While running the myrocks regression suite, I found that while
dropping a table soon after inserting rows into it resulted in an
assertion failure in CheckConsistencyForDeletes for not finding
a file which was recently added or moved. Marking the files to be
deleted as being compacted before calling LogAndApplyChange
fixed the assertion failures.
Test Plan: DBCompactionTest.DeleteFileRange
Reviewers: IslamAbdelRahman, anthony, yhchiang, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: dhruba, yoshinorim, leveldb
Differential Revision: https://reviews.facebook.net/D52599
Summary: DBImpl::GetLatestSequenceForKey() can do memcpy's to load a value that will never be used. This can be optimized by changing all the Get() functions called to optionally not fetch the value (and only fetch the sequencenumber).
Test Plan: optimistic_transaction_test and transaction_test
Reviewers: anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, hermanlee4
Differential Revision: https://reviews.facebook.net/D52227
Summary:
We need to clean the job context if we end up not deleting any
files because no files are in the range specified.
Test Plan: DBCompactionTest.DeleteFileRange
Reviewers: sdong, anthony, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D52467
Summary:
myrocks seems to build rocksdb using
-Wmissing-field-initializers (and treats warnings as errors). This diff
adds that flag to the rocksdb build, and fixes the compilation failures
that result. I have not checked for any other differences in the build
flags for rocksdb build as part of myrocks.
Test Plan: make check
Reviewers: sdong, rven
Reviewed By: rven
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D52443
Summary:
This is an initial diff for providing the ability to delete
files which are completely within a given range of keys.
Test Plan: DBCompactionTest.DeleteRange
Reviewers: IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D52293
Summary: Fix some CLANG errors introduced in 7d87f02799
Test Plan: Build with both of CLANG and gcc
Reviewers: rven, yhchiang, kradhakrishnan, anthony, IslamAbdelRahman, ngbronson
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52329
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
Summary:
Add CompactionReason to CompactionJobInfo
This will allow users to understand why compaction started which will help options tuning
Test Plan:
added new tests
make check -j64
Reviewers: yhchiang, anthony, kradhakrishnan, sdong, rven
Reviewed By: rven
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D51975
Summary:
This patch fixes https://github.com/facebook/mysql-5.6/issues/121
There is a recent change in rocksdb to disable auto compactions on startup: https://reviews.facebook.net/D51147. However, there is a small timing window where a column family needs to be compacted and schedules a compaction, but the scheduled compaction fails when it checks the disable_auto_compactions setting. The expectation is once the application is ready, it will call EnableAutoCompactions() to allow new compactions to go through. However, if the Column family is stalled because L0 is full, and no writes can go through, it is possible the column family may never have a new compaction request get scheduled. EnableAutoCompaction() should probably schedule an new flush and compaction event when it resets disable_auto_compaction.
Using InstallSuperVersionAndScheduleWork, we call SchedulePendingFlush,
SchedulePendingCompaction, as well as MaybeScheduleFlushOrcompaction on all the
column families to avoid the situation above.
This is still a first pass for feedback.
Could also just call SchedePendingFlush and SchedulePendingCompaction directly.
Test Plan:
Run on Asan build
cd _build-5.6-ASan/ && ./mysql-test/mtr --mem --big --testcase-timeout=36000 --suite-timeout=12000 --parallel=16 --suite=rocksdb,rocksdb_rpl,rocksdb_sys_vars --mysqld=--default-storage-engine=rocksdb --mysqld=--skip-innodb --mysqld=--default-tmp-storage-engine=MyISAM --mysqld=--rocksdb rocksdb_rpl.rpl_rocksdb_stress_crash --repeat=1000
Ensure that it no longer hangs during the test.
Reviewers: hermanlee4, yhchiang, anthony
Reviewed By: anthony
Subscribers: leveldb, yhchiang, dhruba
Differential Revision: https://reviews.facebook.net/D51747
Summary:
When there are waiting manual compactions, we need to signal
them after removing the current manual compaction from the deque.
Test Plan: ColumnFamilytTest.SameCFManualManualCommaction
Reviewers: anthony, IslamAbdelRahman, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D52119
Summary:
This patch update the Iterator API to introduce new functions that allow users to keep the Slices returned by key() valid as long as the Iterator is not deleted
ReadOptions::pin_data : If true keep loaded blocks in memory as long as the iterator is not deleted
Iterator::IsKeyPinned() : If true, this mean that the Slice returned by key() is valid as long as the iterator is not deleted
Also add a new option BlockBasedTableOptions::use_delta_encoding to allow users to disable delta_encoding if needed.
Benchmark results (using https://phabricator.fb.com/P20083553)
```
// $ du -h /home/tec/local/normal.4K.Snappy/db10077
// 6.1G /home/tec/local/normal.4K.Snappy/db10077
// $ du -h /home/tec/local/zero.8K.LZ4/db10077
// 6.4G /home/tec/local/zero.8K.LZ4/db10077
// Benchmarks for shard db10077
// _build/opt/rocks/benchmark/rocks_copy_benchmark \
// --normal_db_path="/home/tec/local/normal.4K.Snappy/db10077" \
// --zero_db_path="/home/tec/local/zero.8K.LZ4/db10077"
// First run
// ============================================================================
// rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s
// ============================================================================
// BM_StringCopy 1.73s 576.97m
// BM_StringPiece 103.74% 1.67s 598.55m
// ============================================================================
// Match rate : 1000000 / 1000000
// Second run
// ============================================================================
// rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s
// ============================================================================
// BM_StringCopy 611.99ms 1.63
// BM_StringPiece 203.76% 300.35ms 3.33
// ============================================================================
// Match rate : 1000000 / 1000000
```
Test Plan: Unit tests
Reviewers: sdong, igor, anthony, yhchiang, rven
Reviewed By: rven
Subscribers: dhruba, lovro, adsharma
Differential Revision: https://reviews.facebook.net/D48999
Summary:
List of changes:
1) Fix the snprintf() usage in cases where wrong variable was used to determine the output buffer size.
2) Remove unnecessary checks before calling delete operator.
3) Increase code correctness by using size_t type when getting vector's size.
4) Unify the coding style by removing namespace::std usage at the top of the file to confirm to the majority usage.
5) Fix various lint errors pointed out by 'arc lint'.
Test Plan:
Code review and build:
git diff
make clean
make -j 32 commit-prereq
arc lint
Reviewers: kradhakrishnan, sdong, rven, anthony, yhchiang, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D51849
Summary:
This diff provides a framework for doing manual
compactions in parallel with other compactions. We now have a deque of manual compactions. We also pass manual compactions as an argument from RunManualCompactions down to
BackgroundCompactions, so that RunManualCompactions can be reentrant.
Parallelism is controlled by the two routines
ConflictingManualCompaction to allow/disallow new parallel/manual
compactions based on already existing ManualCompactions. In this diff, by default manual compactions still have to run exclusive of other compactions. However, by setting the compaction option, exclusive_manual_compaction to false, it is possible to run other compactions in parallel with a manual compaction. However, we are still restricted to one manual compaction per column family at a time. All of these restrictions will be relaxed in future diffs.
I will be adding more tests later.
Test Plan: Rocksdb regression + new tests + valgrind
Reviewers: igor, anthony, IslamAbdelRahman, kradhakrishnan, yhchiang, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D47973
Summary:
Currently, transactions can fail even if there is no actual write conflict. This is due to relying on only the memtables to check for write-conflicts. Users have to tune memtable settings to try to avoid this, but it's hard to figure out exactly how to tune these settings.
With this diff, TransactionDB will use both memtables and SST files to determine if there are any write conflicts. This relies on the fact that BlockBasedTable stores sequence numbers for all writes that happen after any open snapshot. Also, D50295 is needed to prevent SingleDelete from disappearing writes (the TODOs in this test code will be fixed once the other diff is approved and merged).
Note that Optimistic transactions will still rely on tuning memtable settings as we do not want to read from SST while on the write thread. Also, memtable settings can still be used to reduce how often TransactionDB needs to read SST files.
Test Plan: unit tests, db bench
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb, yoshinorim
Differential Revision: https://reviews.facebook.net/D50475
Summary: For Transactions, we want to start using the SST files to do write conflict checking. To do this, we need to make sure that compaction never removes all writes if an earlier snapshot exists. So I had to change the way we process SingleDeletes to sometimes leave a SingleDelete behind when we encounter a Put followed by a SingleDelete. See the comments in this diff for a more detailed explanation.
Test Plan: added more unit tests
Reviewers: rven, igor, kradhakrishnan, IslamAbdelRahman, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D50295
Summary:
This patch fix a race condition in persisting options which will cause a crash when:
* Thread A obtain cf options and start to persist options based on that cf options.
* Thread B kicks in and finish DropColumnFamily and delete cf_handle.
* Thread A wakes up and tries to finish the persisting options and crashes.
Test Plan: Add a test in column_family_test that can reproduce the crash
Reviewers: anthony, IslamAbdelRahman, rven, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D51717
Summary:
D51183 was reverted due to breaking the LITE build.
This diff is the same as D51183 but with a fix for the LITE BUILD(D51693)
Test Plan: run all unit tests
Reviewers: sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D51711
Summary:
D50475 enables using SST files for transaction write-conflict checking. In order for this to work, we need to make sure not to compact out SingleDeletes when there is an earlier transaction snapshot(D50295). If there is a long-held snapshot, this could reduce the benefit of the SingleDelete optimization.
This diff allows Transactions to mark snapshots as being used for write-conflict checking. Then, during compaction, we will be able to optimize SingleDeletes better in the future.
This diff adds a flag to SnapshotImpl which is used by Transactions. This diff also passes the earliest write-conflict snapshot's sequence number to CompactionIterator. This diff does not actually change Compaction (after this diff is pushed, D50295 will be able to use this information).
Test Plan: no behavior change, ran existing tests
Reviewers: rven, kradhakrishnan, yhchiang, IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D51183
Summary:
This patch fix a race condition in persisting options which will cause a crash when:
* Thread A obtain cf options and start to persist options based on that cf options.
* Thread B kicks in and finish DropColumnFamily and delete cf_handle.
* Thread A wakes up and tries to finish the persisting options and crashes.
Test Plan: Add a test in column_family_test that can reproduce the crash
Reviewers: anthony, IslamAbdelRahman, rven, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D51609
Summary:
Fixes T8781168.
Added a new function EnableAutoCompactions in db.h to be publicly
avialable. This allows compaction to be re-enabled after disabling it via
SetOptions
Refactored code to set the dbptr earlier on in TransactionDB::Open and DB::Open
Temporarily disable auto_compaction in TransactionDB::Open until dbptr is set to
prevent race condition.
Test Plan:
Ran make all check
verified fix on myrocks side:
was able to reproduce the seg fault with
../tools/mysqltest.sh --mem --force rocksdb.drop_table
method was to manually sleep the thread after DB::Open but before TransactionDB ptr was
assigned in transaction_db_impl.cc:
DB::Open(db_options, dbname, column_families_copy, handles, &db);
clock_t goal = (60000 * 10) + clock();
while (goal > clock());
...dbptr(aka rdb) gets assigned below
verified my changes fixed the issue.
Also added unit test 'ToggleAutoCompaction' in transaction_test.cc
Reviewers: hermanlee4, anthony
Reviewed By: anthony
Subscribers: alex, dhruba
Differential Revision: https://reviews.facebook.net/D51147
Summary: When option.db_write_buffer_size is hit, we currently flush all column families. Move to flush the column family with the largest active memt table instead. In this way, we can avoid too many small files in some cases.
Test Plan: Modify test DBTest.SharedWriteBuffer to work with the updated behavior
Reviewers: kradhakrishnan, yhchiang, rven, anthony, IslamAbdelRahman, igor
Reviewed By: igor
Subscribers: march, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D51291
Summary: Handle multiple calls to DBImpl::PauseBackgroundWork() and DBImpl::ContinueBackgroundWork()
Test Plan: rocksdb.information_schema handles this case.
Reviewers: igor
Reviewed By: igor
Subscribers: hermanlee4, jkedgar, dhruba
Differential Revision: https://reviews.facebook.net/D50781
Summary:
Since level 0 files can overlap, two level 0 compactions cannot
run in parallel. Compact files needs to check this before running a
compaction.
Test Plan: CompactFilesTest.L0ConflictsFiles
Reviewers: igor, IslamAbdelRahman, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D50079
Summary:
There's no need for WriteImpl to flatten the write batch group
into a single WriteBatch if the WAL is disabled. This diff moves the
flattening into the WAL step, and skips flattening entirely if it isn't
needed. It's good for about 5% speedup on a multi-threaded workload
with no WAL.
This diff also adds clarifying comments about the chance for partial
failure of WriteBatchInternal::InsertInto, and always sets bg_error_ if
the memtable state diverges from the logged state or if a WriteBatch
succeeds only partially.
Benchmark for speedup:
db_bench -benchmarks=fillrandom -threads=16 -batch_size=1 -memtablerep=skip_list -value_size=0 --num=200000 -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000
Test Plan: asserts + make check
Reviewers: sdong, igor
Reviewed By: igor
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D50583
Summary:
This patch allows rocksdb to persist options into a file on
DB::Open, SetOptions, and Create / Drop ColumnFamily.
Options files are created under the same directory as the rocksdb
instance.
In addition, this patch also adds a fail_if_missing_options_file in DBOptions
that makes any function call return non-ok status when it is not able to
persist options properly.
// If true, then DB::Open / CreateColumnFamily / DropColumnFamily
// / SetOptions will fail if options file is not detected or properly
// persisted.
//
// DEFAULT: false
bool fail_if_missing_options_file;
Options file names are formatted as OPTIONS-<number>, and RocksDB
will always keep the latest two options files.
Test Plan:
Add options_file_test.
options_test
column_family_test
Reviewers: igor, IslamAbdelRahman, sdong, anthony
Reviewed By: anthony
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D48285
Summary:
MyRocks testing found an issue that while iterating over keys
that are outside the prefix, sometimes wrong results were seen for keys
outside the prefix. We now tighten the range of keys seen with a new
read option called prefix_seen_at_start. This remembers the starting
prefix and then compares it on a Next for equality of prefix. If they
are from a different prefix, it sets valid to false.
Test Plan: PrefixTest.PrefixValid
Reviewers: IslamAbdelRahman, sdong, yhchiang, anthony
Reviewed By: anthony
Subscribers: spetrunia, hermanlee4, yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D50211