rocksdb/db/db_iter.cc

1595 lines
54 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "db/db_iter.h"
#include <string>
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
#include <iostream>
#include <limits>
#include "db/dbformat.h"
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
#include "db/merge_context.h"
#include "db/merge_helper.h"
#include "db/pinned_iterators_manager.h"
#include "monitoring/perf_context_imp.h"
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/options.h"
#include "table/internal_iterator.h"
#include "util/arena.h"
#include "util/filename.h"
#include "util/logging.h"
#include "util/mutexlock.h"
#include "util/string_util.h"
#include "util/trace_replay.h"
#include "util/user_comparator_wrapper.h"
namespace rocksdb {
#if 0
static void DumpInternalIter(Iterator* iter) {
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
ParsedInternalKey k;
if (!ParseInternalKey(iter->key(), &k)) {
fprintf(stderr, "Corrupt '%s'\n", EscapeString(iter->key()).c_str());
} else {
fprintf(stderr, "@ '%s'\n", k.DebugString().c_str());
}
}
}
#endif
// Memtables and sstables that make the DB representation contain
// (userkey,seq,type) => uservalue entries. DBIter
// combines multiple entries for the same userkey found in the DB
// representation into a single entry while accounting for sequence
// numbers, deletion markers, overwrites, etc.
class DBIter final: public Iterator {
public:
// The following is grossly complicated. TODO: clean it up
// Which direction is the iterator currently moving?
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// (1) When moving forward:
// (1a) if current_entry_is_merged_ = false, the internal iterator is
// positioned at the exact entry that yields this->key(), this->value()
// (1b) if current_entry_is_merged_ = true, the internal iterator is
// positioned immediately after the last entry that contributed to the
// current this->value(). That entry may or may not have key equal to
// this->key().
// (2) When moving backwards, the internal iterator is positioned
// just before all entries whose user key == this->key().
enum Direction {
kForward,
kReverse
};
// LocalStatistics contain Statistics counters that will be aggregated per
// each iterator instance and then will be sent to the global statistics when
// the iterator is destroyed.
//
// The purpose of this approach is to avoid perf regression happening
// when multiple threads bump the atomic counters from a DBIter::Next().
struct LocalStatistics {
explicit LocalStatistics() { ResetCounters(); }
void ResetCounters() {
next_count_ = 0;
next_found_count_ = 0;
prev_count_ = 0;
prev_found_count_ = 0;
bytes_read_ = 0;
skip_count_ = 0;
}
void BumpGlobalStatistics(Statistics* global_statistics) {
RecordTick(global_statistics, NUMBER_DB_NEXT, next_count_);
RecordTick(global_statistics, NUMBER_DB_NEXT_FOUND, next_found_count_);
RecordTick(global_statistics, NUMBER_DB_PREV, prev_count_);
RecordTick(global_statistics, NUMBER_DB_PREV_FOUND, prev_found_count_);
RecordTick(global_statistics, ITER_BYTES_READ, bytes_read_);
RecordTick(global_statistics, NUMBER_ITER_SKIP, skip_count_);
PERF_COUNTER_ADD(iter_read_bytes, bytes_read_);
ResetCounters();
}
// Map to Tickers::NUMBER_DB_NEXT
uint64_t next_count_;
// Map to Tickers::NUMBER_DB_NEXT_FOUND
uint64_t next_found_count_;
// Map to Tickers::NUMBER_DB_PREV
uint64_t prev_count_;
// Map to Tickers::NUMBER_DB_PREV_FOUND
uint64_t prev_found_count_;
// Map to Tickers::ITER_BYTES_READ
uint64_t bytes_read_;
// Map to Tickers::NUMBER_ITER_SKIP
uint64_t skip_count_;
};
DBIter(Env* _env, const ReadOptions& read_options,
const ImmutableCFOptions& cf_options,
const MutableCFOptions& mutable_cf_options, const Comparator* cmp,
InternalIterator* iter, SequenceNumber s, bool arena_mode,
uint64_t max_sequential_skip_in_iterations,
ReadCallback* read_callback, DBImpl* db_impl, ColumnFamilyData* cfd,
bool allow_blob)
: env_(_env),
logger_(cf_options.info_log),
user_comparator_(cmp),
merge_operator_(cf_options.merge_operator),
iter_(iter),
read_callback_(read_callback),
sequence_(s),
statistics_(cf_options.statistics),
num_internal_keys_skipped_(0),
iterate_lower_bound_(read_options.iterate_lower_bound),
iterate_upper_bound_(read_options.iterate_upper_bound),
direction_(kForward),
valid_(false),
current_entry_is_merged_(false),
prefix_same_as_start_(read_options.prefix_same_as_start),
pin_thru_lifetime_(read_options.pin_data),
total_order_seek_(read_options.total_order_seek),
allow_blob_(allow_blob),
is_blob_(false),
arena_mode_(arena_mode),
range_del_agg_(&cf_options.internal_comparator, s),
db_impl_(db_impl),
cfd_(cfd),
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
start_seqnum_(read_options.iter_start_seqnum) {
RecordTick(statistics_, NO_ITERATOR_CREATED);
prefix_extractor_ = mutable_cf_options.prefix_extractor.get();
max_skip_ = max_sequential_skip_in_iterations;
max_skippable_internal_keys_ = read_options.max_skippable_internal_keys;
if (pin_thru_lifetime_) {
pinned_iters_mgr_.StartPinning();
}
if (iter_) {
iter_->SetPinnedItersMgr(&pinned_iters_mgr_);
}
}
~DBIter() override {
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
// Release pinned data if any
if (pinned_iters_mgr_.PinningEnabled()) {
pinned_iters_mgr_.ReleasePinnedData();
}
RecordTick(statistics_, NO_ITERATOR_DELETED);
ResetInternalKeysSkippedCounter();
local_stats_.BumpGlobalStatistics(statistics_);
if (!arena_mode_) {
delete iter_;
} else {
iter_->~InternalIterator();
}
}
virtual void SetIter(InternalIterator* iter) {
assert(iter_ == nullptr);
iter_ = iter;
iter_->SetPinnedItersMgr(&pinned_iters_mgr_);
}
virtual ReadRangeDelAggregator* GetRangeDelAggregator() {
return &range_del_agg_;
}
bool Valid() const override { return valid_; }
Slice key() const override {
assert(valid_);
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
if(start_seqnum_ > 0) {
return saved_key_.GetInternalKey();
} else {
return saved_key_.GetUserKey();
}
}
Slice value() const override {
assert(valid_);
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
if (current_entry_is_merged_) {
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
// If pinned_value_ is set then the result of merge operator is one of
// the merge operands and we should return it.
return pinned_value_.data() ? pinned_value_ : saved_value_;
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
} else if (direction_ == kReverse) {
return pinned_value_;
} else {
return iter_->value();
}
}
Status status() const override {
if (status_.ok()) {
return iter_->status();
} else {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(!valid_);
return status_;
}
}
bool IsBlob() const {
assert(valid_ && (allow_blob_ || !is_blob_));
return is_blob_;
}
Status GetProperty(std::string prop_name, std::string* prop) override {
if (prop == nullptr) {
return Status::InvalidArgument("prop is nullptr");
}
if (prop_name == "rocksdb.iterator.super-version-number") {
// First try to pass the value returned from inner iterator.
return iter_->GetProperty(prop_name, prop);
} else if (prop_name == "rocksdb.iterator.is-key-pinned") {
if (valid_) {
*prop = (pin_thru_lifetime_ && saved_key_.IsKeyPinned()) ? "1" : "0";
} else {
*prop = "Iterator is not valid.";
}
return Status::OK();
} else if (prop_name == "rocksdb.iterator.internal-key") {
*prop = saved_key_.GetUserKey().ToString();
return Status::OK();
}
return Status::InvalidArgument("Unidentified property.");
Introduce ReadOptions::pin_data (support zero copy for keys) 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
2015-12-16 20:08:30 +00:00
}
void Next() override;
void Prev() override;
void Seek(const Slice& target) override;
void SeekForPrev(const Slice& target) override;
void SeekToFirst() override;
void SeekToLast() override;
Env* env() { return env_; }
void set_sequence(uint64_t s) { sequence_ = s; }
void set_valid(bool v) { valid_ = v; }
private:
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// For all methods in this block:
// PRE: iter_->Valid() && status_.ok()
// Return false if there was an error, and status() is non-ok, valid_ = false;
// in this case callers would usually stop what they were doing and return.
bool ReverseToForward();
bool ReverseToBackward();
bool FindValueForCurrentKey();
bool FindValueForCurrentKeyUsingSeek();
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool FindUserKeyBeforeSavedKey();
inline bool FindNextUserEntry(bool skipping, bool prefix_check);
bool FindNextUserEntryInternal(bool skipping, bool prefix_check);
bool ParseKey(ParsedInternalKey* key);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool MergeValuesNewToOld();
void PrevInternal();
bool TooManyInternalKeysSkipped(bool increment = true);
bool IsVisible(SequenceNumber sequence);
// CanReseekToSkip() returns whether the iterator can use the optimization
// where it reseek by sequence number to get the next key when there are too
// many versions. This is disabled for write unprepared because seeking to
// sequence number does not guarantee that it is visible.
inline bool CanReseekToSkip();
// MaxVisibleSequenceNumber() returns the maximum visible sequence number
// for this snapshot. This sequence number may be greater than snapshot
// seqno because uncommitted data written to DB for write unprepared will
// have a higher sequence number.
inline SequenceNumber MaxVisibleSequenceNumber();
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
// Temporarily pin the blocks that we encounter until ReleaseTempPinnedData()
// is called
void TempPinData() {
if (!pin_thru_lifetime_) {
pinned_iters_mgr_.StartPinning();
}
}
// Release blocks pinned by TempPinData()
void ReleaseTempPinnedData() {
if (!pin_thru_lifetime_ && pinned_iters_mgr_.PinningEnabled()) {
pinned_iters_mgr_.ReleasePinnedData();
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
}
}
inline void ClearSavedValue() {
if (saved_value_.capacity() > 1048576) {
std::string empty;
swap(empty, saved_value_);
} else {
saved_value_.clear();
}
}
inline void ResetInternalKeysSkippedCounter() {
local_stats_.skip_count_ += num_internal_keys_skipped_;
if (valid_) {
local_stats_.skip_count_--;
}
num_internal_keys_skipped_ = 0;
}
const SliceTransform* prefix_extractor_;
Env* const env_;
Logger* logger_;
UserComparatorWrapper user_comparator_;
const MergeOperator* const merge_operator_;
InternalIterator* iter_;
ReadCallback* read_callback_;
SequenceNumber sequence_;
IterKey saved_key_;
// Reusable internal key data structure. This is only used inside one function
// and should not be used across functions. Reusing this object can reduce
// overhead of calling construction of the function if creating it each time.
ParsedInternalKey ikey_;
std::string saved_value_;
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
Slice pinned_value_;
// for prefix seek mode to support prev()
Statistics* statistics_;
uint64_t max_skip_;
uint64_t max_skippable_internal_keys_;
uint64_t num_internal_keys_skipped_;
const Slice* iterate_lower_bound_;
const Slice* iterate_upper_bound_;
IterKey prefix_start_buf_;
Status status_;
Slice prefix_start_key_;
Direction direction_;
bool valid_;
bool current_entry_is_merged_;
const bool prefix_same_as_start_;
// Means that we will pin all data blocks we read as long the Iterator
// is not deleted, will be true if ReadOptions::pin_data is true
const bool pin_thru_lifetime_;
const bool total_order_seek_;
bool allow_blob_;
bool is_blob_;
bool arena_mode_;
2016-01-07 15:59:14 +00:00
// List of operands for merge operator.
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
MergeContext merge_context_;
ReadRangeDelAggregator range_del_agg_;
LocalStatistics local_stats_;
PinnedIteratorsManager pinned_iters_mgr_;
DBImpl* db_impl_;
ColumnFamilyData* cfd_;
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
// for diff snapshots we want the lower bound on the seqnum;
// if this value > 0 iterator will return internal keys
SequenceNumber start_seqnum_;
// No copying allowed
DBIter(const DBIter&);
void operator=(const DBIter&);
};
inline bool DBIter::ParseKey(ParsedInternalKey* ikey) {
if (!ParseInternalKey(iter_->key(), ikey)) {
status_ = Status::Corruption("corrupted internal key in DBIter");
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
valid_ = false;
ROCKS_LOG_ERROR(logger_, "corrupted internal key in DBIter: %s",
iter_->key().ToString(true).c_str());
return false;
} else {
return true;
}
}
void DBIter::Next() {
assert(valid_);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(status_.ok());
PERF_CPU_TIMER_GUARD(iter_next_cpu_nanos, env_);
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
// Release temporarily pinned blocks from last operation
ReleaseTempPinnedData();
ResetInternalKeysSkippedCounter();
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool ok = true;
if (direction_ == kReverse) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!ReverseToForward()) {
ok = false;
}
} else if (iter_->Valid() && !current_entry_is_merged_) {
// If the current value is not a merge, the iter position is the
// current key, which is already returned. We can safely issue a
// Next() without checking the current key.
// If the current key is a merge, very likely iter already points
// to the next internal position.
iter_->Next();
PERF_COUNTER_ADD(internal_key_skipped_count, 1);
}
if (statistics_ != nullptr) {
local_stats_.next_count_++;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (ok && iter_->Valid()) {
FindNextUserEntry(true /* skipping the current user key */,
prefix_same_as_start_);
} else {
valid_ = false;
}
if (statistics_ != nullptr && valid_) {
local_stats_.next_found_count_++;
local_stats_.bytes_read_ += (key().size() + value().size());
}
}
// PRE: saved_key_ has the current user key if skipping
// POST: saved_key_ should have the next user key if valid_,
// if the current entry is a result of merge
// current_entry_is_merged_ => true
// saved_value_ => the merged value
//
// NOTE: In between, saved_key_ can point to a user key that has
// a delete marker or a sequence number higher than sequence_
// saved_key_ MUST have a proper user_key before calling this function
//
// The prefix_check parameter controls whether we check the iterated
// keys against the prefix of the seeked key. Set to false when
// performing a seek without a key (e.g. SeekToFirst). Set to
// prefix_same_as_start_ for other iterations.
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
inline bool DBIter::FindNextUserEntry(bool skipping, bool prefix_check) {
PERF_TIMER_GUARD(find_next_user_entry_time);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return FindNextUserEntryInternal(skipping, prefix_check);
}
// Actual implementation of DBIter::FindNextUserEntry()
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool DBIter::FindNextUserEntryInternal(bool skipping, bool prefix_check) {
// Loop until we hit an acceptable entry to yield
assert(iter_->Valid());
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(status_.ok());
assert(direction_ == kForward);
current_entry_is_merged_ = false;
// How many times in a row we have skipped an entry with user key less than
// or equal to saved_key_. We could skip these entries either because
// sequence numbers were too high or because skipping = true.
// What saved_key_ contains throughout this method:
// - if skipping : saved_key_ contains the key that we need to skip,
// and we haven't seen any keys greater than that,
// - if num_skipped > 0 : saved_key_ contains the key that we have skipped
// num_skipped times, and we haven't seen any keys
// greater than that,
// - none of the above : saved_key_ can contain anything, it doesn't matter.
uint64_t num_skipped = 0;
is_blob_ = false;
do {
if (!ParseKey(&ikey_)) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
if (iterate_upper_bound_ != nullptr &&
user_comparator_.Compare(ikey_.user_key, *iterate_upper_bound_) >= 0) {
break;
}
if (prefix_extractor_ && prefix_check &&
prefix_extractor_->Transform(ikey_.user_key)
.compare(prefix_start_key_) != 0) {
break;
}
if (TooManyInternalKeysSkipped()) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
if (IsVisible(ikey_.sequence)) {
if (skipping && user_comparator_.Compare(ikey_.user_key,
saved_key_.GetUserKey()) <= 0) {
num_skipped++; // skip this entry
PERF_COUNTER_ADD(internal_key_skipped_count, 1);
} else {
num_skipped = 0;
switch (ikey_.type) {
case kTypeDeletion:
case kTypeSingleDeletion:
// Arrange to skip all upcoming entries for this key since
// they are hidden by this deletion.
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
// if iterartor specified start_seqnum we
// 1) return internal key, including the type
// 2) return ikey only if ikey.seqnum >= start_seqnum_
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// note that if deletion seqnum is < start_seqnum_ we
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
// just skip it like in normal iterator.
if (start_seqnum_ > 0 && ikey_.sequence >= start_seqnum_) {
saved_key_.SetInternalKey(ikey_);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
valid_ = true;
return true;
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
} else {
saved_key_.SetUserKey(
ikey_.user_key,
!pin_thru_lifetime_ || !iter_->IsKeyPinned() /* copy */);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
skipping = true;
PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
}
break;
case kTypeValue:
case kTypeBlobIndex:
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
if (start_seqnum_ > 0) {
// we are taking incremental snapshot here
// incremental snapshots aren't supported on DB with range deletes
assert(!(
(ikey_.type == kTypeBlobIndex) && (start_seqnum_ > 0)
));
if (ikey_.sequence >= start_seqnum_) {
saved_key_.SetInternalKey(ikey_);
valid_ = true;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
} else {
// this key and all previous versions shouldn't be included,
// skipping
saved_key_.SetUserKey(ikey_.user_key,
!pin_thru_lifetime_ || !iter_->IsKeyPinned() /* copy */);
skipping = true;
}
} else {
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
saved_key_.SetUserKey(
ikey_.user_key,
!pin_thru_lifetime_ || !iter_->IsKeyPinned() /* copy */);
if (range_del_agg_.ShouldDelete(
ikey_, RangeDelPositioningMode::kForwardTraversal)) {
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
// Arrange to skip all upcoming entries for this key since
// they are hidden by this deletion.
skipping = true;
num_skipped = 0;
PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
} else if (ikey_.type == kTypeBlobIndex) {
if (!allow_blob_) {
ROCKS_LOG_ERROR(logger_, "Encounter unexpected blob index.");
status_ = Status::NotSupported(
"Encounter unexpected blob index. Please open DB with "
"rocksdb::blob_db::BlobDB instead.");
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
is_blob_ = true;
valid_ = true;
return true;
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
} else {
valid_ = true;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 01:43:29 +00:00
}
}
break;
case kTypeMerge:
saved_key_.SetUserKey(
ikey_.user_key,
!pin_thru_lifetime_ || !iter_->IsKeyPinned() /* copy */);
if (range_del_agg_.ShouldDelete(
ikey_, RangeDelPositioningMode::kForwardTraversal)) {
// Arrange to skip all upcoming entries for this key since
// they are hidden by this deletion.
skipping = true;
num_skipped = 0;
PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
} else {
// By now, we are sure the current ikey is going to yield a
// value
current_entry_is_merged_ = true;
valid_ = true;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return MergeValuesNewToOld(); // Go to a different state machine
}
break;
default:
assert(false);
break;
}
}
} else {
PERF_COUNTER_ADD(internal_recent_skipped_count, 1);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// This key was inserted after our snapshot was taken.
// If this happens too many times in a row for the same user key, we want
// to seek to the target sequence number.
int cmp =
user_comparator_.Compare(ikey_.user_key, saved_key_.GetUserKey());
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (cmp == 0 || (skipping && cmp <= 0)) {
num_skipped++;
} else {
saved_key_.SetUserKey(
ikey_.user_key,
!iter_->IsKeyPinned() || !pin_thru_lifetime_ /* copy */);
skipping = false;
num_skipped = 0;
}
}
// If we have sequentially iterated via numerous equal keys, then it's
// better to seek so that we can avoid too many key comparisons.
if (num_skipped > max_skip_ && CanReseekToSkip()) {
num_skipped = 0;
std::string last_key;
if (skipping) {
// We're looking for the next user-key but all we see are the same
// user-key with decreasing sequence numbers. Fast forward to
// sequence number 0 and type deletion (the smallest type).
AppendInternalKey(&last_key, ParsedInternalKey(saved_key_.GetUserKey(),
0, kTypeDeletion));
// Don't set skipping = false because we may still see more user-keys
// equal to saved_key_.
} else {
// We saw multiple entries with this user key and sequence numbers
// higher than sequence_. Fast forward to sequence_.
// Note that this only covers a case when a higher key was overwritten
// many times since our snapshot was taken, not the case when a lot of
// different keys were inserted after our snapshot was taken.
AppendInternalKey(&last_key,
ParsedInternalKey(saved_key_.GetUserKey(), sequence_,
kValueTypeForSeek));
}
iter_->Seek(last_key);
RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
} else {
iter_->Next();
}
} while (iter_->Valid());
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return iter_->status().ok();
}
// Merge values of the same user key starting from the current iter_ position
// Scan from the newer entries to older entries.
// PRE: iter_->key() points to the first merge type entry
// saved_key_ stores the user key
// POST: saved_value_ has the merged value for the user key
// iter_ points to the next entry (or invalid)
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool DBIter::MergeValuesNewToOld() {
if (!merge_operator_) {
ROCKS_LOG_ERROR(logger_, "Options::merge_operator is null.");
status_ = Status::InvalidArgument("merge_operator_ must be set.");
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
// Temporarily pin the blocks that hold merge operands
TempPinData();
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
merge_context_.Clear();
[RocksDB] [MergeOperator] The new Merge Interface! Uses merge sequences. Summary: Here are the major changes to the Merge Interface. It has been expanded to handle cases where the MergeOperator is not associative. It does so by stacking up merge operations while scanning through the key history (i.e.: during Get() or Compaction), until a valid Put/Delete/end-of-history is encountered; it then applies all of the merge operations in the correct sequence starting with the base/sentinel value. I have also introduced an "AssociativeMerge" function which allows the user to take advantage of associative merge operations (such as in the case of counters). The implementation will always attempt to merge the operations/operands themselves together when they are encountered, and will resort to the "stacking" method if and only if the "associative-merge" fails. This implementation is conjectured to allow MergeOperator to handle the general case, while still providing the user with the ability to take advantage of certain efficiencies in their own merge-operator / data-structure. NOTE: This is a preliminary diff. This must still go through a lot of review, revision, and testing. Feedback welcome! Test Plan: -This is a preliminary diff. I have only just begun testing/debugging it. -I will be testing this with the existing MergeOperator use-cases and unit-tests (counters, string-append, and redis-lists) -I will be "desk-checking" and walking through the code with the help gdb. -I will find a way of stress-testing the new interface / implementation using db_bench, db_test, merge_test, and/or db_stress. -I will ensure that my tests cover all cases: Get-Memtable, Get-Immutable-Memtable, Get-from-Disk, Iterator-Range-Scan, Flush-Memtable-to-L0, Compaction-L0-L1, Compaction-Ln-L(n+1), Put/Delete found, Put/Delete not-found, end-of-history, end-of-file, etc. -A lot of feedback from the reviewers. Reviewers: haobo, dhruba, zshao, emayanke Reviewed By: haobo CC: leveldb Differential Revision: https://reviews.facebook.net/D11499
2013-08-06 03:14:32 +00:00
// Start the merge process by pushing the first operand
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
merge_context_.PushOperand(iter_->value(),
iter_->IsValuePinned() /* operand_pinned */);
TEST_SYNC_POINT("DBIter::MergeValuesNewToOld:PushedFirstOperand");
ParsedInternalKey ikey;
Status s;
for (iter_->Next(); iter_->Valid(); iter_->Next()) {
TEST_SYNC_POINT("DBIter::MergeValuesNewToOld:SteppedToNextOperand");
if (!ParseKey(&ikey)) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
if (!user_comparator_.Equal(ikey.user_key, saved_key_.GetUserKey())) {
// hit the next user key, stop right here
break;
} else if (kTypeDeletion == ikey.type || kTypeSingleDeletion == ikey.type ||
range_del_agg_.ShouldDelete(
ikey, RangeDelPositioningMode::kForwardTraversal)) {
// hit a delete with the same user key, stop right here
// iter_ is positioned after delete
iter_->Next();
break;
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
} else if (kTypeValue == ikey.type) {
[RocksDB] [MergeOperator] The new Merge Interface! Uses merge sequences. Summary: Here are the major changes to the Merge Interface. It has been expanded to handle cases where the MergeOperator is not associative. It does so by stacking up merge operations while scanning through the key history (i.e.: during Get() or Compaction), until a valid Put/Delete/end-of-history is encountered; it then applies all of the merge operations in the correct sequence starting with the base/sentinel value. I have also introduced an "AssociativeMerge" function which allows the user to take advantage of associative merge operations (such as in the case of counters). The implementation will always attempt to merge the operations/operands themselves together when they are encountered, and will resort to the "stacking" method if and only if the "associative-merge" fails. This implementation is conjectured to allow MergeOperator to handle the general case, while still providing the user with the ability to take advantage of certain efficiencies in their own merge-operator / data-structure. NOTE: This is a preliminary diff. This must still go through a lot of review, revision, and testing. Feedback welcome! Test Plan: -This is a preliminary diff. I have only just begun testing/debugging it. -I will be testing this with the existing MergeOperator use-cases and unit-tests (counters, string-append, and redis-lists) -I will be "desk-checking" and walking through the code with the help gdb. -I will find a way of stress-testing the new interface / implementation using db_bench, db_test, merge_test, and/or db_stress. -I will ensure that my tests cover all cases: Get-Memtable, Get-Immutable-Memtable, Get-from-Disk, Iterator-Range-Scan, Flush-Memtable-to-L0, Compaction-L0-L1, Compaction-Ln-L(n+1), Put/Delete found, Put/Delete not-found, end-of-history, end-of-file, etc. -A lot of feedback from the reviewers. Reviewers: haobo, dhruba, zshao, emayanke Reviewed By: haobo CC: leveldb Differential Revision: https://reviews.facebook.net/D11499
2013-08-06 03:14:32 +00:00
// hit a put, merge the put value with operands and store the
// final result in saved_value_. We are done!
const Slice val = iter_->value();
s = MergeHelper::TimedFullMerge(
merge_operator_, ikey.user_key, &val, merge_context_.GetOperands(),
&saved_value_, logger_, statistics_, env_, &pinned_value_, true);
if (!s.ok()) {
valid_ = false;
status_ = s;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
// iter_ is positioned after put
iter_->Next();
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!iter_->status().ok()) {
valid_ = false;
return false;
}
return true;
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
} else if (kTypeMerge == ikey.type) {
[RocksDB] [MergeOperator] The new Merge Interface! Uses merge sequences. Summary: Here are the major changes to the Merge Interface. It has been expanded to handle cases where the MergeOperator is not associative. It does so by stacking up merge operations while scanning through the key history (i.e.: during Get() or Compaction), until a valid Put/Delete/end-of-history is encountered; it then applies all of the merge operations in the correct sequence starting with the base/sentinel value. I have also introduced an "AssociativeMerge" function which allows the user to take advantage of associative merge operations (such as in the case of counters). The implementation will always attempt to merge the operations/operands themselves together when they are encountered, and will resort to the "stacking" method if and only if the "associative-merge" fails. This implementation is conjectured to allow MergeOperator to handle the general case, while still providing the user with the ability to take advantage of certain efficiencies in their own merge-operator / data-structure. NOTE: This is a preliminary diff. This must still go through a lot of review, revision, and testing. Feedback welcome! Test Plan: -This is a preliminary diff. I have only just begun testing/debugging it. -I will be testing this with the existing MergeOperator use-cases and unit-tests (counters, string-append, and redis-lists) -I will be "desk-checking" and walking through the code with the help gdb. -I will find a way of stress-testing the new interface / implementation using db_bench, db_test, merge_test, and/or db_stress. -I will ensure that my tests cover all cases: Get-Memtable, Get-Immutable-Memtable, Get-from-Disk, Iterator-Range-Scan, Flush-Memtable-to-L0, Compaction-L0-L1, Compaction-Ln-L(n+1), Put/Delete found, Put/Delete not-found, end-of-history, end-of-file, etc. -A lot of feedback from the reviewers. Reviewers: haobo, dhruba, zshao, emayanke Reviewed By: haobo CC: leveldb Differential Revision: https://reviews.facebook.net/D11499
2013-08-06 03:14:32 +00:00
// hit a merge, add the value as an operand and run associative merge.
// when complete, add result to operands and continue.
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
merge_context_.PushOperand(iter_->value(),
iter_->IsValuePinned() /* operand_pinned */);
PERF_COUNTER_ADD(internal_merge_count, 1);
} else if (kTypeBlobIndex == ikey.type) {
if (!allow_blob_) {
ROCKS_LOG_ERROR(logger_, "Encounter unexpected blob index.");
status_ = Status::NotSupported(
"Encounter unexpected blob index. Please open DB with "
"rocksdb::blob_db::BlobDB instead.");
} else {
status_ =
Status::NotSupported("Blob DB does not support merge operator.");
}
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
} else {
assert(false);
}
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!iter_->status().ok()) {
valid_ = false;
return false;
}
// we either exhausted all internal keys under this user key, or hit
// a deletion marker.
// feed null as the existing value to the merge operator, such that
// client can differentiate this scenario and do things accordingly.
s = MergeHelper::TimedFullMerge(merge_operator_, saved_key_.GetUserKey(),
nullptr, merge_context_.GetOperands(),
&saved_value_, logger_, statistics_, env_,
&pinned_value_, true);
if (!s.ok()) {
valid_ = false;
status_ = s;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(status_.ok());
return true;
}
void DBIter::Prev() {
assert(valid_);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(status_.ok());
PERF_CPU_TIMER_GUARD(iter_prev_cpu_nanos, env_);
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
ReleaseTempPinnedData();
ResetInternalKeysSkippedCounter();
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool ok = true;
if (direction_ == kForward) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!ReverseToBackward()) {
ok = false;
}
}
if (ok) {
PrevInternal();
}
if (statistics_ != nullptr) {
local_stats_.prev_count_++;
if (valid_) {
local_stats_.prev_found_count_++;
local_stats_.bytes_read_ += (key().size() + value().size());
}
}
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
bool DBIter::ReverseToForward() {
assert(iter_->status().ok());
// When moving backwards, iter_ is positioned on _previous_ key, which may
// not exist or may have different prefix than the current key().
// If that's the case, seek iter_ to current key.
if ((prefix_extractor_ != nullptr && !total_order_seek_) || !iter_->Valid()) {
IterKey last_key;
last_key.SetInternalKey(ParsedInternalKey(
saved_key_.GetUserKey(), kMaxSequenceNumber, kValueTypeForSeek));
iter_->Seek(last_key.GetInternalKey());
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
direction_ = kForward;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Skip keys less than the current key() (a.k.a. saved_key_).
while (iter_->Valid()) {
ParsedInternalKey ikey;
if (!ParseKey(&ikey)) {
return false;
}
if (user_comparator_.Compare(ikey.user_key, saved_key_.GetUserKey()) >= 0) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
}
iter_->Next();
}
if (!iter_->status().ok()) {
valid_ = false;
return false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Move iter_ to the key before saved_key_.
bool DBIter::ReverseToBackward() {
assert(iter_->status().ok());
// When current_entry_is_merged_ is true, iter_ may be positioned on the next
// key, which may not exist or may have prefix different from current.
// If that's the case, seek to saved_key_.
if (current_entry_is_merged_ &&
((prefix_extractor_ != nullptr && !total_order_seek_) ||
!iter_->Valid())) {
IterKey last_key;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Using kMaxSequenceNumber and kValueTypeForSeek
// (not kValueTypeForSeekForPrev) to seek to a key strictly smaller
// than saved_key_.
last_key.SetInternalKey(ParsedInternalKey(
saved_key_.GetUserKey(), kMaxSequenceNumber, kValueTypeForSeek));
if (prefix_extractor_ != nullptr && !total_order_seek_) {
iter_->SeekForPrev(last_key.GetInternalKey());
} else {
// Some iterators may not support SeekForPrev(), so we avoid using it
// when prefix seek mode is disabled. This is somewhat expensive
// (an extra Prev(), as well as an extra change of direction of iter_),
// so we may need to reconsider it later.
iter_->Seek(last_key.GetInternalKey());
if (!iter_->Valid() && iter_->status().ok()) {
iter_->SeekToLast();
}
}
}
direction_ = kReverse;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return FindUserKeyBeforeSavedKey();
}
void DBIter::PrevInternal() {
while (iter_->Valid()) {
saved_key_.SetUserKey(
ExtractUserKey(iter_->key()),
!iter_->IsKeyPinned() || !pin_thru_lifetime_ /* copy */);
if (prefix_extractor_ && prefix_same_as_start_ &&
prefix_extractor_->Transform(saved_key_.GetUserKey())
.compare(prefix_start_key_) != 0) {
// Current key does not have the same prefix as start
valid_ = false;
return;
}
if (iterate_lower_bound_ != nullptr &&
user_comparator_.Compare(saved_key_.GetUserKey(),
*iterate_lower_bound_) < 0) {
// We've iterated earlier than the user-specified lower bound.
valid_ = false;
return;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!FindValueForCurrentKey()) { // assigns valid_
return;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Whether or not we found a value for current key, we need iter_ to end up
// on a smaller key.
if (!FindUserKeyBeforeSavedKey()) {
return;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (valid_) {
// Found the value.
return;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (TooManyInternalKeysSkipped(false)) {
return;
}
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// We haven't found any key - iterator is not valid
valid_ = false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Used for backwards iteration.
// Looks at the entries with user key saved_key_ and finds the most up-to-date
// value for it, or executes a merge, or determines that the value was deleted.
// Sets valid_ to true if the value is found and is ready to be presented to
// the user through value().
// Sets valid_ to false if the value was deleted, and we should try another key.
// Returns false if an error occurred, and !status().ok() and !valid_.
//
// PRE: iter_ is positioned on the last entry with user key equal to saved_key_.
// POST: iter_ is positioned on one of the entries equal to saved_key_, or on
// the entry just before them, or on the entry just after them.
bool DBIter::FindValueForCurrentKey() {
assert(iter_->Valid());
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
merge_context_.Clear();
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
current_entry_is_merged_ = false;
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
// last entry before merge (could be kTypeDeletion, kTypeSingleDeletion or
// kTypeValue)
ValueType last_not_merge_type = kTypeDeletion;
ValueType last_key_entry_type = kTypeDeletion;
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
// Temporarily pin blocks that hold (merge operands / the value)
ReleaseTempPinnedData();
TempPinData();
size_t num_skipped = 0;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
while (iter_->Valid()) {
ParsedInternalKey ikey;
if (!ParseKey(&ikey)) {
return false;
}
if (!IsVisible(ikey.sequence) ||
!user_comparator_.Equal(ikey.user_key, saved_key_.GetUserKey())) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
break;
}
if (TooManyInternalKeysSkipped()) {
return false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// This user key has lots of entries.
// We're going from old to new, and it's taking too long. Let's do a Seek()
// and go from new to old. This helps when a key was overwritten many times.
if (num_skipped >= max_skip_ && CanReseekToSkip()) {
return FindValueForCurrentKeyUsingSeek();
}
last_key_entry_type = ikey.type;
switch (last_key_entry_type) {
case kTypeValue:
case kTypeBlobIndex:
if (range_del_agg_.ShouldDelete(
ikey, RangeDelPositioningMode::kBackwardTraversal)) {
last_key_entry_type = kTypeRangeDeletion;
PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
} else {
assert(iter_->IsValuePinned());
pinned_value_ = iter_->value();
}
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
merge_context_.Clear();
last_not_merge_type = last_key_entry_type;
break;
case kTypeDeletion:
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
case kTypeSingleDeletion:
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
merge_context_.Clear();
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
last_not_merge_type = last_key_entry_type;
PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
break;
case kTypeMerge:
if (range_del_agg_.ShouldDelete(
ikey, RangeDelPositioningMode::kBackwardTraversal)) {
merge_context_.Clear();
last_key_entry_type = kTypeRangeDeletion;
last_not_merge_type = last_key_entry_type;
PERF_COUNTER_ADD(internal_delete_skipped_count, 1);
} else {
assert(merge_operator_ != nullptr);
merge_context_.PushOperandBack(
iter_->value(), iter_->IsValuePinned() /* operand_pinned */);
PERF_COUNTER_ADD(internal_merge_count, 1);
}
break;
default:
assert(false);
}
PERF_COUNTER_ADD(internal_key_skipped_count, 1);
iter_->Prev();
++num_skipped;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
}
if (!iter_->status().ok()) {
valid_ = false;
return false;
}
Status s;
is_blob_ = false;
switch (last_key_entry_type) {
case kTypeDeletion:
Support for SingleDelete() 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
2015-09-17 18:42:56 +00:00
case kTypeSingleDeletion:
case kTypeRangeDeletion:
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
case kTypeMerge:
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
current_entry_is_merged_ = true;
if (last_not_merge_type == kTypeDeletion ||
last_not_merge_type == kTypeSingleDeletion ||
last_not_merge_type == kTypeRangeDeletion) {
s = MergeHelper::TimedFullMerge(
merge_operator_, saved_key_.GetUserKey(), nullptr,
merge_context_.GetOperands(), &saved_value_, logger_, statistics_,
env_, &pinned_value_, true);
} else if (last_not_merge_type == kTypeBlobIndex) {
if (!allow_blob_) {
ROCKS_LOG_ERROR(logger_, "Encounter unexpected blob index.");
status_ = Status::NotSupported(
"Encounter unexpected blob index. Please open DB with "
"rocksdb::blob_db::BlobDB instead.");
} else {
status_ =
Status::NotSupported("Blob DB does not support merge operator.");
}
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
} else {
assert(last_not_merge_type == kTypeValue);
s = MergeHelper::TimedFullMerge(
merge_operator_, saved_key_.GetUserKey(), &pinned_value_,
merge_context_.GetOperands(), &saved_value_, logger_, statistics_,
env_, &pinned_value_, true);
}
break;
case kTypeValue:
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// do nothing - we've already has value in pinned_value_
break;
case kTypeBlobIndex:
if (!allow_blob_) {
ROCKS_LOG_ERROR(logger_, "Encounter unexpected blob index.");
status_ = Status::NotSupported(
"Encounter unexpected blob index. Please open DB with "
"rocksdb::blob_db::BlobDB instead.");
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
is_blob_ = true;
break;
default:
assert(false);
break;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!s.ok()) {
valid_ = false;
status_ = s;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
valid_ = true;
return true;
}
// This function is used in FindValueForCurrentKey.
// We use Seek() function instead of Prev() to find necessary value
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// TODO: This is very similar to FindNextUserEntry() and MergeValuesNewToOld().
// Would be nice to reuse some code.
bool DBIter::FindValueForCurrentKeyUsingSeek() {
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
2016-07-20 16:49:03 +00:00
// FindValueForCurrentKey will enable pinning before calling
// FindValueForCurrentKeyUsingSeek()
assert(pinned_iters_mgr_.PinningEnabled());
std::string last_key;
AppendInternalKey(&last_key, ParsedInternalKey(saved_key_.GetUserKey(),
sequence_, kValueTypeForSeek));
iter_->Seek(last_key);
RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// In case read_callback presents, the value we seek to may not be visible.
// Find the next value that's visible.
ParsedInternalKey ikey;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
while (true) {
if (!iter_->Valid()) {
valid_ = false;
return iter_->status().ok();
}
if (!ParseKey(&ikey)) {
return false;
}
if (!user_comparator_.Equal(ikey.user_key, saved_key_.GetUserKey())) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// No visible values for this key, even though FindValueForCurrentKey()
// has seen some. This is possible if we're using a tailing iterator, and
// the entries were discarded in a compaction.
valid_ = false;
return true;
}
if (IsVisible(ikey.sequence)) {
break;
}
iter_->Next();
}
if (ikey.type == kTypeDeletion || ikey.type == kTypeSingleDeletion ||
range_del_agg_.ShouldDelete(
ikey, RangeDelPositioningMode::kBackwardTraversal)) {
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
}
if (ikey.type == kTypeBlobIndex && !allow_blob_) {
ROCKS_LOG_ERROR(logger_, "Encounter unexpected blob index.");
status_ = Status::NotSupported(
"Encounter unexpected blob index. Please open DB with "
"rocksdb::blob_db::BlobDB instead.");
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
if (ikey.type == kTypeValue || ikey.type == kTypeBlobIndex) {
assert(iter_->IsValuePinned());
pinned_value_ = iter_->value();
valid_ = true;
return true;
}
// kTypeMerge. We need to collect all kTypeMerge values and save them
// in operands
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(ikey.type == kTypeMerge);
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
current_entry_is_merged_ = true;
Eliminate std::deque initialization while iterating over merge operands Summary: This patch is similar to D52563, When we iterate over a DB with merge operands we keep creating std::queue to store the operands, optimize this by reusing merge_operands_ data member Before the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.757 micros/op 266141 ops/sec; 29.4 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.413 micros/op 2423538 ops/sec; 268.1 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.451 micros/op 2219071 ops/sec; 245.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.420 micros/op 2382039 ops/sec; 263.5 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.408 micros/op 2452017 ops/sec; 271.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.947 micros/op 253376 ops/sec; 28.0 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.441 micros/op 2266473 ops/sec; 250.7 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.471 micros/op 2122033 ops/sec; 234.8 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.440 micros/op 2271407 ops/sec; 251.3 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.429 micros/op 2331471 ops/sec; 257.9 MB/s ``` with the patch ``` ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq" --db="/dev/shm/bench_merge_memcpy_on_the_fly/" --merge_operator="put" --merge_keys=10000 --num=10000 DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 4.080 micros/op 245092 ops/sec; 27.1 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.308 micros/op 3241843 ops/sec; 358.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.312 micros/op 3200408 ops/sec; 354.0 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.332 micros/op 3013962 ops/sec; 333.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.300 micros/op 3328017 ops/sec; 368.2 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] mergerandom : 3.973 micros/op 251705 ops/sec; 27.8 MB/s ( updates:10000) DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.320 micros/op 3123752 ops/sec; 345.6 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.335 micros/op 2986641 ops/sec; 330.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.339 micros/op 2950047 ops/sec; 326.4 MB/s DB path: [/dev/shm/bench_merge_memcpy_on_the_fly/] readseq : 0.319 micros/op 3131565 ops/sec; 346.4 MB/s ``` Test Plan: make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D56031
2016-04-01 22:48:55 +00:00
merge_context_.Clear();
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
merge_context_.PushOperand(iter_->value(),
iter_->IsValuePinned() /* operand_pinned */);
while (true) {
iter_->Next();
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!iter_->Valid()) {
if (!iter_->status().ok()) {
valid_ = false;
return false;
}
break;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!ParseKey(&ikey)) {
return false;
}
if (!user_comparator_.Equal(ikey.user_key, saved_key_.GetUserKey())) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
break;
}
if (ikey.type == kTypeDeletion || ikey.type == kTypeSingleDeletion ||
range_del_agg_.ShouldDelete(
ikey, RangeDelPositioningMode::kForwardTraversal)) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
break;
} else if (ikey.type == kTypeValue) {
const Slice val = iter_->value();
Status s = MergeHelper::TimedFullMerge(
merge_operator_, saved_key_.GetUserKey(), &val,
merge_context_.GetOperands(), &saved_value_, logger_, statistics_,
env_, &pinned_value_, true);
if (!s.ok()) {
valid_ = false;
status_ = s;
return false;
}
valid_ = true;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
} else if (ikey.type == kTypeMerge) {
merge_context_.PushOperand(iter_->value(),
iter_->IsValuePinned() /* operand_pinned */);
PERF_COUNTER_ADD(internal_merge_count, 1);
} else if (ikey.type == kTypeBlobIndex) {
if (!allow_blob_) {
ROCKS_LOG_ERROR(logger_, "Encounter unexpected blob index.");
status_ = Status::NotSupported(
"Encounter unexpected blob index. Please open DB with "
"rocksdb::blob_db::BlobDB instead.");
} else {
status_ =
Status::NotSupported("Blob DB does not support merge operator.");
}
valid_ = false;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
} else {
assert(false);
}
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
Status s = MergeHelper::TimedFullMerge(
merge_operator_, saved_key_.GetUserKey(), nullptr,
merge_context_.GetOperands(), &saved_value_, logger_, statistics_, env_,
&pinned_value_, true);
if (!s.ok()) {
valid_ = false;
status_ = s;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Make sure we leave iter_ in a good state. If it's valid and we don't care
// about prefixes, that's already good enough. Otherwise it needs to be
// seeked to the current key.
if ((prefix_extractor_ != nullptr && !total_order_seek_) || !iter_->Valid()) {
if (prefix_extractor_ != nullptr && !total_order_seek_) {
iter_->SeekForPrev(last_key);
} else {
iter_->Seek(last_key);
if (!iter_->Valid() && iter_->status().ok()) {
iter_->SeekToLast();
}
}
RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
valid_ = true;
return true;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Move backwards until the key smaller than saved_key_.
// Changes valid_ only if return value is false.
bool DBIter::FindUserKeyBeforeSavedKey() {
assert(status_.ok());
size_t num_skipped = 0;
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
while (iter_->Valid()) {
ParsedInternalKey ikey;
if (!ParseKey(&ikey)) {
return false;
}
if (user_comparator_.Compare(ikey.user_key, saved_key_.GetUserKey()) < 0) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
return true;
}
if (TooManyInternalKeysSkipped()) {
return false;
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
assert(ikey.sequence != kMaxSequenceNumber);
if (!IsVisible(ikey.sequence)) {
PERF_COUNTER_ADD(internal_recent_skipped_count, 1);
} else {
PERF_COUNTER_ADD(internal_key_skipped_count, 1);
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (num_skipped >= max_skip_ && CanReseekToSkip()) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
num_skipped = 0;
IterKey last_key;
last_key.SetInternalKey(ParsedInternalKey(
saved_key_.GetUserKey(), kMaxSequenceNumber, kValueTypeForSeek));
// It would be more efficient to use SeekForPrev() here, but some
// iterators may not support it.
iter_->Seek(last_key.GetInternalKey());
RecordTick(statistics_, NUMBER_OF_RESEEKS_IN_ITERATION);
if (!iter_->Valid()) {
break;
}
} else {
++num_skipped;
}
iter_->Prev();
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (!iter_->status().ok()) {
valid_ = false;
return false;
}
return true;
}
bool DBIter::TooManyInternalKeysSkipped(bool increment) {
if ((max_skippable_internal_keys_ > 0) &&
(num_internal_keys_skipped_ > max_skippable_internal_keys_)) {
valid_ = false;
status_ = Status::Incomplete("Too many internal keys skipped.");
return true;
} else if (increment) {
num_internal_keys_skipped_++;
}
return false;
}
bool DBIter::IsVisible(SequenceNumber sequence) {
return sequence <= MaxVisibleSequenceNumber() &&
(read_callback_ == nullptr || read_callback_->IsVisible(sequence));
}
bool DBIter::CanReseekToSkip() {
return read_callback_ == nullptr ||
read_callback_->MaxUnpreparedSequenceNumber() == 0;
}
SequenceNumber DBIter::MaxVisibleSequenceNumber() {
if (read_callback_ == nullptr) {
return sequence_;
}
return std::max(sequence_, read_callback_->MaxUnpreparedSequenceNumber());
}
void DBIter::Seek(const Slice& target) {
PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_);
StopWatch sw(env_, statistics_, DB_SEEK);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
status_ = Status::OK();
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
ReleaseTempPinnedData();
ResetInternalKeysSkippedCounter();
SequenceNumber seq = MaxVisibleSequenceNumber();
saved_key_.Clear();
saved_key_.SetInternalKey(target, seq);
#ifndef ROCKSDB_LITE
if (db_impl_ != nullptr && cfd_ != nullptr) {
db_impl_->TraceIteratorSeek(cfd_->GetID(), target);
}
#endif // ROCKSDB_LITE
if (iterate_lower_bound_ != nullptr &&
user_comparator_.Compare(saved_key_.GetUserKey(), *iterate_lower_bound_) <
0) {
saved_key_.Clear();
saved_key_.SetInternalKey(*iterate_lower_bound_, seq);
}
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->Seek(saved_key_.GetInternalKey());
range_del_agg_.InvalidateRangeDelMapPositions();
}
RecordTick(statistics_, NUMBER_DB_SEEK);
if (iter_->Valid()) {
if (prefix_extractor_ && prefix_same_as_start_) {
prefix_start_key_ = prefix_extractor_->Transform(target);
}
direction_ = kForward;
ClearSavedValue();
FindNextUserEntry(false /* not skipping */, prefix_same_as_start_);
if (!valid_) {
prefix_start_key_.clear();
}
if (statistics_ != nullptr) {
if (valid_) {
// Decrement since we don't want to count this key as skipped
RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
}
}
} else {
valid_ = false;
}
if (valid_ && prefix_extractor_ && prefix_same_as_start_) {
prefix_start_buf_.SetUserKey(prefix_start_key_);
prefix_start_key_ = prefix_start_buf_.GetUserKey();
}
}
void DBIter::SeekForPrev(const Slice& target) {
PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_);
StopWatch sw(env_, statistics_, DB_SEEK);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
status_ = Status::OK();
ReleaseTempPinnedData();
ResetInternalKeysSkippedCounter();
saved_key_.Clear();
// now saved_key is used to store internal key.
saved_key_.SetInternalKey(target, 0 /* sequence_number */,
kValueTypeForSeekForPrev);
if (iterate_upper_bound_ != nullptr &&
user_comparator_.Compare(saved_key_.GetUserKey(),
*iterate_upper_bound_) >= 0) {
saved_key_.Clear();
saved_key_.SetInternalKey(*iterate_upper_bound_, kMaxSequenceNumber);
}
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->SeekForPrev(saved_key_.GetInternalKey());
range_del_agg_.InvalidateRangeDelMapPositions();
}
#ifndef ROCKSDB_LITE
if (db_impl_ != nullptr && cfd_ != nullptr) {
db_impl_->TraceIteratorSeekForPrev(cfd_->GetID(), target);
}
#endif // ROCKSDB_LITE
RecordTick(statistics_, NUMBER_DB_SEEK);
if (iter_->Valid()) {
if (prefix_extractor_ && prefix_same_as_start_) {
prefix_start_key_ = prefix_extractor_->Transform(target);
}
direction_ = kReverse;
ClearSavedValue();
PrevInternal();
if (!valid_) {
prefix_start_key_.clear();
}
if (statistics_ != nullptr) {
if (valid_) {
RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
}
}
} else {
valid_ = false;
}
if (valid_ && prefix_extractor_ && prefix_same_as_start_) {
prefix_start_buf_.SetUserKey(prefix_start_key_);
prefix_start_key_ = prefix_start_buf_.GetUserKey();
}
}
void DBIter::SeekToFirst() {
if (iterate_lower_bound_ != nullptr) {
Seek(*iterate_lower_bound_);
return;
}
PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_);
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
// Don't use iter_::Seek() if we set a prefix extractor
// because prefix seek will be used.
if (prefix_extractor_ != nullptr && !total_order_seek_) {
max_skip_ = std::numeric_limits<uint64_t>::max();
}
status_ = Status::OK();
direction_ = kForward;
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
ReleaseTempPinnedData();
ResetInternalKeysSkippedCounter();
ClearSavedValue();
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->SeekToFirst();
range_del_agg_.InvalidateRangeDelMapPositions();
}
RecordTick(statistics_, NUMBER_DB_SEEK);
if (iter_->Valid()) {
saved_key_.SetUserKey(
ExtractUserKey(iter_->key()),
!iter_->IsKeyPinned() || !pin_thru_lifetime_ /* copy */);
FindNextUserEntry(false /* not skipping */, false /* no prefix check */);
if (statistics_ != nullptr) {
if (valid_) {
RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
}
}
} else {
valid_ = false;
}
if (valid_ && prefix_extractor_ && prefix_same_as_start_) {
prefix_start_buf_.SetUserKey(
prefix_extractor_->Transform(saved_key_.GetUserKey()));
prefix_start_key_ = prefix_start_buf_.GetUserKey();
}
}
void DBIter::SeekToLast() {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (iterate_upper_bound_ != nullptr) {
// Seek to last key strictly less than ReadOptions.iterate_upper_bound.
SeekForPrev(*iterate_upper_bound_);
if (Valid() && user_comparator_.Equal(*iterate_upper_bound_, key())) {
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
ReleaseTempPinnedData();
PrevInternal();
}
return;
}
PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_);
// Don't use iter_::Seek() if we set a prefix extractor
// because prefix seek will be used.
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
if (prefix_extractor_ != nullptr && !total_order_seek_) {
max_skip_ = std::numeric_limits<uint64_t>::max();
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
status_ = Status::OK();
direction_ = kReverse;
Eliminate memcpy in Iterator::Prev() by pinning blocks for keys spanning multiple blocks Summary: This diff is stacked on top of this diff https://reviews.facebook.net/D56493 The current Iterator::Prev() implementation need to copy every value since the underlying Iterator may move after reading the value. This can be optimized by making sure that the block containing the value is pinned until the Iterator move. which will improve the throughput by up to 1.5X master ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.449 micros/op 2225887 ops/sec; 246.2 MB/s readreverse : 0.433 micros/op 2311508 ops/sec; 255.7 MB/s readreverse : 0.436 micros/op 2294335 ops/sec; 253.8 MB/s readreverse : 0.471 micros/op 2121295 ops/sec; 234.7 MB/s readreverse : 0.465 micros/op 2152227 ops/sec; 238.1 MB/s readreverse : 0.454 micros/op 2203011 ops/sec; 243.7 MB/s readreverse : 0.451 micros/op 2216095 ops/sec; 245.2 MB/s readreverse : 0.462 micros/op 2162447 ops/sec; 239.2 MB/s readreverse : 0.476 micros/op 2099151 ops/sec; 232.2 MB/s readreverse : 0.472 micros/op 2120710 ops/sec; 234.6 MB/s avg : 242.34 MB/s ==> 1000000_Keys_1KB.txt <== readreverse : 1.013 micros/op 986793 ops/sec; 978.7 MB/s readreverse : 0.942 micros/op 1061136 ops/sec; 1052.5 MB/s readreverse : 0.951 micros/op 1051901 ops/sec; 1043.3 MB/s readreverse : 0.932 micros/op 1072894 ops/sec; 1064.1 MB/s readreverse : 1.024 micros/op 976720 ops/sec; 968.7 MB/s readreverse : 0.935 micros/op 1069169 ops/sec; 1060.4 MB/s readreverse : 1.012 micros/op 988132 ops/sec; 980.1 MB/s readreverse : 0.962 micros/op 1039579 ops/sec; 1031.1 MB/s readreverse : 0.991 micros/op 1008924 ops/sec; 1000.7 MB/s readreverse : 1.004 micros/op 996144 ops/sec; 988.0 MB/s avg : 1016.76 MB/s ==> 1000000_Keys_10KB.txt <== readreverse : 4.167 micros/op 239952 ops/sec; 2346.9 MB/s readreverse : 4.070 micros/op 245713 ops/sec; 2403.3 MB/s readreverse : 4.572 micros/op 218733 ops/sec; 2139.4 MB/s readreverse : 4.497 micros/op 222388 ops/sec; 2175.2 MB/s readreverse : 4.203 micros/op 237920 ops/sec; 2327.1 MB/s readreverse : 4.206 micros/op 237756 ops/sec; 2325.5 MB/s readreverse : 4.181 micros/op 239149 ops/sec; 2339.1 MB/s readreverse : 4.157 micros/op 240552 ops/sec; 2352.8 MB/s readreverse : 4.187 micros/op 238848 ops/sec; 2336.1 MB/s readreverse : 4.106 micros/op 243575 ops/sec; 2382.4 MB/s avg : 2312.78 MB/s ==> 100000_Keys_100KB.txt <== readreverse : 41.281 micros/op 24224 ops/sec; 2366.0 MB/s readreverse : 39.722 micros/op 25175 ops/sec; 2458.9 MB/s readreverse : 40.319 micros/op 24802 ops/sec; 2422.5 MB/s readreverse : 39.762 micros/op 25149 ops/sec; 2456.4 MB/s readreverse : 40.916 micros/op 24440 ops/sec; 2387.1 MB/s readreverse : 41.188 micros/op 24278 ops/sec; 2371.4 MB/s readreverse : 40.061 micros/op 24962 ops/sec; 2438.1 MB/s readreverse : 40.221 micros/op 24862 ops/sec; 2428.4 MB/s readreverse : 40.084 micros/op 24947 ops/sec; 2436.7 MB/s readreverse : 40.655 micros/op 24597 ops/sec; 2402.4 MB/s avg : 2416.79 MB/s ==> 10000_Keys_1MB.txt <== readreverse : 298.038 micros/op 3355 ops/sec; 3355.3 MB/s readreverse : 335.001 micros/op 2985 ops/sec; 2985.1 MB/s readreverse : 286.956 micros/op 3484 ops/sec; 3484.9 MB/s readreverse : 329.954 micros/op 3030 ops/sec; 3030.8 MB/s readreverse : 306.428 micros/op 3263 ops/sec; 3263.5 MB/s readreverse : 330.749 micros/op 3023 ops/sec; 3023.5 MB/s readreverse : 328.903 micros/op 3040 ops/sec; 3040.5 MB/s readreverse : 324.853 micros/op 3078 ops/sec; 3078.4 MB/s readreverse : 320.488 micros/op 3120 ops/sec; 3120.3 MB/s readreverse : 320.536 micros/op 3119 ops/sec; 3119.8 MB/s avg : 3150.21 MB/s ``` After memcpy elimination ``` ==> 1000000_Keys_100Byte.txt <== readreverse : 0.395 micros/op 2529890 ops/sec; 279.9 MB/s readreverse : 0.368 micros/op 2715922 ops/sec; 300.5 MB/s readreverse : 0.384 micros/op 2603929 ops/sec; 288.1 MB/s readreverse : 0.375 micros/op 2663286 ops/sec; 294.6 MB/s readreverse : 0.357 micros/op 2802180 ops/sec; 310.0 MB/s readreverse : 0.363 micros/op 2757684 ops/sec; 305.1 MB/s readreverse : 0.372 micros/op 2689603 ops/sec; 297.5 MB/s readreverse : 0.379 micros/op 2638599 ops/sec; 291.9 MB/s readreverse : 0.375 micros/op 2663803 ops/sec; 294.7 MB/s readreverse : 0.375 micros/op 2665579 ops/sec; 294.9 MB/s avg: 295.72 MB/s (1.22 X) ==> 1000000_Keys_1KB.txt <== readreverse : 0.879 micros/op 1138112 ops/sec; 1128.8 MB/s readreverse : 0.842 micros/op 1187998 ops/sec; 1178.3 MB/s readreverse : 0.837 micros/op 1194915 ops/sec; 1185.1 MB/s readreverse : 0.845 micros/op 1182983 ops/sec; 1173.3 MB/s readreverse : 0.877 micros/op 1140308 ops/sec; 1131.0 MB/s readreverse : 0.849 micros/op 1177581 ops/sec; 1168.0 MB/s readreverse : 0.915 micros/op 1093284 ops/sec; 1084.3 MB/s readreverse : 0.863 micros/op 1159418 ops/sec; 1149.9 MB/s readreverse : 0.895 micros/op 1117670 ops/sec; 1108.5 MB/s readreverse : 0.852 micros/op 1174116 ops/sec; 1164.5 MB/s avg: 1147.17 MB/s (1.12 X) ==> 1000000_Keys_10KB.txt <== readreverse : 3.870 micros/op 258386 ops/sec; 2527.2 MB/s readreverse : 3.568 micros/op 280296 ops/sec; 2741.5 MB/s readreverse : 4.005 micros/op 249694 ops/sec; 2442.2 MB/s readreverse : 3.550 micros/op 281719 ops/sec; 2755.5 MB/s readreverse : 3.562 micros/op 280758 ops/sec; 2746.1 MB/s readreverse : 3.507 micros/op 285125 ops/sec; 2788.8 MB/s readreverse : 3.463 micros/op 288739 ops/sec; 2824.1 MB/s readreverse : 3.428 micros/op 291734 ops/sec; 2853.4 MB/s readreverse : 3.553 micros/op 281491 ops/sec; 2753.2 MB/s readreverse : 3.535 micros/op 282885 ops/sec; 2766.9 MB/s avg : 2719.89 MB/s (1.17 X) ==> 100000_Keys_100KB.txt <== readreverse : 22.815 micros/op 43830 ops/sec; 4281.0 MB/s readreverse : 29.957 micros/op 33381 ops/sec; 3260.4 MB/s readreverse : 25.334 micros/op 39473 ops/sec; 3855.4 MB/s readreverse : 23.037 micros/op 43409 ops/sec; 4239.8 MB/s readreverse : 27.810 micros/op 35958 ops/sec; 3512.1 MB/s readreverse : 30.327 micros/op 32973 ops/sec; 3220.6 MB/s readreverse : 29.704 micros/op 33665 ops/sec; 3288.2 MB/s readreverse : 29.423 micros/op 33987 ops/sec; 3319.6 MB/s readreverse : 23.334 micros/op 42856 ops/sec; 4185.9 MB/s readreverse : 29.969 micros/op 33368 ops/sec; 3259.1 MB/s avg : 3642.21 MB/s (1.5 X) ==> 10000_Keys_1MB.txt <== readreverse : 244.748 micros/op 4085 ops/sec; 4085.9 MB/s readreverse : 230.208 micros/op 4343 ops/sec; 4344.0 MB/s readreverse : 235.655 micros/op 4243 ops/sec; 4243.6 MB/s readreverse : 235.730 micros/op 4242 ops/sec; 4242.2 MB/s readreverse : 237.346 micros/op 4213 ops/sec; 4213.3 MB/s readreverse : 227.306 micros/op 4399 ops/sec; 4399.4 MB/s readreverse : 194.957 micros/op 5129 ops/sec; 5129.4 MB/s readreverse : 238.359 micros/op 4195 ops/sec; 4195.4 MB/s readreverse : 221.588 micros/op 4512 ops/sec; 4513.0 MB/s readreverse : 235.911 micros/op 4238 ops/sec; 4239.0 MB/s avg : 4360.52 MB/s (1.38 X) ``` 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/D56511
2016-05-03 04:46:30 +00:00
ReleaseTempPinnedData();
ResetInternalKeysSkippedCounter();
ClearSavedValue();
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->SeekToLast();
range_del_agg_.InvalidateRangeDelMapPositions();
}
Change and clarify the relationship between Valid(), status() and Seek*() for all iterators. Also fix some bugs Summary: Before this PR, Iterator/InternalIterator may simultaneously have non-ok status() and Valid() = true. That state means that the last operation failed, but the iterator is nevertheless positioned on some unspecified record. Likely intended uses of that are: * If some sst files are corrupted, a normal iterator can be used to read the data from files that are not corrupted. * When using read_tier = kBlockCacheTier, read the data that's in block cache, skipping over the data that is not. However, this behavior wasn't documented well (and until recently the wiki on github had misleading incorrect information). In the code there's a lot of confusion about the relationship between status() and Valid(), and about whether Seek()/SeekToLast()/etc reset the status or not. There were a number of bugs caused by this confusion, both inside rocksdb and in the code that uses rocksdb (including ours). This PR changes the convention to: * If status() is not ok, Valid() always returns false. * Any seek operation resets status. (Before the PR, it depended on iterator type and on particular error.) This does sacrifice the two use cases listed above, but siying said it's ok. Overview of the changes: * A commit that adds missing status checks in MergingIterator. This fixes a bug that actually affects us, and we need it fixed. `DBIteratorTest.NonBlockingIterationBugRepro` explains the scenario. * Changes to lots of iterator types to make all of them conform to the new convention. Some bug fixes along the way. By far the biggest changes are in DBIter, which is a big messy piece of code; I tried to make it less big and messy but mostly failed. * A stress-test for DBIter, to gain some confidence that I didn't break it. It does a few million random operations on the iterator, while occasionally modifying the underlying data (like ForwardIterator does) and occasionally returning non-ok status from internal iterator. To find the iterator types that needed changes I searched for "public .*Iterator" in the code. Here's an overview of all 27 iterator types: Iterators that didn't need changes: * status() is always ok(), or Valid() is always false: MemTableIterator, ModelIter, TestIterator, KVIter (2 classes with this name anonymous namespaces), LoggingForwardVectorIterator, VectorIterator, MockTableIterator, EmptyIterator, EmptyInternalIterator. * Thin wrappers that always pass through Valid() and status(): ArenaWrappedDBIter, TtlIterator, InternalIteratorFromIterator. Iterators with changes (see inline comments for details): * DBIter - an overhaul: - It used to silently skip corrupted keys (`FindParseableKey()`), which seems dangerous. This PR makes it just stop immediately after encountering a corrupted key, just like it would for other kinds of corruption. Let me know if there was actually some deeper meaning in this behavior and I should put it back. - It had a few code paths silently discarding subiterator's status. The stress test caught a few. - The backwards iteration code path was expecting the internal iterator's set of keys to be immutable. It's probably always true in practice at the moment, since ForwardIterator doesn't support backwards iteration, but this PR fixes it anyway. See added DBIteratorTest.ReverseToForwardBug for an example. - Some parts of backwards iteration code path even did things like `assert(iter_->Valid())` after a seek, which is never a safe assumption. - It used to not reset status on seek for some types of errors. - Some simplifications and better comments. - Some things got more complicated from the added error handling. I'm open to ideas for how to make it nicer. * MergingIterator - check status after every operation on every subiterator, and in some places assert that valid subiterators have ok status. * ForwardIterator - changed to the new convention, also slightly simplified. * ForwardLevelIterator - fixed some bugs and simplified. * LevelIterator - simplified. * TwoLevelIterator - changed to the new convention. Also fixed a bug that would make SeekForPrev() sometimes silently ignore errors from first_level_iter_. * BlockBasedTableIterator - minor changes. * BlockIter - replaced `SetStatus()` with `Invalidate()` to make sure non-ok BlockIter is always invalid. * PlainTableIterator - some seeks used to not reset status. * CuckooTableIterator - tiny code cleanup. * ManagedIterator - fixed some bugs. * BaseDeltaIterator - changed to the new convention and fixed a bug. * BlobDBIterator - seeks used to not reset status. * KeyConvertingIterator - some small change. Closes https://github.com/facebook/rocksdb/pull/3810 Differential Revision: D7888019 Pulled By: al13n321 fbshipit-source-id: 4aaf6d3421c545d16722a815b2fa2e7912bc851d
2018-05-17 09:44:14 +00:00
PrevInternal();
if (statistics_ != nullptr) {
RecordTick(statistics_, NUMBER_DB_SEEK);
if (valid_) {
RecordTick(statistics_, NUMBER_DB_SEEK_FOUND);
RecordTick(statistics_, ITER_BYTES_READ, key().size() + value().size());
PERF_COUNTER_ADD(iter_read_bytes, key().size() + value().size());
}
}
if (valid_ && prefix_extractor_ && prefix_same_as_start_) {
prefix_start_buf_.SetUserKey(
prefix_extractor_->Transform(saved_key_.GetUserKey()));
prefix_start_key_ = prefix_start_buf_.GetUserKey();
}
}
Iterator* NewDBIterator(Env* env, const ReadOptions& read_options,
const ImmutableCFOptions& cf_options,
const MutableCFOptions& mutable_cf_options,
const Comparator* user_key_comparator,
InternalIterator* internal_iter,
const SequenceNumber& sequence,
uint64_t max_sequential_skip_in_iterations,
ReadCallback* read_callback, DBImpl* db_impl,
ColumnFamilyData* cfd, bool allow_blob) {
DBIter* db_iter = new DBIter(
env, read_options, cf_options, mutable_cf_options, user_key_comparator,
internal_iter, sequence, false, max_sequential_skip_in_iterations,
read_callback, db_impl, cfd, allow_blob);
Introduce ReadOptions::pin_data (support zero copy for keys) 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
2015-12-16 20:08:30 +00:00
return db_iter;
}
ArenaWrappedDBIter::~ArenaWrappedDBIter() { db_iter_->~DBIter(); }
ReadRangeDelAggregator* ArenaWrappedDBIter::GetRangeDelAggregator() {
return db_iter_->GetRangeDelAggregator();
}
void ArenaWrappedDBIter::SetIterUnderDBIter(InternalIterator* iter) {
static_cast<DBIter*>(db_iter_)->SetIter(iter);
}
inline bool ArenaWrappedDBIter::Valid() const { return db_iter_->Valid(); }
inline void ArenaWrappedDBIter::SeekToFirst() { db_iter_->SeekToFirst(); }
inline void ArenaWrappedDBIter::SeekToLast() { db_iter_->SeekToLast(); }
inline void ArenaWrappedDBIter::Seek(const Slice& target) {
db_iter_->Seek(target);
}
inline void ArenaWrappedDBIter::SeekForPrev(const Slice& target) {
db_iter_->SeekForPrev(target);
}
inline void ArenaWrappedDBIter::Next() { db_iter_->Next(); }
inline void ArenaWrappedDBIter::Prev() { db_iter_->Prev(); }
inline Slice ArenaWrappedDBIter::key() const { return db_iter_->key(); }
inline Slice ArenaWrappedDBIter::value() const { return db_iter_->value(); }
inline Status ArenaWrappedDBIter::status() const { return db_iter_->status(); }
bool ArenaWrappedDBIter::IsBlob() const { return db_iter_->IsBlob(); }
inline Status ArenaWrappedDBIter::GetProperty(std::string prop_name,
std::string* prop) {
if (prop_name == "rocksdb.iterator.super-version-number") {
// First try to pass the value returned from inner iterator.
if (!db_iter_->GetProperty(prop_name, prop).ok()) {
*prop = ToString(sv_number_);
}
return Status::OK();
}
return db_iter_->GetProperty(prop_name, prop);
}
void ArenaWrappedDBIter::Init(Env* env, const ReadOptions& read_options,
const ImmutableCFOptions& cf_options,
const MutableCFOptions& mutable_cf_options,
const SequenceNumber& sequence,
uint64_t max_sequential_skip_in_iteration,
uint64_t version_number,
ReadCallback* read_callback, DBImpl* db_impl,
ColumnFamilyData* cfd, bool allow_blob,
bool allow_refresh) {
auto mem = arena_.AllocateAligned(sizeof(DBIter));
db_iter_ = new (mem) DBIter(env, read_options, cf_options, mutable_cf_options,
cf_options.user_comparator, nullptr, sequence,
true, max_sequential_skip_in_iteration,
read_callback, db_impl, cfd, allow_blob);
sv_number_ = version_number;
allow_refresh_ = allow_refresh;
}
Status ArenaWrappedDBIter::Refresh() {
if (cfd_ == nullptr || db_impl_ == nullptr || !allow_refresh_) {
return Status::NotSupported("Creating renew iterator is not allowed.");
}
assert(db_iter_ != nullptr);
// TODO(yiwu): For last_seq_same_as_publish_seq_==false, this is not the
// correct behavior. Will be corrected automatically when we take a snapshot
// here for the case of WritePreparedTxnDB.
SequenceNumber latest_seq = db_impl_->GetLatestSequenceNumber();
uint64_t cur_sv_number = cfd_->GetSuperVersionNumber();
if (sv_number_ != cur_sv_number) {
Env* env = db_iter_->env();
db_iter_->~DBIter();
arena_.~Arena();
new (&arena_) Arena();
SuperVersion* sv = cfd_->GetReferencedSuperVersion(db_impl_->mutex());
Init(env, read_options_, *(cfd_->ioptions()), sv->mutable_cf_options,
latest_seq, sv->mutable_cf_options.max_sequential_skip_in_iterations,
cur_sv_number, read_callback_, db_impl_, cfd_, allow_blob_,
allow_refresh_);
InternalIterator* internal_iter = db_impl_->NewInternalIterator(
read_options_, cfd_, sv, &arena_, db_iter_->GetRangeDelAggregator(),
latest_seq);
SetIterUnderDBIter(internal_iter);
} else {
db_iter_->set_sequence(latest_seq);
db_iter_->set_valid(false);
}
return Status::OK();
}
ArenaWrappedDBIter* NewArenaWrappedDbIterator(
Env* env, const ReadOptions& read_options,
const ImmutableCFOptions& cf_options,
const MutableCFOptions& mutable_cf_options, const SequenceNumber& sequence,
uint64_t max_sequential_skip_in_iterations, uint64_t version_number,
ReadCallback* read_callback, DBImpl* db_impl, ColumnFamilyData* cfd,
bool allow_blob, bool allow_refresh) {
ArenaWrappedDBIter* iter = new ArenaWrappedDBIter();
iter->Init(env, read_options, cf_options, mutable_cf_options, sequence,
max_sequential_skip_in_iterations, version_number, read_callback,
db_impl, cfd, allow_blob, allow_refresh);
if (db_impl != nullptr && cfd != nullptr && allow_refresh) {
iter->StoreRefreshInfo(read_options, db_impl, cfd, read_callback,
allow_blob);
}
return iter;
}
} // namespace rocksdb