From 534e50de716f1a5c4604f11f77aefaee9bbe26c2 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 5 Dec 2018 22:41:04 -0800 Subject: [PATCH 001/128] Added counters for page writes, sets, and commits. --- fdbserver/VersionedBTree.actor.cpp | 37 ++++++++++++++++++++++++++++-- fdbserver/storageserver.actor.cpp | 3 +++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index c092ab3ba5..499d4a14fc 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -447,6 +447,29 @@ public: static Key beginKey; static Key endKey; + struct Counts { + Counts() { + memset(this, 0, sizeof(Counts)); + } + + void clear() { + *this = Counts(); + } + + int64_t pageWrites; + int64_t blockWrites; + int64_t sets; + int64_t clears; + int64_t commits; + + std::string toString() const { + std::string s = format("sets=%lld clears=%lld commits=%lld pages=%lld blocks=%lld\n", sets, clears, commits, pageWrites, blockWrites); + return s; + } + }; + + Counts counts; + // All async opts on the btree are based on pager reads, writes, and commits, so // we can mostly forward these next few functions to the pager virtual Future getError() { @@ -484,6 +507,7 @@ public: // A write is considered part of (a change leading to) the version determined by the previous call to setWriteVersion() // A write shall not become durable until the following call to commit() begins, and shall be durable once the following call to commit() returns virtual void set(KeyValueRef keyValue) { + ++counts.sets; SingleKeyMutationsByVersion &changes = insertMutationBoundary(keyValue.key)->second.startKeyMutations; // Add the set if the changes set is empty or the last entry isn't a set to exactly the same value @@ -492,6 +516,7 @@ public: } } virtual void clear(KeyRangeRef range) { + ++counts.clears; MutationBufferT::iterator iBegin = insertMutationBoundary(range.begin); MutationBufferT::iterator iEnd = insertMutationBoundary(range.end); @@ -818,6 +843,8 @@ private: debug_printf("%p: writePages(): Writing %lu replacement pages for %d at version %lld\n", actor_debug, pages.size(), originalID, version); for(int i=0; iextensionPages[e] = eid; // If replacing the primary page below (version == 0) then pass the primary page's ID as the reference page ID m_pager->writePage(eid, extPages[e], version, (version == 0) ? id : invalidLogicalPageID); + ++counts.blockWrites; } debug_printf("%p: writePages(): Writing primary page op=write id=%u @%lld (+%lu extension pages)\n", actor_debug, id, version, extPages.size()); @@ -842,6 +870,7 @@ private: else { debug_printf("%p: writePages(): Writing normal page op=write id=%u @%lld\n", actor_debug, id, version); writePage(id, pages[i].firstPage, version, pages[i].lowerBound, (i == pages.size() - 1) ? upperBound : pages[i + 1].lowerBound); + ++counts.blockWrites; } } @@ -1330,6 +1359,7 @@ private: self->m_mutationBuffers.erase(self->m_mutationBuffers.begin()); self->m_lastCommittedVersion = writeVersion; + ++self->counts.commits; committed.send(Void()); return Void(); @@ -2428,15 +2458,18 @@ TEST_CASE("!/redwood/performance/set") { ++records; } - if(g_random->random01() < (1.0 / 300)) { + if(g_random->random01() < (1.0 / 1000)) { wait(commit); commit = btree->commit(); + wait(commit); double elapsed = now() - startTime; - printf("Committed (cumulative) %lld bytes in %d records in %f seconds, %.2f MB/s\n", kvBytes, records, elapsed, kvBytes / elapsed / 1e6); + printf("Committed (cumulative) %lld bytes in %d records in %f seconds, %.2f MB/s %s\n", kvBytes, records, elapsed, kvBytes / elapsed / 1e6, btree->counts.toString().c_str()); } } wait(btree->commit()); + double elapsed = now() - startTime; + printf("Committed (cumulative) %lld bytes in %d records in %f seconds, %.2f MB/s %s\n", kvBytes, records, elapsed, kvBytes / elapsed / 1e6, btree->counts.toString().c_str()); Future closedFuture = btree->onClosed(); btree->close(); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index eeec67403d..2e1ca7da94 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -2656,6 +2656,8 @@ ACTOR Future updateStorage(StorageServer* data) { state Version newOldestVersion = data->storageVersion(); state Version desiredVersion = data->desiredOldestVersion.get(); state int64_t bytesLeft = SERVER_KNOBS->STORAGE_COMMIT_BYTES; + + // Write mutations to storage until we reach the desiredVersion or have written too much (bytesleft) loop { state bool done = data->storage.makeVersionMutationsDurable(newOldestVersion, desiredVersion, bytesLeft); // We want to forget things from these data structures atomically with changing oldestVersion (and "before", since oldestVersion.set() may trigger waiting actors) @@ -2667,6 +2669,7 @@ ACTOR Future updateStorage(StorageServer* data) { if (done) break; } + // Set the new durable version as part of the outstanding change set, before commit if (startOldestVersion != newOldestVersion) data->storage.makeVersionDurable( newOldestVersion ); From 6660ccd936d744381c2e88dd9c7fcef133517f91 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Thu, 21 Feb 2019 02:46:30 -0800 Subject: [PATCH 002/128] Large rewrite/refactor from the page format outward. Replaced the in-page format, PrefixTree, with more general and templated implementation of the same concept, DeltaTree, which will be incrementally modifiable soon. New implementation for in-page cursors to use a per-page arena and share decoded binary tree nodes between cursors in the same page. New implementation for internal record B+Tree cursor which uses reference counted path hops which makes cloning a cursor very cheap. New implementation of external facing B+Tree cursor which makes efficient use of two internal cursors to find user-visible KV pairs and return them without copying keys and values except in the case of a large sharded value. --- fdbserver/DeltaTree.h | 368 +++++ fdbserver/IPager.h | 16 +- fdbserver/IndirectShadowPager.actor.cpp | 14 +- fdbserver/KeyValueStoreMemory.actor.cpp | 2 +- fdbserver/VersionedBTree.actor.cpp | 1796 ++++++++++++++--------- flow/flow.h | 6 +- 6 files changed, 1492 insertions(+), 710 deletions(-) create mode 100644 fdbserver/DeltaTree.h diff --git a/fdbserver/DeltaTree.h b/fdbserver/DeltaTree.h new file mode 100644 index 0000000000..682f7ebb99 --- /dev/null +++ b/fdbserver/DeltaTree.h @@ -0,0 +1,368 @@ +/* + * MutablePrefixTree.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "flow/flow.h" +#include "flow/Arena.h" +#include "fdbclient/FDBTypes.h" +#include "fdbserver/Knobs.h" +#include "fdbserver/PrefixTree.h" +#include + +// Delta Tree is a memory mappable binary tree of T objects such that each node's item is +// stored as a Delta which can reproduce the node's T item given the node's greatest +// lesser ancestor and the node's least greater ancestor. +// +// The Delta type is intended to make use of ordered prefix compression and borrow all +// available prefix bytes from the ancestor T which shares the most prefix bytes with +// the item T being encoded. +// +// T requirements +// +// Must be compatible with Standalone and must implement the following additional methods: +// +// // Writes to d a delta which can create *this from prev and next +// void writeDelta(dT &d, const T &prev, const T &next) +// +// // Compare *this to t, returns < 0 for less than, 0 for equal, > 0 for greater than +// int compare(const T &rhs) +// +// // Returns the delta size needed to make *this from base +// // TODO: Explain contract required for deltaSize to be used to predict final +// // balanced tree size incrementally while adding sorted items to a build set +// int deltaSize(const T &base) +// +// DeltaT requirements +// +// // Returns the size of this dT instance +// int size(); +// +// // Returns the T created by applying the delta to prev or next +// T apply(const T &prev, const T &next, Arena &localStorage) +// +template +struct DeltaTree { + + static int MaximumTreeSize() { + return std::numeric_limits::max(); + }; + +#pragma pack(push,1) + struct Node { + OffsetT leftChildOffset; + OffsetT rightChildOffset; + DeltaT delta[0]; + + Node * rightChild() const { + //printf("Node(%p): leftOffset=%d rightOffset=%d deltaSize=%d\n", this, (int)leftChildOffset, (int)rightChildOffset, (int)delta->size()); + return rightChildOffset == 0 ? nullptr : (Node *)((uint8_t *)delta + rightChildOffset); + } + + Node * leftChild() const { + //printf("Node(%p): leftOffset=%d rightOffset=%d deltaSize=%d\n", this, (int)leftChildOffset, (int)rightChildOffset, (int)delta->size()); + return leftChildOffset == 0 ? nullptr : (Node *)((uint8_t *)delta + leftChildOffset); + } + + int size() const { + return sizeof(Node) + delta->size(); + } + }; +#pragma pack(pop) + +#pragma pack(push,1) + struct { + OffsetT nodeBytes; // Total size of all Nodes including the root + uint8_t initialDepth; // Levels in the tree as of the last rebuild + Node root[0]; + }; +#pragma pack(pop) + + int size() const { + return sizeof(DeltaTree) + nodeBytes; + } + +public: + // Get count of total overhead bytes (everything but the user-formatted Delta) for a tree given size n + static inline int GetTreeOverhead(int n = 0) { + return sizeof(DeltaTree) + (n * sizeof(Node)); + } + + struct DecodedNode { + DecodedNode(Node *raw, const T *prev, const T *next, Arena &arena) + : raw(raw), parent(nullptr), left(nullptr), right(nullptr), prev(prev), next(next), + item(raw->delta->apply(*prev, *next, arena)) + { + //printf("DecodedNode1 raw=%p delta=%s\n", raw, raw->delta->toString().c_str()); + } + + DecodedNode(Node *raw, DecodedNode *parent, bool left, Arena &arena) + : parent(parent), raw(raw), left(nullptr), right(nullptr), + prev(left ? parent->prev : &parent->item), + next(left ? &parent->item : parent->next), + item(raw->delta->apply(*prev, *next, arena)) + { + //printf("DecodedNode2 raw=%p delta=%s\n", raw, raw->delta->toString().c_str()); + } + + Node *raw; + DecodedNode *parent; + DecodedNode *left; + DecodedNode *right; + const T *prev; // greatest ancestor to the left + const T *next; // least ancestor to the right + T item; + + DecodedNode *getRight(Arena &arena) { + if(right == nullptr) { + Node *n = raw->rightChild(); + if(n != nullptr) { + right = new (arena) DecodedNode(n, this, false, arena); + } + } + return right; + } + + DecodedNode *getLeft(Arena &arena) { + if(left == nullptr) { + Node *n = raw->leftChild(); + if(n != nullptr) { + left = new (arena) DecodedNode(n, this, true, arena); + } + } + return left; + } + }; + + struct Cursor; + + // A Reader is used to read a Tree by getting cursors into it. + // Any node decoded by any cursor is placed in cache for use + // by other cursors. + struct Reader : FastAllocated { + Reader(const void *treePtr = nullptr, const T *lowerBound = nullptr, const T *upperBound = nullptr) + : tree((DeltaTree *)treePtr), lower(lowerBound), upper(upperBound) { + + // TODO: Remove these copies into arena and require users of Reader to keep prev and next alive during its lifetime + lower = new(arena) T(arena, *lower); + upper = new(arena) T(arena, *upper); + + root = (tree->nodeBytes == 0) ? nullptr : new (arena) DecodedNode(tree->root, lower, upper, arena); + } + + const T *lowerBound() const { + return lower; + } + + const T *upperBound() const { + return upper; + } + + Arena arena; + DeltaTree *tree; + DecodedNode *root; + const T *lower; + const T *upper; + + Cursor getCursor() { + return Cursor(this); + } + }; + + // Cursor provides a way to seek into a PrefixTree and iterate over its contents + // All Cursors from a Reader share the same decoded node 'cache' (tree of DecodedNodes) + struct Cursor { + Cursor() : reader(nullptr), node(nullptr) { + } + + Cursor(Reader *r) : reader(r), node(reader->root) { + } + + Reader *reader; + DecodedNode *node; + + bool valid() const { + return node != nullptr; + } + + const T & get() const { + return node->item; + } + + const T & getOrUpperBound() const { + return valid() ? node->item : *reader->upperBound(); + } + + // Moves the cursor to the node with the greatest key less than or equal to s. If successful, + // returns true, otherwise returns false and the cursor will be at the node with the next key + // greater than s. + bool seekLessThanOrEqual(const T &s) { + node = nullptr; + DecodedNode *n = reader->root; + + while(n != nullptr) { + int cmp = s.compare(n->item); + + if(cmp == 0) { + node = n; + return true; + } + + if(cmp < 0) { + n = n->getLeft(reader->arena); + } + else { + // n < s so store it in node as a potential result + node = n; + n = n->getRight(reader->arena); + } + } + + return node != nullptr; + } + + bool moveFirst() { + DecodedNode *n = reader->root; + node = n; + while(n != nullptr) { + n = n->getLeft(reader->arena); + if(n != nullptr) + node = n; + } + return node != nullptr; + } + + bool moveLast() { + DecodedNode *n = reader->root; + node = n; + while(n != nullptr) { + n = n->getRight(reader->arena); + if(n != nullptr) + node = n; + } + return node != nullptr; + } + + bool moveNext() { + // Try to go right + DecodedNode *n = node->getRight(reader->arena); + if(n != nullptr) { + // Go left as far as possible + while(n != nullptr) { + node = n; + n = n->getLeft(reader->arena); + } + return true; + } + + // Follow parent links until a greater parent is found + while(node->parent != nullptr) { + bool greaterParent = node->parent->left == node; + node = node->parent; + if(greaterParent) { + return true; + } + } + + node = nullptr; + return false; + } + + bool movePrev() { + // Try to go left + DecodedNode *n = node->getLeft(reader->arena); + if(n != nullptr) { + // Go right as far as possible + while(n != nullptr) { + node = n; + n = n->getRight(reader->arena); + } + return true; + } + + // Follow parent links until a lesser parent is found + while(node->parent != nullptr) { + bool lesserParent = node->parent->right == node; + node = node->parent; + if(lesserParent) { + return true; + } + } + + node = nullptr; + return false; + } + }; + + // Returns number of bytes written + int build(const T *begin, const T *end, const T *prev, const T *next) { + //printf("tree size: %d node size: %d\n", sizeof(DeltaTree), sizeof(Node)); + int count = end - begin; + initialDepth = (uint8_t)log2(count) + 1; + + // The boundary leading to the new page acts as the last time we branched right + if(begin != end) { + nodeBytes = build(*root, begin, end, prev, next); + } + else { + nodeBytes = 0; + } + return size(); + } + +private: + static OffsetT build(Node &root, const T *begin, const T *end, const T *prev, const T *next) { + //printf("build: %s to %s\n", begin->toString().c_str(), (end - 1)->toString().c_str()); + //printf("build: root at %p sizeof(Node) %d delta at %p \n", &root, sizeof(Node), root.delta); + ASSERT(end != begin); + int count = end - begin; + + // Find key to be stored in root + int mid = perfectSubtreeSplitPointCached(count); + const T &item = begin[mid]; + + item.writeDelta(*root.delta, *prev, *next); + //printf("Serialized %s to %p\n", item.toString().c_str(), root.delta); + + // Delta can have variable size, so calculate wptr which is where to write to next + int deltaSize = root.delta->size(); + uint8_t *wptr = (uint8_t *)root.delta + deltaSize; + + // Serialize left child + if(count > 1) { + wptr += build(*(Node *)wptr, begin, begin + mid, prev, &item); + root.leftChildOffset = deltaSize; + } + else { + root.leftChildOffset = 0; + } + + // Serialize right child + if(count > 2) { + root.rightChildOffset = wptr - (uint8_t *)root.delta; + wptr += build(*(Node *)wptr, begin + mid + 1, end, &item, next); + } + else { + root.rightChildOffset = 0; + } + + return wptr - (uint8_t *)&root; + } +}; diff --git a/fdbserver/IPager.h b/fdbserver/IPager.h index 6089089f9f..c78f96349d 100644 --- a/fdbserver/IPager.h +++ b/fdbserver/IPager.h @@ -39,11 +39,18 @@ #define BEACON fprintf(stderr, "%s: %s line %d \n", __FUNCTION__, __FILE__, __LINE__) +#ifndef VALGRIND +#define VALGRIND_MAKE_MEM_UNDEFINED(x, y) +#define VALGRIND_MAKE_MEM_DEFINED(x, y) +#endif + typedef uint32_t LogicalPageID; // uint64_t? static const int invalidLogicalPageID = LogicalPageID(-1); class IPage { public: + IPage() : userData(nullptr) {} + virtual uint8_t const* begin() const = 0; virtual uint8_t* mutate() = 0; @@ -54,10 +61,17 @@ public: return StringRef(begin(), size()); } - virtual ~IPage() {} + virtual ~IPage() { + if(userData != nullptr && userDataDestructor != nullptr) { + userDataDestructor(userData); + } + } virtual void addref() const = 0; virtual void delref() const = 0; + + mutable void *userData; + mutable void (*userDataDestructor)(void *); }; class IPagerSnapshot { diff --git a/fdbserver/IndirectShadowPager.actor.cpp b/fdbserver/IndirectShadowPager.actor.cpp index 6b7d6c2e06..d76b2c289f 100644 --- a/fdbserver/IndirectShadowPager.actor.cpp +++ b/fdbserver/IndirectShadowPager.actor.cpp @@ -41,15 +41,19 @@ bool checksum(IAsyncFile *file, uint8_t *page, int pageSize, LogicalPageID logic pageSize -= IndirectShadowPage::PAGE_OVERHEAD_BYTES; SumType sum; SumType *pSumInPage = (SumType *)(page + pageSize); - // Write sum directly to page or to sum variable based on mode SumType *sumOut = write ? pSumInPage : ∑ sumOut->part1 = physical; - sumOut->part2 = logical; + sumOut->part2 = logical; hashlittle2(page, pageSize, &sumOut->part1, &sumOut->part2); + VALGRIND_MAKE_MEM_DEFINED(sumOut, sizeof(SumType)); debug_printf("checksum %s%s logical %d physical %d size %d checksums page %s calculated %s data at %p %s\n", - write ? "write" : "read", (!write && sum != *pSumInPage) ? " MISMATCH" : "", logical, physical, pageSize, write ? "NA" : pSumInPage->toString().c_str(), sumOut->toString().c_str(), page, "" /*StringRef((uint8_t *)page, pageSize).toHexString().c_str()*/); + write ? "write" : "read", + (!write && sum != *pSumInPage) ? " MISMATCH" : "", + logical, physical, pageSize, + write ? "NA" : pSumInPage->toString().c_str(), + sumOut->toString().c_str(), page, ""); // Verify if not in write mode if(!write && sum != *pSumInPage) { @@ -75,10 +79,6 @@ inline void checksumWrite(IAsyncFile *file, uint8_t *page, int pageSize, Logical IndirectShadowPage::IndirectShadowPage() : fastAllocated(true) { data = (uint8_t*)FastAllocator<4096>::allocate(); -#if VALGRIND - // Prevent valgrind errors caused by writing random unneeded bytes to disk. - memset(data, 0, size()); -#endif } IndirectShadowPage::~IndirectShadowPage() { diff --git a/fdbserver/KeyValueStoreMemory.actor.cpp b/fdbserver/KeyValueStoreMemory.actor.cpp index 94db4e1bf5..78ce449e47 100644 --- a/fdbserver/KeyValueStoreMemory.actor.cpp +++ b/fdbserver/KeyValueStoreMemory.actor.cpp @@ -407,7 +407,7 @@ private: bool ok = count < 1e6; if( !ok ) { - TraceEvent(/*ok ? SevInfo : */SevWarnAlways, "KVSMemCommit_queue", id) + TraceEvent(/*ok ? SevInfo : */SevWarnAlways, "KVSMemCommitQueue", id) .detail("Bytes", total) .detail("Log", log) .detail("Ops", count) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 499d4a14fc..d8dc24ad96 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -31,72 +31,274 @@ #include #include "fdbclient/CommitTransaction.h" #include "fdbserver/IKeyValueStore.h" -#include "fdbserver/PrefixTree.h" +#include "fdbserver/DeltaTree.h" #include #include "flow/actorcompiler.h" -// Convenience method for converting a Standalone to a Ref while adding its arena to another arena. -template inline const Standalone & dependsOn(Arena &arena, const Standalone &s) { - arena.dependsOn(s.arena()); - return s; -} +#define STR(x) LiteralStringRef(x) +struct RedwoodRecordRef { -struct BTreePage { - enum EPageFlags { IS_LEAF = 1}; - -#pragma pack(push,4) - uint8_t flags; - uint16_t count; - uint32_t kvBytes; - uint8_t extensionPageCount; - LogicalPageID extensionPages[0]; +#pragma pack(push,1) + struct ValuePart { + int32_t total; + int32_t start; + }; #pragma pack(pop) - PrefixTree & tree() { - return *(PrefixTree *)(extensionPages + extensionPageCount); + RedwoodRecordRef(KeyRef key = KeyRef(), Version ver = std::numeric_limits::max(), Optional value = {}, Optional part = {}) + : key(key), version(ver), value(value), valuePart(part) + { + ASSERT(!part.present() || value.present()); } - const PrefixTree & tree() const { - return *(const PrefixTree *)(extensionPages + extensionPageCount); + RedwoodRecordRef(Arena &arena, const RedwoodRecordRef &toCopy) { + *this = toCopy; + key = KeyRef(arena, key); + if(value.present()) { + value = ValueRef(arena, toCopy.value.get()); + } + } + + KeyRef key; + Version version; + Optional value; + Optional valuePart; + + int expectedSize() const { + return key.expectedSize() + value.expectedSize() + sizeof(version) + sizeof(valuePart); + } + + bool isMultiPart() const { + return valuePart.present(); + } + + // Generate a kv shard from a complete kv + RedwoodRecordRef split(int start, int len) { + ASSERT(!isMultiPart() && value.present()); + return RedwoodRecordRef(key, version, value.get().substr(start, len), ValuePart({value.get().size(), start})); + } + +#pragma pack(push,1) + struct Delta { + // TODO: Make this actually a delta + enum EFlags {HAS_VALUE = 1, HAS_VALUE_PART = 4}; + + uint8_t flags; + uint16_t keySize; + Version version; + uint8_t bytes[]; + + RedwoodRecordRef apply(const RedwoodRecordRef &prev, const RedwoodRecordRef &next, Arena arena) { + RedwoodRecordRef r; + const uint8_t *rptr = bytes; + r.key = StringRef(rptr, keySize); + rptr += keySize; + r.version = version; + if(flags & HAS_VALUE) { + uint16_t valueSize = *(uint16_t *)rptr; + rptr += 2; + r.value = StringRef(rptr, valueSize); + rptr += valueSize; + if(flags & HAS_VALUE_PART) { + r.valuePart = *(ValuePart *)rptr; + } + } + return r; + } + + int size() const { + int s = sizeof(Delta) + keySize; + if(flags & HAS_VALUE) { + s += 2; + s += *(uint16_t *)(bytes + keySize); + if(flags & HAS_VALUE_PART) { + s += sizeof(ValuePart); + } + } + return s; + } + + std::string toString() const { + return format("DELTA{ %s | %s }", + StringRef((const uint8_t *)this, sizeof(Delta)).toHexString().c_str(), + StringRef(bytes, size() - sizeof(Delta)).toHexString().c_str() + ); + } + }; +#pragma pack(pop) + + int compare(const RedwoodRecordRef &rhs) const { + //printf("compare %s to %s\n", toString().c_str(), rhs.toString().c_str()); + int cmp = key.compare(rhs.key); + if(cmp == 0) { + cmp = version - rhs.version; + if(cmp == 0) { + // Absent value is greater than present (for reasons) + cmp = (value.present() ? 0 : 1) - (rhs.value.present() ? 0 : 1); + if(cmp == 0) { + // Chunked is greater than whole + cmp = (valuePart.present() ? 1 : 0) - (rhs.valuePart.present() ? 1 : 0); + if(cmp == 0 && valuePart.present()) { + // Larger total size is greater + cmp = valuePart.get().total - rhs.valuePart.get().total; + if(cmp == 0) { + // Order by start + cmp = valuePart.get().start - rhs.valuePart.get().start; + } + } + } + } + } + return cmp; + } + + bool operator==(const RedwoodRecordRef &rhs) const { + return compare(rhs) == 0; + } + + bool operator<(const RedwoodRecordRef &rhs) const { + return compare(rhs) < 0; + } + + bool operator>(const RedwoodRecordRef &rhs) const { + return compare(rhs) > 0; + } + + bool operator<=(const RedwoodRecordRef &rhs) const { + return compare(rhs) <= 0; + } + + bool operator>=(const RedwoodRecordRef &rhs) const { + return compare(rhs) >= 0; + } + + int deltaSize(const RedwoodRecordRef &base) const { + int s = sizeof(Delta) + key.size(); + if(value.present()) { + s += 2; + s += value.get().size(); + if(valuePart.present()) { + s += sizeof(ValuePart); + } + } + return s; + } + + void writeDelta(Delta &d, const RedwoodRecordRef &prev, const RedwoodRecordRef &next) const { + d.flags = value.present() ? Delta::EFlags::HAS_VALUE : 0; + if(valuePart.present()) + d.flags |= Delta::EFlags::HAS_VALUE_PART; + d.keySize = key.size(); + d.version = version; + uint8_t *wptr = d.bytes; + memcpy(wptr, key.begin(), key.size()); + wptr += key.size(); + if(value.present()) { + *(uint16_t *)wptr = value.get().size(); + wptr += 2; + memcpy(wptr, value.get().begin(), value.get().size()); + wptr += value.get().size(); + if(valuePart.present()) { + *(ValuePart *)wptr = valuePart.get(); + } + } + } + + static std::string kvformat(StringRef s, int hexLimit = -1) { + bool hex = false; + + for(auto c : s) { + if(!isprint(c)) { + hex = true; + break; + } + } + + return hex ? s.toHexString(hexLimit) : s.toString(); + } + + std::string toString(int hexLimit = 15) const { + std::string r; + r += format("'%s' @%lld ", kvformat(key, hexLimit).c_str(), version); + if(valuePart.present()) { + r += format("[%d/%d] ", valuePart.get().start, valuePart.get().total); + } + if(value.present()) { + r += format("-> '%s'", kvformat(value.get(), hexLimit).c_str()); + } + else { + r += "-> "; + } + return r; + } +}; + +struct BTreePage { + + enum EPageFlags { IS_LEAF = 1}; + + typedef DeltaTree BinaryTree; + +#pragma pack(push,1) + struct { + uint8_t flags; + uint16_t count; + uint32_t kvBytes; + uint8_t extensionPageCount; + LogicalPageID extensionPages[0]; + }; +#pragma pack(pop) + + int size() const { + const BinaryTree *t = &tree(); + return (uint8_t *)t - (uint8_t *)this + t->size(); + } + + bool isLeaf() const { + return flags & IS_LEAF; + } + + BinaryTree & tree() { + return *(BinaryTree *)(extensionPages + extensionPageCount); + } + + const BinaryTree & tree() const { + return *(const BinaryTree *)(extensionPages + extensionPageCount); } static inline int GetHeaderSize(int extensionPages = 0) { return sizeof(BTreePage) + extensionPages + sizeof(LogicalPageID); } - std::string toString(bool write, LogicalPageID id, Version ver, StringRef lowerBoundKey, StringRef upperBoundKey) const { + std::string toString(bool write, LogicalPageID id, Version ver, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) const { std::string r; - r += format("BTreePage op=%s id=%d ver=%lld ptr=%p flags=0x%X count=%d kvBytes=%d\nlowerBoundKey='%s'\nupperBoundKey='%s'", - write ? "write" : "read", id, ver, this, (int)flags, (int)count, (int)kvBytes, - lowerBoundKey.toHexString(20).c_str(), upperBoundKey.toHexString(20).c_str()); + r += format("BTreePage op=%s id=%d ver=%lld ptr=%p flags=0x%X count=%d kvBytes=%d extPages=%d\n lowerBound: %s\n upperBound: %s\n", + write ? "write" : "read", id, ver, this, (int)flags, (int)count, (int)kvBytes, (int)extensionPageCount, + lowerBound->toString().c_str(), upperBound->toString().c_str()); try { if(count > 0) { - PrefixTree::Cursor c = tree().getCursor(lowerBoundKey, upperBoundKey); + // This doesn't use the cached reader for the page but it is only for debugging purposes + BinaryTree::Reader reader(&tree(), lowerBound, upperBound); + BinaryTree::Cursor c = reader.getCursor(); + c.moveFirst(); ASSERT(c.valid()); do { - r += "\n "; - Tuple t; - try { - t = Tuple::unpack(c.getKeyRef()); - for(int i = 0; i < t.size(); ++i) { - if(i != 0) - r += ","; - if(t.getType(i) == Tuple::ElementType::BYTES) - r += format("'%s'", t.getString(i).printable().c_str()); - if(t.getType(i) == Tuple::ElementType::INT) - r += format("%lld", t.getInt(i, true)); - } - } catch(Error &e) { + r += " "; + if(!(flags & IS_LEAF)) { + RedwoodRecordRef rec = c.get(); + ASSERT(rec.value.present() && rec.value.get().size() == sizeof(uint32_t)); + uint32_t id = *(const uint32_t *)rec.value.get().begin(); + std::string val = format("[Page id=%u]", id); + rec.value = val; + r += rec.toString(); + } + else { + r += c.get().toString(); } - r += format("['%s']", c.getKeyRef().toHexString(20).c_str()); - r += " -> "; - if(flags && IS_LEAF) - r += format("'%s'", c.getValueRef().toHexString(20).c_str()); - else - r += format("Page id=%u", *(const uint32_t *)c.getValueRef().begin()); + r += "\n"; } while(c.moveNext()); } @@ -111,26 +313,35 @@ struct BTreePage { }; static void writeEmptyPage(Reference page, uint8_t newFlags, int pageSize) { + VALGRIND_MAKE_MEM_DEFINED(page->begin(), page->size()); BTreePage *btpage = (BTreePage *)page->begin(); btpage->flags = newFlags; btpage->kvBytes = 0; btpage->count = 0; btpage->extensionPageCount = 0; - btpage->tree().build(nullptr, nullptr, StringRef(), StringRef()); + btpage->tree().build(nullptr, nullptr, nullptr, nullptr); +} + +BTreePage::BinaryTree::Reader * getReader(Reference page) { + return (BTreePage::BinaryTree::Reader *)page->userData; } struct BoundaryAndPage { - Key lowerBound; + Standalone lowerBound; // Only firstPage or multiPage will be in use at once Reference firstPage; std::vector> extPages; }; // Returns a std::vector of pairs of lower boundary key indices within kvPairs and encoded pages. +// TODO: Refactor this as an accumulator you add sorted keys to which makes pages. template -static std::vector buildPages(bool minimalBoundaries, StringRef lowerBound, StringRef upperBound, std::vector entries, uint8_t newFlags, Allocator const &newBlockFn, int usableBlockSize) { +static std::vector buildPages(bool minimalBoundaries, const RedwoodRecordRef &lowerBound, const RedwoodRecordRef &upperBound, std::vector entries, uint8_t newFlags, Allocator const &newBlockFn, int usableBlockSize) { + // TODO: Figure out how to do minimal boundaries with RedwoodRecordRef + minimalBoundaries = false; + // This is how much space for the binary tree exists in the page, after the header - int pageSize = usableBlockSize - (BTreePage::GetHeaderSize() + PrefixTree::GetHeaderSize()); + int pageSize = usableBlockSize - BTreePage::GetHeaderSize(); // Each new block adds (usableBlockSize - sizeof(LogicalPageID)) more net usable space *for the binary tree* to pageSize. int netTreeBlockSize = usableBlockSize - sizeof(LogicalPageID); @@ -138,16 +349,15 @@ static std::vector buildPages(bool minimalBoundaries, StringRef int blockCount = 1; std::vector pages; - // TODO: Move all of this abstraction breaking stuff into PrefixTree in the form of a helper function or class. - int kvBytes = 0; // User key/value bytes in page - int compressedBytes = 0; // Conservative estimate of size of compressed page. TODO: Make this exactly right if possible + int kvBytes = 0; + int compressedBytes = BTreePage::BinaryTree::GetTreeOverhead(); int start = 0; int i = 0; const int iEnd = entries.size(); // Lower bound of the page being added to - Key pageLowerBound = lowerBound; - Key pageUpperBound; + RedwoodRecordRef pageLowerBound = lowerBound; + RedwoodRecordRef pageUpperBound; while(i <= iEnd) { bool end = i == iEnd; @@ -158,17 +368,17 @@ static std::vector buildPages(bool minimalBoundaries, StringRef pageUpperBound = upperBound; } else { - // Common prefix with previous record - const PrefixTree::EntryRef &entry = entries[i]; - int prefixLen = commonPrefixLength(entry.key, (i == start) ? pageLowerBound : entries[i - 1].key); + // Get delta from previous record + const RedwoodRecordRef &entry = entries[i]; + int deltaSize = entry.deltaSize((i == start) ? pageLowerBound : entries[i - 1]); int keySize = entry.key.size(); - int valueSize = entry.value.size(); + int valueSize = entry.value.present() ? entry.value.get().size() : 0; - int spaceNeeded = valueSize + keySize - prefixLen + PrefixTree::Node::getMaxOverhead(i, entry.key.size(), entry.value.size()); + int spaceNeeded = sizeof(BTreePage::BinaryTree::Node) + deltaSize; - debug_printf("Trying to add record %d of %lu (i=%d) klen %d vlen %d prefixLen %d spaceNeeded %d usedSoFar %d/%d '%s'\n", - i + 1, entries.size(), i, keySize, valueSize, prefixLen, - spaceNeeded, compressedBytes, pageSize, entry.key.toHexString(15).c_str()); + debug_printf("Trying to add record %3d of %3lu (i=%3d) klen %4d vlen %3d deltaSize %4d spaceNeeded %4d compressed %4d / page %4d bytes %s\n", + i + 1, entries.size(), i, keySize, valueSize, deltaSize, + spaceNeeded, compressedBytes, pageSize, entry.toString().c_str()); int spaceAvailable = pageSize - compressedBytes; @@ -186,7 +396,7 @@ static std::vector buildPages(bool minimalBoundaries, StringRef // Figure out how many additional whole or partial blocks are needed int newBlocks = 1 + (spaceNeeded - spaceAvailable - 1) / netTreeBlockSize; int newPageSize = pageSize + (newBlocks * netTreeBlockSize); - if(newPageSize <= PrefixTree::MaximumTreeSize()) { + if(newPageSize <= BTreePage::BinaryTree::MaximumTreeSize()) { blockCount += newBlocks; pageSize = newPageSize; fits = true; @@ -196,13 +406,13 @@ static std::vector buildPages(bool minimalBoundaries, StringRef // Flush page if(minimalBoundaries) { // Note that prefixLen is guaranteed to be < entry.key.size() because entries are in increasing order and cannot repeat. - int len = prefixLen + 1; - if(entry.key[prefixLen] == 0) - len = std::min(len + 1, entry.key.size()); - pageUpperBound = entry.key.substr(0, len); +// int len = prefixLen + 1; +// if(entry.key[prefixLen] == 0) +// len = std::min(len + 1, entry.key.size()); +// pageUpperBound = entry.key.substr(0, len); } else { - pageUpperBound = entry.key; + pageUpperBound = entry; } } } @@ -221,29 +431,35 @@ static std::vector buildPages(bool minimalBoundaries, StringRef if(flush) { end = i == iEnd; // i could have been moved above int count = i - start; - debug_printf("Flushing page start=%d i=%d\nlower='%s'\nupper='%s'\n", start, i, pageLowerBound.toHexString(20).c_str(), pageUpperBound.toHexString(20).c_str()); - ASSERT(pageLowerBound <= pageUpperBound); + debug_printf("Flushing page start=%d i=%d\nlower: %s\nupper: %s\n", start, i, pageLowerBound.toString().c_str(), pageUpperBound.toString().c_str()); +#if REDWOOD_DEBUG for(int j = start; j < i; ++j) { - debug_printf(" %d: %s -> %s\n", j, entries[j].key.toHexString(15).c_str(), entries[j].value.toHexString(15).c_str()); + debug_printf(" %3d: %s\n", j, entries[j].toString().c_str()); + if(j > start) { + ASSERT(entries[j] > entries[j - 1]); + } } +#endif + ASSERT(pageLowerBound <= pageUpperBound); union { BTreePage *btPage; uint8_t *btPageMem; }; + int allocatedSize; if(blockCount == 1) { Reference page = newBlockFn(); + VALGRIND_MAKE_MEM_DEFINED(page->begin(), page->size()); btPageMem = page->mutate(); - pages.push_back({std::move(pageLowerBound), std::move(page)}); + allocatedSize = page->size(); + pages.push_back({pageLowerBound, page}); } else { ASSERT(blockCount > 1); - btPageMem = new uint8_t[usableBlockSize * blockCount]; -#if VALGRIND - // Prevent valgrind errors caused by writing random unneeded bytes to disk. - memset(btPageMem, 0, usableBlockSize * blockCount); -#endif + allocatedSize = usableBlockSize * blockCount; + btPageMem = new uint8_t[allocatedSize]; + VALGRIND_MAKE_MEM_DEFINED(btPageMem, allocatedSize); } btPage->flags = newFlags; @@ -251,7 +467,7 @@ static std::vector buildPages(bool minimalBoundaries, StringRef btPage->count = i - start; btPage->extensionPageCount = blockCount - 1; - int written = btPage->tree().build(&entries[start], &entries[i], pageLowerBound, pageUpperBound); + int written = btPage->tree().build(&entries[start], &entries[i], &pageLowerBound, &pageUpperBound); if(written > pageSize) { fprintf(stderr, "ERROR: Wrote %d bytes to %d byte page (%d blocks). recs %d kvBytes %d compressed %d\n", written, pageSize, blockCount, i - start, kvBytes, compressedBytes); ASSERT(false); @@ -259,6 +475,8 @@ static std::vector buildPages(bool minimalBoundaries, StringRef if(blockCount != 1) { Reference page = newBlockFn(); + VALGRIND_MAKE_MEM_DEFINED(page->begin(), page->size()); + const uint8_t *rptr = btPageMem; memcpy(page->mutate(), rptr, usableBlockSize); rptr += usableBlockSize; @@ -266,6 +484,8 @@ static std::vector buildPages(bool minimalBoundaries, StringRef std::vector> extPages; for(int b = 1; b < blockCount; ++b) { Reference extPage = newBlockFn(); + VALGRIND_MAKE_MEM_DEFINED(page->begin(), page->size()); + //debug_printf("block %d write offset %d\n", b, firstBlockSize + (b - 1) * usableBlockSize); memcpy(extPage->mutate(), rptr, usableBlockSize); rptr += usableBlockSize; @@ -280,7 +500,7 @@ static std::vector buildPages(bool minimalBoundaries, StringRef break; start = i; kvBytes = 0; - compressedBytes = 0; + compressedBytes = BTreePage::BinaryTree::GetTreeOverhead(); pageLowerBound = pageUpperBound; } } @@ -289,163 +509,14 @@ static std::vector buildPages(bool minimalBoundaries, StringRef return pages; } -// Internal key/value records represent either a cleared key at a version or a shard of a value of a key at a version. -// When constructing and packing these it is assumed that the key and value memory is being held elsewhere. -struct KeyVersionValueRef { - KeyVersionValueRef() : version(invalidVersion) {} - // Cleared key at version - KeyVersionValueRef(KeyRef key, Version ver, Optional val = {}) - : key(key), version(ver), value(val), valueIndex(0) - { - if(value.present()) - valueTotalSize = value.get().size(); - } - - KeyVersionValueRef(Arena &a, const KeyVersionValueRef &toCopy) { - key = KeyRef(a, toCopy.key); - version = toCopy.version; - if(toCopy.value.present()) { - value = ValueRef(a, toCopy.value.get()); - } - valueTotalSize = toCopy.valueTotalSize; - valueIndex = toCopy.valueIndex; - } - - static inline Key searchKey(StringRef key, Version ver) { - Tuple t; - t.append(key); - t.append(ver); - Standalone> packed = t.getData(); - packed.append(packed.arena(), (const uint8_t *)"\xff", 1); - return Key(KeyRef(packed.begin(), packed.size()), packed.arena()); - } - - KeyRef key; - Version version; - int64_t valueTotalSize; // Total size of value, including all other KVV parts if multipart - int64_t valueIndex; // Index within reconstituted value of this part - Optional value; - - // Result undefined if value is not present - bool isMultiPart() const { return value.get().size() != valueTotalSize; } - bool valid() const { return version != invalidVersion; } - - // Generate a kv shard from a complete kv - KeyVersionValueRef split(int start, int len) { - ASSERT(value.present()); - KeyVersionValueRef r(key, version); - r.value = value.get().substr(start, len); - r.valueIndex = start; - r.valueTotalSize = valueTotalSize; - return r; - } - - // Encode the record for writing to a btree page. - // If copyValue is false, the value is not copied into the returned arena. - // - // Encoded forms: - // userKey, version - the value is present and complete (which includes an empty value) - // userKey, version, valueSize=0 - the key was deleted as of this version - // userKey, version, valueSize>=0, valuePart - the value is present and spans multiple records - inline PrefixTree::Entry pack(bool copyValue = true) const { - Tuple t; - t.append(key); - t.append(version); - - if(!value.present()) { - t.append(0); - } - else { - if(isMultiPart()) { - t.append(valueTotalSize); - t.append(valueIndex); - } - } - - Key k = t.getDataAsStandalone(); - ValueRef v; - if(value.present()) { - v = copyValue ? StringRef(k.arena(), value.get()) : value.get(); - } - - return PrefixTree::Entry({k, v}, k.arena()); - } - - // Supports partial/incomplete encoded sequences. - // Unpack an encoded key/value pair. - // Both key and value will be in the returned arena unless copyValue is false in which case - // the value will not be copied to the arena. - static Standalone unpack(KeyValueRef kv, bool copyValue = true) { - //debug_printf("Unpacking: '%s' -> '%s' \n", kv.key.toHexString(15).c_str(), kv.value.toHexString(15).c_str()); - Standalone result; - if(kv.key.size() != 0) { -#if REDWOOD_DEBUG - try { Tuple t = Tuple::unpack(kv.key); } catch(Error &e) { debug_printf("UNPACK FAIL %s %s\n", kv.key.toHexString(20).c_str(), platform::get_backtrace().c_str()); } -#endif - Tuple k = Tuple::unpack(kv.key); - int s = k.size(); - switch(s) { - case 4: - // Value shard - result.valueTotalSize = k.getInt(2); - result.valueIndex = k.getInt(3, true); - result.value = kv.value; - break; - case 3: - // Deleted or Complete value - result.valueIndex = 0; - result.valueTotalSize = k.getInt(2, true); - // If not a clear, set the value, otherwise it remains non-present - if(result.valueTotalSize != 0) - result.value = kv.value; - break; - default: - result.valueIndex = 0; - result.valueTotalSize = kv.value.size(); - result.value = kv.value; - break; - }; - if(s > 0) { - Key sk = k.getString(0); - result.arena().dependsOn(sk.arena()); - result.key = sk; - if(s > 1) { - result.version = k.getInt(1, true); - } - } - } - if(copyValue && result.value.present()) { - result.value = StringRef(result.arena(), result.value.get()); - } - return result; - } - - static Standalone unpack(KeyRef k) { - return unpack(KeyValueRef(k, StringRef())); - } - - std::string toString() const { - std::string r; - r += format("'%s' @%lld -> ", key.toHexString(15).c_str(), version); - r += value.present() ? format("'%s' %d/%d", value.get().toHexString(15).c_str(), valueIndex, valueTotalSize).c_str() : ""; - return r; - } -}; - -typedef Standalone KeyVersionValue; - #define NOT_IMPLEMENTED { UNSTOPPABLE_ASSERT(false); } class VersionedBTree : public IVersionedStore { public: // The first possible internal record possible in the tree - static KeyVersionValueRef beginKVV; + static RedwoodRecordRef dbBegin; // A record which is greater than the last possible record in the tree - static KeyVersionValueRef endKVV; - - // The encoded key form of the above two things. - static Key beginKey; - static Key endKey; + static RedwoodRecordRef dbEnd; struct Counts { Counts() { @@ -576,7 +647,7 @@ public: ++latest; Reference page = self->m_pager->newPageBuffer(); writeEmptyPage(page, BTreePage::IS_LEAF, self->m_usablePageSizeOverride); - self->writePage(self->m_root, page, latest, StringRef(), StringRef()); + self->writePage(self->m_root, page, latest, &dbBegin, &dbEnd); self->m_pager->setLatestVersion(latest); wait(self->m_pager->commit()); } @@ -604,7 +675,7 @@ public: // TODO: Use the buffer to return uncommitted data // For now, only committed versions can be read. ASSERT(v <= m_lastCommittedVersion); - return Reference(new Cursor(v, m_pager, m_root, m_usablePageSizeOverride)); + return Reference(new Cursor(m_pager->getReadSnapshot(v), m_root, m_usablePageSizeOverride)); } // Must be nondecreasing @@ -617,8 +688,8 @@ public: m_pBuffer = &m_mutationBuffers[v]; // Create range representing the entire keyspace. This reduces edge cases to applying mutations // because now all existing keys are within some range in the mutation map. - (*m_pBuffer)[beginKVV.key]; - (*m_pBuffer)[endKVV.key]; + (*m_pBuffer)[dbBegin.key]; + (*m_pBuffer)[dbEnd.key]; } else { // It's OK to set the write version to the same version repeatedly so long as m_pBuffer is not null @@ -634,14 +705,14 @@ public: } private: - void writePage(LogicalPageID id, Reference page, Version ver, StringRef pageLowerBound, StringRef pageUpperBound) { + void writePage(LogicalPageID id, Reference page, Version ver, const RedwoodRecordRef *pageLowerBound, const RedwoodRecordRef *pageUpperBound) { debug_printf("writePage(): %s\n", ((const BTreePage *)page->begin())->toString(true, id, ver, pageLowerBound, pageUpperBound).c_str()); m_pager->writePage(id, page, ver); } LogicalPageID m_root; - typedef std::pair KeyPagePairT; + typedef std::pair, LogicalPageID> KeyPagePairT; typedef std::pair> VersionedKeyToPageSetT; typedef std::vector VersionedChildrenT; @@ -663,15 +734,14 @@ private: inline bool equalToSet(ValueRef val) { return isSet() && value == val; } - // The returned packed key will be added to arena, the value will just point to the SingleKeyMutation's memory - inline KeyVersionValueRef toKVV(KeyRef userKey, Version version) const { + inline RedwoodRecordRef toRecord(KeyRef userKey, Version version) const { // No point in serializing an atomic op, it needs to be coalesced to a real value. ASSERT(!isAtomicOp()); if(isClear()) - return KeyVersionValueRef(userKey, version); + return RedwoodRecordRef(userKey, version); - return KeyVersionValueRef(userKey, version, value); + return RedwoodRecordRef(userKey, version, value); } std::string toString() const { @@ -812,20 +882,23 @@ private: //debug_printf("buildNewRoot start %lu\n", pages.size()); // While there are multiple child pages for this version we must write new tree levels. while(pages.size() > 1) { - std::vector childEntries; - for(int i=0; i childEntries; + for(int i=0; inewPageBuffer(); }, m_usablePageSizeOverride); + pages = buildPages(false, dbBegin, dbEnd, childEntries, 0, [=](){ return m_pager->newPageBuffer(); }, m_usablePageSizeOverride); debug_printf("Writing a new root level at version %lld with %lu children across %lu pages\n", version, childEntries.size(), pages.size()); - logicalPageIDs = writePages(pages, version, m_root, pPage, endKey, nullptr); + logicalPageIDs = writePages(pages, version, m_root, pPage, &dbEnd, nullptr); } } - std::vector writePages(std::vector pages, Version version, LogicalPageID originalID, const BTreePage *originalPage, StringRef upperBound, void *actor_debug) { + std::vector writePages(std::vector pages, Version version, LogicalPageID originalID, const BTreePage *originalPage, const RedwoodRecordRef *upperBound, void *actor_debug) { debug_printf("%p: writePages(): %u @%lld -> %lu replacement pages\n", actor_debug, originalID, version, pages.size()); ASSERT(version != 0 || pages.size() == 1); @@ -869,7 +942,7 @@ private: } else { debug_printf("%p: writePages(): Writing normal page op=write id=%u @%lld\n", actor_debug, id, version); - writePage(id, pages[i].firstPage, version, pages[i].lowerBound, (i == pages.size() - 1) ? upperBound : pages[i + 1].lowerBound); + writePage(id, pages[i].firstPage, version, &pages[i].lowerBound, (i == pages.size() - 1) ? upperBound : &pages[i + 1].lowerBound); ++counts.blockWrites; } } @@ -926,60 +999,70 @@ private: int m_size; }; - ACTOR static Future> readPage(Reference snapshot, LogicalPageID id, int usablePageSize) { + ACTOR static Future> readPage(Reference snapshot, LogicalPageID id, int usablePageSize, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { debug_printf("readPage() op=read id=%u @%lld\n", id, snapshot->getVersion()); - Reference raw = wait(snapshot->getPhysicalPage(id)); - const BTreePage *pTreePage = (const BTreePage *)raw->begin(); + state Reference result = wait(snapshot->getPhysicalPage(id)); + state const BTreePage *pTreePage = (const BTreePage *)result->begin(); + if(pTreePage->extensionPageCount == 0) { debug_printf("readPage() Found normal page for op=read id=%u @%lld\n", id, snapshot->getVersion()); - return raw; + } + else { + std::vector>> pageGets; + pageGets.push_back(std::move(result)); + + for(int i = 0; i < pTreePage->extensionPageCount; ++i) { + debug_printf("readPage() Reading extension page op=read id=%u @%lld ext=%d/%d\n", pTreePage->extensionPages[i], snapshot->getVersion(), i + 1, (int)pTreePage->extensionPageCount); + pageGets.push_back(snapshot->getPhysicalPage(pTreePage->extensionPages[i])); + } + + std::vector> pages = wait(getAll(pageGets)); + result = Reference(new SuperPage(pages, usablePageSize)); + pTreePage = (const BTreePage *)result->begin(); } - std::vector>> pageGets; - pageGets.push_back(std::move(raw)); - - for(int i = 0; i < pTreePage->extensionPageCount; ++i) { - debug_printf("readPage() Reading extension page op=read id=%u @%lld ext=%d/%d\n", pTreePage->extensionPages[i], snapshot->getVersion(), i + 1, (int)pTreePage->extensionPageCount); - pageGets.push_back(snapshot->getPhysicalPage(pTreePage->extensionPages[i])); + if(result->userData == nullptr) { + result->userData = new BTreePage::BinaryTree::Reader(&pTreePage->tree(), lowerBound, upperBound); + result->userDataDestructor = [](void *ptr) { delete (BTreePage::BinaryTree::Reader *)ptr; }; } - std::vector> pages = wait(getAll(pageGets)); + debug_printf("readPage() %s\n", pTreePage->toString(false, id, snapshot->getVersion(), lowerBound, upperBound).c_str()); - return Reference(new SuperPage(pages, usablePageSize)); + // Nothing should attempt to read bytes in the page outside the BTreePage structure + VALGRIND_MAKE_MEM_UNDEFINED(result->begin() + pTreePage->size(), result->size() - pTreePage->size()); + + return result; } // Returns list of (version, list of (lower_bound, list of children) ) - ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, Key lowerBoundKey, Key upperBoundKey) { - debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", this, root, lowerBoundKey.toHexString(20).c_str(), upperBoundKey.toHexString(20).c_str()); - - // Decode the (likely truncate) upper and lower bound keys for this subtree. - state KeyVersionValue lowerBoundKVV = KeyVersionValue::unpack(lowerBoundKey); - state KeyVersionValue upperBoundKVV = KeyVersionValue::unpack(upperBoundKey); + // TODO: Probably should pass prev/next records by pointer in many places + ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { + debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", this, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); // Find the slice of the mutation buffer that is relevant to this subtree - // TODO: Rather than two lower_bound searches, perhaps just compare each mutation to the upperBound key - state MutationBufferT::const_iterator iMutationBoundary = mutationBuffer->lower_bound(lowerBoundKVV.key); - state MutationBufferT::const_iterator iMutationBoundaryEnd = mutationBuffer->lower_bound(upperBoundKVV.key); + // TODO: Rather than two lower_bound searches, perhaps just compare each mutation to the upperBound key while iterating + state MutationBufferT::const_iterator iMutationBoundary = mutationBuffer->lower_bound(lowerBound->key); + state MutationBufferT::const_iterator iMutationBoundaryEnd = mutationBuffer->lower_bound(upperBound->key); // If the lower bound key and the upper bound key are the same then there can't be any changes to // this subtree since changes would happen after the upper bound key as the mutated versions would // necessarily be higher. - if(lowerBoundKVV.key == upperBoundKVV.key) { + if(lowerBound->key == upperBound->key) { debug_printf("%p no changes, lower and upper bound keys are the same.\n", this); - return VersionedChildrenT({ {0,{{lowerBoundKey,root}}} }); + return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } // If the mutation buffer key found is greater than the lower bound key then go to the previous mutation // buffer key because it may cover deletion of some keys at the start of this subtree. - if(iMutationBoundary != mutationBuffer->begin() && iMutationBoundary->first > lowerBoundKVV.key) { + if(iMutationBoundary != mutationBuffer->begin() && iMutationBoundary->first > lowerBound->key) { --iMutationBoundary; } else { // If the there are no mutations, we're done if(iMutationBoundary == iMutationBoundaryEnd) { debug_printf("%p no changes, mutation buffer start/end are the same\n", this); - return VersionedChildrenT({ {0,{{lowerBoundKey,root}}} }); + return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } } @@ -992,20 +1075,20 @@ private: ++iMutationBoundaryNext; if(iMutationBoundaryNext == iMutationBoundaryEnd && iMutationBoundary->second.noChanges()) { debug_printf("%p no changes because sole mutation range was empty\n", this); - return VersionedChildrenT({ {0,{{lowerBoundKey,root}}} }); + return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } - state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride)); + state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride, lowerBound, upperBound)); state BTreePage *page = (BTreePage *) rawPage->begin(); - debug_printf("%p commitSubtree(): %s\n", this, page->toString(false, root, snapshot->getVersion(), lowerBoundKey, upperBoundKey).c_str()); + debug_printf("%p commitSubtree(): %s\n", this, page->toString(false, root, snapshot->getVersion(), lowerBound, upperBound).c_str()); - PrefixTree::Cursor existingCursor = page->tree().getCursor(lowerBoundKey, upperBoundKey); - bool existingCursorValid = existingCursor.moveFirst(); + BTreePage::BinaryTree::Cursor cursor = getReader(rawPage)->getCursor(); + cursor.moveFirst(); // Leaf Page if(page->flags & BTreePage::IS_LEAF) { VersionedChildrenT results; - std::vector merged; + std::vector merged; Arena mergedArena; debug_printf("%p MERGING EXISTING DATA WITH MUTATIONS:\n", this); @@ -1014,12 +1097,6 @@ private: // It's a given that the mutation map is not empty so it's safe to do this Key mutationRangeStart = iMutationBoundary->first; - // There will be multiple loops advancing existing cursor, existing KVV will track its current value - KeyVersionValue existing; - - if(existingCursorValid) { - existing = KeyVersionValue::unpack(existingCursor.getKVRef()); - } // If replacement pages are written they will be at the minimum version seen in the mutations for this leaf Version minVersion = invalidVersion; @@ -1031,26 +1108,22 @@ private: // If the mutation boundary key is less than the lower bound key then skip startKeyMutations for // this bounary, we're only processing this mutation range here to apply any clears to existing data. - if(iMutationBoundary->first < lowerBoundKVV.key) + if(iMutationBoundary->first < lowerBound->key) iMutations = iMutationBoundary->second.startKeyMutations.end(); // If the mutation boundary key is the same as the page lowerBound key then start reading single - // key mutations at the first version greater than the lowerBoundKey version. - else if(iMutationBoundary->first == lowerBoundKVV.key) - iMutations = iMutationBoundary->second.startKeyMutations.upper_bound(lowerBoundKVV.version); + // key mutations at the first version greater than the lowerBound key's version. + else if(iMutationBoundary->first == lowerBound->key) + iMutations = iMutationBoundary->second.startKeyMutations.upper_bound(lowerBound->version); else iMutations = iMutationBoundary->second.startKeyMutations.begin(); SingleKeyMutationsByVersion::const_iterator iMutationsEnd = iMutationBoundary->second.startKeyMutations.end(); // Output old versions of the mutation boundary key - while(existingCursorValid && existing.key == iMutationBoundary->first) { - // Don't copy the value because this page will stay in memory until after we've built new version(s) of it - merged.push_back(dependsOn(mergedArena, existingCursor.getKV(false))); - debug_printf("%p: Added %s [existing, boundary start]\n", this, KeyVersionValue::unpack(merged.back()).toString().c_str()); - - existingCursorValid = existingCursor.moveNext(); - if(existingCursorValid) - existing = KeyVersionValue::unpack(existingCursor.getKVRef()); + while(cursor.valid() && cursor.get().key == iMutationBoundary->first) { + merged.push_back(cursor.get()); + debug_printf("%p: Added %s [existing, boundary start]\n", this, merged.back().toString().c_str()); + cursor.moveNext(); } // TODO: If a mutation set is equal to the previous existing value of the key, maybe don't write it. @@ -1061,23 +1134,22 @@ private: if(m.isClear() || m.value.size() <= maxPartSize) { if(iMutations->first < minVersion || minVersion == invalidVersion) minVersion = iMutations->first; - // Don't copy the value because this page will stay in memory until after we've built new version(s) of it - merged.push_back(dependsOn(mergedArena, iMutations->second.toKVV(iMutationBoundary->first, iMutations->first).pack(false))); - debug_printf("%p: Added %s [mutation, boundary start]\n", this, KeyVersionValue::unpack(merged.back()).toString().c_str()); + merged.push_back(iMutations->second.toRecord(iMutationBoundary->first, iMutations->first)); + debug_printf("%p: Added non-split %s [mutation, boundary start]\n", this, merged.back().toString().c_str()); } else { if(iMutations->first < minVersion || minVersion == invalidVersion) minVersion = iMutations->first; int bytesLeft = m.value.size(); int start = 0; - KeyVersionValueRef whole(iMutationBoundary->first, iMutations->first, m.value); + RedwoodRecordRef whole(iMutationBoundary->first, iMutations->first, m.value); while(bytesLeft > 0) { int partSize = std::min(bytesLeft, maxPartSize); // Don't copy the value chunk because this page will stay in memory until after we've built new version(s) of it - merged.push_back(dependsOn(mergedArena, whole.split(start, partSize).pack(false))); + merged.push_back(whole.split(start, partSize)); bytesLeft -= partSize; start += partSize; - debug_printf("%p: Added %s [mutation, boundary start]\n", this, KeyVersionValue::unpack(merged.back()).toString().c_str()); + debug_printf("%p: Added split %s [mutation, boundary start]\n", this, merged.back().toString().c_str()); } } ++iMutations; @@ -1091,40 +1163,32 @@ private: debug_printf("%p Mutation range end: '%s'\n", this, printable(iMutationBoundary->first).c_str()); // Write existing keys which are less than the next mutation boundary key, clearing if needed. - while(existingCursorValid && existing.key < iMutationBoundary->first) { - merged.push_back(dependsOn(mergedArena, existingCursor.getKV(false))); - debug_printf("%p: Added %s [existing, middle]\n", this, KeyVersionValue::unpack(merged.back()).toString().c_str()); + while(cursor.valid() && cursor.get().key < iMutationBoundary->first) { + merged.push_back(cursor.get()); + debug_printf("%p: Added %s [existing, middle]\n", this, merged.back().toString().c_str()); - // Write a clear of this key if needed. A clear is required if clearRangeVersion is set and the next key is different - // than this one. Note that the next key might be the in our right sibling, we can use the page upperBound to get that. - existingCursorValid = existingCursor.moveNext(); - KeyVersionValue nextEntry; - if(existingCursorValid) - nextEntry = KeyVersionValue::unpack(existingCursor.getKVRef()); - else - nextEntry = upperBoundKVV; + // Write a clear of this key if needed. A clear is required if clearRangeVersion is set and the next cursor + // key is different than the current one. If the last cursor key in the page is different from the + // first key in the right sibling page then the page's upper bound will reflect that. + auto nextCursor = cursor; + nextCursor.moveNext(); - if(clearRangeVersion.present() && existing.key != nextEntry.key) { + if(clearRangeVersion.present() && cursor.get().key != nextCursor.getOrUpperBound().key) { Version clearVersion = clearRangeVersion.get(); if(clearVersion < minVersion || minVersion == invalidVersion) minVersion = clearVersion; - merged.push_back(dependsOn(mergedArena, KeyVersionValueRef(existing.key, clearVersion).pack(false))); - debug_printf("%p: Added %s [existing, middle clear]\n", this, KeyVersionValue::unpack(merged.back()).toString().c_str()); + merged.push_back(RedwoodRecordRef(cursor.get().key, clearVersion)); + debug_printf("%p: Added %s [existing, middle clear]\n", this, merged.back().toString().c_str()); } - - if(existingCursorValid) - existing = nextEntry; + cursor = nextCursor; } } // Write any remaining existing keys, which are not subject to clears as they are beyond the cleared range. - while(existingCursorValid) { - merged.push_back(dependsOn(mergedArena, existingCursor.getKV(false))); - debug_printf("%p: Added %s [existing, tail]\n", this, KeyVersionValue::unpack(merged.back()).toString().c_str()); - - existingCursorValid = existingCursor.moveNext(); - if(existingCursorValid) - existing = KeyVersionValue::unpack(existingCursor.getKVRef()); + while(cursor.valid()) { + merged.push_back(cursor.get()); + debug_printf("%p: Added %s [existing, tail]\n", this, merged.back().toString().c_str()); + cursor.moveNext(); } debug_printf("%p Done merging mutations into existing leaf contents\n", this); @@ -1133,26 +1197,26 @@ private: // not which turns out to not match any existing data in the leaf. if(minVersion == invalidVersion) { debug_printf("%p No changes were made during mutation merge\n", this); - return VersionedChildrenT({ {0,{{lowerBoundKey,root}}} }); + return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } // TODO: Make version and key splits based on contents of merged list, if keeping history IPager *pager = self->m_pager; - std::vector pages = buildPages(true, lowerBoundKey, upperBoundKey, merged, BTreePage::IS_LEAF, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); + std::vector pages = buildPages(true, *lowerBound, *upperBound, merged, BTreePage::IS_LEAF, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); // If there isn't still just a single page of data then this page became too large and was split. // The new split pages will be valid as of minVersion, but the old page remains valid at the old version // (TODO: unless history isn't being kept at all) if(pages.size() != 1) { - results.push_back( {0, {{lowerBoundKey, root}}} ); + results.push_back( {0, {{*lowerBound, root}}} ); } if(pages.size() == 1) minVersion = 0; // Write page(s), get new page IDs - std::vector newPageIDs = self->writePages(pages, minVersion, root, page, upperBoundKey, this); + std::vector newPageIDs = self->writePages(pages, minVersion, root, page, upperBound, this); // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root && pages.size() > 1) { @@ -1166,9 +1230,9 @@ private: // TODO: This can probably be skipped for root for(int i=0; i %d\n", this, lowerBound.toHexString(20).c_str(), newPageIDs[i]); - results.back().second.push_back( {lowerBound, newPageIDs[i]} ); + const RedwoodRecordRef &lower = (i == 0) ? *lowerBound : pages[i].lowerBound; + debug_printf("%p Adding page to results: %s => %d\n", this, lower.toString().c_str(), newPageIDs[i]); + results.back().second.push_back( {lower, newPageIDs[i]} ); } debug_printf("%p DONE.\n", this); @@ -1182,27 +1246,30 @@ private: // TODO: Make this much more efficient with a skip-merge through the two sorted sets (mutations, existing cursor) bool first = true; - while(existingCursorValid) { - // The lower bound for the first child is lowerBoundKey - Key childLowerBound = first ? lowerBoundKey : existingCursor.getKey(); + while(cursor.valid()) { + // The lower bound for the first child is the lowerBound arg + const RedwoodRecordRef &childLowerBound = first ? *lowerBound : cursor.get(); if(first) first = false; - uint32_t pageID = *(uint32_t*)existingCursor.getValueRef().begin(); + uint32_t pageID = *(uint32_t*)cursor.get().value.get().begin(); ASSERT(pageID != 0); - existingCursorValid = existingCursor.moveNext(); - Key childUpperBound = existingCursorValid ? existingCursor.getKey() : upperBoundKey; + const RedwoodRecordRef &childUpperBound = cursor.moveNext() ? cursor.get() : *upperBound; - debug_printf("lower '%s'\n", childLowerBound.toHexString(20).c_str()); - debug_printf("upper '%s'\n", childUpperBound.toHexString(20).c_str()); + debug_printf("lower '%s'\n", childLowerBound.toString().c_str()); + debug_printf("upper '%s'\n", childUpperBound.toString().c_str()); ASSERT(childLowerBound <= childUpperBound); - futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, childLowerBound, childUpperBound)); + futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound)); childPageIDs.push_back(pageID); } - wait(waitForAll(futureChildren)); + //wait(waitForAll(futureChildren)); + state int k; + for(k = 0; k < futureChildren.size(); ++k) { + wait(success(futureChildren[k])); + } bool modified = false; for(int i = 0; i < futureChildren.size(); ++i) { @@ -1216,7 +1283,7 @@ private: if(!modified) { debug_printf("%p not modified.\n", this); - return VersionedChildrenT({{0, {{lowerBoundKey, root}}}}); + return VersionedChildrenT({{0, {{*lowerBound, root}}}}); } Version version = 0; @@ -1225,7 +1292,7 @@ private: loop { // over version splits of this page Version nextVersion = std::numeric_limits::max(); - std::vector childEntries; // Logically std::vector> childEntries; + std::vector childEntries; // For each Future debug_printf("%p creating replacement pages for id=%d at Version %lld\n", this, root, version); @@ -1241,7 +1308,7 @@ private: for(auto &versionedPageSet : children) { debug_printf("%p version: Page id=%lld\n", this, versionedPageSet.first); for(auto &boundaryPage : versionedPageSet.second) { - debug_printf("%p '%s' -> Page id=%u\n", this, printable(boundaryPage.first).c_str(), boundaryPage.second); + debug_printf("%p '%s' -> Page id=%u\n", this, boundaryPage.first.toString().c_str(), boundaryPage.second); } } @@ -1272,8 +1339,10 @@ private: // Add the children at this version to the child entries list for the current version being built. for (auto &childPage : cv->second) { - debug_printf("%p Adding child page '%s'\n", this, printable(childPage.first).c_str()); - childEntries.emplace_back(childPage.first, StringRef((unsigned char *)&childPage.second, sizeof(uint32_t))); + debug_printf("%p Adding child page %s\n", this, childPage.first.toString().c_str()); + RedwoodRecordRef entry = childPage.first; + entry.value = StringRef((unsigned char *)&childPage.second, sizeof(uint32_t)); + childEntries.push_back(entry); } } @@ -1284,10 +1353,10 @@ private: // cause unnecessary path copying IPager *pager = self->m_pager; - std::vector pages = buildPages(false, lowerBoundKey, upperBoundKey, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); + std::vector pages = buildPages(false, *lowerBound, *upperBound, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); // Write page(s), use version 0 to replace latest version if only writing one page - std::vector newPageIDs = self->writePages(pages, version, root, page, upperBoundKey, this); + std::vector newPageIDs = self->writePages(pages, version, root, page, upperBound, this); // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root) @@ -1307,7 +1376,7 @@ private: } else { debug_printf("%p Version 0 has no changes\n", this); - result.push_back({0, {{lowerBoundKey, root}}}); + result.push_back({0, {{*lowerBound, root}}}); } if (nextVersion == std::numeric_limits::max()) @@ -1346,7 +1415,7 @@ private: self->printMutationBuffer(mutations); - VersionedChildrenT _ = wait(commitSubtree(self, mutations, self->m_pager->getReadSnapshot(latestVersion), self->m_root, beginKey, endKey)); + VersionedChildrenT _ = wait(commitSubtree(self, mutations, self->m_pager->getReadSnapshot(latestVersion), self->m_root, &dbBegin, &dbEnd)); self->m_pager->setLatestVersion(writeVersion); debug_printf("%s: Committing pager %lld\n", self->m_name.c_str(), writeVersion); @@ -1365,239 +1434,278 @@ private: return Void(); } - // InternalCursor is for seeking to and iterating over the internal / low level records in the Btree. - // This records are versioned and they can represent deletions or partial values so they must be - // post processed to obtain keys returnable to the user. - class InternalCursor { + // InternalCursor is for seeking to and iterating over the 'internal' records (not user-visible) in the Btree. + // These records are versioned and they can represent deletedness or partial values. + struct InternalCursor { + private: + // Each InternalCursor's position is represented by a reference counted PageCursor, which links + // to its parent PageCursor, up to a PageCursor representing a cursor on the root page. + // PageCursors can be shared by many InternalCursors, making InternalCursor copying low overhead + struct PageCursor : ReferenceCounted, FastAllocated { + Reference parent; + LogicalPageID pageID; // Only needed for debugging purposes + Reference page; + BTreePage::BinaryTree::Cursor cursor; + + PageCursor(LogicalPageID id, Reference page, Reference parent = {}) + : pageID(id), page(page), parent(parent), cursor(getReader().getCursor()) + { + } + + PageCursor(const PageCursor &toCopy) : parent(toCopy.parent), pageID(toCopy.pageID), page(toCopy.page), cursor(toCopy.cursor) { + } + + // Convenience method for copying a PageCursor + Reference copy() const { + return Reference(new PageCursor(*this)); + } + + // Multiple InternalCursors can share a Page + BTreePage::BinaryTree::Reader & getReader() const { + return *(BTreePage::BinaryTree::Reader *)page->userData; + } + + bool isLeaf() const { + const BTreePage *p = ((const BTreePage *)page->begin()); + return p->isLeaf(); + } + + Future> getChild(Reference pager, int usablePageSizeOverride) { + ASSERT(!isLeaf()); + BTreePage::BinaryTree::Cursor next = cursor; + next.moveNext(); + const RedwoodRecordRef &rec = cursor.get(); + LogicalPageID id = *(LogicalPageID *)rec.value.get().begin(); + Future> child = readPage(pager, id, usablePageSizeOverride, &rec, &next.getOrUpperBound()); + return map(child, [=](Reference page) { + return Reference(new PageCursor(id, page, Reference::addRef(this))); + }); + } + + std::string toString() const { + return format("Page %lu, %s", pageID, cursor.valid() ? cursor.get().toString().c_str() : ""); + } + }; + + LogicalPageID rootPageID; + int usablePageSizeOverride; + Reference pager; + Reference pageCursor; + public: - InternalCursor() {} - InternalCursor(Reference pages, LogicalPageID root, int usablePageSizeOverride) : m_pages(pages), m_root(root), outOfBound(0), m_usablePageSizeOverride(usablePageSizeOverride) { - m_path.reserve(6); + InternalCursor() { } - bool valid() const { - return (outOfBound == 0) && kvv.valid(); + InternalCursor(Reference pager, LogicalPageID root, int usablePageSizeOverride) + : pager(pager), rootPageID(root), usablePageSizeOverride(usablePageSizeOverride) { } - Future seekLessThanOrEqual(KeyRef key) { - return seekLessThanOrEqual_impl(this, key); - } - - Future move(bool fwd) { - return move_impl(this, fwd); - } - - Standalone kvv; // The decoded current internal record in the tree - - std::string toString(const char *wrapPrefix = "") const { + std::string toString() const { std::string r; - r += format("InternalCursor(%p) ver=%lld oob=%d valid=%d", this, m_pages->getVersion(), outOfBound, valid()); - r += format("\n%s KVV: %s", wrapPrefix, kvv.toString().c_str()); - for(const PageEntryLocation &p : m_path) { - std::string cur = p.cursor.valid() ? format("'%s' -> '%s'", p.cursor.getKey().toHexString(20).c_str(), p.cursor.getValueRef().toHexString(20).c_str()) : "invalid"; - r += format("\n%s Page id=%d (%d records, %d bytes) Cursor %s", wrapPrefix, p.pageNumber, p.btPage->count, p.btPage->kvBytes, cur.c_str()); + Reference c = pageCursor; + while(c) { + r = format("[%s] ", c->toString().c_str()) + r; + c = c->parent; } return r; } - private: - Reference m_pages; - LogicalPageID m_root; - int m_usablePageSizeOverride; + // Returns true if cursor position is a valid leaf page record + bool valid() const { + return pageCursor && pageCursor->isLeaf() && pageCursor->cursor.valid(); + } - struct PageEntryLocation { - PageEntryLocation() {} - PageEntryLocation(Key lowerBound, Key upperBound, Reference page, LogicalPageID id) - : pageLowerBound(lowerBound), pageUpperBound(upperBound), page(page), pageNumber(id), btPage((BTreePage *)page->begin()), cursor(btPage->tree().getCursor(pageLowerBound, pageUpperBound)) - { + // Returns true if cursor position is valid() and has a present record value + bool present() { + return valid() && pageCursor->cursor.get().value.present(); + } + + // Returns true if cursor position is present() and has an effective version <= v + bool presentAtVersion(Version v) { + return present() && pageCursor->cursor.get().version <= v; + } + + // Returns true if cursor position is present() and has an effective version <= v + bool validAtVersion(Version v) { + return valid() && pageCursor->cursor.get().version <= v; + } + + const RedwoodRecordRef & get() const { + return pageCursor->cursor.get(); + } + + // Ensure that pageCursor is not shared with other cursors so we can modify it + void ensureUnshared() { + if(!pageCursor->isSoleOwner()) { + pageCursor = pageCursor->copy(); } + } - Key getNextOrUpperBound() { - if(cursor.moveNext()) { - Key r = cursor.getKey(); - cursor.movePrev(); - return r; + Future moveToRoot() { + // If pageCursor exists follow parent links to the root + if(pageCursor) { + while(pageCursor->parent) { + pageCursor = pageCursor->parent; } - return pageUpperBound; + return Void(); } - Key pageLowerBound; - Key pageUpperBound; - Reference page; - BTreePage *btPage; - PrefixTree::Cursor cursor; - // For easier debugging - LogicalPageID pageNumber; - }; - - typedef std::vector TraversalPathT; - TraversalPathT m_path; - int outOfBound; - - ACTOR static Future pushPage(InternalCursor *self, Key lowerBound, Key upperBound, LogicalPageID id) { - Reference rawPage = wait(readPage(self->m_pages, id, self->m_usablePageSizeOverride)); - debug_printf("InternalCursor::pushPage() %s\n", ((const BTreePage *)rawPage->begin())->toString(false, id, self->m_pages->getVersion(), lowerBound, upperBound).c_str()); - self->m_path.emplace_back(lowerBound, upperBound, rawPage, id); - return Void(); + // Otherwise read the root page + Future> root = readPage(pager, rootPageID, usablePageSizeOverride, &dbBegin, &dbEnd); + return map(root, [=](Reference p) { + pageCursor = Reference(new PageCursor(rootPageID, p)); + return Void(); + }); } - ACTOR static Future reset(InternalCursor *self) { - if(self->m_path.empty()) { - wait(pushPage(self, beginKey, endKey, self->m_root)); - } - else { - self->m_path.resize(1); - } - self->outOfBound = 0; - return Void(); - } + ACTOR Future seekLessThanOrEqual_impl(InternalCursor *self, RedwoodRecordRef query) { + Future f = self->moveToRoot(); - ACTOR static Future seekLessThanOrEqual_impl(InternalCursor *self, KeyRef key) { - state TraversalPathT &path = self->m_path; - wait(reset(self)); + // f will almost always be ready + if(!f.isReady()) { + wait(f); + } - debug_printf("InternalCursor::seekLTE(%s): start %s\n", key.toHexString(20).c_str(), self->toString(" ").c_str()); + self->ensureUnshared(); loop { - state PageEntryLocation *p = &path.back(); - - if(p->btPage->count == 0) { - ASSERT(path.size() == 1); // This must be the root page. - self->outOfBound = -1; - self->kvv.version = invalidVersion; - debug_printf("InternalCursor::seekLTE(%s): Exit, root page empty. %s\n", key.toHexString(20).c_str(), self->toString(" ").c_str()); - return Void(); - } - - state bool foundLTE = p->cursor.seekLessThanOrEqual(key); - debug_printf("InternalCursor::seekLTE(%s): Seek on path tail, result %d. %s\n", key.toHexString(20).c_str(), foundLTE, self->toString(" ").c_str()); - - if(p->btPage->flags & BTreePage::IS_LEAF) { - // It is possible for the current leaf key to be between the page's lower bound (in the parent page) and the - // first record in the leaf page, which means we must move backwards 1 step in the database to find the - // record < key, if such a record exists. - if(!foundLTE) { - wait(self->move(false)); + if(self->pageCursor->cursor.seekLessThanOrEqual(query)) { + // If we found a record <= query at a leaf page then return success + if(self->pageCursor->isLeaf()) { + return true; } - else { - // Found the target record - self->kvv = KeyVersionValue::unpack(p->cursor.getKVRef()); - } - debug_printf("InternalCursor::seekLTE(%s): Exit, Found leaf page. %s\n", key.toHexString(20).c_str(), self->toString(" ").c_str()); - return Void(); + + // Otherwise move to next child page + Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); + self->pageCursor = child; } else { - // We don't have to check foundLTE here because if it's false then cursor will be at the first record in the page. - // TODO: It would, however, be more efficient to check foundLTE and if false move to the previous sibling page. - // But the page should NOT be empty so let's assert that the cursor is valid. - ASSERT(p->cursor.valid()); - - state LogicalPageID newPage = (LogicalPageID)*(uint32_t *)p->cursor.getValueRef().begin(); - debug_printf("InternalCursor::seekLTE(%s): Found internal page, going to Page id=%d. %s\n", - key.toHexString(20).c_str(), newPage, self->toString(" ").c_str()); - wait(pushPage(self, p->cursor.getKey(), p->getNextOrUpperBound(), newPage)); + // No records <= query on this page, so move to immediate previous record at leaf level + bool success = wait(self->move(false)); + return success; } } } - // Move one 'internal' key/value/version/valueindex/value record. - // Iterating with this function will "see" all parts of all values and clears at all versions (that is, within the cursor's version of btree pages) - ACTOR static Future move_impl(InternalCursor *self, bool fwd) { - state TraversalPathT &path = self->m_path; - state const char *dir = fwd ? "forward" : "backward"; + Future seekLTE(RedwoodRecordRef query) { + return seekLessThanOrEqual_impl(this, query); + } - debug_printf("InternalCursor::move(%s) start %s\n", dir, self->toString(" ").c_str()); + ACTOR Future move_impl(InternalCursor *self, bool forward) { + // Try to move pageCursor, if it fails to go parent, repeat until it works or root cursor can't be moved + while(1) { + self->ensureUnshared(); + bool success = self->pageCursor->cursor.valid() && (forward ? self->pageCursor->cursor.moveNext() : self->pageCursor->cursor.movePrev()); - // If cursor was out of bound, adjust out of boundness by 1 in the correct direction - if(self->outOfBound != 0) { - self->outOfBound += fwd ? 1 : -1; - // If we appear to be inbounds, see if we're off the other end of the db or if the page cursor is valid. - if(self->outOfBound == 0) { - if(!path.empty() && path.back().cursor.valid()) { - self->kvv = KeyVersionValue::unpack(path.back().cursor.getKVRef()); - } - else { - self->outOfBound = fwd ? 1 : -1; - } - } - debug_printf("InternalCursor::move(%s) was out of bound, exiting %s\n", dir, self->toString(" ").c_str()); - return Void(); - } - - int i = path.size(); - // Find the closest path part to the end where the index can be moved in the correct direction. - while(--i >= 0) { - PrefixTree::Cursor &c = path[i].cursor; - bool success = fwd ? c.moveNext() : c.movePrev(); - if(success) { - debug_printf("InternalCursor::move(%s) Move successful on path index %d\n", dir, i); - path.resize(i + 1); + // Stop if successful or there's no parent to move to + if(success || !self->pageCursor->parent) { break; - } else { - debug_printf("InternalCursor::move(%s) Move failed on path index %d\n", dir, i); + } + + // Move to parent + self->pageCursor = self->pageCursor->parent; + } + + // If pageCursor not valid we've reached an end of the tree + if(!self->pageCursor->cursor.valid()) { + return false; + } + + // While not on a leaf page, move down to get to one. + while(!self->pageCursor->isLeaf()) { + Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); + bool success = forward ? child->cursor.moveFirst() : child->cursor.moveLast(); + self->pageCursor = child; + } + + return true; + } + + Future move(bool forward) { + return move_impl(this, forward); + } + + Future moveNext() { + return move_impl(this, true); + } + Future movePrev() { + return move_impl(this, false); + } + + // Move to the first or last record of the database. + ACTOR Future move_end(InternalCursor *self, bool begin) { + Future f = self->moveToRoot(); + + // f will almost always be ready + if(!f.isReady()) { + wait(f); + } + + self->ensureUnshared(); + + loop { + // Move to first or last record in the page + bool success = begin ? self->pageCursor->cursor.moveFirst() : self->pageCursor->cursor.moveLast(); + + // If it worked, return true if we've reached a leaf page otherwise go to the next child + if(success) { + if(self->pageCursor->isLeaf()) { + return true; + } + Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); + self->pageCursor = child; + } + else { + return false; } } - - // If no path part could be moved without going out of range then the - // new cursor position is either before the first record or after the last. - // Leave the path steps in place and set outOfBound to 1 or -1 based on fwd. - // This makes the cursor not valid() but a move in the opposite direction - // will make it valid again, pointing to the previous target record. - if(i < 0) { - self->outOfBound = fwd ? 1 : -1; - debug_printf("InternalCursor::move(%s) Passed an end of the database %s\n", dir, self->toString(" ").c_str()); - return Void(); - } - - // We were able to advance the cursor on one of the pages in the page traversal path, so now traverse down to leaf level - state PageEntryLocation *p = &(path.back()); - - debug_printf("InternalCursor::move(%s): Descending if needed to find a leaf\n", dir); - - // Now we must traverse downward if needed until we are at a leaf level. - // Each movement down will start on the far left or far right depending on fwd - while(!(p->btPage->flags & BTreePage::IS_LEAF)) { - // Get the page that the path's last entry points to - LogicalPageID childPageID = (LogicalPageID)*(uint32_t *)p->cursor.getValueRef().begin(); - - wait(pushPage(self, p->cursor.getKey(), p->getNextOrUpperBound(), childPageID)); - p = &(path.back()); - // No page traversed to in this manner should be empty. - ASSERT(p->btPage->count != 0); - // Go to the first or last entry in the page depending on traversal direction - if(fwd) - p->cursor.moveFirst(); - else - p->cursor.moveLast(); - - debug_printf("InternalCursor::move(%s) Descended one level %s\n", dir, self->toString(" ").c_str()); - } - - // Found the target record, unpack it - ASSERT(p->cursor.valid()); - self->kvv = KeyVersionValue::unpack(p->cursor.getKVRef()); - - debug_printf("InternalCursor::move(%s) Exiting %s\n", dir, self->toString(" ").c_str()); - return Void(); } + + Future moveFirst() { + return move_end(this, true); + } + Future moveLast() { + return move_end(this, false); + } + }; // Cursor is for reading and interating over user visible KV pairs at a specific version - // Keys and values returned are only valid until one of the move methods is called (find*, next, prev) - // TODO: Make an option to copy all returned strings into an arena? - class Cursor : public IStoreCursor, public ReferenceCounted, public NonCopyable { + // KeyValueRefs returned become invalid once the cursor is moved + class Cursor : public IStoreCursor, public ReferenceCounted, public FastAllocated, NonCopyable { public: - Cursor(Version version, IPager *pager, LogicalPageID root, int usablePageSizeOverride) - : m_version(version), m_pagerSnapshot(pager->getReadSnapshot(version)), m_icursor(m_pagerSnapshot, root, usablePageSizeOverride) { + Cursor(Reference pageSource, LogicalPageID root, int usablePageSizeOverride) + : m_version(pageSource->getVersion()), + m_cur1(pageSource, root, usablePageSizeOverride), + m_cur2(m_cur1) + { } - virtual ~Cursor() {} - virtual Future findEqual(KeyRef key) { return find_impl(Reference::addRef(this), key, true, 0); } - virtual Future findFirstEqualOrGreater(KeyRef key, bool needValue, int prefetchNextBytes) { return find_impl(Reference::addRef(this), key, needValue, 1); } - virtual Future findLastLessOrEqual(KeyRef key, bool needValue, int prefetchPriorBytes) { return find_impl(Reference::addRef(this), key, needValue, -1); } + void addref() { ReferenceCounted::addref(); } + void delref() { ReferenceCounted::delref(); } - virtual Future next(bool needValue) { return next_impl(Reference::addRef(this), needValue); } - virtual Future prev(bool needValue) { return prev_impl(Reference::addRef(this), needValue); } + private: + Version m_version; + // If kv is valid + // - kv.key references memory held by cur1 + // - If cur1 points to a non split KV pair + // - kv.value references memory held by cur1 + // - cur2 points to the next internal record after cur1 + // Else + // - kv.value references memory in arena + // - cur2 points to the first internal record of the split KV pair + InternalCursor m_cur1; + InternalCursor m_cur2; + Arena m_arena; + Optional m_kv; + + public: + virtual Future findEqual(KeyRef key) { return find_impl(this, key, true, 0); } + virtual Future findFirstEqualOrGreater(KeyRef key, bool needValue, int prefetchNextBytes) { return find_impl(this, key, needValue, 1); } + virtual Future findLastLessOrEqual(KeyRef key, bool needValue, int prefetchPriorBytes) { return find_impl(this, key, needValue, -1); } + + virtual Future next(bool needValue) { return move(this, true, needValue); } + virtual Future prev(bool needValue) { return move(this, false, needValue); } virtual bool isValid() { return m_kv.present(); @@ -1606,6 +1714,7 @@ private: virtual KeyRef getKey() { return m_kv.get().key; } + //virtual StringRef getCompressedKey() = 0; virtual ValueRef getValue() { return m_kv.get().value; @@ -1616,63 +1725,79 @@ private: virtual void invalidateReturnedStrings() { } - void addref() { ReferenceCounted::addref(); } - void delref() { ReferenceCounted::delref(); } - - std::string toString(const char *wrapPrefix = "") const { + std::string toString() const { std::string r; - r += format("Cursor(%p) ver: %lld key: %s value: %s", this, m_version, - (m_kv.present() ? m_kv.get().key.printable().c_str() : ""), - (m_kv.present() ? m_kv.get().value.printable().c_str() : "")); - r += format("\n%s InternalCursor: %s", wrapPrefix, m_icursor.toString(format("%s ", wrapPrefix).c_str()).c_str()); + r += format("Cursor(%p) ver: %lld ", this, m_version); + r += format(" KV: '%s' -> '%s'\n", + m_kv.present() ? m_kv.get().key.printable().c_str() : "", + m_kv.present() ? m_kv.get().value.printable().c_str() : ""); + r += format(" Cur1: %s\n", m_cur1.toString().c_str()); + r += format(" Cur2: %s\n", m_cur2.toString().c_str()); return r; } private: - Version m_version; - Reference m_pagerSnapshot; - InternalCursor m_icursor; - Optional m_kv; // The current user-level key/value in the tree - Arena m_arena; - // find key in tree closest to or equal to key (at this cursor's version) // for less than or equal use cmp < 0 // for greater than or equal use cmp > 0 // for equal use cmp == 0 - ACTOR static Future find_impl(Reference self, KeyRef key, bool needValue, int cmp) { - state InternalCursor &icur = self->m_icursor; - + ACTOR static Future find_impl(Cursor *self, KeyRef key, bool needValue, int cmp) { // Search for the last key at or before (key, version, \xff) - state Key target = KeyVersionValueRef::searchKey(key, self->m_version); - self->m_kv = Optional(); + state RedwoodRecordRef query(key, self->m_version); + self->m_kv.reset(); - wait(icur.seekLessThanOrEqual(target)); - debug_printf("find%sE('%s'): %s\n", cmp > 0 ? "GT" : (cmp == 0 ? "" : "LT"), target.toHexString(15).c_str(), icur.toString().c_str()); + wait(success(self->m_cur1.seekLTE(query))); + debug_printf("find%sE(%s): %s\n", cmp > 0 ? "GT" : (cmp == 0 ? "" : "LT"), query.toString().c_str(), self->toString().c_str()); - // If we found the target key, return it as it is valid for any cmp option - if(icur.valid() && icur.kvv.value.present() && icur.kvv.key == key) { - debug_printf("Reading full kv pair starting from: %s\n", icur.kvv.toString().c_str()); + // If we found the target key with a present value then return it as it is valid for any cmp type + if(self->m_cur1.present() && self->m_cur1.get().key == key) { + debug_printf("Target key found, reading full KV pair. Cursor: %s\n", self->toString().c_str()); wait(self->readFullKVPair(self)); return Void(); } - // FindEqual, so if we're still here we didn't find it. + // Mode is ==, so if we're still here we didn't find it. if(cmp == 0) { return Void(); } - // FindEqualOrGreaterThan, so if we're here we have to go to the next present record at the target version. + // Mode is >=, so if we're here we have to go to the next present record at the target version + // because the seek done above was <= query if(cmp > 0) { - // icur is at a record < key, possibly before the start of the tree so move forward at least once. - loop { - wait(icur.move(true)); - if(!icur.valid() || icur.kvv.key > key) - break; + // icur is at a record < query or invalid. + + // If cursor is invalid, try to go to start of tree + if(!self->m_cur1.valid()) { + bool valid = wait(self->m_cur1.moveFirst()); + if(!valid) { + self->m_kv.reset(); + return Void(); + } } + else { + loop { + bool valid = wait(self->m_cur1.move(true)); + if(!valid) { + self->m_kv.reset(); + return Void(); + } + + if(self->m_cur1.get().key > key) { + break; + } + } + } + // Get the next present key at the target version. Handles invalid cursor too. wait(self->next(needValue)); } else if(cmp < 0) { + // Mode is <=, which is the same as the seekLTE(query) + if(!self->m_cur1.valid()) { + self->m_kv.reset(); + return Void(); + } + // Move to previous present kv pair at the target version wait(self->prev(needValue)); } @@ -1680,136 +1805,117 @@ private: return Void(); } - ACTOR static Future next_impl(Reference self, bool needValue) { - // TODO: use needValue - state InternalCursor &i = self->m_icursor; + // TODO: use needValue + ACTOR static Future move(Cursor *self, bool fwd, bool needValue) { + debug_printf("Cursor::move(%d): Cursor = %s\n", fwd, self->toString().c_str()); + ASSERT(self->m_cur1.valid()); - debug_printf("Cursor::next(): cursor %s\n", i.toString().c_str()); - - // Make sure we are one record past the last user key + // If kv is present then the key/version at cur1 was already returned so move to a new key + // Move cur1 until failure or a new key is found, keeping prior record visited in cur2 if(self->m_kv.present()) { - while(i.valid() && i.kvv.key <= self->m_kv.get().key) { - debug_printf("Cursor::next(): Advancing internal cursor to get passed previous returned user key. cursor %s\n", i.toString().c_str()); - wait(i.move(true)); + ASSERT(self->m_cur1.valid()); + loop { + self->m_cur2 = self->m_cur1; + bool valid = wait(self->m_cur1.move(fwd)); + if(!valid || self->m_cur1.get().key != self->m_cur2.get().key) { + break; + } } } - state Version v = self->m_pagerSnapshot->getVersion(); - state InternalCursor iLast; - while(1) { - iLast = i; - if(!i.valid()) - break; - wait(i.move(true)); - // If the previous cursor position was a set at a version at or before v and the new cursor position - // is not valid or a newer version of the same key or a different key, then get the full record - // for the previous cursor position - if(iLast.kvv.version <= v - && iLast.kvv.value.present() - && ( - !i.valid() - || i.kvv.key != iLast.kvv.key - || i.kvv.version > v - ) + // Given two consecutive cursors c1 and c2, c1 represents a returnable record if + // c1.presentAtVersion(v) || (!c2.validAtVersion() || c2.get().key != c1.get().key()) + // Note the distinction between 'present' and 'valid'. Present means the value for the key + // exists at the version (but could be the empty string) while valid just means the internal + // record is in effect at that version but it could indicate that the key was cleared and + // no longer exists from the user's perspective at that version + // + // If moving forward, cur2 must be the record after cur1 so we can determine if + // cur1 is to be returned below. + // If moving backward, cur2 is already the record after cur1 + if(fwd && self->m_cur1.valid()) { + self->m_cur2 = self->m_cur1; + wait(success(self->m_cur2.move(true))); + } + + while(self->m_cur1.valid()) { + + if(self->m_cur1.presentAtVersion(self->m_version) && + (!self->m_cur2.validAtVersion(self->m_version) || + self->m_cur2.get().key != self->m_cur1.get().key) ) { - // Assume that next is the most likely next move, so save the one-too-far cursor position. - std::swap(i, iLast); - // readFullKVPair will have to go backwards to read the value wait(readFullKVPair(self)); - std::swap(i, iLast); return Void(); } - } - self->m_kv = Optional(); - return Void(); - } - - ACTOR static Future prev_impl(Reference self, bool needValue) { - // TODO: use needValue - state InternalCursor &i = self->m_icursor; - - debug_printf("Cursor::prev(): cursor %s\n", i.toString().c_str()); - - // Make sure we are one record before the last user key - if(self->m_kv.present()) { - while(i.valid() && i.kvv.key >= self->m_kv.get().key) { - wait(i.move(false)); - } - } - - state Version v = self->m_pagerSnapshot->getVersion(); - while(i.valid()) { - // Once we reach a present value at or before v, return or skip it. - if(i.kvv.version <= v) { - // If it's present, return it - if(i.kvv.value.present()) { - wait(readFullKVPair(self)); - return Void(); - } - // Value wasn't present as of the latest version <= v, so move backward to a new key - state Key clearedKey = i.kvv.key; - while(1) { - wait(i.move(false)); - if(!i.valid() || i.kvv.key != clearedKey) - break; - } + if(fwd) { + // Moving forward, move cur2 forward and keep cur1 pointing to the prior (predecessor) record + self->m_cur1 = self->m_cur2; + wait(success(self->m_cur2.move(true))); } else { - wait(i.move(false)); + // Moving backward, move cur1 backward and keep cur2 pointing to the prior (successor) record + self->m_cur2 = self->m_cur1; + wait(success(self->m_cur1.move(false))); } + } - self->m_kv = Optional(); + self->m_kv.reset(); return Void(); } - // Read all of the current value, if it is split across multiple kv pairs, and set m_kv. - // m_current must be at either the first or the last value part. - ACTOR static Future readFullKVPair(Reference self) { - state KeyVersionValue &kvv = self->m_icursor.kvv; - state KeyValueRef &kv = (self->m_kv = KeyValueRef()).get(); + // Read all of the current key-value record starting at cur1 into kv + ACTOR static Future readFullKVPair(Cursor *self) { + self->m_arena = Arena(); + const RedwoodRecordRef &rec = self->m_cur1.get(); + + debug_printf("readFullKVPair: Starting at %s\n", self->toString().c_str()); - ASSERT(kvv.value.present()); - // Set the key and cursor arena to the arena containing that key - self->m_arena = kvv.arena(); - kv.key = kvv.key; + // Unsplit value, cur1 will hold the key and value memory + if(!rec.isMultiPart()) { + debug_printf("readFullKVPair: Unsplit, exit. %s\n", self->toString().c_str()); - // Unsplit value - if(!kvv.isMultiPart()) { - kv.value = kvv.value.get(); - debug_printf("readFullKVPair: Unsplit, exit. %s\n", self->toString(" ").c_str()); + self->m_kv = KeyValueRef(rec.key, rec.value.get()); + return Void(); } - else { - // Figure out if we should go forward or backward to find all the parts - state bool fwd = kvv.valueIndex == 0; - ASSERT(fwd || kvv.valueIndex + kvv.value.get().size() == kvv.valueTotalSize); - debug_printf("readFullKVPair: Split, fwd %d totalsize %lld %s\n", fwd, kvv.valueTotalSize, self->toString(" ").c_str()); - // Allocate space for the entire value in the same arena as the key - state int bytesLeft = kvv.valueTotalSize; - kv.value = makeString(bytesLeft, self->m_arena); - while(1) { - debug_printf("readFullKVPair: Adding chunk start %lld len %d total %lld dir %d\n", kvv.valueIndex, kvv.value.get().size(), kvv.valueTotalSize, fwd); - int partSize = kvv.value.get().size(); - memcpy(mutateString(kv.value) + kvv.valueIndex, kvv.value.get().begin(), partSize); - bytesLeft -= partSize; - if(bytesLeft == 0) - break; - ASSERT(bytesLeft > 0); - wait(self->m_icursor.move(fwd)); - ASSERT(self->m_icursor.valid()); + + // Split value, need to coalesce split value parts into a buffer in arena, + // after which cur1 will point to the first part and kv.key will reference its key + const RedwoodRecordRef::ValuePart &part = rec.valuePart.get(); + ASSERT(part.start + rec.value.get().size() == part.total); + + debug_printf("readFullKVPair: Split, totalsize %d %s\n", part.total, self->toString().c_str()); + + // Allocate space for the entire value in the same arena as the key + state int bytesLeft = part.total; + state StringRef dst = makeString(bytesLeft, self->m_arena); + + loop { + const RedwoodRecordRef &rec = self->m_cur1.get(); + const RedwoodRecordRef::ValuePart &part = rec.valuePart.get(); + + debug_printf("readFullKVPair: Adding chunk %s\n", rec.toString().c_str()); + + int partSize = rec.value.get().size(); + memcpy(mutateString(dst) + part.start, rec.value.get().begin(), partSize); + bytesLeft -= partSize; + if(bytesLeft == 0) { + self->m_kv = KeyValueRef(rec.key, dst); + return Void(); } + ASSERT(bytesLeft > 0); + // Move backward + bool success = wait(self->m_cur1.move(false)); + ASSERT(success); } - - return Void(); } }; + }; -KeyVersionValueRef VersionedBTree::beginKVV(StringRef(), 0, StringRef()); -KeyVersionValueRef VersionedBTree::endKVV(LiteralStringRef("\xff\xff\xff\xff"), std::numeric_limits::max(), StringRef()); -Key VersionedBTree::beginKey(beginKVV.pack().key); -Key VersionedBTree::endKey(endKVV.pack().key); +RedwoodRecordRef VersionedBTree::dbBegin(StringRef(), 0); +RedwoodRecordRef VersionedBTree::dbEnd(LiteralStringRef("\xff\xff\xff\xff"), std::numeric_limits::max()); ACTOR template Future catchError(Promise error, Future f) { @@ -1993,9 +2099,7 @@ IKeyValueStore* keyValueStoreRedwoodV1( std::string const& filename, UID logID) } int randomSize(int max) { - int exp = g_random->randomInt(0, 6); - int limit = (pow(10.0, exp) / 1e5 * max) + 1; - int n = g_random->randomInt(0, max); + int n = pow(g_random->random01(), 3) * max; return n; } @@ -2157,7 +2261,8 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, state Reference cur = btree->readAtVersion(ver); debug_printf("Verifying @%lld '%s'\n", ver, key.c_str()); - wait(cur->findEqual(key)); + state Arena arena; + wait(cur->findEqual(KeyRef(arena, key))); if(val.present()) { if(!(cur->isValid() && cur->getKey() == key && cur->getValue() == val.get())) { @@ -2217,7 +2322,8 @@ ACTOR Future randomReader(VersionedBTree *btree) { cur = btree->readAtVersion(v); } - wait(cur->findFirstEqualOrGreater(randomKV(10, 0).key, true, 0)); + state KeyValue kv = randomKV(10, 0); + wait(cur->findFirstEqualOrGreater(kv.key, true, 0)); state int c = g_random->randomInt(0, 100); while(cur->isValid() && c-- > 0) { wait(success(cur->next(true))); @@ -2226,6 +2332,278 @@ ACTOR Future randomReader(VersionedBTree *btree) { } } +struct SplitStringRef { + StringRef a; + StringRef b; + + SplitStringRef(StringRef a = StringRef(), StringRef b = StringRef()) : a(a), b(b) { + } + + SplitStringRef getSplitPrefix(int len) const { + if(len <= a.size()) { + return SplitStringRef(a.substr(0, len)); + } + len -= a.size(); + ASSERT(b.size() <= len); + return SplitStringRef(a, b.substr(0, len)); + } + + StringRef getContiguousPrefix(int len, Arena &arena) const { + if(len <= a.size()) { + return a.substr(0, len); + } + StringRef c = makeString(len, arena); + memcpy(mutateString(c), a.begin(), a.size()); + len -= a.size(); + memcpy(mutateString(c) + a.size(), b.begin(), len); + return c; + } + + int compare(const SplitStringRef &rhs) const { + // TODO: Rewrite this.. + Arena a; + StringRef self = getContiguousPrefix(size(), a); + StringRef other = rhs.getContiguousPrefix(rhs.size(), a); + return self.compare(other); + } + + int compare(const StringRef &rhs) const { + // TODO: Rewrite this.. + Arena a; + StringRef self = getContiguousPrefix(size(), a); + return self.compare(rhs); + } + + int size() const { + return a.size() + b.size(); + } + + std::string toString() const { + return format("%s%s", a.toString().c_str(), b.toString().c_str()); + } + + std::string toHexString() const { + return format("%s%s", a.toHexString().c_str(), b.toHexString().c_str()); + } +}; + +struct IntIntPair { + IntIntPair() {} + IntIntPair(int k, int v) : k(k), v(v) {} + + IntIntPair(Arena &arena, const IntIntPair &toCopy) { + *this = toCopy; + } + + struct Delta { + int dk; + int dv; + + IntIntPair apply(const IntIntPair &prev, const IntIntPair &next, Arena arena) { + return {prev.k + dk, prev.v + dv}; + } + + int size() const { + return sizeof(Delta); + } + + std::string toString() const { + return format("DELTA{dk=%d(0x%x) dv=%d(0x%x)}", dk, dk, dv, dv); + } + }; + + int compare(const IntIntPair &rhs) const { + //printf("compare %s to %s\n", toString().c_str(), rhs.toString().c_str()); + return k - rhs.k; + } + + bool operator==(const IntIntPair &rhs) const { + return k == rhs.k; + } + + int deltaSize(const IntIntPair &base) const { + return sizeof(Delta); + } + + void writeDelta(Delta &d, const IntIntPair &prev, const IntIntPair &next) const { + // Always borrow from prev + d.dk = k - prev.k; + d.dv = v - prev.v; + } + + int k; + int v; + + std::string toString() const { + return format("{k=%d(0x%x) v=%d(0x%x)}", k, k, v, v); + } +}; + +TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessRedwoodRecord") { + const int N = 200; + + RedwoodRecordRef prev; + RedwoodRecordRef next(LiteralStringRef("\xff\xff\xff\xff")); + + Arena arena; + std::vector items; + for(int i = 0; i < N; ++i) { + std::string k = g_random->randomAlphaNumeric(30); + std::string v = g_random->randomAlphaNumeric(30); + RedwoodRecordRef rec; + rec.key = StringRef(arena, k); + rec.version = g_random->coinflip() ? g_random->randomInt64(0, std::numeric_limits::max()) : invalidVersion; + if(g_random->coinflip()) { + rec.value = StringRef(arena, v); + if(g_random->coinflip()) { + RedwoodRecordRef::ValuePart part; + part.start = g_random->randomInt(0, 5000); + part.total = part.start + v.size() + g_random->randomInt(0, 5000); + rec.valuePart = part; + } + } + items.push_back(rec); + //printf("i=%d %s\n", i, items.back().toString().c_str()); + } + std::sort(items.begin(), items.end()); + + DeltaTree *tree = (DeltaTree *) new uint8_t[N * 100]; + + tree->build(&items[0], &items[items.size()], &prev, &next); + + printf("Count=%d Size=%d InitialDepth=%d\n", (int)items.size(), (int)tree->size(), (int)tree->initialDepth); + debug_printf("Data(%p): %s\n", tree, StringRef((uint8_t *)tree, tree->size()).toHexString().c_str()); + + DeltaTree::Reader r(tree, &prev, &next); + DeltaTree::Cursor fwd = r.getCursor(); + DeltaTree::Cursor rev = r.getCursor(); + + ASSERT(fwd.moveFirst()); + ASSERT(rev.moveLast()); + int i = 0; + while(1) { + if(fwd.get() != items[i]) { + printf("forward iterator i=%d\n %s found\n %s expected\n", i, fwd.get().toString().c_str(), items[i].toString().c_str()); + printf("Delta: %s\n", fwd.node->raw->delta->toString().c_str()); + ASSERT(false); + } + if(rev.get() != items[items.size() - 1 - i]) { + printf("reverse iterator i=%d\n %s found\n %s expected\n", i, rev.get().toString().c_str(), items[items.size() - 1 - i].toString().c_str()); + printf("Delta: %s\n", rev.node->raw->delta->toString().c_str()); + ASSERT(false); + } + ++i; + ASSERT(fwd.moveNext() == rev.movePrev()); + ASSERT(fwd.valid() == rev.valid()); + if(!fwd.valid()) { + break; + } + } + ASSERT(i == items.size()); + + double start = timer(); + DeltaTree::Cursor c = r.getCursor(); + + for(int i = 0; i < 20000000; ++i) { + const RedwoodRecordRef &query = items[g_random->randomInt(0, items.size())]; + if(!c.seekLessThanOrEqual(query)) { + printf("Not found! query=%s\n", query.toString().c_str()); + ASSERT(false); + } + if(c.get() != query) { + printf("Found incorrect node! query=%s found=%s\n", query.toString().c_str(), c.get().toString().c_str()); + ASSERT(false); + } + } + double elapsed = timer() - start; + printf("Elapsed %f\n", elapsed); + + return Void(); +} + +TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessIntInt") { + const int N = 200; + IntIntPair prev = {0, 0}; + IntIntPair next = {1000, 0}; + + std::vector items; + for(int i = 0; i < N; ++i) { + items.push_back({i*10, i*1000}); + //printf("i=%d %s\n", i, items.back().toString().c_str()); + } + + DeltaTree *tree = (DeltaTree *) new uint8_t[10000]; + + tree->build(&items[0], &items[items.size()], &prev, &next); + + printf("Count=%d Size=%d InitialDepth=%d\n", (int)items.size(), (int)tree->size(), (int)tree->initialDepth); + debug_printf("Data(%p): %s\n", tree, StringRef((uint8_t *)tree, tree->size()).toHexString().c_str()); + + DeltaTree::Reader r(tree, &prev, &next); + DeltaTree::Cursor fwd = r.getCursor(); + DeltaTree::Cursor rev = r.getCursor(); + + ASSERT(fwd.moveFirst()); + ASSERT(rev.moveLast()); + int i = 0; + while(1) { + if(fwd.get() != items[i]) { + printf("forward iterator i=%d\n %s found\n %s expected\n", i, fwd.get().toString().c_str(), items[i].toString().c_str()); + ASSERT(false); + } + if(rev.get() != items[items.size() - 1 - i]) { + printf("reverse iterator i=%d\n %s found\n %s expected\n", i, rev.get().toString().c_str(), items[items.size() - 1 - i].toString().c_str()); + ASSERT(false); + } + ++i; + ASSERT(fwd.moveNext() == rev.movePrev()); + ASSERT(fwd.valid() == rev.valid()); + if(!fwd.valid()) { + break; + } + } + ASSERT(i == items.size()); + + DeltaTree::Cursor c = r.getCursor(); + + double start = timer(); + for(int i = 0; i < 20000000; ++i) { + IntIntPair p({g_random->randomInt(0, items.size() * 10), 0}); + if(!c.seekLessThanOrEqual(p)) { + printf("Not found! query=%s\n", p.toString().c_str()); + ASSERT(false); + } + if(c.get().k != (p.k - (p.k % 10))) { + printf("Found incorrect node! query=%s found=%s\n", p.toString().c_str(), c.get().toString().c_str()); + ASSERT(false); + } + } + double elapsed = timer() - start; + printf("Elapsed %f\n", elapsed); + + return Void(); +} + +struct SimpleCounter { + SimpleCounter() : x(0), xt(0), t(timer()), start(t) {} + void operator+=(int n) { x += n; } + void operator++() { x++; } + int64_t get() { return x; } + double rate() { + double t2 = timer(); + int r = (x - xt) / (t2 - t); + xt = x; + t = t2; + return r; + } + double avgRate() { return x / (timer() - start); } + int64_t x; + double t; + double start; + int64_t xt; + std::string toString() { return format("%lld/%.2f/%.2f", x, rate() / 1e6, avgRate() / 1e6); } +}; + TEST_CASE("!/redwood/correctness") { state bool useDisk = true; // MemoryPager is not being maintained currently. @@ -2245,14 +2623,13 @@ TEST_CASE("!/redwood/correctness") { state VersionedBTree *btree = new VersionedBTree(pager, pagerFile, pageSize); wait(btree->init()); - state int mutationBytesTarget = g_random->randomInt(100, 20e6); - // We must be able to fit at least two any two keys plus overhead in a page to prevent // a situation where the tree cannot be grown upward with decreasing level size. // TODO: Handle arbitrarily large keys - state int maxKeySize = std::min(pageSize * 8, 30000); - ASSERT(maxKeySize > 0); - state int maxValueSize = std::min(pageSize * 25, 100000); + state int maxKeySize = g_random->randomInt(4, pageSize * 2); + state int maxValueSize = g_random->randomInt(0, pageSize * 2); + + state int mutationBytesTarget = g_random->randomInt(100, (maxKeySize + maxValueSize) * 2000); printf("Using page size %d, max key size %d, max value size %d, total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, mutationBytesTarget); @@ -2263,12 +2640,15 @@ TEST_CASE("!/redwood/correctness") { printf("Starting from version: %lld\n", lastVer); state Version version = lastVer + 1; - state int mutationBytes = 0; - btree->setWriteVersion(version); - state int64_t keyBytesInserted = 0; - state int64_t ValueBytesInserted = 0; + state SimpleCounter mutationBytes; + state SimpleCounter keyBytesInserted; + state SimpleCounter valueBytesInserted; + state SimpleCounter sets; + state SimpleCounter rangeClears; + state SimpleCounter keyBytesCleared; + state SimpleCounter valueBytesCleared; state int errorCount; state PromiseStream committedVersions; @@ -2277,7 +2657,7 @@ TEST_CASE("!/redwood/correctness") { state Future commit = Void(); - while(mutationBytes < mutationBytesTarget) { + while(mutationBytes.get() < mutationBytesTarget) { // Sometimes advance the version if(g_random->random01() < 0.10) { ++version; @@ -2307,6 +2687,7 @@ TEST_CASE("!/redwood/correctness") { std::swap(end, start); } + ++rangeClears; KeyRangeRef range(start, end); debug_printf(" Clear '%s' to '%s' @%lld\n", start.toString().c_str(), end.toString().c_str(), version); auto e = written.lower_bound(std::make_pair(start.toString(), 0)); @@ -2319,13 +2700,17 @@ TEST_CASE("!/redwood/correctness") { // If e key is different from last and last was present then insert clear for last's key at version if(last != eEnd && ((e == eEnd || e->first.first != last->first.first) && last->second.present())) { debug_printf(" Clearing key '%s' @%lld\n", last->first.first.c_str(), version); + + keyBytesCleared += last->first.first.size(); + valueBytesCleared += last->second.get().size(); mutationBytes += (last->first.first.size() + last->second.get().size()); + // If the last set was at version then just make it not present if(last->first.second == version) { - last->second = Optional(); + last->second.reset(); } else { - written[std::make_pair(last->first.first, version)] = Optional(); + written[std::make_pair(last->first.first, version)].reset(); } } last = e; @@ -2343,17 +2728,21 @@ TEST_CASE("!/redwood/correctness") { if(i != keys.end()) kv.key = StringRef(kv.arena(), *i); } - keyBytesInserted += kv.key.size(); - ValueBytesInserted += kv.value.size(); - mutationBytes += (kv.key.size() + kv.value.size()); + debug_printf(" Set '%s' -> '%s' @%lld\n", kv.key.toString().c_str(), kv.value.toString().c_str(), version); + + ++sets; + keyBytesInserted += kv.key.size(); + valueBytesInserted += kv.value.size(); + mutationBytes += (kv.key.size() + kv.value.size()); + btree->set(kv); written[std::make_pair(kv.key.toString(), version)] = kv.value.toString(); keys.insert(kv.key); } - // Sometimes (and at end) commit then check all results - if(mutationBytes >= std::min(mutationBytesTarget, (int)20e6) || g_random->random01() < .002) { + // Sometimes (and at end) commit + if(mutationBytes.get() >= mutationBytesTarget || g_random->random01() < .002) { // Wait for btree commit and send the new version to committedVersions. // Avoid capture of version as a member of *this Version v = version; @@ -2363,7 +2752,15 @@ TEST_CASE("!/redwood/correctness") { return Void(); }); - printf("Cumulative: %d total mutation bytes, %lu key changes, %lld key bytes, %lld value bytes\n", mutationBytes, written.size(), keyBytesInserted, ValueBytesInserted); + printf("Committing. Stats: sets %s clears %s setKey %s setVal %s clearKey %s clearVal %s mutations %s\n", + sets.toString().c_str(), + rangeClears.toString().c_str(), + keyBytesInserted.toString().c_str(), + valueBytesInserted.toString().c_str(), + keyBytesCleared.toString().c_str(), + valueBytesCleared.toString().c_str(), + mutationBytes.toString().c_str() + ); // Recover from disk at random if(useDisk && g_random->random01() < .1) { @@ -2404,7 +2801,6 @@ TEST_CASE("!/redwood/correctness") { ++version; btree->setWriteVersion(version); } - } debug_printf("Waiting for outstanding commit\n"); @@ -2431,7 +2827,7 @@ TEST_CASE("!/redwood/performance/set") { state int nodeCount = 10000000; state int maxChangesPerVersion = 1000; - state int versions = 5000; + state int versions = 15000; int maxKeySize = 50; int maxValueSize = 100; diff --git a/flow/flow.h b/flow/flow.h index 3118a52eeb..bf9b7b7183 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -142,11 +142,15 @@ public: if (valid) ((T*)&value)->~T(); } - Optional & operator=(Optional const& o) { + void reset() { if (valid) { valid = false; ((T*)&value)->~T(); } + } + + Optional & operator=(Optional const& o) { + reset(); if (o.valid) { new (&value) T(o.get()); valid = true; From 767de5aa4d8434442299efbc8ef9da3da30975eb Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Sun, 24 Feb 2019 03:47:32 -0800 Subject: [PATCH 003/128] Redwood unit test improvements. Correctness errors now cause test to terminate earlier, and errors occurring after final commit are no longer ignored. Fixed bug in reverse iterator movement in Cursor. Several debug output improvements. --- fdbserver/VersionedBTree.actor.cpp | 159 +++++++++++++++++------------ 1 file changed, 96 insertions(+), 63 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index d8dc24ad96..881f28caae 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -1728,11 +1728,15 @@ private: std::string toString() const { std::string r; r += format("Cursor(%p) ver: %lld ", this, m_version); - r += format(" KV: '%s' -> '%s'\n", - m_kv.present() ? m_kv.get().key.printable().c_str() : "", - m_kv.present() ? m_kv.get().value.printable().c_str() : ""); + if(m_kv.present()) { + r += format(" KV: '%s' -> '%s'\n", m_kv.get().key.printable().c_str(), m_kv.get().value.printable().c_str()); + } + else { + r += " KV: "; + } r += format(" Cur1: %s\n", m_cur1.toString().c_str()); r += format(" Cur2: %s\n", m_cur2.toString().c_str()); + return r; } @@ -1830,10 +1834,9 @@ private: // record is in effect at that version but it could indicate that the key was cleared and // no longer exists from the user's perspective at that version // - // If moving forward, cur2 must be the record after cur1 so we can determine if - // cur1 is to be returned below. - // If moving backward, cur2 is already the record after cur1 - if(fwd && self->m_cur1.valid()) { + // cur2 must be the record immediately after cur1 + // TODO: This may already be the case, store state to track this condition and avoid the reset here + if(self->m_cur1.valid()) { self->m_cur2 = self->m_cur1; wait(success(self->m_cur2.move(true))); } @@ -1850,11 +1853,13 @@ private: if(fwd) { // Moving forward, move cur2 forward and keep cur1 pointing to the prior (predecessor) record + debug_printf("Cursor::move(%d): Moving forward, Cursor = %s\n", fwd, self->toString().c_str()); self->m_cur1 = self->m_cur2; wait(success(self->m_cur2.move(true))); } else { // Moving backward, move cur1 backward and keep cur2 pointing to the prior (successor) record + debug_printf("Cursor::move(%d): Moving backward, Cursor = %s\n", fwd, self->toString().c_str()); self->m_cur2 = self->m_cur1; wait(success(self->m_cur1.move(false))); } @@ -1862,6 +1867,7 @@ private: } self->m_kv.reset(); + debug_printf("Cursor::move(%d): Exit, end of db reached. Cursor = %s\n", fwd, self->toString().c_str()); return Void(); } @@ -1874,9 +1880,9 @@ private: // Unsplit value, cur1 will hold the key and value memory if(!rec.isMultiPart()) { + self->m_kv = KeyValueRef(rec.key, rec.value.get()); debug_printf("readFullKVPair: Unsplit, exit. %s\n", self->toString().c_str()); - self->m_kv = KeyValueRef(rec.key, rec.value.get()); return Void(); } @@ -2116,7 +2122,7 @@ KeyValue randomKV(int keySize = 10, int valueSize = 5) { return kv; } -ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::map, Optional> *written) { +ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::map, Optional> *written, int *pErrorCount) { state int errors = 0; state Key start = randomKV().key; state Key end = randomKV().key; @@ -2161,18 +2167,21 @@ ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::mapgetKey().toString().c_str()); break; } if(cur->getKey() != iLast->first.first) { - errors += 1; + ++errors; + ++*pErrorCount; printf("VerifyRange(@%lld, %s, %s) ERROR: Tree key '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), iLast->first.first.c_str()); break; } if(cur->getValue() != iLast->second.get()) { - errors += 1; + ++errors; + ++*pErrorCount; printf("VerifyRange(@%lld, %s, %s) ERROR: Tree key '%s' has tree value '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), cur->getValue().toString().c_str(), iLast->second.get().c_str()); break; } @@ -2199,7 +2208,8 @@ ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::mapfirst.second, iLast->first.first.c_str()); } @@ -2218,18 +2228,21 @@ ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::mapisValid() && cur->getKey() >= start) { if(r == results.rend()) { - errors += 1; + ++errors; + ++*pErrorCount; printf("VerifyRangeReverse(@%lld, %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str()); break; } if(cur->getKey() != r->key) { - errors += 1; + ++errors; + ++*pErrorCount; printf("VerifyRangeReverse(@%lld, %s, %s) ERROR: Tree key '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), r->key.toString().c_str()); break; } if(cur->getValue() != r->value) { - errors += 1; + ++errors; + ++*pErrorCount; printf("VerifyRangeReverse(@%lld, %s, %s) ERROR: Tree key '%s' has tree value '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), cur->getValue().toString().c_str(), r->value.toString().c_str()); break; } @@ -2239,14 +2252,15 @@ ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::mapkey.toString().c_str()); } return errors; } -ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, std::map, Optional> *written) { +ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, std::map, Optional> *written, int *pErrorCount) { // Read back every key at every version set or cleared and verify the result. state std::map, Optional>::const_iterator i = written->cbegin(); state std::map, Optional>::const_iterator iEnd = written->cend(); @@ -2267,6 +2281,7 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, if(val.present()) { if(!(cur->isValid() && cur->getKey() == key && cur->getValue() == val.get())) { ++errors; + ++*pErrorCount; if(!cur->isValid()) printf("Verify ERROR: key_not_found: '%s' -> '%s' @%lld\n", key.c_str(), val.get().c_str(), ver); else if(cur->getKey() != key) @@ -2277,6 +2292,7 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, } else { if(cur->isValid() && cur->getKey() == key) { ++errors; + ++*pErrorCount; printf("Verify ERROR: cleared_key_found: '%s' -> '%s' @%lld\n", key.c_str(), cur->getValue().toString().c_str(), ver); } } @@ -2292,12 +2308,11 @@ ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, state Version v = waitNext(vStream); debug_printf("Verifying through version %lld\n", v); - state Future vall = verifyAll(btree, v, written); - state Future vrange = verifyRandomRange(btree, g_random->randomInt(1, v + 1), written); + state Future vall = verifyAll(btree, v, written, pErrorCount); + state Future vrange = verifyRandomRange(btree, g_random->randomInt(1, v + 1), written, pErrorCount); wait(success(vall) && success(vrange)); int errors = vall.get() + vrange.get(); - *pErrorCount += errors; debug_printf("Verified through version %lld, %d errors\n", v, errors); @@ -2611,6 +2626,7 @@ TEST_CASE("!/redwood/correctness") { IPager *pager; if(useDisk) { + printf("Deleting existing test data...\n"); deleteFile(pagerFile); deleteFile(pagerFile + "0.pagerlog"); deleteFile(pagerFile + "1.pagerlog"); @@ -2619,6 +2635,7 @@ TEST_CASE("!/redwood/correctness") { else pager = createMemoryPager(); + printf("Initializing...\n"); state int pageSize = g_random->coinflip() ? pager->getUsablePageSize() : g_random->randomInt(200, 400); state VersionedBTree *btree = new VersionedBTree(pager, pagerFile, pageSize); wait(btree->init()); @@ -2628,10 +2645,11 @@ TEST_CASE("!/redwood/correctness") { // TODO: Handle arbitrarily large keys state int maxKeySize = g_random->randomInt(4, pageSize * 2); state int maxValueSize = g_random->randomInt(0, pageSize * 2); + state int maxCommitSize = 5e6; + state int mutationBytesTarget = randomSize(50e6); + state double clearChance = g_random->random01() * .001; // at most 1 in 1000 - state int mutationBytesTarget = g_random->randomInt(100, (maxKeySize + maxValueSize) * 2000); - - printf("Using page size %d, max key size %d, max value size %d, total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, mutationBytesTarget); + printf("Using page size %d, max key size %d, max value size %d, clearchance %f total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, clearChance, mutationBytesTarget); state std::map, Optional> written; state std::set keys; @@ -2648,8 +2666,9 @@ TEST_CASE("!/redwood/correctness") { state SimpleCounter sets; state SimpleCounter rangeClears; state SimpleCounter keyBytesCleared; - state SimpleCounter valueBytesCleared; state int errorCount; + state int mutationBytesThisCommit = 0; + state int mutationBytesTargetThisCommit = randomSize(maxCommitSize); state PromiseStream committedVersions; state Future verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount); @@ -2665,7 +2684,7 @@ TEST_CASE("!/redwood/correctness") { } // Sometimes do a clear range - if(g_random->random01() < .10) { + if(g_random->random01() < clearChance) { Key start = randomKV(maxKeySize, 1).key; Key end = (g_random->random01() < .01) ? keyAfter(start) : randomKV(maxKeySize, 1).key; @@ -2702,8 +2721,8 @@ TEST_CASE("!/redwood/correctness") { debug_printf(" Clearing key '%s' @%lld\n", last->first.first.c_str(), version); keyBytesCleared += last->first.first.size(); - valueBytesCleared += last->second.get().size(); - mutationBytes += (last->first.first.size() + last->second.get().size()); + mutationBytes += last->first.first.size(); + mutationBytesThisCommit += last->first.first.size(); // If the last set was at version then just make it not present if(last->first.second == version) { @@ -2735,35 +2754,40 @@ TEST_CASE("!/redwood/correctness") { keyBytesInserted += kv.key.size(); valueBytesInserted += kv.value.size(); mutationBytes += (kv.key.size() + kv.value.size()); + mutationBytesThisCommit += (kv.key.size() + kv.value.size()); btree->set(kv); written[std::make_pair(kv.key.toString(), version)] = kv.value.toString(); keys.insert(kv.key); } - // Sometimes (and at end) commit - if(mutationBytes.get() >= mutationBytesTarget || g_random->random01() < .002) { - // Wait for btree commit and send the new version to committedVersions. - // Avoid capture of version as a member of *this - Version v = version; - commit = map(commit && btree->commit(), [=](Void) { + // Commit at end or after this commit's mutation bytes are reached + if(mutationBytes.get() >= mutationBytesTarget || mutationBytesThisCommit >= mutationBytesTargetThisCommit) { + // Wait for previous commit to finish + wait(commit); + printf("Committed. Next commit %d bytes, %lld/%d (%.2f%%) Stats: Insert %.2f MB/s ClearedKeys %.2f MB/s Total %.2f\n", + mutationBytesThisCommit, + mutationBytes.get(), + mutationBytesTarget, + (double)mutationBytes.get() / mutationBytesTarget * 100, + (keyBytesInserted.rate() + valueBytesInserted.rate()) / 1e6, + keyBytesCleared.rate() / 1e6, + mutationBytes.rate() / 1e6 + ); + + Version v = version; // Avoid capture of version as a member of *this + + commit = map(btree->commit(), [=](Void) { // Notify the background verifier that version is committed and therefore readable committedVersions.send(v); return Void(); }); - printf("Committing. Stats: sets %s clears %s setKey %s setVal %s clearKey %s clearVal %s mutations %s\n", - sets.toString().c_str(), - rangeClears.toString().c_str(), - keyBytesInserted.toString().c_str(), - valueBytesInserted.toString().c_str(), - keyBytesCleared.toString().c_str(), - valueBytesCleared.toString().c_str(), - mutationBytes.toString().c_str() - ); + mutationBytesThisCommit = 0; + mutationBytesTargetThisCommit = randomSize(maxCommitSize); // Recover from disk at random - if(useDisk && g_random->random01() < .1) { + if(useDisk && g_random->random01() < .02) { printf("Recovering from disk.\n"); // Wait for outstanding commit @@ -2794,13 +2818,13 @@ TEST_CASE("!/redwood/correctness") { randomTask = randomReader(btree) || btree->getError(); } - // Check for errors - if(errorCount != 0) - throw internal_error(); - ++version; btree->setWriteVersion(version); } + + // Check for errors + if(errorCount != 0) + throw internal_error(); } debug_printf("Waiting for outstanding commit\n"); @@ -2809,6 +2833,10 @@ TEST_CASE("!/redwood/correctness") { debug_printf("Waiting for verification to complete.\n"); wait(verifyTask); + // Check for errors + if(errorCount != 0) + throw internal_error(); + Future closedFuture = btree->onClosed(); btree->close(); wait(closedFuture); @@ -2818,6 +2846,7 @@ TEST_CASE("!/redwood/correctness") { TEST_CASE("!/redwood/performance/set") { state std::string pagerFile = "unittest_pageFile"; + printf("Deleting old test data\n"); deleteFile(pagerFile); deleteFile(pagerFile + "0.pagerlog"); deleteFile(pagerFile + "1.pagerlog"); @@ -2825,11 +2854,11 @@ TEST_CASE("!/redwood/performance/set") { state VersionedBTree *btree = new VersionedBTree(pager, "unittest_pageFile"); wait(btree->init()); - state int nodeCount = 10000000; + state int nodeCount = 10e6; state int maxChangesPerVersion = 1000; state int versions = 15000; - int maxKeySize = 50; - int maxValueSize = 100; + int maxKeySize = 25; + int maxValueSize = 75; state std::string key(maxKeySize, 'k'); state std::string value(maxKeySize, 'v'); @@ -2837,7 +2866,9 @@ TEST_CASE("!/redwood/performance/set") { state int records = 0; state Future commit = Void(); - state double startTime = now(); + printf("Starting.\n"); + state double intervalStart = timer(); + while(--versions) { Version lastVer = wait(btree->getLatestVersion()); state Version version = lastVer + 1; @@ -2854,25 +2885,27 @@ TEST_CASE("!/redwood/performance/set") { ++records; } - if(g_random->random01() < (1.0 / 1000)) { + if(kvBytes > 5e6 || versions == 1) { wait(commit); - commit = btree->commit(); - wait(commit); - double elapsed = now() - startTime; - printf("Committed (cumulative) %lld bytes in %d records in %f seconds, %.2f MB/s %s\n", kvBytes, records, elapsed, kvBytes / elapsed / 1e6, btree->counts.toString().c_str()); + // Avoid capturing this to freeze counter values + int recs = records; + int kvb = kvBytes; + commit = map(btree->commit(), [=](Void result) { + double elapsed = timer() - intervalStart; + printf("Committed %d kvBytes in %d records in %f seconds, %.2f MB/s\n", kvb, recs, elapsed, kvb / elapsed / 1e6); + intervalStart = timer(); + return Void(); + }); + records = 0; + kvBytes = 0; } } - wait(btree->commit()); - double elapsed = now() - startTime; - printf("Committed (cumulative) %lld bytes in %d records in %f seconds, %.2f MB/s %s\n", kvBytes, records, elapsed, kvBytes / elapsed / 1e6, btree->counts.toString().c_str()); + wait(commit); Future closedFuture = btree->onClosed(); btree->close(); wait(closedFuture); - double elapsed = now() - startTime; - printf("Wrote (final) %lld bytes in %d records in %f seconds, %.2f MB/s\n", kvBytes, records, elapsed, kvBytes / elapsed / 1e6); - return Void(); } From 126252a2740ffcd6ef1feb95ee53258bcfaec2ad Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Fri, 15 Mar 2019 00:46:09 -0700 Subject: [PATCH 004/128] Changed checksum to crc32. Disabled pager housekeeping for now. Added more btree read/write/commit metrics. Changed readPage to use disk read priority. Bug fix in CommitSubtree causing it to recurse to children unnecessarily. Added point read speed test at the end of set performance unit test. --- fdbserver/IndirectShadowPager.actor.cpp | 14 +-- fdbserver/VersionedBTree.actor.cpp | 140 +++++++++++++++++------- 2 files changed, 109 insertions(+), 45 deletions(-) diff --git a/fdbserver/IndirectShadowPager.actor.cpp b/fdbserver/IndirectShadowPager.actor.cpp index d76b2c289f..b9b436be04 100644 --- a/fdbserver/IndirectShadowPager.actor.cpp +++ b/fdbserver/IndirectShadowPager.actor.cpp @@ -23,12 +23,12 @@ #include "flow/UnitTest.h" #include "flow/actorcompiler.h" +#include "fdbrpc/crc32c.h" struct SumType { - bool operator==(const SumType &rhs) const { return part1 == rhs.part1 && part2 == rhs.part2; } - uint32_t part1; - uint32_t part2; - std::string toString() { return format("0x%08x%08x", part1, part2); } + bool operator==(const SumType &rhs) const { return crc == rhs.crc; } + uint32_t crc; + std::string toString() { return format("0x%08x", crc); } }; bool checksum(IAsyncFile *file, uint8_t *page, int pageSize, LogicalPageID logical, PhysicalPageID physical, bool write) { @@ -43,9 +43,7 @@ bool checksum(IAsyncFile *file, uint8_t *page, int pageSize, LogicalPageID logic SumType *pSumInPage = (SumType *)(page + pageSize); // Write sum directly to page or to sum variable based on mode SumType *sumOut = write ? pSumInPage : ∑ - sumOut->part1 = physical; - sumOut->part2 = logical; - hashlittle2(page, pageSize, &sumOut->part1, &sumOut->part2); + sumOut->crc = crc32c_append(logical, page, pageSize); VALGRIND_MAKE_MEM_DEFINED(sumOut, sizeof(SumType)); debug_printf("checksum %s%s logical %d physical %d size %d checksums page %s calculated %s data at %p %s\n", @@ -276,7 +274,7 @@ ACTOR Future recover(IndirectShadowPager *pager) { ACTOR Future housekeeper(IndirectShadowPager *pager) { wait(pager->recovery); - + wait(Never()); loop { state LogicalPageID pageID = 0; for(; pageID < pager->pageTable.size(); ++pageID) { diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 881f28caae..a5e14b29e9 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -430,8 +430,15 @@ static std::vector buildPages(bool minimalBoundaries, const Red // If flush then write a page using records from start to i. It's guaranteed that pageUpperBound has been set above. if(flush) { end = i == iEnd; // i could have been moved above + int count = i - start; - debug_printf("Flushing page start=%d i=%d\nlower: %s\nupper: %s\n", start, i, pageLowerBound.toString().c_str(), pageUpperBound.toString().c_str()); + // If not writing the final page, reduce entry count of page by a third + if(!end) { + i -= count / 3; + pageUpperBound = entries[i]; + } + + debug_printf("Flushing page start=%d i=%d count=%d\nlower: %s\nupper: %s\n", start, i, count, pageLowerBound.toString().c_str(), pageUpperBound.toString().c_str()); #if REDWOOD_DEBUG for(int j = start; j < i; ++j) { debug_printf(" %3d: %s\n", j, entries[j].toString().c_str()); @@ -527,19 +534,31 @@ public: *this = Counts(); } + int64_t pageReads; + int64_t extPageReads; + int64_t setBytes; int64_t pageWrites; - int64_t blockWrites; + int64_t extPageWrites; int64_t sets; int64_t clears; int64_t commits; + int64_t gets; + int64_t getRanges; + int64_t commitToPage; + int64_t commitToPageStart; - std::string toString() const { - std::string s = format("sets=%lld clears=%lld commits=%lld pages=%lld blocks=%lld\n", sets, clears, commits, pageWrites, blockWrites); + std::string toString(bool clearAfter = false) { + std::string s = format("set=%lld clear=%lld get=%lld getRange=%lld commit=%lld pageRead=%lld extPageRead=%lld pageWrite=%lld extPageWrite=%lld commitPage=%lld commitPageStart=%lld", + sets, clears, gets, getRanges, commits, pageReads, extPageReads, pageWrites, extPageWrites, commitToPage, commitToPageStart); + if(clearAfter) { + clear(); + } return s; } }; - Counts counts; + // Using a static for metrics because a single process shouldn't normally have multiple storage engines + static Counts counts; // All async opts on the btree are based on pager reads, writes, and commits, so // we can mostly forward these next few functions to the pager @@ -934,7 +953,7 @@ private: newPage->extensionPages[e] = eid; // If replacing the primary page below (version == 0) then pass the primary page's ID as the reference page ID m_pager->writePage(eid, extPages[e], version, (version == 0) ? id : invalidLogicalPageID); - ++counts.blockWrites; + ++counts.extPageWrites; } debug_printf("%p: writePages(): Writing primary page op=write id=%u @%lld (+%lu extension pages)\n", actor_debug, id, version, extPages.size()); @@ -943,7 +962,6 @@ private: else { debug_printf("%p: writePages(): Writing normal page op=write id=%u @%lld\n", actor_debug, id, version); writePage(id, pages[i].firstPage, version, &pages[i].lowerBound, (i == pages.size() - 1) ? upperBound : &pages[i + 1].lowerBound); - ++counts.blockWrites; } } @@ -1001,8 +1019,10 @@ private: ACTOR static Future> readPage(Reference snapshot, LogicalPageID id, int usablePageSize, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { debug_printf("readPage() op=read id=%u @%lld\n", id, snapshot->getVersion()); + wait(delay(0, TaskDiskRead)); state Reference result = wait(snapshot->getPhysicalPage(id)); + ++counts.pageReads; state const BTreePage *pTreePage = (const BTreePage *)result->begin(); if(pTreePage->extensionPageCount == 0) { @@ -1018,6 +1038,7 @@ private: } std::vector> pages = wait(getAll(pageGets)); + counts.extPageReads += pTreePage->extensionPageCount; result = Reference(new SuperPage(pages, usablePageSize)); pTreePage = (const BTreePage *)result->begin(); } @@ -1039,20 +1060,21 @@ private: // TODO: Probably should pass prev/next records by pointer in many places ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", this, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); + self->counts.commitToPageStart++; + + // If the lower bound key and the upper bound key are the same then there can't be any changes to + // this subtree since changes would happen after the upper bound key as the mutated versions would + // necessarily be higher than all previous versions + if(lowerBound->key == upperBound->key) { + debug_printf("%p no changes, lower and upper bound keys are the same.\n", this); + return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); + } // Find the slice of the mutation buffer that is relevant to this subtree // TODO: Rather than two lower_bound searches, perhaps just compare each mutation to the upperBound key while iterating state MutationBufferT::const_iterator iMutationBoundary = mutationBuffer->lower_bound(lowerBound->key); state MutationBufferT::const_iterator iMutationBoundaryEnd = mutationBuffer->lower_bound(upperBound->key); - // If the lower bound key and the upper bound key are the same then there can't be any changes to - // this subtree since changes would happen after the upper bound key as the mutated versions would - // necessarily be higher. - if(lowerBound->key == upperBound->key) { - debug_printf("%p no changes, lower and upper bound keys are the same.\n", this); - return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); - } - // If the mutation buffer key found is greater than the lower bound key then go to the previous mutation // buffer key because it may cover deletion of some keys at the start of this subtree. if(iMutationBoundary != mutationBuffer->begin() && iMutationBoundary->first > lowerBound->key) { @@ -1073,11 +1095,17 @@ private: // subtree but have no changes in it MutationBufferT::const_iterator iMutationBoundaryNext = iMutationBoundary; ++iMutationBoundaryNext; - if(iMutationBoundaryNext == iMutationBoundaryEnd && iMutationBoundary->second.noChanges()) { - debug_printf("%p no changes because sole mutation range was empty\n", this); + if(iMutationBoundaryNext == iMutationBoundaryEnd && + ( iMutationBoundary->second.noChanges() || + ( !iMutationBoundary->second.rangeClearVersion.present() && + iMutationBoundary->first < lowerBound->key) + ) + ) { + debug_printf("%p no changes because sole mutation range was not cleared\n", this); return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } + self->counts.commitToPage++; state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride, lowerBound, upperBound)); state BTreePage *page = (BTreePage *) rawPage->begin(); debug_printf("%p commitSubtree(): %s\n", this, page->toString(false, root, snapshot->getVersion(), lowerBound, upperBound).c_str()); @@ -1089,16 +1117,18 @@ private: if(page->flags & BTreePage::IS_LEAF) { VersionedChildrenT results; std::vector merged; - Arena mergedArena; debug_printf("%p MERGING EXISTING DATA WITH MUTATIONS:\n", this); - self->printMutationBuffer(iMutationBoundary, iMutationBoundaryEnd); + if(REDWOOD_DEBUG) { + self->printMutationBuffer(iMutationBoundary, iMutationBoundaryEnd); + } // It's a given that the mutation map is not empty so it's safe to do this Key mutationRangeStart = iMutationBoundary->first; // If replacement pages are written they will be at the minimum version seen in the mutations for this leaf Version minVersion = invalidVersion; + int changes = 0; // Now, process each mutation range and merge changes with existing data. while(iMutationBoundary != iMutationBoundaryEnd) { @@ -1134,12 +1164,14 @@ private: if(m.isClear() || m.value.size() <= maxPartSize) { if(iMutations->first < minVersion || minVersion == invalidVersion) minVersion = iMutations->first; + ++changes; merged.push_back(iMutations->second.toRecord(iMutationBoundary->first, iMutations->first)); debug_printf("%p: Added non-split %s [mutation, boundary start]\n", this, merged.back().toString().c_str()); } else { if(iMutations->first < minVersion || minVersion == invalidVersion) minVersion = iMutations->first; + ++changes; int bytesLeft = m.value.size(); int start = 0; RedwoodRecordRef whole(iMutationBoundary->first, iMutations->first, m.value); @@ -1177,6 +1209,7 @@ private: Version clearVersion = clearRangeVersion.get(); if(clearVersion < minVersion || minVersion == invalidVersion) minVersion = clearVersion; + ++changes; merged.push_back(RedwoodRecordRef(cursor.get().key, clearVersion)); debug_printf("%p: Added %s [existing, middle clear]\n", this, merged.back().toString().c_str()); } @@ -1191,12 +1224,12 @@ private: cursor.moveNext(); } - debug_printf("%p Done merging mutations into existing leaf contents\n", this); + debug_printf("%p Done merging mutations into existing leaf contents, made %d changes\n", this, changes); - // No changes were actually made. This could happen if there is a clear which does not cover an entire leaf but also does - // not which turns out to not match any existing data in the leaf. + // No changes were actually made. This could happen if the only mutations are clear ranges which do not match any records. if(minVersion == invalidVersion) { debug_printf("%p No changes were made during mutation merge\n", this); + ASSERT(changes == 0); return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } @@ -1265,7 +1298,8 @@ private: childPageIDs.push_back(pageID); } - //wait(waitForAll(futureChildren)); + // Waiting one at a time makes debugging easier + // TODO: Is it better to use waitForAll()? state int k; for(k = 0; k < futureChildren.size(); ++k) { wait(success(futureChildren[k])); @@ -1413,7 +1447,9 @@ private: Version latestVersion = wait(self->m_pager->getLatestVersion()); debug_printf("%s: pager latestVersion %lld\n", self->m_name.c_str(), latestVersion); - self->printMutationBuffer(mutations); + if(REDWOOD_DEBUG) { + self->printMutationBuffer(mutations); + } VersionedChildrenT _ = wait(commitSubtree(self, mutations, self->m_pager->getReadSnapshot(latestVersion), self->m_root, &dbBegin, &dbEnd)); @@ -1922,6 +1958,7 @@ private: RedwoodRecordRef VersionedBTree::dbBegin(StringRef(), 0); RedwoodRecordRef VersionedBTree::dbEnd(LiteralStringRef("\xff\xff\xff\xff"), std::numeric_limits::max()); +VersionedBTree::Counts VersionedBTree::counts; ACTOR template Future catchError(Promise error, Future f) { @@ -2005,15 +2042,22 @@ public: }; void clear(KeyRangeRef range, const Arena* arena = 0) { + debug_printf("CLEAR %s\n", printable(range).c_str()); m_tree->clear(range); } virtual void set( KeyValueRef keyValue, const Arena* arena = NULL ) { - //printf("SET write version %lld %s\n", m_tree->getWriteVersion(), printable(keyValue).c_str()); + debug_printf("SET %s\n", keyValue.key.printable().c_str()); m_tree->set(keyValue); } + virtual Future< Standalone< VectorRef< KeyValueRef > > > readRange(KeyRangeRef keys, int rowLimit = 1<<30, int byteLimit = 1<<30) { + debug_printf("READRANGE %s\n", printable(keys).c_str()); + return catchError(readRange_impl(this, keys, rowLimit, byteLimit)); + } + ACTOR static Future< Standalone< VectorRef< KeyValueRef > > > readRange_impl(KeyValueStoreRedwoodUnversioned *self, KeyRange keys, int rowLimit, int byteLimit) { + self->m_tree->counts.getRanges++; state Standalone> result; state int accumulatedBytes = 0; ASSERT( byteLimit > 0 ); @@ -2050,11 +2094,8 @@ public: return result; } - virtual Future< Standalone< VectorRef< KeyValueRef > > > readRange(KeyRangeRef keys, int rowLimit = 1<<30, int byteLimit = 1<<30) { - return catchError(readRange_impl(this, keys, rowLimit, byteLimit)); - } - ACTOR static Future< Optional > readValue_impl(KeyValueStoreRedwoodUnversioned *self, Key key, Optional< UID > debugID) { + self->m_tree->counts.gets++; state Reference cur = self->m_tree->readAtVersion(self->m_tree->getLastCommittedVersion()); state Version readVersion = self->m_tree->getLastCommittedVersion(); @@ -2070,6 +2111,7 @@ public: } ACTOR static Future< Optional > readValuePrefix_impl(KeyValueStoreRedwoodUnversioned *self, Key key, int maxLength, Optional< UID > debugID) { + self->m_tree->counts.gets++; state Reference cur = self->m_tree->readAtVersion(self->m_tree->getLastCommittedVersion()); wait(cur->findEqual(key)); @@ -2649,7 +2691,7 @@ TEST_CASE("!/redwood/correctness") { state int mutationBytesTarget = randomSize(50e6); state double clearChance = g_random->random01() * .001; // at most 1 in 1000 - printf("Using page size %d, max key size %d, max value size %d, clearchance %f total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, clearChance, mutationBytesTarget); + printf("Using page size %d, max key size %d, max value size %d, clearchance %f, total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, clearChance, mutationBytesTarget); state std::map, Optional> written; state std::set keys; @@ -2844,6 +2886,25 @@ TEST_CASE("!/redwood/correctness") { return Void(); } +ACTOR Future pointReads(VersionedBTree *btree, int count, int nodeCount) { + state Version readVer = wait(btree->getLatestVersion()); + state int c = 0; + state double readStart = timer(); + printf("Executing %d point reads\n", count); + state Key k = makeString(4); + state Reference cur = btree->readAtVersion(readVer); + while(c < count) { + //cur = btree->readAtVersion(readVer); + *(uint32_t *)k.begin() = g_random->randomInt(0, nodeCount); + wait(success(cur->findFirstEqualOrGreater(k, true, 0))); + ++c; + } + double elapsed = timer() - readStart; + printf("Point read speed %d/s\n", int(count / elapsed)); + return Void(); +} + + TEST_CASE("!/redwood/performance/set") { state std::string pagerFile = "unittest_pageFile"; printf("Deleting old test data\n"); @@ -2854,22 +2915,23 @@ TEST_CASE("!/redwood/performance/set") { state VersionedBTree *btree = new VersionedBTree(pager, "unittest_pageFile"); wait(btree->init()); - state int nodeCount = 10e6; + state int nodeCount = 1e9; state int maxChangesPerVersion = 1000; - state int versions = 15000; - int maxKeySize = 25; - int maxValueSize = 75; + state int64_t kvBytesTarget = 100e6; + int maxKeySize = 19; + int maxValueSize = 1000; state std::string key(maxKeySize, 'k'); - state std::string value(maxKeySize, 'v'); + state std::string value(maxValueSize, 'v'); state int64_t kvBytes = 0; + state int64_t kvBytesTotal = 0; state int records = 0; state Future commit = Void(); printf("Starting.\n"); state double intervalStart = timer(); - while(--versions) { + while(kvBytesTotal < kvBytesTarget) { Version lastVer = wait(btree->getLatestVersion()); state Version version = lastVer + 1; btree->setWriteVersion(version); @@ -2882,10 +2944,11 @@ TEST_CASE("!/redwood/performance/set") { kv.value = StringRef((uint8_t *)value.data(), g_random->randomInt(0, value.size())); btree->set(kv); kvBytes += kv.key.size() + kv.value.size(); + kvBytesTotal += kv.key.size() + kv.value.size(); ++records; } - if(kvBytes > 5e6 || versions == 1) { + if(kvBytes > 1e6 || kvBytesTotal >= kvBytesTarget) { wait(commit); // Avoid capturing this to freeze counter values int recs = records; @@ -2903,6 +2966,9 @@ TEST_CASE("!/redwood/performance/set") { wait(commit); + state int reads = 3e5; + wait(pointReads(btree, reads, nodeCount) && pointReads(btree, reads, nodeCount) && pointReads(btree, reads, nodeCount)); + Future closedFuture = btree->onClosed(); btree->close(); wait(closedFuture); From 64554e90d4eb59cbff2ab7ef404f3f95c5b65953 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 27 Mar 2019 13:42:49 -0700 Subject: [PATCH 005/128] Change this to THIS in actors for IDE compatibility. --- fdbserver/VersionedBTree.actor.cpp | 70 +++++++++++++++--------------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index a5e14b29e9..31aa6e2be8 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -1059,14 +1059,14 @@ private: // Returns list of (version, list of (lower_bound, list of children) ) // TODO: Probably should pass prev/next records by pointer in many places ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { - debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", this, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); + debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", THIS, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); self->counts.commitToPageStart++; // If the lower bound key and the upper bound key are the same then there can't be any changes to // this subtree since changes would happen after the upper bound key as the mutated versions would // necessarily be higher than all previous versions if(lowerBound->key == upperBound->key) { - debug_printf("%p no changes, lower and upper bound keys are the same.\n", this); + debug_printf("%p no changes, lower and upper bound keys are the same.\n", THIS); return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } @@ -1083,7 +1083,7 @@ private: else { // If the there are no mutations, we're done if(iMutationBoundary == iMutationBoundaryEnd) { - debug_printf("%p no changes, mutation buffer start/end are the same\n", this); + debug_printf("%p no changes, mutation buffer start/end are the same\n", THIS); return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } } @@ -1101,14 +1101,14 @@ private: iMutationBoundary->first < lowerBound->key) ) ) { - debug_printf("%p no changes because sole mutation range was not cleared\n", this); + debug_printf("%p no changes because sole mutation range was not cleared\n", THIS); return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } self->counts.commitToPage++; state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride, lowerBound, upperBound)); state BTreePage *page = (BTreePage *) rawPage->begin(); - debug_printf("%p commitSubtree(): %s\n", this, page->toString(false, root, snapshot->getVersion(), lowerBound, upperBound).c_str()); + debug_printf("%p commitSubtree(): %s\n", THIS, page->toString(false, root, snapshot->getVersion(), lowerBound, upperBound).c_str()); BTreePage::BinaryTree::Cursor cursor = getReader(rawPage)->getCursor(); cursor.moveFirst(); @@ -1118,7 +1118,7 @@ private: VersionedChildrenT results; std::vector merged; - debug_printf("%p MERGING EXISTING DATA WITH MUTATIONS:\n", this); + debug_printf("%p MERGING EXISTING DATA WITH MUTATIONS:\n", THIS); if(REDWOOD_DEBUG) { self->printMutationBuffer(iMutationBoundary, iMutationBoundaryEnd); } @@ -1132,7 +1132,7 @@ private: // Now, process each mutation range and merge changes with existing data. while(iMutationBoundary != iMutationBoundaryEnd) { - debug_printf("%p New mutation boundary: '%s': %s\n", this, printable(iMutationBoundary->first).c_str(), iMutationBoundary->second.toString().c_str()); + debug_printf("%p New mutation boundary: '%s': %s\n", THIS, printable(iMutationBoundary->first).c_str(), iMutationBoundary->second.toString().c_str()); SingleKeyMutationsByVersion::const_iterator iMutations; @@ -1152,7 +1152,7 @@ private: // Output old versions of the mutation boundary key while(cursor.valid() && cursor.get().key == iMutationBoundary->first) { merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, boundary start]\n", this, merged.back().toString().c_str()); + debug_printf("%p: Added %s [existing, boundary start]\n", THIS, merged.back().toString().c_str()); cursor.moveNext(); } @@ -1166,7 +1166,7 @@ private: minVersion = iMutations->first; ++changes; merged.push_back(iMutations->second.toRecord(iMutationBoundary->first, iMutations->first)); - debug_printf("%p: Added non-split %s [mutation, boundary start]\n", this, merged.back().toString().c_str()); + debug_printf("%p: Added non-split %s [mutation, boundary start]\n", THIS, merged.back().toString().c_str()); } else { if(iMutations->first < minVersion || minVersion == invalidVersion) @@ -1181,7 +1181,7 @@ private: merged.push_back(whole.split(start, partSize)); bytesLeft -= partSize; start += partSize; - debug_printf("%p: Added split %s [mutation, boundary start]\n", this, merged.back().toString().c_str()); + debug_printf("%p: Added split %s [mutation, boundary start]\n", THIS, merged.back().toString().c_str()); } } ++iMutations; @@ -1192,12 +1192,12 @@ private: // Advance to the next boundary because we need to know the end key for the current range. ++iMutationBoundary; - debug_printf("%p Mutation range end: '%s'\n", this, printable(iMutationBoundary->first).c_str()); + debug_printf("%p Mutation range end: '%s'\n", THIS, printable(iMutationBoundary->first).c_str()); // Write existing keys which are less than the next mutation boundary key, clearing if needed. while(cursor.valid() && cursor.get().key < iMutationBoundary->first) { merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, middle]\n", this, merged.back().toString().c_str()); + debug_printf("%p: Added %s [existing, middle]\n", THIS, merged.back().toString().c_str()); // Write a clear of this key if needed. A clear is required if clearRangeVersion is set and the next cursor // key is different than the current one. If the last cursor key in the page is different from the @@ -1211,7 +1211,7 @@ private: minVersion = clearVersion; ++changes; merged.push_back(RedwoodRecordRef(cursor.get().key, clearVersion)); - debug_printf("%p: Added %s [existing, middle clear]\n", this, merged.back().toString().c_str()); + debug_printf("%p: Added %s [existing, middle clear]\n", THIS, merged.back().toString().c_str()); } cursor = nextCursor; } @@ -1220,15 +1220,15 @@ private: // Write any remaining existing keys, which are not subject to clears as they are beyond the cleared range. while(cursor.valid()) { merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, tail]\n", this, merged.back().toString().c_str()); + debug_printf("%p: Added %s [existing, tail]\n", THIS, merged.back().toString().c_str()); cursor.moveNext(); } - debug_printf("%p Done merging mutations into existing leaf contents, made %d changes\n", this, changes); + debug_printf("%p Done merging mutations into existing leaf contents, made %d changes\n", THIS, changes); // No changes were actually made. This could happen if the only mutations are clear ranges which do not match any records. if(minVersion == invalidVersion) { - debug_printf("%p No changes were made during mutation merge\n", this); + debug_printf("%p No changes were made during mutation merge\n", THIS); ASSERT(changes == 0); return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); } @@ -1249,11 +1249,11 @@ private: minVersion = 0; // Write page(s), get new page IDs - std::vector newPageIDs = self->writePages(pages, minVersion, root, page, upperBound, this); + std::vector newPageIDs = self->writePages(pages, minVersion, root, page, upperBound, THIS); // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root && pages.size() > 1) { - debug_printf("%p Building new root\n", this); + debug_printf("%p Building new root\n", THIS); self->buildNewRoot(minVersion, pages, newPageIDs, page); } @@ -1264,11 +1264,11 @@ private: for(int i=0; i %d\n", this, lower.toString().c_str(), newPageIDs[i]); + debug_printf("%p Adding page to results: %s => %d\n", THIS, lower.toString().c_str(), newPageIDs[i]); results.back().second.push_back( {lower, newPageIDs[i]} ); } - debug_printf("%p DONE.\n", this); + debug_printf("%p DONE.\n", THIS); return results; } else { @@ -1316,7 +1316,7 @@ private: } if(!modified) { - debug_printf("%p not modified.\n", this); + debug_printf("%p not modified.\n", THIS); return VersionedChildrenT({{0, {{*lowerBound, root}}}}); } @@ -1329,7 +1329,7 @@ private: std::vector childEntries; // For each Future - debug_printf("%p creating replacement pages for id=%d at Version %lld\n", this, root, version); + debug_printf("%p creating replacement pages for id=%d at Version %lld\n", THIS, root, version); // If we're writing version 0, there is a chance that we don't have to write ourselves, if there are no changes bool modified = version != 0; @@ -1338,11 +1338,11 @@ private: LogicalPageID pageID = childPageIDs[i]; const VersionedChildrenT &children = futureChildren[i].get(); - debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", this, pageID, children.size()); + debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, pageID, children.size()); for(auto &versionedPageSet : children) { - debug_printf("%p version: Page id=%lld\n", this, versionedPageSet.first); + debug_printf("%p version: Page id=%lld\n", THIS, versionedPageSet.first); for(auto &boundaryPage : versionedPageSet.second) { - debug_printf("%p '%s' -> Page id=%u\n", this, boundaryPage.first.toString().c_str(), boundaryPage.second); + debug_printf("%p '%s' -> Page id=%u\n", THIS, boundaryPage.first.toString().c_str(), boundaryPage.second); } } @@ -1351,36 +1351,36 @@ private: // If there are no versions before the one we found, just update nextVersion and continue. if(cv == children.begin()) { - debug_printf("%p First version (%lld) in set is greater than current, setting nextVersion and continuing\n", this, cv->first); + debug_printf("%p First version (%lld) in set is greater than current, setting nextVersion and continuing\n", THIS, cv->first); nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr %lld next %lld\n", this, version, nextVersion); + debug_printf("%p curr %lld next %lld\n", THIS, version, nextVersion); continue; } // If a version greater than the current version being written was found, update nextVersion if(cv != children.end()) { nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr %lld next %lld\n", this, version, nextVersion); + debug_printf("%p curr %lld next %lld\n", THIS, version, nextVersion); } // Go back one to the last version that was valid prior to or at the current version we are writing --cv; - debug_printf("%p Using children for version %lld from this set, building version %lld\n", this, cv->first, version); + debug_printf("%p Using children for version %lld from this set, building version %lld\n", THIS, cv->first, version); // If page count isn't 1 then the root is definitely modified modified = modified || cv->second.size() != 1; // Add the children at this version to the child entries list for the current version being built. for (auto &childPage : cv->second) { - debug_printf("%p Adding child page %s\n", this, childPage.first.toString().c_str()); + debug_printf("%p Adding child page %s\n", THIS, childPage.first.toString().c_str()); RedwoodRecordRef entry = childPage.first; entry.value = StringRef((unsigned char *)&childPage.second, sizeof(uint32_t)); childEntries.push_back(entry); } } - debug_printf("%p Finished pass through futurechildren. childEntries=%lu version=%lld nextVersion=%lld\n", this, childEntries.size(), version, nextVersion); + debug_printf("%p Finished pass through futurechildren. childEntries=%lu version=%lld nextVersion=%lld\n", THIS, childEntries.size(), version, nextVersion); if(modified) { // TODO: Track split points across iterations of this loop, so that they don't shift unnecessarily and @@ -1390,7 +1390,7 @@ private: std::vector pages = buildPages(false, *lowerBound, *upperBound, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); // Write page(s), use version 0 to replace latest version if only writing one page - std::vector newPageIDs = self->writePages(pages, version, root, page, upperBound, this); + std::vector newPageIDs = self->writePages(pages, version, root, page, upperBound, THIS); // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root) @@ -1404,12 +1404,12 @@ private: // TODO: figure this out earlier instead of writing replacement page more than once if (result.size() > 1 && result.back().second == result.end()[-2].second) { - debug_printf("%p Output same as last version, popping it.\n", this); + debug_printf("%p Output same as last version, popping it.\n", THIS); result.pop_back(); } } else { - debug_printf("%p Version 0 has no changes\n", this); + debug_printf("%p Version 0 has no changes\n", THIS); result.push_back({0, {{*lowerBound, root}}}); } @@ -1418,7 +1418,7 @@ private: version = nextVersion; } - debug_printf("%p DONE.\n", this); + debug_printf("%p DONE.\n", THIS); return result; } } From 2801298ae86966e35ed540a02b96afb2340e0e39 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Mon, 29 Apr 2019 17:00:29 -0700 Subject: [PATCH 006/128] Checkpointing incomplete and correctness-breaking progress on adding single-version mode to VersionedBTree. --- fdbserver/IVersionedStore.h | 2 + fdbserver/VersionedBTree.actor.cpp | 569 ++++++++++++++++++++--------- 2 files changed, 397 insertions(+), 174 deletions(-) diff --git a/fdbserver/IVersionedStore.h b/fdbserver/IVersionedStore.h index 1394ee01c4..dd7b0f4bea 100644 --- a/fdbserver/IVersionedStore.h +++ b/fdbserver/IVersionedStore.h @@ -44,6 +44,8 @@ public: virtual void addref() = 0; virtual void delref() = 0; + + virtual std::string toString() const = 0; }; class IVersionedStore : public IClosable { diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 31aa6e2be8..5c926f1b52 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -38,17 +38,10 @@ #define STR(x) LiteralStringRef(x) struct RedwoodRecordRef { -#pragma pack(push,1) - struct ValuePart { - int32_t total; - int32_t start; - }; -#pragma pack(pop) - - RedwoodRecordRef(KeyRef key = KeyRef(), Version ver = std::numeric_limits::max(), Optional value = {}, Optional part = {}) - : key(key), version(ver), value(value), valuePart(part) + RedwoodRecordRef(KeyRef key = KeyRef(), Version ver = 0, Optional value = {}, uint32_t chunkTotal = 0, uint32_t chunkStart = 0) + : key(key), version(ver), value(value), chunk({chunkTotal, chunkStart}) { - ASSERT(!part.present() || value.present()); + ASSERT(value.present() || !isMultiPart()); } RedwoodRecordRef(Arena &arena, const RedwoodRecordRef &toCopy) { @@ -62,30 +55,32 @@ struct RedwoodRecordRef { KeyRef key; Version version; Optional value; - Optional valuePart; + struct { + uint32_t total; + uint32_t start; + } chunk; int expectedSize() const { - return key.expectedSize() + value.expectedSize() + sizeof(version) + sizeof(valuePart); + return key.expectedSize() + value.expectedSize(); } bool isMultiPart() const { - return valuePart.present(); + return value.present() && chunk.total != 0; } // Generate a kv shard from a complete kv RedwoodRecordRef split(int start, int len) { - ASSERT(!isMultiPart() && value.present()); - return RedwoodRecordRef(key, version, value.get().substr(start, len), ValuePart({value.get().size(), start})); + ASSERT(!isMultiPart()); + return RedwoodRecordRef(key, version, value.get().substr(start, len), value.get().size(), start); } #pragma pack(push,1) struct Delta { // TODO: Make this actually a delta - enum EFlags {HAS_VALUE = 1, HAS_VALUE_PART = 4}; + enum EFlags {HAS_VALUE = 1, HAS_VERSION = 2, IS_MULTIPART = 4}; uint8_t flags; uint16_t keySize; - Version version; uint8_t bytes[]; RedwoodRecordRef apply(const RedwoodRecordRef &prev, const RedwoodRecordRef &next, Arena arena) { @@ -93,14 +88,26 @@ struct RedwoodRecordRef { const uint8_t *rptr = bytes; r.key = StringRef(rptr, keySize); rptr += keySize; - r.version = version; + if(flags & HAS_VERSION) { + r.version = (*(Version *)rptr); + rptr += sizeof(Version); + } + else { + r.version = 0; + } if(flags & HAS_VALUE) { uint16_t valueSize = *(uint16_t *)rptr; rptr += 2; r.value = StringRef(rptr, valueSize); rptr += valueSize; - if(flags & HAS_VALUE_PART) { - r.valuePart = *(ValuePart *)rptr; + if(flags & IS_MULTIPART) { + r.chunk.total = *(uint32_t *)rptr; + rptr += sizeof(uint32_t); + r.chunk.start = *(uint32_t *)rptr; + } + else { + r.chunk.total = 0; + r.chunk.start = 0; } } return r; @@ -108,11 +115,14 @@ struct RedwoodRecordRef { int size() const { int s = sizeof(Delta) + keySize; + if(flags & HAS_VERSION) { + s += sizeof(Version); + } if(flags & HAS_VALUE) { - s += 2; - s += *(uint16_t *)(bytes + keySize); - if(flags & HAS_VALUE_PART) { - s += sizeof(ValuePart); + s += *(uint16_t *)((uint8_t *)this + s); + s += sizeof(uint16_t); + if(flags & IS_MULTIPART) { + s += (2 * sizeof(uint32_t)); } } return s; @@ -133,18 +143,15 @@ struct RedwoodRecordRef { if(cmp == 0) { cmp = version - rhs.version; if(cmp == 0) { - // Absent value is greater than present (for reasons) - cmp = (value.present() ? 0 : 1) - (rhs.value.present() ? 0 : 1); - if(cmp == 0) { - // Chunked is greater than whole - cmp = (valuePart.present() ? 1 : 0) - (rhs.valuePart.present() ? 1 : 0); - if(cmp == 0 && valuePart.present()) { - // Larger total size is greater - cmp = valuePart.get().total - rhs.valuePart.get().total; - if(cmp == 0) { - // Order by start - cmp = valuePart.get().start - rhs.valuePart.get().start; - } + // Absent value sorts higher than present (for reasons) + if(value.present() != rhs.value.present()) { + cmp = value.present() ? -1 : 1; + } + else { + // Chunked (represented by chunk.total > 0) sorts higher than whole + cmp = chunk.total - rhs.chunk.total; + if(cmp == 0) { + cmp = chunk.start - rhs.chunk.start; } } } @@ -174,11 +181,14 @@ struct RedwoodRecordRef { int deltaSize(const RedwoodRecordRef &base) const { int s = sizeof(Delta) + key.size(); + if(version != 0) { + s += sizeof(Version); + } if(value.present()) { s += 2; s += value.get().size(); - if(valuePart.present()) { - s += sizeof(ValuePart); + if(isMultiPart()) { + s += (2 * sizeof(uint32_t)); } } return s; @@ -186,20 +196,25 @@ struct RedwoodRecordRef { void writeDelta(Delta &d, const RedwoodRecordRef &prev, const RedwoodRecordRef &next) const { d.flags = value.present() ? Delta::EFlags::HAS_VALUE : 0; - if(valuePart.present()) - d.flags |= Delta::EFlags::HAS_VALUE_PART; d.keySize = key.size(); - d.version = version; uint8_t *wptr = d.bytes; memcpy(wptr, key.begin(), key.size()); wptr += key.size(); + if(version != 0) { + d.flags |= Delta::EFlags::HAS_VERSION; + *(Version *)wptr = (version); + wptr += sizeof(Version); + } if(value.present()) { *(uint16_t *)wptr = value.get().size(); wptr += 2; memcpy(wptr, value.get().begin(), value.get().size()); wptr += value.get().size(); - if(valuePart.present()) { - *(ValuePart *)wptr = valuePart.get(); + if(isMultiPart()) { + d.flags |= Delta::EFlags::IS_MULTIPART; + *(uint32_t *)wptr = chunk.total; + wptr += sizeof(uint32_t); + *(uint32_t *)wptr = chunk.start; } } } @@ -220,8 +235,8 @@ struct RedwoodRecordRef { std::string toString(int hexLimit = 15) const { std::string r; r += format("'%s' @%lld ", kvformat(key, hexLimit).c_str(), version); - if(valuePart.present()) { - r += format("[%d/%d] ", valuePart.get().start, valuePart.get().total); + if(isMultiPart()) { + r += format("[%d/%d] ", chunk.start, chunk.total); } if(value.present()) { r += format("-> '%s'", kvformat(value.get(), hexLimit).c_str()); @@ -600,9 +615,19 @@ public: ++counts.sets; SingleKeyMutationsByVersion &changes = insertMutationBoundary(keyValue.key)->second.startKeyMutations; - // Add the set if the changes set is empty or the last entry isn't a set to exactly the same value - if(changes.empty() || !changes.rbegin()->second.equalToSet(keyValue.value)) { - changes[m_writeVersion] = SingleKeyMutation(keyValue.value); + if(singleVersion) { + if(changes.empty()) { + changes[0] = SingleKeyMutation(keyValue.value); + } + else { + changes.begin()->second.value = keyValue.value; + } + } + else { + // Add the set if the changes set is empty or the last entry isn't a set to exactly the same value + if(changes.empty() || !changes.rbegin()->second.equalToSet(keyValue.value)) { + changes[m_writeVersion] = SingleKeyMutation(keyValue.value); + } } } virtual void clear(KeyRangeRef range) { @@ -610,19 +635,29 @@ public: MutationBufferT::iterator iBegin = insertMutationBoundary(range.begin); MutationBufferT::iterator iEnd = insertMutationBoundary(range.end); - // For each boundary in the cleared range - while(iBegin != iEnd) { + // In single version mode, clear all pending updates in the affected range + if(singleVersion) { RangeMutation &range = iBegin->second; - - // Set the rangeClearedVersion if not set - if(!range.rangeClearVersion.present()) - range.rangeClearVersion = m_writeVersion; - - // Add a clear to the startKeyMutations map if it's empty or the last item is not a clear - if(range.startKeyMutations.empty() || !range.startKeyMutations.rbegin()->second.isClear()) - range.startKeyMutations[m_writeVersion] = SingleKeyMutation(); - + range.startKeyMutations.clear(); + range.rangeClearVersion = 0; ++iBegin; + m_pBuffer->erase(iBegin, iEnd); + } + else { + // For each boundary in the cleared range + while(iBegin != iEnd) { + RangeMutation &range = iBegin->second; + + // Set the rangeClearedVersion if not set + if(!range.rangeClearVersion.present()) + range.rangeClearVersion = m_writeVersion; + + // Add a clear to the startKeyMutations map if it's empty or the last item is not a clear + if(range.startKeyMutations.empty() || !range.startKeyMutations.rbegin()->second.isClear()) + range.startKeyMutations[m_writeVersion] = SingleKeyMutation(); + + ++iBegin; + } } } @@ -645,13 +680,14 @@ public: return m_lastCommittedVersion; } - VersionedBTree(IPager *pager, std::string name, int target_page_size = -1) + VersionedBTree(IPager *pager, std::string name, bool singleVersion = false, int target_page_size = -1) : m_pager(pager), m_writeVersion(invalidVersion), m_usablePageSizeOverride(pager->getUsablePageSize()), m_lastCommittedVersion(invalidVersion), m_pBuffer(nullptr), - m_name(name) + m_name(name), + singleVersion(singleVersion) { if(target_page_size > 0 && target_page_size < m_usablePageSizeOverride) m_usablePageSizeOverride = target_page_size; @@ -693,8 +729,12 @@ public: virtual Reference readAtVersion(Version v) { // TODO: Use the buffer to return uncommitted data // For now, only committed versions can be read. + Version recordVersion = singleVersion ? 0 : v; ASSERT(v <= m_lastCommittedVersion); - return Reference(new Cursor(m_pager->getReadSnapshot(v), m_root, m_usablePageSizeOverride)); + if(singleVersion) { + ASSERT(v == m_lastCommittedVersion); + } + return Reference(new Cursor(m_pager->getReadSnapshot(v), m_root, recordVersion, m_usablePageSizeOverride)); } // Must be nondecreasing @@ -723,6 +763,10 @@ public: return commit_impl(this); } + bool isSingleVersion() const { + return singleVersion; + } + private: void writePage(LogicalPageID id, Reference page, Version ver, const RedwoodRecordRef *pageLowerBound, const RedwoodRecordRef *pageUpperBound) { debug_printf("writePage(): %s\n", ((const BTreePage *)page->begin())->toString(true, id, ver, pageLowerBound, pageUpperBound).c_str()); @@ -735,6 +779,26 @@ private: typedef std::pair> VersionedKeyToPageSetT; typedef std::vector VersionedChildrenT; + static std::string toString(const KeyPagePairT &c) { + return format("(%s, Page %u)", c.first.toString(-1).c_str(), c.second); + } + + static std::string toString(const VersionedKeyToPageSetT &c) { + std::string r = format("Version %lld => [", c.first); + for(auto &o : c.second) { + r += toString(o) + " "; + } + return r + "]"; + } + + static std::string toString(const VersionedChildrenT &c) { + std::string r = "{ "; + for(auto &o : c) { + r += toString(o) + ", "; + } + return r + " }"; + } + // Represents a change to a single key - set, clear, or atomic op struct SingleKeyMutation { // Clear @@ -800,9 +864,13 @@ private: /* Mutation Buffer Overview * - * MutationBuffer maps the start of a range to a RangeMutation. The end of the range is - * the next range start in the map. - * + * This structure's organization is meant to put pending updates for the btree in an order + * that makes it efficient to query all pending mutations across all pending versions which are + * relevant to a particular subtree of the btree. + * + * At the top level, it is a map of the start of a range being modified to a RangeMutation. + * The end of the range is map key (which is the next range start in the map). + * * - The buffer starts out with keys '' and endKVV.key already populated. * * - When a new key is inserted into the buffer map, it is by definition @@ -837,7 +905,11 @@ private: * but it would also be valid to see if the last key before startKey is equal to * keyBefore(startKey), and if so that mutation buffer boundary key can be used instead * without adding an additional key to the buffer. - */ + + * TODO: A possible optimization here could be to only use existing btree leaf page boundaries as keys, + * with mutation point keys being stored in an unsorted strucutre under those boundary map keys, + * to be sorted later just before being merged into the existing leaf page. + */ IPager *m_pager; MutationBufferT *m_pBuffer; @@ -849,6 +921,7 @@ private: int m_usablePageSizeOverride; Future m_init; std::string m_name; + bool singleVersion; void printMutationBuffer(MutationBufferT::const_iterator begin, MutationBufferT::const_iterator end) const { #if REDWOOD_DEBUG @@ -1058,16 +1131,18 @@ private: // Returns list of (version, list of (lower_bound, list of children) ) // TODO: Probably should pass prev/next records by pointer in many places - ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { + ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound, const RedwoodRecordRef *newLowerBound = nullptr, const RedwoodRecordRef *newUpperBound = nullptr) { debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", THIS, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); self->counts.commitToPageStart++; // If the lower bound key and the upper bound key are the same then there can't be any changes to // this subtree since changes would happen after the upper bound key as the mutated versions would // necessarily be higher than all previous versions + // TODO: Avoid calling commitSubtree() when this is true to avoid creating the rather large state of this actor if(lowerBound->key == upperBound->key) { - debug_printf("%p no changes, lower and upper bound keys are the same.\n", THIS); - return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); + VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); + debug_printf("%p id=%u no changes, lower and upper bound keys are the same, returning %s\n", THIS, root, toString(c).c_str()); + return c; } // Find the slice of the mutation buffer that is relevant to this subtree @@ -1083,8 +1158,9 @@ private: else { // If the there are no mutations, we're done if(iMutationBoundary == iMutationBoundaryEnd) { - debug_printf("%p no changes, mutation buffer start/end are the same\n", THIS); - return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); + VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); + debug_printf("%p id=%d no changes, mutation buffer start/end are the same, returning %s\n", THIS, root, toString(c).c_str()); + return c; } } @@ -1101,8 +1177,9 @@ private: iMutationBoundary->first < lowerBound->key) ) ) { - debug_printf("%p no changes because sole mutation range was not cleared\n", THIS); - return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); + VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); + debug_printf("%p no changes because sole mutation range was not cleared, returning %s\n", THIS, toString(c).c_str()); + return c; } self->counts.commitToPage++; @@ -1138,21 +1215,32 @@ private: // If the mutation boundary key is less than the lower bound key then skip startKeyMutations for // this bounary, we're only processing this mutation range here to apply any clears to existing data. - if(iMutationBoundary->first < lowerBound->key) + if(iMutationBoundary->first < lowerBound->key) { iMutations = iMutationBoundary->second.startKeyMutations.end(); + } // If the mutation boundary key is the same as the page lowerBound key then start reading single // key mutations at the first version greater than the lowerBound key's version. - else if(iMutationBoundary->first == lowerBound->key) + else if(!self->singleVersion && iMutationBoundary->first == lowerBound->key) { iMutations = iMutationBoundary->second.startKeyMutations.upper_bound(lowerBound->version); - else + } + else { iMutations = iMutationBoundary->second.startKeyMutations.begin(); + } SingleKeyMutationsByVersion::const_iterator iMutationsEnd = iMutationBoundary->second.startKeyMutations.end(); - // Output old versions of the mutation boundary key + // Iterate over old versions of the mutation boundary key, outputting if necessary while(cursor.valid() && cursor.get().key == iMutationBoundary->first) { - merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, boundary start]\n", THIS, merged.back().toString().c_str()); + // If not in single version mode or there were no changes to the key + if(!self->singleVersion || iMutationBoundary->second.noChanges()) { + merged.push_back(cursor.get()); + debug_printf("%p: Added %s [existing, boundary start]\n", THIS, merged.back().toString().c_str()); + } + else { + ASSERT(self->singleVersion); + debug_printf("%p: Skipped %s [existing, boundary start, singleVersion mode]\n", THIS, cursor.get().toString().c_str()); + minVersion = 0; + } cursor.moveNext(); } @@ -1196,24 +1284,42 @@ private: // Write existing keys which are less than the next mutation boundary key, clearing if needed. while(cursor.valid() && cursor.get().key < iMutationBoundary->first) { - merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, middle]\n", THIS, merged.back().toString().c_str()); + // TODO: Remove old versions that are too old - // Write a clear of this key if needed. A clear is required if clearRangeVersion is set and the next cursor - // key is different than the current one. If the last cursor key in the page is different from the - // first key in the right sibling page then the page's upper bound will reflect that. - auto nextCursor = cursor; - nextCursor.moveNext(); - - if(clearRangeVersion.present() && cursor.get().key != nextCursor.getOrUpperBound().key) { + bool remove = self->singleVersion && clearRangeVersion.present(); + if(!remove) { + merged.push_back(cursor.get()); + debug_printf("%p: Added %s [existing, middle]\n", THIS, merged.back().toString().c_str()); + } + else { + ASSERT(self->singleVersion); + debug_printf("%p: Skipped %s [existing, boundary start, singleVersion mode]\n", THIS, cursor.get().toString().c_str()); Version clearVersion = clearRangeVersion.get(); if(clearVersion < minVersion || minVersion == invalidVersion) minVersion = clearVersion; - ++changes; - merged.push_back(RedwoodRecordRef(cursor.get().key, clearVersion)); - debug_printf("%p: Added %s [existing, middle clear]\n", THIS, merged.back().toString().c_str()); } - cursor = nextCursor; + + // If keeping version history, write clears for records that exist in this range if the range was cleared + if(!self->singleVersion) { + // Write a clear of this key if needed. A clear is required if clearRangeVersion is set and the next cursor + // key is different than the current one. If the last cursor key in the page is different from the + // first key in the right sibling page then the page's upper bound will reflect that. + auto nextCursor = cursor; + nextCursor.moveNext(); + + if(clearRangeVersion.present() && cursor.get().key != nextCursor.getOrUpperBound().key) { + Version clearVersion = clearRangeVersion.get(); + if(clearVersion < minVersion || minVersion == invalidVersion) + minVersion = clearVersion; + ++changes; + merged.push_back(RedwoodRecordRef(cursor.get().key, clearVersion)); + debug_printf("%p: Added %s [existing, middle clear]\n", THIS, merged.back().toString().c_str()); + } + cursor = nextCursor; + } + else { + cursor.moveNext(); + } } } @@ -1228,45 +1334,61 @@ private: // No changes were actually made. This could happen if the only mutations are clear ranges which do not match any records. if(minVersion == invalidVersion) { - debug_printf("%p No changes were made during mutation merge\n", THIS); + VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); + debug_printf("%p No changes were made during mutation merge, returning %s\n", THIS, toString(c).c_str()); ASSERT(changes == 0); - return VersionedChildrenT({ {0,{{*lowerBound,root}}} }); + return c; } // TODO: Make version and key splits based on contents of merged list, if keeping history + // If everything in the page was deleted then this page should be deleted as of the new version + // Note that if a single range clear covered the entire page then we should not get this far + if(merged.empty()) { + // TODO: For multi version mode only delete this page as of the new version + VersionedChildrenT c({}); + debug_printf("%p All leaf page contents were cleared, returning %s\n", THIS, toString(c).c_str()); + return c; + } + IPager *pager = self->m_pager; std::vector pages = buildPages(true, *lowerBound, *upperBound, merged, BTreePage::IS_LEAF, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); - // If there isn't still just a single page of data then this page became too large and was split. - // The new split pages will be valid as of minVersion, but the old page remains valid at the old version - // (TODO: unless history isn't being kept at all) - if(pages.size() != 1) { - results.push_back( {0, {{*lowerBound, root}}} ); + if(!self->singleVersion) { + // If there isn't still just a single page of data then this page became too large and was split. + // The new split pages will be valid as of minVersion, but the old page remains valid at the old version + if(pages.size() != 1) { + results.push_back( {0, {{*lowerBound, root}}} ); + debug_printf("%p Added versioned child set: %s\n", THIS, toString(results.back()).c_str()); + } + else { + // The page was updated but not size-split or version-split so the last page version's data + // can be replaced with the new page contents + if(pages.size() == 1) + minVersion = 0; + } } - if(pages.size() == 1) - minVersion = 0; - // Write page(s), get new page IDs - std::vector newPageIDs = self->writePages(pages, minVersion, root, page, upperBound, THIS); + Version writeVersion = self->singleVersion ? self->getLastCommittedVersion() + 1 : minVersion; + std::vector newPageIDs = self->writePages(pages, writeVersion, root, page, upperBound, THIS); // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root && pages.size() > 1) { debug_printf("%p Building new root\n", THIS); - self->buildNewRoot(minVersion, pages, newPageIDs, page); + self->buildNewRoot(writeVersion, pages, newPageIDs, page); } - results.push_back({minVersion, {}}); - // TODO: Can this be moved into writePages? // TODO: This can probably be skipped for root + results.push_back({writeVersion, {}}); for(int i=0; i %d\n", THIS, lower.toString().c_str(), newPageIDs[i]); results.back().second.push_back( {lower, newPageIDs[i]} ); } + debug_printf("%p Added versioned child set: %s\n", THIS, toString(results.back()).c_str()); debug_printf("%p DONE.\n", THIS); return results; @@ -1290,10 +1412,40 @@ private: const RedwoodRecordRef &childUpperBound = cursor.moveNext() ? cursor.get() : *upperBound; - debug_printf("lower '%s'\n", childLowerBound.toString().c_str()); - debug_printf("upper '%s'\n", childUpperBound.toString().c_str()); + debug_printf("%p lower '%s'\n", THIS, childLowerBound.toString().c_str()); + debug_printf("%p upper '%s'\n", THIS, childUpperBound.toString().c_str()); ASSERT(childLowerBound <= childUpperBound); + /* + // TODO: If lower bound and upper bound have the same key, do something intelligent if possible + // + if(childLowerBound.key == childUpperBound.key) { + if(key is modified or cleared) { + if(self->singleVersion) { + // In single version mode, don't keep any records with the old key if the key is modified, so return + // an empty page set to replace the child page + futureChildren.push_back(VersionedChildrenT({ {0,{} } })); + } + else { + // In versioned mode, there is no need to recurse to this page because new versions of key + // will go in the right most page that has the same lowerBound key, but since the key is + // being changed the new version of this page should exclude the old subtree + + } + else { + // Return the child page as-is, no need to visit it + futureChildren.push_back(VersionedChildrenT({ {0,{{childLowerBound, pageID}}} })); + } + } + else { + // No changes + futureChildren.push_back(VersionedChildrenT({ {0,{{childLowerBound, pageID}}} })); + } + } + else { + futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound)); + } + */ futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound)); childPageIDs.push_back(pageID); } @@ -1306,23 +1458,37 @@ private: } bool modified = false; + + if(REDWOOD_DEBUG) { + debug_printf("%p Subtree update results for id=%d\n", THIS, root); + for(int i = 0; i < futureChildren.size(); ++i) { + const VersionedChildrenT &children = futureChildren[i].get(); + debug_printf("%p subtree for child page id=%u: %s\n", THIS, childPageIDs[i], toString(children).c_str()); + } + } + for(int i = 0; i < futureChildren.size(); ++i) { const VersionedChildrenT &children = futureChildren[i].get(); - // Handle multipages - if(children.size() != 1 || children[0].second.size() != 1) { + + // If the merge resulted in 1 versioned child set with exactly one child + // page, and its id is the same as the original, then no changes were made. + if(!(children.size() == 1 && children.front().second.size() == 1 && children.front().second.front().second == childPageIDs[i])) { modified = true; break; } } if(!modified) { - debug_printf("%p not modified.\n", THIS); - return VersionedChildrenT({{0, {{*lowerBound, root}}}}); + VersionedChildrenT c({{0, {{*lowerBound, root}}}}); + debug_printf("%p not modified, returning %s\n", THIS, toString(c).c_str()); + return c; } Version version = 0; VersionedChildrenT result; + debug_printf("%p: Internal page %u modified, creating replacements.\n", THIS, root); + loop { // over version splits of this page Version nextVersion = std::numeric_limits::max(); @@ -1337,40 +1503,57 @@ private: for(int i = 0; i < futureChildren.size(); ++i) { LogicalPageID pageID = childPageIDs[i]; const VersionedChildrenT &children = futureChildren[i].get(); - - debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, pageID, children.size()); - for(auto &versionedPageSet : children) { - debug_printf("%p version: Page id=%lld\n", THIS, versionedPageSet.first); - for(auto &boundaryPage : versionedPageSet.second) { - debug_printf("%p '%s' -> Page id=%u\n", THIS, boundaryPage.first.toString().c_str(), boundaryPage.second); - } - } - - // Find the first version greater than the current version we are writing - auto cv = std::upper_bound( children.begin(), children.end(), version, [](Version a, VersionedChildrenT::value_type const &b) { return a < b.first; } ); - - // If there are no versions before the one we found, just update nextVersion and continue. - if(cv == children.begin()) { - debug_printf("%p First version (%lld) in set is greater than current, setting nextVersion and continuing\n", THIS, cv->first); - nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr %lld next %lld\n", THIS, version, nextVersion); + if(children.empty()) { + modified = true; continue; } - // If a version greater than the current version being written was found, update nextVersion - if(cv != children.end()) { - nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr %lld next %lld\n", THIS, version, nextVersion); + debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, pageID, children.size()); + if(REDWOOD_DEBUG) { + for(auto &versionedPageSet : children) { + debug_printf("%p version %lld\n", THIS, versionedPageSet.first); + for(auto &boundaryPage : versionedPageSet.second) { + debug_printf("%p '%s' -> Page id=%u\n", THIS, boundaryPage.first.toString().c_str(), boundaryPage.second); + } + } } - // Go back one to the last version that was valid prior to or at the current version we are writing - --cv; + vector::const_iterator cv; + + if(self->singleVersion) { + ASSERT(children.size() == 1); + cv = children.begin(); + nextVersion = std::numeric_limits::max(); + } + else { + // Find the first version greater than the current version we are writing + cv = std::upper_bound( children.begin(), children.end(), version, [](Version a, VersionedChildrenT::value_type const &b) { return a < b.first; } ); + + // If there are no versions before the one we found, just update nextVersion and continue. + if(cv == children.begin()) { + debug_printf("%p First version (%lld) in set is greater than current, setting nextVersion and continuing\n", THIS, cv->first); + nextVersion = std::min(nextVersion, cv->first); + debug_printf("%p curr %lld next %lld\n", THIS, version, nextVersion); + continue; + } + + // If a version greater than the current version being written was found, update nextVersion + if(cv != children.end()) { + nextVersion = std::min(nextVersion, cv->first); + debug_printf("%p curr %lld next %lld\n", THIS, version, nextVersion); + } + + // Go back one to the last version that was valid prior to or at the current version we are writing + --cv; + } debug_printf("%p Using children for version %lld from this set, building version %lld\n", THIS, cv->first, version); // If page count isn't 1 then the root is definitely modified modified = modified || cv->second.size() != 1; + // TODO: If page ID changed in singleVersion mode then root is modified + // Add the children at this version to the child entries list for the current version being built. for (auto &childPage : cv->second) { debug_printf("%p Adding child page %s\n", THIS, childPage.first.toString().c_str()); @@ -1383,6 +1566,15 @@ private: debug_printf("%p Finished pass through futurechildren. childEntries=%lu version=%lld nextVersion=%lld\n", THIS, childEntries.size(), version, nextVersion); if(modified) { + // If all children were deleted then this page should be deleted as of the new version + // Note that if a single range clear covered the entire page then we should not get this far + if(childEntries.empty()) { + // TODO: delete page as of new version + VersionedChildrenT c({}); + debug_printf("%p All internal page children were deleted, returning %s\n", THIS, toString(c).c_str()); + return c; + } + // TODO: Track split points across iterations of this loop, so that they don't shift unnecessarily and // cause unnecessary path copying @@ -1390,14 +1582,15 @@ private: std::vector pages = buildPages(false, *lowerBound, *upperBound, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); // Write page(s), use version 0 to replace latest version if only writing one page - std::vector newPageIDs = self->writePages(pages, version, root, page, upperBound, THIS); + Version writeVersion = self->singleVersion ? self->getLastCommittedVersion() + 1 : version; + std::vector newPageIDs = self->writePages(pages, writeVersion, root, page, upperBound, THIS); // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root) - self->buildNewRoot(version, pages, newPageIDs, page); + self->buildNewRoot(writeVersion, pages, newPageIDs, page); result.resize(result.size()+1); - result.back().first = version; + result.back().first = writeVersion; for(int i=0; i::max()) @@ -1538,9 +1733,18 @@ private: std::string toString() const { std::string r; + Reference c = pageCursor; + int maxDepth = 0; while(c) { - r = format("[%s] ", c->toString().c_str()) + r; + c = c->parent; + ++maxDepth; + } + + c = pageCursor; + int depth = maxDepth; + while(c) { + r = format("[%d/%d: %s] ", depth--, maxDepth, c->toString().c_str()) + r; c = c->parent; } return r; @@ -1710,8 +1914,8 @@ private: // KeyValueRefs returned become invalid once the cursor is moved class Cursor : public IStoreCursor, public ReferenceCounted, public FastAllocated, NonCopyable { public: - Cursor(Reference pageSource, LogicalPageID root, int usablePageSizeOverride) - : m_version(pageSource->getVersion()), + Cursor(Reference pageSource, LogicalPageID root, Version recordVersion, int usablePageSizeOverride) + : m_version(recordVersion), m_cur1(pageSource, root, usablePageSizeOverride), m_cur2(m_cur1) { @@ -1768,7 +1972,7 @@ private: r += format(" KV: '%s' -> '%s'\n", m_kv.get().key.printable().c_str(), m_kv.get().value.printable().c_str()); } else { - r += " KV: "; + r += " KV: \n"; } r += format(" Cur1: %s\n", m_cur1.toString().c_str()); r += format(" Cur2: %s\n", m_cur2.toString().c_str()); @@ -1924,23 +2128,21 @@ private: // Split value, need to coalesce split value parts into a buffer in arena, // after which cur1 will point to the first part and kv.key will reference its key - const RedwoodRecordRef::ValuePart &part = rec.valuePart.get(); - ASSERT(part.start + rec.value.get().size() == part.total); + ASSERT(rec.chunk.start + rec.value.get().size() == rec.chunk.total); - debug_printf("readFullKVPair: Split, totalsize %d %s\n", part.total, self->toString().c_str()); + debug_printf("readFullKVPair: Split, totalsize %d %s\n", rec.chunk.total, self->toString().c_str()); // Allocate space for the entire value in the same arena as the key - state int bytesLeft = part.total; + state int bytesLeft = rec.chunk.total; state StringRef dst = makeString(bytesLeft, self->m_arena); loop { const RedwoodRecordRef &rec = self->m_cur1.get(); - const RedwoodRecordRef::ValuePart &part = rec.valuePart.get(); debug_printf("readFullKVPair: Adding chunk %s\n", rec.toString().c_str()); int partSize = rec.value.get().size(); - memcpy(mutateString(dst) + part.start, rec.value.get().begin(), partSize); + memcpy(mutateString(dst) + rec.chunk.start, rec.value.get().begin(), partSize); bytesLeft -= partSize; if(bytesLeft == 0) { self->m_kv = KeyValueRef(rec.key, dst); @@ -1977,7 +2179,7 @@ public: KeyValueStoreRedwoodUnversioned(std::string filePrefix, UID logID) : m_filePrefix(filePrefix) { // TODO: This constructor should really just take an IVersionedStore IPager *pager = new IndirectShadowPager(filePrefix); - m_tree = new VersionedBTree(pager, filePrefix, pager->getUsablePageSize()); + m_tree = new VersionedBTree(pager, filePrefix, true, pager->getUsablePageSize()); m_init = catchError(init_impl(this)); } @@ -2164,28 +2366,26 @@ KeyValue randomKV(int keySize = 10, int valueSize = 5) { return kv; } -ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::map, Optional> *written, int *pErrorCount) { +ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version v, std::map, Optional> *written, int *pErrorCount) { state int errors = 0; - state Key start = randomKV().key; - state Key end = randomKV().key; if(end <= start) end = keyAfter(start); - debug_printf("VerifyRange '%s' to '%s' @%lld\n", printable(start).c_str(), printable(end).c_str(), v); state std::map, Optional>::const_iterator i = written->lower_bound(std::make_pair(start.toString(), 0)); state std::map, Optional>::const_iterator iEnd = written->upper_bound(std::make_pair(end.toString(), 0)); state std::map, Optional>::const_iterator iLast; state Reference cur = btree->readAtVersion(v); + debug_printf("VerifyRange(@%lld, %s, %s): Start cur=%p\n", v, start.toString().c_str(), end.toString().c_str(), cur.getPtr()); // Randomly use the cursor for something else first. if(g_random->coinflip()) { - debug_printf("VerifyRange: Dummy seek\n"); + debug_printf("VerifyRange(@%lld, %s, %s): Dummy seek\n", v, start.toString().c_str(), end.toString().c_str()); state Key randomKey = randomKV().key; wait(g_random->coinflip() ? cur->findFirstEqualOrGreater(randomKey, true, 0) : cur->findLastLessOrEqual(randomKey, true, 0)); } - debug_printf("VerifyRange: Actual seek\n"); + debug_printf("VerifyRange(@%lld, %s, %s): Actual seek\n", v, start.toString().c_str(), end.toString().c_str()); wait(cur->findFirstEqualOrGreater(start, true, 0)); state std::vector results; @@ -2255,9 +2455,10 @@ ACTOR Future verifyRandomRange(VersionedBTree *btree, Version v, std::mapfirst.second, iLast->first.first.c_str()); } - debug_printf("VerifyRangeReverse '%s' to '%s' @%lld\n", printable(start).c_str(), printable(end).c_str(), v); - // Randomly use a new cursor for the revere range read - if(g_random->coinflip()) { + debug_printf("VerifyRangeReverse(@%lld, %s, %s): start\n", v, start.toString().c_str(), end.toString().c_str()); + + // Randomly use a new cursor for the reverse range read but only if version history is available + if(!btree->isSingleVersion() && g_random->coinflip()) { cur = btree->readAtVersion(v); } @@ -2345,13 +2546,24 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, } ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, std::map, Optional> *written, int *pErrorCount) { + state Future vall; + state Future vrange; + try { loop { state Version v = waitNext(vStream); - debug_printf("Verifying through version %lld\n", v); - state Future vall = verifyAll(btree, v, written, pErrorCount); - state Future vrange = verifyRandomRange(btree, g_random->randomInt(1, v + 1), written, pErrorCount); + if(btree->isSingleVersion()) { + v = btree->getLastCommittedVersion(); + debug_printf("Verifying at latest committed version %lld\n", v); + vall = verifyRange(btree, LiteralStringRef(""), LiteralStringRef("\xff\xff"), v, written, pErrorCount); + vrange = verifyRange(btree, randomKV().key, randomKV().key, v, written, pErrorCount); + } + else { + debug_printf("Verifying through version %lld\n", v); + vall = verifyAll(btree, v, written, pErrorCount); + vrange = verifyRange(btree, randomKV().key, randomKV().key, g_random->randomInt(1, v + 1), written, pErrorCount); + } wait(success(vall) && success(vrange)); int errors = vall.get() + vrange.get(); @@ -2375,7 +2587,10 @@ ACTOR Future randomReader(VersionedBTree *btree) { loop { wait(yield()); if(!cur || g_random->random01() > .1) { - Version v = g_random->randomInt(1, btree->getLastCommittedVersion() + 1); + Version v = btree->getLastCommittedVersion(); + if(!btree->isSingleVersion()) { + v = g_random->randomInt(1, v + 1); + } cur = btree->readAtVersion(v); } @@ -2513,10 +2728,8 @@ TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessRedwoodRecord") { if(g_random->coinflip()) { rec.value = StringRef(arena, v); if(g_random->coinflip()) { - RedwoodRecordRef::ValuePart part; - part.start = g_random->randomInt(0, 5000); - part.total = part.start + v.size() + g_random->randomInt(0, 5000); - rec.valuePart = part; + rec.chunk.start = g_random->randomInt(0, 5000); + rec.chunk.total = rec.chunk.start + v.size() + g_random->randomInt(0, 5000); } } items.push_back(rec); @@ -2679,7 +2892,8 @@ TEST_CASE("!/redwood/correctness") { printf("Initializing...\n"); state int pageSize = g_random->coinflip() ? pager->getUsablePageSize() : g_random->randomInt(200, 400); - state VersionedBTree *btree = new VersionedBTree(pager, pagerFile, pageSize); + state bool singleVersion = g_random->random01() < .05; + state VersionedBTree *btree = new VersionedBTree(pager, pagerFile, singleVersion, pageSize); wait(btree->init()); // We must be able to fit at least two any two keys plus overhead in a page to prevent @@ -2847,7 +3061,7 @@ TEST_CASE("!/redwood/correctness") { debug_printf("Reopening btree\n"); IPager *pager = new IndirectShadowPager(pagerFile); - btree = new VersionedBTree(pager, pagerFile, pageSize); + btree = new VersionedBTree(pager, pagerFile, singleVersion, pageSize); wait(btree->init()); Version v = wait(btree->getLatestVersion()); @@ -2895,7 +3109,7 @@ ACTOR Future pointReads(VersionedBTree *btree, int count, int nodeCount) { state Reference cur = btree->readAtVersion(readVer); while(c < count) { //cur = btree->readAtVersion(readVer); - *(uint32_t *)k.begin() = g_random->randomInt(0, nodeCount); + *(uint32_t *)k.begin() = bigEndian32(g_random->randomInt(0, nodeCount)); wait(success(cur->findFirstEqualOrGreater(k, true, 0))); ++c; } @@ -2911,15 +3125,18 @@ TEST_CASE("!/redwood/performance/set") { deleteFile(pagerFile); deleteFile(pagerFile + "0.pagerlog"); deleteFile(pagerFile + "1.pagerlog"); + IPager *pager = new IndirectShadowPager(pagerFile); - state VersionedBTree *btree = new VersionedBTree(pager, "unittest_pageFile"); + state bool singleVersion = true; + state VersionedBTree *btree = new VersionedBTree(pager, "unittest_pageFile", singleVersion); wait(btree->init()); state int nodeCount = 1e9; state int maxChangesPerVersion = 1000; - state int64_t kvBytesTarget = 100e6; - int maxKeySize = 19; - int maxValueSize = 1000; + state int64_t kvBytesTarget = 200e6; + int maxKeySize = 50; + int maxValueSize = 100; + state int maxConsecutiveRun = 6; state std::string key(maxKeySize, 'k'); state std::string value(maxValueSize, 'v'); @@ -2936,10 +3153,14 @@ TEST_CASE("!/redwood/performance/set") { state Version version = lastVer + 1; btree->setWriteVersion(version); int changes = g_random->randomInt(0, maxChangesPerVersion); + int baseKey = g_random->randomInt(0, nodeCount); while(changes--) { KeyValue kv; // Change first 4 bytes of key to an int - *(uint32_t *)key.data() = g_random->randomInt(0, nodeCount); + *(uint32_t *)key.data() = bigEndian32(baseKey++); + if(g_random->randomInt(0, maxConsecutiveRun) == 0) { + baseKey = g_random->randomInt(0, nodeCount); + } kv.key = StringRef((uint8_t *)key.data(), g_random->randomInt(10, key.size())); kv.value = StringRef((uint8_t *)value.data(), g_random->randomInt(0, value.size())); btree->set(kv); From 4eb4c03ce5fd142ed07ec87d3a2b817ac8c45755 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Tue, 14 May 2019 14:07:49 -1000 Subject: [PATCH 007/128] Save TLog resources by letting peek request only spilled data. If a peek is entirely fulfilled from spilled data, then it's likely that the next peek will be also. It is thus wasteful for each of these peeks to call peekMessagesFromMemory, which memcpy's excessively, and then throw all that data away without using it. Now, TLogs will give a hint back to peek cursors about if the provided reply was served entirely from the spilled data, which peek curors then feed back as the hint into their next request. At some point, a cursor will send a request for only spilled data, get an incomplete response, and then be told to send its next request as one that peeks from memory as well, and then it will fully catch up. --- fdbserver/LogRouter.actor.cpp | 1 + fdbserver/LogSystem.h | 1 + fdbserver/LogSystemPeekCursor.actor.cpp | 15 ++++++++----- fdbserver/OldTLogServer_6_0.actor.cpp | 15 ++++++++++--- fdbserver/TLogInterface.h | 8 ++++--- fdbserver/TLogServer.actor.cpp | 28 +++++++++++++++++++------ 6 files changed, 51 insertions(+), 17 deletions(-) diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 2dc8194d3a..5ab13ee1a8 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -357,6 +357,7 @@ ACTOR Future logRouterPeekMessages( LogRouterData* self, TLogPeekRequest r reply.messages = messages.toValue(); reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; reply.end = endVersion; + reply.onlySpilled = false; req.reply.send( reply ); //TraceEvent("LogRouterPeek4", self->dbgid); diff --git a/fdbserver/LogSystem.h b/fdbserver/LogSystem.h index fb9c5506fa..1b998ff769 100644 --- a/fdbserver/LogSystem.h +++ b/fdbserver/LogSystem.h @@ -379,6 +379,7 @@ struct ILogSystem { UID randomID; bool returnIfBlocked; + bool onlySpilled; bool parallelGetMore; int sequence; Deque> futureResults; diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 0ce171274e..d73789c3fa 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -25,14 +25,14 @@ #include "flow/actorcompiler.h" // has to be last include ILogSystem::ServerPeekCursor::ServerPeekCursor( Reference>> const& interf, Tag tag, Version begin, Version end, bool returnIfBlocked, bool parallelGetMore ) - : interf(interf), tag(tag), messageVersion(begin), end(end), hasMsg(false), rd(results.arena, results.messages, Unversioned()), randomID(g_random->randomUniqueID()), poppedVersion(0), returnIfBlocked(returnIfBlocked), sequence(0), parallelGetMore(parallelGetMore) { + : interf(interf), tag(tag), messageVersion(begin), end(end), hasMsg(false), rd(results.arena, results.messages, Unversioned()), randomID(g_random->randomUniqueID()), poppedVersion(0), returnIfBlocked(returnIfBlocked), sequence(0), onlySpilled(false), parallelGetMore(parallelGetMore) { this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; //TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace(); } ILogSystem::ServerPeekCursor::ServerPeekCursor( TLogPeekReply const& results, LogMessageVersion const& messageVersion, LogMessageVersion const& end, int32_t messageLength, int32_t rawLength, bool hasMsg, Version poppedVersion, Tag tag ) - : results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion), end(end), messageLength(messageLength), rawLength(rawLength), hasMsg(hasMsg), randomID(g_random->randomUniqueID()), poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), parallelGetMore(false) + : results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion), end(end), messageLength(messageLength), rawLength(rawLength), hasMsg(hasMsg), randomID(g_random->randomUniqueID()), poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), onlySpilled(false), parallelGetMore(false) { //TraceEvent("SPC_Clone", randomID); this->results.maxKnownVersion = 0; @@ -147,7 +147,7 @@ ACTOR Future serverPeekParallelGetMore( ILogSystem::ServerPeekCursor* self state Version expectedBegin = self->messageVersion.version; try { while(self->futureResults.size() < SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && self->interf->get().present()) { - self->futureResults.push_back( brokenPromiseToNever( self->interf->get().interf().peekMessages.getReply(TLogPeekRequest(self->messageVersion.version,self->tag,self->returnIfBlocked, std::make_pair(self->randomID, self->sequence++)), taskID) ) ); + self->futureResults.push_back( brokenPromiseToNever( self->interf->get().interf().peekMessages.getReply(TLogPeekRequest(self->messageVersion.version,self->tag,self->returnIfBlocked, self->onlySpilled, std::make_pair(self->randomID, self->sequence++)), taskID) ) ); } choose { @@ -158,6 +158,7 @@ ACTOR Future serverPeekParallelGetMore( ILogSystem::ServerPeekCursor* self expectedBegin = res.end; self->futureResults.pop_front(); self->results = res; + self->onlySpilled = res.onlySpilled; if(res.popped.present()) self->poppedVersion = std::min( std::max(self->poppedVersion, res.popped.get()), self->end.version ); self->rd = ArenaReader( self->results.arena, self->results.messages, Unversioned() ); @@ -172,6 +173,7 @@ ACTOR Future serverPeekParallelGetMore( ILogSystem::ServerPeekCursor* self self->interfaceChanged = self->interf->onChange(); self->randomID = g_random->randomUniqueID(); self->sequence = 0; + self->onlySpilled = false; self->futureResults.clear(); } } @@ -201,8 +203,9 @@ ACTOR Future serverPeekGetMore( ILogSystem::ServerPeekCursor* self, int ta loop { choose { when( TLogPeekReply res = wait( self->interf->get().present() ? - brokenPromiseToNever( self->interf->get().interf().peekMessages.getReply(TLogPeekRequest(self->messageVersion.version,self->tag,self->returnIfBlocked), taskID) ) : Never() ) ) { + brokenPromiseToNever( self->interf->get().interf().peekMessages.getReply(TLogPeekRequest(self->messageVersion.version,self->tag,self->returnIfBlocked, self->onlySpilled), taskID) ) : Never() ) ) { self->results = res; + self->onlySpilled = res.onlySpilled; if(res.popped.present()) self->poppedVersion = std::min( std::max(self->poppedVersion, res.popped.get()), self->end.version ); self->rd = ArenaReader( self->results.arena, self->results.messages, Unversioned() ); @@ -213,7 +216,9 @@ ACTOR Future serverPeekGetMore( ILogSystem::ServerPeekCursor* self, int ta //TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0); return Void(); } - when( wait( self->interf->onChange() ) ) {} + when( wait( self->interf->onChange() ) ) { + self->onlySpilled = false; + } } } } catch( Error &e ) { diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 9f86bd7574..aa7b43b169 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1018,6 +1018,7 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; rep.popped = poppedVer; rep.end = poppedVer; + rep.onlySpilled = false; if(req.sequence.present()) { auto& trackerData = self->peekTracker[peekId]; @@ -1044,6 +1045,7 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere } state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; //grab messages from disk //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); @@ -1053,7 +1055,11 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere // SOMEDAY: Only do this if an initial attempt to read from disk results in insufficient data and the required data is no longer in memory // SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the result? - peekMessagesFromMemory( logData, req, messages2, endVersion ); + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory( logData, req, messages2, endVersion ); + } Standalone> kvs = wait( self->persistentData->readRange(KeyRangeRef( @@ -1068,10 +1074,12 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere messages.serializeBytes(kv.value); } - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - else + onlySpilled = true; + } else { messages.serializeBytes( messages2.toValue() ); + } } else { peekMessagesFromMemory( logData, req, messages, endVersion ); //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); @@ -1082,6 +1090,7 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; reply.messages = messages.toValue(); reply.end = endVersion; + reply.onlySpilled = onlySpilled; //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().address); diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index 585caa23a3..f9c3db645b 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -150,10 +150,11 @@ struct TLogPeekReply { Version maxKnownVersion; Version minKnownCommittedVersion; Optional begin; + bool onlySpilled; template void serialize(Ar& ar) { - serializer(ar, arena, messages, end, popped, maxKnownVersion, minKnownCommittedVersion, begin); + serializer(ar, arena, messages, end, popped, maxKnownVersion, minKnownCommittedVersion, begin, onlySpilled); } }; @@ -163,15 +164,16 @@ struct TLogPeekRequest { Version begin; Tag tag; bool returnIfBlocked; + bool onlySpilled; Optional> sequence; ReplyPromise reply; - TLogPeekRequest( Version begin, Tag tag, bool returnIfBlocked, Optional> sequence = Optional>() ) : begin(begin), tag(tag), returnIfBlocked(returnIfBlocked), sequence(sequence) {} + TLogPeekRequest( Version begin, Tag tag, bool returnIfBlocked, bool onlySpilled, Optional> sequence = Optional>() ) : begin(begin), tag(tag), returnIfBlocked(returnIfBlocked), sequence(sequence), onlySpilled(onlySpilled) {} TLogPeekRequest() {} template void serialize(Ar& ar) { - serializer(ar, arena, begin, tag, returnIfBlocked, sequence, reply); + serializer(ar, arena, begin, tag, returnIfBlocked, onlySpilled, sequence, reply); } }; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 55051ff6e4..3f602a028b 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1304,6 +1304,7 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; rep.popped = poppedVer; rep.end = poppedVer; + rep.onlySpilled = false; if(req.sequence.present()) { auto& trackerData = self->peekTracker[peekId]; @@ -1330,6 +1331,7 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere } state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; //grab messages from disk //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); @@ -1339,7 +1341,11 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere // SOMEDAY: Only do this if an initial attempt to read from disk results in insufficient data and the required data is no longer in memory // SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the result? - peekMessagesFromMemory( logData, req, messages2, endVersion ); + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory( logData, req, messages2, endVersion ); + } if (req.tag == txsTag) { Standalone> kvs = wait( @@ -1353,10 +1359,12 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere messages.serializeBytes(kv.value); } - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - else + onlySpilled = true; + } else { messages.serializeBytes( messages2.toValue() ); + } } else { // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. Standalone> kvrefs = wait( @@ -1433,13 +1441,20 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere messageReads.clear(); memoryReservation.release(); - if (earlyEnd) + if (earlyEnd) { endVersion = lastRefMessageVersion + 1; - else + onlySpilled = true; + } else { messages.serializeBytes( messages2.toValue() ); + } } } else { - peekMessagesFromMemory( logData, req, messages, endVersion ); + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory( logData, req, messages, endVersion ); + } + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } @@ -1448,6 +1463,7 @@ ACTOR Future tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; reply.messages = messages.toValue(); reply.end = endVersion; + reply.onlySpilled = onlySpilled; //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); From 658e61b39448b0975e0b07cbc169cc73b6a68bea Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Tue, 14 May 2019 21:03:44 -1000 Subject: [PATCH 008/128] And now use spilledOnly as a hint to do parallel peeks. If there's some spilled data, there's probably a lot of spilled data, and now we can pull all of it faster. --- fdbserver/LogSystemPeekCursor.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index d73789c3fa..2d62088672 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -235,7 +235,7 @@ Future ILogSystem::ServerPeekCursor::getMore(int taskID) { if( hasMessage() ) return Void(); if( !more.isValid() || more.isReady() ) { - more = parallelGetMore ? serverPeekParallelGetMore(this, taskID) : serverPeekGetMore(this, taskID); + more = (parallelGetMore || onlySpilled) ? serverPeekParallelGetMore(this, taskID) : serverPeekGetMore(this, taskID); } return more; } From 3f8fce029693db87ba5884480ce4dd931563d487 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Tue, 21 May 2019 19:16:32 -0700 Subject: [PATCH 009/128] Checkpointing progress on single-version mode in VersionedBTree. Subtree clears now work, preserving internal page boundary keys when necessary. Multi-version mode is unfortunately now broken, in addition to being incomplete. Added serial and simple btree unit test options. --- fdbserver/VersionedBTree.actor.cpp | 402 ++++++++++++++++++++--------- 1 file changed, 277 insertions(+), 125 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 5c926f1b52..f66ab5aa9f 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -52,6 +52,10 @@ struct RedwoodRecordRef { } } + RedwoodRecordRef withoutValue() const { + return RedwoodRecordRef(key, version, {}, chunk.total, chunk.start); + } + KeyRef key; Version version; Optional value; @@ -138,20 +142,19 @@ struct RedwoodRecordRef { #pragma pack(pop) int compare(const RedwoodRecordRef &rhs) const { - //printf("compare %s to %s\n", toString().c_str(), rhs.toString().c_str()); int cmp = key.compare(rhs.key); if(cmp == 0) { cmp = version - rhs.version; if(cmp == 0) { - // Absent value sorts higher than present (for reasons) - if(value.present() != rhs.value.present()) { - cmp = value.present() ? -1 : 1; - } - else { - // Chunked (represented by chunk.total > 0) sorts higher than whole + // It is assumed that in any data set there will never be more than one + // unique chunk total size for the same key and version, so sort by start, total + // Chunked (represented by chunk.total > 0) sorts higher than whole + cmp = chunk.start - rhs.chunk.start; + if(cmp == 0) { cmp = chunk.total - rhs.chunk.total; if(cmp == 0) { - cmp = chunk.start - rhs.chunk.start; + // No-value sorts AFTER having a value. + cmp = (value.present() ? 0 : 1) - (rhs.value.present() ? 0 : 1); } } } @@ -235,14 +238,12 @@ struct RedwoodRecordRef { std::string toString(int hexLimit = 15) const { std::string r; r += format("'%s' @%lld ", kvformat(key, hexLimit).c_str(), version); - if(isMultiPart()) { - r += format("[%d/%d] ", chunk.start, chunk.total); - } + r += format("[%d/%d] ", chunk.start, chunk.total); if(value.present()) { r += format("-> '%s'", kvformat(value.get(), hexLimit).c_str()); } else { - r += "-> "; + r += "-> "; } return r; } @@ -303,10 +304,13 @@ struct BTreePage { r += " "; if(!(flags & IS_LEAF)) { RedwoodRecordRef rec = c.get(); - ASSERT(rec.value.present() && rec.value.get().size() == sizeof(uint32_t)); - uint32_t id = *(const uint32_t *)rec.value.get().begin(); - std::string val = format("[Page id=%u]", id); - rec.value = val; + std::string val; + if(rec.value.present()) { + ASSERT(rec.value.get().size() == sizeof(uint32_t)); + uint32_t id = *(const uint32_t *)rec.value.get().begin(); + val = format("[Page id=%u]", id); + rec.value = val; + } r += rec.toString(); } else { @@ -315,6 +319,8 @@ struct BTreePage { r += "\n"; + ASSERT(c.get().key >= lowerBound->key && c.get().key <= upperBound->key); + } while(c.moveNext()); } } catch(Error &e) { @@ -327,7 +333,7 @@ struct BTreePage { } }; -static void writeEmptyPage(Reference page, uint8_t newFlags, int pageSize) { +static void makeEmptyPage(Reference page, uint8_t newFlags, int pageSize) { VALGRIND_MAKE_MEM_DEFINED(page->begin(), page->size()); BTreePage *btpage = (BTreePage *)page->begin(); btpage->flags = newFlags; @@ -371,7 +377,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red int i = 0; const int iEnd = entries.size(); // Lower bound of the page being added to - RedwoodRecordRef pageLowerBound = lowerBound; + RedwoodRecordRef pageLowerBound = lowerBound.withoutValue(); RedwoodRecordRef pageUpperBound; while(i <= iEnd) { @@ -380,7 +386,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red // If not the end, add i to the page if necessary if(end) { - pageUpperBound = upperBound; + pageUpperBound = upperBound.withoutValue(); } else { // Get delta from previous record @@ -420,6 +426,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red if(!fits) { // Flush page if(minimalBoundaries) { + // TODO: Write minimal boundaries // Note that prefixLen is guaranteed to be < entry.key.size() because entries are in increasing order and cannot repeat. // int len = prefixLen + 1; // if(entry.key[prefixLen] == 0) @@ -427,7 +434,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red // pageUpperBound = entry.key.substr(0, len); } else { - pageUpperBound = entry; + pageUpperBound = entry.withoutValue(); } } } @@ -450,7 +457,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red // If not writing the final page, reduce entry count of page by a third if(!end) { i -= count / 3; - pageUpperBound = entries[i]; + pageUpperBound = entries[i].withoutValue(); } debug_printf("Flushing page start=%d i=%d count=%d\nlower: %s\nupper: %s\n", start, i, count, pageLowerBound.toString().c_str(), pageUpperBound.toString().c_str()); @@ -458,11 +465,11 @@ static std::vector buildPages(bool minimalBoundaries, const Red for(int j = start; j < i; ++j) { debug_printf(" %3d: %s\n", j, entries[j].toString().c_str()); if(j > start) { - ASSERT(entries[j] > entries[j - 1]); + //ASSERT(entries[j] > entries[j - 1]); } } + ASSERT(pageLowerBound.key <= pageUpperBound.key); #endif - ASSERT(pageLowerBound <= pageUpperBound); union { BTreePage *btPage; @@ -523,7 +530,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red start = i; kvBytes = 0; compressedBytes = BTreePage::BinaryTree::GetTreeOverhead(); - pageLowerBound = pageUpperBound; + pageLowerBound = pageUpperBound.withoutValue(); } } @@ -620,7 +627,7 @@ public: changes[0] = SingleKeyMutation(keyValue.value); } else { - changes.begin()->second.value = keyValue.value; + changes.begin()->second = SingleKeyMutation(keyValue.value); } } else { @@ -639,6 +646,7 @@ public: if(singleVersion) { RangeMutation &range = iBegin->second; range.startKeyMutations.clear(); + range.startKeyMutations[0] = SingleKeyMutation(); range.rangeClearVersion = 0; ++iBegin; m_pBuffer->erase(iBegin, iEnd); @@ -701,7 +709,7 @@ public: if(latest == 0) { ++latest; Reference page = self->m_pager->newPageBuffer(); - writeEmptyPage(page, BTreePage::IS_LEAF, self->m_usablePageSizeOverride); + makeEmptyPage(page, BTreePage::IS_LEAF, self->m_usablePageSizeOverride); self->writePage(self->m_root, page, latest, &dbBegin, &dbEnd); self->m_pager->setLatestVersion(latest); wait(self->m_pager->commit()); @@ -1091,7 +1099,7 @@ private: }; ACTOR static Future> readPage(Reference snapshot, LogicalPageID id, int usablePageSize, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { - debug_printf("readPage() op=read id=%u @%lld\n", id, snapshot->getVersion()); + debug_printf("readPage() op=read id=%u @%lld lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); wait(delay(0, TaskDiskRead)); state Reference result = wait(snapshot->getPhysicalPage(id)); @@ -1117,6 +1125,7 @@ private: } if(result->userData == nullptr) { + debug_printf("readPage() Creating Reader for page id=%u @%lld lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); result->userData = new BTreePage::BinaryTree::Reader(&pTreePage->tree(), lowerBound, upperBound); result->userDataDestructor = [](void *ptr) { delete (BTreePage::BinaryTree::Reader *)ptr; }; } @@ -1131,47 +1140,47 @@ private: // Returns list of (version, list of (lower_bound, list of children) ) // TODO: Probably should pass prev/next records by pointer in many places - ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound, const RedwoodRecordRef *newLowerBound = nullptr, const RedwoodRecordRef *newUpperBound = nullptr) { - debug_printf("%p commitSubtree: root=%d lower='%s' upper='%s'\n", THIS, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); + ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound, const RedwoodRecordRef *decodeLowerBound, const RedwoodRecordRef *decodeUpperBound) { + debug_printf("%p commitSubtree: root=%d lower=%s upper=%s\n", THIS, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); + debug_printf("%p commitSubtree: root=%d decodeLower=%s decodeUpper=%s\n", THIS, root, decodeLowerBound->toString().c_str(), decodeUpperBound->toString().c_str()); self->counts.commitToPageStart++; - // If the lower bound key and the upper bound key are the same then there can't be any changes to - // this subtree since changes would happen after the upper bound key as the mutated versions would - // necessarily be higher than all previous versions - // TODO: Avoid calling commitSubtree() when this is true to avoid creating the rather large state of this actor - if(lowerBound->key == upperBound->key) { - VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); - debug_printf("%p id=%u no changes, lower and upper bound keys are the same, returning %s\n", THIS, root, toString(c).c_str()); - return c; - } + // If a boundary changed, the page must be rewritten regardless of KV mutations + state bool boundaryChanged = (lowerBound != decodeLowerBound) || (upperBound != decodeUpperBound); + debug_printf("%p id=%u boundaryChanged=%d\n", THIS, root, boundaryChanged); // Find the slice of the mutation buffer that is relevant to this subtree // TODO: Rather than two lower_bound searches, perhaps just compare each mutation to the upperBound key while iterating - state MutationBufferT::const_iterator iMutationBoundary = mutationBuffer->lower_bound(lowerBound->key); + state MutationBufferT::const_iterator iMutationBoundary = mutationBuffer->upper_bound(lowerBound->key); + --iMutationBoundary; state MutationBufferT::const_iterator iMutationBoundaryEnd = mutationBuffer->lower_bound(upperBound->key); - // If the mutation buffer key found is greater than the lower bound key then go to the previous mutation - // buffer key because it may cover deletion of some keys at the start of this subtree. - if(iMutationBoundary != mutationBuffer->begin() && iMutationBoundary->first > lowerBound->key) { - --iMutationBoundary; + if(REDWOOD_DEBUG) { + self->printMutationBuffer(iMutationBoundary, iMutationBoundaryEnd); } - else { - // If the there are no mutations, we're done - if(iMutationBoundary == iMutationBoundaryEnd) { + + // If the boundary range iterators are the same then upperbound and lowerbound have the same key. + // If the key is being mutated, them remove this subtree. + if(iMutationBoundary == iMutationBoundaryEnd) { + if(!iMutationBoundary->second.startKeyMutations.empty()) { + VersionedChildrenT c; + debug_printf("%p id=%u lower and upper bound key/version match and key is modified so deleting page, returning %s\n", THIS, root, toString(c).c_str()); + return c; + } + + // If there are no forced boundary changes then this subtree is unchanged. + if(!boundaryChanged) { VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); - debug_printf("%p id=%d no changes, mutation buffer start/end are the same, returning %s\n", THIS, root, toString(c).c_str()); + debug_printf("%p id=%d page contains a single key '%s' which is not changing, returning %s\n", THIS, root, lowerBound->key.toString().c_str(), toString(c).c_str()); return c; } } - // TODO: Check if entire subtree is erased and return no pages, also have the previous pages deleted as of - // the cleared version. - // Another way to have no mutations is to have a single mutation range cover this // subtree but have no changes in it MutationBufferT::const_iterator iMutationBoundaryNext = iMutationBoundary; ++iMutationBoundaryNext; - if(iMutationBoundaryNext == iMutationBoundaryEnd && + if(!boundaryChanged && iMutationBoundaryNext == iMutationBoundaryEnd && ( iMutationBoundary->second.noChanges() || ( !iMutationBoundary->second.rangeClearVersion.present() && iMutationBoundary->first < lowerBound->key) @@ -1183,9 +1192,9 @@ private: } self->counts.commitToPage++; - state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride, lowerBound, upperBound)); + state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride, decodeLowerBound, decodeUpperBound)); state BTreePage *page = (BTreePage *) rawPage->begin(); - debug_printf("%p commitSubtree(): %s\n", THIS, page->toString(false, root, snapshot->getVersion(), lowerBound, upperBound).c_str()); + debug_printf("%p commitSubtree(): %s\n", THIS, page->toString(false, root, snapshot->getVersion(), decodeLowerBound, decodeUpperBound).c_str()); BTreePage::BinaryTree::Cursor cursor = getReader(rawPage)->getCursor(); cursor.moveFirst(); @@ -1195,7 +1204,7 @@ private: VersionedChildrenT results; std::vector merged; - debug_printf("%p MERGING EXISTING DATA WITH MUTATIONS:\n", THIS); + debug_printf("%p id=%u MERGING EXISTING DATA WITH MUTATIONS:\n", THIS, root); if(REDWOOD_DEBUG) { self->printMutationBuffer(iMutationBoundary, iMutationBoundaryEnd); } @@ -1253,7 +1262,7 @@ private: if(iMutations->first < minVersion || minVersion == invalidVersion) minVersion = iMutations->first; ++changes; - merged.push_back(iMutations->second.toRecord(iMutationBoundary->first, iMutations->first)); + merged.push_back(m.toRecord(iMutationBoundary->first, iMutations->first)); debug_printf("%p: Added non-split %s [mutation, boundary start]\n", THIS, merged.back().toString().c_str()); } else { @@ -1333,7 +1342,7 @@ private: debug_printf("%p Done merging mutations into existing leaf contents, made %d changes\n", THIS, changes); // No changes were actually made. This could happen if the only mutations are clear ranges which do not match any records. - if(minVersion == invalidVersion) { + if(!boundaryChanged && minVersion == invalidVersion) { VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); debug_printf("%p No changes were made during mutation merge, returning %s\n", THIS, toString(c).c_str()); ASSERT(changes == 0); @@ -1344,10 +1353,10 @@ private: // If everything in the page was deleted then this page should be deleted as of the new version // Note that if a single range clear covered the entire page then we should not get this far - if(merged.empty()) { + if(merged.empty() && root != 0) { // TODO: For multi version mode only delete this page as of the new version VersionedChildrenT c({}); - debug_printf("%p All leaf page contents were cleared, returning %s\n", THIS, toString(c).c_str()); + debug_printf("%p id=%u All leaf page contents were cleared, returning %s\n", THIS, root, toString(c).c_str()); return c; } @@ -1359,7 +1368,7 @@ private: // The new split pages will be valid as of minVersion, but the old page remains valid at the old version if(pages.size() != 1) { results.push_back( {0, {{*lowerBound, root}}} ); - debug_printf("%p Added versioned child set: %s\n", THIS, toString(results.back()).c_str()); + debug_printf("%p Added versioned child set #1: %s\n", THIS, toString(results.back()).c_str()); } else { // The page was updated but not size-split or version-split so the last page version's data @@ -1379,16 +1388,14 @@ private: self->buildNewRoot(writeVersion, pages, newPageIDs, page); } - // TODO: Can this be moved into writePages? - // TODO: This can probably be skipped for root results.push_back({writeVersion, {}}); for(int i=0; i %d\n", THIS, lower.toString().c_str(), newPageIDs[i]); + debug_printf("%p Adding page to results: %s => Page %d\n", THIS, lower.toString().c_str(), newPageIDs[i]); results.back().second.push_back( {lower, newPageIDs[i]} ); } - debug_printf("%p Added versioned child set: %s\n", THIS, toString(results.back()).c_str()); + debug_printf("%p Added versioned child set #2: %s\n", THIS, toString(results.back()).c_str()); debug_printf("%p DONE.\n", THIS); return results; @@ -1396,25 +1403,53 @@ private: else { // Internal Page + // TODO: Combine these into one vector and/or do something more elegant state std::vector> futureChildren; state std::vector childPageIDs; + state std::vector lowerBoundaries; + state std::vector decodeLowerBoundaries; + + // Track whether or not any child has had its boundaries forcibly changed + state bool childBoundariesChanged = false; // TODO: Make this much more efficient with a skip-merge through the two sorted sets (mutations, existing cursor) bool first = true; while(cursor.valid()) { // The lower bound for the first child is the lowerBound arg const RedwoodRecordRef &childLowerBound = first ? *lowerBound : cursor.get(); - if(first) - first = false; + first = false; - uint32_t pageID = *(uint32_t*)cursor.get().value.get().begin(); + // Skip over any children that do not link to a page. They exist to preserve the ancestors from + // which adjacent children can borrow prefix bytes. + // If there are any, then the first valid child page will incur a boundary change to move + // its lower bound to the left so we can delete the non-linking entry from this page to free up space. + while(!cursor.get().value.present()) { + // There should be an internal page written that has no valid child pages. This loop will find + // the first valid child link, and if there are no more then execution will not return to this loop. + ASSERT(cursor.moveNext()); + childBoundariesChanged = true; + } + + ASSERT(cursor.valid()); + + const RedwoodRecordRef &decodeChildLowerBound = cursor.get(); + + const Optional &childValue = cursor.get().value; + uint32_t pageID = *(uint32_t*)childValue.get().begin(); ASSERT(pageID != 0); - const RedwoodRecordRef &childUpperBound = cursor.moveNext() ? cursor.get() : *upperBound; + const RedwoodRecordRef &decodeChildUpperBound = cursor.moveNext() ? cursor.get() : *upperBound; - debug_printf("%p lower '%s'\n", THIS, childLowerBound.toString().c_str()); - debug_printf("%p upper '%s'\n", THIS, childUpperBound.toString().c_str()); - ASSERT(childLowerBound <= childUpperBound); + // Skip over any next-children which do not actually link to child pages + while(cursor.valid() && !cursor.get().value.present()) { + cursor.moveNext(); + childBoundariesChanged = true; + } + + const RedwoodRecordRef &childUpperBound = cursor.valid() ? cursor.get() : *upperBound; + + debug_printf("%p internal page id=%d child page id=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", + THIS, root, pageID, childLowerBound.toString().c_str(), childUpperBound.toString().c_str(), decodeChildLowerBound.toString().c_str(), decodeChildUpperBound.toString().c_str()); /* // TODO: If lower bound and upper bound have the same key, do something intelligent if possible @@ -1446,8 +1481,10 @@ private: futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound)); } */ - futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound)); + futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound, &decodeChildLowerBound, &decodeChildUpperBound)); childPageIDs.push_back(pageID); + lowerBoundaries.push_back(&childLowerBound); + decodeLowerBoundaries.push_back(&decodeChildLowerBound); } // Waiting one at a time makes debugging easier @@ -1457,28 +1494,67 @@ private: wait(success(futureChildren[k])); } + // Were any children modified? bool modified = false; + // Were all children deleted? + bool all_deleted = true; if(REDWOOD_DEBUG) { debug_printf("%p Subtree update results for id=%d\n", THIS, root); for(int i = 0; i < futureChildren.size(); ++i) { const VersionedChildrenT &children = futureChildren[i].get(); - debug_printf("%p subtree for child page id=%u: %s\n", THIS, childPageIDs[i], toString(children).c_str()); + debug_printf("%p subtree for child page id=%u lowerBound=%s: %s\n", THIS, childPageIDs[i], lowerBoundaries[i]->toString(50).c_str(), toString(children).c_str()); } } for(int i = 0; i < futureChildren.size(); ++i) { const VersionedChildrenT &children = futureChildren[i].get(); - // If the merge resulted in 1 versioned child set with exactly one child - // page, and its id is the same as the original, then no changes were made. - if(!(children.size() == 1 && children.front().second.size() == 1 && children.front().second.front().second == childPageIDs[i])) { + if(children.empty()) { modified = true; - break; + } + else { + // Not all children were deleted since this branch has a child + all_deleted = false; + + // If the merge resulted in 1 versioned child set with exactly one child + // page, and its page ID and lower bound are the same as the original, then no changes were made. + // Otherwise, a change was made so we know the page was modified and we can stop iterating. + if(!( children.size() == 1 + && children.front().second.size() == 1 + && children.front().second.front().second == childPageIDs[i] + && children.front().second.front().first == *decodeLowerBoundaries[i] + ) + ) + { + modified = true; + break; + } } } - if(!modified) { + if(childBoundariesChanged) { + modified = true; + } + + if(all_deleted) { + ASSERT(modified); + if(root == 0) { + Reference page = self->m_pager->newPageBuffer(); + makeEmptyPage(page, BTreePage::IS_LEAF, self->m_usablePageSizeOverride); + self->writePage(0, page, self->getLastCommittedVersion() + 1, &dbBegin, &dbEnd); + VersionedChildrenT c({ {0, { {dbBegin, 0} } } }); + debug_printf("%p id=%u All root page children were deleted, rewrote root as leaf, returning %s\n", THIS, root, toString(c).c_str()); + return c; + } + else { + VersionedChildrenT c({}); + debug_printf("%p id=%u All internal page children were deleted #1 so deleting this page too, returning %s\n", THIS, root, toString(c).c_str()); + return c; + } + } + + if(!boundaryChanged && !modified) { VersionedChildrenT c({{0, {{*lowerBound, root}}}}); debug_printf("%p not modified, returning %s\n", THIS, toString(c).c_str()); return c; @@ -1497,14 +1573,23 @@ private: // For each Future debug_printf("%p creating replacement pages for id=%d at Version %lld\n", THIS, root, version); - // If we're writing version 0, there is a chance that we don't have to write ourselves, if there are no changes - bool modified = version != 0; + // In multi version mode if we're writing version 0 there is a chance that we don't have to write ourselves, if there are no changes in any child subtrees + bool modified = self->singleVersion || version != 0; for(int i = 0; i < futureChildren.size(); ++i) { LogicalPageID pageID = childPageIDs[i]; const VersionedChildrenT &children = futureChildren[i].get(); if(children.empty()) { + // Subtree was deleted, but due to prefix dependencies the boundary key might still need to exist modified = true; + + // If there is a previous child and it has a subtree then this boundary key must still exist + // to continue being the upper bound of the previous child + if(!childEntries.empty() && childEntries.back().value.present()) { + RedwoodRecordRef entry(*lowerBoundaries[i]); + entry.value = Optional(); + childEntries.push_back(entry); + } continue; } @@ -1556,9 +1641,9 @@ private: // Add the children at this version to the child entries list for the current version being built. for (auto &childPage : cv->second) { - debug_printf("%p Adding child page %s\n", THIS, childPage.first.toString().c_str()); RedwoodRecordRef entry = childPage.first; entry.value = StringRef((unsigned char *)&childPage.second, sizeof(uint32_t)); + debug_printf("%p Adding child page %s\n", THIS, entry.toString().c_str()); childEntries.push_back(entry); } } @@ -1569,43 +1654,44 @@ private: // If all children were deleted then this page should be deleted as of the new version // Note that if a single range clear covered the entire page then we should not get this far if(childEntries.empty()) { - // TODO: delete page as of new version - VersionedChildrenT c({}); - debug_printf("%p All internal page children were deleted, returning %s\n", THIS, toString(c).c_str()); - return c; + if(self->singleVersion) { + debug_printf("%p All internal page children were deleted #2 at version %lld\n", THIS, version); + } + else { + VersionedKeyToPageSetT c({version, {} }); + debug_printf("%p All internal page children were deleted #3 at version %lld, adding %s\n", THIS, version, toString(c).c_str()); + result.push_back(c); + } } + else { + // TODO: Track split points across iterations of this loop, so that they don't shift unnecessarily and + // cause unnecessary path copying - // TODO: Track split points across iterations of this loop, so that they don't shift unnecessarily and - // cause unnecessary path copying + IPager *pager = self->m_pager; + std::vector pages = buildPages(false, *lowerBound, *upperBound, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); - IPager *pager = self->m_pager; - std::vector pages = buildPages(false, *lowerBound, *upperBound, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); + // Write page(s), use version 0 to replace latest version if only writing one page + Version writeVersion = self->singleVersion ? self->getLastCommittedVersion() + 1 : version; + std::vector newPageIDs = self->writePages(pages, writeVersion, root, page, upperBound, THIS); - // Write page(s), use version 0 to replace latest version if only writing one page - Version writeVersion = self->singleVersion ? self->getLastCommittedVersion() + 1 : version; - std::vector newPageIDs = self->writePages(pages, writeVersion, root, page, upperBound, THIS); + // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page + if(root == self->m_root) { + self->buildNewRoot(writeVersion, pages, newPageIDs, page); + } - // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page - if(root == self->m_root) - self->buildNewRoot(writeVersion, pages, newPageIDs, page); + result.resize(result.size()+1); + result.back().first = writeVersion; - result.resize(result.size()+1); - result.back().first = writeVersion; + for(int i=0; i 1 && result.back().second == result.end()[-2].second) { - debug_printf("%p Output same as last version, popping it.\n", THIS); - result.pop_back(); + debug_printf("%p Added versioned child set #3: %s\n", THIS, toString(result.back()).c_str()); } - debug_printf("%p Added versioned child set: %s\n", THIS, toString(result.back()).c_str()); } else { debug_printf("%p Version 0 has no changes\n", THIS); result.push_back({0, {{*lowerBound, root}}}); - debug_printf("%p Added versioned child set: %s\n", THIS, toString(result.back()).c_str()); + debug_printf("%p Added versioned child set #4: %s\n", THIS, toString(result.back()).c_str()); } if (nextVersion == std::numeric_limits::max()) @@ -1646,7 +1732,7 @@ private: self->printMutationBuffer(mutations); } - VersionedChildrenT _ = wait(commitSubtree(self, mutations, self->m_pager->getReadSnapshot(latestVersion), self->m_root, &dbBegin, &dbEnd)); + VersionedChildrenT newRoot = wait(commitSubtree(self, mutations, self->m_pager->getReadSnapshot(latestVersion), self->m_root, &dbBegin, &dbEnd, &dbBegin, &dbEnd)); self->m_pager->setLatestVersion(writeVersion); debug_printf("%s: Committing pager %lld\n", self->m_name.c_str(), writeVersion); @@ -1809,13 +1895,22 @@ private: self->ensureUnshared(); loop { - if(self->pageCursor->cursor.seekLessThanOrEqual(query)) { + bool success = self->pageCursor->cursor.seekLessThanOrEqual(query); + + // Skip backwards over internal page entries that do not link to child pages + if(!self->pageCursor->isLeaf()) { + // While record has no value, move again + while(success && !self->pageCursor->cursor.get().value.present()) { + success = self->pageCursor->cursor.movePrev(); + } + } + + if(success) { // If we found a record <= query at a leaf page then return success if(self->pageCursor->isLeaf()) { return true; } - // Otherwise move to next child page Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); self->pageCursor = child; } @@ -1837,6 +1932,14 @@ private: self->ensureUnshared(); bool success = self->pageCursor->cursor.valid() && (forward ? self->pageCursor->cursor.moveNext() : self->pageCursor->cursor.movePrev()); + // Skip over internal page entries that do not link to child pages + if(!self->pageCursor->isLeaf()) { + // While record has no value, move again + while(success && !self->pageCursor->cursor.get().value.present()) { + success = forward ? self->pageCursor->cursor.moveNext() : self->pageCursor->cursor.movePrev(); + } + } + // Stop if successful or there's no parent to move to if(success || !self->pageCursor->parent) { break; @@ -1853,6 +1956,14 @@ private: // While not on a leaf page, move down to get to one. while(!self->pageCursor->isLeaf()) { + // Skip over internal page entries that do not link to child pages + while(!self->pageCursor->cursor.get().value.present()) { + bool success = forward ? self->pageCursor->cursor.moveNext() : self->pageCursor->cursor.movePrev(); + if(!success) { + return false; + } + } + Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); bool success = forward ? child->cursor.moveFirst() : child->cursor.moveLast(); self->pageCursor = child; @@ -1887,11 +1998,20 @@ private: // Move to first or last record in the page bool success = begin ? self->pageCursor->cursor.moveFirst() : self->pageCursor->cursor.moveLast(); + // Skip over internal page entries that do not link to child pages + if(!self->pageCursor->isLeaf()) { + // While record has no value, move past it + while(success && !self->pageCursor->cursor.get().value.present()) { + success = begin ? self->pageCursor->cursor.moveNext() : self->pageCursor->cursor.movePrev(); + } + } + // If it worked, return true if we've reached a leaf page otherwise go to the next child if(success) { if(self->pageCursor->isLeaf()) { return true; } + Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); self->pageCursor = child; } @@ -1987,7 +2107,7 @@ private: // for equal use cmp == 0 ACTOR static Future find_impl(Cursor *self, KeyRef key, bool needValue, int cmp) { // Search for the last key at or before (key, version, \xff) - state RedwoodRecordRef query(key, self->m_version); + state RedwoodRecordRef query(key, self->m_version, {}, 0, std::numeric_limits::max()); self->m_kv.reset(); wait(success(self->m_cur1.seekLTE(query))); @@ -2380,8 +2500,8 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version // Randomly use the cursor for something else first. if(g_random->coinflip()) { - debug_printf("VerifyRange(@%lld, %s, %s): Dummy seek\n", v, start.toString().c_str(), end.toString().c_str()); state Key randomKey = randomKV().key; + debug_printf("VerifyRange(@%lld, %s, %s): Dummy seek to '%s'\n", v, start.toString().c_str(), end.toString().c_str(), randomKey.toString().c_str()); wait(g_random->coinflip() ? cur->findFirstEqualOrGreater(randomKey, true, 0) : cur->findLastLessOrEqual(randomKey, true, 0)); } @@ -2397,6 +2517,7 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version if(i == iEnd) break; ++i; + if(iLast->first.second <= v && iLast->second.present() && ( @@ -2404,8 +2525,10 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version || i->first.first != iLast->first.first || i->first.second > v ) - ) + ) { + debug_printf("VerifyRange(@%lld, %s, %s) Found key in written map: %s\n", v, start.toString().c_str(), end.toString().c_str(), iLast->first.first.c_str()); break; + } } if(iLast == iEnd) { @@ -2428,6 +2551,8 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version break; } + ASSERT(errors == 0); + results.push_back(KeyValue(KeyValueRef(cur->getKey(), cur->getValue()))); wait(cur->next(true)); } @@ -2545,7 +2670,7 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, return errors; } -ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, std::map, Optional> *written, int *pErrorCount) { +ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, std::map, Optional> *written, int *pErrorCount, bool serial) { state Future vall; state Future vrange; @@ -2557,12 +2682,24 @@ ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, v = btree->getLastCommittedVersion(); debug_printf("Verifying at latest committed version %lld\n", v); vall = verifyRange(btree, LiteralStringRef(""), LiteralStringRef("\xff\xff"), v, written, pErrorCount); + if(serial) { + wait(success(vall)); + } vrange = verifyRange(btree, randomKV().key, randomKV().key, v, written, pErrorCount); + if(serial) { + wait(success(vrange)); + } } else { debug_printf("Verifying through version %lld\n", v); vall = verifyAll(btree, v, written, pErrorCount); + if(serial) { + wait(success(vall)); + } vrange = verifyRange(btree, randomKV().key, randomKV().key, g_random->randomInt(1, v + 1), written, pErrorCount); + if(serial) { + wait(success(vrange)); + } } wait(success(vall) && success(vrange)); @@ -2880,6 +3017,12 @@ TEST_CASE("!/redwood/correctness") { state std::string pagerFile = "unittest_pageFile"; IPager *pager; + state bool serialTest = g_random->coinflip(); + state bool shortTest = g_random->coinflip(); + state bool singleVersion = true; // Multi-version mode is broken / not finished + + printf("serialTest: %d shortTest: %d singleVersion: %d\n", serialTest, shortTest, singleVersion); + if(useDisk) { printf("Deleting existing test data...\n"); deleteFile(pagerFile); @@ -2891,8 +3034,7 @@ TEST_CASE("!/redwood/correctness") { pager = createMemoryPager(); printf("Initializing...\n"); - state int pageSize = g_random->coinflip() ? pager->getUsablePageSize() : g_random->randomInt(200, 400); - state bool singleVersion = g_random->random01() < .05; + state int pageSize = shortTest ? 200 : (g_random->coinflip() ? pager->getUsablePageSize() : g_random->randomInt(200, 400)); state VersionedBTree *btree = new VersionedBTree(pager, pagerFile, singleVersion, pageSize); wait(btree->init()); @@ -2900,10 +3042,10 @@ TEST_CASE("!/redwood/correctness") { // a situation where the tree cannot be grown upward with decreasing level size. // TODO: Handle arbitrarily large keys state int maxKeySize = g_random->randomInt(4, pageSize * 2); - state int maxValueSize = g_random->randomInt(0, pageSize * 2); - state int maxCommitSize = 5e6; - state int mutationBytesTarget = randomSize(50e6); - state double clearChance = g_random->random01() * .001; // at most 1 in 1000 + state int maxValueSize = g_random->randomInt(0, pageSize * 4); + state int maxCommitSize = shortTest ? 1000 : randomSize(10e6); + state int mutationBytesTarget = shortTest ? 5000 : randomSize(50e6); + state double clearChance = g_random->random01() * .01; // at most 1 in 100 printf("Using page size %d, max key size %d, max value size %d, clearchance %f, total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, clearChance, mutationBytesTarget); @@ -2927,8 +3069,8 @@ TEST_CASE("!/redwood/correctness") { state int mutationBytesTargetThisCommit = randomSize(maxCommitSize); state PromiseStream committedVersions; - state Future verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount); - state Future randomTask = randomReader(btree) || btree->getError(); + state Future verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount, serialTest); + state Future randomTask = serialTest ? Void() : (randomReader(btree) || btree->getError()); state Future commit = Void(); @@ -2964,7 +3106,7 @@ TEST_CASE("!/redwood/correctness") { ++rangeClears; KeyRangeRef range(start, end); - debug_printf(" Clear '%s' to '%s' @%lld\n", start.toString().c_str(), end.toString().c_str(), version); + debug_printf(" Mutation: Clear '%s' to '%s' @%lld\n", start.toString().c_str(), end.toString().c_str(), version); auto e = written.lower_bound(std::make_pair(start.toString(), 0)); if(e != written.end()) { auto last = e; @@ -2974,7 +3116,7 @@ TEST_CASE("!/redwood/correctness") { ++e; // If e key is different from last and last was present then insert clear for last's key at version if(last != eEnd && ((e == eEnd || e->first.first != last->first.first) && last->second.present())) { - debug_printf(" Clearing key '%s' @%lld\n", last->first.first.c_str(), version); + debug_printf(" Mutation: Clearing key '%s' @%lld\n", last->first.first.c_str(), version); keyBytesCleared += last->first.first.size(); mutationBytes += last->first.first.size(); @@ -3004,7 +3146,7 @@ TEST_CASE("!/redwood/correctness") { kv.key = StringRef(kv.arena(), *i); } - debug_printf(" Set '%s' -> '%s' @%lld\n", kv.key.toString().c_str(), kv.value.toString().c_str(), version); + debug_printf(" Mutation: Set '%s' -> '%s' @%lld\n", kv.key.toString().c_str(), kv.value.toString().c_str(), version); ++sets; keyBytesInserted += kv.key.size(); @@ -3039,11 +3181,21 @@ TEST_CASE("!/redwood/correctness") { return Void(); }); + if(serialTest) { + // Wait for commit, wait for verification, then start new verification + wait(commit); + committedVersions.sendError(end_of_stream()); + debug_printf("Waiting for verification to complete.\n"); + wait(verifyTask); + committedVersions = PromiseStream(); + verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount, serialTest); + } + mutationBytesThisCommit = 0; mutationBytesTargetThisCommit = randomSize(maxCommitSize); // Recover from disk at random - if(useDisk && g_random->random01() < .02) { + if(!serialTest && useDisk && g_random->random01() < .02) { printf("Recovering from disk.\n"); // Wait for outstanding commit @@ -3070,7 +3222,7 @@ TEST_CASE("!/redwood/correctness") { // Create new promise stream and start the verifier again committedVersions = PromiseStream(); - verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount); + verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount, serialTest); randomTask = randomReader(btree) || btree->getError(); } From e9197a8f7061df06ef3969ba8538442b82bab9b7 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Tue, 21 May 2019 22:19:14 -0700 Subject: [PATCH 010/128] Added time limit. --- fdbserver/VersionedBTree.actor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index f66ab5aa9f..1775ed8c19 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -3020,6 +3020,7 @@ TEST_CASE("!/redwood/correctness") { state bool serialTest = g_random->coinflip(); state bool shortTest = g_random->coinflip(); state bool singleVersion = true; // Multi-version mode is broken / not finished + state double startTime = now(); printf("serialTest: %d shortTest: %d singleVersion: %d\n", serialTest, shortTest, singleVersion); @@ -3075,6 +3076,10 @@ TEST_CASE("!/redwood/correctness") { state Future commit = Void(); while(mutationBytes.get() < mutationBytesTarget) { + if(now() - startTime > 600) { + mutationBytesTarget = mutationBytes.get(); + } + // Sometimes advance the version if(g_random->random01() < 0.10) { ++version; From f99c36aad24ff048692afb5bf68da7a1f527419d Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 22 May 2019 00:23:31 -0700 Subject: [PATCH 011/128] Fixed merge mistake. --- fdbserver/VersionedBTree.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 775ad5692e..bc6747d491 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -1591,7 +1591,7 @@ private: continue; } - debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, pageID, children.size()); + debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, childPageIDs[i], children.size()); if(REDWOOD_DEBUG) { for(auto &versionedPageSet : children) { debug_printf("%p version " PRId64 "\n", THIS, versionedPageSet.first); From 0fb8612ef51126b07efb34d7bc0a7f94eeb4e19d Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 22 May 2019 03:40:18 -0700 Subject: [PATCH 012/128] debug_printf_noop() was incorrectly defined as a function, which still has a runtime cost of argument evaluation. --- fdbserver/IPager.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/fdbserver/IPager.h b/fdbserver/IPager.h index 5cb30dd3f5..5823588a17 100644 --- a/fdbserver/IPager.h +++ b/fdbserver/IPager.h @@ -31,8 +31,7 @@ #define debug_printf_always(...) { fprintf(stdout, "%s %f ", g_network->getLocalAddress().toString().c_str(), now()), fprintf(stdout, __VA_ARGS__); fflush(stdout); } -template -void debug_printf_noop(T&&...) {} +#define debug_printf_noop(...) #if REDWOOD_DEBUG #define debug_printf debug_printf_always From 02882dbf0088d2c2c9ecb9d55cc83ca97cf5e479 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 29 May 2019 06:23:32 -0700 Subject: [PATCH 013/128] Checkpointing progress, RedwoodRecordRef and DeltaTree tests pass but BTree test does not. RedwoodRecordRef::Delta rewritten to actually do prefix compression on key and integer fields. Added related unit tests and benchmarks. Some improvements to DeltaTree and requirements on its T and Delta types to avoid repeated common prefix discovery. --- fdbserver/DeltaTree.h | 54 +- fdbserver/VersionedBTree.actor.cpp | 933 ++++++++++++++++++++++------- flow/Platform.h | 24 + 3 files changed, 796 insertions(+), 215 deletions(-) diff --git a/fdbserver/DeltaTree.h b/fdbserver/DeltaTree.h index 682f7ebb99..6ab0da62f1 100644 --- a/fdbserver/DeltaTree.h +++ b/fdbserver/DeltaTree.h @@ -39,16 +39,21 @@ // // Must be compatible with Standalone and must implement the following additional methods: // -// // Writes to d a delta which can create *this from prev and next -// void writeDelta(dT &d, const T &prev, const T &next) +// // Writes to d a delta which can create *this from base +// // commonPrefix can be passed in if known +// void writeDelta(dT &d, const T &base, int commonPrefix = -1) const; // // // Compare *this to t, returns < 0 for less than, 0 for equal, > 0 for greater than -// int compare(const T &rhs) +// int compare(const T &rhs) const; // -// // Returns the delta size needed to make *this from base +// // Get the common prefix bytes between *this and base +// // skip is a hint of how many prefix bytes are already known to be the same +// int getCommonPrefixLen(const T &base, int skip) const; +// +// // Returns the size of the delta object needed to make *this from base // // TODO: Explain contract required for deltaSize to be used to predict final // // balanced tree size incrementally while adding sorted items to a build set -// int deltaSize(const T &base) +// int deltaSize(const T &base) const; // // DeltaT requirements // @@ -56,7 +61,13 @@ // int size(); // // // Returns the T created by applying the delta to prev or next -// T apply(const T &prev, const T &next, Arena &localStorage) +// T apply(const T &base, Arena &localStorage) const; +// +// // Stores a boolean which DeltaTree will later use to determine the base node for a node's delta +// void setPrefixSource(bool val); +// +// // Retrieves the previously stored boolean +// bool getPrefixSource() const; // template struct DeltaTree { @@ -108,7 +119,7 @@ public: struct DecodedNode { DecodedNode(Node *raw, const T *prev, const T *next, Arena &arena) : raw(raw), parent(nullptr), left(nullptr), right(nullptr), prev(prev), next(next), - item(raw->delta->apply(*prev, *next, arena)) + item(raw->delta->apply(raw->delta->getPrefixSource() ? *prev : *next, arena)) { //printf("DecodedNode1 raw=%p delta=%s\n", raw, raw->delta->toString().c_str()); } @@ -117,7 +128,7 @@ public: : parent(parent), raw(raw), left(nullptr), right(nullptr), prev(left ? parent->prev : &parent->item), next(left ? &parent->item : parent->next), - item(raw->delta->apply(*prev, *next, arena)) + item(raw->delta->apply(raw->delta->getPrefixSource() ? *prev : *next, arena)) { //printf("DecodedNode2 raw=%p delta=%s\n", raw, raw->delta->toString().c_str()); } @@ -338,11 +349,32 @@ private: int mid = perfectSubtreeSplitPointCached(count); const T &item = begin[mid]; - item.writeDelta(*root.delta, *prev, *next); + // Get the common prefix length between next and prev + // Since mid is between them, we can skip that length to determine the common prefix length + // between mid and prev and between mid and next. + int nextPrevCommon = prev->getCommonPrefixLen(*next, 0); + int commonWithPrev = begin[mid].getCommonPrefixLen(*prev, nextPrevCommon); + int commonWithNext = begin[mid].getCommonPrefixLen(*next, nextPrevCommon); + + bool prefixSourcePrev; + int commonPrefix; + const T *base; + if(commonWithNext > commonWithPrev) { + prefixSourcePrev = true; + commonPrefix = commonWithPrev; + base = prev; + } + else { + prefixSourcePrev = false; + commonPrefix = commonWithNext; + base = next; + } + root.delta->setPrefixSource(prefixSourcePrev); + + int deltaSize = item.writeDelta(*root.delta, *base, commonPrefix); //printf("Serialized %s to %p\n", item.toString().c_str(), root.delta); - // Delta can have variable size, so calculate wptr which is where to write to next - int deltaSize = root.delta->size(); + // Continue writing after the serialized Delta. uint8_t *wptr = (uint8_t *)root.delta + deltaSize; // Serialize left child diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index bc6747d491..08dbf5caad 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -36,32 +36,200 @@ #include "flow/actorcompiler.h" #include +// TODO: Move this to a flow header once it is mature. +struct SplitStringRef { + StringRef a; + StringRef b; + + SplitStringRef(StringRef a = StringRef(), StringRef b = StringRef()) : a(a), b(b) { + } + + SplitStringRef(Arena &arena, const SplitStringRef &toCopy) + : a(toStringRef(arena)), b() { + } + + SplitStringRef prefix(int len) const { + if(len <= a.size()) { + return SplitStringRef(a.substr(0, len)); + } + len -= a.size(); + return SplitStringRef(a, b.substr(0, len)); + } + + StringRef toStringRef(Arena &arena) const { + StringRef c = makeString(size(), arena); + memcpy(mutateString(c), a.begin(), a.size()); + memcpy(mutateString(c) + a.size(), b.begin(), b.size()); + return c; + } + + Standalone toStringRef() const { + Arena a; + return Standalone(toStringRef(a), a); + } + + int size() const { + return a.size() + b.size(); + } + + int expectedSize() const { + return size(); + } + + std::string toString() const { + return format("%s%s", a.toString().c_str(), b.toString().c_str()); + } + + std::string toHexString() const { + return format("%s%s", a.toHexString().c_str(), b.toHexString().c_str()); + } + + struct const_iterator { + const uint8_t *ptr; + const uint8_t *end; + const uint8_t *next; + + inline bool operator==(const const_iterator &rhs) const { + return ptr == rhs.ptr; + } + + inline const_iterator & operator++() { + ++ptr; + if(ptr == end) { + ptr = next; + } + return *this; + } + + inline const_iterator & operator+(int n) { + ptr += n; + if(ptr >= end) { + ptr = next + (ptr - end); + } + return *this; + } + + inline uint8_t operator *() const { + return *ptr; + } + }; + + inline const_iterator begin() const { + return {a.begin(), a.end(), b.begin()}; + } + + inline const_iterator end() const { + return {b.end()}; + } + + template + int compare(const StringT &rhs) const { + auto j = begin(); + auto k = rhs.begin(); + auto jEnd = end(); + auto kEnd = rhs.end(); + + while(j != jEnd && k != kEnd) { + int cmp = *j - *k; + if(cmp != 0) { + return cmp; + } + } + + // If we've reached the end of *this, then values are equal if rhs is also exhausted, otherwise *this is less than rhs + if(j == jEnd) { + return k == kEnd ? 0 : -1; + } + + return 1; + } + +}; + #define STR(x) LiteralStringRef(x) struct RedwoodRecordRef { + typedef uint8_t byte; RedwoodRecordRef(KeyRef key = KeyRef(), Version ver = 0, Optional value = {}, uint32_t chunkTotal = 0, uint32_t chunkStart = 0) : key(key), version(ver), value(value), chunk({chunkTotal, chunkStart}) { - ASSERT(value.present() || !isMultiPart()); } - RedwoodRecordRef(Arena &arena, const RedwoodRecordRef &toCopy) { - *this = toCopy; - key = KeyRef(arena, key); + RedwoodRecordRef(Arena &arena, const RedwoodRecordRef &toCopy) + : key(arena, toCopy.key), version(toCopy.version), chunk(toCopy.chunk) { if(value.present()) { value = ValueRef(arena, toCopy.value.get()); } } + RedwoodRecordRef(KeyRef key, Optional value, const byte intFields[14]) + : key(key), value(value) + { + deserializeIntFields(intFields); + } + + // TODO: This probably is not actually needed, it only helps make debug output more clear RedwoodRecordRef withoutValue() const { return RedwoodRecordRef(key, version, {}, chunk.total, chunk.start); } + // Returns how many bytes are in common between the integer fields of *this and other, assuming that + // all values are BigEndian, version is 64 bits, chunk total is 24 bits, and chunk start is 24 bits + int getCommonIntFieldPrefix(const RedwoodRecordRef &other) const { + if(version != other.version) { + return clzll(version ^ other.version) >> 3; + } + + if(chunk.total != other.chunk.total) { + // the -1 is because we are only considering the lower 3 bytes + return 8 + (clz(chunk.total ^ other.chunk.total) >> 3) - 1; + } + + if(chunk.start != other.chunk.start) { + // the -1 is because we are only considering the lower 3 bytes + return 11 + (clz(chunk.start ^ other.chunk.start) >> 3) - 1; + } + + return 14; + } + + // Find the common prefix between two records, assuming that the first + // skip bytes are the same. + inline int getCommonPrefixLen(const RedwoodRecordRef &other, int skip) const { + int skipStart = std::min(skip, key.size()); + int common = skipStart + commonPrefixLength(key.begin() + skipStart, other.key.begin() + skipStart, std::min(other.key.size(), key.size()) - skipStart); + + if(common == key.size() && key.size() == other.key.size()) { + common += getCommonIntFieldPrefix(other); + } + + return common; + } + + static const int intFieldArraySize = 14; + + // Write big endian values of version (64 bits), total (24 bits), and start (24 bits) fields + // to an array of 14 bytes + void serializeIntFields(byte *dst) const { + *(uint32_t *)(dst + 10) = bigEndian32(chunk.start); + *(uint32_t *)(dst + 7) = bigEndian32(chunk.total); + *(uint64_t *)dst = bigEndian64(version); + } + + // Initialize int fields from the array format that serializeIntFields produces + void deserializeIntFields(const byte *src) { + version = bigEndian64(*(uint64_t *)src); + chunk.total = bigEndian32(*(uint32_t *)(src + 7)) & 0xffffff; + chunk.start = bigEndian32(*(uint32_t *)(src + 10)) & 0xffffff; + } + + // TODO: Use SplitStringRef (unless it ends up being slower) KeyRef key; - Version version; Optional value; + Version version; struct { uint32_t total; + // TODO: Change start to chunk number. uint32_t start; } chunk; @@ -70,7 +238,7 @@ struct RedwoodRecordRef { } bool isMultiPart() const { - return value.present() && chunk.total != 0; + return chunk.total != 0; } // Generate a kv shard from a complete kv @@ -79,65 +247,205 @@ struct RedwoodRecordRef { return RedwoodRecordRef(key, version, value.get().substr(start, len), value.get().size(), start); } -#pragma pack(push,1) - struct Delta { - // TODO: Make this actually a delta - enum EFlags {HAS_VALUE = 1, HAS_VERSION = 2, IS_MULTIPART = 4}; + class Writer { + public: + Writer(byte *ptr) : wptr(ptr) {} - uint8_t flags; - uint16_t keySize; - uint8_t bytes[]; + byte *wptr; - RedwoodRecordRef apply(const RedwoodRecordRef &prev, const RedwoodRecordRef &next, Arena arena) { - RedwoodRecordRef r; - const uint8_t *rptr = bytes; - r.key = StringRef(rptr, keySize); - rptr += keySize; - if(flags & HAS_VERSION) { - r.version = (*(Version *)rptr); - rptr += sizeof(Version); - } - else { - r.version = 0; - } - if(flags & HAS_VALUE) { - uint16_t valueSize = *(uint16_t *)rptr; - rptr += 2; - r.value = StringRef(rptr, valueSize); - rptr += valueSize; - if(flags & IS_MULTIPART) { - r.chunk.total = *(uint32_t *)rptr; - rptr += sizeof(uint32_t); - r.chunk.start = *(uint32_t *)rptr; - } - else { - r.chunk.total = 0; - r.chunk.start = 0; - } + template void write(const T &in) { + *(T *)wptr = in; + wptr += sizeof(T); + } + + // Write a big endian 1 or 2 byte integer using the high bit of the first byte as an "extension" bit. + // Values > 15 bits in length are not valid input but this is not checked for. + void writeVarInt(int x) { + if(x >= 128) { + *wptr++ = (uint8_t)( (x >> 8) & 0x7f ); } + *wptr++ = (uint8_t)x; + } + + void writeString(StringRef s) { + memcpy(wptr, s.begin(), s.size()); + wptr += s.size(); + } + + }; + + class Reader { + public: + Reader(const void *ptr) : rptr((const byte *)ptr) {} + + const byte *rptr; + + template T read() { + T r = *(const T *)rptr; + rptr += sizeof(T); return r; } - int size() const { - int s = sizeof(Delta) + keySize; - if(flags & HAS_VERSION) { - s += sizeof(Version); - } - if(flags & HAS_VALUE) { - s += *(uint16_t *)((uint8_t *)this + s); - s += sizeof(uint16_t); - if(flags & IS_MULTIPART) { - s += (2 * sizeof(uint32_t)); - } + // Read a big endian 1 or 2 byte integer using the high bit of the first byte as an "extension" bit. + int readVarInt() { + int x = *rptr++; + // If the high bit is set + if(x & 0x80) { + // Clear the high bit + x &= 0x7f; + // Shift low byte left + x <<= 8; + // Read the new low byte and OR it in + x |= *rptr++; } + + return x; + } + + StringRef readString(int len) { + StringRef s(rptr, len); + rptr += len; return s; } + const byte * readBytes(int len) { + const byte *b = rptr; + rptr += len; + return b; + } + }; + +#pragma pack(push,1) + struct Delta { + + // Serialized Format + // + // 1 byte for Flags + a 4 bit length + // borrow source is prev ancestor - 0 or 1 + // has_key_suffix + // has_value + // has_version + // other_fields suffix len - 4 bits + // + // If has value and value is not 4 bytes + // 1 byte value length + // + // 1 or 2 bytes for Prefix Borrow Length (hi bit indicates presence of second byte) + // + // IF has_key_suffix is set + // 1 or 2 bytes for Key Suffix Length + // + // Key suffix bytes + // Meta suffix bytes + // Value bytes + // + // For a series of RedwoodRecordRef's containing shards of the same KV pair where the key size is < 104 bytes, + // the overhead per middle chunk is 7 bytes: + // 4 bytes of child pointers in the DeltaTree Node + // 1 flag byte + // 1 prefix borrow length byte + // 1 meta suffix byte describing chunk start position + + enum EFlags { + PREFIX_SOURCE = 0x80, + HAS_KEY_SUFFIX = 0x40, + HAS_VALUE = 0x20, + HAS_VERSION = 0x10, + INT_FIELD_SUFFIX_BITS = 0x0f + }; + + uint8_t flags; + byte data[]; + + void setPrefixSource(bool val) { + if(val) { + flags |= PREFIX_SOURCE; + } + else { + flags &= ~PREFIX_SOURCE; + } + } + + bool getPrefixSource() const { + return flags & PREFIX_SOURCE; + } + + RedwoodRecordRef apply(const RedwoodRecordRef &base, Arena &arena) const { + Reader r(data); + + int intFieldSuffixLen = flags & INT_FIELD_SUFFIX_BITS; + int prefixLen = r.readVarInt(); + int valueLen = (flags & HAS_VALUE) ? r.read() : 0; + + StringRef k; + + int keyPrefixLen = std::min(prefixLen, base.key.size()); + int intFieldPrefixLen = prefixLen - keyPrefixLen; + int keySuffixLen = (flags & HAS_KEY_SUFFIX) ? r.readVarInt() : 0; + + if(keySuffixLen > 0) { + k = makeString(keyPrefixLen + keySuffixLen, arena); + memcpy(mutateString(k), base.key.begin(), keyPrefixLen); + memcpy(mutateString(k) + keyPrefixLen, r.readString(keySuffixLen).begin(), keySuffixLen); + } + else { + k = base.key.substr(0, keyPrefixLen); + } + + // Now decode the integer fields + const byte *intFieldSuffix = r.readBytes(intFieldSuffixLen); + + // Create big endian array in which to reassemble the integer fields from prefix and suffix bytes + byte intFields[intFieldArraySize]; + + // If borrowing any bytes, get the source's integer field array + if(intFieldPrefixLen > 0) { + base.serializeIntFields(intFields); + } + else { + memset(intFields, 0, intFieldArraySize); + } + + // Version offset is used to skip the version bytes in the int field array when version is missing (aka 0) + int versionOffset = flags & HAS_VERSION ? 0 : 8; + + // If there are suffix bytes, copy those into place after the prefix + if(intFieldSuffixLen > 0) { + memcpy(intFields + versionOffset + intFieldPrefixLen, intFieldSuffix, intFieldSuffixLen); + } + + // Zero out any remaining bytes if the array was initialized from base + if(intFieldPrefixLen > 0) { + for(int i = versionOffset + intFieldPrefixLen + intFieldSuffixLen; i < intFieldArraySize; ++i) { + intFields[i] = 0; + } + } + + return RedwoodRecordRef(k, flags & HAS_VALUE ? r.readString(valueLen) : Optional(), intFields); + } + + int size() const { + Reader r(data); + + int intFieldSuffixLen = flags & INT_FIELD_SUFFIX_BITS; + int prefixLen = r.readVarInt(); + int valueLen = (flags & HAS_VALUE) ? r.read() : 0; + int keySuffixLen = (flags & HAS_KEY_SUFFIX) ? r.readVarInt() : 0; + + return sizeof(Delta) + r.rptr - data + intFieldSuffixLen + valueLen + keySuffixLen; + } + + // Delta can't be determined without the RedwoodRecordRef upon which the Delta is based. std::string toString() const { - return format("DELTA{ %s | %s }", - StringRef((const uint8_t *)this, sizeof(Delta)).toHexString().c_str(), - StringRef(bytes, size() - sizeof(Delta)).toHexString().c_str() - ); + Reader r(data); + + int intFieldSuffixLen = flags & INT_FIELD_SUFFIX_BITS; + int prefixLen = r.readVarInt(); + int valueLen = (flags & HAS_VALUE) ? r.read() : 0; + int keySuffixLen = (flags & HAS_KEY_SUFFIX) ? r.readVarInt() : 0; + + return format("flags: %02x prefixLen: %d keySuffixLen: %d intFieldSuffix: %d valueLen %d raw: %s", + flags, prefixLen, keySuffixLen, intFieldSuffixLen, valueLen, StringRef((const uint8_t *)this, size()).toHexString().c_str()); } }; #pragma pack(pop) @@ -184,46 +492,111 @@ struct RedwoodRecordRef { } int deltaSize(const RedwoodRecordRef &base) const { - int s = sizeof(Delta) + key.size(); - if(version != 0) { - s += sizeof(Version); - } + int size = sizeof(Delta); + if(value.present()) { - s += 2; - s += value.get().size(); - if(isMultiPart()) { - s += (2 * sizeof(uint32_t)); - } + size += value.get().size(); + ++size; } - return s; + + int prefixLen = getCommonPrefixLen(base, 0); + size += (prefixLen > 128) ? 2 : 1; + + int intFieldPrefixLen; + + // Currently using a worst-guess guess where int fields in suffix are stored in their entirety if nonzero. + if(prefixLen < key.size()) { + int keySuffixLen = key.size() - prefixLen; + size += (keySuffixLen > 128) ? 2 : 1; + size += keySuffixLen; + intFieldPrefixLen = 0; + } + else { + intFieldPrefixLen = prefixLen - key.size(); + } + + if(version == 0 && chunk.total == 0 && chunk.start == 0) { + // No int field suffix needed + } + else { + byte fields[intFieldArraySize]; + serializeIntFields(fields); + + const byte *end = fields + intFieldArraySize - 1; + int trailingNulls = 0; + while(*end-- == 0) { + ++trailingNulls; + } + + size += std::max(0, intFieldArraySize - intFieldPrefixLen - trailingNulls); + } + + return size; } - void writeDelta(Delta &d, const RedwoodRecordRef &prev, const RedwoodRecordRef &next) const { - d.flags = value.present() ? Delta::EFlags::HAS_VALUE : 0; - d.keySize = key.size(); - uint8_t *wptr = d.bytes; - memcpy(wptr, key.begin(), key.size()); - wptr += key.size(); - if(version != 0) { - d.flags |= Delta::EFlags::HAS_VERSION; - *(Version *)wptr = (version); - wptr += sizeof(Version); + // commonPrefix between *this and base can be passed if known + int writeDelta(Delta &d, const RedwoodRecordRef &base, int commonPrefix = -1) const { + d.flags = version == 0 ? 0 : Delta::HAS_VERSION; + + if(commonPrefix < 0) { + commonPrefix = getCommonPrefixLen(base, 0); } + + Writer w(d.data); + + // prefixLen + w.writeVarInt(commonPrefix); + + // valueLen if(value.present()) { - *(uint16_t *)wptr = value.get().size(); - wptr += 2; - memcpy(wptr, value.get().begin(), value.get().size()); - wptr += value.get().size(); - if(isMultiPart()) { - d.flags |= Delta::EFlags::IS_MULTIPART; - *(uint32_t *)wptr = chunk.total; - wptr += sizeof(uint32_t); - *(uint32_t *)wptr = chunk.start; + d.flags |= Delta::HAS_VALUE; + w.write(value.get().size()); + } + + // keySuffixLen + if(key.size() > commonPrefix) { + d.flags |= Delta::HAS_KEY_SUFFIX; + + StringRef keySuffix = key.substr(commonPrefix); + w.writeVarInt(keySuffix.size()); + + // keySuffix + w.writeString(keySuffix); + } + + // This is a common case, where no int suffix is needed + if(version == 0 && chunk.total == 0 && chunk.start == 0) { + // The suffixLen bits in flags are already zero, so nothing to do here. + } + else { + byte fields[intFieldArraySize]; + serializeIntFields(fields); + + // Find the position of the first null byte from the right + // This for loop has no endPos > 0 check because it is known that the array contains non-null bytes + int endPos; + for(endPos = intFieldArraySize; fields[endPos - 1] == 0; --endPos); + + // Start copying after any prefix bytes that matched the int fields of the base + int intFieldPrefixLen = std::max(0, commonPrefix - key.size()); + int startPos = intFieldPrefixLen + (version == 0 ? 8 : 0); + int suffixLen = std::max(0, endPos - startPos); + + if(suffixLen > 0) { + w.writeString(StringRef(fields + startPos, suffixLen)); + d.flags |= suffixLen; } } + + if(value.present()) { + w.writeString(value.get()); + } + + return w.wptr - d.data + sizeof(Delta); } - static std::string kvformat(StringRef s, int hexLimit = -1) { + template + static std::string kvformat(StringRefT s, int hexLimit = -1) { bool hex = false; for(auto c : s) { @@ -238,7 +611,7 @@ struct RedwoodRecordRef { std::string toString(int hexLimit = 15) const { std::string r; - r += format("'%s' @" PRId64 " ", kvformat(key, hexLimit).c_str(), version); + r += format("'%s' @%" PRId64 " ", kvformat(key, hexLimit).c_str(), version); r += format("[%d/%d] ", chunk.start, chunk.total); if(value.present()) { r += format("-> '%s'", kvformat(value.get(), hexLimit).c_str()); @@ -289,7 +662,7 @@ struct BTreePage { std::string toString(bool write, LogicalPageID id, Version ver, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) const { std::string r; - r += format("BTreePage op=%s id=%d ver=" PRId64 " ptr=%p flags=0x%X count=%d kvBytes=%d extPages=%d\n lowerBound: %s\n upperBound: %s\n", + r += format("BTreePage op=%s id=%d ver=%" PRId64 " ptr=%p flags=0x%X count=%d kvBytes=%d extPages=%d\n lowerBound: %s\n upperBound: %s\n", write ? "write" : "read", id, ver, this, (int)flags, (int)count, (int)kvBytes, (int)extensionPageCount, lowerBound->toString().c_str(), upperBound->toString().c_str()); try { @@ -571,7 +944,7 @@ public: int64_t commitToPageStart; std::string toString(bool clearAfter = false) { - std::string s = format("set=" PRId64 " clear=" PRId64 " get=" PRId64 " getRange=" PRId64 " commit=" PRId64 " pageRead=" PRId64 " extPageRead=" PRId64 " pageWrite=" PRId64 " extPageWrite=" PRId64 " commitPage=" PRId64 " commitPageStart=" PRId64 "", + std::string s = format("set=%" PRId64 " clear=%" PRId64 " get=%" PRId64 " getRange=%" PRId64 " commit=%" PRId64 " pageRead=%" PRId64 " extPageRead=%" PRId64 " pageWrite=%" PRId64 " extPageWrite=%" PRId64 " commitPage=%" PRId64 " commitPageStart=%" PRId64 "", sets, clears, gets, getRanges, commits, pageReads, extPageReads, pageWrites, extPageWrites, commitToPage, commitToPageStart); if(clearAfter) { clear(); @@ -793,7 +1166,7 @@ private: } static std::string toString(const VersionedKeyToPageSetT &c) { - std::string r = format("Version " PRId64 " => [", c.first); + std::string r = format("Version %" PRId64 " => [", c.first); for(auto &o : c.second) { r += toString(o) + " "; } @@ -859,12 +1232,12 @@ private: std::string result; result.append("rangeClearVersion: "); if(rangeClearVersion.present()) - result.append(format("" PRId64 "", rangeClearVersion.get())); + result.append(format("%" PRId64 "", rangeClearVersion.get())); else result.append(""); result.append(" startKeyMutations: "); for(SingleKeyMutationsByVersion::value_type const &m : startKeyMutations) - result.append(format("[" PRId64 " => %s] ", m.first, m.second.toString().c_str())); + result.append(format("[%" PRId64 " => %s] ", m.first, m.second.toString().c_str())); return result; } }; @@ -993,14 +1366,14 @@ private: int oldPages = pages.size(); pages = buildPages(false, dbBegin, dbEnd, childEntries, 0, [=](){ return m_pager->newPageBuffer(); }, m_usablePageSizeOverride); - debug_printf("Writing a new root level at version " PRId64 " with %lu children across %lu pages\n", version, childEntries.size(), pages.size()); + debug_printf("Writing a new root level at version %" PRId64 " with %lu children across %lu pages\n", version, childEntries.size(), pages.size()); logicalPageIDs = writePages(pages, version, m_root, pPage, &dbEnd, nullptr); } } std::vector writePages(std::vector pages, Version version, LogicalPageID originalID, const BTreePage *originalPage, const RedwoodRecordRef *upperBound, void *actor_debug) { - debug_printf("%p: writePages(): %u @" PRId64 " -> %lu replacement pages\n", actor_debug, originalID, version, pages.size()); + debug_printf("%p: writePages(): %u @%" PRId64 " -> %lu replacement pages\n", actor_debug, originalID, version, pages.size()); ASSERT(version != 0 || pages.size() == 1); @@ -1015,7 +1388,7 @@ private: primaryLogicalPageIDs.push_back(m_pager->allocateLogicalPage()); } - debug_printf("%p: writePages(): Writing %lu replacement pages for %d at version " PRId64 "\n", actor_debug, pages.size(), originalID, version); + debug_printf("%p: writePages(): Writing %lu replacement pages for %d at version %" PRId64 "\n", actor_debug, pages.size(), originalID, version); for(int i=0; iallocateLogicalPage(); - debug_printf("%p: writePages(): Writing extension page op=write id=%u @" PRId64 " (%d of %lu) referencePage=%u\n", actor_debug, eid, version, e + 1, extPages.size(), id); + debug_printf("%p: writePages(): Writing extension page op=write id=%u @%" PRId64 " (%d of %lu) referencePage=%u\n", actor_debug, eid, version, e + 1, extPages.size(), id); newPage->extensionPages[e] = eid; // If replacing the primary page below (version == 0) then pass the primary page's ID as the reference page ID m_pager->writePage(eid, extPages[e], version, (version == 0) ? id : invalidLogicalPageID); ++counts.extPageWrites; } - debug_printf("%p: writePages(): Writing primary page op=write id=%u @" PRId64 " (+%lu extension pages)\n", actor_debug, id, version, extPages.size()); + debug_printf("%p: writePages(): Writing primary page op=write id=%u @%" PRId64 " (+%lu extension pages)\n", actor_debug, id, version, extPages.size()); m_pager->writePage(id, pages[i].firstPage, version); } else { - debug_printf("%p: writePages(): Writing normal page op=write id=%u @" PRId64 "\n", actor_debug, id, version); + debug_printf("%p: writePages(): Writing normal page op=write id=%u @%" PRId64 "\n", actor_debug, id, version); writePage(id, pages[i].firstPage, version, &pages[i].lowerBound, (i == pages.size() - 1) ? upperBound : &pages[i + 1].lowerBound); } } @@ -1098,7 +1471,7 @@ private: }; ACTOR static Future> readPage(Reference snapshot, LogicalPageID id, int usablePageSize, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound) { - debug_printf("readPage() op=read id=%u @" PRId64 " lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); + debug_printf("readPage() op=read id=%u @%" PRId64 " lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); wait(delay(0, TaskDiskRead)); state Reference result = wait(snapshot->getPhysicalPage(id)); @@ -1106,14 +1479,14 @@ private: state const BTreePage *pTreePage = (const BTreePage *)result->begin(); if(pTreePage->extensionPageCount == 0) { - debug_printf("readPage() Found normal page for op=read id=%u @" PRId64 "\n", id, snapshot->getVersion()); + debug_printf("readPage() Found normal page for op=read id=%u @%" PRId64 "\n", id, snapshot->getVersion()); } else { std::vector>> pageGets; pageGets.push_back(std::move(result)); for(int i = 0; i < pTreePage->extensionPageCount; ++i) { - debug_printf("readPage() Reading extension page op=read id=%u @" PRId64 " ext=%d/%d\n", pTreePage->extensionPages[i], snapshot->getVersion(), i + 1, (int)pTreePage->extensionPageCount); + debug_printf("readPage() Reading extension page op=read id=%u @%" PRId64 " ext=%d/%d\n", pTreePage->extensionPages[i], snapshot->getVersion(), i + 1, (int)pTreePage->extensionPageCount); pageGets.push_back(snapshot->getPhysicalPage(pTreePage->extensionPages[i])); } @@ -1124,7 +1497,7 @@ private: } if(result->userData == nullptr) { - debug_printf("readPage() Creating Reader for page id=%u @" PRId64 " lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); + debug_printf("readPage() Creating Reader for page id=%u @%" PRId64 " lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); result->userData = new BTreePage::BinaryTree::Reader(&pTreePage->tree(), lowerBound, upperBound); result->userDataDestructor = [](void *ptr) { delete (BTreePage::BinaryTree::Reader *)ptr; }; } @@ -1570,7 +1943,7 @@ private: std::vector childEntries; // For each Future - debug_printf("%p creating replacement pages for id=%d at Version " PRId64 "\n", THIS, root, version); + debug_printf("%p creating replacement pages for id=%d at Version %" PRId64 "\n", THIS, root, version); // In multi version mode if we're writing version 0 there is a chance that we don't have to write ourselves, if there are no changes in any child subtrees bool modified = self->singleVersion || version != 0; @@ -1594,7 +1967,7 @@ private: debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, childPageIDs[i], children.size()); if(REDWOOD_DEBUG) { for(auto &versionedPageSet : children) { - debug_printf("%p version " PRId64 "\n", THIS, versionedPageSet.first); + debug_printf("%p version %" PRId64 "\n", THIS, versionedPageSet.first); for(auto &boundaryPage : versionedPageSet.second) { debug_printf("%p '%s' -> Page id=%u\n", THIS, boundaryPage.first.toString().c_str(), boundaryPage.second); } @@ -1614,23 +1987,23 @@ private: // If there are no versions before the one we found, just update nextVersion and continue. if(cv == children.begin()) { - debug_printf("%p First version (" PRId64 ") in set is greater than current, setting nextVersion and continuing\n", THIS, cv->first); + debug_printf("%p First version (%" PRId64 ") in set is greater than current, setting nextVersion and continuing\n", THIS, cv->first); nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr " PRId64 " next " PRId64 "\n", THIS, version, nextVersion); + debug_printf("%p curr %" PRId64 " next %" PRId64 "\n", THIS, version, nextVersion); continue; } // If a version greater than the current version being written was found, update nextVersion if(cv != children.end()) { nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr " PRId64 " next " PRId64 "\n", THIS, version, nextVersion); + debug_printf("%p curr %" PRId64 " next %" PRId64 "\n", THIS, version, nextVersion); } // Go back one to the last version that was valid prior to or at the current version we are writing --cv; } - debug_printf("%p Using children for version " PRId64 " from this set, building version " PRId64 "\n", THIS, cv->first, version); + debug_printf("%p Using children for version %" PRId64 " from this set, building version %" PRId64 "\n", THIS, cv->first, version); // If page count isn't 1 then the root is definitely modified modified = modified || cv->second.size() != 1; @@ -1646,18 +2019,18 @@ private: } } - debug_printf("%p Finished pass through futurechildren. childEntries=%lu version=" PRId64 " nextVersion=" PRId64 "\n", THIS, childEntries.size(), version, nextVersion); + debug_printf("%p Finished pass through futurechildren. childEntries=%lu version=%" PRId64 " nextVersion=%" PRId64 "\n", THIS, childEntries.size(), version, nextVersion); if(modified) { // If all children were deleted then this page should be deleted as of the new version // Note that if a single range clear covered the entire page then we should not get this far if(childEntries.empty()) { if(self->singleVersion) { - debug_printf("%p All internal page children were deleted #2 at version " PRId64 "\n", THIS, version); + debug_printf("%p All internal page children were deleted #2 at version %" PRId64 "\n", THIS, version); } else { VersionedKeyToPageSetT c({version, {} }); - debug_printf("%p All internal page children were deleted #3 at version " PRId64 ", adding %s\n", THIS, version, toString(c).c_str()); + debug_printf("%p All internal page children were deleted #3 at version %" PRId64 ", adding %s\n", THIS, version, toString(c).c_str()); result.push_back(c); } } @@ -1720,11 +2093,11 @@ private: // Wait for the latest commit that started to be finished. wait(previousCommit); - debug_printf("%s: Beginning commit of version " PRId64 "\n", self->m_name.c_str(), writeVersion); + debug_printf("%s: Beginning commit of version %" PRId64 "\n", self->m_name.c_str(), writeVersion); // Get the latest version from the pager, which is what we will read at Version latestVersion = wait(self->m_pager->getLatestVersion()); - debug_printf("%s: pager latestVersion " PRId64 "\n", self->m_name.c_str(), latestVersion); + debug_printf("%s: pager latestVersion %" PRId64 "\n", self->m_name.c_str(), latestVersion); if(REDWOOD_DEBUG) { self->printMutationBuffer(mutations); @@ -1733,9 +2106,9 @@ private: VersionedChildrenT newRoot = wait(commitSubtree(self, mutations, self->m_pager->getReadSnapshot(latestVersion), self->m_root, &dbBegin, &dbEnd, &dbBegin, &dbEnd)); self->m_pager->setLatestVersion(writeVersion); - debug_printf("%s: Committing pager " PRId64 "\n", self->m_name.c_str(), writeVersion); + debug_printf("%s: Committing pager %" PRId64 "\n", self->m_name.c_str(), writeVersion); wait(self->m_pager->commit()); - debug_printf("%s: Committed version " PRId64 "\n", self->m_name.c_str(), writeVersion); + debug_printf("%s: Committed version %" PRId64 "\n", self->m_name.c_str(), writeVersion); // Now that everything is committed we must delete the mutation buffer. // Our buffer's start version should be the oldest mutation buffer version in the map. @@ -1744,6 +2117,7 @@ private: self->m_lastCommittedVersion = writeVersion; ++self->counts.commits; +printf("\nCommitted: %s\n", self->counts.toString(true).c_str()); committed.send(Void()); return Void(); @@ -2085,7 +2459,7 @@ private: std::string toString() const { std::string r; - r += format("Cursor(%p) ver: " PRId64 " ", this, m_version); + r += format("Cursor(%p) ver: %" PRId64 " ", this, m_version); if(m_kv.present()) { r += format(" KV: '%s' -> '%s'\n", m_kv.get().key.printable().c_str(), m_kv.get().value.printable().c_str()); } @@ -2492,16 +2866,16 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version state std::map, Optional>::const_iterator iLast; state Reference cur = btree->readAtVersion(v); - debug_printf("VerifyRange(@" PRId64 ", %s, %s): Start cur=%p\n", v, start.toString().c_str(), end.toString().c_str(), cur.getPtr()); + debug_printf("VerifyRange(@%" PRId64 ", %s, %s): Start cur=%p\n", v, start.toString().c_str(), end.toString().c_str(), cur.getPtr()); // Randomly use the cursor for something else first. if(g_random->coinflip()) { state Key randomKey = randomKV().key; - debug_printf("VerifyRange(@" PRId64 ", %s, %s): Dummy seek to '%s'\n", v, start.toString().c_str(), end.toString().c_str(), randomKey.toString().c_str()); + debug_printf("VerifyRange(@%" PRId64 ", %s, %s): Dummy seek to '%s'\n", v, start.toString().c_str(), end.toString().c_str(), randomKey.toString().c_str()); wait(g_random->coinflip() ? cur->findFirstEqualOrGreater(randomKey, true, 0) : cur->findLastLessOrEqual(randomKey, true, 0)); } - debug_printf("VerifyRange(@" PRId64 ", %s, %s): Actual seek\n", v, start.toString().c_str(), end.toString().c_str()); + debug_printf("VerifyRange(@%" PRId64 ", %s, %s): Actual seek\n", v, start.toString().c_str(), end.toString().c_str()); wait(cur->findFirstEqualOrGreater(start, true, 0)); state std::vector results; @@ -2522,7 +2896,7 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version || i->first.second > v ) ) { - debug_printf("VerifyRange(@" PRId64 ", %s, %s) Found key in written map: %s\n", v, start.toString().c_str(), end.toString().c_str(), iLast->first.first.c_str()); + debug_printf("VerifyRange(@%" PRId64 ", %s, %s) Found key in written map: %s\n", v, start.toString().c_str(), end.toString().c_str(), iLast->first.first.c_str()); break; } } @@ -2530,20 +2904,20 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version if(iLast == iEnd) { ++errors; ++*pErrorCount; - printf("VerifyRange(@" PRId64 ", %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str()); + printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str()); break; } if(cur->getKey() != iLast->first.first) { ++errors; ++*pErrorCount; - printf("VerifyRange(@" PRId64 ", %s, %s) ERROR: Tree key '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), iLast->first.first.c_str()); + printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), iLast->first.first.c_str()); break; } if(cur->getValue() != iLast->second.get()) { ++errors; ++*pErrorCount; - printf("VerifyRange(@" PRId64 ", %s, %s) ERROR: Tree key '%s' has tree value '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), cur->getValue().toString().c_str(), iLast->second.get().c_str()); + printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' has tree value '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), cur->getValue().toString().c_str(), iLast->second.get().c_str()); break; } @@ -2573,10 +2947,10 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version if(iLast != iEnd) { ++errors; ++*pErrorCount; - printf("VerifyRange(@" PRId64 ", %s, %s) ERROR: Tree range ended but written has @" PRId64 " '%s'\n", v, start.toString().c_str(), end.toString().c_str(), iLast->first.second, iLast->first.first.c_str()); + printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree range ended but written has @%" PRId64 " '%s'\n", v, start.toString().c_str(), end.toString().c_str(), iLast->first.second, iLast->first.first.c_str()); } - debug_printf("VerifyRangeReverse(@" PRId64 ", %s, %s): start\n", v, start.toString().c_str(), end.toString().c_str()); + debug_printf("VerifyRangeReverse(@%" PRId64 ", %s, %s): start\n", v, start.toString().c_str(), end.toString().c_str()); // Randomly use a new cursor for the reverse range read but only if version history is available if(!btree->isSingleVersion() && g_random->coinflip()) { @@ -2594,20 +2968,20 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version if(r == results.rend()) { ++errors; ++*pErrorCount; - printf("VerifyRangeReverse(@" PRId64 ", %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str()); + printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str()); break; } if(cur->getKey() != r->key) { ++errors; ++*pErrorCount; - printf("VerifyRangeReverse(@" PRId64 ", %s, %s) ERROR: Tree key '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), r->key.toString().c_str()); + printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), r->key.toString().c_str()); break; } if(cur->getValue() != r->value) { ++errors; ++*pErrorCount; - printf("VerifyRangeReverse(@" PRId64 ", %s, %s) ERROR: Tree key '%s' has tree value '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), cur->getValue().toString().c_str(), r->value.toString().c_str()); + printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' has tree value '%s' vs written '%s'\n", v, start.toString().c_str(), end.toString().c_str(), cur->getKey().toString().c_str(), cur->getValue().toString().c_str(), r->value.toString().c_str()); break; } @@ -2618,7 +2992,7 @@ ACTOR Future verifyRange(VersionedBTree *btree, Key start, Key end, Version if(r != results.rend()) { ++errors; ++*pErrorCount; - printf("VerifyRangeReverse(@" PRId64 ", %s, %s) ERROR: Tree range ended but written has '%s'\n", v, start.toString().c_str(), end.toString().c_str(), r->key.toString().c_str()); + printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree range ended but written has '%s'\n", v, start.toString().c_str(), end.toString().c_str(), r->key.toString().c_str()); } return errors; @@ -2638,7 +3012,7 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, state Reference cur = btree->readAtVersion(ver); - debug_printf("Verifying @" PRId64 " '%s'\n", ver, key.c_str()); + debug_printf("Verifying @%" PRId64 " '%s'\n", ver, key.c_str()); state Arena arena; wait(cur->findEqual(KeyRef(arena, key))); @@ -2657,7 +3031,7 @@ ACTOR Future verifyAll(VersionedBTree *btree, Version maxCommittedVersion, if(cur->isValid() && cur->getKey() == key) { ++errors; ++*pErrorCount; - printf("Verify ERROR: cleared_key_found: '%s' -> '%s' @" PRId64 "\n", key.c_str(), cur->getValue().toString().c_str(), ver); + printf("Verify ERROR: cleared_key_found: '%s' -> '%s' @%" PRId64 "\n", key.c_str(), cur->getValue().toString().c_str(), ver); } } } @@ -2676,7 +3050,7 @@ ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, if(btree->isSingleVersion()) { v = btree->getLastCommittedVersion(); - debug_printf("Verifying at latest committed version " PRId64 "\n", v); + debug_printf("Verifying at latest committed version %" PRId64 "\n", v); vall = verifyRange(btree, LiteralStringRef(""), LiteralStringRef("\xff\xff"), v, written, pErrorCount); if(serial) { wait(success(vall)); @@ -2687,7 +3061,7 @@ ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, } } else { - debug_printf("Verifying through version " PRId64 "\n", v); + debug_printf("Verifying through version %" PRId64 "\n", v); vall = verifyAll(btree, v, written, pErrorCount); if(serial) { wait(success(vall)); @@ -2701,7 +3075,7 @@ ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, int errors = vall.get() + vrange.get(); - debug_printf("Verified through version " PRId64 ", %d errors\n", v, errors); + debug_printf("Verified through version %" PRId64 ", %d errors\n", v, errors); if(*pErrorCount != 0) break; @@ -2737,61 +3111,6 @@ ACTOR Future randomReader(VersionedBTree *btree) { } } -struct SplitStringRef { - StringRef a; - StringRef b; - - SplitStringRef(StringRef a = StringRef(), StringRef b = StringRef()) : a(a), b(b) { - } - - SplitStringRef getSplitPrefix(int len) const { - if(len <= a.size()) { - return SplitStringRef(a.substr(0, len)); - } - len -= a.size(); - ASSERT(b.size() <= len); - return SplitStringRef(a, b.substr(0, len)); - } - - StringRef getContiguousPrefix(int len, Arena &arena) const { - if(len <= a.size()) { - return a.substr(0, len); - } - StringRef c = makeString(len, arena); - memcpy(mutateString(c), a.begin(), a.size()); - len -= a.size(); - memcpy(mutateString(c) + a.size(), b.begin(), len); - return c; - } - - int compare(const SplitStringRef &rhs) const { - // TODO: Rewrite this.. - Arena a; - StringRef self = getContiguousPrefix(size(), a); - StringRef other = rhs.getContiguousPrefix(rhs.size(), a); - return self.compare(other); - } - - int compare(const StringRef &rhs) const { - // TODO: Rewrite this.. - Arena a; - StringRef self = getContiguousPrefix(size(), a); - return self.compare(rhs); - } - - int size() const { - return a.size() + b.size(); - } - - std::string toString() const { - return format("%s%s", a.toString().c_str(), b.toString().c_str()); - } - - std::string toHexString() const { - return format("%s%s", a.toHexString().c_str(), b.toHexString().c_str()); - } -}; - struct IntIntPair { IntIntPair() {} IntIntPair(int k, int v) : k(k), v(v) {} @@ -2801,11 +3120,20 @@ struct IntIntPair { } struct Delta { + bool prefixSource; int dk; int dv; - IntIntPair apply(const IntIntPair &prev, const IntIntPair &next, Arena arena) { - return {prev.k + dk, prev.v + dv}; + IntIntPair apply(const IntIntPair &base, Arena &arena) { + return {base.k + dk, base.v + dv}; + } + + void setPrefixSource(bool val) { + prefixSource = val; + } + + bool getPrefixSource() const { + return prefixSource; } int size() const { @@ -2813,7 +3141,7 @@ struct IntIntPair { } std::string toString() const { - return format("DELTA{dk=%d(0x%x) dv=%d(0x%x)}", dk, dk, dv, dv); + return format("DELTA{prefixSource=%d dk=%d(0x%x) dv=%d(0x%x)}", prefixSource, dk, dk, dv, dv); } }; @@ -2826,14 +3154,18 @@ struct IntIntPair { return k == rhs.k; } + int getCommonPrefixLen(const IntIntPair &other, int skip) const { + return 0; + } + int deltaSize(const IntIntPair &base) const { return sizeof(Delta); } - void writeDelta(Delta &d, const IntIntPair &prev, const IntIntPair &next) const { - // Always borrow from prev - d.dk = k - prev.k; - d.dv = v - prev.v; + int writeDelta(Delta &d, const IntIntPair &base, int commonPrefix = -1) const { + d.dk = k - base.k; + d.dv = v - base.v; + return sizeof(Delta); } int k; @@ -2844,7 +3176,200 @@ struct IntIntPair { } }; -TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessRedwoodRecord") { +int getCommonIntFieldPrefix2(const RedwoodRecordRef &a, const RedwoodRecordRef &b) { + RedwoodRecordRef::byte aFields[RedwoodRecordRef::intFieldArraySize]; + RedwoodRecordRef::byte bFields[RedwoodRecordRef::intFieldArraySize]; + + a.serializeIntFields(aFields); + b.serializeIntFields(bFields); + + //printf("a: %s\n", StringRef(aFields, RedwoodRecordRef::intFieldArraySize).toHexString().c_str()); + //printf("b: %s\n", StringRef(bFields, RedwoodRecordRef::intFieldArraySize).toHexString().c_str()); + + int i = 0; + while(i < RedwoodRecordRef::intFieldArraySize && aFields[i] == bFields[i]) { + ++i; + } + + //printf("%d\n", i); + return i; +} + +void deltaTest(RedwoodRecordRef rec, RedwoodRecordRef base) { + char buf[500]; + RedwoodRecordRef::Delta &d = *(RedwoodRecordRef::Delta *)buf; + + Arena mem; + rec.writeDelta(d, base); + RedwoodRecordRef decoded = d.apply(base, mem); + + if(decoded != rec) { + printf("RedwoodRecordRef::Delta test failure!\n"); + printf("BASE: %s\n", base.toString().c_str()); + printf("DELTA: %s\n", d.toString().c_str()); + printf("REC: %s\n", rec.toString().c_str()); + printf("DECODED: %s\n", decoded.toString().c_str()); + printf("\n"); + ASSERT(false); + } +} + +TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { + // Testing common prefix calculation for integer fields using the member function that calculates this directly + // and by serializing the integer fields to arrays and finding the common prefix length of the two arrays + + deltaTest(RedwoodRecordRef(LiteralStringRef(""), 0, LiteralStringRef(""), 0, 0), + RedwoodRecordRef(LiteralStringRef(""), 0, LiteralStringRef(""), 0, 0) + ); + + deltaTest(RedwoodRecordRef(LiteralStringRef("abc"), 0, LiteralStringRef(""), 0, 0), + RedwoodRecordRef(LiteralStringRef("abc"), 0, LiteralStringRef(""), 0, 0) + ); + + deltaTest(RedwoodRecordRef(LiteralStringRef("abc"), 0, LiteralStringRef(""), 0, 0), + RedwoodRecordRef(LiteralStringRef("abcd"), 0, LiteralStringRef(""), 0, 0) + ); + + deltaTest(RedwoodRecordRef(LiteralStringRef("abc"), 2, LiteralStringRef(""), 0, 0), + RedwoodRecordRef(LiteralStringRef("abc"), 2, LiteralStringRef(""), 0, 0) + ); + + deltaTest(RedwoodRecordRef(LiteralStringRef("abc"), 2, LiteralStringRef(""), 0, 0), + RedwoodRecordRef(LiteralStringRef("ab"), 2, LiteralStringRef(""), 1, 3) + ); + + deltaTest(RedwoodRecordRef(LiteralStringRef("abc"), 2, LiteralStringRef(""), 5, 0), + RedwoodRecordRef(LiteralStringRef("abc"), 2, LiteralStringRef(""), 5, 1) + ); + + RedwoodRecordRef rec1; + RedwoodRecordRef rec2; + + rec1.version = 0x12345678; + rec2.version = 0x12995678; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 5); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + rec1.version = 0x12345678; + rec2.version = 0x12345678; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 14); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + rec1.version = invalidVersion; + rec2.version = 0; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 0); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + rec1.version = 0x12345678; + rec2.version = 0x12345678; + rec1.chunk.total = 4; + rec2.chunk.total = 4; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 14); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + rec1.version = 0x12345678; + rec2.version = 0x12345678; + rec1.chunk.start = 4; + rec2.chunk.start = 4; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 14); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + rec1.version = 0x12345678; + rec2.version = 0x12345678; + rec1.chunk.start = 4; + rec2.chunk.start = 5; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 13); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + rec1.version = 0x12345678; + rec2.version = 0x12345678; + rec1.chunk.total = 256; + rec2.chunk.total = 512; + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == 9); + ASSERT(rec1.getCommonIntFieldPrefix(rec2) == getCommonIntFieldPrefix2(rec1, rec2)); + + Arena mem; + double start; + uint64_t total; + uint64_t count; + uint64_t i; + + start = timer(); + total = 0; + count = 1000000000; + for(i = 0; i < count; ++i) { + rec1.chunk.total = i & 0xffffff; + rec2.chunk.total = i & 0xffffff; + rec1.chunk.start = i & 0xffffff; + rec2.chunk.start = (i + 1) & 0xffffff; + total += rec1.getCommonIntFieldPrefix(rec2); + } + printf("%" PRId64 " getCommonIntFieldPrefix() %g M/s\n", total, count / (timer() - start) / 1e6); + + rec1.key = LiteralStringRef("alksdfjaklsdfjlkasdjflkasdjfklajsdflk;ajsdflkajdsflkjadsf"); + rec2.key = LiteralStringRef("alksdfjaklsdfjlkasdjflkasdjfklajsdflk;ajsdflkajdsflkjadsf"); + + start = timer(); + total = 0; + count = 1000000000; + for(i = 0; i < count; ++i) { + RedwoodRecordRef::byte fields[RedwoodRecordRef::intFieldArraySize]; + rec1.chunk.start = i & 0xffffff; + rec2.chunk.start = (i + 1) & 0xffffff; + rec1.serializeIntFields(fields); + total += fields[RedwoodRecordRef::intFieldArraySize - 1]; + } + printf("%" PRId64 " serializeIntFields() %g M/s\n", total, count / (timer() - start) / 1e6); + + start = timer(); + total = 0; + count = 1000000000; + for(i = 0; i < count; ++i) { + rec1.chunk.start = i & 0xffffff; + rec2.chunk.start = (i + 1) & 0xffffff; + total += rec1.getCommonPrefixLen(rec2, 50); + } + printf("%" PRId64 " getCommonPrefixLen(skip=50) %g M/s\n", total, count / (timer() - start) / 1e6); + + start = timer(); + total = 0; + count = 1000000000; + for(i = 0; i < count; ++i) { + rec1.chunk.start = i & 0xffffff; + rec2.chunk.start = (i + 1) & 0xffffff; + total += rec1.getCommonPrefixLen(rec2, 0); + } + printf("%" PRId64 " getCommonPrefixLen(skip=0) %g M/s\n", total, count / (timer() - start) / 1e6); + + char buf[1000]; + RedwoodRecordRef::Delta &d = *(RedwoodRecordRef::Delta *)buf; + + start = timer(); + total = 0; + count = 100000000; + int commonPrefix = rec1.getCommonPrefixLen(rec2, 0); + + for(i = 0; i < count; ++i) { + rec1.chunk.start = i & 0xffffff; + rec2.chunk.start = (i + 1) & 0xffffff; + total += rec1.writeDelta(d, rec2, commonPrefix); + } + printf("%" PRId64 " writeDelta(commonPrefix=%d) %g M/s\n", total, commonPrefix, count / (timer() - start) / 1e6); + + start = timer(); + total = 0; + count = 100000000; + for(i = 0; i < count; ++i) { + rec1.chunk.start = i & 0xffffff; + rec2.chunk.start = (i + 1) & 0xffffff; + total += rec1.writeDelta(d, rec2); + } + printf("%" PRId64 " writeDelta() %g M/s\n", total, count / (timer() - start) / 1e6); + + return Void(); +} + +TEST_CASE("!/redwood/correctness/unit/deltaTree/RedwoodRecordRef") { const int N = 200; RedwoodRecordRef prev; @@ -2861,8 +3386,8 @@ TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessRedwoodRecord") { if(g_random->coinflip()) { rec.value = StringRef(arena, v); if(g_random->coinflip()) { - rec.chunk.start = g_random->randomInt(0, 5000); - rec.chunk.total = rec.chunk.start + v.size() + g_random->randomInt(0, 5000); + rec.chunk.start = g_random->randomInt(0, 100000); + rec.chunk.total = rec.chunk.start + v.size() + g_random->randomInt(0, 100000); } } items.push_back(rec); @@ -2924,7 +3449,7 @@ TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessRedwoodRecord") { return Void(); } -TEST_CASE("!/redwood/mutableDeltaTreeCorrectnessIntInt") { +TEST_CASE("!/redwood/correctness/unit/deltaTree/IntIntPair") { const int N = 200; IntIntPair prev = {0, 0}; IntIntPair next = {1000, 0}; @@ -3004,10 +3529,10 @@ struct SimpleCounter { double t; double start; int64_t xt; - std::string toString() { return format("" PRId64 "/%.2f/%.2f", x, rate() / 1e6, avgRate() / 1e6); } + std::string toString() { return format("%" PRId64 "/%.2f/%.2f", x, rate() / 1e6, avgRate() / 1e6); } }; -TEST_CASE("!/redwood/correctness") { +TEST_CASE("!/redwood/correctness/btree") { state bool useDisk = true; // MemoryPager is not being maintained currently. state std::string pagerFile = "unittest_pageFile"; @@ -3107,7 +3632,7 @@ TEST_CASE("!/redwood/correctness") { ++rangeClears; KeyRangeRef range(start, end); - debug_printf(" Mutation: Clear '%s' to '%s' @" PRId64 "\n", start.toString().c_str(), end.toString().c_str(), version); + debug_printf(" Mutation: Clear '%s' to '%s' @%" PRId64 "\n", start.toString().c_str(), end.toString().c_str(), version); auto e = written.lower_bound(std::make_pair(start.toString(), 0)); if(e != written.end()) { auto last = e; @@ -3117,7 +3642,7 @@ TEST_CASE("!/redwood/correctness") { ++e; // If e key is different from last and last was present then insert clear for last's key at version if(last != eEnd && ((e == eEnd || e->first.first != last->first.first) && last->second.present())) { - debug_printf(" Mutation: Clearing key '%s' @" PRId64 "\n", last->first.first.c_str(), version); + debug_printf(" Mutation: Clearing key '%s' @%" PRId64 "\n", last->first.first.c_str(), version); keyBytesCleared += last->first.first.size(); mutationBytes += last->first.first.size(); @@ -3147,7 +3672,7 @@ TEST_CASE("!/redwood/correctness") { kv.key = StringRef(kv.arena(), *i); } - debug_printf(" Mutation: Set '%s' -> '%s' @" PRId64 "\n", kv.key.toString().c_str(), kv.value.toString().c_str(), version); + debug_printf(" Mutation: Set '%s' -> '%s' @%" PRId64 "\n", kv.key.toString().c_str(), kv.value.toString().c_str(), version); ++sets; keyBytesInserted += kv.key.size(); @@ -3164,7 +3689,7 @@ TEST_CASE("!/redwood/correctness") { if(mutationBytes.get() >= mutationBytesTarget || mutationBytesThisCommit >= mutationBytesTargetThisCommit) { // Wait for previous commit to finish wait(commit); - printf("Committed. Next commit %d bytes, " PRId64 "/%d (%.2f%%) Stats: Insert %.2f MB/s ClearedKeys %.2f MB/s Total %.2f\n", + printf("Committed. Next commit %d bytes, %" PRId64 "/%d (%.2f%%) Stats: Insert %.2f MB/s ClearedKeys %.2f MB/s Total %.2f\n", mutationBytesThisCommit, mutationBytes.get(), mutationBytesTarget, diff --git a/flow/Platform.h b/flow/Platform.h index 291583d2cb..49d2ac39af 100644 --- a/flow/Platform.h +++ b/flow/Platform.h @@ -558,8 +558,32 @@ inline static int ctzll( uint64_t value ) { } return 64; } +inline static int clzll( uint64_t value ) { + unsigned long count = 0; + if( _BitScanReverse64( &count, value ) ) { + return 63 - count; + } + return 64; +} +inline static int ctz( uint32_t value ) { + unsigned long count = 0; + if( _BitScanForward( &count, value ) ) { + return count; + } + return 64; +} +inline static int clz( uint32_t value ) { + unsigned long count = 0; + if( _BitScanReverse( &count, value ) ) { + return 63 - count; + } + return 64; +} #else #define ctzll __builtin_ctzll +#define clzll __builtin_clzll +#define ctz __builtin_ctz +#define clz __builtin_clz #endif #include From 1e5b9faa11cef7e37851354f1eb3cfc1469b34d9 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 29 May 2019 16:26:58 -0700 Subject: [PATCH 014/128] Bug fixes in RedwoodRecordRef::Delta. --- fdbserver/VersionedBTree.actor.cpp | 58 +++++++++++++++++++++++------- 1 file changed, 46 insertions(+), 12 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 08dbf5caad..6671b08d5b 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -407,7 +407,7 @@ struct RedwoodRecordRef { } // Version offset is used to skip the version bytes in the int field array when version is missing (aka 0) - int versionOffset = flags & HAS_VERSION ? 0 : 8; + int versionOffset = ( (intFieldPrefixLen == 0) && (~flags & HAS_VERSION) ) ? 8 : 0; // If there are suffix bytes, copy those into place after the prefix if(intFieldSuffixLen > 0) { @@ -439,13 +439,19 @@ struct RedwoodRecordRef { std::string toString() const { Reader r(data); + std::string flagString; + if(flags & PREFIX_SOURCE) flagString += "prefixSource "; + if(flags & HAS_KEY_SUFFIX) flagString += "keySuffix "; + if(flags & HAS_VERSION) flagString += "Version "; + if(flags & HAS_VALUE) flagString += "Value "; + int intFieldSuffixLen = flags & INT_FIELD_SUFFIX_BITS; int prefixLen = r.readVarInt(); int valueLen = (flags & HAS_VALUE) ? r.read() : 0; int keySuffixLen = (flags & HAS_KEY_SUFFIX) ? r.readVarInt() : 0; - return format("flags: %02x prefixLen: %d keySuffixLen: %d intFieldSuffix: %d valueLen %d raw: %s", - flags, prefixLen, keySuffixLen, intFieldSuffixLen, valueLen, StringRef((const uint8_t *)this, size()).toHexString().c_str()); + return format("flags: %s prefixLen: %d keySuffixLen: %d intFieldSuffix: %d valueLen %d raw: %s", + flagString.c_str(), prefixLen, keySuffixLen, intFieldSuffixLen, valueLen, StringRef((const uint8_t *)this, size()).toHexString().c_str()); } }; #pragma pack(pop) @@ -579,7 +585,7 @@ struct RedwoodRecordRef { // Start copying after any prefix bytes that matched the int fields of the base int intFieldPrefixLen = std::max(0, commonPrefix - key.size()); - int startPos = intFieldPrefixLen + (version == 0 ? 8 : 0); + int startPos = intFieldPrefixLen + (intFieldPrefixLen == 0 && version == 0 ? 8 : 0); int suffixLen = std::max(0, endPos - startPos); if(suffixLen > 0) { @@ -3204,16 +3210,34 @@ void deltaTest(RedwoodRecordRef rec, RedwoodRecordRef base) { RedwoodRecordRef decoded = d.apply(base, mem); if(decoded != rec) { - printf("RedwoodRecordRef::Delta test failure!\n"); printf("BASE: %s\n", base.toString().c_str()); printf("DELTA: %s\n", d.toString().c_str()); printf("REC: %s\n", rec.toString().c_str()); printf("DECODED: %s\n", decoded.toString().c_str()); - printf("\n"); + printf("RedwoodRecordRef::Delta test failure!\n"); ASSERT(false); } } +Standalone randomRedwoodRecordRef(int maxKeySize = 3, int maxValueSize = 255) { + RedwoodRecordRef rec; + KeyValue kv = randomKV(3, 10); + rec.key = kv.key; + + if(g_random->random01() < .9) { + rec.value = kv.value; + } + + rec.version = g_random->coinflip() ? 0 : g_random->randomInt64(0, std::numeric_limits::max()); + + if(g_random->coinflip()) { + rec.chunk.total = g_random->randomInt(1, 100000); + rec.chunk.start = g_random->randomInt(0, rec.chunk.total); + } + + return Standalone(rec, kv.arena()); +} + TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { // Testing common prefix calculation for integer fields using the member function that calculates this directly // and by serializing the integer fields to arrays and finding the common prefix length of the two arrays @@ -3296,7 +3320,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 1000000000; + count = 1e9; for(i = 0; i < count; ++i) { rec1.chunk.total = i & 0xffffff; rec2.chunk.total = i & 0xffffff; @@ -3311,7 +3335,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 1000000000; + count = 1e9; for(i = 0; i < count; ++i) { RedwoodRecordRef::byte fields[RedwoodRecordRef::intFieldArraySize]; rec1.chunk.start = i & 0xffffff; @@ -3323,7 +3347,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 1000000000; + count = 100e6; for(i = 0; i < count; ++i) { rec1.chunk.start = i & 0xffffff; rec2.chunk.start = (i + 1) & 0xffffff; @@ -3333,7 +3357,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 1000000000; + count = 100e6; for(i = 0; i < count; ++i) { rec1.chunk.start = i & 0xffffff; rec2.chunk.start = (i + 1) & 0xffffff; @@ -3346,7 +3370,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 100000000; + count = 100e6; int commonPrefix = rec1.getCommonPrefixLen(rec2, 0); for(i = 0; i < count; ++i) { @@ -3358,7 +3382,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 100000000; + count = 10e6; for(i = 0; i < count; ++i) { rec1.chunk.start = i & 0xffffff; rec2.chunk.start = (i + 1) & 0xffffff; @@ -3366,6 +3390,16 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { } printf("%" PRId64 " writeDelta() %g M/s\n", total, count / (timer() - start) / 1e6); + start = timer(); + total = 0; + count = 5e6; + for(i = 0; i < count; ++i) { + Standalone a = randomRedwoodRecordRef(); + Standalone b = randomRedwoodRecordRef(); + deltaTest(a, b); + } + printf("Random deltaTest() %g M/s\n", count / (timer() - start) / 1e6); + return Void(); } From cedcfcddd00413873d4511fb7bae6562d3daeca9 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 29 May 2019 16:47:53 -0700 Subject: [PATCH 015/128] Bug fix in RedwoodRecordRef::Delta var int writer, new tests. --- fdbserver/VersionedBTree.actor.cpp | 29 +++++++++++++++++++++-------- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 6671b08d5b..4a00f39409 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -262,7 +262,7 @@ struct RedwoodRecordRef { // Values > 15 bits in length are not valid input but this is not checked for. void writeVarInt(int x) { if(x >= 128) { - *wptr++ = (uint8_t)( (x >> 8) & 0x7f ); + *wptr++ = (uint8_t)( (x >> 8) | 0x80 ); } *wptr++ = (uint8_t)x; } @@ -3206,14 +3206,17 @@ void deltaTest(RedwoodRecordRef rec, RedwoodRecordRef base) { RedwoodRecordRef::Delta &d = *(RedwoodRecordRef::Delta *)buf; Arena mem; - rec.writeDelta(d, base); + int expectedSize = rec.deltaSize(base); + int deltaSize = rec.writeDelta(d, base); RedwoodRecordRef decoded = d.apply(base, mem); - if(decoded != rec) { - printf("BASE: %s\n", base.toString().c_str()); - printf("DELTA: %s\n", d.toString().c_str()); - printf("REC: %s\n", rec.toString().c_str()); - printf("DECODED: %s\n", decoded.toString().c_str()); + if(decoded != rec || expectedSize < deltaSize) { + printf("Base: %s\n", base.toString().c_str()); + printf("ExpectedSize: %d\n", expectedSize); + printf("DeltaSize: %d\n", deltaSize); + printf("Delta: %s\n", d.toString().c_str()); + printf("Record: %s\n", rec.toString().c_str()); + printf("Decoded: %s\n", decoded.toString().c_str()); printf("RedwoodRecordRef::Delta test failure!\n"); ASSERT(false); } @@ -3266,6 +3269,16 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { RedwoodRecordRef(LiteralStringRef("abc"), 2, LiteralStringRef(""), 5, 1) ); + RedwoodRecordRef::byte varInts[100]; + RedwoodRecordRef::Writer w(varInts); + RedwoodRecordRef::Reader r(varInts); + w.writeVarInt(1); + w.writeVarInt(128); + w.writeVarInt(32000); + ASSERT(r.readVarInt() == 1); + ASSERT(r.readVarInt() == 128); + ASSERT(r.readVarInt() == 32000); + RedwoodRecordRef rec1; RedwoodRecordRef rec2; @@ -3392,7 +3405,7 @@ TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { start = timer(); total = 0; - count = 5e6; + count = 1e6; for(i = 0; i < count; ++i) { Standalone a = randomRedwoodRecordRef(); Standalone b = randomRedwoodRecordRef(); From 3e155a2563bd9d565497bb3a7b5e71faee6f0bc1 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 29 May 2019 17:38:55 -0700 Subject: [PATCH 016/128] Bug fixes. --- fdbserver/DeltaTree.h | 8 ++++---- fdbserver/VersionedBTree.actor.cpp | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/fdbserver/DeltaTree.h b/fdbserver/DeltaTree.h index 6ab0da62f1..4a9bee5c98 100644 --- a/fdbserver/DeltaTree.h +++ b/fdbserver/DeltaTree.h @@ -353,13 +353,13 @@ private: // Since mid is between them, we can skip that length to determine the common prefix length // between mid and prev and between mid and next. int nextPrevCommon = prev->getCommonPrefixLen(*next, 0); - int commonWithPrev = begin[mid].getCommonPrefixLen(*prev, nextPrevCommon); - int commonWithNext = begin[mid].getCommonPrefixLen(*next, nextPrevCommon); + int commonWithPrev = item.getCommonPrefixLen(*prev, nextPrevCommon); + int commonWithNext = item.getCommonPrefixLen(*next, nextPrevCommon); bool prefixSourcePrev; int commonPrefix; const T *base; - if(commonWithNext > commonWithPrev) { + if(commonWithPrev >= commonWithNext) { prefixSourcePrev = true; commonPrefix = commonWithPrev; base = prev; @@ -369,9 +369,9 @@ private: commonPrefix = commonWithNext; base = next; } - root.delta->setPrefixSource(prefixSourcePrev); int deltaSize = item.writeDelta(*root.delta, *base, commonPrefix); + root.delta->setPrefixSource(prefixSourcePrev); //printf("Serialized %s to %p\n", item.toString().c_str(), root.delta); // Continue writing after the serialized Delta. diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 4a00f39409..694840ecc5 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -506,14 +506,14 @@ struct RedwoodRecordRef { } int prefixLen = getCommonPrefixLen(base, 0); - size += (prefixLen > 128) ? 2 : 1; + size += (prefixLen >= 128) ? 2 : 1; int intFieldPrefixLen; // Currently using a worst-guess guess where int fields in suffix are stored in their entirety if nonzero. if(prefixLen < key.size()) { int keySuffixLen = key.size() - prefixLen; - size += (keySuffixLen > 128) ? 2 : 1; + size += (keySuffixLen >= 128) ? 2 : 1; size += keySuffixLen; intFieldPrefixLen = 0; } From 098ac46af9ac8a98e00ed617e61922145296932b Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 29 May 2019 18:06:11 -0700 Subject: [PATCH 017/128] RedwoodRecordRef::deltaSize() now calculates actual delta size instead of a conservative estimate. --- fdbserver/VersionedBTree.actor.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 694840ecc5..46a29c4659 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -535,6 +535,9 @@ struct RedwoodRecordRef { } size += std::max(0, intFieldArraySize - intFieldPrefixLen - trailingNulls); + if(intFieldPrefixLen == 0 && version == 0) { + size -= 8; + } } return size; @@ -3210,7 +3213,8 @@ void deltaTest(RedwoodRecordRef rec, RedwoodRecordRef base) { int deltaSize = rec.writeDelta(d, base); RedwoodRecordRef decoded = d.apply(base, mem); - if(decoded != rec || expectedSize < deltaSize) { + if(decoded != rec || expectedSize != deltaSize) { + printf("\n"); printf("Base: %s\n", base.toString().c_str()); printf("ExpectedSize: %d\n", expectedSize); printf("DeltaSize: %d\n", deltaSize); From 9f064ad7cf839acd0930827a5c8de57e54a583f9 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Thu, 30 May 2019 02:10:07 -0700 Subject: [PATCH 018/128] Added back minimal btree internal page boundaries using RedwoodRecordRef. --- fdbserver/VersionedBTree.actor.cpp | 48 ++++++++++++++++++------------ 1 file changed, 29 insertions(+), 19 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 46a29c4659..af41e20382 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -193,6 +193,24 @@ struct RedwoodRecordRef { return 14; } + // Truncate (key, version, chunk.total, chunk.start) tuple to len bytes. + void truncate(int len) { + if(len <= key.size()) { + key = key.substr(0, len); + version = 0; + chunk.total = 0; + chunk.start = 0; + } + else { + byte fields[intFieldArraySize]; + serializeIntFields(fields); + int end = len - key.size(); + for(int i = intFieldArraySize - 1; i >= end; --i) { + fields[i] = 0; + } + } + } + // Find the common prefix between two records, assuming that the first // skip bytes are the same. inline int getCommonPrefixLen(const RedwoodRecordRef &other, int skip) const { @@ -439,7 +457,7 @@ struct RedwoodRecordRef { std::string toString() const { Reader r(data); - std::string flagString; + std::string flagString = " "; if(flags & PREFIX_SOURCE) flagString += "prefixSource "; if(flags & HAS_KEY_SUFFIX) flagString += "keySuffix "; if(flags & HAS_VERSION) flagString += "Version "; @@ -450,8 +468,8 @@ struct RedwoodRecordRef { int valueLen = (flags & HAS_VALUE) ? r.read() : 0; int keySuffixLen = (flags & HAS_KEY_SUFFIX) ? r.readVarInt() : 0; - return format("flags: %s prefixLen: %d keySuffixLen: %d intFieldSuffix: %d valueLen %d raw: %s", - flagString.c_str(), prefixLen, keySuffixLen, intFieldSuffixLen, valueLen, StringRef((const uint8_t *)this, size()).toHexString().c_str()); + return format("len: %d flags: %s prefixLen: %d keySuffixLen: %d intFieldSuffix: %d valueLen %d raw: %s", + size(), flagString.c_str(), prefixLen, keySuffixLen, intFieldSuffixLen, valueLen, StringRef((const uint8_t *)this, size()).toHexString().c_str()); } }; #pragma pack(pop) @@ -741,9 +759,6 @@ struct BoundaryAndPage { // TODO: Refactor this as an accumulator you add sorted keys to which makes pages. template static std::vector buildPages(bool minimalBoundaries, const RedwoodRecordRef &lowerBound, const RedwoodRecordRef &upperBound, std::vector entries, uint8_t newFlags, Allocator const &newBlockFn, int usableBlockSize) { - // TODO: Figure out how to do minimal boundaries with RedwoodRecordRef - minimalBoundaries = false; - // This is how much space for the binary tree exists in the page, after the header int pageSize = usableBlockSize - BTreePage::GetHeaderSize(); @@ -807,18 +822,7 @@ static std::vector buildPages(bool minimalBoundaries, const Red } } if(!fits) { - // Flush page - if(minimalBoundaries) { - // TODO: Write minimal boundaries - // Note that prefixLen is guaranteed to be < entry.key.size() because entries are in increasing order and cannot repeat. -// int len = prefixLen + 1; -// if(entry.key[prefixLen] == 0) -// len = std::min(len + 1, entry.key.size()); -// pageUpperBound = entry.key.substr(0, len); - } - else { - pageUpperBound = entry.withoutValue(); - } + pageUpperBound = entry.withoutValue(); } } @@ -843,6 +847,12 @@ static std::vector buildPages(bool minimalBoundaries, const Red pageUpperBound = entries[i].withoutValue(); } + // If this isn't the final page, shorten the upper boundary + if(!end && minimalBoundaries) { + int commonPrefix = pageUpperBound.getCommonPrefixLen(entries[i - 1], 0); + pageUpperBound.truncate(commonPrefix + 1); + } + debug_printf("Flushing page start=%d i=%d count=%d\nlower: %s\nupper: %s\n", start, i, count, pageLowerBound.toString().c_str(), pageUpperBound.toString().c_str()); #if REDWOOD_DEBUG for(int j = start; j < i; ++j) { @@ -2660,7 +2670,7 @@ printf("\nCommitted: %s\n", self->counts.toString(true).c_str()); }; RedwoodRecordRef VersionedBTree::dbBegin(StringRef(), 0); -RedwoodRecordRef VersionedBTree::dbEnd(LiteralStringRef("\xff\xff\xff\xff"), std::numeric_limits::max()); +RedwoodRecordRef VersionedBTree::dbEnd(LiteralStringRef("\xff\xff\xff\xff\xff")); VersionedBTree::Counts VersionedBTree::counts; ACTOR template From 653440d54c27b44191d6775f86142f6b866723be Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Tue, 4 Jun 2019 04:03:52 -0700 Subject: [PATCH 019/128] Changes and bug fixes in how boundary keys are modified during clears in internal pages by rewriting how internal pages are modified, making edge cases much easier to handle. Several debug output improvements. Page numbers stored on disk are now big endian. --- fdbserver/VersionedBTree.actor.cpp | 640 +++++++++++++++-------------- 1 file changed, 340 insertions(+), 300 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index af41e20382..9ae706a212 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -156,9 +156,15 @@ struct RedwoodRecordRef { } RedwoodRecordRef(Arena &arena, const RedwoodRecordRef &toCopy) - : key(arena, toCopy.key), version(toCopy.version), chunk(toCopy.chunk) { - if(value.present()) { - value = ValueRef(arena, toCopy.value.get()); + : key(arena, toCopy.key), version(toCopy.version), chunk(toCopy.chunk) + { + if(toCopy.value.present()) { + if(toCopy.localValue()) { + setPageID(toCopy.getPageID()); + } + else { + value = ValueRef(arena, toCopy.value.get()); + } } } @@ -168,8 +174,57 @@ struct RedwoodRecordRef { deserializeIntFields(intFields); } - // TODO: This probably is not actually needed, it only helps make debug output more clear - RedwoodRecordRef withoutValue() const { + RedwoodRecordRef(const RedwoodRecordRef &toCopy) : key(toCopy.key), version(toCopy.version), chunk(toCopy.chunk) { + if(toCopy.value.present()) { + if(toCopy.localValue()) { + setPageID(toCopy.getPageID()); + } + else { + value = toCopy.value; + } + } + } + + RedwoodRecordRef & operator= (const RedwoodRecordRef &toCopy) { + key = toCopy.key; + version = toCopy.version; + chunk = toCopy.chunk; + if(toCopy.value.present()) { + if(toCopy.localValue()) { + setPageID(toCopy.getPageID()); + } + else { + value = toCopy.value; + } + } + + return *this; + } + + bool localValue() const { + return value.get().begin() == bigEndianPageIDSpace; + } + + // RedwoodRecordRefs are used for both internal and leaf pages of the BTree. + // Boundary records in internal pages are made from leaf records. + // These functions make creating and working with internal page records more convenient. + inline LogicalPageID getPageID() const { + ASSERT(value.present()); + return bigEndian32(*(LogicalPageID *)value.get().begin()); + } + + inline void setPageID(LogicalPageID id) { + *(LogicalPageID *)bigEndianPageIDSpace = bigEndian32(id); + value = ValueRef(bigEndianPageIDSpace, sizeof(bigEndianPageIDSpace)); + } + + inline RedwoodRecordRef withPageID(LogicalPageID id) const { + RedwoodRecordRef rec(key, version, {}, chunk.total, chunk.start); + rec.setPageID(id); + return rec; + } + + inline RedwoodRecordRef withoutValue() const { return RedwoodRecordRef(key, version, {}, chunk.total, chunk.start); } @@ -251,6 +306,9 @@ struct RedwoodRecordRef { uint32_t start; } chunk; + // If the value is a page ID it will be stored here + uint8_t bigEndianPageIDSpace[sizeof(LogicalPageID)]; + int expectedSize() const { return key.expectedSize() + value.expectedSize(); } @@ -474,6 +532,9 @@ struct RedwoodRecordRef { }; #pragma pack(pop) + // Compares and orders by key, version, chunk.start, chunk.total. + // Value is not considered, as it is does not make sense for a container + // to have two records which differ only in value. int compare(const RedwoodRecordRef &rhs) const { int cmp = key.compare(rhs.key); if(cmp == 0) { @@ -485,21 +546,26 @@ struct RedwoodRecordRef { cmp = chunk.start - rhs.chunk.start; if(cmp == 0) { cmp = chunk.total - rhs.chunk.total; - if(cmp == 0) { - // No-value sorts AFTER having a value. - cmp = (value.present() ? 0 : 1) - (rhs.value.present() ? 0 : 1); - } } } } return cmp; } + // Compares key fields and value for equality + bool identical(const RedwoodRecordRef &rhs) const { + return compare(rhs) == 0 && value == rhs.value; + } + bool operator==(const RedwoodRecordRef &rhs) const { return compare(rhs) == 0; } - bool operator<(const RedwoodRecordRef &rhs) const { + bool operator!=(const RedwoodRecordRef &rhs) const { + return compare(rhs) != 0; + } + + bool operator<(const RedwoodRecordRef &rhs) const { return compare(rhs) < 0; } @@ -638,13 +704,19 @@ struct RedwoodRecordRef { std::string toString(int hexLimit = 15) const { std::string r; - r += format("'%s' @%" PRId64 " ", kvformat(key, hexLimit).c_str(), version); - r += format("[%d/%d] ", chunk.start, chunk.total); + r += format("'%s'@%" PRId64, kvformat(key, hexLimit).c_str(), version); + r += format("[%d/%d]->", chunk.start, chunk.total); if(value.present()) { - r += format("-> '%s'", kvformat(value.get(), hexLimit).c_str()); + // Assume that values the size of a page ID are page IDs. It's not perfect but it's just for debugging. + if(value.get().size() == sizeof(LogicalPageID)) { + r += format("[Page id=%u]", getPageID()); + } + else { + r += format("'%s'", kvformat(value.get(), hexLimit).c_str()); + } } else { - r += "-> "; + r += "null"; } return r; } @@ -703,21 +775,7 @@ struct BTreePage { do { r += " "; - if(!(flags & IS_LEAF)) { - RedwoodRecordRef rec = c.get(); - std::string val; - if(rec.value.present()) { - ASSERT(rec.value.get().size() == sizeof(uint32_t)); - uint32_t id = *(const uint32_t *)rec.value.get().begin(); - val = format("[Page id=%u]", id); - rec.value = val; - } - r += rec.toString(); - } - else { - r += c.get().toString(); - } - + r += c.get().toString(); r += "\n"; ASSERT(c.get().key >= lowerBound->key && c.get().key <= upperBound->key); @@ -1176,25 +1234,116 @@ private: LogicalPageID m_root; - typedef std::pair, LogicalPageID> KeyPagePairT; - typedef std::pair> VersionedKeyToPageSetT; - typedef std::vector VersionedChildrenT; + // TODO: Don't use Standalone + struct VersionedChildPageSet { + Version version; + std::vector> children; + Standalone upperBound; + }; - static std::string toString(const KeyPagePairT &c) { - return format("(%s, Page %u)", c.first.toString(-1).c_str(), c.second); - } + typedef std::vector VersionedChildrenT; - static std::string toString(const VersionedKeyToPageSetT &c) { - std::string r = format("Version %" PRId64 " => [", c.first); - for(auto &o : c.second) { - r += toString(o) + " "; + // Utility class for building a vector of internal page entries. + // Entries must be added in version order. Modified will be set to true + // if any entries differ from the original ones. Additional entries will be + // added when necessary to reconcile differences between the upper and lower + // boundaries of consecutive entries. + struct InternalPageBuilder { + // Cursor must be at first entry in page + InternalPageBuilder(const BTreePage::BinaryTree::Cursor &c) + : cursor(c), modified(false), childPageCount(0) + { } - return r + "]"; + + inline void addEntry(const RedwoodRecordRef &rec) { + if(rec.value.present()) { + ++childPageCount; + } + + // If no modification detected yet then check that this record is identical to the next + // record from the original page which is at the current cursor position. + if(!modified) { + if(cursor.valid()) { + if(!rec.identical(cursor.get())) { + modified = true; + } + else { + cursor.moveNext(); + } + } + else { + modified = true; + } + } + + entries.push_back(rec); + } + + void addEntries(const VersionedChildPageSet &newSet) { + // If there are already entries, the last one links to a child page, and its upper bound is not the same + // as the first lowerBound in newSet (or newSet is empty, as the next newSet is necessarily greater) + // then add the upper bound of the previous set as a value-less record so that on future reads + // the previous child page can be decoded correctly. + if(!entries.empty() && entries.back().value.present() + && (newSet.children.empty() || newSet.children.front() != lastUpperBound)) + { + addEntry(lastUpperBound.withoutValue()); + } + + for(auto &child : newSet.children) { + addEntry(child); + } + + lastUpperBound = newSet.upperBound; + } + + // Finish comparison to existing data if necesary. + // Handle possible page upper bound changes. + // If modified is set (see below) and our rightmost entry has a child page and its upper bound + // (currently in lastUpperBound) does not match the new desired page upper bound, passed as newUpperBound, + // then write lastUpperBound with no value to allow correct decoding of the rightmost entry. + // This is only done if modified is set to avoid rewriting this page for this purpose only. + // + // After this call, lastUpperBound is internal page's upper bound. + void finalize(const RedwoodRecordRef &newUpperBound) { + if(!modified) { + // If the original set has any more entries then the page content has been modified. + if(cursor.valid()) { + modified = true; + } + } + + if(modified) { + if(!entries.empty() && entries.back().value.present() && lastUpperBound != newUpperBound) { + addEntry(lastUpperBound.withoutValue()); + } + lastUpperBound = newUpperBound; + } + } + + BTreePage::BinaryTree::Cursor cursor; + std::vector> entries; + Standalone lastUpperBound; + bool modified; + int childPageCount; + Arena arena; + }; + + + template + static std::string toString(const T &o) { + return o.toString(); } - static std::string toString(const VersionedChildrenT &c) { + template<> + std::string toString(const VersionedChildPageSet &c) { + return format("Version=%" PRId64 " children=%s upperBound=%s", c.version, toString(c.children).c_str(), c.upperBound.toString().c_str()); + } + + template + static std::string toString(const std::vector &v) { std::string r = "{ "; - for(auto &o : c) { + for(auto &o : v) { r += toString(o) + ", "; } return r + " }"; @@ -1377,8 +1526,8 @@ private: while(pages.size() > 1) { std::vector childEntries; for(int i=0; iallocateLogicalPage(); debug_printf("%p: writePages(): Writing extension page op=write id=%u @%" PRId64 " (%d of %lu) referencePage=%u\n", actor_debug, eid, version, e + 1, extPages.size(), id); - newPage->extensionPages[e] = eid; + newPage->extensionPages[e] = bigEndian32(eid); // If replacing the primary page below (version == 0) then pass the primary page's ID as the reference page ID m_pager->writePage(eid, extPages[e], version, (version == 0) ? id : invalidLogicalPageID); ++counts.extPageWrites; @@ -1441,8 +1590,8 @@ private: // Free the old extension pages now that all replacement pages have been written for(int i = 0; i < originalPage->extensionPageCount; ++i) { - //debug_printf("%p: writePages(): Freeing old extension op=del id=%u @latest\n", actor_debug, originalPage->extensionPages[i]); - //m_pager->freeLogicalPage(originalPage->extensionPages[i], version); + //debug_printf("%p: writePages(): Freeing old extension op=del id=%u @latest\n", actor_debug, bigEndian32(originalPage->extensionPages[i])); + //m_pager->freeLogicalPage(bigEndian32(originalPage->extensionPages[i]), version); } return primaryLogicalPageIDs; @@ -1505,8 +1654,8 @@ private: pageGets.push_back(std::move(result)); for(int i = 0; i < pTreePage->extensionPageCount; ++i) { - debug_printf("readPage() Reading extension page op=read id=%u @%" PRId64 " ext=%d/%d\n", pTreePage->extensionPages[i], snapshot->getVersion(), i + 1, (int)pTreePage->extensionPageCount); - pageGets.push_back(snapshot->getPhysicalPage(pTreePage->extensionPages[i])); + debug_printf("readPage() Reading extension page op=read id=%u @%" PRId64 " ext=%d/%d\n", bigEndian32(pTreePage->extensionPages[i]), snapshot->getVersion(), i + 1, (int)pTreePage->extensionPageCount); + pageGets.push_back(snapshot->getPhysicalPage(bigEndian32(pTreePage->extensionPages[i]))); } std::vector> pages = wait(getAll(pageGets)); @@ -1532,13 +1681,18 @@ private: // Returns list of (version, list of (lower_bound, list of children) ) // TODO: Probably should pass prev/next records by pointer in many places ACTOR static Future commitSubtree(VersionedBTree *self, MutationBufferT *mutationBuffer, Reference snapshot, LogicalPageID root, const RedwoodRecordRef *lowerBound, const RedwoodRecordRef *upperBound, const RedwoodRecordRef *decodeLowerBound, const RedwoodRecordRef *decodeUpperBound) { - debug_printf("%p commitSubtree: root=%d lower=%s upper=%s\n", THIS, root, lowerBound->toString().c_str(), upperBound->toString().c_str()); - debug_printf("%p commitSubtree: root=%d decodeLower=%s decodeUpper=%s\n", THIS, root, decodeLowerBound->toString().c_str(), decodeUpperBound->toString().c_str()); + state std::string context; + if(REDWOOD_DEBUG) { + context = format("CommitSubtree(root=%u): ", root); + } + + debug_printf("%s root=%d lower=%s upper=%s\n", context.c_str(), root, lowerBound->toString().c_str(), upperBound->toString().c_str()); + debug_printf("%s root=%d decodeLower=%s decodeUpper=%s\n", context.c_str(), root, decodeLowerBound->toString().c_str(), decodeUpperBound->toString().c_str()); self->counts.commitToPageStart++; // If a boundary changed, the page must be rewritten regardless of KV mutations state bool boundaryChanged = (lowerBound != decodeLowerBound) || (upperBound != decodeUpperBound); - debug_printf("%p id=%u boundaryChanged=%d\n", THIS, root, boundaryChanged); + debug_printf("%s id=%u boundaryChanged=%d\n", context.c_str(), root, boundaryChanged); // Find the slice of the mutation buffer that is relevant to this subtree // TODO: Rather than two lower_bound searches, perhaps just compare each mutation to the upperBound key while iterating @@ -1555,14 +1709,14 @@ private: if(iMutationBoundary == iMutationBoundaryEnd) { if(!iMutationBoundary->second.startKeyMutations.empty()) { VersionedChildrenT c; - debug_printf("%p id=%u lower and upper bound key/version match and key is modified so deleting page, returning %s\n", THIS, root, toString(c).c_str()); + debug_printf("%s id=%u lower and upper bound key/version match and key is modified so deleting page, returning %s\n", context.c_str(), root, toString(c).c_str()); return c; } // If there are no forced boundary changes then this subtree is unchanged. if(!boundaryChanged) { - VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); - debug_printf("%p id=%d page contains a single key '%s' which is not changing, returning %s\n", THIS, root, lowerBound->key.toString().c_str(), toString(c).c_str()); + VersionedChildrenT c({ {0, {*decodeLowerBound}, *decodeUpperBound} }); + debug_printf("%s id=%d page contains a single key '%s' which is not changing, returning %s\n", context.c_str(), root, lowerBound->key.toString().c_str(), toString(c).c_str()); return c; } } @@ -1577,17 +1731,17 @@ private: iMutationBoundary->first < lowerBound->key) ) ) { - VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); - debug_printf("%p no changes because sole mutation range was not cleared, returning %s\n", THIS, toString(c).c_str()); + VersionedChildrenT c({ {0, {*decodeLowerBound}, *decodeUpperBound} }); + debug_printf("%s no changes because sole mutation range was not cleared, returning %s\n", context.c_str(), toString(c).c_str()); return c; } self->counts.commitToPage++; state Reference rawPage = wait(readPage(snapshot, root, self->m_usablePageSizeOverride, decodeLowerBound, decodeUpperBound)); state BTreePage *page = (BTreePage *) rawPage->begin(); - debug_printf("%p commitSubtree(): %s\n", THIS, page->toString(false, root, snapshot->getVersion(), decodeLowerBound, decodeUpperBound).c_str()); + debug_printf("%s commitSubtree(): %s\n", context.c_str(), page->toString(false, root, snapshot->getVersion(), decodeLowerBound, decodeUpperBound).c_str()); - BTreePage::BinaryTree::Cursor cursor = getReader(rawPage)->getCursor(); + state BTreePage::BinaryTree::Cursor cursor = getReader(rawPage)->getCursor(); cursor.moveFirst(); // Leaf Page @@ -1595,7 +1749,7 @@ private: VersionedChildrenT results; std::vector merged; - debug_printf("%p id=%u MERGING EXISTING DATA WITH MUTATIONS:\n", THIS, root); + debug_printf("%s id=%u MERGING EXISTING DATA WITH MUTATIONS:\n", context.c_str(), root); if(REDWOOD_DEBUG) { self->printMutationBuffer(iMutationBoundary, iMutationBoundaryEnd); } @@ -1609,7 +1763,7 @@ private: // Now, process each mutation range and merge changes with existing data. while(iMutationBoundary != iMutationBoundaryEnd) { - debug_printf("%p New mutation boundary: '%s': %s\n", THIS, printable(iMutationBoundary->first).c_str(), iMutationBoundary->second.toString().c_str()); + debug_printf("%s New mutation boundary: '%s': %s\n", context.c_str(), printable(iMutationBoundary->first).c_str(), iMutationBoundary->second.toString().c_str()); SingleKeyMutationsByVersion::const_iterator iMutations; @@ -1634,11 +1788,11 @@ private: // If not in single version mode or there were no changes to the key if(!self->singleVersion || iMutationBoundary->second.noChanges()) { merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, boundary start]\n", THIS, merged.back().toString().c_str()); + debug_printf("%s Added %s [existing, boundary start]\n", context.c_str(), merged.back().toString().c_str()); } else { ASSERT(self->singleVersion); - debug_printf("%p: Skipped %s [existing, boundary start, singleVersion mode]\n", THIS, cursor.get().toString().c_str()); + debug_printf("%s Skipped %s [existing, boundary start, singleVersion mode]\n", context.c_str(), cursor.get().toString().c_str()); minVersion = 0; } cursor.moveNext(); @@ -1654,7 +1808,7 @@ private: minVersion = iMutations->first; ++changes; merged.push_back(m.toRecord(iMutationBoundary->first, iMutations->first)); - debug_printf("%p: Added non-split %s [mutation, boundary start]\n", THIS, merged.back().toString().c_str()); + debug_printf("%s Added non-split %s [mutation, boundary start]\n", context.c_str(), merged.back().toString().c_str()); } else { if(iMutations->first < minVersion || minVersion == invalidVersion) @@ -1669,7 +1823,7 @@ private: merged.push_back(whole.split(start, partSize)); bytesLeft -= partSize; start += partSize; - debug_printf("%p: Added split %s [mutation, boundary start]\n", THIS, merged.back().toString().c_str()); + debug_printf("%s Added split %s [mutation, boundary start]\n", context.c_str(), merged.back().toString().c_str()); } } ++iMutations; @@ -1680,7 +1834,7 @@ private: // Advance to the next boundary because we need to know the end key for the current range. ++iMutationBoundary; - debug_printf("%p Mutation range end: '%s'\n", THIS, printable(iMutationBoundary->first).c_str()); + debug_printf("%s Mutation range end: '%s'\n", context.c_str(), printable(iMutationBoundary->first).c_str()); // Write existing keys which are less than the next mutation boundary key, clearing if needed. while(cursor.valid() && cursor.get().key < iMutationBoundary->first) { @@ -1689,11 +1843,11 @@ private: bool remove = self->singleVersion && clearRangeVersion.present(); if(!remove) { merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, middle]\n", THIS, merged.back().toString().c_str()); + debug_printf("%s Added %s [existing, middle]\n", context.c_str(), merged.back().toString().c_str()); } else { ASSERT(self->singleVersion); - debug_printf("%p: Skipped %s [existing, boundary start, singleVersion mode]\n", THIS, cursor.get().toString().c_str()); + debug_printf("%s Skipped %s [existing, boundary start, singleVersion mode]\n", context.c_str(), cursor.get().toString().c_str()); Version clearVersion = clearRangeVersion.get(); if(clearVersion < minVersion || minVersion == invalidVersion) minVersion = clearVersion; @@ -1713,7 +1867,7 @@ private: minVersion = clearVersion; ++changes; merged.push_back(RedwoodRecordRef(cursor.get().key, clearVersion)); - debug_printf("%p: Added %s [existing, middle clear]\n", THIS, merged.back().toString().c_str()); + debug_printf("%s Added %s [existing, middle clear]\n", context.c_str(), merged.back().toString().c_str()); } cursor = nextCursor; } @@ -1726,16 +1880,17 @@ private: // Write any remaining existing keys, which are not subject to clears as they are beyond the cleared range. while(cursor.valid()) { merged.push_back(cursor.get()); - debug_printf("%p: Added %s [existing, tail]\n", THIS, merged.back().toString().c_str()); + debug_printf("%s Added %s [existing, tail]\n", context.c_str(), merged.back().toString().c_str()); cursor.moveNext(); } - debug_printf("%p Done merging mutations into existing leaf contents, made %d changes\n", THIS, changes); + debug_printf("%s Done merging mutations into existing leaf contents, made %d changes\n", context.c_str(), changes); // No changes were actually made. This could happen if the only mutations are clear ranges which do not match any records. + // But if a boundary was changed then we must rewrite the page anyway. if(!boundaryChanged && minVersion == invalidVersion) { - VersionedChildrenT c({ {0,{{*lowerBound,root}}} }); - debug_printf("%p No changes were made during mutation merge, returning %s\n", THIS, toString(c).c_str()); + VersionedChildrenT c({ {0, {*decodeLowerBound}, *decodeUpperBound} }); + debug_printf("%s No changes were made during mutation merge, returning %s\n", context.c_str(), toString(c).c_str()); ASSERT(changes == 0); return c; } @@ -1747,7 +1902,7 @@ private: if(merged.empty() && root != 0) { // TODO: For multi version mode only delete this page as of the new version VersionedChildrenT c({}); - debug_printf("%p id=%u All leaf page contents were cleared, returning %s\n", THIS, root, toString(c).c_str()); + debug_printf("%s id=%u All leaf page contents were cleared, returning %s\n", context.c_str(), root, toString(c).c_str()); return c; } @@ -1755,18 +1910,19 @@ private: std::vector pages = buildPages(true, *lowerBound, *upperBound, merged, BTreePage::IS_LEAF, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); if(!self->singleVersion) { - // If there isn't still just a single page of data then this page became too large and was split. - // The new split pages will be valid as of minVersion, but the old page remains valid at the old version - if(pages.size() != 1) { - results.push_back( {0, {{*lowerBound, root}}} ); - debug_printf("%p Added versioned child set #1: %s\n", THIS, toString(results.back()).c_str()); - } - else { - // The page was updated but not size-split or version-split so the last page version's data - // can be replaced with the new page contents - if(pages.size() == 1) - minVersion = 0; - } + ASSERT(false); +// // If there isn't still just a single page of data then this page became too large and was split. +// // The new split pages will be valid as of minVersion, but the old page remains valid at the old version +// if(pages.size() != 1) { +// results.push_back( {0, {*decodeLowerBound}, ??} ); +// debug_printf("%s Added versioned child set #1: %s\n", context.c_str(), toString(results.back()).c_str()); +// } +// else { +// // The page was updated but not size-split or version-split so the last page version's data +// // can be replaced with the new page contents +// if(pages.size() == 1) +// minVersion = 0; +// } } // Write page(s), get new page IDs @@ -1775,20 +1931,21 @@ private: // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page if(root == self->m_root && pages.size() > 1) { - debug_printf("%p Building new root\n", THIS); + debug_printf("%s Building new root\n", context.c_str()); self->buildNewRoot(writeVersion, pages, newPageIDs, page); } - results.push_back({writeVersion, {}}); + results.push_back({writeVersion, {}, *upperBound}); for(int i=0; i Page %d\n", THIS, lower.toString().c_str(), newPageIDs[i]); - results.back().second.push_back( {lower, newPageIDs[i]} ); + RedwoodRecordRef entry = lower.withPageID(newPageIDs[i]); + debug_printf("%s Adding child page link: %s\n", context.c_str(), entry.toString().c_str()); + results.back().children.push_back(entry); } - debug_printf("%p Added versioned child set #2: %s\n", THIS, toString(results.back()).c_str()); + debug_printf("%s Merge complete, returning %s\n", context.c_str(), toString(results).c_str()); - debug_printf("%p DONE.\n", THIS); + debug_printf("%s DONE.\n", context.c_str()); return results; } else { @@ -1796,14 +1953,7 @@ private: // TODO: Combine these into one vector and/or do something more elegant state std::vector> futureChildren; - state std::vector childPageIDs; - state std::vector lowerBoundaries; - state std::vector decodeLowerBoundaries; - // Track whether or not any child has had its boundaries forcibly changed - state bool childBoundariesChanged = false; - - // TODO: Make this much more efficient with a skip-merge through the two sorted sets (mutations, existing cursor) bool first = true; while(cursor.valid()) { // The lower bound for the first child is the lowerBound arg @@ -1815,18 +1965,16 @@ private: // If there are any, then the first valid child page will incur a boundary change to move // its lower bound to the left so we can delete the non-linking entry from this page to free up space. while(!cursor.get().value.present()) { - // There should be an internal page written that has no valid child pages. This loop will find + // There should never be an internal page written that has no valid child pages. This loop will find // the first valid child link, and if there are no more then execution will not return to this loop. ASSERT(cursor.moveNext()); - childBoundariesChanged = true; } ASSERT(cursor.valid()); const RedwoodRecordRef &decodeChildLowerBound = cursor.get(); - const Optional &childValue = cursor.get().value; - uint32_t pageID = *(uint32_t*)childValue.get().begin(); + LogicalPageID pageID = cursor.get().getPageID(); ASSERT(pageID != 0); const RedwoodRecordRef &decodeChildUpperBound = cursor.moveNext() ? cursor.get() : *upperBound; @@ -1834,13 +1982,12 @@ private: // Skip over any next-children which do not actually link to child pages while(cursor.valid() && !cursor.get().value.present()) { cursor.moveNext(); - childBoundariesChanged = true; } const RedwoodRecordRef &childUpperBound = cursor.valid() ? cursor.get() : *upperBound; - debug_printf("%p internal page id=%d child page id=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", - THIS, root, pageID, childLowerBound.toString().c_str(), childUpperBound.toString().c_str(), decodeChildLowerBound.toString().c_str(), decodeChildUpperBound.toString().c_str()); + debug_printf("%s internal page id=%d child page id=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", + context.c_str(), root, pageID, childLowerBound.toString().c_str(), childUpperBound.toString().c_str(), decodeChildLowerBound.toString().c_str(), decodeChildUpperBound.toString().c_str()); /* // TODO: If lower bound and upper bound have the same key, do something intelligent if possible @@ -1873,9 +2020,6 @@ private: } */ futureChildren.push_back(self->commitSubtree(self, mutationBuffer, snapshot, pageID, &childLowerBound, &childUpperBound, &decodeChildLowerBound, &decodeChildUpperBound)); - childPageIDs.push_back(pageID); - lowerBoundaries.push_back(&childLowerBound); - decodeLowerBoundaries.push_back(&decodeChildLowerBound); } // Waiting one at a time makes debugging easier @@ -1885,212 +2029,91 @@ private: wait(success(futureChildren[k])); } - // Were any children modified? - bool modified = false; - // Were all children deleted? - bool all_deleted = true; - if(REDWOOD_DEBUG) { - debug_printf("%p Subtree update results for id=%d\n", THIS, root); + debug_printf("%s Subtree update results for root Page id=%u\n", context.c_str(), root); for(int i = 0; i < futureChildren.size(); ++i) { - const VersionedChildrenT &children = futureChildren[i].get(); - debug_printf("%p subtree for child page id=%u lowerBound=%s: %s\n", THIS, childPageIDs[i], lowerBoundaries[i]->toString(50).c_str(), toString(children).c_str()); + const VersionedChildrenT &vc = futureChildren[i].get(); + debug_printf("%s subtree result parent=%u %s\n", context.c_str(), root, toString(vc).c_str()); } } + // TODO: Handle multi-versioned results + ASSERT(self->singleVersion); + cursor.moveFirst(); + InternalPageBuilder pageBuilder(cursor); + for(int i = 0; i < futureChildren.size(); ++i) { - const VersionedChildrenT &children = futureChildren[i].get(); + const VersionedChildrenT &versionedChildren = futureChildren[i].get(); + ASSERT(versionedChildren.size() <= 1); - if(children.empty()) { - modified = true; + if(!versionedChildren.empty()) { + pageBuilder.addEntries(versionedChildren.front()); } - else { - // Not all children were deleted since this branch has a child - all_deleted = false; + } - // If the merge resulted in 1 versioned child set with exactly one child - // page, and its page ID and lower bound are the same as the original, then no changes were made. - // Otherwise, a change was made so we know the page was modified and we can stop iterating. - if(!( children.size() == 1 - && children.front().second.size() == 1 - && children.front().second.front().second == childPageIDs[i] - && children.front().second.front().first == *decodeLowerBoundaries[i] - ) - ) - { - modified = true; - break; + pageBuilder.finalize(*upperBound); + + // If page contents have changed + if(pageBuilder.modified) { + // If the page now has no children + if(pageBuilder.childPageCount == 0) { + // If we are the root, write a new empty btree + if(root == 0) { + Reference page = self->m_pager->newPageBuffer(); + makeEmptyPage(page, BTreePage::IS_LEAF, self->m_usablePageSizeOverride); + RedwoodRecordRef rootEntry = dbBegin.withPageID(0); + self->writePage(0, page, self->getLastCommittedVersion() + 1, &dbBegin, &dbEnd); + VersionedChildrenT c({ {0, {dbBegin}, dbEnd } }); + debug_printf("%s id=%u All root page children were deleted, rewrote root as leaf, returning %s\n", context.c_str(), root, toString(c).c_str()); + return c; + } + else { + VersionedChildrenT c({}); + debug_printf("%s id=%u All internal page children were deleted #1 so deleting this page too, returning %s\n", context.c_str(), root, toString(c).c_str()); + return c; } } - } - - if(childBoundariesChanged) { - modified = true; - } - - if(all_deleted) { - ASSERT(modified); - if(root == 0) { - Reference page = self->m_pager->newPageBuffer(); - makeEmptyPage(page, BTreePage::IS_LEAF, self->m_usablePageSizeOverride); - self->writePage(0, page, self->getLastCommittedVersion() + 1, &dbBegin, &dbEnd); - VersionedChildrenT c({ {0, { {dbBegin, 0} } } }); - debug_printf("%p id=%u All root page children were deleted, rewrote root as leaf, returning %s\n", THIS, root, toString(c).c_str()); - return c; - } else { - VersionedChildrenT c({}); - debug_printf("%p id=%u All internal page children were deleted #1 so deleting this page too, returning %s\n", THIS, root, toString(c).c_str()); - return c; + debug_printf("%s Internal page id=%u modified, creating replacements.\n", context.c_str(), root); + debug_printf("%s newChildren=%s lastUpperBound=%s upperBound=%s\n", context.c_str(), toString(pageBuilder.entries).c_str(), pageBuilder.lastUpperBound.toString().c_str(), upperBound->toString().c_str()); + + ASSERT(pageBuilder.lastUpperBound == *upperBound); + + // TODO: Don't do this! + std::vector entries; + for(auto &o : pageBuilder.entries) { + entries.push_back(o); + } + + std::vector pages = buildPages(false, *lowerBound, *upperBound, entries, 0, [=](){ return self->m_pager->newPageBuffer(); }, self->m_usablePageSizeOverride); + + Version writeVersion = self->getLastCommittedVersion() + 1; + std::vector newPageIDs = self->writePages(pages, writeVersion, root, page, upperBound, THIS); + + // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page + if(root == self->m_root) { + self->buildNewRoot(writeVersion, pages, newPageIDs, page); + } + + VersionedChildrenT vc(1); + vc.resize(1); + VersionedChildPageSet &c = vc.front(); + c.version = writeVersion; + c.upperBound = *upperBound; + + for(int i=0; i::max(); - - std::vector childEntries; - - // For each Future - debug_printf("%p creating replacement pages for id=%d at Version %" PRId64 "\n", THIS, root, version); - - // In multi version mode if we're writing version 0 there is a chance that we don't have to write ourselves, if there are no changes in any child subtrees - bool modified = self->singleVersion || version != 0; - - for(int i = 0; i < futureChildren.size(); ++i) { - const VersionedChildrenT &children = futureChildren[i].get(); - if(children.empty()) { - // Subtree was deleted, but due to prefix dependencies the boundary key might still need to exist - modified = true; - - // If there is a previous child and it has a subtree then this boundary key must still exist - // to continue being the upper bound of the previous child - if(!childEntries.empty() && childEntries.back().value.present()) { - RedwoodRecordRef entry(*lowerBoundaries[i]); - entry.value = Optional(); - childEntries.push_back(entry); - } - continue; - } - - debug_printf("%p Versioned page set that replaced Page id=%d: %lu versions\n", THIS, childPageIDs[i], children.size()); - if(REDWOOD_DEBUG) { - for(auto &versionedPageSet : children) { - debug_printf("%p version %" PRId64 "\n", THIS, versionedPageSet.first); - for(auto &boundaryPage : versionedPageSet.second) { - debug_printf("%p '%s' -> Page id=%u\n", THIS, boundaryPage.first.toString().c_str(), boundaryPage.second); - } - } - } - - vector::const_iterator cv; - - if(self->singleVersion) { - ASSERT(children.size() == 1); - cv = children.begin(); - nextVersion = std::numeric_limits::max(); - } - else { - // Find the first version greater than the current version we are writing - cv = std::upper_bound( children.begin(), children.end(), version, [](Version a, VersionedChildrenT::value_type const &b) { return a < b.first; } ); - - // If there are no versions before the one we found, just update nextVersion and continue. - if(cv == children.begin()) { - debug_printf("%p First version (%" PRId64 ") in set is greater than current, setting nextVersion and continuing\n", THIS, cv->first); - nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr %" PRId64 " next %" PRId64 "\n", THIS, version, nextVersion); - continue; - } - - // If a version greater than the current version being written was found, update nextVersion - if(cv != children.end()) { - nextVersion = std::min(nextVersion, cv->first); - debug_printf("%p curr %" PRId64 " next %" PRId64 "\n", THIS, version, nextVersion); - } - - // Go back one to the last version that was valid prior to or at the current version we are writing - --cv; - } - - debug_printf("%p Using children for version %" PRId64 " from this set, building version %" PRId64 "\n", THIS, cv->first, version); - - // If page count isn't 1 then the root is definitely modified - modified = modified || cv->second.size() != 1; - - // TODO: If page ID changed in singleVersion mode then root is modified - - // Add the children at this version to the child entries list for the current version being built. - for (auto &childPage : cv->second) { - RedwoodRecordRef entry = childPage.first; - entry.value = StringRef((unsigned char *)&childPage.second, sizeof(uint32_t)); - debug_printf("%p Adding child page %s\n", THIS, entry.toString().c_str()); - childEntries.push_back(entry); - } - } - - debug_printf("%p Finished pass through futurechildren. childEntries=%lu version=%" PRId64 " nextVersion=%" PRId64 "\n", THIS, childEntries.size(), version, nextVersion); - - if(modified) { - // If all children were deleted then this page should be deleted as of the new version - // Note that if a single range clear covered the entire page then we should not get this far - if(childEntries.empty()) { - if(self->singleVersion) { - debug_printf("%p All internal page children were deleted #2 at version %" PRId64 "\n", THIS, version); - } - else { - VersionedKeyToPageSetT c({version, {} }); - debug_printf("%p All internal page children were deleted #3 at version %" PRId64 ", adding %s\n", THIS, version, toString(c).c_str()); - result.push_back(c); - } - } - else { - // TODO: Track split points across iterations of this loop, so that they don't shift unnecessarily and - // cause unnecessary path copying - - IPager *pager = self->m_pager; - std::vector pages = buildPages(false, *lowerBound, *upperBound, childEntries, 0, [pager](){ return pager->newPageBuffer(); }, self->m_usablePageSizeOverride); - - // Write page(s), use version 0 to replace latest version if only writing one page - Version writeVersion = self->singleVersion ? self->getLastCommittedVersion() + 1 : version; - std::vector newPageIDs = self->writePages(pages, writeVersion, root, page, upperBound, THIS); - - // If this commitSubtree() is operating on the root, write new levels if needed until until we're returning a single page - if(root == self->m_root) { - self->buildNewRoot(writeVersion, pages, newPageIDs, page); - } - - result.resize(result.size()+1); - result.back().first = writeVersion; - - for(int i=0; i::max()) - break; - version = nextVersion; - } - - debug_printf("%p DONE.\n", THIS); - return result; } } @@ -2183,7 +2206,7 @@ printf("\nCommitted: %s\n", self->counts.toString(true).c_str()); BTreePage::BinaryTree::Cursor next = cursor; next.moveNext(); const RedwoodRecordRef &rec = cursor.get(); - LogicalPageID id = *(LogicalPageID *)rec.value.get().begin(); + LogicalPageID id = rec.getPageID(); Future> child = readPage(pager, id, usablePageSizeOverride, &rec, &next.getOrUpperBound()); return map(child, [=](Reference page) { return Reference(new PageCursor(id, page, Reference::addRef(this))); @@ -3256,6 +3279,23 @@ Standalone randomRedwoodRecordRef(int maxKeySize = 3, int maxV } TEST_CASE("!/redwood/correctness/unit/RedwoodRecordRef") { + + // Test pageID stuff. + { + LogicalPageID id = 1; + RedwoodRecordRef r; + r.setPageID(id); + ASSERT(r.getPageID() == id); + RedwoodRecordRef s; + s = r; + ASSERT(s.getPageID() == id); + RedwoodRecordRef t(r); + ASSERT(t.getPageID() == id); + r.setPageID(id + 1); + ASSERT(s.getPageID() == id); + ASSERT(t.getPageID() == id); + } + // Testing common prefix calculation for integer fields using the member function that calculates this directly // and by serializing the integer fields to arrays and finding the common prefix length of the two arrays From 6aad34620d7c3363885151e0442cee1ed220e469 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Tue, 4 Jun 2019 04:55:09 -0700 Subject: [PATCH 020/128] Bug fix in upper boundary selection in commitSubtree(). More debug output. --- fdbserver/VersionedBTree.actor.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 9ae706a212..051221802e 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -1265,6 +1265,7 @@ private: if(!modified) { if(cursor.valid()) { if(!rec.identical(cursor.get())) { + debug_printf("InternalPageBuilder: Found internal page difference. new: %s old: %s\n", rec.toString().c_str(), cursor.get().toString().c_str()); modified = true; } else { @@ -1272,6 +1273,7 @@ private: } } else { + debug_printf("InternalPageBuilder: Found internal page difference. new: %s old: \n", rec.toString().c_str()); modified = true; } } @@ -1287,14 +1289,17 @@ private: if(!entries.empty() && entries.back().value.present() && (newSet.children.empty() || newSet.children.front() != lastUpperBound)) { + debug_printf("InternalPageBuilder: Added placeholder %s\n", lastUpperBound.withoutValue().toString().c_str()); addEntry(lastUpperBound.withoutValue()); } for(auto &child : newSet.children) { + debug_printf("InternalPageBuilder: Adding child entry %s\n", child.toString().c_str()); addEntry(child); } lastUpperBound = newSet.upperBound; + debug_printf("InternalPageBuilder: New upper bound: %s\n", lastUpperBound.toString().c_str()); } // Finish comparison to existing data if necesary. @@ -1313,11 +1318,15 @@ private: } } + debug_printf("InternalPageBuilder: finalizing. modified=%d newUpperBound=%s\n", modified, newUpperBound.toString().c_str()); if(modified) { if(!entries.empty() && entries.back().value.present() && lastUpperBound != newUpperBound) { + debug_printf("InternalPageBuilder: Added placeholder %s\n", lastUpperBound.withoutValue().toString().c_str()); addEntry(lastUpperBound.withoutValue()); } + lastUpperBound = newUpperBound; + debug_printf("InternalPageBuilder: New upper bound: %s\n", lastUpperBound.toString().c_str()); } } @@ -1977,7 +1986,7 @@ private: LogicalPageID pageID = cursor.get().getPageID(); ASSERT(pageID != 0); - const RedwoodRecordRef &decodeChildUpperBound = cursor.moveNext() ? cursor.get() : *upperBound; + const RedwoodRecordRef &decodeChildUpperBound = cursor.moveNext() ? cursor.get() : *decodeUpperBound; // Skip over any next-children which do not actually link to child pages while(cursor.valid() && !cursor.get().value.present()) { @@ -1986,8 +1995,8 @@ private: const RedwoodRecordRef &childUpperBound = cursor.valid() ? cursor.get() : *upperBound; - debug_printf("%s internal page id=%d child page id=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", - context.c_str(), root, pageID, childLowerBound.toString().c_str(), childUpperBound.toString().c_str(), decodeChildLowerBound.toString().c_str(), decodeChildUpperBound.toString().c_str()); + debug_printf("%s recursing to page id=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", + context.c_str(), pageID, childLowerBound.toString().c_str(), childUpperBound.toString().c_str(), decodeChildLowerBound.toString().c_str(), decodeChildUpperBound.toString().c_str()); /* // TODO: If lower bound and upper bound have the same key, do something intelligent if possible From 100789b354587669cc64a7ec745fd3ea360ad1c8 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Wed, 5 Jun 2019 20:58:47 -0700 Subject: [PATCH 021/128] More bug fixes in handling upperBound changes in modified pages and worst-case delta size calculation. Normalized some formatting in debug statements. Fixed compile error on linux. Updated test specs. --- fdbserver/VersionedBTree.actor.cpp | 88 ++++++++++++++++---------- tests/RedwoodCorrectnessBTree.txt | 6 ++ tests/RedwoodCorrectnessUnits.txt | 6 ++ tests/fast/RedwoodCorrectnessBTree.txt | 6 ++ 4 files changed, 73 insertions(+), 33 deletions(-) create mode 100644 tests/RedwoodCorrectnessBTree.txt create mode 100644 tests/RedwoodCorrectnessUnits.txt create mode 100644 tests/fast/RedwoodCorrectnessBTree.txt diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 051221802e..4d365049eb 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -581,7 +581,7 @@ struct RedwoodRecordRef { return compare(rhs) >= 0; } - int deltaSize(const RedwoodRecordRef &base) const { + int deltaSize(const RedwoodRecordRef &base, bool worstCase = true) const { int size = sizeof(Delta); if(value.present()) { @@ -590,19 +590,22 @@ struct RedwoodRecordRef { } int prefixLen = getCommonPrefixLen(base, 0); - size += (prefixLen >= 128) ? 2 : 1; + size += (worstCase || prefixLen >= 128) ? 2 : 1; int intFieldPrefixLen; // Currently using a worst-guess guess where int fields in suffix are stored in their entirety if nonzero. if(prefixLen < key.size()) { int keySuffixLen = key.size() - prefixLen; - size += (keySuffixLen >= 128) ? 2 : 1; + size += (worstCase || keySuffixLen >= 128) ? 2 : 1; size += keySuffixLen; intFieldPrefixLen = 0; } else { intFieldPrefixLen = prefixLen - key.size(); + if(worstCase) { + size += 2; + } } if(version == 0 && chunk.total == 0 && chunk.start == 0) { @@ -705,11 +708,11 @@ struct RedwoodRecordRef { std::string toString(int hexLimit = 15) const { std::string r; r += format("'%s'@%" PRId64, kvformat(key, hexLimit).c_str(), version); - r += format("[%d/%d]->", chunk.start, chunk.total); + r += format("[%u/%u]->", chunk.start, chunk.total); if(value.present()) { // Assume that values the size of a page ID are page IDs. It's not perfect but it's just for debugging. if(value.get().size() == sizeof(LogicalPageID)) { - r += format("[Page id=%u]", getPageID()); + r += format("[PageID=%u]", getPageID()); } else { r += format("'%s'", kvformat(value.get(), hexLimit).c_str()); @@ -773,14 +776,26 @@ struct BTreePage { c.moveFirst(); ASSERT(c.valid()); + bool anyOutOfRange = false; do { r += " "; r += c.get().toString(); + + bool tooLow = c.get().key < lowerBound->key; + bool tooHigh = c.get().key > upperBound->key; + if(tooLow || tooHigh) { + anyOutOfRange = true; + if(tooLow) { + r += " (too low)"; + } + if(tooHigh) { + r += " (too high)"; + } + } r += "\n"; - ASSERT(c.get().key >= lowerBound->key && c.get().key <= upperBound->key); - } while(c.moveNext()); + ASSERT(!anyOutOfRange); } } catch (Error& e) { debug_printf("BTreePage::toString ERROR: %s\n", e.what()); @@ -1310,24 +1325,32 @@ private: // This is only done if modified is set to avoid rewriting this page for this purpose only. // // After this call, lastUpperBound is internal page's upper bound. - void finalize(const RedwoodRecordRef &newUpperBound) { - if(!modified) { - // If the original set has any more entries then the page content has been modified. - if(cursor.valid()) { + void finalize(const RedwoodRecordRef &upperBound, const RedwoodRecordRef &decodeUpperBound) { + debug_printf("InternalPageBuilder::end modified=%d upperBound=%s decodeUpperBound=%s lastUpperBound=%s\n", modified, upperBound.toString().c_str(), decodeUpperBound.toString().c_str(), lastUpperBound.toString().c_str()); + modified = modified || cursor.valid(); + debug_printf("InternalPageBuilder::end modified=%d after cursor check\n", modified); + + // If there are boundary key entries and the last one has a child page then the + // upper bound for this internal page must match the required upper bound for + // the last child entry. + if(!entries.empty() && entries.back().value.present()) { + debug_printf("InternalPageBuilder::end last entry is not null\n"); + + // If the page contents were not modified so far and the upper bound required + // for the last child page (lastUpperBound) does not match what the page + // was encoded with then the page must be modified. + if(!modified && lastUpperBound != decodeUpperBound) { + debug_printf("InternalPageBuilder::end modified set true because lastUpperBound does not match decodeUpperBound\n"); modified = true; } - } - debug_printf("InternalPageBuilder: finalizing. modified=%d newUpperBound=%s\n", modified, newUpperBound.toString().c_str()); - if(modified) { - if(!entries.empty() && entries.back().value.present() && lastUpperBound != newUpperBound) { - debug_printf("InternalPageBuilder: Added placeholder %s\n", lastUpperBound.withoutValue().toString().c_str()); + if(modified && lastUpperBound != upperBound) { + debug_printf("InternalPageBuilder::end Modified is true but lastUpperBound does not match upperBound so adding placeholder\n"); addEntry(lastUpperBound.withoutValue()); + lastUpperBound = upperBound; } - - lastUpperBound = newUpperBound; - debug_printf("InternalPageBuilder: New upper bound: %s\n", lastUpperBound.toString().c_str()); } + debug_printf("InternalPageBuilder::end exit. modified=%d upperBound=%s decodeUpperBound=%s lastUpperBound=%s\n", modified, upperBound.toString().c_str(), decodeUpperBound.toString().c_str(), lastUpperBound.toString().c_str()); } BTreePage::BinaryTree::Cursor cursor; @@ -1344,8 +1367,7 @@ private: return o.toString(); } - template<> - std::string toString(const VersionedChildPageSet &c) { + static std::string toString(const VersionedChildPageSet &c) { return format("Version=%" PRId64 " children=%s upperBound=%s", c.version, toString(c.children).c_str(), c.upperBound.toString().c_str()); } @@ -1581,7 +1603,7 @@ private: for(int e = 0, eEnd = extPages.size(); e < eEnd; ++e) { LogicalPageID eid = m_pager->allocateLogicalPage(); - debug_printf("%p: writePages(): Writing extension page op=write id=%u @%" PRId64 " (%d of %lu) referencePage=%u\n", actor_debug, eid, version, e + 1, extPages.size(), id); + debug_printf("%p: writePages(): Writing extension page op=write id=%u @%" PRId64 " (%d of %lu) referencePageID=%u\n", actor_debug, eid, version, e + 1, extPages.size(), id); newPage->extensionPages[e] = bigEndian32(eid); // If replacing the primary page below (version == 0) then pass the primary page's ID as the reference page ID m_pager->writePage(eid, extPages[e], version, (version == 0) ? id : invalidLogicalPageID); @@ -1674,7 +1696,7 @@ private: } if(result->userData == nullptr) { - debug_printf("readPage() Creating Reader for page id=%u @%" PRId64 " lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); + debug_printf("readPage() Creating Reader for PageID=%u @%" PRId64 " lower=%s upper=%s\n", id, snapshot->getVersion(), lowerBound->toString().c_str(), upperBound->toString().c_str()); result->userData = new BTreePage::BinaryTree::Reader(&pTreePage->tree(), lowerBound, upperBound); result->userDataDestructor = [](void *ptr) { delete (BTreePage::BinaryTree::Reader *)ptr; }; } @@ -1995,7 +2017,7 @@ private: const RedwoodRecordRef &childUpperBound = cursor.valid() ? cursor.get() : *upperBound; - debug_printf("%s recursing to page id=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", + debug_printf("%s recursing to PageID=%u lower=%s upper=%s decodeLower=%s decodeUpper=%s\n", context.c_str(), pageID, childLowerBound.toString().c_str(), childUpperBound.toString().c_str(), decodeChildLowerBound.toString().c_str(), decodeChildUpperBound.toString().c_str()); /* @@ -2039,10 +2061,10 @@ private: } if(REDWOOD_DEBUG) { - debug_printf("%s Subtree update results for root Page id=%u\n", context.c_str(), root); + debug_printf("%s Subtree update results for root PageID=%u\n", context.c_str(), root); for(int i = 0; i < futureChildren.size(); ++i) { const VersionedChildrenT &vc = futureChildren[i].get(); - debug_printf("%s subtree result parent=%u %s\n", context.c_str(), root, toString(vc).c_str()); + debug_printf("%s subtree result %s\n", context.c_str(), toString(vc).c_str()); } } @@ -2060,7 +2082,7 @@ private: } } - pageBuilder.finalize(*upperBound); + pageBuilder.finalize(*upperBound, *decodeUpperBound); // If page contents have changed if(pageBuilder.modified) { @@ -2083,7 +2105,7 @@ private: } } else { - debug_printf("%s Internal page id=%u modified, creating replacements.\n", context.c_str(), root); + debug_printf("%s Internal PageID=%u modified, creating replacements.\n", context.c_str(), root); debug_printf("%s newChildren=%s lastUpperBound=%s upperBound=%s\n", context.c_str(), toString(pageBuilder.entries).c_str(), pageBuilder.lastUpperBound.toString().c_str(), upperBound->toString().c_str()); ASSERT(pageBuilder.lastUpperBound == *upperBound); @@ -2114,13 +2136,13 @@ private: c.children.push_back(pages[i].lowerBound.withPageID(newPageIDs[i])); } - debug_printf("%s Internal Page id=%u modified, returning %s\n", context.c_str(), root, toString(c).c_str()); + debug_printf("%s Internal PageID=%u modified, returning %s\n", context.c_str(), root, toString(c).c_str()); return vc; } } else { VersionedChildrenT c( { {0, {*decodeLowerBound}, *decodeUpperBound} }); - debug_printf("%s Page id=%u has no changes, returning %s\n", context.c_str(), root, toString(c).c_str()); + debug_printf("%s PageID=%u has no changes, returning %s\n", context.c_str(), root, toString(c).c_str()); return c; } } @@ -2223,7 +2245,7 @@ printf("\nCommitted: %s\n", self->counts.toString(true).c_str()); } std::string toString() const { - return format("Page %lu, %s", pageID, cursor.valid() ? cursor.get().toString().c_str() : ""); + return format("PageID=%u, %s", pageID, cursor.valid() ? cursor.get().toString().c_str() : ""); } }; @@ -3251,7 +3273,7 @@ void deltaTest(RedwoodRecordRef rec, RedwoodRecordRef base) { RedwoodRecordRef::Delta &d = *(RedwoodRecordRef::Delta *)buf; Arena mem; - int expectedSize = rec.deltaSize(base); + int expectedSize = rec.deltaSize(base, false); int deltaSize = rec.writeDelta(d, base); RedwoodRecordRef decoded = d.apply(base, mem); @@ -3677,7 +3699,7 @@ TEST_CASE("!/redwood/correctness/btree") { state int maxValueSize = g_random->randomInt(0, pageSize * 4); state int maxCommitSize = shortTest ? 1000 : randomSize(10e6); state int mutationBytesTarget = shortTest ? 5000 : randomSize(50e6); - state double clearChance = g_random->random01() * .01; // at most 1 in 100 + state double clearChance = g_random->random01() * .1; printf("Using page size %d, max key size %d, max value size %d, clearchance %f, total mutation byte target %d\n", pageSize, maxKeySize, maxValueSize, clearChance, mutationBytesTarget); diff --git a/tests/RedwoodCorrectnessBTree.txt b/tests/RedwoodCorrectnessBTree.txt new file mode 100644 index 0000000000..3bde204032 --- /dev/null +++ b/tests/RedwoodCorrectnessBTree.txt @@ -0,0 +1,6 @@ +testTitle=UnitTests +testName=UnitTests +startDelay=0 +useDB=false +maxTestCases=0 +testsMatching=!/redwood/correctness/btree diff --git a/tests/RedwoodCorrectnessUnits.txt b/tests/RedwoodCorrectnessUnits.txt new file mode 100644 index 0000000000..d5dabe2216 --- /dev/null +++ b/tests/RedwoodCorrectnessUnits.txt @@ -0,0 +1,6 @@ +testTitle=UnitTests +testName=UnitTests +startDelay=0 +useDB=false +maxTestCases=0 +testsMatching=!/redwood/correctness/unit/ diff --git a/tests/fast/RedwoodCorrectnessBTree.txt b/tests/fast/RedwoodCorrectnessBTree.txt new file mode 100644 index 0000000000..3bde204032 --- /dev/null +++ b/tests/fast/RedwoodCorrectnessBTree.txt @@ -0,0 +1,6 @@ +testTitle=UnitTests +testName=UnitTests +startDelay=0 +useDB=false +maxTestCases=0 +testsMatching=!/redwood/correctness/btree From 8e0cbd299b495c1f913b053f2d8f230093245539 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Wed, 22 May 2019 16:09:04 -0700 Subject: [PATCH 022/128] Use 'map' for finding install paths --- cmake/InstallLayout.cmake | 98 +++++++++++++++++++-------------------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/cmake/InstallLayout.cmake b/cmake/InstallLayout.cmake index 6a7b75256c..380cb7ab1b 100644 --- a/cmake/InstallLayout.cmake +++ b/cmake/InstallLayout.cmake @@ -79,6 +79,49 @@ function(install_symlink) endif() endfunction() +# 'map' from (destination, package) to path +# format vars like install_destination_for_${destination}_${package} +set(install_destination_for_bin_tgz "bin") +set(install_destination_for_bin_deb "usr/bin") +set(install_destination_for_bin_el6 "usr/bin") +set(install_destination_for_bin_el7 "usr/bin") +set(install_destination_for_bin_pm "usr/local/bin") +set(install_destination_for_sbin_tgz "sbin") +set(install_destination_for_sbin_deb "usr/sbin") +set(install_destination_for_sbin_el6 "usr/sbin") +set(install_destination_for_sbin_el7 "usr/sbin") +set(install_destination_for_sbin_pm "usr/local/libexec") +set(install_destination_for_lib_tgz "lib") +set(install_destination_for_lib_deb "usr/lib") +set(install_destination_for_lib_el6 "usr/lib64") +set(install_destination_for_lib_el7 "usr/lib64") +set(install_destination_for_lib_pm "lib") +set(install_destination_for_fdbmonitor_tgz "libexec") +set(install_destination_for_fdbmonitor_deb "usr/lib/foundationdb") +set(install_destination_for_fdbmonitor_el6 "usr/lib/foundationdb") +set(install_destination_for_fdbmonitor_el7 "usr/lib/foundationdb") +set(install_destination_for_fdbmonitor_pm "usr/local/libexec") +set(install_destination_for_include_tgz "include") +set(install_destination_for_include_deb "usr/include") +set(install_destination_for_include_el6 "usr/include") +set(install_destination_for_include_el7 "usr/include") +set(install_destination_for_include_pm "usr/local/include") +set(install_destination_for_etc_tgz "etc/foundationdb") +set(install_destination_for_etc_deb "etc/foundationdb") +set(install_destination_for_etc_el6 "etc/foundationdb") +set(install_destination_for_etc_el7 "etc/foundationdb") +set(install_destination_for_etc_pm "usr/local/etc/foundationdb") +set(install_destination_for_log_tgz "log/foundationdb") +set(install_destination_for_log_deb "var/log/foundationdb") +set(install_destination_for_log_el6 "var/log/foundationdb") +set(install_destination_for_log_el7 "var/log/foundationdb") +set(install_destination_for_log_pm "") +set(install_destination_for_data_tgz "lib/foundationdb") +set(install_destination_for_data_deb "var/lib/foundationdb") +set(install_destination_for_data_el6 "var/lib/foundationdb") +set(install_destination_for_data_el7 "var/lib/foundationdb") +set(install_destination_for_data_pm "") + function(fdb_install) if(NOT WIN32 AND NOT OPEN_FOR_IDE) set(one_value_options COMPONENT DESTINATION) @@ -94,55 +137,12 @@ function(fdb_install) else() message(FATAL_ERROR "Expected FILES or TARGETS") endif() - if("${IN_DESTINATION}" STREQUAL "bin") - install(${args} DESTINATION "bin" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "usr/bin" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "usr/bin" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "usr/bin" COMPONENT "${IN_COMPONENT}-el7") - install(${args} DESTINATION "usr/local/bin" COMPONENT "${IN_COMPONENT}-pm") - elseif("${IN_DESTINATION}" STREQUAL "sbin") - install(${args} DESTINATION "sbin" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "usr/sbin" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "usr/sbin" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "usr/sbin" COMPONENT "${IN_COMPONENT}-el7") - install(${args} DESTINATION "usr/local/libexec" COMPONENT "${IN_COMPONENT}-pm") - elseif("${IN_DESTINATION}" STREQUAL "lib") - install(${args} DESTINATION "lib" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "usr/lib" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "usr/lib64" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "usr/lib64" COMPONENT "${IN_COMPONENT}-el7") - install(${args} DESTINATION "lib" COMPONENT "${IN_COMPONENT}-pm") - elseif("${IN_DESTINATION}" STREQUAL "fdbmonitor") - install(${args} DESTINATION "libexec" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "usr/lib/foundationdb" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "usr/lib/foundationdb" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "usr/lib/foundationdb" COMPONENT "${IN_COMPONENT}-el7") - install(${args} DESTINATION "usr/local/libexec" COMPONENT "${IN_COMPONENT}-pm") - elseif("${IN_DESTINATION}" STREQUAL "include") - install(${args} DESTINATION "include" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "usr/include" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "usr/include" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "usr/include" COMPONENT "${IN_COMPONENT}-el7") - install(${args} DESTINATION "usr/local/include" COMPONENT "${IN_COMPONENT}-pm") - elseif("${IN_DESTINATION}" STREQUAL "etc") - install(${args} DESTINATION "etc/foundationdb" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "etc/foundationdb" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "etc/foundationdb" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "etc/foundationdb" COMPONENT "${IN_COMPONENT}-el7") - install(${args} DESTINATION "usr/local/etc/foundationdb" COMPONENT "${IN_COMPONENT}-pm") - elseif("${IN_DESTINATION}" STREQUAL "log") - install(${args} DESTINATION "log/foundationdb" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "var/log/foundationdb" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "var/log/foundationdb" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "var/log/foundationdb" COMPONENT "${IN_COMPONENT}-el7") - elseif("${IN_DESTINATION}" STREQUAL "data") - install(${args} DESTINATION "lib/foundationdb" COMPONENT "${IN_COMPONENT}-tgz") - install(${args} DESTINATION "var/lib/foundationdb/data" COMPONENT "${IN_COMPONENT}-deb") - install(${args} DESTINATION "var/lib/foundationdb/data" COMPONENT "${IN_COMPONENT}-el6") - install(${args} DESTINATION "var/lib/foundationdb/data" COMPONENT "${IN_COMPONENT}-el7") - else() - message(FATAL_ERROR "unrecognized destination ${IN_DESTINATION}") - endif() + foreach(package tgz deb el6 el7 pm) + set(install_path "${install_destination_for_${IN_DESTINATION}_${package}}") + if(NOT ${install_path} STREQUAL "") + install(${args} DESTINATION "${install_path}" COMPONENT "${IN_COMPONENT}-${package}") + endif() + endforeach() endif() endfunction() From 2b07a0559ee8d0600b38ed7648f83aa1fa0b8a30 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 10:43:15 -0700 Subject: [PATCH 023/128] Install headers in foundationdb dir --- bindings/c/CMakeLists.txt | 4 +++- cmake/InstallLayout.cmake | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/bindings/c/CMakeLists.txt b/bindings/c/CMakeLists.txt index 2807b11b9b..faf29ffb05 100644 --- a/bindings/c/CMakeLists.txt +++ b/bindings/c/CMakeLists.txt @@ -72,5 +72,7 @@ fdb_install( FILES foundationdb/fdb_c.h ${CMAKE_CURRENT_BINARY_DIR}/foundationdb/fdb_c_options.g.h ${CMAKE_SOURCE_DIR}/fdbclient/vexillographer/fdb.options - DESTINATION include COMPONENT clients) + DESTINATION include + DESTINATION_SUFFIX /foundationdb + COMPONENT clients) #install(EXPORT fdbc DESTINATION ${FDB_LIB_DIR}/foundationdb COMPONENT clients) diff --git a/cmake/InstallLayout.cmake b/cmake/InstallLayout.cmake index 380cb7ab1b..e5d3f3ca62 100644 --- a/cmake/InstallLayout.cmake +++ b/cmake/InstallLayout.cmake @@ -124,7 +124,7 @@ set(install_destination_for_data_pm "") function(fdb_install) if(NOT WIN32 AND NOT OPEN_FOR_IDE) - set(one_value_options COMPONENT DESTINATION) + set(one_value_options COMPONENT DESTINATION DESTINATION_SUFFIX) set(multi_value_options TARGETS FILES DIRECTORY) cmake_parse_arguments(IN "${options}" "${one_value_options}" "${multi_value_options}" "${ARGN}") @@ -140,7 +140,7 @@ function(fdb_install) foreach(package tgz deb el6 el7 pm) set(install_path "${install_destination_for_${IN_DESTINATION}_${package}}") if(NOT ${install_path} STREQUAL "") - install(${args} DESTINATION "${install_path}" COMPONENT "${IN_COMPONENT}-${package}") + install(${args} DESTINATION "${install_path}${IN_DESTINATION_SUFFIX}" COMPONENT "${IN_COMPONENT}-${package}") endif() endforeach() endif() From cabe41626cc5137c51856fcc342550c574d7f5a3 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 11:15:39 -0700 Subject: [PATCH 024/128] Prepare for exporting fdb_c --- bindings/c/CMakeLists.txt | 10 ++++++---- cmake/InstallLayout.cmake | 12 ++++++++++-- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/bindings/c/CMakeLists.txt b/bindings/c/CMakeLists.txt index faf29ffb05..25b593488f 100644 --- a/bindings/c/CMakeLists.txt +++ b/bindings/c/CMakeLists.txt @@ -39,9 +39,9 @@ endif() add_dependencies(fdb_c fdb_c_generated fdb_c_options) target_link_libraries(fdb_c PUBLIC fdbclient) target_include_directories(fdb_c PUBLIC - ${CMAKE_CURRENT_BINARY_DIR} - ${CMAKE_CURRENT_SOURCE_DIR} - ${CMAKE_CURRENT_BINARY_DIR}/foundationdb) + $ + $ + $) if(WIN32) enable_language(ASM_MASM) set_property(SOURCE ${asm_file} PROPERTY LANGUAGE ASM_MASM) @@ -65,7 +65,9 @@ endif() # TODO: re-enable once the old vcxproj-based build system is removed. #generate_export_header(fdb_c EXPORT_MACRO_NAME "DLLEXPORT" # EXPORT_FILE_NAME ${CMAKE_CURRENT_BINARY_DIR}/foundationdb/fdb_c_export.h) -fdb_install(TARGETS fdb_c +fdb_install( + TARGETS fdb_c + EXPORT FoundationDB-Client DESTINATION lib COMPONENT clients) fdb_install( diff --git a/cmake/InstallLayout.cmake b/cmake/InstallLayout.cmake index e5d3f3ca62..1ae26b9e5f 100644 --- a/cmake/InstallLayout.cmake +++ b/cmake/InstallLayout.cmake @@ -124,7 +124,7 @@ set(install_destination_for_data_pm "") function(fdb_install) if(NOT WIN32 AND NOT OPEN_FOR_IDE) - set(one_value_options COMPONENT DESTINATION DESTINATION_SUFFIX) + set(one_value_options COMPONENT DESTINATION EXPORT DESTINATION_SUFFIX) set(multi_value_options TARGETS FILES DIRECTORY) cmake_parse_arguments(IN "${options}" "${one_value_options}" "${multi_value_options}" "${ARGN}") @@ -139,8 +139,16 @@ function(fdb_install) endif() foreach(package tgz deb el6 el7 pm) set(install_path "${install_destination_for_${IN_DESTINATION}_${package}}") + set(export_args "") + if (IN_EXPORT) + set(export_args EXPORT "${IN_EXPORT}") + endif() if(NOT ${install_path} STREQUAL "") - install(${args} DESTINATION "${install_path}${IN_DESTINATION_SUFFIX}" COMPONENT "${IN_COMPONENT}-${package}") + install( + ${args} + ${export_args} + DESTINATION "${install_path}${IN_DESTINATION_SUFFIX}" + COMPONENT "${IN_COMPONENT}-${package}") endif() endforeach() endif() From 882523330cc4d6762d69478bb3a2ce19a1c948df Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 11:46:31 -0700 Subject: [PATCH 025/128] include(/path/to/cmake/file) now works for fdb_c --- bindings/c/CMakeLists.txt | 8 ++++++-- cmake/InstallLayout.cmake | 25 ++++++++++++++++++------- 2 files changed, 24 insertions(+), 9 deletions(-) diff --git a/bindings/c/CMakeLists.txt b/bindings/c/CMakeLists.txt index 25b593488f..7aa5dccb40 100644 --- a/bindings/c/CMakeLists.txt +++ b/bindings/c/CMakeLists.txt @@ -37,7 +37,7 @@ else() strip_debug_symbols(fdb_c) endif() add_dependencies(fdb_c fdb_c_generated fdb_c_options) -target_link_libraries(fdb_c PUBLIC fdbclient) +target_link_libraries(fdb_c PUBLIC $) target_include_directories(fdb_c PUBLIC $ $ @@ -77,4 +77,8 @@ fdb_install( DESTINATION include DESTINATION_SUFFIX /foundationdb COMPONENT clients) -#install(EXPORT fdbc DESTINATION ${FDB_LIB_DIR}/foundationdb COMPONENT clients) +fdb_install( + EXPORT FoundationDB-Client + DESTINATION lib + DESTINATION_SUFFIX /cmake + COMPONENT clients) diff --git a/cmake/InstallLayout.cmake b/cmake/InstallLayout.cmake index 1ae26b9e5f..0cb1cc3a12 100644 --- a/cmake/InstallLayout.cmake +++ b/cmake/InstallLayout.cmake @@ -128,27 +128,38 @@ function(fdb_install) set(multi_value_options TARGETS FILES DIRECTORY) cmake_parse_arguments(IN "${options}" "${one_value_options}" "${multi_value_options}" "${ARGN}") + set(install_export 0) if(IN_TARGETS) set(args TARGETS ${IN_TARGETS}) elseif(IN_FILES) set(args FILES ${IN_FILES}) elseif(IN_DIRECTORY) set(args DIRECTORY ${IN_DIRECTORY}) + elseif(IN_EXPORT) + set(install_export 1) else() message(FATAL_ERROR "Expected FILES or TARGETS") endif() foreach(package tgz deb el6 el7 pm) set(install_path "${install_destination_for_${IN_DESTINATION}_${package}}") - set(export_args "") - if (IN_EXPORT) - set(export_args EXPORT "${IN_EXPORT}") - endif() - if(NOT ${install_path} STREQUAL "") + if(install_export) install( - ${args} - ${export_args} + EXPORT "${IN_EXPORT}-${package}" DESTINATION "${install_path}${IN_DESTINATION_SUFFIX}" + FILE "${IN_EXPORT}.cmake" COMPONENT "${IN_COMPONENT}-${package}") + else() + set(export_args "") + if (IN_EXPORT) + set(export_args EXPORT "${IN_EXPORT}-${package}") + endif() + if(NOT ${install_path} STREQUAL "") + install( + ${args} + ${export_args} + DESTINATION "${install_path}${IN_DESTINATION_SUFFIX}" + COMPONENT "${IN_COMPONENT}-${package}") + endif() endif() endforeach() endif() From 9af6326e1b6e6e629d4613d9071fd8ea4ea76061 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 12:39:27 -0700 Subject: [PATCH 026/128] Make find_package work --- bindings/c/CMakeLists.txt | 22 +++++++++++++++++++--- cmake/Config.cmake.in | 1 + 2 files changed, 20 insertions(+), 3 deletions(-) create mode 100644 cmake/Config.cmake.in diff --git a/bindings/c/CMakeLists.txt b/bindings/c/CMakeLists.txt index 7aa5dccb40..1dc4d04b56 100644 --- a/bindings/c/CMakeLists.txt +++ b/bindings/c/CMakeLists.txt @@ -65,9 +65,20 @@ endif() # TODO: re-enable once the old vcxproj-based build system is removed. #generate_export_header(fdb_c EXPORT_MACRO_NAME "DLLEXPORT" # EXPORT_FILE_NAME ${CMAKE_CURRENT_BINARY_DIR}/foundationdb/fdb_c_export.h) + +set(targets_export_name "FoundationDB-Client") +set(generated_dir "${CMAKE_CURRENT_BINARY_DIR}/generated") +set(version_config "${generated_dir}/${targets_export_name}ConfigVersion.cmake") +set(project_config "${generated_dir}/${targets_export_name}Config.cmake") +include(CMakePackageConfigHelpers) +write_basic_package_version_file( + "${version_config}" VERSION ${GENERIC_LIB_VERSION} COMPATIBILITY AnyNewerVersion +) +configure_file("${PROJECT_SOURCE_DIR}/cmake/Config.cmake.in" "${project_config}" @ONLY) + fdb_install( TARGETS fdb_c - EXPORT FoundationDB-Client + EXPORT ${targets_export_name} DESTINATION lib COMPONENT clients) fdb_install( @@ -78,7 +89,12 @@ fdb_install( DESTINATION_SUFFIX /foundationdb COMPONENT clients) fdb_install( - EXPORT FoundationDB-Client + FILES "${project_config}" "${version_config}" DESTINATION lib - DESTINATION_SUFFIX /cmake + DESTINATION_SUFFIX "/cmake/${targets_export_name}" + COMPONENT clients) +fdb_install( + EXPORT ${targets_export_name} + DESTINATION lib + DESTINATION_SUFFIX "/cmake/${targets_export_name}" COMPONENT clients) diff --git a/cmake/Config.cmake.in b/cmake/Config.cmake.in new file mode 100644 index 0000000000..6e9256eea8 --- /dev/null +++ b/cmake/Config.cmake.in @@ -0,0 +1 @@ +include("${CMAKE_CURRENT_LIST_DIR}/@targets_export_name@.cmake") From f5e7b8b1655126c925496f4fe502f26debf4c21c Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 14:13:07 -0700 Subject: [PATCH 027/128] Support pkg-config --- bindings/c/CMakeLists.txt | 5 +++++ cmake/InstallLayout.cmake | 21 +++++++++++++++++++++ cmake/foundationdb-client.pc.in | 6 ++++++ 3 files changed, 32 insertions(+) create mode 100644 cmake/foundationdb-client.pc.in diff --git a/bindings/c/CMakeLists.txt b/bindings/c/CMakeLists.txt index 1dc4d04b56..615027a304 100644 --- a/bindings/c/CMakeLists.txt +++ b/bindings/c/CMakeLists.txt @@ -93,6 +93,11 @@ fdb_install( DESTINATION lib DESTINATION_SUFFIX "/cmake/${targets_export_name}" COMPONENT clients) +fdb_configure_and_install( + FILE "${PROJECT_SOURCE_DIR}/cmake/foundationdb-client.pc.in" + DESTINATION lib + DESTINATION_SUFFIX "/pkgconfig" + COMPONENT clients) fdb_install( EXPORT ${targets_export_name} DESTINATION lib diff --git a/cmake/InstallLayout.cmake b/cmake/InstallLayout.cmake index 0cb1cc3a12..28d789a073 100644 --- a/cmake/InstallLayout.cmake +++ b/cmake/InstallLayout.cmake @@ -122,6 +122,27 @@ set(install_destination_for_data_el6 "var/lib/foundationdb") set(install_destination_for_data_el7 "var/lib/foundationdb") set(install_destination_for_data_pm "") +set(generated_dir "${CMAKE_CURRENT_BINARY_DIR}/generated") +function(fdb_configure_and_install) + if(NOT WIN32 AND NOT OPEN_FOR_IDE) + set(one_value_options COMPONENT DESTINATION FILE DESTINATION_SUFFIX) + cmake_parse_arguments(IN "${options}" "${one_value_options}" "${multi_value_options}" "${ARGN}") + foreach(package tgz deb el6 el7 pm) + set(INCLUDE_DIR "${install_destination_for_include_${package}}") + set(LIB_DIR "${install_destination_for_lib_${package}}") + set(install_path "${install_destination_for_${IN_DESTINATION}_${package}}") + string(REGEX REPLACE "\.in$" "" name "${IN_FILE}") + get_filename_component(name "${name}" NAME) + set(generated_file_name "${generated_dir}/${package}/${name}") + configure_file("${IN_FILE}" "${generated_file_name}" @ONLY) + install( + FILES "${generated_file_name}" + DESTINATION "${install_path}${IN_DESTINATION_SUFFIX}" + COMPONENT "${IN_COMPONENT}-${package}") + endforeach() + endif() +endfunction() + function(fdb_install) if(NOT WIN32 AND NOT OPEN_FOR_IDE) set(one_value_options COMPONENT DESTINATION EXPORT DESTINATION_SUFFIX) diff --git a/cmake/foundationdb-client.pc.in b/cmake/foundationdb-client.pc.in new file mode 100644 index 0000000000..738a52c7c6 --- /dev/null +++ b/cmake/foundationdb-client.pc.in @@ -0,0 +1,6 @@ +Name: foundationdb-client +Description: FoundationDB c client +Version: @PROJECT_VERSION@ + +Libs: -L/@LIB_DIR@ -lfdb_c +Cflags: -I/@INCLUDE_DIR@ From 2544f2d1a424aa053bd731d68f783157e4b46cc6 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 14:52:09 -0700 Subject: [PATCH 028/128] Add package tests --- build/cmake/package_tester/fdb_c_app/CMakeLists.txt | 5 +++++ build/cmake/package_tester/fdb_c_app/app.c | 6 ++++++ build/cmake/package_tester/modules/tests.sh | 11 +++++++++++ 3 files changed, 22 insertions(+) create mode 100644 build/cmake/package_tester/fdb_c_app/CMakeLists.txt create mode 100644 build/cmake/package_tester/fdb_c_app/app.c diff --git a/build/cmake/package_tester/fdb_c_app/CMakeLists.txt b/build/cmake/package_tester/fdb_c_app/CMakeLists.txt new file mode 100644 index 0000000000..60ed25f0ca --- /dev/null +++ b/build/cmake/package_tester/fdb_c_app/CMakeLists.txt @@ -0,0 +1,5 @@ +cmake_minimum_required(VERSION 2.8.0) +project(fdb_c_app C) +find_package(FoundationDB-Client REQUIRED) +add_executable(app app.c) +target_link_libraries(app PRIVATE fdb_c) diff --git a/build/cmake/package_tester/fdb_c_app/app.c b/build/cmake/package_tester/fdb_c_app/app.c new file mode 100644 index 0000000000..6d276abad3 --- /dev/null +++ b/build/cmake/package_tester/fdb_c_app/app.c @@ -0,0 +1,6 @@ +#define FDB_API_VERSION 610 +#include + +int main(int argc, char* argv[]) { + return 0; +} diff --git a/build/cmake/package_tester/modules/tests.sh b/build/cmake/package_tester/modules/tests.sh index 77383cfa3b..55300319ff 100644 --- a/build/cmake/package_tester/modules/tests.sh +++ b/build/cmake/package_tester/modules/tests.sh @@ -67,6 +67,17 @@ then popd python -c 'import fdb; fdb.api_version(610)' successOr "Loading python bindings failed" + + # Test cmake and pkg-config integration: https://github.com/apple/foundationdb/issues/1483 + cd /foundationdb/build/cmake/package_tester/fdb_c_app + rm -rf build + mkdir build + cd build + cmake .. && make + successOr "FoundationDB-Client cmake integration failed" + + cc ../app.c `pkg-config --libs --cflags foundationdb-client` + successOr "FoundationDB-Client pkg-config integration failed" } keep_config() { From c9dc2f98f00ab16b34518987a39aa50755abe1a4 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Thu, 23 May 2019 15:00:32 -0700 Subject: [PATCH 029/128] Make pkg-config look more like example in man page --- cmake/foundationdb-client.pc.in | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/cmake/foundationdb-client.pc.in b/cmake/foundationdb-client.pc.in index 738a52c7c6..735dd3abb0 100644 --- a/cmake/foundationdb-client.pc.in +++ b/cmake/foundationdb-client.pc.in @@ -1,6 +1,9 @@ +libdir=/@LIB_DIR@ +includedir=/@INCLUDE_DIR@ + Name: foundationdb-client Description: FoundationDB c client Version: @PROJECT_VERSION@ -Libs: -L/@LIB_DIR@ -lfdb_c -Cflags: -I/@INCLUDE_DIR@ +Libs: -L${libdir} -lfdb_c +Cflags: -I${includedir} From 49f3f589dc735ebb6075039a62914c6fee28f846 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Fri, 24 May 2019 11:30:37 -0700 Subject: [PATCH 030/128] Don't build in source tree for package test --- build/cmake/package_tester/modules/tests.sh | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/build/cmake/package_tester/modules/tests.sh b/build/cmake/package_tester/modules/tests.sh index 55300319ff..dbef5cf423 100644 --- a/build/cmake/package_tester/modules/tests.sh +++ b/build/cmake/package_tester/modules/tests.sh @@ -69,15 +69,14 @@ then successOr "Loading python bindings failed" # Test cmake and pkg-config integration: https://github.com/apple/foundationdb/issues/1483 - cd /foundationdb/build/cmake/package_tester/fdb_c_app - rm -rf build - mkdir build - cd build - cmake .. && make + rm -rf build-fdb_c_app + mkdir build-fdb_c_app + pushd build-fdb_c_app + cmake /foundationdb/build/cmake/package_tester/fdb_c_app && make successOr "FoundationDB-Client cmake integration failed" - - cc ../app.c `pkg-config --libs --cflags foundationdb-client` + cc /foundationdb/build/cmake/package_tester/fdb_c_app/app.c `pkg-config --libs --cflags foundationdb-client` successOr "FoundationDB-Client pkg-config integration failed" + popd } keep_config() { From 578ae0d59567b39c02313a1f29d56e261986d216 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Tue, 28 May 2019 11:48:13 -0700 Subject: [PATCH 031/128] Call function from libfdb_c --- build/cmake/package_tester/fdb_c_app/app.c | 1 + 1 file changed, 1 insertion(+) diff --git a/build/cmake/package_tester/fdb_c_app/app.c b/build/cmake/package_tester/fdb_c_app/app.c index 6d276abad3..549bd935d3 100644 --- a/build/cmake/package_tester/fdb_c_app/app.c +++ b/build/cmake/package_tester/fdb_c_app/app.c @@ -2,5 +2,6 @@ #include int main(int argc, char* argv[]) { + fdb_select_api_version(610); return 0; } From 7bb262d21a666372d90bbb41757073b39d2a1cf8 Mon Sep 17 00:00:00 2001 From: Andrew Noyes Date: Sun, 9 Jun 2019 15:20:56 -0700 Subject: [PATCH 032/128] Install build tools before running fdb_c_app test --- build/cmake/package_tester/modules/deb.sh | 4 ++++ build/cmake/package_tester/modules/rpm.sh | 4 ++++ build/cmake/package_tester/modules/tests.sh | 1 + 3 files changed, 9 insertions(+) diff --git a/build/cmake/package_tester/modules/deb.sh b/build/cmake/package_tester/modules/deb.sh index 575a2388de..fcef96805a 100644 --- a/build/cmake/package_tester/modules/deb.sh +++ b/build/cmake/package_tester/modules/deb.sh @@ -6,6 +6,10 @@ then source ${source_dir}/modules/util.sh + install_build_tools() { + apt-get -y install cmake gcc + } + install() { local __res=0 enterfun diff --git a/build/cmake/package_tester/modules/rpm.sh b/build/cmake/package_tester/modules/rpm.sh index bd3348d102..866bde558e 100644 --- a/build/cmake/package_tester/modules/rpm.sh +++ b/build/cmake/package_tester/modules/rpm.sh @@ -8,6 +8,10 @@ then conf_save_extension=".rpmsave" + install_build_tools() { + yum -y install cmake gcc + } + install() { local __res=0 enterfun diff --git a/build/cmake/package_tester/modules/tests.sh b/build/cmake/package_tester/modules/tests.sh index dbef5cf423..5dd39b21e0 100644 --- a/build/cmake/package_tester/modules/tests.sh +++ b/build/cmake/package_tester/modules/tests.sh @@ -69,6 +69,7 @@ then successOr "Loading python bindings failed" # Test cmake and pkg-config integration: https://github.com/apple/foundationdb/issues/1483 + install_build_tools rm -rf build-fdb_c_app mkdir build-fdb_c_app pushd build-fdb_c_app From 5eb833759e391818d1f1280e31ec3041139d966a Mon Sep 17 00:00:00 2001 From: Balachandar Namasivayam Date: Fri, 14 Jun 2019 11:35:38 -0700 Subject: [PATCH 033/128] Extend RebootRequest API to include time to suspend the process before reboot. This is intended to be used for testing purposes to simulate failures. --- fdbbackup/backup.actor.cpp | 1 - fdbclient/ClientWorkerInterface.h | 6 +- fdbserver/CMakeLists.txt | 1 + fdbserver/fdbserver.vcxproj | 1 + fdbserver/fdbserver.vcxproj.filters | 3 + fdbserver/worker.actor.cpp | 7 ++ .../workloads/SuspendProcesses.actor.cpp | 68 +++++++++++++++++++ flow/Error.cpp | 2 - flow/Trace.h | 1 + 9 files changed, 85 insertions(+), 5 deletions(-) create mode 100644 fdbserver/workloads/SuspendProcesses.actor.cpp diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index 89754cab18..e90c3eda29 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -779,7 +779,6 @@ const KeyRef exeRestore = LiteralStringRef("fdbrestore"); const KeyRef exeDatabaseAgent = LiteralStringRef("dr_agent"); const KeyRef exeDatabaseBackup = LiteralStringRef("fdbdr"); -extern void flushTraceFileVoid(); extern const char* getHGVersion(); #ifdef _WIN32 diff --git a/fdbclient/ClientWorkerInterface.h b/fdbclient/ClientWorkerInterface.h index 4e3efde268..d5f83b86ec 100644 --- a/fdbclient/ClientWorkerInterface.h +++ b/fdbclient/ClientWorkerInterface.h @@ -49,12 +49,14 @@ struct RebootRequest { constexpr static FileIdentifier file_identifier = 11913957; bool deleteData; bool checkData; + uint32_t waitForDuration; - explicit RebootRequest(bool deleteData = false, bool checkData = false) : deleteData(deleteData), checkData(checkData) {} + explicit RebootRequest(bool deleteData = false, bool checkData = false, uint32_t waitForDuration = 0) + : deleteData(deleteData), checkData(checkData), waitForDuration(waitForDuration) {} template void serialize(Ar& ar) { - serializer(ar, deleteData, checkData); + serializer(ar, deleteData, checkData, waitForDuration); } }; diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index bf264c030c..bf61207a16 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -107,6 +107,7 @@ set(FDBSERVER_SRCS workloads/ChangeConfig.actor.cpp workloads/ClientTransactionProfileCorrectness.actor.cpp workloads/TriggerRecovery.actor.cpp + workloads/SuspendProcesses.actor.cpp workloads/CommitBugCheck.actor.cpp workloads/ConfigureDatabase.actor.cpp workloads/ConflictRange.actor.cpp diff --git a/fdbserver/fdbserver.vcxproj b/fdbserver/fdbserver.vcxproj index da4a62d62f..44e752cdb3 100644 --- a/fdbserver/fdbserver.vcxproj +++ b/fdbserver/fdbserver.vcxproj @@ -107,6 +107,7 @@ + diff --git a/fdbserver/fdbserver.vcxproj.filters b/fdbserver/fdbserver.vcxproj.filters index 78e9d35e16..23b68d2c38 100644 --- a/fdbserver/fdbserver.vcxproj.filters +++ b/fdbserver/fdbserver.vcxproj.filters @@ -234,6 +234,9 @@ workloads + + workloads + workloads diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index aa2b37b404..97428b9802 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -866,6 +866,13 @@ ACTOR Future workerServer( when( RebootRequest req = waitNext( interf.clientInterface.reboot.getFuture() ) ) { state RebootRequest rebootReq = req; + if(req.waitForDuration) { + TraceEvent("RebootRequestSuspendingProcess").detail("Duration", req.waitForDuration); + flushTraceFileVoid(); + setProfilingEnabled(0); + g_network->stop(); + threadSleep(req.waitForDuration); + } if(rebootReq.checkData) { Reference checkFile = wait( IAsyncFileSystem::filesystem()->open( joinPath(folder, validationFilename), IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_READWRITE, 0600 ) ); wait( checkFile->sync() ); diff --git a/fdbserver/workloads/SuspendProcesses.actor.cpp b/fdbserver/workloads/SuspendProcesses.actor.cpp new file mode 100644 index 0000000000..4581f02fa1 --- /dev/null +++ b/fdbserver/workloads/SuspendProcesses.actor.cpp @@ -0,0 +1,68 @@ +#include + +#include "fdbserver/workloads/workloads.actor.h" +#include "fdbserver/ServerDBInfo.h" +#include "fdbclient/Status.h" +#include "fdbclient/StatusClient.h" +#include "fdbclient/ManagementAPI.actor.h" +#include "fdbclient/RunTransaction.actor.h" +#include "flow/actorcompiler.h" // has to be last include + +struct SuspendProcessesWorkload : TestWorkload { + std::vector prefixSuspendProcesses; + double suspendTimeDuration; + double waitTimeDuration; + + SuspendProcessesWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { + prefixSuspendProcesses = getOption(options, LiteralStringRef("prefixesSuspendProcesses"), std::vector()); + waitTimeDuration = getOption(options, LiteralStringRef("waitTimeDuration"), 0); + suspendTimeDuration = getOption(options, LiteralStringRef("suspendTimeDuration"), 0); + } + + virtual std::string description() { return "SuspendProcesses"; } + + virtual Future setup(Database const& cx) { return Void(); } + + ACTOR Future _start(Database cx, SuspendProcessesWorkload* self) { + wait(delay(self->waitTimeDuration)); + state ReadYourWritesTransaction tr(cx); + loop { + try { + tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr.setOption(FDBTransactionOptions::LOCK_AWARE); + Standalone kvs = wait(tr.getRange( + KeyRangeRef(LiteralStringRef("\xff\xff/worker_interfaces"), LiteralStringRef("\xff\xff\xff")), 1)); + std::vector> suspendProcessInterfaces; + for (auto it : kvs) { + auto ip_port = it.key.endsWith(LiteralStringRef(":tls")) + ? it.key.removeSuffix(LiteralStringRef(":tls")) + : it.key; + for (auto& killProcess : self->prefixSuspendProcesses) { + if (boost::starts_with(ip_port.toString().c_str(), killProcess.c_str())) { + suspendProcessInterfaces.push_back(it.value); + TraceEvent("SuspendProcessSelectedProcess").detail("IpPort", printable(ip_port)); + } + } + } + for (auto& interf : suspendProcessInterfaces) { + BinaryReader::fromStringRef(interf, IncludeVersion()) + .reboot.send(RebootRequest(false, false, self->suspendTimeDuration)); + } + return Void(); + } catch (Error& e) { + wait(tr.onError(e)); + } + } + } + + virtual Future start(Database const& cx) { + if (clientId != 0) return Void(); + return _start(cx, this); + } + + virtual Future check(Database const& cx) { return true; } + + virtual void getMetrics(vector& m) {} +}; + +WorkloadFactory SuspendProcessesWorkloadFactory("SuspendProcesses"); diff --git a/flow/Error.cpp b/flow/Error.cpp index e2d096b8dd..3edb81adf9 100644 --- a/flow/Error.cpp +++ b/flow/Error.cpp @@ -35,8 +35,6 @@ std::map& Error::errorCounts() { #include -extern void flushTraceFileVoid(); - Error Error::fromUnvalidatedCode(int code) { if (code < 0 || code > 30000) { Error e = Error::fromCode(error_code_unknown_error); diff --git a/flow/Trace.h b/flow/Trace.h index 6ab4a857c6..6ba7e0e7db 100644 --- a/flow/Trace.h +++ b/flow/Trace.h @@ -537,6 +537,7 @@ void openTraceFile(const NetworkAddress& na, uint64_t rollsize, uint64_t maxLogs void initTraceEventMetrics(); void closeTraceFile(); bool traceFileIsOpen(); +void flushTraceFileVoid(); // Changes the format of trace files. Returns false if the format is unrecognized. No longer safe to call after a call // to openTraceFile. From d4b7f9b60690546395acedde8a658661c93cf568 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Mon, 17 Jun 2019 18:55:49 -0700 Subject: [PATCH 034/128] Fixed some cmake, compile, and IDE warnings. --- fdbserver/VersionedBTree.actor.cpp | 20 ++++++++++---------- tests/CMakeLists.txt | 3 +++ 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 4d365049eb..15b17256e2 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -504,7 +504,7 @@ struct RedwoodRecordRef { Reader r(data); int intFieldSuffixLen = flags & INT_FIELD_SUFFIX_BITS; - int prefixLen = r.readVarInt(); + r.readVarInt(); // prefixlen int valueLen = (flags & HAS_VALUE) ? r.read() : 0; int keySuffixLen = (flags & HAS_KEY_SUFFIX) ? r.readVarInt() : 0; @@ -1562,7 +1562,6 @@ private: childEntries.push_back(entry); } - int oldPages = pages.size(); pages = buildPages(false, dbBegin, dbEnd, childEntries, 0, [=](){ return m_pager->newPageBuffer(); }, m_usablePageSizeOverride); debug_printf("Writing a new root level at version %" PRId64 " with %lu children across %lu pages\n", version, childEntries.size(), pages.size()); @@ -2063,8 +2062,7 @@ private: if(REDWOOD_DEBUG) { debug_printf("%s Subtree update results for root PageID=%u\n", context.c_str(), root); for(int i = 0; i < futureChildren.size(); ++i) { - const VersionedChildrenT &vc = futureChildren[i].get(); - debug_printf("%s subtree result %s\n", context.c_str(), toString(vc).c_str()); + debug_printf("%s subtree result %s\n", context.c_str(), toString(futureChildren[i].get()).c_str()); } } @@ -2410,7 +2408,7 @@ printf("\nCommitted: %s\n", self->counts.toString(true).c_str()); } Reference child = wait(self->pageCursor->getChild(self->pager, self->usablePageSizeOverride)); - bool success = forward ? child->cursor.moveFirst() : child->cursor.moveLast(); + forward ? child->cursor.moveFirst() : child->cursor.moveLast(); self->pageCursor = child; } @@ -3146,9 +3144,7 @@ ACTOR Future verify(VersionedBTree *btree, FutureStream vStream, } wait(success(vall) && success(vrange)); - int errors = vall.get() + vrange.get(); - - debug_printf("Verified through version %" PRId64 ", %d errors\n", v, errors); + debug_printf("Verified through version %" PRId64 ", %d errors\n", v, *pErrorCount); if(*pErrorCount != 0) break; @@ -3985,10 +3981,14 @@ TEST_CASE("!/redwood/performance/set") { // Avoid capturing this to freeze counter values int recs = records; int kvb = kvBytes; + + // Capturing invervalStart via this->intervalStart makes IDE's unhappy as they do not know about the actor state object + double *pIntervalStart = &intervalStart; + commit = map(btree->commit(), [=](Void result) { - double elapsed = timer() - intervalStart; + double elapsed = timer() - *pIntervalStart; printf("Committed %d kvBytes in %d records in %f seconds, %.2f MB/s\n", kvb, recs, elapsed, kvb / elapsed / 1e6); - intervalStart = timer(); + *pIntervalStart = timer(); return Void(); }); records = 0; diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index af7fe413c8..e569ac222c 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -60,6 +60,9 @@ add_fdb_test(TEST_FILES RandomRead.txt IGNORE) add_fdb_test(TEST_FILES RandomReadWrite.txt IGNORE) add_fdb_test(TEST_FILES ReadAbsent.txt IGNORE) add_fdb_test(TEST_FILES ReadHalfAbsent.txt IGNORE) +add_fdb_test(TEST_FILES RedwoodCorrectnessUnits.txt IGNORE) +add_fdb_test(TEST_FILES RedwoodCorrectnessBTree.txt IGNORE) +add_fdb_test(TEST_FILES fast/RedwoodCorrectnessBTree.txt IGNORE) add_fdb_test(TEST_FILES RedwoodCorrectness.txt IGNORE) add_fdb_test(TEST_FILES RedwoodPerfTests.txt IGNORE) add_fdb_test(TEST_FILES SlowTask.txt IGNORE) From 4832404c859fd6544b626eac887403b8763551b3 Mon Sep 17 00:00:00 2001 From: Balachandar Namasivayam Date: Tue, 18 Jun 2019 18:15:15 -0700 Subject: [PATCH 035/128] Make public address and listen address a comma separated list --- fdbserver/fdbserver.actor.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 4d7f58796e..302257e5d7 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -22,6 +22,7 @@ // a macro that makes boost interprocess break on Windows. #define BOOST_DATE_TIME_NO_LIB #include +#include #include "fdbrpc/simulator.h" #include "flow/DeterministicRandom.h" @@ -983,6 +984,7 @@ int main(int argc, char* argv[]) { } const char *sRole; Optional ti; + std::string argStr; switch (args.OptionId()) { case OPT_HELP: @@ -1050,10 +1052,12 @@ int main(int argc, char* argv[]) { } break; case OPT_PUBLICADDR: - publicAddressStrs.push_back(args.OptionArg()); + argStr = args.OptionArg(); + boost::split(publicAddressStrs, argStr, [](char c){return c == ',';}); break; case OPT_LISTEN: - listenAddressStrs.push_back(args.OptionArg()); + argStr = args.OptionArg(); + boost::split(listenAddressStrs, argStr, [](char c){return c == ',';}); break; case OPT_CONNFILE: connFile = args.OptionArg(); From ce24db3c53d0073fb6cbc8c38d34140c2ef0e9b9 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Wed, 19 Jun 2019 01:30:49 -0700 Subject: [PATCH 036/128] Fully consume parallelPeekMore results before switching back. --- fdbserver/LogSystemPeekCursor.actor.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 77e8d27625..40f9ef85e8 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -146,8 +146,12 @@ ACTOR Future serverPeekParallelGetMore( ILogSystem::ServerPeekCursor* self loop { state Version expectedBegin = self->messageVersion.version; try { - while(self->futureResults.size() < SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && self->interf->get().present()) { - self->futureResults.push_back( brokenPromiseToNever( self->interf->get().interf().peekMessages.getReply(TLogPeekRequest(self->messageVersion.version,self->tag,self->returnIfBlocked, self->onlySpilled, std::make_pair(self->randomID, self->sequence++)), taskID) ) ); + if (self->parallelGetMore || self->onlySpilled) { + while(self->futureResults.size() < SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && self->interf->get().present()) { + self->futureResults.push_back( brokenPromiseToNever( self->interf->get().interf().peekMessages.getReply(TLogPeekRequest(self->messageVersion.version,self->tag,self->returnIfBlocked, self->onlySpilled, std::make_pair(self->randomID, self->sequence++)), taskID) ) ); + } + } else if (self->futureResults.size() == 0) { + return Void(); } choose { @@ -235,7 +239,11 @@ Future ILogSystem::ServerPeekCursor::getMore(int taskID) { if( hasMessage() ) return Void(); if( !more.isValid() || more.isReady() ) { - more = (parallelGetMore || onlySpilled) ? serverPeekParallelGetMore(this, taskID) : serverPeekGetMore(this, taskID); + if (parallelGetMore || onlySpilled || futureResults.size()) { + more = serverPeekParallelGetMore(this, taskID); + } else { + more = serverPeekGetMore(this, taskID); + } } return more; } From 6e753dc614dac09bad5338137a2e58b1ba3a108f Mon Sep 17 00:00:00 2001 From: Vishesh Yadav Date: Wed, 19 Jun 2019 02:44:30 -0700 Subject: [PATCH 037/128] Fix setting enClientFailureMonitor global for client --- fdbclient/NativeAPI.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 5c97174b5f..2d61be40e2 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -1038,7 +1038,7 @@ void setupNetwork(uint64_t transportId, bool useMetrics) { networkOptions.logClientInfo = true; g_network = newNet2(false, useMetrics || networkOptions.traceDirectory.present(), networkOptions.useObjectSerializer); - FlowTransport::createInstance(transportId); + FlowTransport::createInstance(true, transportId); Net2FileSystem::newFileSystem(); initTLSOptions(); From 9c2257a0e50e81c7a144cfaab21f077e56b0deea Mon Sep 17 00:00:00 2001 From: Jingyu Zhou Date: Wed, 19 Jun 2019 07:40:54 -0700 Subject: [PATCH 038/128] Add transaction size option --- bindings/flow/tester/Tester.actor.cpp | 2 ++ bindings/go/src/fdb/generated.go | 28 ++++++++++++++++------ bindings/python/tests/tester.py | 1 + bindings/ruby/tests/tester.rb | 1 + documentation/sphinx/source/api-python.rst | 5 ++++ documentation/sphinx/source/api-ruby.rst | 5 ++++ fdbclient/DatabaseContext.h | 1 + fdbclient/Knobs.h | 2 +- fdbclient/NativeAPI.actor.cpp | 11 ++++++++- fdbclient/ReadYourWrites.h | 2 +- fdbclient/vexillographer/fdb.options | 8 ++++++- 11 files changed, 55 insertions(+), 11 deletions(-) diff --git a/bindings/flow/tester/Tester.actor.cpp b/bindings/flow/tester/Tester.actor.cpp index a20718b15d..fa309a59e8 100644 --- a/bindings/flow/tester/Tester.actor.cpp +++ b/bindings/flow/tester/Tester.actor.cpp @@ -1550,6 +1550,7 @@ struct UnitTestsFunc : InstructionFunc { const uint64_t retryLimit = 50; const uint64_t noRetryLimit = -1; const uint64_t maxRetryDelay = 100; + const uint64_t sizeLimit = 100000; data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_LOCATION_CACHE_SIZE, Optional(StringRef((const uint8_t*)&locationCacheSize, 8))); data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_MAX_WATCHES, Optional(StringRef((const uint8_t*)&maxWatches, 8))); @@ -1558,6 +1559,7 @@ struct UnitTestsFunc : InstructionFunc { data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_TIMEOUT, Optional(StringRef((const uint8_t*)&timeout, 8))); data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_TIMEOUT, Optional(StringRef((const uint8_t*)&noTimeout, 8))); data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_MAX_RETRY_DELAY, Optional(StringRef((const uint8_t*)&maxRetryDelay, 8))); + data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_SIZE_LIMIT, Optional(StringRef((const uint8_t*)&sizeLimit, 8))); data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_RETRY_LIMIT, Optional(StringRef((const uint8_t*)&retryLimit, 8))); data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_TRANSACTION_RETRY_LIMIT, Optional(StringRef((const uint8_t*)&noRetryLimit, 8))); data->db->setDatabaseOption(FDBDatabaseOption::FDB_DB_OPTION_SNAPSHOT_RYW_ENABLE); diff --git a/bindings/go/src/fdb/generated.go b/bindings/go/src/fdb/generated.go index e059e7da63..77599d407b 100644 --- a/bindings/go/src/fdb/generated.go +++ b/bindings/go/src/fdb/generated.go @@ -277,6 +277,13 @@ func (o DatabaseOptions) SetTransactionMaxRetryDelay(param int64) error { return o.setOpt(502, int64ToBytes(param)) } +// Set the maximum transaction size which, if exceeded, will cause the transaction to be cancelled. Default to 10,000,000 bytes. +// +// Parameter: value in bytes +func (o DatabaseOptions) SetTransactionSizeLimit(param int64) error { + return o.setOpt(503, int64ToBytes(param)) +} + // Snapshot read operations will see the results of writes done in the same transaction. This is the default behavior. func (o DatabaseOptions) SetSnapshotRywEnable() error { return o.setOpt(26, nil) @@ -402,6 +409,13 @@ func (o TransactionOptions) SetMaxRetryDelay(param int64) error { return o.setOpt(502, int64ToBytes(param)) } +// Set the maximum transaction size which, if exceeded, will cause the transaction to be cancelled. Valid parameter values are ``[32, 10,000,000]```. +// +// Parameter: value in bytes +func (o TransactionOptions) SetMaxTransactionSize(param int64) error { + return o.setOpt(503, int64ToBytes(param)) +} + // Snapshot read operations will see the results of writes done in the same transaction. This is the default behavior. func (o TransactionOptions) SetSnapshotRywEnable() error { return o.setOpt(600, nil) @@ -451,7 +465,7 @@ const ( // Infrequently used. The client has passed a specific row limit and wants // that many rows delivered in a single batch. Because of iterator operation // in client drivers make request batches transparent to the user, consider - // “WANT_ALL“ StreamingMode instead. A row limit must be specified if this + // ``WANT_ALL`` StreamingMode instead. A row limit must be specified if this // mode is used. StreamingModeExact StreamingMode = 1 @@ -568,15 +582,15 @@ type ErrorPredicate int const ( - // Returns “true“ if the error indicates the operations in the transactions - // should be retried because of transient error. + // Returns ``true`` if the error indicates the operations in the + // transactions should be retried because of transient error. ErrorPredicateRetryable ErrorPredicate = 50000 - // Returns “true“ if the error indicates the transaction may have succeeded, - // though not in a way the system can verify. + // Returns ``true`` if the error indicates the transaction may have + // succeeded, though not in a way the system can verify. ErrorPredicateMaybeCommitted ErrorPredicate = 50001 - // Returns “true“ if the error indicates the transaction has not committed, - // though in a way that can be retried. + // Returns ``true`` if the error indicates the transaction has not + // committed, though in a way that can be retried. ErrorPredicateRetryableNotCommitted ErrorPredicate = 50002 ) diff --git a/bindings/python/tests/tester.py b/bindings/python/tests/tester.py index 32ae2c01a3..3023cc5cb8 100644 --- a/bindings/python/tests/tester.py +++ b/bindings/python/tests/tester.py @@ -133,6 +133,7 @@ def test_db_options(db): db.options.set_transaction_timeout(0) db.options.set_transaction_timeout(0) db.options.set_transaction_max_retry_delay(100) + db.options.set_transaction_size_limit(100000) db.options.set_transaction_retry_limit(10) db.options.set_transaction_retry_limit(-1) db.options.set_snapshot_ryw_enable() diff --git a/bindings/ruby/tests/tester.rb b/bindings/ruby/tests/tester.rb index 54303075db..829ecf8a5f 100755 --- a/bindings/ruby/tests/tester.rb +++ b/bindings/ruby/tests/tester.rb @@ -459,6 +459,7 @@ class Tester @db.options.set_transaction_timeout(100000) @db.options.set_transaction_timeout(0) @db.options.set_transaction_max_retry_delay(100) + @db.options.set_transaction_size_limit(100000) @db.options.set_transaction_retry_limit(10) @db.options.set_transaction_retry_limit(-1) @db.options.set_snapshot_ryw_enable() diff --git a/documentation/sphinx/source/api-python.rst b/documentation/sphinx/source/api-python.rst index 2488ab6039..431b6f0393 100644 --- a/documentation/sphinx/source/api-python.rst +++ b/documentation/sphinx/source/api-python.rst @@ -24,6 +24,7 @@ .. |retry-limit-database-option| replace:: :func:`Database.options.set_transaction_retry_limit` .. |timeout-database-option| replace:: :func:`Database.options.set_transaction_timeout` .. |max-retry-delay-database-option| replace:: :func:`Database.options.set_transaction_max_retry_delay` +.. |transaction-size-limit-database-option| replace:: :func:`Database.options.set_transaction_size_limit` .. |snapshot-ryw-enable-database-option| replace:: :func:`Database.options.set_snapshot_ryw_enable` .. |snapshot-ryw-disable-database-option| replace:: :func:`Database.options.set_snapshot_ryw_disable` .. |future-type-string| replace:: a :ref:`future ` @@ -378,6 +379,10 @@ Database options |option-db-tr-max-retry-delay-blurb| +.. method:: Database.options.set_transaction_size_limit(size_limit) + + |option-db-tr-size-limit-blurb| + .. method:: Database.options.set_snapshot_ryw_enable() |option-db-snapshot-ryw-enable-blurb| diff --git a/documentation/sphinx/source/api-ruby.rst b/documentation/sphinx/source/api-ruby.rst index 2389c911e5..8ebdecdbab 100644 --- a/documentation/sphinx/source/api-ruby.rst +++ b/documentation/sphinx/source/api-ruby.rst @@ -29,6 +29,7 @@ .. |retry-limit-transaction-option| replace:: :meth:`Transaction.options.set_retry_limit` .. |timeout-transaction-option| replace:: :meth:`Transaction.options.set_timeout` .. |max-retry-delay-transaction-option| replace:: :meth:`Transaction.options.set_max_retry_delay` +.. |size-limit-transaction-option| replace:: :meth:`Transaction.options.set_transaction_size_limit` .. |snapshot-ryw-enable-transaction-option| replace:: :meth:`Transaction.options.set_snapshot_ryw_enable` .. |snapshot-ryw-disable-transaction-option| replace:: :meth:`Transaction.options.set_snapshot_ryw_disable` .. |lazy-iterator-object| replace:: :class:`Enumerator` @@ -374,6 +375,10 @@ Database options |option-db-tr-max-retry-delay-blurb| +.. method:: Database.options.set_transaction_size_limit(size_limit) -> nil + + |option-db-tr-size-limit-blurb| + .. method:: Database.options.set_snapshot_ryw_enable() -> nil |option-db-snapshot-ryw-enable-blurb| diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 0245c2abdb..f5f0ab0e94 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -157,6 +157,7 @@ public: double transactionTimeout; int transactionMaxRetries; double transactionMaxBackoff; + int transactionMaxBytes; int snapshotRywEnabled; Future logger; diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index cd4be9c339..5a5d34adf6 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -51,7 +51,7 @@ public: double RESOURCE_CONSTRAINED_MAX_BACKOFF; int PROXY_COMMIT_OVERHEAD_BYTES; - int64_t TRANSACTION_SIZE_LIMIT; + int TRANSACTION_SIZE_LIMIT; int64_t KEY_SIZE_LIMIT; int64_t SYSTEM_KEY_SIZE_LIMIT; int64_t VALUE_SIZE_LIMIT; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index e6d708aefe..fe904c5063 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -522,6 +522,7 @@ DatabaseContext::DatabaseContext( maxOutstandingWatches = CLIENT_KNOBS->DEFAULT_MAX_OUTSTANDING_WATCHES; transactionMaxBackoff = CLIENT_KNOBS->FAILURE_MAX_DELAY; + transactionMaxBytes = CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT; snapshotRywEnabled = apiVersionAtLeast(300) ? 1 : 0; logger = databaseLogger( this ); @@ -747,6 +748,10 @@ void DatabaseContext::setOption( FDBDatabaseOptions::Option option, Optional::max()) / 1000.0; break; + case FDBDatabaseOptions::TRANSACTION_SIZE_LIMIT: + validateOptionValue(value, true); + transactionMaxBytes = extractIntOption(value, 32, CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT); + break; case FDBDatabaseOptions::SNAPSHOT_RYW_ENABLE: validateOptionValue(value, false); snapshotRywEnabled++; @@ -2398,6 +2403,7 @@ double Transaction::getBackoff(int errCode) { TransactionOptions::TransactionOptions(Database const& cx) { maxBackoff = cx->transactionMaxBackoff; + customTransactionSizeLimit = cx->transactionMaxBytes; reset(cx); if (BUGGIFY) { commitOnFirstProxy = true; @@ -2411,6 +2417,7 @@ TransactionOptions::TransactionOptions(Database const& cx) { TransactionOptions::TransactionOptions() { memset(this, 0, sizeof(*this)); maxBackoff = CLIENT_KNOBS->DEFAULT_MAX_BACKOFF; + customTransactionSizeLimit = CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT; } void TransactionOptions::reset(Database const& cx) { @@ -2418,6 +2425,7 @@ void TransactionOptions::reset(Database const& cx) { double oldMaxRetries = maxRetries; memset(this, 0, sizeof(*this)); maxBackoff = cx->apiVersionAtLeast(610) ? oldMaxBackoff : cx->transactionMaxBackoff; + customTransactionSizeLimit = cx->transactionMaxBytes; maxRetries = oldMaxRetries; lockAware = cx->lockAware; } @@ -2753,8 +2761,9 @@ Future Transaction::commitMutations() { transactionSize = tr.transaction.mutations.expectedSize(); // Old API versions didn't account for conflict ranges when determining whether to throw transaction_too_large } - if (transactionSize > (options.customTransactionSizeLimit == 0 ? (uint64_t)CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT : (uint64_t)options.customTransactionSizeLimit)) + if (transactionSize > options.customTransactionSizeLimit) { return transaction_too_large(); + } if( !readVersion.isValid() ) getReadVersion( GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY ); // sets up readVersion field. We had no reads, so no need for (expensive) full causal consistency. diff --git a/fdbclient/ReadYourWrites.h b/fdbclient/ReadYourWrites.h index c57b59a42c..c5d4e0fafc 100644 --- a/fdbclient/ReadYourWrites.h +++ b/fdbclient/ReadYourWrites.h @@ -126,7 +126,7 @@ public: void getWriteConflicts( KeyRangeMap *result ); - Database getDatabase() { + Database getDatabase() const { return tr.getDatabase(); } private: diff --git a/fdbclient/vexillographer/fdb.options b/fdbclient/vexillographer/fdb.options index a84ba589dd..eab1b81755 100644 --- a/fdbclient/vexillographer/fdb.options +++ b/fdbclient/vexillographer/fdb.options @@ -145,6 +145,9 @@ description is not currently required but encouraged.