You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db/memtable.h

352 lines
13 KiB

// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same 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.
#pragma once
#include <string>
#include <memory>
#include <functional>
[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
11 years ago
#include <deque>
#include <vector>
#include "db/dbformat.h"
#include "db/skiplist.h"
#include "db/version_edit.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/memtablerep.h"
#include "rocksdb/immutable_options.h"
#include "db/memtable_allocator.h"
#include "util/arena.h"
#include "util/dynamic_bloom.h"
#include "util/mutable_cf_options.h"
namespace rocksdb {
class Mutex;
class MemTableIterator;
class MergeContext;
class WriteBuffer;
struct MemTableOptions {
explicit MemTableOptions(
const ImmutableCFOptions& ioptions,
const MutableCFOptions& mutable_cf_options);
size_t write_buffer_size;
size_t arena_block_size;
uint32_t memtable_prefix_bloom_bits;
uint32_t memtable_prefix_bloom_probes;
size_t memtable_prefix_bloom_huge_page_tlb_size;
bool inplace_update_support;
size_t inplace_update_num_locks;
UpdateStatus (*inplace_callback)(char* existing_value,
uint32_t* existing_value_size,
Slice delta_value,
std::string* merged_value);
size_t max_successive_merges;
bool filter_deletes;
Statistics* statistics;
MergeOperator* merge_operator;
Logger* info_log;
};
// Note: Many of the methods in this class have comments indicating that
// external synchromization is required as these methods are not thread-safe.
// It is up to higher layers of code to decide how to prevent concurrent
// invokation of these methods. This is usually done by acquiring either
// the db mutex or the single writer thread.
//
// Some of these methods are documented to only require external
// synchronization if this memtable is immutable. Calling MarkImmutable() is
// not sufficient to guarantee immutability. It is up to higher layers of
// code to determine if this MemTable can still be modified by other threads.
// Eg: The Superversion stores a pointer to the current MemTable (that can
// be modified) and a separate list of the MemTables that can no longer be
// written to (aka the 'immutable memtables').
class MemTable {
public:
struct KeyComparator : public MemTableRep::KeyComparator {
const InternalKeyComparator comparator;
explicit KeyComparator(const InternalKeyComparator& c) : comparator(c) { }
virtual int operator()(const char* prefix_len_key1,
const char* prefix_len_key2) const override;
virtual int operator()(const char* prefix_len_key,
const Slice& key) const override;
};
// MemTables are reference counted. The initial reference count
// is zero and the caller must call Ref() at least once.
//
// earliest_seq should be the current SequenceNumber in the db such that any
// key inserted into this memtable will have an equal or larger seq number.
// (When a db is first created, the earliest sequence number will be 0).
// If the earliest sequence number is not known, kMaxSequenceNumber may be
// used, but this may prevent some transactions from succeeding until the
// first key is inserted into the memtable.
explicit MemTable(const InternalKeyComparator& comparator,
const ImmutableCFOptions& ioptions,
const MutableCFOptions& mutable_cf_options,
WriteBuffer* write_buffer, SequenceNumber earliest_seq);
// Do not delete this MemTable unless Unref() indicates it not in use.
~MemTable();
// Increase reference count.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
void Ref() { ++refs_; }
// Drop reference count.
// If the refcount goes to zero return this memtable, otherwise return null.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
MemTable* Unref() {
--refs_;
assert(refs_ >= 0);
if (refs_ <= 0) {
return this;
}
return nullptr;
}
// Returns an estimate of the number of bytes of data in use by this
// data structure.
//
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable (unless this Memtable is immutable).
size_t ApproximateMemoryUsage();
// This method heuristically determines if the memtable should continue to
// host more data.
bool ShouldScheduleFlush() const {
return flush_scheduled_ == false && should_flush_;
}
void MarkFlushScheduled() { flush_scheduled_ = true; }
// Return an iterator that yields the contents of the memtable.
//
// The caller must ensure that the underlying MemTable remains live
// while the returned iterator is live. The keys returned by this
// iterator are internal keys encoded by AppendInternalKey in the
// db/dbformat.{h,cc} module.
//
// By default, it returns an iterator for prefix seek if prefix_extractor
// is configured in Options.
// arena: If not null, the arena needs to be used to allocate the Iterator.
// Calling ~Iterator of the iterator will destroy all the states but
// those allocated in arena.
Iterator* NewIterator(const ReadOptions& read_options, Arena* arena);
// Add an entry into memtable that maps key to value at the
// specified sequence number and with the specified type.
// Typically value will be empty if type==kTypeDeletion.
//
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
void Add(SequenceNumber seq, ValueType type,
const Slice& key,
const Slice& value);
// If memtable contains a value for key, store it in *value and return true.
// If memtable contains a deletion for key, store a NotFound() error
// in *status and return true.
// If memtable contains Merge operation as the most recent entry for a key,
// and the merge process does not stop (not reaching a value or delete),
[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
11 years ago
// prepend the current merge operand to *operands.
// store MergeInProgress in s, and return false.
// Else, return false.
// If any operation was found, its most recent sequence number
// will be stored in *seq on success (regardless of whether true/false is
// returned). Otherwise, *seq will be set to kMaxSequenceNumber.
// On success, *s may be set to OK, NotFound, or MergeInProgress. Any other
// status returned indicates a corruption or other unexpected error.
bool Get(const LookupKey& key, std::string* value, Status* s,
MergeContext* merge_context, SequenceNumber* seq);
bool Get(const LookupKey& key, std::string* value, Status* s,
MergeContext* merge_context) {
SequenceNumber seq;
return Get(key, value, s, merge_context, &seq);
}
// Attempts to update the new_value inplace, else does normal Add
// Pseudocode
// if key exists in current memtable && prev_value is of type kTypeValue
// if new sizeof(new_value) <= sizeof(prev_value)
// update inplace
// else add(key, new_value)
// else add(key, new_value)
//
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
void Update(SequenceNumber seq,
const Slice& key,
const Slice& value);
// If prev_value for key exits, attempts to update it inplace.
// else returns false
// Pseudocode
// if key exists in current memtable && prev_value is of type kTypeValue
// new_value = delta(prev_value)
// if sizeof(new_value) <= sizeof(prev_value)
// update inplace
// else add(key, new_value)
// else return false
//
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
bool UpdateCallback(SequenceNumber seq,
const Slice& key,
const Slice& delta);
// Returns the number of successive merge entries starting from the newest
// entry for the key up to the last non-merge entry or last entry for the
// key in the memtable.
size_t CountSuccessiveMergeEntries(const LookupKey& key);
// Get total number of entries in the mem table.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable (unless this Memtable is immutable).
uint64_t num_entries() const {
return num_entries_.load(std::memory_order_relaxed);
}
// Get total number of deletes in the mem table.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable (unless this Memtable is immutable).
uint64_t num_deletes() const { return num_deletes_; }
// Returns the edits area that is needed for flushing the memtable
VersionEdit* GetEdits() { return &edit_; }
// Returns if there is no entry inserted to the mem table.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable (unless this Memtable is immutable).
bool IsEmpty() const { return first_seqno_ == 0; }
// Returns the sequence number of the first element that was inserted
// into the memtable.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable (unless this Memtable is immutable).
SequenceNumber GetFirstSequenceNumber() { return first_seqno_; }
// Returns the sequence number that is guaranteed to be smaller than or equal
// to the sequence number of any key that could be inserted into this
// memtable. It can then be assumed that any write with a larger(or equal)
// sequence number will be present in this memtable or a later memtable.
//
// If the earliest sequence number could not be determined,
// kMaxSequenceNumber will be returned.
SequenceNumber GetEarliestSequenceNumber() { return earliest_seqno_; }
// Returns the next active logfile number when this memtable is about to
// be flushed to storage
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
uint64_t GetNextLogNumber() { return mem_next_logfile_number_; }
// Sets the next active logfile number when this memtable is about to
// be flushed to storage
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
void SetNextLogNumber(uint64_t num) { mem_next_logfile_number_ = num; }
// Notify the underlying storage that no more items will be added.
// REQUIRES: external synchronization to prevent simultaneous
// operations on the same MemTable.
// After MarkImmutable() is called, you should not attempt to
// write anything to this MemTable(). (Ie. do not call Add() or Update()).
void MarkImmutable() {
table_->MarkReadOnly();
allocator_.DoneAllocating();
}
Add a new mem-table representation based on cuckoo hash. Summary: = Major Changes = * Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash. Cuckoo hash uses multiple hash functions. This allows each key to have multiple possible locations in the mem-table. - Put: When insert a key, it will try to find whether one of its possible locations is vacant and store the key. If none of its possible locations are available, then it will kick out a victim key and store at that location. The kicked-out victim key will then be stored at a vacant space of its possible locations or kick-out another victim. In this diff, the kick-out path (known as cuckoo-path) is found using BFS, which guarantees to be the shortest. - Get: Simply tries all possible locations of a key --- this guarantees worst-case constant time complexity. - Time complexity: O(1) for Get, and average O(1) for Put if the fullness of the mem-table is below 80%. - Default using two hash functions, the number of hash functions used by the cuckoo-hash may dynamically increase if it fails to find a short-enough kick-out path. - Currently, HashCuckooRep does not support iteration and snapshots, as our current main purpose of this is to optimize point access. = Minor Changes = * Add IsSnapshotSupported() to DB to indicate whether the current DB supports snapshots. If it returns false, then DB::GetSnapshot() will always return nullptr. Test Plan: Run existing tests. Will develop a test specifically for cuckoo hash in the next diff. Reviewers: sdong, haobo Reviewed By: sdong CC: leveldb, dhruba, igor Differential Revision: https://reviews.facebook.net/D16155
10 years ago
// return true if the current MemTableRep supports merge operator.
bool IsMergeOperatorSupported() const {
return table_->IsMergeOperatorSupported();
}
// return true if the current MemTableRep supports snapshots.
// inplace update prevents snapshots,
bool IsSnapshotSupported() const {
return table_->IsSnapshotSupported() && !moptions_.inplace_update_support;
}
Add a new mem-table representation based on cuckoo hash. Summary: = Major Changes = * Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash. Cuckoo hash uses multiple hash functions. This allows each key to have multiple possible locations in the mem-table. - Put: When insert a key, it will try to find whether one of its possible locations is vacant and store the key. If none of its possible locations are available, then it will kick out a victim key and store at that location. The kicked-out victim key will then be stored at a vacant space of its possible locations or kick-out another victim. In this diff, the kick-out path (known as cuckoo-path) is found using BFS, which guarantees to be the shortest. - Get: Simply tries all possible locations of a key --- this guarantees worst-case constant time complexity. - Time complexity: O(1) for Get, and average O(1) for Put if the fullness of the mem-table is below 80%. - Default using two hash functions, the number of hash functions used by the cuckoo-hash may dynamically increase if it fails to find a short-enough kick-out path. - Currently, HashCuckooRep does not support iteration and snapshots, as our current main purpose of this is to optimize point access. = Minor Changes = * Add IsSnapshotSupported() to DB to indicate whether the current DB supports snapshots. If it returns false, then DB::GetSnapshot() will always return nullptr. Test Plan: Run existing tests. Will develop a test specifically for cuckoo hash in the next diff. Reviewers: sdong, haobo Reviewed By: sdong CC: leveldb, dhruba, igor Differential Revision: https://reviews.facebook.net/D16155
10 years ago
uint64_t ApproximateSize(const Slice& start_ikey, const Slice& end_ikey);
// Get the lock associated for the key
port::RWMutex* GetLock(const Slice& key);
const InternalKeyComparator& GetInternalKeyComparator() const {
return comparator_.comparator;
}
const MemTableOptions* GetMemTableOptions() const { return &moptions_; }
private:
// Dynamically check if we can add more incoming entries
bool ShouldFlushNow() const;
friend class MemTableIterator;
friend class MemTableBackwardIterator;
friend class MemTableList;
KeyComparator comparator_;
const MemTableOptions moptions_;
int refs_;
const size_t kArenaBlockSize;
Arena arena_;
MemTableAllocator allocator_;
unique_ptr<MemTableRep> table_;
// Total data size of all data inserted
std::atomic<uint64_t> data_size_;
std::atomic<uint64_t> num_entries_;
uint64_t num_deletes_;
// These are used to manage memtable flushes to storage
bool flush_in_progress_; // started the flush
bool flush_completed_; // finished the flush
uint64_t file_number_; // filled up after flush is complete
// The updates to be applied to the transaction log when this
// memtable is flushed to storage.
VersionEdit edit_;
// The sequence number of the kv that was inserted first
SequenceNumber first_seqno_;
// The db sequence number at the time of creation or kMaxSequenceNumber
// if not set.
SequenceNumber earliest_seqno_;
// The log files earlier than this number can be deleted.
uint64_t mem_next_logfile_number_;
// rw locks for inplace updates
std::vector<port::RWMutex> locks_;
// No copying allowed
MemTable(const MemTable&);
void operator=(const MemTable&);
const SliceTransform* const prefix_extractor_;
std::unique_ptr<DynamicBloom> prefix_bloom_;
// a flag indicating if a memtable has met the criteria to flush
bool should_flush_;
// a flag indicating if flush has been scheduled
bool flush_scheduled_;
Env* env_;
};
extern const char* EncodeKey(std::string* scratch, const Slice& target);
} // namespace rocksdb