diff --git a/db/db_bench.cc b/db/db_bench.cc index 9b3665694..c130063eb 100644 --- a/db/db_bench.cc +++ b/db/db_bench.cc @@ -353,6 +353,18 @@ static auto FLAGS_bytes_per_sync = // On true, deletes use bloom-filter and drop the delete if key not present static bool FLAGS_filter_deletes = false; +// Control the prefix size for PrefixHashRep +static bool FLAGS_prefix_size = 0; + +enum RepFactory { + kSkipList, + kPrefixHash, + kUnsorted, + kVectorRep +}; + +static enum RepFactory FLAGS_rep_factory; + // The merge operator to use with the database. // If a new merge operator is specified, be sure to use fresh database // The possible merge operators are defined in utilities/merge_operators.h @@ -673,6 +685,21 @@ class Benchmark { break; } + switch (FLAGS_rep_factory) { + case kPrefixHash: + fprintf(stdout, "Memtablerep: prefix_hash\n"); + break; + case kSkipList: + fprintf(stdout, "Memtablerep: skip_list\n"); + break; + case kUnsorted: + fprintf(stdout, "Memtablerep: unsorted\n"); + break; + case kVectorRep: + fprintf(stdout, "Memtablerep: vector\n"); + break; + } + PrintWarnings(); fprintf(stdout, "------------------------------------------------\n"); } @@ -1159,6 +1186,31 @@ class Benchmark { options.max_bytes_for_level_multiplier = FLAGS_max_bytes_for_level_multiplier; options.filter_deletes = FLAGS_filter_deletes; + if ((FLAGS_prefix_size == 0) == (FLAGS_rep_factory == kPrefixHash)) { + fprintf(stderr, + "prefix_size should be non-zero iff memtablerep == prefix_hash\n"); + exit(1); + } + switch (FLAGS_rep_factory) { + case kPrefixHash: + options.memtable_factory.reset( + new PrefixHashRepFactory(NewFixedPrefixTransform(FLAGS_prefix_size)) + ); + break; + case kUnsorted: + options.memtable_factory.reset( + new UnsortedRepFactory + ); + break; + case kSkipList: + // no need to do anything + break; + case kVectorRep: + options.memtable_factory.reset( + new VectorRepFactory + ); + break; + } if (FLAGS_max_bytes_for_level_multiplier_additional.size() > 0) { if (FLAGS_max_bytes_for_level_multiplier_additional.size() != (unsigned int)FLAGS_num_levels) { @@ -2324,6 +2376,19 @@ int main(int argc, char** argv) { else { fprintf(stdout, "Cannot parse %s\n", argv[i]); } + } else if (strncmp(argv[i], "--memtablerep=", 14) == 0) { + const char* ctype = argv[i] + 14; + if (!strcasecmp(ctype, "skip_list")) + FLAGS_rep_factory = kSkipList; + else if (!strcasecmp(ctype, "prefix_hash")) + FLAGS_rep_factory = kPrefixHash; + else if (!strcasecmp(ctype, "unsorted")) + FLAGS_rep_factory = kUnsorted; + else if (!strcasecmp(ctype, "vector")) + FLAGS_rep_factory = kVectorRep; + else { + fprintf(stdout, "Cannot parse %s\n", argv[i]); + } } else if (sscanf(argv[i], "--min_level_to_compress=%d%c", &n, &junk) == 1 && n >= 0) { FLAGS_min_level_to_compress = n; @@ -2338,6 +2403,9 @@ int main(int argc, char** argv) { } else if (sscanf(argv[i], "--stats_per_interval=%d%c", &n, &junk) == 1 && (n == 0 || n == 1)) { FLAGS_stats_per_interval = n; + } else if (sscanf(argv[i], "--prefix_size=%d%c", &n, &junk) == 1 && + n >= 0 && n < 2000000000) { + FLAGS_prefix_size = n; } else if (sscanf(argv[i], "--soft_rate_limit=%lf%c", &d, &junk) == 1 && d > 0.0) { FLAGS_soft_rate_limit = d; diff --git a/db/db_impl.cc b/db/db_impl.cc index 6c9980ec0..30b781873 100644 --- a/db/db_impl.cc +++ b/db/db_impl.cc @@ -163,6 +163,19 @@ Options SanitizeOptions(const std::string& dbname, result.compaction_filter_factory->CreateCompactionFilter().get()) { Log(result.info_log, "Both filter and factory specified. Using filter"); } + if (result.prefix_extractor) { + // If a prefix extractor has been supplied and a PrefixHashRepFactory is + // being used, make sure that the latter uses the former as its transform + // function. + auto factory = dynamic_cast( + result.memtable_factory.get()); + if (factory != nullptr && factory->transform_ != result.prefix_extractor) { + Log(result.info_log, "A prefix hash representation factory was supplied " + "whose prefix extractor does not match options.prefix_extractor. " + "Falling back to skip list representation factory"); + result.memtable_factory = std::make_shared(); + } + } return result; } @@ -2143,7 +2156,8 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options, // Collect together all needed child iterators for mem std::vector list; mem_->Ref(); - list.push_back(mem_->NewIterator()); + list.push_back(mem_->NewIterator(options.prefix)); + cleanup->mem.push_back(mem_); // Collect together all needed child iterators for imm_ @@ -2152,7 +2166,7 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options, for (unsigned int i = 0; i < immutables.size(); i++) { MemTable* m = immutables[i]; m->Ref(); - list.push_back(m->NewIterator()); + list.push_back(m->NewIterator(options.prefix)); cleanup->mem.push_back(m); } diff --git a/db/db_statistics.h b/db/db_statistics.h index 725b75798..d11eb3145 100644 --- a/db/db_statistics.h +++ b/db/db_statistics.h @@ -62,5 +62,3 @@ std::shared_ptr CreateDBStatistics() { } // namespace leveldb #endif // LEVELDB_STORAGE_DB_DB_STATISTICS_H_ - - diff --git a/db/db_test.cc b/db/db_test.cc index 870f9fd98..227e44b6c 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -207,9 +207,12 @@ class DBTest { private: const FilterPolicy* filter_policy_; + protected: // Sequence of option configurations to try enum OptionConfig { kDefault, + kVectorRep, + kUnsortedRep, kMergePut, kFilter, kUncompressed, @@ -219,6 +222,7 @@ class DBTest { kCompactOnFlush, kPerfOptions, kDeletesFilterFirst, + kPrefixHashRep, kUniversalCompaction, kEnd }; @@ -293,6 +297,10 @@ class DBTest { Options CurrentOptions() { Options options; switch (option_config_) { + case kPrefixHashRep: + options.memtable_factory.reset(new + PrefixHashRepFactory(NewFixedPrefixTransform(1))); + break; case kMergePut: options.merge_operator = MergeOperators::CreatePutOperator(); break; @@ -321,6 +329,12 @@ class DBTest { case kDeletesFilterFirst: options.filter_deletes = true; break; + case kUnsortedRep: + options.memtable_factory.reset(new UnsortedRepFactory); + break; + case kVectorRep: + options.memtable_factory.reset(new VectorRepFactory); + break; case kUniversalCompaction: options.compaction_style = kCompactionStyleUniversal; break; @@ -3509,10 +3523,13 @@ class ModelDB: public DB { KVMap map_; }; -static std::string RandomKey(Random* rnd) { - int len = (rnd->OneIn(3) - ? 1 // Short sometimes to encourage collisions - : (rnd->OneIn(100) ? rnd->Skewed(10) : rnd->Uniform(10))); +static std::string RandomKey(Random* rnd, int minimum = 0) { + int len; + do { + len = (rnd->OneIn(3) + ? 1 // Short sometimes to encourage collisions + : (rnd->OneIn(100) ? rnd->Skewed(10) : rnd->Uniform(10))); + } while (len < minimum); return test::RandomKey(rnd, len); } @@ -3574,8 +3591,12 @@ TEST(DBTest, Randomized) { for (int step = 0; step < N; step++) { // TODO(sanjay): Test Get() works int p = rnd.Uniform(100); + int minimum = 0; + if (option_config_ == kPrefixHashRep) { + minimum = 1; + } if (p < 45) { // Put - k = RandomKey(&rnd); + k = RandomKey(&rnd, minimum); v = RandomString(&rnd, rnd.OneIn(20) ? 100 + rnd.Uniform(100) @@ -3584,7 +3605,7 @@ TEST(DBTest, Randomized) { ASSERT_OK(db_->Put(WriteOptions(), k, v)); } else if (p < 90) { // Delete - k = RandomKey(&rnd); + k = RandomKey(&rnd, minimum); ASSERT_OK(model.Delete(WriteOptions(), k)); ASSERT_OK(db_->Delete(WriteOptions(), k)); @@ -3594,7 +3615,7 @@ TEST(DBTest, Randomized) { const int num = rnd.Uniform(8); for (int i = 0; i < num; i++) { if (i == 0 || !rnd.OneIn(10)) { - k = RandomKey(&rnd); + k = RandomKey(&rnd, minimum); } else { // Periodically re-use the same key from the previous iter, so // we have multiple entries in the write batch for the same key @@ -3750,6 +3771,9 @@ TEST(DBTest, PrefixScan) { snprintf(buf, sizeof(buf), "03______:"); prefix = Slice(buf, 8); key = Slice(buf, 9); + auto prefix_extractor = NewFixedPrefixTransform(8); + auto memtable_factory = + std::make_shared(prefix_extractor); // db configs env_->count_random_reads_ = true; @@ -3757,12 +3781,13 @@ TEST(DBTest, PrefixScan) { options.env = env_; options.block_cache = NewLRUCache(0); // Prevent cache hits options.filter_policy = NewBloomFilterPolicy(10); - options.prefix_extractor = NewFixedPrefixTransform(8); + options.prefix_extractor = prefix_extractor; options.whole_key_filtering = false; options.disable_auto_compactions = true; options.max_background_compactions = 2; options.create_if_missing = true; options.disable_seek_compaction = true; + options.memtable_factory = memtable_factory; // prefix specified, with blooms: 2 RAND I/Os // SeekToFirst diff --git a/db/memtable.cc b/db/memtable.cc index e2fa482f0..e3c19f3fc 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -12,16 +12,10 @@ #include "leveldb/iterator.h" #include "leveldb/merge_operator.h" #include "util/coding.h" +#include "util/murmurhash.h" namespace leveldb { -static Slice GetLengthPrefixedSlice(const char* data) { - uint32_t len; - const char* p = data; - p = GetVarint32Ptr(p, p + 5, &len); // +5: we assume "p" is not corrupted - return Slice(p, len); -} - MemTable::MemTable(const InternalKeyComparator& cmp, std::shared_ptr table_factory, int numlevel, @@ -42,7 +36,8 @@ MemTable::~MemTable() { } size_t MemTable::ApproximateMemoryUsage() { - return arena_impl_.ApproximateMemoryUsage(); + return arena_impl_.ApproximateMemoryUsage() + + table_->ApproximateMemoryUsage(); } int MemTable::KeyComparator::operator()(const char* aptr, const char* bptr) @@ -53,6 +48,11 @@ int MemTable::KeyComparator::operator()(const char* aptr, const char* bptr) return comparator.Compare(a, b); } +Slice MemTableRep::UserKey(const char* key) const { + Slice slice = GetLengthPrefixedSlice(key); + return Slice(slice.data(), slice.size() - 8); +} + // Encode a suitable internal key target for "target" and return it. // Uses *scratch as scratch space, and the returned pointer will point // into this scratch space. @@ -68,6 +68,9 @@ class MemTableIterator: public Iterator { explicit MemTableIterator(MemTableRep* table) : iter_(table->GetIterator()) { } + MemTableIterator(MemTableRep* table, const Slice* prefix) + : iter_(table->GetPrefixIterator(*prefix)) { } + virtual bool Valid() const { return iter_->Valid(); } virtual void Seek(const Slice& k) { iter_->Seek(EncodeKey(&tmp_, k)); } virtual void SeekToFirst() { iter_->SeekToFirst(); } @@ -93,8 +96,12 @@ class MemTableIterator: public Iterator { void operator=(const MemTableIterator&); }; -Iterator* MemTable::NewIterator() { - return new MemTableIterator(table_.get()); +Iterator* MemTable::NewIterator(const Slice* prefix) { + if (prefix) { + return new MemTableIterator(table_.get(), prefix); + } else { + return new MemTableIterator(table_.get()); + } } void MemTable::Add(SequenceNumber s, ValueType type, @@ -132,7 +139,8 @@ void MemTable::Add(SequenceNumber s, ValueType type, bool MemTable::Get(const LookupKey& key, std::string* value, Status* s, std::deque* operands, const Options& options) { Slice memkey = key.memtable_key(); - std::shared_ptr iter(table_.get()->GetIterator()); + std::shared_ptr iter( + table_->GetIterator(key.user_key())); iter->Seek(memkey.data()); // It is the caller's responsibility to allocate/delete operands list diff --git a/db/memtable.h b/db/memtable.h index 481f84079..14b043643 100644 --- a/db/memtable.h +++ b/db/memtable.h @@ -61,7 +61,11 @@ class MemTable { // 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. - Iterator* NewIterator(); + // + // If a prefix is supplied, it is passed to the underlying MemTableRep as a + // hint that the iterator only need to support access to keys with that + // prefix. + Iterator* NewIterator(const Slice* prefix = nullptr); // Add an entry into memtable that maps key to value at the // specified sequence number and with the specified type. @@ -96,6 +100,9 @@ class MemTable { // memstore is flushed to storage void SetLogNumber(uint64_t num) { mem_logfile_number_ = num; } + // Notify the underlying storage that no more items will be added + void MarkImmutable() { table_->MarkReadOnly(); } + private: ~MemTable(); // Private since only Unref() should be used to delete it friend class MemTableIterator; diff --git a/db/memtablelist.cc b/db/memtablelist.cc index 256afe361..56c1de545 100644 --- a/db/memtablelist.cc +++ b/db/memtablelist.cc @@ -172,6 +172,7 @@ void MemTableList::Add(MemTable* m) { assert(size_ >= num_flush_not_started_); size_++; memlist_.push_front(m); + m->MarkImmutable(); num_flush_not_started_++; if (num_flush_not_started_ == 1) { imm_flush_needed.Release_Store((void *)1); diff --git a/db/write_batch_test.cc b/db/write_batch_test.cc index c20c9c0c0..0a4b97eee 100644 --- a/db/write_batch_test.cc +++ b/db/write_batch_test.cc @@ -5,10 +5,10 @@ #include "leveldb/db.h" #include -#include "db/skiplistrep.h" #include "db/memtable.h" #include "db/write_batch_internal.h" #include "leveldb/env.h" +#include "leveldb/memtablerep.h" #include "util/logging.h" #include "util/testharness.h" diff --git a/include/leveldb/arena.h b/include/leveldb/arena.h index 6e3a1f00b..fb55de98d 100644 --- a/include/leveldb/arena.h +++ b/include/leveldb/arena.h @@ -8,6 +8,9 @@ #ifndef STORAGE_LEVELDB_INCLUDE_ARENA_H_ #define STORAGE_LEVELDB_INCLUDE_ARENA_H_ +#include +#include + namespace leveldb { class Arena { diff --git a/include/leveldb/memtablerep.h b/include/leveldb/memtablerep.h index bf769f543..bf1b8abdc 100644 --- a/include/leveldb/memtablerep.h +++ b/include/leveldb/memtablerep.h @@ -4,26 +4,58 @@ // (1) It does not store duplicate items. // (2) It uses MemTableRep::KeyComparator to compare items for iteration and // equality. -// (3) It can be accessed concurrently by multiple readers but need not support -// concurrent writes. +// (3) It can be accessed concurrently by multiple readers and can support +// during reads. However, it needn't support multiple concurrent writes. // (4) Items are never deleted. // The liberal use of assertions is encouraged to enforce (1). - -#ifndef STORAGE_LEVELDB_DB_TABLE_H_ -#define STORAGE_LEVELDB_DB_TABLE_H_ +// +// The factory will be passed an Arena object when a new MemTableRep is +// requested. The API for this object is in leveldb/arena.h. +// +// Users can implement their own memtable representations. We include four +// types built in: +// - SkipListRep: This is the default; it is backed by a skip list. +// - TransformRep: This is backed by an std::unordered_map. On construction, they are given a SliceTransform object. This +// object is applied to the user key of stored items which indexes into the +// unordered map to yield a set containing all records that share the same user +// key under the transform function. +// - UnsortedRep: A subclass of TransformRep where the transform function is +// the identity function. Optimized for point lookups. +// - PrefixHashRep: A subclass of TransformRep where the transform function is +// a fixed-size prefix extractor. If you use PrefixHashRepFactory, the transform +// must be identical to options.prefix_extractor, otherwise it will be discarded +// and the default will be used. It is optimized for ranged scans over a +// prefix. +// - VectorRep: This is backed by an unordered std::vector. On iteration, the +// vector is sorted. It is intelligent about sorting; once the MarkReadOnly() +// has been called, the vector will only be sorted once. It is optimized for +// random-write-heavy workloads. +// +// The last four implementations are designed for situations in which +// iteration over the entire collection is rare since doing so requires all the +// keys to be copied into a sorted data structure. + +#ifndef STORAGE_LEVELDB_DB_MEMTABLEREP_H_ +#define STORAGE_LEVELDB_DB_MEMTABLEREP_H_ #include #include "leveldb/arena.h" +#include "leveldb/slice.h" +#include "leveldb/slice_transform.h" namespace leveldb { class MemTableRep { public: - // KeyComparator(a, b) returns a negative value if a is less than b, 0 if they - // are equal, and a positive value if b is greater than a + // KeyComparator provides a means to compare keys, which are internal keys + // concatenated with values. class KeyComparator { - public: + public: + // Compare a and b. Return a negative value if a is less than b, 0 if they + // are equal, and a positive value if a is greater than b virtual int operator()(const char* a, const char* b) const = 0; + virtual ~KeyComparator() { } }; @@ -36,6 +68,14 @@ class MemTableRep { // Returns true iff an entry that compares equal to key is in the collection. virtual bool Contains(const char* key) const = 0; + // Notify this table rep that it will no longer be added to. By default, does + // nothing. + virtual void MarkReadOnly() { } + + // Report an approximation of how much memory has been used other than memory + // that was allocated through the arena. + virtual size_t ApproximateMemoryUsage() = 0; + virtual ~MemTableRep() { } // Iteration over the contents of a skip collection @@ -73,16 +113,118 @@ class MemTableRep { virtual void SeekToLast() = 0; }; + // Return an iterator over the keys in this representation. virtual std::shared_ptr GetIterator() = 0; + + // Return an iterator over at least the keys with the specified user key. The + // iterator may also allow access to other keys, but doesn't have to. Default: + // GetIterator(). + virtual std::shared_ptr GetIterator(const Slice& user_key) { + return GetIterator(); + } + + // Return an iterator over at least the keys with the specified prefix. The + // iterator may also allow access to other keys, but doesn't have to. Default: + // GetIterator(). + virtual std::shared_ptr GetPrefixIterator(const Slice& prefix) { + return GetIterator(); + } + + protected: + // When *key is an internal key concatenated with the value, returns the + // user key. + virtual Slice UserKey(const char* key) const; }; +// This is the base class for all factories that are used by RocksDB to create +// new MemTableRep objects class MemTableRepFactory { public: virtual ~MemTableRepFactory() { }; virtual std::shared_ptr CreateMemTableRep( - MemTableRep::KeyComparator&, Arena* arena) = 0; + MemTableRep::KeyComparator&, Arena*) = 0; +}; + +// This creates MemTableReps that are backed by an std::vector. On iteration, +// the vector is sorted. This is useful for workloads where iteration is very +// rare and writes are generally not issued after reads begin. +// +// Parameters: +// count: Passed to the constructor of the underlying std::vector of each +// VectorRep. On initialization, the underlying array will be at least count +// size. +class VectorRepFactory : public MemTableRepFactory { + const size_t count_; +public: + explicit VectorRepFactory(size_t count = 0) : count_(count) { } + virtual std::shared_ptr CreateMemTableRep( + MemTableRep::KeyComparator&, Arena*) override; +}; + +// This uses a skip list to store keys. It is the default. +class SkipListFactory : public MemTableRepFactory { +public: + virtual std::shared_ptr CreateMemTableRep( + MemTableRep::KeyComparator&, Arena*) override; +}; + +// TransformReps are backed by an unordered map of buffers to buckets. When +// looking up a key, the user key is extracted and a user-supplied transform +// function (see leveldb/slice_transform.h) is applied to get the key into the +// unordered map. This allows the user to bin user keys based on arbitrary +// criteria. Two example implementations are UnsortedRepFactory and +// PrefixHashRepFactory. +// +// Iteration over the entire collection is implemented by dumping all the keys +// into an std::set. Thus, these data structures are best used when iteration +// over the entire collection is rare. +// +// Parameters: +// transform: The SliceTransform to bucket user keys on. +// bucket_count: Passed to the constructor of the underlying +// std::unordered_map of each TransformRep. On initialization, the +// underlying array will be at least bucket_count size. +// num_locks: Number of read-write locks to have for the rep. Each bucket is +// hashed onto a read-write lock which controls access to that lock. More +// locks means finer-grained concurrency but more memory overhead. +class TransformRepFactory : public MemTableRepFactory { +public: + const SliceTransform* transform_; + const size_t bucket_count_; + const size_t num_locks_; + explicit TransformRepFactory(const SliceTransform* transform, + size_t bucket_count, size_t num_locks = 1000) + : transform_(transform), + bucket_count_(bucket_count), + num_locks_(num_locks) { } + virtual std::shared_ptr CreateMemTableRep( + MemTableRep::KeyComparator&, Arena*) override; +}; + +// UnsortedReps bin user keys based on an identity function transform -- that +// is, transform(key) = key. This optimizes for point look-ups. +// +// Parameters: See TransformRepFactory. +class UnsortedRepFactory : public TransformRepFactory { +public: + explicit UnsortedRepFactory(size_t bucket_count = 0, size_t num_locks = 1000) + : TransformRepFactory(NewNoopTransform(), bucket_count, num_locks) { } +}; + +// PrefixHashReps bin user keys based on a fixed-size prefix. This optimizes for +// short ranged scans over a given prefix. +// +// Parameters: See TransformRepFactory. +class PrefixHashRepFactory : public TransformRepFactory { +public: + explicit PrefixHashRepFactory(const SliceTransform* prefix_extractor, + size_t bucket_count = 0, size_t num_locks = 1000) + : TransformRepFactory(prefix_extractor, bucket_count, num_locks) + { } + virtual std::shared_ptr CreateMemTableRep( + MemTableRep::KeyComparator&, Arena*) override; }; } -#endif // STORAGE_LEVELDB_DB_TABLE_H_ +#endif // STORAGE_LEVELDB_DB_MEMTABLEREP_H_ diff --git a/include/leveldb/slice.h b/include/leveldb/slice.h index e0fc9aa25..ba0f3467b 100644 --- a/include/leveldb/slice.h +++ b/include/leveldb/slice.h @@ -31,9 +31,11 @@ class Slice { Slice(const char* d, size_t n) : data_(d), size_(n) { } // Create a slice that refers to the contents of "s" + /* implicit */ Slice(const std::string& s) : data_(s.data()), size_(s.size()) { } // Create a slice that refers to s[0,strlen(s)-1] + /* implicit */ Slice(const char* s) : data_(s), size_(strlen(s)) { } // Return a pointer to the beginning of the referenced data @@ -117,5 +119,4 @@ inline int Slice::compare(const Slice& b) const { } // namespace leveldb - #endif // STORAGE_LEVELDB_INCLUDE_SLICE_H_ diff --git a/include/leveldb/slice_transform.h b/include/leveldb/slice_transform.h index fa94d2141..08dce5825 100644 --- a/include/leveldb/slice_transform.h +++ b/include/leveldb/slice_transform.h @@ -36,6 +36,8 @@ class SliceTransform { extern const SliceTransform* NewFixedPrefixTransform(size_t prefix_len); +extern const SliceTransform* NewNoopTransform(); + } #endif // STORAGE_LEVELDB_INCLUDE_SLICE_TRANSFORM_H_ diff --git a/table/table_test.cc b/table/table_test.cc index 118ffa232..d78725dac 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -6,12 +6,12 @@ #include #include "db/dbformat.h" #include "db/memtable.h" -#include "db/skiplistrep.h" #include "db/write_batch_internal.h" #include "leveldb/db.h" #include "leveldb/env.h" #include "leveldb/iterator.h" #include "leveldb/table_builder.h" +#include "leveldb/memtablerep.h" #include "table/block.h" #include "table/block_builder.h" #include "table/format.h" diff --git a/tools/db_stress.cc b/tools/db_stress.cc index b8a251e2f..e9f31e208 100644 --- a/tools/db_stress.cc +++ b/tools/db_stress.cc @@ -199,6 +199,18 @@ static bool FLAGS_filter_deletes = false; // Level0 compaction start trigger static int FLAGS_level0_file_num_compaction_trigger = 0; +enum RepFactory { + kSkipList, + kPrefixHash, + kUnsorted, + kVectorRep +}; + +static enum RepFactory FLAGS_rep_factory; + +// Control the prefix size for PrefixHashRep +static bool FLAGS_prefix_size = 0; + // On true, replaces all writes with a Merge that behaves like a Put static bool FLAGS_use_merge_put = false; @@ -1094,6 +1106,25 @@ class StressTest { } fprintf(stdout, "Compression : %s\n", compression); + + const char* memtablerep = ""; + switch (FLAGS_rep_factory) { + case kSkipList: + memtablerep = "skip_list"; + break; + case kPrefixHash: + memtablerep = "prefix_hash"; + break; + case kUnsorted: + memtablerep = "unsorted"; + break; + case kVectorRep: + memtablerep = "vector"; + break; + } + + fprintf(stdout, "Memtablerep : %s\n", memtablerep); + fprintf(stdout, "------------------------------------------------\n"); } @@ -1132,6 +1163,31 @@ class StressTest { FLAGS_delete_obsolete_files_period_micros; options.max_manifest_file_size = 1024; options.filter_deletes = FLAGS_filter_deletes; + if ((FLAGS_prefix_size == 0) == (FLAGS_rep_factory == kPrefixHash)) { + fprintf(stderr, + "prefix_size should be non-zero iff memtablerep == prefix_hash\n"); + exit(1); + } + switch (FLAGS_rep_factory) { + case kPrefixHash: + options.memtable_factory.reset( + new PrefixHashRepFactory(NewFixedPrefixTransform(FLAGS_prefix_size)) + ); + break; + case kUnsorted: + options.memtable_factory.reset( + new UnsortedRepFactory() + ); + break; + case kSkipList: + // no need to do anything + break; + case kVectorRep: + options.memtable_factory.reset( + new VectorRepFactory() + ); + break; + } static Random purge_percent(1000); // no benefit from non-determinism here if (purge_percent.Uniform(100) < FLAGS_purge_redundant_percent - 1) { options.purge_redundant_kvs_while_flush = false; @@ -1339,6 +1395,19 @@ int main(int argc, char** argv) { else { fprintf(stdout, "Cannot parse %s\n", argv[i]); } + } else if (strncmp(argv[i], "--memtablerep=", 14) == 0) { + const char* ctype = argv[i] + 14; + if (!strcasecmp(ctype, "skip_list")) + FLAGS_rep_factory = kSkipList; + else if (!strcasecmp(ctype, "prefix_hash")) + FLAGS_rep_factory = kPrefixHash; + else if (!strcasecmp(ctype, "unsorted")) + FLAGS_rep_factory = kUnsorted; + else if (!strcasecmp(ctype, "vector")) + FLAGS_rep_factory = kVectorRep; + else { + fprintf(stdout, "Cannot parse %s\n", argv[i]); + } } else if (sscanf(argv[i], "--disable_seek_compaction=%d%c", &n, &junk) == 1 && (n == 0 || n == 1)) { FLAGS_disable_seek_compaction = n; @@ -1351,6 +1420,9 @@ int main(int argc, char** argv) { } else if (sscanf(argv[i], "--filter_deletes=%d%c", &n, &junk) == 1 && (n == 0 || n == 1)) { FLAGS_filter_deletes = n; + } else if (sscanf(argv[i], "--prefix_size=%d%c", &n, &junk) == 1 && + n >= 0 && n < 2000000000) { + FLAGS_prefix_size = n; } else if (sscanf(argv[i], "--use_merge=%d%c", &n, &junk) == 1 && (n == 0 || n == 1)) { FLAGS_use_merge_put = n; diff --git a/util/coding.cc b/util/coding.cc index 58b2c4751..4aefb2688 100644 --- a/util/coding.cc +++ b/util/coding.cc @@ -193,6 +193,13 @@ bool GetLengthPrefixedSlice(Slice* input, Slice* result) { } } +Slice GetLengthPrefixedSlice(const char* data) { + uint32_t len; + const char* p = data; + p = GetVarint32Ptr(p, p + 5, &len); // +5: we assume "p" is not corrupted + return Slice(p, len); +} + void BitStreamPutInt(char* dst, size_t dstlen, size_t offset, uint32_t bits, uint64_t value) { assert((offset + bits + 7)/8 <= dstlen); diff --git a/util/coding.h b/util/coding.h index 59c61b1b0..336d1efde 100644 --- a/util/coding.h +++ b/util/coding.h @@ -13,7 +13,6 @@ #include #include #include -#include "leveldb/slice.h" #include "port/port.h" namespace leveldb { @@ -34,6 +33,7 @@ extern void PutLengthPrefixedSlice(std::string* dst, const Slice& value); extern bool GetVarint32(Slice* input, uint32_t* value); extern bool GetVarint64(Slice* input, uint64_t* value); extern bool GetLengthPrefixedSlice(Slice* input, Slice* result); +extern Slice GetLengthPrefixedSlice(const char* data); // Pointer-based variants of GetVarint... These either store a value // in *v and return a pointer just past the parsed value, or return diff --git a/util/options.cc b/util/options.cc index 268b04313..18b708bea 100644 --- a/util/options.cc +++ b/util/options.cc @@ -12,7 +12,6 @@ #include "leveldb/env.h" #include "leveldb/filter_policy.h" #include "leveldb/merge_operator.h" -#include "db/skiplistrep.h" namespace leveldb { diff --git a/db/skiplistrep.h b/util/skiplistrep.cc similarity index 62% rename from db/skiplistrep.h rename to util/skiplistrep.cc index d22768f4d..0c3c81987 100644 --- a/db/skiplistrep.h +++ b/util/skiplistrep.cc @@ -1,14 +1,9 @@ -#ifndef STORAGE_LEVELDB_DB_SKIPLISTREP_H_ -#define STORAGE_LEVELDB_DB_SKIPLISTREP_H_ - #include "leveldb/memtablerep.h" #include "db/memtable.h" #include "db/skiplist.h" namespace leveldb { - -class Arena; - +namespace { class SkipListRep : public MemTableRep { SkipList skip_list_; public: @@ -18,16 +13,21 @@ public: // Insert key into the list. // REQUIRES: nothing that compares equal to key is currently in the list. - virtual void Insert(const char* key) { + virtual void Insert(const char* key) override { skip_list_.Insert(key); } // Returns true iff an entry that compares equal to key is in the list. - virtual bool Contains(const char* key) const { + virtual bool Contains(const char* key) const override { return skip_list_.Contains(key); } - virtual ~SkipListRep() { } + virtual size_t ApproximateMemoryUsage() override { + // All memory is allocated through arena; nothing to report here + return 0; + } + + virtual ~SkipListRep() override { } // Iteration over the contents of a skip list class Iterator : public MemTableRep::Iterator { @@ -39,64 +39,61 @@ public: const SkipList* list ) : iter_(list) { } - virtual ~Iterator() { } + virtual ~Iterator() override { } // Returns true iff the iterator is positioned at a valid node. - virtual bool Valid() const { + virtual bool Valid() const override { return iter_.Valid(); } // Returns the key at the current position. // REQUIRES: Valid() - virtual const char* key() const { + virtual const char* key() const override { return iter_.key(); } // Advances to the next position. // REQUIRES: Valid() - virtual void Next() { + virtual void Next() override { iter_.Next(); } // Advances to the previous position. // REQUIRES: Valid() - virtual void Prev() { + virtual void Prev() override { iter_.Prev(); } // Advance to the first entry with a key >= target - virtual void Seek(const char* target) { + virtual void Seek(const char* target) override { iter_.Seek(target); } // Position at the first entry in list. // Final state of iterator is Valid() iff list is not empty. - virtual void SeekToFirst() { + virtual void SeekToFirst() override { iter_.SeekToFirst(); } // Position at the last entry in list. // Final state of iterator is Valid() iff list is not empty. - virtual void SeekToLast() { + virtual void SeekToLast() override { iter_.SeekToLast(); } }; - virtual std::shared_ptr GetIterator() { - return std::shared_ptr( - new SkipListRep::Iterator(&skip_list_) - ); - } -}; + // Unhide default implementations of GetIterator + using MemTableRep::GetIterator; -class SkipListFactory : public MemTableRepFactory { -public: - virtual std::shared_ptr CreateMemTableRep ( - MemTableRep::KeyComparator& compare, Arena* arena) { - return std::shared_ptr(new SkipListRep(compare, arena)); + virtual std::shared_ptr GetIterator() override { + return std::make_shared(&skip_list_); } }; +} +std::shared_ptr SkipListFactory::CreateMemTableRep ( + MemTableRep::KeyComparator& compare, Arena* arena) { + return std::shared_ptr(new SkipListRep(compare, arena)); } -#endif // STORAGE_LEVELDB_DB_SKIPLISTREP_H_ +} // namespace leveldb diff --git a/util/fixed_prefix_transform.cc b/util/slice.cc similarity index 69% rename from util/fixed_prefix_transform.cc rename to util/slice.cc index 2eadb8b0a..a9cdaa5f7 100644 --- a/util/fixed_prefix_transform.cc +++ b/util/slice.cc @@ -3,7 +3,6 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "leveldb/slice_transform.h" - #include "leveldb/slice.h" namespace leveldb { @@ -34,10 +33,36 @@ class FixedPrefixTransform : public SliceTransform { return (dst.size() == prefix_len_); } }; + +class NoopTransform : public SliceTransform { + public: + explicit NoopTransform() { } + + virtual const char* Name() const { + return "rocksdb.Noop"; + } + + virtual Slice Transform(const Slice& src) const { + return src; + } + + virtual bool InDomain(const Slice& src) const { + return true; + } + + virtual bool InRange(const Slice& dst) const { + return true; + } +}; + } const SliceTransform* NewFixedPrefixTransform(size_t prefix_len) { return new FixedPrefixTransform(prefix_len); } +const SliceTransform* NewNoopTransform() { + return new NoopTransform; +} + } // namespace leveldb diff --git a/util/stl_wrappers.h b/util/stl_wrappers.h new file mode 100644 index 000000000..412341978 --- /dev/null +++ b/util/stl_wrappers.h @@ -0,0 +1,49 @@ +#ifndef LEVELDB_UTIL_STL_WRAPPERS_H_ +#define LEVELDB_UTIL_STL_WRAPPERS_H_ + +#include "util/murmurhash.h" +#include "util/coding.h" + +#include "leveldb/memtablerep.h" +#include "leveldb/slice.h" + +namespace leveldb { +namespace stl_wrappers { + class Base { + protected: + const MemTableRep::KeyComparator& compare_; + explicit Base(const MemTableRep::KeyComparator& compare) + : compare_(compare) { } + }; + + struct Compare : private Base { + explicit Compare(const MemTableRep::KeyComparator& compare) + : Base(compare) { } + inline bool operator()(const char* a, const char* b) const { + return compare_(a, b) < 0; + } + }; + + struct Hash { + inline size_t operator()(const char* buf) const { + Slice internal_key = GetLengthPrefixedSlice(buf); + Slice value = + GetLengthPrefixedSlice(internal_key.data() + internal_key.size()); + unsigned int hval = MurmurHash(internal_key.data(), internal_key.size(), + 0); + hval = MurmurHash(value.data(), value.size(), hval); + return hval; + } + }; + + struct KeyEqual : private Base { + explicit KeyEqual(const MemTableRep::KeyComparator& compare) + : Base(compare) { } + inline bool operator()(const char* a, const char* b) const { + return this->compare_(a, b) == 0; + } + }; +} +} + +#endif // LEVELDB_UTIL_STL_WRAPPERS_H_ diff --git a/util/transformrep.cc b/util/transformrep.cc new file mode 100644 index 000000000..10c988427 --- /dev/null +++ b/util/transformrep.cc @@ -0,0 +1,336 @@ +#include +#include +#include +#include +#include + +#include "leveldb/memtablerep.h" +#include "leveldb/arena.h" +#include "leveldb/slice.h" +#include "leveldb/slice_transform.h" +#include "port/port.h" +#include "util/mutexlock.h" +#include "util/murmurhash.h" +#include "util/stl_wrappers.h" + +namespace std { +template <> +struct hash { + size_t operator()(const leveldb::Slice& slice) const { + return MurmurHash(slice.data(), slice.size(), 0); + } +}; +} + +namespace leveldb { +namespace { + +using namespace stl_wrappers; + +class TransformRep : public MemTableRep { + public: + TransformRep(const KeyComparator& compare, Arena* arena, + const SliceTransform* transform, size_t bucket_size, + size_t num_locks); + + virtual void Insert(const char* key) override; + + virtual bool Contains(const char* key) const override; + + virtual size_t ApproximateMemoryUsage() override; + + virtual ~TransformRep() { } + + virtual std::shared_ptr GetIterator() override; + + virtual std::shared_ptr GetIterator( + const Slice& slice) override; + + std::shared_ptr GetTransformIterator( + const Slice& transformed); + + private: + typedef std::set Bucket; + typedef std::unordered_map> BucketMap; + + // Maps slices (which are transformed user keys) to buckets of keys sharing + // the same transform. + BucketMap buckets_; + + // rwlock_ protects access to the buckets_ data structure itself. Each bucket + // has its own read-write lock as well. + mutable port::RWMutex rwlock_; + + // Keep track of approximately how much memory is being used. + size_t memory_usage_ = 0; + + // The user-supplied transform whose domain is the user keys. + const SliceTransform* transform_; + + // Get a bucket from buckets_. If the bucket hasn't been initialized yet, + // initialize it before returning. Must be externally synchronized. + std::shared_ptr& GetBucket(const Slice& transformed); + + port::RWMutex* GetLock(const Slice& transformed) const; + + mutable std::vector locks_; + + const KeyComparator& compare_; + + class Iterator : public MemTableRep::Iterator { + public: + explicit Iterator(std::shared_ptr items); + + virtual ~Iterator() { }; + + // Returns true iff the iterator is positioned at a valid node. + virtual bool Valid() const; + + // Returns the key at the current position. + // REQUIRES: Valid() + virtual const char* key() const; + + // Advances to the next position. + // REQUIRES: Valid() + virtual void Next(); + + // Advances to the previous position. + // REQUIRES: Valid() + virtual void Prev(); + + // Advance to the first entry with a key >= target + virtual void Seek(const char* target); + + // Position at the first entry in collection. + // Final state of iterator is Valid() iff collection is not empty. + virtual void SeekToFirst(); + + // Position at the last entry in collection. + // Final state of iterator is Valid() iff collection is not empty. + virtual void SeekToLast(); + private: + std::shared_ptr items_; + Bucket::const_iterator cit_; + }; + + class EmptyIterator : public MemTableRep::Iterator { + // This is used when there wasn't a bucket. It is cheaper than + // instantiating an empty bucket over which to iterate. + public: + virtual bool Valid() const { + return false; + } + virtual const char* key() const { + assert(false); + return nullptr; + } + virtual void Next() { } + virtual void Prev() { } + virtual void Seek(const char* target) { } + virtual void SeekToFirst() { } + virtual void SeekToLast() { } + static std::shared_ptr GetInstance(); + private: + static std::shared_ptr instance; + EmptyIterator() { } + }; + + class TransformIterator : public Iterator { + public: + explicit TransformIterator(std::shared_ptr items, + port::RWMutex* rwlock); + virtual ~TransformIterator() { } + private: + const ReadLock l_; + }; +}; + +class PrefixHashRep : public TransformRep { + public: + PrefixHashRep(const KeyComparator& compare, Arena* arena, + const SliceTransform* transform, size_t bucket_size, + size_t num_locks) : TransformRep(compare, arena, transform, + bucket_size, num_locks) { } + + virtual std::shared_ptr GetPrefixIterator( + const Slice& prefix) override; +}; + +std::shared_ptr& TransformRep::GetBucket( + const Slice& transformed) { + WriteLock l(&rwlock_); + auto& bucket = buckets_[transformed]; + if (!bucket) { + bucket.reset( + new decltype(buckets_)::mapped_type::element_type(Compare(compare_))); + // To memory_usage_ we add the size of the std::set and the size of the + // std::pair (decltype(buckets_)::value_type) which includes the + // Slice and the std::shared_ptr + memory_usage_ += sizeof(*bucket) + + sizeof(decltype(buckets_)::value_type); + } + return bucket; +} + +port::RWMutex* TransformRep::GetLock(const Slice& transformed) const { + return &locks_[std::hash()(transformed) % locks_.size()]; +} + +TransformRep::TransformRep(const KeyComparator& compare, Arena* arena, + const SliceTransform* transform, size_t bucket_size, + size_t num_locks) + : buckets_(bucket_size), + transform_(transform), + locks_(num_locks), + compare_(compare) { } + +void TransformRep::Insert(const char* key) { + assert(!Contains(key)); + auto transformed = transform_->Transform(UserKey(key)); + auto& bucket = GetBucket(transformed); + WriteLock bl(GetLock(transformed)); + bucket->insert(key); + memory_usage_ += sizeof(key); +} + +bool TransformRep::Contains(const char* key) const { + ReadLock l(&rwlock_); + auto transformed = transform_->Transform(UserKey(key)); + auto bucket = buckets_.find(transformed); + if (bucket == buckets_.end()) { + return false; + } + ReadLock bl(GetLock(transformed)); + return bucket->second->count(key) != 0; +} + +size_t TransformRep::ApproximateMemoryUsage() { + return memory_usage_; +} + +std::shared_ptr + TransformRep::EmptyIterator::GetInstance() { + if (!instance) { + instance.reset(new TransformRep::EmptyIterator); + } + return instance; +} + +TransformRep::Iterator::Iterator(std::shared_ptr items) + : items_(items), + cit_(items_->begin()) { } + +// Returns true iff the iterator is positioned at a valid node. +bool TransformRep::Iterator::Valid() const { + return cit_ != items_->end(); +} + +// Returns the key at the current position. +// REQUIRES: Valid() +const char* TransformRep::Iterator::key() const { + assert(Valid()); + return *cit_; +} + +// Advances to the next position. +// REQUIRES: Valid() +void TransformRep::Iterator::Next() { + assert(Valid()); + if (cit_ == items_->end()) { + return; + } + ++cit_; +} + +// Advances to the previous position. +// REQUIRES: Valid() +void TransformRep::Iterator::Prev() { + assert(Valid()); + if (cit_ == items_->begin()) { + // If you try to go back from the first element, the iterator should be + // invalidated. So we set it to past-the-end. This means that you can + // treat the container circularly. + cit_ = items_->end(); + } else { + --cit_; + } +} + +// Advance to the first entry with a key >= target +void TransformRep::Iterator::Seek(const char* target) { + cit_ = items_->lower_bound(target); +} + +// Position at the first entry in collection. +// Final state of iterator is Valid() iff collection is not empty. +void TransformRep::Iterator::SeekToFirst() { + cit_ = items_->begin(); +} + +void TransformRep::Iterator::SeekToLast() { + cit_ = items_->end(); + if (items_->size() != 0) { + --cit_; + } +} + +TransformRep::TransformIterator::TransformIterator( + std::shared_ptr items, port::RWMutex* rwlock) + : Iterator(items), l_(rwlock) { } + +std::shared_ptr TransformRep::GetIterator() { + auto items = std::make_shared(Compare(compare_)); + // Hold read locks on all locks + ReadLock l(&rwlock_); + std::for_each(locks_.begin(), locks_.end(), [] (port::RWMutex& lock) { + lock.ReadLock(); + }); + for (auto& bucket : buckets_) { + items->insert(bucket.second->begin(), bucket.second->end()); + } + std::for_each(locks_.begin(), locks_.end(), [] (port::RWMutex& lock) { + lock.Unlock(); + }); + return std::make_shared(std::move(items)); +} + +std::shared_ptr TransformRep::GetTransformIterator( + const Slice& transformed) { + ReadLock l(&rwlock_); + auto bucket = buckets_.find(transformed); + if (bucket == buckets_.end()) { + return EmptyIterator::GetInstance(); + } + return std::make_shared(bucket->second, + GetLock(transformed)); +} + +std::shared_ptr TransformRep::GetIterator( + const Slice& slice) { + auto transformed = transform_->Transform(slice); + return GetTransformIterator(transformed); +} + +std::shared_ptr + TransformRep::EmptyIterator::instance; + +} // anon namespace + +std::shared_ptr TransformRepFactory::CreateMemTableRep( + MemTableRep::KeyComparator& compare, Arena* arena) { + return std::make_shared(compare, arena, transform_, + bucket_count_, num_locks_); +} + +std::shared_ptr PrefixHashRepFactory::CreateMemTableRep( + MemTableRep::KeyComparator& compare, Arena* arena) { + return std::make_shared(compare, arena, transform_, + bucket_count_, num_locks_); +} + +std::shared_ptr PrefixHashRep::GetPrefixIterator( + const Slice& prefix) { + return TransformRep::GetTransformIterator(prefix); +} + +} // namespace leveldb diff --git a/util/vectorrep.cc b/util/vectorrep.cc new file mode 100644 index 000000000..c38ffad77 --- /dev/null +++ b/util/vectorrep.cc @@ -0,0 +1,215 @@ +#include "leveldb/memtablerep.h" + +#include +#include +#include +#include +#include + +#include "leveldb/arena.h" +#include "port/port.h" +#include "util/mutexlock.h" +#include "util/stl_wrappers.h" + +namespace leveldb { +namespace { + +using namespace stl_wrappers; + +class VectorRep : public MemTableRep { + public: + VectorRep(const KeyComparator& compare, Arena* arena, size_t count); + + // Insert key into the collection. (The caller will pack key and value into a + // single buffer and pass that in as the parameter to Insert) + // REQUIRES: nothing that compares equal to key is currently in the + // collection. + virtual void Insert(const char* key) override; + + // Returns true iff an entry that compares equal to key is in the collection. + virtual bool Contains(const char* key) const override; + + virtual void MarkReadOnly() override; + + virtual size_t ApproximateMemoryUsage() override; + + virtual ~VectorRep() override { } + + class Iterator : public MemTableRep::Iterator { + std::shared_ptr> bucket_; + typename std::vector::const_iterator cit_; + const KeyComparator& compare_; + public: + explicit Iterator(std::shared_ptr> bucket, + const KeyComparator& compare); + + // Initialize an iterator over the specified collection. + // The returned iterator is not valid. + // explicit Iterator(const MemTableRep* collection); + virtual ~Iterator() override { }; + + // Returns true iff the iterator is positioned at a valid node. + virtual bool Valid() const override; + + // Returns the key at the current position. + // REQUIRES: Valid() + virtual const char* key() const override; + + // Advances to the next position. + // REQUIRES: Valid() + virtual void Next() override; + + // Advances to the previous position. + // REQUIRES: Valid() + virtual void Prev() override; + + // Advance to the first entry with a key >= target + virtual void Seek(const char* target) override; + + // Position at the first entry in collection. + // Final state of iterator is Valid() iff collection is not empty. + virtual void SeekToFirst() override; + + // Position at the last entry in collection. + // Final state of iterator is Valid() iff collection is not empty. + virtual void SeekToLast() override; + }; + + // Unhide default implementations of GetIterator() + using MemTableRep::GetIterator; + + // Return an iterator over the keys in this representation. + virtual std::shared_ptr GetIterator() override; + + private: + typedef std::vector Bucket; + std::shared_ptr bucket_; + mutable port::RWMutex rwlock_; + bool immutable_ = false; + bool sorted_ = false; + const KeyComparator& compare_; +}; + +void VectorRep::Insert(const char* key) { + assert(!Contains(key)); + WriteLock l(&rwlock_); + assert(!immutable_); + bucket_->push_back(key); +} + +// Returns true iff an entry that compares equal to key is in the collection. +bool VectorRep::Contains(const char* key) const { + ReadLock l(&rwlock_); + return std::find(bucket_->begin(), bucket_->end(), key) != bucket_->end(); +} + +void VectorRep::MarkReadOnly() { + WriteLock l(&rwlock_); + immutable_ = true; +} + +size_t VectorRep::ApproximateMemoryUsage() { + return + sizeof(bucket_) + sizeof(*bucket_) + + bucket_->size() * + sizeof( + std::remove_reference::type::value_type + ); +} + +VectorRep::VectorRep(const KeyComparator& compare, Arena* arena, size_t count) + : bucket_(new Bucket(count)), + compare_(compare) { } + +VectorRep::Iterator::Iterator(std::shared_ptr> bucket, + const KeyComparator& compare) +: bucket_(bucket), + cit_(bucket_->begin()), + compare_(compare) { } + +// Returns true iff the iterator is positioned at a valid node. +bool VectorRep::Iterator::Valid() const { + return cit_ != bucket_->end(); +} + +// Returns the key at the current position. +// REQUIRES: Valid() +const char* VectorRep::Iterator::key() const { + assert(Valid()); + return *cit_; +} + +// Advances to the next position. +// REQUIRES: Valid() +void VectorRep::Iterator::Next() { + assert(Valid()); + if (cit_ == bucket_->end()) { + return; + } + ++cit_; +} + +// Advances to the previous position. +// REQUIRES: Valid() +void VectorRep::Iterator::Prev() { + assert(Valid()); + if (cit_ == bucket_->begin()) { + // If you try to go back from the first element, the iterator should be + // invalidated. So we set it to past-the-end. This means that you can + // treat the container circularly. + cit_ = bucket_->end(); + } else { + --cit_; + } +} + +// Advance to the first entry with a key >= target +void VectorRep::Iterator::Seek(const char* target) { + // Do binary search to find first value not less than the target + cit_ = std::equal_range(bucket_->begin(), + bucket_->end(), + target, + [this] (const char* a, const char* b) { + return compare_(a, b) < 0; + }).first; +} + +// Position at the first entry in collection. +// Final state of iterator is Valid() iff collection is not empty. +void VectorRep::Iterator::SeekToFirst() { + cit_ = bucket_->begin(); +} + +// Position at the last entry in collection. +// Final state of iterator is Valid() iff collection is not empty. +void VectorRep::Iterator::SeekToLast() { + cit_ = bucket_->end(); + if (bucket_->size() != 0) { + --cit_; + } +} + +std::shared_ptr VectorRep::GetIterator() { + std::shared_ptr tmp; + ReadLock l(&rwlock_); + if (immutable_) { + rwlock_.Unlock(); + rwlock_.WriteLock(); + tmp = bucket_; + if (!sorted_) { + std::sort(tmp->begin(), tmp->end(), Compare(compare_)); + sorted_ = true; + } + } else { + tmp.reset(new Bucket(*bucket_)); // make a copy + std::sort(tmp->begin(), tmp->end(), Compare(compare_)); + } + return std::make_shared(tmp, compare_); +} +} // anon namespace + +std::shared_ptr VectorRepFactory::CreateMemTableRep( + MemTableRep::KeyComparator& compare, Arena* arena) { + return std::make_shared(compare, arena, count_); +} +} // namespace leveldb