summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorHenrik Edin <henrik.edin@mongodb.com>2020-06-04 15:03:23 -0400
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2020-06-23 00:50:44 +0000
commitde515ed57a4cec1b2748618d5b5e62217ac9103a (patch)
tree981851b231e02275160e294b8fcd8ce8c0b5713b
parente94ca2a3bc234c8f340330217d89da6e73d1f026 (diff)
downloadmongo-de515ed57a4cec1b2748618d5b5e62217ac9103a.tar.gz
SERVER-48626 Change biggie index format, it can now properly handle unindexing of unique indexes where duplicates are allowed
Unique indexes now have a binary representation that allowes for duplicates, and there's no fallback to embedding the record id in the keys that causes issues. Unique and standard indexes have separate formats, similar to WT.
-rw-r--r--src/mongo/db/storage/biggie/biggie_kv_engine.cpp7
-rw-r--r--src/mongo/db/storage/biggie/biggie_sorted_impl.cpp1868
-rw-r--r--src/mongo/db/storage/biggie/biggie_sorted_impl.h210
-rw-r--r--src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp4
4 files changed, 1297 insertions, 792 deletions
diff --git a/src/mongo/db/storage/biggie/biggie_kv_engine.cpp b/src/mongo/db/storage/biggie/biggie_kv_engine.cpp
index 58e16b2c227..52ff2400224 100644
--- a/src/mongo/db/storage/biggie/biggie_kv_engine.cpp
+++ b/src/mongo/db/storage/biggie/biggie_kv_engine.cpp
@@ -119,7 +119,10 @@ std::unique_ptr<mongo::SortedDataInterface> KVEngine::getSortedDataInterface(
stdx::lock_guard lock(_identsLock);
_idents[ident.toString()] = false;
}
- return std::make_unique<SortedDataInterface>(opCtx, ident, desc);
+ if (desc->unique())
+ return std::make_unique<SortedDataInterfaceUnique>(opCtx, ident, desc);
+ else
+ return std::make_unique<SortedDataInterfaceStandard>(opCtx, ident, desc);
}
Status KVEngine::dropIdent(OperationContext* opCtx, mongo::RecoveryUnit* ru, StringData ident) {
@@ -137,7 +140,7 @@ Status KVEngine::dropIdent(OperationContext* opCtx, mongo::RecoveryUnit* ru, Str
checked_cast<RecordStore*>(rs.get())->truncateWithoutUpdatingCount(ru).getStatus();
} else { // ident is SortedDataInterface.
auto sdi =
- std::make_unique<SortedDataInterface>(Ordering::make(BSONObj()), true, ident);
+ std::make_unique<SortedDataInterfaceUnique>(Ordering::make(BSONObj()), ident);
dropStatus = sdi->truncate(ru);
}
lock.lock();
diff --git a/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp b/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp
index d5150f064c3..fef5c046eca 100644
--- a/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp
+++ b/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp
@@ -31,6 +31,7 @@
#include "mongo/platform/basic.h"
+#include <boost/iterator/iterator_facade.hpp>
#include <cstring>
#include <memory>
#include <string>
@@ -54,525 +55,477 @@ namespace mongo {
namespace biggie {
namespace {
-const Ordering allAscending = Ordering::make(BSONObj());
-
-// This just checks to see if the field names are empty or not.
-bool hasFieldNames(const BSONObj& obj) {
- BSONForEach(e, obj) {
- if (e.fieldName()[0])
- return true;
+// Helper to interpret index data buffer
+class IndexDataEntry {
+public:
+ IndexDataEntry() : _buffer(nullptr) {}
+ IndexDataEntry(const uint8_t* buffer);
+ IndexDataEntry(const std::string& indexDataEntry);
+
+ static std::string create(RecordId loc, const KeyString::TypeBits& typeBits);
+
+ const uint8_t* buffer() const;
+ size_t size() const; // returns buffer size
+ RecordId loc() const;
+ KeyString::TypeBits typeBits() const;
+
+private:
+ const uint8_t* _buffer;
+};
+
+// Forward iterator for IndexDataEntry with contigous memory layout
+class IndexDataEntryIterator : public boost::iterator_facade<IndexDataEntryIterator,
+ IndexDataEntry const,
+ boost::forward_traversal_tag> {
+public:
+ IndexDataEntryIterator() = default;
+ IndexDataEntryIterator(const uint8_t* entryData);
+
+private:
+ friend class boost::iterator_core_access;
+
+ void increment();
+ bool equal(IndexDataEntryIterator const& other) const;
+ const IndexDataEntry& dereference() const;
+
+ IndexDataEntry _entry;
+};
+
+// Helper to interpret index data buffer for unique index
+class UniqueIndexData {
+public:
+ UniqueIndexData() : _size(0), _begin(nullptr), _end(nullptr) {}
+ UniqueIndexData(const std::string& indexData) {
+ std::memcpy(&_size, indexData.data(), sizeof(uint64_t));
+ _begin = reinterpret_cast<const uint8_t*>(indexData.data() + sizeof(uint64_t));
+ _end = reinterpret_cast<const uint8_t*>(indexData.data() + indexData.size());
}
- return false;
-}
-// This function converts a key and an ordering to a KeyString::Builder.
-std::unique_ptr<KeyString::Builder> keyToKeyStringBuilder(const BSONObj& key, Ordering order) {
- KeyString::Version version = KeyString::Version::V1;
- std::unique_ptr<KeyString::Builder> retKs =
- std::make_unique<KeyString::Builder>(version, key, order);
- return retKs;
-}
+ using const_iterator = IndexDataEntryIterator;
-void prefixKeyString(KeyString::Builder* keyString,
- const RecordId& loc,
- std::string prefixToUse,
- bool isUnique) {
- BSONObjBuilder b;
- b.append("", prefixToUse); // prefix
- b.append("", std::string(keyString->getBuffer(), keyString->getSize())); // key
-
- if (isUnique)
- keyString->resetToKey(b.obj(), allAscending);
- else
- keyString->resetToKey(b.obj(), allAscending, loc);
-}
-
-std::string createKeyString(const BSONObj& key,
- const RecordId& loc,
- std::string prefixToUse,
- Ordering order,
- bool isUnique) {
- KeyString::Version version = KeyString::Version::V1;
- KeyString::Builder ks(version, key, order);
+ size_t size() const {
+ return _size;
+ }
+ bool empty() const {
+ return _size == 0;
+ }
+ const_iterator begin() const {
+ return IndexDataEntryIterator(_begin);
+ }
+ const_iterator end() const {
+ return IndexDataEntryIterator(_end);
+ }
+ const_iterator lower_bound(RecordId loc) const;
+ const_iterator upper_bound(RecordId loc) const;
- prefixKeyString(&ks, loc, prefixToUse, isUnique);
- return std::string(ks.getBuffer(), ks.getSize());
-}
+ // Creates a new UniqueIndexData buffer containing an additional item. Returns boost::none if
+ // entry already exists.
+ boost::optional<std::string> add(RecordId loc, const KeyString::TypeBits& typeBits);
-std::string createKeyString(const KeyString::Value& keyString,
- const size_t size,
- const RecordId& loc,
- std::string prefixToUse,
- bool isUnique) {
- KeyString::Builder ks(KeyString::Version::V1);
- ks.resetFromBuffer(keyString.getBuffer(), size);
- prefixKeyString(&ks, loc, prefixToUse, isUnique);
- return std::string(ks.getBuffer(), ks.getSize());
-}
+ // Creates a new UniqueIndexData buffer with item with RecordId removed. Returns boost::none if
+ // entry did not exist.
+ boost::optional<std::string> remove(RecordId loc);
-bool keysAreIdentical(std::string ks1, std::string ks2, bool isUnique) {
- size_t size1 =
- isUnique ? ks1.length() : KeyString::sizeWithoutRecordIdAtEnd(ks1.c_str(), ks1.length());
- size_t size2 =
- isUnique ? ks2.length() : KeyString::sizeWithoutRecordIdAtEnd(ks2.c_str(), ks2.length());
+private:
+ size_t _memoryUsage() const;
- if (size1 != size2)
- return false;
- return !ks1.compare(0, size2, ks2);
-}
+ size_t _size;
+ const uint8_t* _begin;
+ const uint8_t* _end;
+};
-/**
- * This function converts a std::string KeyString into a Key. We don't need to store the
- * typebits for the outer key string (the one consisting of the prefix, the key, and the recordId)
- * since those will not be used. However, we do need to store the typebits for the internal
- * keystring (made from the key itself), as those typebits are potentially important.
- *
- * The data which is serialized as a byte array, has the following structure:
- * [RecordId][TypeBits of internal keystring]
- */
-BSONObj getKeyFromKeyString(const std::string& keyString,
- const std::string& data,
- const Ordering& order) {
- std::string typeBitsString(data.length() - sizeof(int64_t), '\0');
- std::memcpy(&typeBitsString[0], data.data() + sizeof(int64_t), data.length() - sizeof(int64_t));
+IndexDataEntry::IndexDataEntry(const uint8_t* buffer) : _buffer(buffer) {}
+IndexDataEntry::IndexDataEntry(const std::string& indexDataEntry)
+ : _buffer(reinterpret_cast<const uint8_t*>(indexDataEntry.data())) {}
- BufReader brTbInternal(typeBitsString.c_str(), typeBitsString.length());
+std::string IndexDataEntry::create(RecordId loc, const KeyString::TypeBits& typeBits) {
+ uint64_t repr = loc.repr();
+ uint64_t typebitsSize = typeBits.getSize();
+ std::string output(sizeof(loc) + sizeof(typebitsSize) + typebitsSize, '\0');
- KeyString::Version version = KeyString::Version::V1;
- KeyString::TypeBits tbInternal = KeyString::TypeBits(version);
+ // RecordId
+ std::memcpy(output.data(), &repr, sizeof(repr));
- tbInternal.resetFromBuffer(&brTbInternal);
+ // TypeBits size
+ std::memcpy(output.data() + sizeof(repr), &typebitsSize, sizeof(typebitsSize));
- KeyString::TypeBits tbOuter = KeyString::TypeBits(version);
- BSONObj bsonObj =
- KeyString::toBsonSafe(keyString.c_str(), keyString.length(), allAscending, tbOuter);
+ // TypeBits data
+ std::memcpy(
+ output.data() + sizeof(repr) + sizeof(typebitsSize), typeBits.getBuffer(), typebitsSize);
- SharedBuffer sb;
- auto it = BSONObjIterator(bsonObj);
- ++it; // We want the second part
- KeyString::Builder ks(version);
- ks.resetFromBuffer((*it).valuestr(), (*it).valuestrsize());
+ return output;
+}
- BSONObj originalKey = KeyString::toBsonSafe(ks.getBuffer(), ks.getSize(), order, tbInternal);
+const uint8_t* IndexDataEntry::buffer() const {
+ return _buffer;
+}
- sb = SharedBuffer::allocate(originalKey.objsize());
- std::memcpy(sb.get(), originalKey.objdata(), originalKey.objsize());
+size_t IndexDataEntry::size() const {
+ uint64_t typeBitsSize;
+ std::memcpy(&typeBitsSize, _buffer + sizeof(uint64_t), sizeof(uint64_t));
- BSONObj key(ConstSharedBuffer{sb});
+ // RecordId + TypeBits size + TypeBits buffer
+ return sizeof(uint64_t) * 2 + typeBitsSize;
+}
- return key;
+RecordId IndexDataEntry::loc() const {
+ uint64_t repr;
+ std::memcpy(&repr, _buffer, sizeof(uint64_t));
+ return RecordId(repr);
}
-IndexKeyEntry keyStringToIndexKeyEntry(const std::string keyString,
- std::string data,
- const Ordering order) {
+KeyString::TypeBits IndexDataEntry::typeBits() const {
+ uint64_t size;
+ std::memcpy(&size, _buffer + sizeof(uint64_t), sizeof(uint64_t));
- auto key = getKeyFromKeyString(keyString, data, order);
- int64_t ridRepr;
- std::memcpy(&ridRepr, data.data(), sizeof(int64_t));
- RecordId rid(ridRepr);
- return IndexKeyEntry(key, rid);
+ BufReader reader(_buffer + sizeof(uint64_t) * 2, size);
+ return KeyString::TypeBits::fromBuffer(KeyString::Version::kLatestVersion, &reader);
}
-boost::optional<KeyStringEntry> keyStringToKeyStringEntry(const std::string keyString,
- std::string data,
- const Ordering order) {
- auto key = getKeyFromKeyString(keyString, data, order);
- int64_t ridRepr;
- std::memcpy(&ridRepr, data.data(), sizeof(int64_t));
- RecordId rid(ridRepr);
- KeyString::Builder ksFinal(KeyString::Version::V1, key, order);
- ksFinal.appendRecordId(rid);
- return KeyStringEntry(ksFinal.getValueCopy(), rid);
+IndexDataEntryIterator::IndexDataEntryIterator(const uint8_t* entryData)
+ : _entry(IndexDataEntry(entryData)) {}
+void IndexDataEntryIterator::increment() {
+ _entry = IndexDataEntry(_entry.buffer() + _entry.size());
}
-} // namespace
-
-SortedDataBuilderInterface::SortedDataBuilderInterface(OperationContext* opCtx,
- bool unique,
- bool dupsAllowed,
- Ordering order,
- const std::string& prefix,
- const std::string& identEnd,
- const IndexDescriptor* desc,
- const std::string& indexName,
- const BSONObj& keyPattern,
- const BSONObj& collation)
- : _opCtx(opCtx),
- _unique(unique),
- _dupsAllowed(dupsAllowed),
- _order(order),
- _prefix(prefix),
- _identEnd(identEnd),
- _desc(desc),
- _indexName(indexName),
- _keyPattern(keyPattern),
- _collation(collation),
- _hasLast(false),
- _lastKeyToString(""),
- _lastRID(-1) {}
-
-void SortedDataBuilderInterface::commit(bool mayInterrupt) {
- WriteUnitOfWork wunit(_opCtx);
- wunit.commit();
+bool IndexDataEntryIterator::equal(IndexDataEntryIterator const& other) const {
+ return _entry.buffer() == other._entry.buffer();
}
-Status SortedDataBuilderInterface::addKey(const KeyString::Value& keyString) {
- dassert(KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize()).isValid());
- RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+const IndexDataEntry& IndexDataEntryIterator::dereference() const {
+ return _entry;
+}
- StringStore* workingCopy(RecoveryUnit::get(_opCtx)->getHead());
- auto sizeWithoutRecordId =
- KeyString::sizeWithoutRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
- std::string newKSToString = std::string(keyString.getBuffer(), sizeWithoutRecordId);
+UniqueIndexData::const_iterator UniqueIndexData::lower_bound(RecordId loc) const {
+ // Linear search to the first item not less than loc
+ return std::find_if_not(
+ begin(), end(), [loc](const IndexDataEntry& entry) { return entry.loc() < loc; });
+}
+UniqueIndexData::const_iterator UniqueIndexData::upper_bound(RecordId loc) const {
+ // Linear search to the first item larger than loc
+ auto lb = lower_bound(loc);
+ return std::find_if(
+ lb, end(), [loc](const IndexDataEntry& entry) { return loc < entry.loc(); });
+}
- int twoKeyCmp = 1;
- int twoRIDCmp = 1;
+size_t UniqueIndexData::_memoryUsage() const {
+ return sizeof(_size) + _end - _begin;
+}
- if (_hasLast) {
- twoKeyCmp = newKSToString.compare(_lastKeyToString);
- twoRIDCmp = loc.repr() - _lastRID;
+boost::optional<std::string> UniqueIndexData::add(RecordId loc,
+ const KeyString::TypeBits& typeBits) {
+ // If entry already exists then nothing to do
+ auto it = lower_bound(loc);
+ if (it != end() && it->loc() == loc)
+ return boost::none;
+
+ auto itBuffer = it->buffer();
+ std::string entry = IndexDataEntry::create(loc, typeBits);
+
+ // Allocate string that fit the new entry
+ std::string output(_memoryUsage() + entry.size(), '\0');
+ auto pos = output.data();
+
+ // Write number of entries
+ uint64_t num = size() + 1;
+ std::memcpy(pos, &num, sizeof(num));
+ pos += sizeof(num);
+
+ // Write old entries smaller than the new one
+ if (auto bytes = itBuffer - _begin) {
+ std::memcpy(pos, _begin, bytes);
+ pos += bytes;
}
- if (twoKeyCmp < 0 || (_dupsAllowed && twoKeyCmp == 0 && twoRIDCmp < 0)) {
- return Status(ErrorCodes::InternalError,
- "expected ascending (key, RecordId) order in bulk builder");
+ // Write new entry
+ std::memcpy(pos, entry.data(), entry.size());
+ pos += entry.size();
+
+ // Write old entries larger than the new one
+ if (auto bytes = _end - itBuffer) {
+ std::memcpy(pos, itBuffer, bytes);
}
- std::string workingCopyInsertKey =
- createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ _unique);
+ return output;
+}
+boost::optional<std::string> UniqueIndexData::remove(RecordId loc) {
+ // If entry doesn't exist then nothing to do
+ auto it = lower_bound(loc);
+ if (it == end() || it->loc() != loc)
+ return boost::none;
+
+ // Allocate string with approrpriate amount of space
+ std::string output(_memoryUsage() - it->size(), '\0');
+ auto pos = output.data();
+
+ // Write number of entries
+ uint64_t num = size() - 1;
+ std::memcpy(pos, &num, sizeof(num));
+ pos += sizeof(num);
+
+ // Write entries before entry to remove
+ std::memcpy(pos, _begin, it->buffer() - _begin);
+ pos += it->buffer() - _begin;
+
+ // Skip entry to remove and write remaining entries
+ ++it;
+ std::memcpy(pos, it->buffer(), _end - it->buffer());
+ return output;
+}
- if (twoKeyCmp == 0 && twoRIDCmp != 0) {
- if (!_dupsAllowed)
- return buildDupKeyErrorStatus(_opCtx, keyString, _order, _desc);
+const Ordering allAscending = Ordering::make(BSONObj());
- // Duplicate index entries are allowed on this unique index, so we put the RecordId in the
- // KeyString until the unique constraint is resolved.
- workingCopyInsertKey =
- createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ false);
- }
+void prefixKeyStringWithoutLoc(KeyString::Builder* keyString, const std::string& prefixToUse) {
+ BSONObjBuilder b;
+ b.append("", prefixToUse); // prefix
+ b.append("", StringData(keyString->getBuffer(), keyString->getSize())); // key
- std::string internalTbString(keyString.getTypeBits().getBuffer(),
- keyString.getTypeBits().getSize());
+ keyString->resetToKey(b.obj(), allAscending);
+}
- // Since this is an in-memory storage engine, we don't need to take endianness into account.
- int64_t recIdRepr = loc.repr();
- std::string data(sizeof(int64_t) + internalTbString.length(), '\0');
- std::memcpy(&data[0], &recIdRepr, sizeof(int64_t));
- std::memcpy(&data[0] + sizeof(int64_t), internalTbString.data(), internalTbString.length());
+void prefixKeyStringWithLoc(KeyString::Builder* keyString,
+ RecordId loc,
+ const std::string& prefixToUse) {
+ BSONObjBuilder b;
+ b.append("", prefixToUse); // prefix
+ b.append("", StringData(keyString->getBuffer(), keyString->getSize())); // key
- workingCopy->insert(StringStore::value_type(workingCopyInsertKey, data));
+ keyString->resetToKey(b.obj(), allAscending, loc);
+}
- _hasLast = true;
- _lastKeyToString = newKSToString;
- _lastRID = loc.repr();
+std::string createRadixKeyWithoutLocFromObj(const BSONObj& key,
+ const std::string& prefixToUse,
+ Ordering order) {
+ KeyString::Version version = KeyString::Version::kLatestVersion;
+ KeyString::Builder ks(version, BSONObj::stripFieldNames(key), order);
- RecoveryUnit::get(_opCtx)->makeDirty();
- return Status::OK();
+ prefixKeyStringWithoutLoc(&ks, prefixToUse);
+ return std::string(ks.getBuffer(), ks.getSize());
}
-SortedDataBuilderInterface* SortedDataInterface::getBulkBuilder(OperationContext* opCtx,
- bool dupsAllowed) {
- return new SortedDataBuilderInterface(opCtx,
- _isUnique,
- dupsAllowed,
- _ordering,
- _prefix,
- _identEnd,
- _desc,
- _indexName,
- _keyPattern,
- _collation);
+std::string createRadixKeyWithoutLocFromKS(const KeyString::Value& keyString,
+ const std::string& prefixToUse) {
+ KeyString::Builder ks(KeyString::Version::kLatestVersion);
+ ks.resetFromBuffer(
+ keyString.getBuffer(),
+ KeyString::sizeWithoutRecordIdAtEnd(keyString.getBuffer(), keyString.getSize()));
+ prefixKeyStringWithoutLoc(&ks, prefixToUse);
+ return std::string(ks.getBuffer(), ks.getSize());
}
-// We append \1 to all idents we get, and therefore the KeyString with ident + \0 will only be
-// before elements in this ident, and the KeyString with ident + \2 will only be after elements in
-// this ident.
-SortedDataInterface::SortedDataInterface(OperationContext* opCtx,
- StringData ident,
- const IndexDescriptor* desc)
- : ::mongo::SortedDataInterface(KeyString::Version::V1, Ordering::make(desc->keyPattern())),
- // All entries in this ident will have a prefix of ident + \1.
- _prefix(ident.toString().append(1, '\1')),
- // Therefore, the string ident + \2 will be greater than all elements in this ident.
- _identEnd(ident.toString().append(1, '\2')),
- _desc(desc),
- _indexName(desc->indexName()),
- _keyPattern(desc->keyPattern()),
- _collation(desc->collation()),
- _isUnique(desc->unique()),
- _isPartial(desc->isPartial()) {
- // This is the string representation of the KeyString before elements in this ident, which is
- // ident + \0. This is before all elements in this ident.
- _KSForIdentStart = createKeyString(
- BSONObj(), RecordId::min(), ident.toString().append(1, '\0'), _ordering, _isUnique);
- // Similarly, this is the string representation of the KeyString for something greater than
- // all other elements in this ident.
- _KSForIdentEnd = createKeyString(BSONObj(), RecordId::min(), _identEnd, _ordering, _isUnique);
+std::string createRadixKeyWithoutLocFromKSWithoutRecordId(const KeyString::Value& keyString,
+ const std::string& prefixToUse) {
+ KeyString::Builder ks(KeyString::Version::kLatestVersion);
+ ks.resetFromBuffer(keyString.getBuffer(), keyString.getSize());
+ prefixKeyStringWithoutLoc(&ks, prefixToUse);
+ return std::string(ks.getBuffer(), ks.getSize());
}
-SortedDataInterface::SortedDataInterface(const Ordering& ordering, bool isUnique, StringData ident)
- : ::mongo::SortedDataInterface(KeyString::Version::V1, ordering),
- _prefix(ident.toString().append(1, '\1')),
- _identEnd(ident.toString().append(1, '\2')),
- _isUnique(isUnique),
- _isPartial(false) {
- _KSForIdentStart = createKeyString(
- BSONObj(), RecordId::min(), ident.toString().append(1, '\0'), _ordering, _isUnique);
- _KSForIdentEnd = createKeyString(BSONObj(), RecordId::min(), _identEnd, _ordering, _isUnique);
+std::string createRadixKeyWithLocFromObj(const BSONObj& key,
+ RecordId loc,
+ const std::string& prefixToUse,
+ Ordering order) {
+ KeyString::Version version = KeyString::Version::kLatestVersion;
+ KeyString::Builder ks(version, BSONObj::stripFieldNames(key), order);
+
+ prefixKeyStringWithLoc(&ks, loc, prefixToUse);
+ return std::string(ks.getBuffer(), ks.getSize());
}
-Status SortedDataInterface::insert(OperationContext* opCtx,
- const KeyString::Value& keyString,
- bool dupsAllowed) {
- RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+std::string createRadixKeyWithLocFromKS(const KeyString::Value& keyString,
+ RecordId loc,
+ const std::string& prefixToUse) {
+ KeyString::Builder ks(KeyString::Version::kLatestVersion);
+ ks.resetFromBuffer(
+ keyString.getBuffer(),
+ KeyString::sizeWithoutRecordIdAtEnd(keyString.getBuffer(), keyString.getSize()));
+ prefixKeyStringWithLoc(&ks, loc, prefixToUse);
+ return std::string(ks.getBuffer(), ks.getSize());
+}
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- auto sizeWithoutRecordId =
- KeyString::sizeWithoutRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
- std::string insertKeyString =
- createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, _isUnique);
- // For unique indexes, if duplicate keys are allowed then we do the following:
- // - Create the KeyString without the RecordId in it and see if anything exists with that.
- // - If the cursor didn't find anything, we index with this KeyString.
- // - If the cursor found a value and it had differing RecordId's, then generate a KeyString
- // with the RecordId in it.
- invariant(_isUnique || dupsAllowed);
- if (_isUnique) {
- // Ensure that another index entry without the RecordId in its KeyString doesn't exist with
- // another RecordId already.
- auto workingCopyIt = workingCopy->find(insertKeyString);
- if (workingCopyIt != workingCopy->end()) {
- IndexKeyEntry entry =
- keyStringToIndexKeyEntry(workingCopyIt->first, workingCopyIt->second, _ordering);
-
- if (entry.loc == loc)
- return Status::OK();
- if (!dupsAllowed)
- return buildDupKeyErrorStatus(opCtx, keyString, _ordering, _desc);
- // Duplicate index entries are allowed on this unique index, so we put the
- // RecordId in the KeyString until the unique constraint is resolved.
- insertKeyString = createKeyString(keyString,
- sizeWithoutRecordId,
- loc,
- _prefix,
- /* isUnique */ false);
- }
- }
- if (workingCopy->find(insertKeyString) != workingCopy->end())
- return Status::OK();
+std::string createRadixKeyWithLocFromKSWithoutRecordId(const KeyString::Value& keyString,
+ RecordId loc,
+ const std::string& prefixToUse) {
+ KeyString::Builder ks(KeyString::Version::kLatestVersion);
+ ks.resetFromBuffer(keyString.getBuffer(), keyString.getSize());
+ prefixKeyStringWithLoc(&ks, loc, prefixToUse);
+ return std::string(ks.getBuffer(), ks.getSize());
+}
- // The value we insert is the RecordId followed by the typebits.
- std::string internalTbString =
- std::string(keyString.getTypeBits().getBuffer(), keyString.getTypeBits().getSize());
+BSONObj createObjFromRadixKey(const std::string& radixKey,
+ const KeyString::TypeBits& typeBits,
+ const Ordering& order) {
+ KeyString::Version version = KeyString::Version::kLatestVersion;
+ KeyString::TypeBits tbOuter = KeyString::TypeBits(version);
+ BSONObj bsonObj =
+ KeyString::toBsonSafe(radixKey.data(), radixKey.size(), allAscending, tbOuter);
- // Since this is an in-memory storage engine, we don't need to take endianness into account.
- int64_t recIdRepr = loc.repr();
- std::string data(sizeof(int64_t) + internalTbString.length(), '\0');
- std::memcpy(&data[0], &recIdRepr, sizeof(int64_t));
- std::memcpy(&data[0] + sizeof(int64_t), internalTbString.data(), internalTbString.length());
+ SharedBuffer sb;
+ auto it = BSONObjIterator(bsonObj);
+ ++it; // We want the second part
+ KeyString::Builder ks(version);
+ ks.resetFromBuffer((*it).valuestr(), (*it).valuestrsize());
- workingCopy->insert(StringStore::value_type(insertKeyString, data));
- RecoveryUnit::get(opCtx)->makeDirty();
+ return KeyString::toBsonSafe(ks.getBuffer(), ks.getSize(), order, typeBits);
+}
- return Status::OK();
+IndexKeyEntry createIndexKeyEntryFromRadixKey(const std::string& radixKey,
+ RecordId loc,
+ const KeyString::TypeBits& typeBits,
+ const Ordering order) {
+ return IndexKeyEntry(createObjFromRadixKey(radixKey, typeBits, order), loc);
}
-void SortedDataInterface::unindex(OperationContext* opCtx,
- const KeyString::Value& keyString,
- bool dupsAllowed) {
- RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+IndexKeyEntry createIndexKeyEntryFromRadixKey(const std::string& radixKey,
+ const std::string& indexDataEntry,
+ const Ordering order) {
+ IndexDataEntry data(indexDataEntry);
+ return IndexKeyEntry(createObjFromRadixKey(radixKey, data.typeBits(), order), data.loc());
+}
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- std::string removeKeyString;
- bool erased;
-
- auto sizeWithoutRecordId =
- KeyString::sizeWithoutRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
- if (_isUnique) {
- // For unique indexes, to unindex them we do the following:
- // - Create the KeyString with or without the RecordId in it depending on dupsAllowed
- // and try to remove the index entry.
- // - If the index entry was removed, we're done.
- // - If the index entry was not removed, we generate a KeyString with or without the
- // RecordId in it.
- // This is required because of the way we insert on unique indexes when dups are allowed.
- if (dupsAllowed)
- removeKeyString =
- createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ false);
- else
- removeKeyString =
- createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ true);
-
- // Check that the record id matches when using partial indexes. We may be called to unindex
- // records that are not present in the index due to the partial filter expression.
- if (!ifPartialCheckRecordIdEquals(opCtx, removeKeyString, loc))
- return;
- erased = workingCopy->erase(removeKeyString);
-
- if (!erased) {
- // If nothing above was erased, then we have to generate the KeyString with or without
- // the RecordId in it, and erase that. This could only happen on unique indexes where
- // duplicate index entries were/are allowed.
- if (dupsAllowed)
- removeKeyString = createKeyString(
- keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ true);
- else
- removeKeyString = createKeyString(
- keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ false);
-
- if (!ifPartialCheckRecordIdEquals(opCtx, removeKeyString, loc))
- return;
- erased = workingCopy->erase(removeKeyString);
- }
- } else {
- removeKeyString =
- createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ false);
- erased = workingCopy->erase(removeKeyString);
- }
+boost::optional<KeyStringEntry> createKeyStringEntryFromRadixKey(
+ const std::string& radixKey,
+ RecordId loc,
+ const KeyString::TypeBits& typeBits,
+ const Ordering& order) {
+ auto key = createObjFromRadixKey(radixKey, typeBits, order);
+ KeyString::Builder ksFinal(KeyString::Version::kLatestVersion, key, order);
+ ksFinal.appendRecordId(loc);
+ return KeyStringEntry(ksFinal.getValueCopy(), loc);
+}
- if (erased)
- RecoveryUnit::get(opCtx)->makeDirty();
+boost::optional<KeyStringEntry> createKeyStringEntryFromRadixKey(const std::string& radixKey,
+ const std::string& indexDataEntry,
+ const Ordering& order) {
+ IndexDataEntry data(indexDataEntry);
+ RecordId loc = data.loc();
+ auto key = createObjFromRadixKey(radixKey, data.typeBits(), order);
+ KeyString::Builder ksFinal(KeyString::Version::kLatestVersion, key, order);
+ ksFinal.appendRecordId(loc);
+ return KeyStringEntry(ksFinal.getValueCopy(), loc);
}
-// This function is, as of now, not in the interface, but there exists a server ticket to add
-// truncate to the list of commands able to be used.
-Status SortedDataInterface::truncate(mongo::RecoveryUnit* ru) {
- auto bRu = checked_cast<biggie::RecoveryUnit*>(ru);
- StringStore* workingCopy(bRu->getHead());
- std::vector<std::string> toDelete;
- auto end = workingCopy->upper_bound(_KSForIdentEnd);
- for (auto it = workingCopy->lower_bound(_KSForIdentStart); it != end; ++it) {
- toDelete.push_back(it->first);
+/*
+ * This is the base cursor class required by the sorted data interface.
+ * Using CRTP (static inheritance) to reuse shared implementation for cursors over unique and
+ * standard indexes
+ */
+template <class CursorImpl>
+class CursorBase : public ::mongo::SortedDataInterface::Cursor {
+public:
+ // All the following public functions just implement the interface.
+ CursorBase(OperationContext* opCtx,
+ bool isForward,
+ // This is the ident.
+ std::string _prefix,
+ // This is a string immediately after the ident and before other idents.
+ std::string _identEnd,
+ StringStore* workingCopy,
+ Ordering order,
+ std::string prefixBSON,
+ std::string KSForIdentEnd);
+ virtual void setEndPosition(const BSONObj& key, bool inclusive) override;
+ virtual boost::optional<IndexKeyEntry> seek(const KeyString::Value& keyString,
+ RequestedInfo parts = kKeyAndLoc) override;
+ virtual boost::optional<KeyStringEntry> seekForKeyString(
+ const KeyString::Value& keyStringValue) override;
+ virtual boost::optional<KeyStringEntry> seekExactForKeyString(
+ const KeyString::Value& keyStringValue) override;
+ virtual boost::optional<IndexKeyEntry> seekExact(const KeyString::Value& keyStringValue,
+ RequestedInfo) override;
+ virtual void save() override;
+ virtual void restore() override;
+ virtual void detachFromOperationContext() override;
+ virtual void reattachToOperationContext(OperationContext* opCtx) override;
+
+private:
+ // CRTP Interface
+ std::string createRadixKeyFromObj(const BSONObj& key,
+ RecordId loc,
+ const std::string& prefixToUse,
+ Ordering order) {
+ return static_cast<CursorImpl*>(this)->createRadixKeyFromObj(key, loc, prefixToUse, order);
}
- if (!toDelete.empty()) {
- for (const auto& key : toDelete)
- workingCopy->erase(key);
- bRu->makeDirty();
+ std::string createRadixKeyFromKSWithoutRecordId(const KeyString::Value& keyString,
+ RecordId loc,
+ const std::string& prefixToUse) {
+ return static_cast<CursorImpl*>(this)->createRadixKeyFromKSWithoutRecordId(
+ keyString, loc, prefixToUse);
}
-
- return Status::OK();
-}
-
-Status SortedDataInterface::dupKeyCheck(OperationContext* opCtx, const KeyString::Value& key) {
- invariant(_isUnique);
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- std::string minKey = createKeyString(key, key.getSize(), RecordId::min(), _prefix, _isUnique);
- std::string maxKey = createKeyString(key, key.getSize(), RecordId::max(), _prefix, _isUnique);
-
- // We effectively do the same check as in insert. However, we also check to make sure that
- // the iterator returned to us by lower_bound also happens to be inside out ident.
- auto lowerBoundIterator = workingCopy->lower_bound(minKey);
- if (lowerBoundIterator == workingCopy->end()) {
- return Status::OK();
+ boost::optional<KeyStringEntry> finishSeekAfterProcessing() {
+ return static_cast<CursorImpl*>(this)->finishSeekAfterProcessing();
}
- if (lowerBoundIterator->first.compare(maxKey) > 0) {
- return Status::OK();
+ bool advanceNextInternal() {
+ return static_cast<CursorImpl*>(this)->advanceNextInternal();
}
- auto lower =
- keyStringToIndexKeyEntry(lowerBoundIterator->first, lowerBoundIterator->second, _ordering);
-
- ++lowerBoundIterator;
- if (lowerBoundIterator == workingCopy->end()) {
- return Status::OK();
+ void finishAdvanceNext() {
+ static_cast<CursorImpl*>(this)->finishAdvanceNext();
}
-
- auto next =
- keyStringToKeyStringEntry(lowerBoundIterator->first, lowerBoundIterator->second, _ordering);
-
- if (KeyString::compare(next->keyString.getBuffer(),
- key.getBuffer(),
- KeyString::sizeWithoutRecordIdAtEnd(next->keyString.getBuffer(),
- next->keyString.getSize()),
- key.getSize()) == 0) {
- return buildDupKeyErrorStatus(opCtx, key, _ordering, _desc);
+ bool checkCursorValid() {
+ return static_cast<CursorImpl*>(this)->checkCursorValid();
}
-
- return Status::OK();
-}
-
-void SortedDataInterface::fullValidate(OperationContext* opCtx,
- long long* numKeysOut,
- ValidateResults* fullResults) const {
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- long long numKeys = 0;
- auto it = workingCopy->lower_bound(_KSForIdentStart);
- while (it != workingCopy->end() && it->first.compare(_KSForIdentEnd) < 0) {
- ++it;
- numKeys++;
+ void saveForward() {
+ return static_cast<CursorImpl*>(this)->saveForward();
}
- *numKeysOut = numKeys;
-}
-
-bool SortedDataInterface::appendCustomStats(OperationContext* opCtx,
- BSONObjBuilder* output,
- double scale) const {
- return false;
-}
-
-long long SortedDataInterface::getSpaceUsedBytes(OperationContext* opCtx) const {
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- size_t totalSize = 0;
- StringStore::const_iterator it = workingCopy->lower_bound(_KSForIdentStart);
- StringStore::const_iterator end = workingCopy->upper_bound(_KSForIdentEnd);
- int64_t numElements = workingCopy->distance(it, end);
- for (int i = 0; i < numElements; i++) {
- totalSize += it->first.length();
- ++it;
+ void saveReverse() {
+ return static_cast<CursorImpl*>(this)->saveReverse();
+ }
+ void restoreForward() {
+ return static_cast<CursorImpl*>(this)->restoreForward();
+ }
+ void restoreReverse() {
+ return static_cast<CursorImpl*>(this)->restoreReverse();
}
- return (long long)totalSize;
-}
-
-bool SortedDataInterface::isEmpty(OperationContext* opCtx) {
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- return workingCopy->distance(workingCopy->lower_bound(_KSForIdentStart),
- workingCopy->upper_bound(_KSForIdentEnd)) == 0;
-}
-
-std::unique_ptr<mongo::SortedDataInterface::Cursor> SortedDataInterface::newCursor(
- OperationContext* opCtx, bool isForward) const {
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
-
- return std::make_unique<SortedDataInterface::Cursor>(opCtx,
- isForward,
- _prefix,
- _identEnd,
- workingCopy,
- _ordering,
- _isUnique,
- _KSForIdentStart,
- _KSForIdentEnd);
-}
-
-Status SortedDataInterface::initAsEmpty(OperationContext* opCtx) {
- return Status::OK();
-}
-
-bool SortedDataInterface::ifPartialCheckRecordIdEquals(OperationContext* opCtx,
- const std::string key,
- const RecordId rid) const {
- if (!_isPartial)
- return true;
-
- StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
- auto workingCopyIt = workingCopy->find(key);
- if (workingCopyIt == workingCopy->end())
- return true;
- IndexKeyEntry entry =
- keyStringToIndexKeyEntry(workingCopyIt->first, workingCopyIt->second, _ordering);
- return entry.loc == rid;
-}
+protected:
+ bool advanceNext();
+ // This is a helper function to check if the cursor was explicitly set by the user or not.
+ bool endPosSet();
+ // This is a helper function for seek.
+ boost::optional<IndexKeyEntry> seekAfterProcessing(BSONObj finalKey);
+ boost::optional<KeyStringEntry> seekAfterProcessing(const KeyString::Value& keyString);
+ OperationContext* _opCtx;
+ // This is the "working copy" of the master "branch" in the git analogy.
+ StringStore* _workingCopy;
+ // These store the end positions.
+ boost::optional<StringStore::const_iterator> _endPos;
+ boost::optional<StringStore::const_reverse_iterator> _endPosReverse;
+ // This means if the cursor is a forward or reverse cursor.
+ bool _forward;
+ // This means whether the cursor has reached the last EOF (with regard to this index).
+ bool _atEOF;
+ // This means whether or not the last move was restore.
+ bool _lastMoveWasRestore;
+ // This is the keystring for the saved location.
+ std::string _saveKey;
+ RecordId _saveLoc;
+ // These are the same as before.
+ std::string _prefix;
+ std::string _identEnd;
+ // These two store the const_iterator, which is the data structure for cursors. The one we
+ // use depends on _forward.
+ StringStore::const_iterator _forwardIt;
+ StringStore::const_reverse_iterator _reverseIt;
+ // This is the ordering for the key's values for multi-field keys.
+ Ordering _order;
+ // This stores whether or not the end position is inclusive for restore.
+ bool _endPosIncl;
+ // This stores the key for the end position.
+ boost::optional<BSONObj> _endPosKey;
+ // The next two are the same as above.
+ std::string _KSForIdentStart;
+ std::string _KSForIdentEnd;
+};
// Cursor
-SortedDataInterface::Cursor::Cursor(OperationContext* opCtx,
- bool isForward,
- std::string _prefix,
- std::string _identEnd,
- StringStore* workingCopy,
- Ordering order,
- bool isUnique,
- std::string _KSForIdentStart,
- std::string identEndBSON)
+template <class CursorImpl>
+CursorBase<CursorImpl>::CursorBase(OperationContext* opCtx,
+ bool isForward,
+ std::string _prefix,
+ std::string _identEnd,
+ StringStore* workingCopy,
+ Ordering order,
+ std::string _KSForIdentStart,
+ std::string identEndBSON)
: _opCtx(opCtx),
_workingCopy(workingCopy),
_endPos(boost::none),
@@ -586,11 +539,11 @@ SortedDataInterface::Cursor::Cursor(OperationContext* opCtx,
_reverseIt(workingCopy->rbegin()),
_order(order),
_endPosIncl(false),
- _isUnique(isUnique),
_KSForIdentStart(_KSForIdentStart),
_KSForIdentEnd(identEndBSON) {}
-bool SortedDataInterface::Cursor::advanceNext() {
+template <class CursorImpl>
+bool CursorBase<CursorImpl>::advanceNext() {
if (!_atEOF) {
// If the last move was restore, then we don't need to advance the cursor, since the user
// never got the value the cursor was pointing to in the first place. However,
@@ -599,6 +552,9 @@ bool SortedDataInterface::Cursor::advanceNext() {
if (_lastMoveWasRestore) {
_lastMoveWasRestore = false;
} else {
+ if (advanceNextInternal())
+ return true;
+
// We basically just check to make sure the cursor is in the ident.
if (_forward && checkCursorValid()) {
++_forwardIt;
@@ -616,128 +572,46 @@ bool SortedDataInterface::Cursor::advanceNext() {
_lastMoveWasRestore = false;
return false;
}
+
+ finishAdvanceNext();
+
return true;
}
// This function checks whether or not the cursor end position was set by the user or not.
-bool SortedDataInterface::Cursor::endPosSet() {
+template <class CursorImpl>
+bool CursorBase<CursorImpl>::endPosSet() {
return (_forward && _endPos != boost::none) || (!_forward && _endPosReverse != boost::none);
}
-// This function checks whether or not a cursor is valid. In particular, it checks 1) whether the
-// cursor is at end() or rend(), 2) whether the cursor is on the wrong side of the end position
-// if it was set, and 3) whether the cursor is still in the ident.
-bool SortedDataInterface::Cursor::checkCursorValid() {
- if (_forward) {
- if (_forwardIt == _workingCopy->end()) {
- return false;
- }
- if (endPosSet()) {
- // The endPos must be in the ident, at most one past the ident, or end. Therefore, the
- // endPos includes the check for being inside the ident
- if (_endPosIncl && _isUnique) {
- if (*_endPos == _workingCopy->end())
- return true;
-
- // For unique indexes, we need to check if the cursor moved up a position when it
- // was restored. This isn't required for non-unique indexes because we store the
- // RecordId in the KeyString and use a "<" comparison instead of "<=" since we know
- // that no RecordId will ever reach RecordId::max() so we don't need to check the
- // equal side of things. This assumption doesn't hold for unique index KeyStrings.
- BSONObj strippedBSON = BSONObj::stripFieldNames(*_endPosKey);
- std::string endPosKeyString =
- createKeyString(strippedBSON, RecordId::max(), _prefix, _order, _isUnique);
-
- if (_forwardIt->first.compare(endPosKeyString) <= 0)
- return true;
- return false;
- }
-
- return *_endPos == _workingCopy->end() ||
- _forwardIt->first.compare((*_endPos)->first) < 0;
- }
- return _forwardIt->first.compare(_KSForIdentEnd) <= 0;
- } else {
- // This is a reverse cursor
- if (_reverseIt == _workingCopy->rend()) {
- return false;
- }
- if (endPosSet()) {
- if (_endPosIncl && _isUnique) {
- if (*_endPosReverse == _workingCopy->rend())
- return true;
-
- BSONObj strippedBSON = BSONObj::stripFieldNames(*_endPosKey);
- std::string endPosKeyString =
- createKeyString(strippedBSON, RecordId::min(), _prefix, _order, _isUnique);
-
- if (_reverseIt->first.compare(endPosKeyString) >= 0)
- return true;
- return false;
- }
-
- return *_endPosReverse == _workingCopy->rend() ||
- _reverseIt->first.compare((*_endPosReverse)->first) > 0;
- }
- return _reverseIt->first.compare(_KSForIdentStart) >= 0;
- }
-}
-
-void SortedDataInterface::Cursor::setEndPosition(const BSONObj& key, bool inclusive) {
- auto finalKey = BSONObj::stripFieldNames(key);
+template <class CursorImpl>
+void CursorBase<CursorImpl>::setEndPosition(const BSONObj& key, bool inclusive) {
StringStore* workingCopy(RecoveryUnit::get(_opCtx)->getHead());
- if (finalKey.isEmpty()) {
+ if (key.isEmpty()) {
_endPos = boost::none;
_endPosReverse = boost::none;
return;
}
_endPosIncl = inclusive;
_endPosKey = key;
- std::string _endPosBound;
+ StringStore::const_iterator it;
// If forward and inclusive or reverse and not inclusive, then we use the last element in this
// ident. Otherwise, we use the first as our bound.
- if (_forward == inclusive) {
- _endPosBound = createKeyString(finalKey, RecordId::max(), _prefix, _order, _isUnique);
- } else {
- _endPosBound = createKeyString(finalKey, RecordId::min(), _prefix, _order, _isUnique);
- }
- if (_forward) {
- _endPos = workingCopy->lower_bound(_endPosBound);
- } else {
- // Reverse iterators work with upper bound since upper bound will return the first element
- // past the argument, so when it becomes a reverse iterator, it goes backwards one,
- // (according to the C++ standard) and we end up in the right place.
- _endPosReverse =
- StringStore::const_reverse_iterator(workingCopy->upper_bound(_endPosBound));
- }
-}
-
-boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::next(RequestedInfo parts) {
- if (!advanceNext()) {
- return {};
- }
-
- if (_forward) {
- return keyStringToIndexKeyEntry(_forwardIt->first, _forwardIt->second, _order);
- }
- return keyStringToIndexKeyEntry(_reverseIt->first, _reverseIt->second, _order);
-}
-
-boost::optional<KeyStringEntry> SortedDataInterface::Cursor::nextKeyString() {
- if (!advanceNext()) {
- return {};
- }
-
- if (_forward) {
- return keyStringToKeyStringEntry(_forwardIt->first, _forwardIt->second, _order);
- }
- return keyStringToKeyStringEntry(_reverseIt->first, _reverseIt->second, _order);
+ if (_forward == inclusive)
+ it = workingCopy->upper_bound(createRadixKeyFromObj(key, RecordId::max(), _prefix, _order));
+ else
+ it = workingCopy->lower_bound(createRadixKeyFromObj(key, RecordId::min(), _prefix, _order));
+ if (_forward)
+ _endPos = it;
+ else
+ _endPosReverse = StringStore::const_reverse_iterator(it);
}
-boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::seekAfterProcessing(BSONObj finalKey) {
+template <class CursorImpl>
+boost::optional<IndexKeyEntry> CursorBase<CursorImpl>::seekAfterProcessing(BSONObj finalKey) {
std::string workingCopyBound;
- KeyString::Builder ks(KeyString::Version::V1, finalKey, _order);
+ KeyString::Builder ks(KeyString::Version::kLatestVersion, finalKey, _order);
auto ksEntry = seekAfterProcessing(ks.getValueCopy());
const BSONObj bson = KeyString::toBson(ksEntry->keyString.getBuffer(),
@@ -747,7 +621,8 @@ boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::seekAfterProcessing(
return IndexKeyEntry(bson, ksEntry->loc);
}
-boost::optional<KeyStringEntry> SortedDataInterface::Cursor::seekAfterProcessing(
+template <class CursorImpl>
+boost::optional<KeyStringEntry> CursorBase<CursorImpl>::seekAfterProcessing(
const KeyString::Value& keyStringVal) {
KeyString::Discriminator discriminator = KeyString::decodeDiscriminator(
@@ -766,77 +641,40 @@ boost::optional<KeyStringEntry> SortedDataInterface::Cursor::seekAfterProcessing
break;
}
- std::string workingCopyBound;
- // Similar to above, if forward and inclusive or reverse and not inclusive, then use min() for
- // recordId. Else, we should use max().
- if (_forward == inclusive) {
- workingCopyBound = createKeyString(
- keyStringVal, keyStringVal.getSize(), RecordId::min(), _prefix, _isUnique);
- } else {
- workingCopyBound = createKeyString(
- keyStringVal, keyStringVal.getSize(), RecordId::max(), _prefix, _isUnique);
- }
- if (keyStringVal.isEmpty()) {
- // If the key is empty and it's not inclusive, then no elements satisfy this seek.
- if (!inclusive) {
- _atEOF = true;
- return boost::none;
- } else {
- // Otherwise, we just try to find the first element in this ident.
- if (_forward) {
- _forwardIt = _workingCopy->lower_bound(workingCopyBound);
- } else {
-
- // Reverse iterators work with upper bound since upper bound will return the first
- // element past the argument, so when it becomes a reverse iterator, it goes
- // backwards one, (according to the C++ standard) and we end up in the right place.
- _reverseIt = StringStore::const_reverse_iterator(
- _workingCopy->upper_bound(workingCopyBound));
- }
- // Here, we check to make sure the iterator doesn't fall off the data structure and is
- // in the ident. We also check to make sure it is on the correct side of the end
- // position, if it was set.
- if (!checkCursorValid()) {
- _atEOF = true;
- return boost::none;
- }
- }
- } else {
- // Otherwise, we seek to the nearest element to our key, but only to the right.
- if (_forward) {
- if (inclusive)
- _forwardIt = _workingCopy->lower_bound(workingCopyBound);
- else
- _forwardIt = _workingCopy->upper_bound(workingCopyBound);
- } else {
- // Reverse iterators work with upper bound since upper bound will return the first
- // element past the argument, so when it becomes a reverse iterator, it goes
- // backwards one, (according to the C++ standard) and we end up in the right place.
- if (inclusive)
- _reverseIt = StringStore::const_reverse_iterator(
- _workingCopy->upper_bound(workingCopyBound));
- else
- _reverseIt = StringStore::const_reverse_iterator(
- _workingCopy->lower_bound(workingCopyBound));
- }
- // Once again, we check to make sure the iterator didn't fall off the data structure and
- // still is in the ident.
- if (!checkCursorValid()) {
- _atEOF = true;
- return boost::none;
- }
+ // If the key is empty and it's not inclusive, then no elements satisfy this seek.
+ if (keyStringVal.isEmpty() && !inclusive) {
+ _atEOF = true;
+ return boost::none;
}
- // Everything checks out, so we have successfullly seeked and now return.
- boost::optional<IndexKeyEntry> indexKeyEntry;
- if (_forward) {
- return keyStringToKeyStringEntry(_forwardIt->first, _forwardIt->second, _order);
+ StringStore::const_iterator it;
+ // Forward inclusive seek uses lower_bound and exclusive upper_bound. For reverse iterators this
+ // is also reversed.
+ if (_forward == inclusive)
+ it = _workingCopy->lower_bound(
+ createRadixKeyFromKSWithoutRecordId(keyStringVal, RecordId::min(), _prefix));
+ else
+ it = _workingCopy->upper_bound(
+ createRadixKeyFromKSWithoutRecordId(keyStringVal, RecordId::max(), _prefix));
+ if (_forward)
+ _forwardIt = it;
+ else
+ _reverseIt = StringStore::const_reverse_iterator(it);
+
+ // Here, we check to make sure the iterator doesn't fall off the data structure and is
+ // in the ident. We also check to make sure it is on the correct side of the end
+ // position, if it was set.
+ if (!checkCursorValid()) {
+ _atEOF = true;
+ return boost::none;
}
- return keyStringToKeyStringEntry(_reverseIt->first, _reverseIt->second, _order);
+
+ return finishSeekAfterProcessing();
}
-boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::seek(const KeyString::Value& keyString,
- RequestedInfo parts) {
+template <class CursorImpl>
+boost::optional<IndexKeyEntry> CursorBase<CursorImpl>::seek(const KeyString::Value& keyString,
+ RequestedInfo parts) {
boost::optional<KeyStringEntry> ksValue = seekForKeyString(keyString);
if (ksValue) {
BSONObj bson = KeyString::toBson(ksValue->keyString.getBuffer(),
@@ -848,14 +686,16 @@ boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::seek(const KeyString
return boost::none;
}
-boost::optional<KeyStringEntry> SortedDataInterface::Cursor::seekForKeyString(
+template <class CursorImpl>
+boost::optional<KeyStringEntry> CursorBase<CursorImpl>::seekForKeyString(
const KeyString::Value& keyStringValue) {
_lastMoveWasRestore = false;
_atEOF = false;
return seekAfterProcessing(keyStringValue);
}
-boost::optional<KeyStringEntry> SortedDataInterface::Cursor::seekExactForKeyString(
+template <class CursorImpl>
+boost::optional<KeyStringEntry> CursorBase<CursorImpl>::seekExactForKeyString(
const KeyString::Value& keyStringValue) {
dassert(KeyString::decodeDiscriminator(keyStringValue.getBuffer(),
keyStringValue.getSize(),
@@ -876,7 +716,8 @@ boost::optional<KeyStringEntry> SortedDataInterface::Cursor::seekExactForKeyStri
return {};
}
-boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::seekExact(
+template <class CursorImpl>
+boost::optional<IndexKeyEntry> CursorBase<CursorImpl>::seekExact(
const KeyString::Value& keyStringValue, RequestedInfo parts) {
auto ksEntry = seekExactForKeyString(keyStringValue);
if (!ksEntry) {
@@ -893,20 +734,25 @@ boost::optional<IndexKeyEntry> SortedDataInterface::Cursor::seekExact(
return IndexKeyEntry(std::move(bson), ksEntry->loc);
}
-void SortedDataInterface::Cursor::save() {
+template <class CursorImpl>
+void CursorBase<CursorImpl>::save() {
_atEOF = false;
if (_lastMoveWasRestore) {
return;
- } else if (_forward && _forwardIt != _workingCopy->end()) {
+ } else if (_forward && checkCursorValid()) {
_saveKey = _forwardIt->first;
- } else if (!_forward && _reverseIt != _workingCopy->rend()) { // reverse
+ saveForward();
+ } else if (!_forward && checkCursorValid()) { // reverse
_saveKey = _reverseIt->first;
+ saveReverse();
} else {
_saveKey = "";
+ _saveLoc = RecordId();
}
}
-void SortedDataInterface::Cursor::restore() {
+template <class CursorImpl>
+void CursorBase<CursorImpl>::restore() {
StringStore* workingCopy(RecoveryUnit::get(_opCtx)->getHead());
this->_workingCopy = workingCopy;
@@ -925,49 +771,763 @@ void SortedDataInterface::Cursor::restore() {
} else {
_forwardIt = workingCopy->lower_bound(_saveKey);
}
- if (!checkCursorValid()) {
- _atEOF = true;
- _lastMoveWasRestore = true;
- return;
+ restoreForward();
+ } else {
+ // Now we are dealing with reverse cursors, and use similar logic.
+ if (_saveKey.length() == 0) {
+ _reverseIt = workingCopy->rend();
+ } else {
+ _reverseIt = StringStore::const_reverse_iterator(workingCopy->upper_bound(_saveKey));
}
+ restoreReverse();
+ }
+}
- if (!_isUnique) {
- _lastMoveWasRestore = (_forwardIt->first.compare(_saveKey) != 0);
+template <class CursorImpl>
+void CursorBase<CursorImpl>::detachFromOperationContext() {
+ _opCtx = nullptr;
+}
+
+template <class CursorImpl>
+void CursorBase<CursorImpl>::reattachToOperationContext(OperationContext* opCtx) {
+ this->_opCtx = opCtx;
+}
+
+/*
+ * This is the cursor class required by the sorted data interface for unique indexes.
+ */
+class CursorUnique final : public CursorBase<CursorUnique> {
+public:
+ using CursorBase::CursorBase;
+
+ virtual boost::optional<IndexKeyEntry> next(RequestedInfo parts = kKeyAndLoc) override;
+ virtual boost::optional<KeyStringEntry> nextKeyString() override;
+
+private:
+ // Implementations of CursorBase interface
+ friend class CursorBase;
+
+ bool advanceNextInternal();
+ void finishAdvanceNext();
+ std::string createRadixKeyFromObj(const BSONObj& key,
+ RecordId loc,
+ const std::string& prefixToUse,
+ Ordering order) {
+ return createRadixKeyWithoutLocFromObj(key, prefixToUse, order);
+ }
+ std::string createRadixKeyFromKSWithoutRecordId(const KeyString::Value& keyString,
+ RecordId loc,
+ const std::string& prefixToUse) {
+ return createRadixKeyWithoutLocFromKSWithoutRecordId(keyString, prefixToUse);
+ }
+ boost::optional<KeyStringEntry> finishSeekAfterProcessing();
+
+ void saveForward();
+ void saveReverse();
+ void restoreForward();
+ void restoreReverse();
+
+ // This is a helper function to check if the cursor is valid or not.
+ bool checkCursorValid();
+ // Helper function to set index data iterators to reverse position, we cannot use reverse
+ // iterators because we only have forward iterator support over this data
+ void initReverseDataIterators();
+ // Unpacked data from current position in the radix tree. Needed to iterate over indexes
+ // containing duplicates
+ UniqueIndexData _indexData;
+ UniqueIndexData::const_iterator _indexDataIt;
+ UniqueIndexData::const_iterator _indexDataEnd;
+ size_t _reversePos;
+};
+
+bool CursorUnique::advanceNextInternal() {
+ // Iterate over duplicates before moving to the next item in the radix tree
+ if (!_indexData.empty()) {
+ if (_forward) {
+ if (++_indexDataIt != _indexDataEnd)
+ return true;
} else {
- // Unique indexes cannot return the same key twice. Therefore, if we would normally not
- // advance on the next call to next() by setting _lastMoveWasRestore, we potentially
- // won't set it if that would cause us to return the same value twice.
- _lastMoveWasRestore = !keysAreIdentical(_forwardIt->first, _saveKey, _isUnique);
+ if (++_reversePos < _indexData.size()) {
+ initReverseDataIterators();
+ return true;
+ }
}
+ }
+ return false;
+}
+void CursorUnique::finishAdvanceNext() {
+ // We have moved to a new position in the tree, initialize index data for iterating over
+ // duplicates
+ if (_forward) {
+ _indexData = UniqueIndexData(_forwardIt->second);
+ _indexDataIt = _indexData.begin();
+ _indexDataEnd = _indexData.end();
} else {
- // Now we are dealing with reverse cursors, and use similar logic.
- if (_saveKey.length() == 0) {
- _reverseIt = workingCopy->rend();
+ _indexData = UniqueIndexData(_reverseIt->second);
+ _reversePos = 0;
+ initReverseDataIterators();
+ }
+}
+
+// This function checks whether or not a cursor is valid. In particular, it checks 1) whether the
+// cursor is at end() or rend(), 2) whether the cursor is on the wrong side of the end position
+// if it was set, and 3) whether the cursor is still in the ident.
+bool CursorUnique::checkCursorValid() {
+ if (_forward) {
+ if (_forwardIt == _workingCopy->end()) {
+ return false;
+ }
+ if (endPosSet()) {
+ // The endPos must be in the ident, at most one past the ident, or end. Therefore, the
+ // endPos includes the check for being inside the ident
+ if (_endPosIncl) {
+ if (*_endPos == _workingCopy->end())
+ return true;
+
+ // For unique indexes, we need to check if the cursor moved up a position when it
+ // was restored. This isn't required for non-unique indexes because we store the
+ // RecordId in the KeyString and use a "<" comparison instead of "<=" since we know
+ // that no RecordId will ever reach RecordId::max() so we don't need to check the
+ // equal side of things. This assumption doesn't hold for unique index KeyStrings.
+ std::string endPosKeyString =
+ createRadixKeyFromObj(*_endPosKey, RecordId::min(), _prefix, _order);
+
+ if (_forwardIt->first.compare(endPosKeyString) <= 0)
+ return true;
+ return false;
+ }
+
+ return *_endPos == _workingCopy->end() ||
+ _forwardIt->first.compare((*_endPos)->first) < 0;
+ }
+ return _forwardIt->first.compare(_KSForIdentEnd) <= 0;
+ } else {
+ // This is a reverse cursor
+ if (_reverseIt == _workingCopy->rend()) {
+ return false;
+ }
+ if (endPosSet()) {
+ if (_endPosIncl) {
+ if (*_endPosReverse == _workingCopy->rend())
+ return true;
+
+ std::string endPosKeyString =
+ createRadixKeyFromObj(*_endPosKey, RecordId::min(), _prefix, _order);
+
+ if (_reverseIt->first.compare(endPosKeyString) >= 0)
+ return true;
+ return false;
+ }
+
+ return *_endPosReverse == _workingCopy->rend() ||
+ _reverseIt->first.compare((*_endPosReverse)->first) > 0;
+ }
+ return _reverseIt->first.compare(_KSForIdentStart) >= 0;
+ }
+}
+
+void CursorUnique::initReverseDataIterators() {
+ _indexDataIt = _indexData.begin();
+ _indexDataEnd = _indexData.end();
+ for (size_t i = 1; i < (_indexData.size() - _reversePos); ++i)
+ ++_indexDataIt;
+}
+
+boost::optional<IndexKeyEntry> CursorUnique::next(RequestedInfo parts) {
+ if (!advanceNext()) {
+ return {};
+ }
+
+ if (_forward) {
+ return createIndexKeyEntryFromRadixKey(
+ _forwardIt->first, _indexDataIt->loc(), _indexDataIt->typeBits(), _order);
+ }
+ return createIndexKeyEntryFromRadixKey(
+ _reverseIt->first, _indexDataIt->loc(), _indexDataIt->typeBits(), _order);
+}
+
+boost::optional<KeyStringEntry> CursorUnique::nextKeyString() {
+ if (!advanceNext()) {
+ return {};
+ }
+
+ if (_forward) {
+ return createKeyStringEntryFromRadixKey(
+ _forwardIt->first, _indexDataIt->loc(), _indexDataIt->typeBits(), _order);
+ }
+ return createKeyStringEntryFromRadixKey(
+ _reverseIt->first, _indexDataIt->loc(), _indexDataIt->typeBits(), _order);
+}
+
+boost::optional<KeyStringEntry> CursorUnique::finishSeekAfterProcessing() {
+ // We have seeked to an entry in the tree. Now unpack the data and initialize iterators to point
+ // to the first entry if this index contains duplicates
+ if (_forward) {
+ _indexData = UniqueIndexData(_forwardIt->second);
+ _indexDataIt = _indexData.begin();
+ _indexDataEnd = _indexData.end();
+ return createKeyStringEntryFromRadixKey(
+ _forwardIt->first, _indexDataIt->loc(), _indexDataIt->typeBits(), _order);
+ } else {
+ _indexData = UniqueIndexData(_reverseIt->second);
+ _reversePos = 0;
+ initReverseDataIterators();
+ return createKeyStringEntryFromRadixKey(
+ _reverseIt->first, _indexDataIt->loc(), _indexDataIt->typeBits(), _order);
+ }
+}
+
+void CursorUnique::saveForward() {
+ if (!_indexData.empty()) {
+ _saveLoc = _indexDataIt->loc();
+ }
+}
+
+void CursorUnique::saveReverse() {
+ if (!_indexData.empty()) {
+ _saveLoc = _indexDataIt->loc();
+ }
+}
+
+void CursorUnique::restoreForward() {
+ _lastMoveWasRestore = true;
+ if (_saveLoc != RecordId() && _forwardIt != _workingCopy->end() &&
+ _forwardIt->first == _saveKey) {
+ _indexData = UniqueIndexData(_forwardIt->second);
+ _indexDataIt = _indexData.lower_bound(_saveLoc);
+ _indexDataEnd = _indexData.end();
+ if (_indexDataIt == _indexDataEnd) {
+ // We reached the end of the index data, so we need to go to the next item in the
+ // radix tree to be positioned on a valid item
+ ++_forwardIt;
+ if (_forwardIt != _workingCopy->end()) {
+ _indexData = UniqueIndexData(_forwardIt->second);
+ _indexDataIt = _indexData.begin();
+ _indexDataEnd = _indexData.end();
+ }
} else {
- _reverseIt = StringStore::const_reverse_iterator(workingCopy->upper_bound(_saveKey));
+ // Unique indexes disregard difference in location and forces the cursor to advance
+ // to guarantee that we never return the same key twice
+ _lastMoveWasRestore = false;
}
- if (!checkCursorValid()) {
- _atEOF = true;
- _lastMoveWasRestore = true;
- return;
+ }
+ if (!checkCursorValid()) {
+ _atEOF = true;
+ }
+}
+
+void CursorUnique::restoreReverse() {
+ _lastMoveWasRestore = true;
+ if (_saveLoc != RecordId() && _reverseIt != _workingCopy->rend() &&
+ _reverseIt->first == _saveKey) {
+ _indexData = UniqueIndexData(_reverseIt->second);
+ _indexDataIt = _indexData.upper_bound(_saveLoc);
+ _indexDataEnd = _indexData.end();
+ if (_indexDataIt == _indexDataEnd) {
+ ++_reverseIt;
+ if (_reverseIt != _workingCopy->rend()) {
+ _indexData = UniqueIndexData(_reverseIt->second);
+ _reversePos = 0;
+ initReverseDataIterators();
+ }
+ } else {
+ _reversePos = _indexData.size() - std::distance(_indexData.begin(), _indexDataIt) - 1;
+ _lastMoveWasRestore = false;
}
+ }
+ if (!checkCursorValid()) {
+ _atEOF = true;
+ }
+}
- if (!_isUnique) {
- _lastMoveWasRestore = (_reverseIt->first.compare(_saveKey) != 0);
+/*
+ * This is the cursor class required by the sorted data interface for standard (non-unique) indexes.
+ */
+class CursorStandard final : public CursorBase<CursorStandard> {
+public:
+ using CursorBase::CursorBase;
+
+ virtual boost::optional<IndexKeyEntry> next(RequestedInfo parts = kKeyAndLoc) override;
+ virtual boost::optional<KeyStringEntry> nextKeyString() override;
+
+protected:
+ // Implementations of CursorBase interface
+ friend class CursorBase;
+
+ bool advanceNextInternal() {
+ return false;
+ }
+ void finishAdvanceNext() {}
+ std::string createRadixKeyFromObj(const BSONObj& key,
+ RecordId loc,
+ const std::string& prefixToUse,
+ Ordering order) {
+ return createRadixKeyWithLocFromObj(key, loc, prefixToUse, order);
+ }
+ std::string createRadixKeyFromKSWithoutRecordId(const KeyString::Value& keyString,
+ RecordId loc,
+ const std::string& prefixToUse) {
+ return createRadixKeyWithLocFromKSWithoutRecordId(keyString, loc, prefixToUse);
+ }
+ boost::optional<KeyStringEntry> finishSeekAfterProcessing();
+ void saveForward() {}
+ void saveReverse() {}
+ void restoreForward();
+ void restoreReverse();
+
+private:
+ // This is a helper function to check if the cursor is valid or not.
+ bool checkCursorValid();
+};
+
+// This function checks whether or not a cursor is valid. In particular, it checks 1) whether the
+// cursor is at end() or rend(), 2) whether the cursor is on the wrong side of the end position
+// if it was set, and 3) whether the cursor is still in the ident.
+bool CursorStandard::checkCursorValid() {
+ if (_forward) {
+ if (_forwardIt == _workingCopy->end()) {
+ return false;
+ }
+ if (endPosSet()) {
+ return *_endPos == _workingCopy->end() ||
+ _forwardIt->first.compare((*_endPos)->first) < 0;
+ }
+ return _forwardIt->first.compare(_KSForIdentEnd) <= 0;
+ } else {
+ // This is a reverse cursor
+ if (_reverseIt == _workingCopy->rend()) {
+ return false;
+ }
+ if (endPosSet()) {
+ return *_endPosReverse == _workingCopy->rend() ||
+ _reverseIt->first.compare((*_endPosReverse)->first) > 0;
+ }
+ return _reverseIt->first.compare(_KSForIdentStart) >= 0;
+ }
+}
+
+
+boost::optional<IndexKeyEntry> CursorStandard::next(RequestedInfo parts) {
+ if (!advanceNext()) {
+ return {};
+ }
+
+ if (_forward) {
+ return createIndexKeyEntryFromRadixKey(_forwardIt->first, _forwardIt->second, _order);
+ }
+ return createIndexKeyEntryFromRadixKey(_reverseIt->first, _reverseIt->second, _order);
+}
+
+boost::optional<KeyStringEntry> CursorStandard::nextKeyString() {
+ if (!advanceNext()) {
+ return {};
+ }
+
+ if (_forward) {
+ return createKeyStringEntryFromRadixKey(_forwardIt->first, _forwardIt->second, _order);
+ }
+ return createKeyStringEntryFromRadixKey(_reverseIt->first, _reverseIt->second, _order);
+}
+
+boost::optional<KeyStringEntry> CursorStandard::finishSeekAfterProcessing() {
+ // We have seeked to an entry in the tree.
+ if (_forward) {
+ return createKeyStringEntryFromRadixKey(_forwardIt->first, _forwardIt->second, _order);
+ } else {
+ return createKeyStringEntryFromRadixKey(_reverseIt->first, _reverseIt->second, _order);
+ }
+}
+
+void CursorStandard::restoreForward() {
+ if (!checkCursorValid()) {
+ _atEOF = true;
+ _lastMoveWasRestore = true;
+ return;
+ }
+ _lastMoveWasRestore = (_forwardIt->first.compare(_saveKey) != 0);
+}
+void CursorStandard::restoreReverse() {
+ if (!checkCursorValid()) {
+ _atEOF = true;
+ _lastMoveWasRestore = true;
+ return;
+ }
+ _lastMoveWasRestore = (_reverseIt->first.compare(_saveKey) != 0);
+}
+
+} // namespace
+
+SortedDataBuilderBase::SortedDataBuilderBase(OperationContext* opCtx,
+ bool dupsAllowed,
+ Ordering order,
+ const std::string& prefix,
+ const std::string& identEnd,
+ const IndexDescriptor* desc,
+ const std::string& indexName,
+ const BSONObj& keyPattern,
+ const BSONObj& collation)
+ : _opCtx(opCtx),
+ _dupsAllowed(dupsAllowed),
+ _order(order),
+ _prefix(prefix),
+ _identEnd(identEnd),
+ _desc(desc),
+ _indexName(indexName),
+ _keyPattern(keyPattern),
+ _collation(collation) {}
+
+void SortedDataBuilderBase::commit(bool mayInterrupt) {
+ WriteUnitOfWork wunit(_opCtx);
+ wunit.commit();
+}
+
+Status SortedDataBuilderUnique::addKey(const KeyString::Value& keyString) {
+ dassert(KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize()).isValid());
+ StringStore* workingCopy(RecoveryUnit::get(_opCtx)->getHead());
+ RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+
+ std::string key = createRadixKeyWithoutLocFromKS(keyString, _prefix);
+ auto it = workingCopy->find(key);
+ if (it != workingCopy->end()) {
+ if (!_dupsAllowed) {
+ // There was an attempt to create an index entry with a different RecordId while dups
+ // were not allowed.
+ auto obj = KeyString::toBson(keyString, _order);
+ return buildDupKeyErrorStatus(_opCtx, keyString, _order, _desc);
+ }
+
+ UniqueIndexData data(it->second);
+ // Bulk builder add keys in ascending order so we should insert at the end
+ auto added = data.add(loc, keyString.getTypeBits());
+ if (!added) {
+ // Already indexed
+ return Status::OK();
+ }
+
+ workingCopy->update({std::move(key), *added});
+ } else {
+ UniqueIndexData data;
+ workingCopy->insert({std::move(key), *data.add(loc, keyString.getTypeBits())});
+ }
+
+ RecoveryUnit::get(_opCtx)->makeDirty();
+ return Status::OK();
+}
+
+// We append \1 to all idents we get, and therefore the KeyString with ident + \0 will only be
+// before elements in this ident, and the KeyString with ident + \2 will only be after elements in
+// this ident.
+SortedDataInterfaceBase::SortedDataInterfaceBase(OperationContext* opCtx,
+ StringData ident,
+ const IndexDescriptor* desc)
+ : ::mongo::SortedDataInterface(KeyString::Version::V1, Ordering::make(desc->keyPattern())),
+ // All entries in this ident will have a prefix of ident + \1.
+ _prefix(ident.toString().append(1, '\1')),
+ // Therefore, the string ident + \2 will be greater than all elements in this ident.
+ _identEnd(ident.toString().append(1, '\2')),
+ _desc(desc),
+ _indexName(desc->indexName()),
+ _keyPattern(desc->keyPattern()),
+ _collation(desc->collation()),
+ _isPartial(desc->isPartial()) {}
+
+SortedDataInterfaceBase::SortedDataInterfaceBase(const Ordering& ordering, StringData ident)
+ : ::mongo::SortedDataInterface(KeyString::Version::V1, ordering),
+ _prefix(ident.toString().append(1, '\1')),
+ _identEnd(ident.toString().append(1, '\2')),
+ _isPartial(false) {}
+
+SortedDataBuilderInterface* SortedDataInterfaceUnique::getBulkBuilder(OperationContext* opCtx,
+ bool dupsAllowed) {
+ return new SortedDataBuilderUnique(opCtx,
+ dupsAllowed,
+ _ordering,
+ _prefix,
+ _identEnd,
+ _desc,
+ _indexName,
+ _keyPattern,
+ _collation);
+}
+
+// We append \1 to all idents we get, and therefore the KeyString with ident + \0 will only be
+// before elements in this ident, and the KeyString with ident + \2 will only be after elements in
+// this ident.
+SortedDataInterfaceUnique::SortedDataInterfaceUnique(OperationContext* opCtx,
+ StringData ident,
+ const IndexDescriptor* desc)
+ : SortedDataInterfaceBase(opCtx, ident, desc) {
+ // This is the string representation of the KeyString before elements in this ident, which is
+ // ident + \0. This is before all elements in this ident.
+ _KSForIdentStart =
+ createRadixKeyWithoutLocFromObj(BSONObj(), ident.toString().append(1, '\0'), _ordering);
+ // Similarly, this is the string representation of the KeyString for something greater than
+ // all other elements in this ident.
+ _KSForIdentEnd = createRadixKeyWithoutLocFromObj(BSONObj(), _identEnd, _ordering);
+}
+
+SortedDataInterfaceUnique::SortedDataInterfaceUnique(const Ordering& ordering, StringData ident)
+ : SortedDataInterfaceBase(ordering, ident) {
+ _KSForIdentStart =
+ createRadixKeyWithoutLocFromObj(BSONObj(), ident.toString().append(1, '\0'), _ordering);
+ _KSForIdentEnd = createRadixKeyWithoutLocFromObj(BSONObj(), _identEnd, _ordering);
+}
+
+Status SortedDataInterfaceUnique::insert(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+
+ std::string key = createRadixKeyWithoutLocFromKS(keyString, _prefix);
+ auto it = workingCopy->find(key);
+ if (it != workingCopy->end()) {
+ if (!dupsAllowed) {
+ // There was an attempt to create an index entry with a different RecordId while
+ // dups were not allowed.
+ return buildDupKeyErrorStatus(opCtx, keyString, _ordering, _desc);
+ }
+
+ UniqueIndexData data(it->second);
+ auto added = data.add(loc, keyString.getTypeBits());
+ if (!added) {
+ // Already indexed
+ return Status::OK();
+ }
+
+ workingCopy->update({std::move(key), *added});
+ } else {
+ UniqueIndexData data;
+ workingCopy->insert({std::move(key), *data.add(loc, keyString.getTypeBits())});
+ }
+ RecoveryUnit::get(opCtx)->makeDirty();
+ return Status::OK();
+}
+
+void SortedDataInterfaceUnique::unindex(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+
+ auto key = createRadixKeyWithoutLocFromKS(keyString, _prefix);
+ auto it = workingCopy->find(key);
+ if (it != workingCopy->end()) {
+ UniqueIndexData data(it->second);
+ auto removed = data.remove(loc);
+ if (!removed)
+ return; // loc not found, nothing to unindex
+
+ if (UniqueIndexData(*removed).empty()) {
+ workingCopy->erase(key);
} else {
- // We use similar logic for reverse cursors on unique indexes.
- _lastMoveWasRestore = !keysAreIdentical(_reverseIt->first, _saveKey, _isUnique);
+ workingCopy->update({std::move(key), *removed});
}
+ RecoveryUnit::get(opCtx)->makeDirty();
}
}
-void SortedDataInterface::Cursor::detachFromOperationContext() {
- _opCtx = nullptr;
+// This function is, as of now, not in the interface, but there exists a server ticket to add
+// truncate to the list of commands able to be used.
+Status SortedDataInterfaceBase::truncate(mongo::RecoveryUnit* ru) {
+ auto bRu = checked_cast<biggie::RecoveryUnit*>(ru);
+ StringStore* workingCopy(bRu->getHead());
+ std::vector<std::string> toDelete;
+ auto end = workingCopy->upper_bound(_KSForIdentEnd);
+ for (auto it = workingCopy->lower_bound(_KSForIdentStart); it != end; ++it) {
+ toDelete.push_back(it->first);
+ }
+ if (!toDelete.empty()) {
+ for (const auto& key : toDelete)
+ workingCopy->erase(key);
+ bRu->makeDirty();
+ }
+
+ return Status::OK();
}
-void SortedDataInterface::Cursor::reattachToOperationContext(OperationContext* opCtx) {
- this->_opCtx = opCtx;
+Status SortedDataInterfaceUnique::dupKeyCheck(OperationContext* opCtx,
+ const KeyString::Value& key) {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+
+ std::string radixKey = createRadixKeyWithoutLocFromKSWithoutRecordId(key, _prefix);
+ auto it = workingCopy->find(radixKey);
+ if (it == workingCopy->end())
+ return Status::OK();
+
+ UniqueIndexData data(it->second);
+ if (data.size() > 1) {
+ return buildDupKeyErrorStatus(opCtx, key, _ordering, _desc);
+ }
+
+ return Status::OK();
+}
+
+void SortedDataInterfaceUnique::fullValidate(OperationContext* opCtx,
+ long long* numKeysOut,
+ ValidateResults* fullResults) const {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ long long numKeys = 0;
+ auto it = workingCopy->lower_bound(_KSForIdentStart);
+ while (it != workingCopy->end() && it->first.compare(_KSForIdentEnd) < 0) {
+ numKeys += UniqueIndexData(it->second).size();
+ ++it;
+ }
+ *numKeysOut = numKeys;
+}
+
+bool SortedDataInterfaceBase::appendCustomStats(OperationContext* opCtx,
+ BSONObjBuilder* output,
+ double scale) const {
+ return false;
+}
+
+long long SortedDataInterfaceBase::getSpaceUsedBytes(OperationContext* opCtx) const {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ size_t totalSize = 0;
+ StringStore::const_iterator it = workingCopy->lower_bound(_KSForIdentStart);
+ StringStore::const_iterator end = workingCopy->upper_bound(_KSForIdentEnd);
+ int64_t numElements = workingCopy->distance(it, end);
+ for (int i = 0; i < numElements; i++) {
+ totalSize += it->first.length();
+ ++it;
+ }
+ return (long long)totalSize;
+}
+
+bool SortedDataInterfaceBase::isEmpty(OperationContext* opCtx) {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ return workingCopy->distance(workingCopy->lower_bound(_KSForIdentStart),
+ workingCopy->upper_bound(_KSForIdentEnd)) == 0;
}
+
+std::unique_ptr<mongo::SortedDataInterface::Cursor> SortedDataInterfaceUnique::newCursor(
+ OperationContext* opCtx, bool isForward) const {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+
+ return std::make_unique<CursorUnique>(opCtx,
+ isForward,
+ _prefix,
+ _identEnd,
+ workingCopy,
+ _ordering,
+ _KSForIdentStart,
+ _KSForIdentEnd);
+}
+
+Status SortedDataInterfaceBase::initAsEmpty(OperationContext* opCtx) {
+ return Status::OK();
+}
+
+Status SortedDataBuilderStandard::addKey(const KeyString::Value& keyString) {
+ dassert(KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize()).isValid());
+ StringStore* workingCopy(RecoveryUnit::get(_opCtx)->getHead());
+ RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+
+ std::string key = createRadixKeyWithLocFromKS(keyString, loc, _prefix);
+ bool inserted =
+ workingCopy->insert({std::move(key), IndexDataEntry::create(loc, keyString.getTypeBits())})
+ .second;
+ if (inserted)
+ RecoveryUnit::get(_opCtx)->makeDirty();
+ return Status::OK();
+}
+
+SortedDataBuilderInterface* SortedDataInterfaceStandard::getBulkBuilder(OperationContext* opCtx,
+ bool dupsAllowed) {
+ return new SortedDataBuilderStandard(opCtx,
+ dupsAllowed,
+ _ordering,
+ _prefix,
+ _identEnd,
+ _desc,
+ _indexName,
+ _keyPattern,
+ _collation);
+}
+
+// We append \1 to all idents we get, and therefore the KeyString with ident + \0 will only be
+// before elements in this ident, and the KeyString with ident + \2 will only be after elements in
+// this ident.
+SortedDataInterfaceStandard::SortedDataInterfaceStandard(OperationContext* opCtx,
+ StringData ident,
+ const IndexDescriptor* desc)
+ : SortedDataInterfaceBase(opCtx, ident, desc) {
+ // This is the string representation of the KeyString before elements in this ident, which is
+ // ident + \0. This is before all elements in this ident.
+ _KSForIdentStart = createRadixKeyWithLocFromObj(
+ BSONObj(), RecordId::min(), ident.toString().append(1, '\0'), _ordering);
+ // Similarly, this is the string representation of the KeyString for something greater than
+ // all other elements in this ident.
+ _KSForIdentEnd = createRadixKeyWithLocFromObj(BSONObj(), RecordId::min(), _identEnd, _ordering);
+}
+
+SortedDataInterfaceStandard::SortedDataInterfaceStandard(const Ordering& ordering, StringData ident)
+ : SortedDataInterfaceBase(ordering, ident) {
+ _KSForIdentStart = createRadixKeyWithLocFromObj(
+ BSONObj(), RecordId::min(), ident.toString().append(1, '\0'), _ordering);
+ _KSForIdentEnd = createRadixKeyWithLocFromObj(BSONObj(), RecordId::min(), _identEnd, _ordering);
+}
+
+Status SortedDataInterfaceStandard::insert(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+
+ std::string key = createRadixKeyWithLocFromKS(keyString, loc, _prefix);
+ bool inserted =
+ workingCopy->insert({std::move(key), IndexDataEntry::create(loc, keyString.getTypeBits())})
+ .second;
+ if (inserted)
+ RecoveryUnit::get(opCtx)->makeDirty();
+ return Status::OK();
+}
+
+void SortedDataInterfaceStandard::unindex(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ RecordId loc = KeyString::decodeRecordIdAtEnd(keyString.getBuffer(), keyString.getSize());
+
+ auto key = createRadixKeyWithLocFromKS(keyString, loc, _prefix);
+ if (workingCopy->erase(key))
+ RecoveryUnit::get(opCtx)->makeDirty();
+}
+
+Status SortedDataInterfaceStandard::dupKeyCheck(OperationContext* opCtx,
+ const KeyString::Value& key) {
+ invariant(false);
+ return Status::OK();
+}
+
+void SortedDataInterfaceStandard::fullValidate(OperationContext* opCtx,
+ long long* numKeysOut,
+ ValidateResults* fullResults) const {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+ long long numKeys = 0;
+ auto it = workingCopy->lower_bound(_KSForIdentStart);
+ while (it != workingCopy->end() && it->first.compare(_KSForIdentEnd) < 0) {
+ ++numKeys;
+ ++it;
+ }
+ *numKeysOut = numKeys;
+}
+
+std::unique_ptr<mongo::SortedDataInterface::Cursor> SortedDataInterfaceStandard::newCursor(
+ OperationContext* opCtx, bool isForward) const {
+ StringStore* workingCopy(RecoveryUnit::get(opCtx)->getHead());
+
+ return std::make_unique<CursorStandard>(opCtx,
+ isForward,
+ _prefix,
+ _identEnd,
+ workingCopy,
+ _ordering,
+ _KSForIdentStart,
+ _KSForIdentEnd);
+}
+
} // namespace biggie
} // namespace mongo
diff --git a/src/mongo/db/storage/biggie/biggie_sorted_impl.h b/src/mongo/db/storage/biggie/biggie_sorted_impl.h
index adf0c2f1890..c2d8919a08d 100644
--- a/src/mongo/db/storage/biggie/biggie_sorted_impl.h
+++ b/src/mongo/db/storage/biggie/biggie_sorted_impl.h
@@ -29,6 +29,7 @@
#pragma once
#include "mongo/db/index/index_descriptor_fwd.h"
+#include "mongo/db/operation_context.h"
#include "mongo/db/storage/biggie/store.h"
#include "mongo/db/storage/key_string.h"
#include "mongo/db/storage/sorted_data_interface.h"
@@ -36,24 +37,21 @@
namespace mongo {
namespace biggie {
-class SortedDataBuilderInterface : public ::mongo::SortedDataBuilderInterface {
+class SortedDataBuilderBase : public SortedDataBuilderInterface {
public:
- SortedDataBuilderInterface(OperationContext* opCtx,
- bool unique,
- bool dupsAllowed,
- Ordering order,
- const std::string& prefix,
- const std::string& identEnd,
- const IndexDescriptor* _desc,
- const std::string& indexName,
- const BSONObj& keyPattern,
- const BSONObj& collation);
+ SortedDataBuilderBase(OperationContext* opCtx,
+ bool dupsAllowed,
+ Ordering order,
+ const std::string& prefix,
+ const std::string& identEnd,
+ const IndexDescriptor* desc,
+ const std::string& indexName,
+ const BSONObj& keyPattern,
+ const BSONObj& collation);
void commit(bool mayInterrupt) override;
- virtual Status addKey(const KeyString::Value& keyString);
-private:
+protected:
OperationContext* _opCtx;
- bool _unique;
bool _dupsAllowed;
// Order of the keys.
Ordering _order;
@@ -65,131 +63,29 @@ private:
const std::string _indexName;
const BSONObj _keyPattern;
const BSONObj _collation;
- // Whether or not we've already added something before.
- bool _hasLast;
- // This is the KeyString of the last key added.
- std::string _lastKeyToString;
- // This is the last recordId added.
- int64_t _lastRID;
};
-class SortedDataInterface : public ::mongo::SortedDataInterface {
+class SortedDataBuilderUnique : public SortedDataBuilderBase {
+public:
+ using SortedDataBuilderBase::SortedDataBuilderBase;
+ Status addKey(const KeyString::Value& keyString) override;
+};
+
+class SortedDataInterfaceBase : public SortedDataInterface {
public:
// Truncate is not required at the time of writing but will be when the truncate command is
// created
Status truncate(RecoveryUnit* ru);
- SortedDataInterface(OperationContext* opCtx, StringData ident, const IndexDescriptor* desc);
- SortedDataInterface(const Ordering& ordering, bool isUnique, StringData ident);
- virtual SortedDataBuilderInterface* getBulkBuilder(OperationContext* opCtx,
- bool dupsAllowed) override;
- virtual Status insert(OperationContext* opCtx,
- const KeyString::Value& keyString,
- bool dupsAllowed) override;
- virtual void unindex(OperationContext* opCtx,
- const KeyString::Value& keyString,
- bool dupsAllowed) override;
- virtual Status dupKeyCheck(OperationContext* opCtx, const KeyString::Value& keyString) override;
- virtual void fullValidate(OperationContext* opCtx,
- long long* numKeysOut,
- ValidateResults* fullResults) const override;
- virtual bool appendCustomStats(OperationContext* opCtx,
- BSONObjBuilder* output,
- double scale) const override;
- virtual long long getSpaceUsedBytes(OperationContext* opCtx) const override;
- virtual bool isEmpty(OperationContext* opCtx) override;
- virtual std::unique_ptr<mongo::SortedDataInterface::Cursor> newCursor(
- OperationContext* opCtx, bool isForward = true) const override;
- virtual Status initAsEmpty(OperationContext* opCtx) override;
-
- /*
- * This is the cursor class required by the sorted data interface.
- */
- class Cursor final : public ::mongo::SortedDataInterface::Cursor {
- public:
- // All the following public functions just implement the interface.
- Cursor(OperationContext* opCtx,
- bool isForward,
- // This is the ident.
- std::string _prefix,
- // This is a string immediately after the ident and before other idents.
- std::string _identEnd,
- StringStore* workingCopy,
- Ordering order,
- bool isUnique,
- std::string prefixBSON,
- std::string KSForIdentEnd);
- virtual void setEndPosition(const BSONObj& key, bool inclusive) override;
- virtual boost::optional<IndexKeyEntry> next(RequestedInfo parts = kKeyAndLoc) override;
- virtual boost::optional<KeyStringEntry> nextKeyString() override;
- virtual boost::optional<IndexKeyEntry> seek(const KeyString::Value& keyString,
- RequestedInfo parts = kKeyAndLoc) override;
- virtual boost::optional<KeyStringEntry> seekForKeyString(
- const KeyString::Value& keyStringValue) override;
- virtual boost::optional<KeyStringEntry> seekExactForKeyString(
- const KeyString::Value& keyStringValue) override;
- virtual boost::optional<IndexKeyEntry> seekExact(const KeyString::Value& keyStringValue,
- RequestedInfo) override;
- virtual void save() override;
- virtual void restore() override;
- virtual void detachFromOperationContext() override;
- virtual void reattachToOperationContext(OperationContext* opCtx) override;
-
- private:
- bool advanceNext();
- // This is a helper function to check if the cursor was explicitly set by the user or not.
- bool endPosSet();
- // This is a helper function to check if the cursor is valid or not.
- bool checkCursorValid();
- // This is a helper function for seek.
- boost::optional<IndexKeyEntry> seekAfterProcessing(BSONObj finalKey);
- boost::optional<KeyStringEntry> seekAfterProcessing(const KeyString::Value& keyString);
- OperationContext* _opCtx;
- // This is the "working copy" of the master "branch" in the git analogy.
- StringStore* _workingCopy;
- // These store the end positions.
- boost::optional<StringStore::const_iterator> _endPos;
- boost::optional<StringStore::const_reverse_iterator> _endPosReverse;
- // This means if the cursor is a forward or reverse cursor.
- bool _forward;
- // This means whether the cursor has reached the last EOF (with regard to this index).
- bool _atEOF;
- // This means whether or not the last move was restore.
- bool _lastMoveWasRestore;
- // This is the keystring for the saved location.
- std::string _saveKey;
- // These are the same as before.
- std::string _prefix;
- std::string _identEnd;
- // These two store the const_iterator, which is the data structure for cursors. The one we
- // use depends on _forward.
- StringStore::const_iterator _forwardIt;
- StringStore::const_reverse_iterator _reverseIt;
- // This is the ordering for the key's values for multi-field keys.
- Ordering _order;
- // This stores whether or not the end position is inclusive for restore.
- bool _endPosIncl;
- // This stores the key for the end position.
- boost::optional<BSONObj> _endPosKey;
- // This stores whether or not the index is unique.
- bool _isUnique;
- // The next two are the same as above.
- std::string _KSForIdentStart;
- std::string _KSForIdentEnd;
- };
-
-private:
- /**
- * Returns false only when the index is partial and the IndexKeyEntry's record id does not match
- * the provided rid from the given key.
- *
- * Returns true in all other cases.
- */
- bool ifPartialCheckRecordIdEquals(OperationContext* opCtx,
- const std::string key,
- const RecordId rid) const;
-
- bool keyExists(OperationContext* opCtx, const BSONObj& key);
+ SortedDataInterfaceBase(OperationContext* opCtx, StringData ident, const IndexDescriptor* desc);
+ SortedDataInterfaceBase(const Ordering& ordering, StringData ident);
+ bool appendCustomStats(OperationContext* opCtx,
+ BSONObjBuilder* output,
+ double scale) const override;
+ long long getSpaceUsedBytes(OperationContext* opCtx) const override;
+ bool isEmpty(OperationContext* opCtx) override;
+ Status initAsEmpty(OperationContext* opCtx) override;
+protected:
// These two are the same as before.
std::string _prefix;
std::string _identEnd;
@@ -201,10 +97,56 @@ private:
// These are the keystring representations of the _prefix and the _identEnd.
std::string _KSForIdentStart;
std::string _KSForIdentEnd;
- // This stores whether or not the end position is inclusive.
- bool _isUnique;
// Whether or not the index is partial
bool _isPartial;
};
+
+class SortedDataInterfaceUnique : public SortedDataInterfaceBase {
+public:
+ SortedDataInterfaceUnique(OperationContext* opCtx,
+ StringData ident,
+ const IndexDescriptor* desc);
+ SortedDataInterfaceUnique(const Ordering& ordering, StringData ident);
+ SortedDataBuilderInterface* getBulkBuilder(OperationContext* opCtx, bool dupsAllowed) override;
+ Status insert(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) override;
+ void unindex(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) override;
+ Status dupKeyCheck(OperationContext* opCtx, const KeyString::Value& keyString) override;
+ void fullValidate(OperationContext* opCtx,
+ long long* numKeysOut,
+ ValidateResults* fullResults) const override;
+ std::unique_ptr<mongo::SortedDataInterface::Cursor> newCursor(
+ OperationContext* opCtx, bool isForward = true) const override;
+};
+
+class SortedDataBuilderStandard : public SortedDataBuilderBase {
+public:
+ using SortedDataBuilderBase::SortedDataBuilderBase;
+ Status addKey(const KeyString::Value& keyString) override;
+};
+
+class SortedDataInterfaceStandard : public SortedDataInterfaceBase {
+public:
+ SortedDataInterfaceStandard(OperationContext* opCtx,
+ StringData ident,
+ const IndexDescriptor* desc);
+ SortedDataInterfaceStandard(const Ordering& ordering, StringData ident);
+ SortedDataBuilderInterface* getBulkBuilder(OperationContext* opCtx, bool dupsAllowed) override;
+ Status insert(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) override;
+ void unindex(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ bool dupsAllowed) override;
+ Status dupKeyCheck(OperationContext* opCtx, const KeyString::Value& keyString) override;
+ void fullValidate(OperationContext* opCtx,
+ long long* numKeysOut,
+ ValidateResults* fullResults) const override;
+ std::unique_ptr<mongo::SortedDataInterface::Cursor> newCursor(
+ OperationContext* opCtx, bool isForward = true) const override;
+};
} // namespace biggie
} // namespace mongo
diff --git a/src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp b/src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp
index 8bedba822cc..a5e5127e0a7 100644
--- a/src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp
+++ b/src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp
@@ -64,7 +64,7 @@ public:
IndexDescriptor desc(collection.get(), "", spec);
invariant(desc.isIdIndex());
- return std::make_unique<SortedDataInterface>(&opCtx, "ident"_sd, &desc);
+ return _kvEngine.getSortedDataInterface(&opCtx, "ident"_sd, &desc);
}
std::unique_ptr<mongo::SortedDataInterface> newSortedDataInterface(bool unique,
@@ -85,7 +85,7 @@ public:
auto collection = std::make_unique<CollectionMock>(NamespaceString(ns));
_descs.emplace_back(collection.get(), "", spec);
- return std::make_unique<SortedDataInterface>(&opCtx, "ident"_sd, &_descs.back());
+ return _kvEngine.getSortedDataInterface(&opCtx, "ident"_sd, &_descs.back());
}
std::unique_ptr<mongo::RecoveryUnit> newRecoveryUnit() final {