Summary:
The patch adds a new BlobDB configuration option `blob_compaction_readahead_size`
that can be used to enable prefetching data from blob files during compaction.
This is important when using storage with higher latencies like HDDs or remote filesystems.
If enabled, prefetching is used for all cases when blobs are read during compaction,
namely garbage collection, compaction filters (when the existing value has to be read from
a blob file), and `Merge` (when the value of the base `Put` is stored in a blob file).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/9187
Test Plan: Ran `make check` and the stress/crash test.
Reviewed By: riversand963
Differential Revision: D32565512
Pulled By: ltamasi
fbshipit-source-id: 87be9cebc3aa01cc227bec6b5f64d827b8164f5d
Summary:
Provide support for Merge operation with base values during
Compaction in IntegratedBlobDB.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8445
Test Plan: Add new unit test
Reviewed By: ltamasi
Differential Revision: D29343949
Pulled By: akankshamahajan15
fbshipit-source-id: 844f6f02f93388a11e6e08bda7bb3a2a28e47c70
Summary:
For performance purposes, the lower level routines were changed to use a SystemClock* instead of a std::shared_ptr<SystemClock>. The shared ptr has some performance degradation on certain hardware classes.
For most of the system, there is no risk of the pointer being deleted/invalid because the shared_ptr will be stored elsewhere. For example, the ImmutableDBOptions stores the Env which has a std::shared_ptr<SystemClock> in it. The SystemClock* within the ImmutableDBOptions is essentially a "short cut" to gain access to this constant resource.
There were a few classes (PeriodicWorkScheduler?) where the "short cut" property did not hold. In those cases, the shared pointer was preserved.
Using db_bench readrandom perf_level=3 on my EC2 box, this change performed as well or better than 6.17:
6.17: readrandom : 28.046 micros/op 854902 ops/sec; 61.3 MB/s (355999 of 355999 found)
6.18: readrandom : 32.615 micros/op 735306 ops/sec; 52.7 MB/s (290999 of 290999 found)
PR: readrandom : 27.500 micros/op 871909 ops/sec; 62.5 MB/s (367999 of 367999 found)
(Note that the times for 6.18 are prior to revert of the SystemClock).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8033
Reviewed By: pdillinger
Differential Revision: D27014563
Pulled By: mrambacher
fbshipit-source-id: ad0459eba03182e454391b5926bf5cdd45657b67
Summary:
Introduces and uses a SystemClock class to RocksDB. This class contains the time-related functions of an Env and these functions can be redirected from the Env to the SystemClock.
Many of the places that used an Env (Timer, PerfStepTimer, RepeatableThread, RateLimiter, WriteController) for time-related functions have been changed to use SystemClock instead. There are likely more places that can be changed, but this is a start to show what can/should be done. Over time it would be nice to migrate most (if not all) of the uses of the time functions from the Env to the SystemClock.
There are several Env classes that implement these functions. Most of these have not been converted yet to SystemClock implementations; that will come in a subsequent PR. It would be good to unify many of the Mock Timer implementations, so that they behave similarly and be tested similarly (some override Sleep, some use a MockSleep, etc).
Additionally, this change will allow new methods to be introduced to the SystemClock (like https://github.com/facebook/rocksdb/issues/7101 WaitFor) in a consistent manner across a smaller number of classes.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7858
Reviewed By: pdillinger
Differential Revision: D26006406
Pulled By: mrambacher
fbshipit-source-id: ed10a8abbdab7ff2e23d69d85bd25b3e7e899e90
Summary:
…when unused. Causes many calls to clock_gettime, impacting performance.
Was looking for something else via Linux "perf" command when I spotted heavy usage of clock_gettime during a compaction. Our product heavily uses the rocksdb::Options::merge_operator. MergeHelper::FilterMerge() properly tests if timing is enabled/disabled upon entry, but not on exit. This patch fixes the exit.
Note: the entry test also verifies if "nullptr!=stats_". This test is redundant to code within ShouldReportDetailedTime(). Therefore I omitted it in my change.
merge_test.cc updated with test that shows failure before merge_helper.cc change ... and fix after change.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7867
Reviewed By: jay-zhuang
Differential Revision: D25960175
Pulled By: ajkr
fbshipit-source-id: 56e66d7eb6ae5eae89c8e0d5a262bd2905a226b6
Summary:
Fixes Issue https://github.com/facebook/rocksdb/issues/7497
When allow_data_in_errors db_options is set, log error key details in `ParseInternalKey()`
Have fixed most of the calls. Have few TODOs still pending - because have to make more deeper changes to pass in the allow_data_in_errors flag. Will do those in a separate PR later.
Tests:
- make check
- some of the existing tests that exercise the "internal key too small" condition are: dbformat_test, cuckoo_table_builder_test
- some of the existing tests that exercise the corrupted key path are: corruption_test, merge_helper_test, compaction_iterator_test
Example of new status returns:
- Key too small - `Corrupted Key: Internal Key too small. Size=5`
- Corrupt key with allow_data_in_errors option set to false: `Corrupted Key: '<redacted>' seq:3, type:3`
- Corrupt key with allow_data_in_errors option set to true: `Corrupted Key: '61' seq:3, type:3`
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7515
Reviewed By: ajkr
Differential Revision: D24240264
Pulled By: ramvadiv
fbshipit-source-id: bc48f5d4475ac19d7713e16df37505b31aac42e7
Summary:
Fix few test cases and add them in ASSERT_STATUS_CHECKED build.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7427
Test Plan:
1. ASSERT_STATUS_CHECKED=1 make -j48 check,
2. travis build for ASSERT_STATUS_CHECKED,
3. Without ASSERT_STATUS_CHECKED: make check -j64, CircleCI build and travis build
Reviewed By: pdillinger
Differential Revision: D23909983
Pulled By: akankshamahajan15
fbshipit-source-id: 42d7e4aea972acb9fcddb7ca73fcb82f93272434
Summary:
When dynamically linking two binaries together, different builds of RocksDB from two sources might cause errors. To provide a tool for user to solve the problem, the RocksDB namespace is changed to a flag which can be overridden in build time.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6433
Test Plan: Build release, all and jtest. Try to build with ROCKSDB_NAMESPACE with another flag.
Differential Revision: D19977691
fbshipit-source-id: aa7f2d0972e1c31d75339ac48478f34f6cfcfb3e
Summary:
Flush/compaction use `MergeUntil` which has a special code path to
handle a merge ending with a non-`Merge` point key. In particular if
that key is a `Put` we forgot to check whether it is covered by a range
tombstone. If it is covered then we must not include it in the following call
to `TimedFullMerge`.
Fixes#5392.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5406
Differential Revision: D15611144
Pulled By: sagar0
fbshipit-source-id: ba6a7863ca2d043f591de78fd0c4f4561f0c500e
Summary:
Statistics cost too much CPU for some use cases. Add two stats levels
so that people can choose to skip two types of expensive stats, timers and
histograms.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5027
Differential Revision: D14252765
Pulled By: siying
fbshipit-source-id: 75ecec9eaa44c06118229df4f80c366115346592
Summary:
The patch fixes the following analyze error by checking the return status of ParseInternalKey.
```
db/merge_helper.cc:306:23: warning: The right operand of '==' is a garbage value
assert(kTypeMerge == orig_ikey.type);
```
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4937
Differential Revision: D13908506
Pulled By: maysamyabandeh
fbshipit-source-id: 68d7771e75519da3d4bd807fd231675ec12093f6
Summary:
Measure CPU time consumed for a compaction and report it in the stats report
Enable NowCPUNanos() to work for MacOS
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4889
Differential Revision: D13701276
Pulled By: zinoale
fbshipit-source-id: 5024e5bbccd4dd10fd90d947870237f436445055
Summary:
Previously compaction was not collapsing operands for a first
key on a layer, even in cases when it was its root of history. Some
tests (CompactionJobTest.NonAssocMerge) was actually accounting
for that bug,
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4909
Differential Revision: D13781169
Pulled By: finik
fbshipit-source-id: d2de353ecf05bec39b942cd8d5b97a8dc445f336
Summary:
Compaction iterator keep a copy of list of live snapshots at the beginning of compaction, and then query snapshot checker to verify if values of a sequence number is visible to these snapshots. However when the snapshot is released in the middle of compaction, the snapshot checker implementation (i.e. WritePreparedSnapshotChecker) may remove info with the snapshot and may report incorrect result, which lead to values being compacted out when it shouldn't. This patch conservatively keep the values if snapshot checker determines that the snapshots is released.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4858
Differential Revision: D13617146
Pulled By: maysamyabandeh
fbshipit-source-id: cf18a94f6f61a94bcff73c280f117b224af5fbc3
Summary:
Now that v2 is fully functional, the v1 aggregator is removed.
The v2 aggregator has been renamed.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4778
Differential Revision: D13495930
Pulled By: abhimadan
fbshipit-source-id: 9d69500a60a283e79b6c4fa938fc68a8aa4d40d6
Summary:
RangeDelAggregatorV2 now supports ShouldDelete calls on
snapshot stripes and creation of range tombstone compaction iterators.
RangeDelAggregator is no longer used on any non-test code path, and will
be removed in a future commit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4758
Differential Revision: D13439254
Pulled By: abhimadan
fbshipit-source-id: fe105bcf8e3d4a2df37a622d5510843cd71b0401
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
Summary:
For example calling CompactionFilter is always timed and gives the user no way to disable.
This PR will disable the timer if `Statistics::stats_level_` (which is part of DBOptions) is `kExceptDetailedTimers`
Closes https://github.com/facebook/rocksdb/pull/4029
Differential Revision: D8583670
Pulled By: miasantreble
fbshipit-source-id: 913be9fe433ae0c06e88193b59d41920a532307f
Summary:
CompactionIterator invoke MergeHelper::MergeUntil() to do partial merge between snapshot boundaries. Previously it only depend on sequence number to tell snapshot boundary, but we also need to make use of snapshot_checker to verify visibility of the merge operands to the snapshots. For example, say there is a snapshot with seq = 2 but only can see data with seq <= 1. There are three merges, each with seq = 1, 2, 3. A correct compaction output would be (1),(2+3). Without taking snapshot_checker into account when generating merge result, compaction will generate output (1+2),(3).
By filtering uncommitted keys with read callback, the read path already take care of merges well and don't need additional updates.
Closes https://github.com/facebook/rocksdb/pull/3475
Differential Revision: D6926087
Pulled By: yiwu-arbug
fbshipit-source-id: 8f539d6f897cfe29b6dc27a8992f68c2a629d40a
Summary:
Added a function `MergeOperator::DoesAllowSingleMergeOperand()` to allow invoking a merge operator even with a single merge operand, if overriden.
This is needed for Cassandra-on-RocksDB work. All Cassandra writes are through merges and this will allow a single merge-value to be updated in the merge-operator invoked via a compaction, if needed, due to an expired TTL.
Closes https://github.com/facebook/rocksdb/pull/2721
Differential Revision: D5608706
Pulled By: sagar0
fbshipit-source-id: f299f9f91c4d1ac26e48bd5906e122c1c5e5f3fc
Summary:
In the condition:
```
if (range_del_agg != nullptr &&
range_del_agg->ShouldDelete(
iter->key(),
RangeDelAggregator::RangePositioningMode::kForwardTraversal) &&
filter != CompactionFilter::Decision::kRemoveAndSkipUntil) {
...
}
```
it could be possible that all the work done in `range_del_agg->ShouldDelete` is wasted due to not having the right `filter` value later on.
Instead, check `filter` value before even calling `range_del_agg->ShouldDelete`, which is a much more involved function.
Closes https://github.com/facebook/rocksdb/pull/2690
Differential Revision: D5568931
Pulled By: sagar0
fbshipit-source-id: 17512d52360425c7ae9de7675383f5d7bc3dad58
Summary:
Add a histogram in statistics to help users understand how many merge operands they merge.
Closes https://github.com/facebook/rocksdb/pull/2373
Differential Revision: D5139983
Pulled By: siying
fbshipit-source-id: 61b9ba8ca83f358530a4833d68f0103b56a0e182
Summary:
Move some files under util/ to new directories env/, monitoring/ options/ and cache/
Closes https://github.com/facebook/rocksdb/pull/2090
Differential Revision: D4833681
Pulled By: siying
fbshipit-source-id: 2fd8bef
Summary:
…action
The two options, min_partial_merge_operands and verify_checksums_in_compaction, are not seldom used. Remove them to reduce the total number of options. Also remove them from Java and C interface.
Closes https://github.com/facebook/rocksdb/pull/1902
Differential Revision: D4601219
Pulled By: siying
fbshipit-source-id: aad4cb2
Summary:
DB shutdown aborts running compactions by setting an atomic shutting_down=true that CompactionJob periodically checks. Without this PR it checks it before processing every _output_ value. If compaction filter filters everything out, the compaction is uninterruptible. This PR adds checks for shutting_down on every _input_ value (in CompactionIterator and MergeHelper).
There's also some minor code cleanup along the way.
Closes https://github.com/facebook/rocksdb/pull/1639
Differential Revision: D4306571
Pulled By: yiwu-arbug
fbshipit-source-id: f050890
Summary:
When deletion-collapsing mode is enabled (i.e., for DBIter/CompactionIterator), we maintain position in the tombstone maps across calls to ShouldDelete(). Since iterators often access keys sequentially (or reverse-sequentially), scanning forward/backward from the last position can be faster than binary-searching the map for every key.
- When Next() is invoked on an iterator, we use kForwardTraversal to scan forwards, if needed, until arriving at the range deletion containing the next key.
- Similarly for Prev(), we use kBackwardTraversal to scan backwards in the range deletion map.
- When the iterator seeks, we use kBinarySearch for repositioning
- After tombstones are added or before the first ShouldDelete() invocation, the current position is set to invalid, which forces kBinarySearch to be used.
- Non-iterator users (i.e., Get()) use kFullScan, which has the same behavior as before---scan the whole map for every key passed to ShouldDelete().
Closes https://github.com/facebook/rocksdb/pull/1701
Differential Revision: D4350318
Pulled By: ajkr
fbshipit-source-id: 5129b76
Summary:
This adds the ability for compaction filter to say "drop this key-value, and also drop everything up to key x". This will cause the compaction to seek input iterator to x, without reading the data. This can make compaction much faster when large consecutive chunks of data are filtered out. See the changes in include/rocksdb/compaction_filter.h for the new API.
Along the way this diff also adds ability for compaction filter changing merge operands, similar to how it can change values; we're not going to use this feature, it just seemed easier and cleaner to implement it than to document that it's not implemented :)
The diff is not as big as it may seem, about half of the lines are a test.
Closes https://github.com/facebook/rocksdb/pull/1599
Differential Revision: D4252092
Pulled By: al13n321
fbshipit-source-id: 41e1e48
Summary:
This diff introduces RangeDelAggregator, which takes ownership of iterators
provided to it via AddTombstones(). The tombstones are organized in a two-level
map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data
copy by holding Slices returned by the iterator, which remain valid thanks to pinning.
For compaction, we create a hierarchical range tombstone iterator with structure
matching the iterator over compaction input data. An aggregator based on that
iterator is used by CompactionIterator to determine which keys are covered by
range tombstones. In case of merge operand, the same aggregator is used by
MergeHelper. Upon finishing each file in the compaction, relevant range tombstones
are added to the output file's range tombstone metablock and file boundaries are
updated accordingly.
To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete()
considers tombstones in the key's snapshot stripe. When this function is used outside of
compaction, it also checks newer stripes, which can contain covering tombstones. Currently
the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges
within a stripe such that binary search can be used.
RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range
to a new table's range tombstone meta-block. Since range tombstones may fall in the gap
between files, we may need to extend some files' key-ranges. The strategy is (1) first file
extends as far left as possible and other files do not extend left, (2) all files extend right
until either the start of the next file or the end of the last range tombstone in the gap,
whichever comes first.
One other notable change is adding release/move semantics to ScopedArenaIterator
such that it can be used to transfer ownership of an arena-allocated iterator, similar to
how unique_ptr is used for malloc'd data.
Depends on D61473
Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927
Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark
Reviewed By: lightmark
Subscribers: andrewkr, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D62205
Summary:
Previously we have an assertion which triggers when we issue Merges
after a single delete. However, merges after a single delete are
unrelated to that single delete. Thus this behavior should be
allowed.
This will address a flakyness of db_stress.
Test Plan: db_stress
Reviewers: IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: andrewkr, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D64923
Summary:
We have alot of code duplication whenever we call FullMerge we keep duplicating the instrumentation and statistics code
This is a simple diff to refactor the code to use TimedFullMerge instead of FullMerge
Test Plan: COMPILE_WITH_ASAN=1 make check -j64
Reviewers: andrewkr, yhchiang, sdong
Reviewed By: sdong
Subscribers: andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D59577
Summary:
Separate a new class InternalIterator from class Iterator, when the look-up is done internally, which also means they operate on key with sequence ID and type.
This change will enable potential future optimizations but for now InternalIterator's functions are still the same as Iterator's.
At the same time, separate the cleanup function to a separate class and let both of InternalIterator and Iterator inherit from it.
Test Plan: Run all existing tests.
Reviewers: igor, yhchiang, anthony, kradhakrishnan, IslamAbdelRahman, rven
Reviewed By: rven
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D48549
Summary:
Since Andres' internship is over, I took over https://reviews.facebook.net/D42555 and rebased and simplified it a bit.
The behavior in this diff is a bit simpler than in D42555:
* only merge operators are passed through FilterMergeValue(). If fitler function returns true, the merge operator is ignored
* compaction filter is *not* called on: 1) results of merge operations and 2) base values that are getting merged with merge operands (the second case was also true in previous diff)
Do we also need a compaction filter to get called on merge results?
Test Plan: make && make check
Reviewers: lovro, tnovak, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: noetzli, kolmike, leveldb, dhruba, sdong
Differential Revision: https://reviews.facebook.net/D47847
Summary:
This patch fixes#7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
Summary:
Builder and CompactionJob share a lot of fairly complex code. This patch
refactors this code into a separate class, the CompactionIterator. Because the
shared code is fairly complex, this patch hopefully improves maintainability.
While there are is a lot of potential for further improvements, the patch is
intentionally pretty close to the original structure because the change is
already complex enough.
Test Plan: make clean all check && ./db_stress
Reviewers: rven, anthony, yhchiang, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D46197
Summary:
In some cases, equality comparisons can be done more efficiently than three-way
comparisons. There are quite a few places in the code where we only care about
equality. This patch adds an Equal() method that defaults to using the
Compare() method.
Test Plan: make clean all check
Reviewers: rven, anthony, yhchiang, igor, sdong
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D46233
Summary:
This diff is a collection of cleanups that were initially part of D43179.
Additionally it adds a unified way of defining key-value maps that use a
Comparator for sorting (this was previously implemented in four different
places).
Test Plan: make clean check all
Reviewers: rven, anthony, yhchiang, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D45993
Summary:
While working on supporting mixing merge operators with
single deletes ( https://reviews.facebook.net/D43179 ),
I realized that returning and dealing with merge results
can be made simpler. Submitting this as a separate diff
because it is not directly related to single deletes.
Before, callers of merge helper had to retrieve the merge
result in one of two ways depending on whether the merge
was successful or not (success = result of merge was single
kTypeValue). For successful merges, the caller could query
the resulting key/value pair and for unsuccessful merges,
the result could be retrieved in the form of two deques of
keys and values. However, with single deletes, a successful merge
does not return a single key/value pair (if merge
operands are merged with a single delete, we have to generate
a value and keep the original single delete around to make
sure that we are not accidentially producing a key overwrite).
In addition, the two existing call sites of the merge
helper were taking the same actions independently from whether
the merge was successful or not, so this patch simplifies that.
Test Plan: make clean all check
Reviewers: rven, sdong, yhchiang, anthony, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43353
Summary:
MergeUntil was not reporting a success when merging an operand with
a Value/Deletion despite the comments in MergeHelper and CompactionJob
indicating otherwise. This lead to operands being written to the compaction
output unnecessarily:
M1 M2 M3 P M4 M5 --> (P+M1+M2+M3) M2 M3 M4 M5 (before the diff)
M1 M2 M3 P M4 M5 --> (P+M1+M2+M3) M4 M5 (after the diff)
In addition, the code handling Values/Deletion was basically identical.
This patch unifies the code. Finally, this patch also adds testing for
merge_helper.
Test Plan: make && make check
Reviewers: sdong, rven, yhchiang, tnovak, igor
Reviewed By: igor
Subscribers: tnovak, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42351
Summary:
We were manipulating `const char*` arrays in CompactionJob to
change the sequence number/types of keys. This patch changes
UpdateInternalKey() to use string methods to do the manipulation
and updates all calls accordingly.
Test Plan:
Added test case for UpdateInternalKey() in dbformat_test.
make && make check
Reviewers: sdong, rven, yhchiang, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D41985
Summary: It's not really nice to call user's API with garbage data in new_value. This diff makes sure that new_value is empty before calling the merge operator.
Test Plan: Added assert to Merge operator in merge_test
Reviewers: sdong, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40773
Summary:
Allow EventListener::OnCompactionCompleted to return CompactionJobStats,
which contains useful information about a compaction.
Example CompactionJobStats returned by OnCompactionCompleted():
smallest_output_key_prefix 05000000
largest_output_key_prefix 06990000
elapsed_time 42419
num_input_records 300
num_input_files 3
num_input_files_at_output_level 2
num_output_records 200
num_output_files 1
actual_bytes_input 167200
actual_bytes_output 110688
total_input_raw_key_bytes 5400
total_input_raw_value_bytes 300000
num_records_replaced 100
is_manual_compaction 1
Test Plan: Developed a mega test in db_test which covers 20 variables in CompactionJobStats.
Reviewers: rven, igor, anthony, sdong
Reviewed By: sdong
Subscribers: tnovak, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D38463
Summary: Remove duplicate code. If this diff looks good, I will cleanup other call sites as well.
Test Plan: unit tests
Reviewers: rven, yhchiang, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37761