summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/mongo/db/catalog/collection_impl.cpp8
-rw-r--r--src/mongo/db/catalog/collection_validation.cpp6
-rw-r--r--src/mongo/db/catalog/index_build_block.cpp3
-rw-r--r--src/mongo/db/catalog/index_catalog.h25
-rw-r--r--src/mongo/db/catalog/index_catalog_entry.cpp43
-rw-r--r--src/mongo/db/catalog/index_catalog_entry.h27
-rw-r--r--src/mongo/db/catalog/index_catalog_entry_impl.cpp76
-rw-r--r--src/mongo/db/catalog/index_catalog_entry_impl.h22
-rw-r--r--src/mongo/db/catalog/index_catalog_impl.cpp122
-rw-r--r--src/mongo/db/catalog/index_catalog_impl.h28
-rw-r--r--src/mongo/db/catalog/index_catalog_noop.h8
-rw-r--r--src/mongo/db/catalog/index_signature_test.cpp22
-rw-r--r--src/mongo/db/catalog/multi_index_block.cpp14
-rw-r--r--src/mongo/db/catalog/validate_adaptor.cpp36
-rw-r--r--src/mongo/db/catalog/validate_adaptor.h2
-rw-r--r--src/mongo/db/commands/count_cmd.cpp2
-rw-r--r--src/mongo/db/commands/distinct.cpp2
-rw-r--r--src/mongo/db/commands/find_and_modify.cpp4
-rw-r--r--src/mongo/db/commands/run_aggregate.cpp3
-rw-r--r--src/mongo/db/exec/count_scan.cpp7
-rw-r--r--src/mongo/db/exec/count_scan.h9
-rw-r--r--src/mongo/db/exec/distinct_scan.cpp7
-rw-r--r--src/mongo/db/exec/distinct_scan.h9
-rw-r--r--src/mongo/db/exec/geo_near.cpp48
-rw-r--r--src/mongo/db/exec/geo_near.h12
-rw-r--r--src/mongo/db/exec/idhack.cpp8
-rw-r--r--src/mongo/db/exec/idhack.h2
-rw-r--r--src/mongo/db/exec/index_scan.cpp3
-rw-r--r--src/mongo/db/exec/index_scan.h5
-rw-r--r--src/mongo/db/exec/near.cpp3
-rw-r--r--src/mongo/db/exec/near.h1
-rw-r--r--src/mongo/db/exec/requires_index_stage.cpp5
-rw-r--r--src/mongo/db/exec/requires_index_stage.h1
-rw-r--r--src/mongo/db/exec/stagedebug_cmd.cpp6
-rw-r--r--src/mongo/db/exec/text.cpp12
-rw-r--r--src/mongo/db/exec/text.h2
-rw-r--r--src/mongo/db/index/btree_access_method.cpp4
-rw-r--r--src/mongo/db/index/index_access_method.cpp29
-rw-r--r--src/mongo/db/index/index_access_method.h14
-rw-r--r--src/mongo/db/index/index_build_interceptor.cpp19
-rw-r--r--src/mongo/db/index/index_build_interceptor.h2
-rw-r--r--src/mongo/db/index/index_descriptor.cpp27
-rw-r--r--src/mongo/db/index/index_descriptor.h34
-rw-r--r--src/mongo/db/index/skipped_record_tracker.cpp7
-rw-r--r--src/mongo/db/index/wildcard_access_method.cpp77
-rw-r--r--src/mongo/db/namespace_string.h2
-rw-r--r--src/mongo/db/ops/write_ops_exec.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_cursor.cpp3
-rw-r--r--src/mongo/db/query/collection_query_info.cpp36
-rw-r--r--src/mongo/db/query/collection_query_info.h18
-rw-r--r--src/mongo/db/query/find.cpp2
-rw-r--r--src/mongo/db/query/get_executor.cpp10
-rw-r--r--src/mongo/db/query/internal_plans.cpp7
-rw-r--r--src/mongo/db/query/stage_builder.cpp13
-rw-r--r--src/mongo/db/repl/storage_interface_impl.cpp2
-rw-r--r--src/mongo/db/repl/storage_interface_impl_test.cpp6
-rw-r--r--src/mongo/db/storage/biggie/biggie_init.cpp6
-rw-r--r--src/mongo/db/storage/biggie/biggie_sorted_impl.cpp51
-rw-r--r--src/mongo/db/storage/biggie/biggie_sorted_impl.h6
-rw-r--r--src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp14
-rw-r--r--src/mongo/db/storage/devnull/devnull_init.cpp6
-rw-r--r--src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_engine.cpp6
-rw-r--r--src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_init.cpp7
-rw-r--r--src/mongo/db/storage/index_entry_comparison.cpp19
-rw-r--r--src/mongo/db/storage/index_entry_comparison.h13
-rw-r--r--src/mongo/db/storage/kv/durable_catalog_test.cpp2
-rw-r--r--src/mongo/db/storage/kv/kv_engine_test_harness.cpp2
-rw-r--r--src/mongo/db/storage/kv/storage_engine_test.cpp3
-rw-r--r--src/mongo/db/storage/storage_engine.h5
-rw-r--r--src/mongo/db/storage/storage_engine_impl.cpp8
-rw-r--r--src/mongo/db/storage/storage_engine_impl.h6
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_index.cpp86
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_index.h9
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_init.cpp30
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp39
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine_test.cpp36
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_prefixed_index_test.cpp19
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_standard_index_test.cpp8
-rw-r--r--src/mongo/db/ttl.cpp10
-rw-r--r--src/mongo/dbtests/query_plan_executor.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_and.cpp78
-rw-r--r--src/mongo/dbtests/query_stage_count.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_count_scan.cpp72
-rw-r--r--src/mongo/dbtests/query_stage_distinct.cpp6
-rw-r--r--src/mongo/dbtests/query_stage_ixscan.cpp4
-rw-r--r--src/mongo/dbtests/query_stage_merge_sort.cpp37
-rw-r--r--src/mongo/dbtests/query_stage_multiplan.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_near.cpp10
-rw-r--r--src/mongo/dbtests/query_stage_tests.cpp4
-rw-r--r--src/mongo/dbtests/validate_tests.cpp4
-rw-r--r--src/mongo/dbtests/wildcard_multikey_persistence_test.cpp3
91 files changed, 785 insertions, 743 deletions
diff --git a/src/mongo/db/catalog/collection_impl.cpp b/src/mongo/db/catalog/collection_impl.cpp
index 7ddbbbaeb86..7317c4d9c89 100644
--- a/src/mongo/db/catalog/collection_impl.cpp
+++ b/src/mongo/db/catalog/collection_impl.cpp
@@ -254,7 +254,7 @@ CollectionImpl::CollectionImpl(OperationContext* opCtx,
_ns.db() != "local"),
_indexCatalog(std::make_unique<IndexCatalogImpl>(this)),
_cappedNotifier(_recordStore && _recordStore->isCapped()
- ? std::make_unique<CappedInsertNotifier>()
+ ? std::make_shared<CappedInsertNotifier>()
: nullptr) {
if (isCapped())
_recordStore->setCappedCallback(this);
@@ -654,7 +654,7 @@ Status CollectionImpl::_insertDocuments(OperationContext* opCtx,
}
int64_t keysInserted;
- status = _indexCatalog->indexRecords(opCtx, bsonRecords, &keysInserted);
+ status = _indexCatalog->indexRecords(opCtx, this, bsonRecords, &keysInserted);
if (opDebug) {
opDebug->additiveMetrics.incrementKeysInserted(keysInserted);
}
@@ -806,7 +806,7 @@ RecordId CollectionImpl::updateDocument(OperationContext* opCtx,
int64_t keysInserted, keysDeleted;
uassertStatusOK(_indexCatalog->updateRecord(
- opCtx, *args->preImageDoc, newDoc, oldLocation, &keysInserted, &keysDeleted));
+ opCtx, this, *args->preImageDoc, newDoc, oldLocation, &keysInserted, &keysDeleted));
if (opDebug) {
opDebug->additiveMetrics.incrementKeysInserted(keysInserted);
@@ -1204,7 +1204,7 @@ void CollectionImpl::setNs(NamespaceString nss) {
void CollectionImpl::indexBuildSuccess(OperationContext* opCtx, IndexCatalogEntry* index) {
DurableCatalog::get(opCtx)->indexBuildSuccess(
opCtx, getCatalogId(), index->descriptor()->indexName());
- _indexCatalog->indexBuildSuccess(opCtx, index);
+ _indexCatalog->indexBuildSuccess(opCtx, this, index);
}
void CollectionImpl::establishOplogCollectionForLogging(OperationContext* opCtx) {
diff --git a/src/mongo/db/catalog/collection_validation.cpp b/src/mongo/db/catalog/collection_validation.cpp
index 8932838957f..2adcaad50bf 100644
--- a/src/mongo/db/catalog/collection_validation.cpp
+++ b/src/mongo/db/catalog/collection_validation.cpp
@@ -96,7 +96,7 @@ std::map<std::string, int64_t> _validateIndexesInternalStructure(
{LogComponent::kIndex},
"Validating internal structure",
"index"_attr = descriptor->indexName(),
- "namespace"_attr = descriptor->parentNS());
+ "namespace"_attr = validateState->nss());
ValidateResults& curIndexResults = (*indexNsResultsMap)[descriptor->indexName()];
@@ -135,7 +135,7 @@ void _validateIndexes(OperationContext* opCtx,
{LogComponent::kIndex},
"Validating index consistency",
"index"_attr = descriptor->indexName(),
- "namespace"_attr = descriptor->parentNS());
+ "namespace"_attr = validateState->nss());
ValidateResults& curIndexResults = (*indexNsResultsMap)[descriptor->indexName()];
int64_t numTraversedKeys;
@@ -242,7 +242,7 @@ void _validateIndexKeyCount(OperationContext* opCtx,
ValidateResults& curIndexResults = (*indexNsResultsMap)[descriptor->indexName()];
if (curIndexResults.valid) {
- indexValidator->validateIndexKeyCount(descriptor, curIndexResults);
+ indexValidator->validateIndexKeyCount(index.get(), curIndexResults);
}
}
}
diff --git a/src/mongo/db/catalog/index_build_block.cpp b/src/mongo/db/catalog/index_build_block.cpp
index 4c1565089b0..f17f541e57e 100644
--- a/src/mongo/db/catalog/index_build_block.cpp
+++ b/src/mongo/db/catalog/index_build_block.cpp
@@ -121,7 +121,8 @@ Status IndexBuildBlock::init(OperationContext* opCtx, Collection* collection) {
// Register this index with the CollectionQueryInfo to regenerate the cache. This way, updates
// occurring while an index is being build in the background will be aware of whether or not
// they need to modify any indexes.
- CollectionQueryInfo::get(collection).addedIndex(opCtx, _indexCatalogEntry->descriptor());
+ CollectionQueryInfo::get(collection)
+ .addedIndex(opCtx, collection, _indexCatalogEntry->descriptor());
return Status::OK();
}
diff --git a/src/mongo/db/catalog/index_catalog.h b/src/mongo/db/catalog/index_catalog.h
index 9a97a935405..00362fb1f1a 100644
--- a/src/mongo/db/catalog/index_catalog.h
+++ b/src/mongo/db/catalog/index_catalog.h
@@ -403,28 +403,12 @@ public:
// ---- modify single index
/**
- * Returns true if the index 'idx' is multikey, and returns false otherwise.
- */
- virtual bool isMultikey(const IndexDescriptor* const idx) = 0;
-
- /**
- * Returns the path components that cause the index 'idx' to be multikey if the index supports
- * path-level multikey tracking, and returns an empty vector if path-level multikey tracking
- * isn't supported.
- *
- * If the index supports path-level multikey tracking but isn't multikey, then this function
- * returns a vector with size equal to the number of elements in the index key pattern where
- * each element in the vector is an empty set.
- */
- virtual MultikeyPaths getMultikeyPaths(OperationContext* const opCtx,
- const IndexDescriptor* const idx) = 0;
-
- /**
* Sets the index 'desc' to be multikey with the provided 'multikeyPaths'.
*
* See IndexCatalogEntry::setMultikey().
*/
virtual void setMultikeyPaths(OperationContext* const opCtx,
+ Collection* coll,
const IndexDescriptor* const desc,
const MultikeyPaths& multikeyPaths) = 0;
@@ -437,6 +421,7 @@ public:
* This method may throw.
*/
virtual Status indexRecords(OperationContext* const opCtx,
+ Collection* collection,
const std::vector<BsonRecord>& bsonRecords,
int64_t* const keysInsertedOut) = 0;
@@ -447,6 +432,7 @@ public:
* This method may throw.
*/
virtual Status updateRecord(OperationContext* const opCtx,
+ Collection* coll,
const BSONObj& oldDoc,
const BSONObj& newDoc,
const RecordId& recordId,
@@ -489,9 +475,12 @@ public:
* index constraints, as needed by replication.
*/
virtual void prepareInsertDeleteOptions(OperationContext* opCtx,
+ const NamespaceString& ns,
const IndexDescriptor* desc,
InsertDeleteOptions* options) const = 0;
- virtual void indexBuildSuccess(OperationContext* opCtx, IndexCatalogEntry* index) = 0;
+ virtual void indexBuildSuccess(OperationContext* opCtx,
+ const Collection* coll,
+ IndexCatalogEntry* index) = 0;
};
} // namespace mongo
diff --git a/src/mongo/db/catalog/index_catalog_entry.cpp b/src/mongo/db/catalog/index_catalog_entry.cpp
index e9690c28174..91c1d6ab9a5 100644
--- a/src/mongo/db/catalog/index_catalog_entry.cpp
+++ b/src/mongo/db/catalog/index_catalog_entry.cpp
@@ -38,49 +38,6 @@
namespace mongo {
-const IndexCatalogEntry* IndexCatalogEntryContainer::find(const IndexDescriptor* desc) const {
- if (desc->_cachedEntry)
- return desc->_cachedEntry;
-
- for (const_iterator i = begin(); i != end(); ++i) {
- const IndexCatalogEntry* e = i->get();
- if (e->descriptor() == desc)
- return e;
- }
- return nullptr;
-}
-
-IndexCatalogEntry* IndexCatalogEntryContainer::find(const IndexDescriptor* desc) {
- if (desc->_cachedEntry)
- return desc->_cachedEntry;
-
- for (iterator i = begin(); i != end(); ++i) {
- IndexCatalogEntry* e = i->get();
- if (e->descriptor() == desc)
- return e;
- }
- return nullptr;
-}
-
-std::shared_ptr<IndexCatalogEntry> IndexCatalogEntryContainer::findShared(
- const IndexDescriptor* desc) const {
- for (auto&& entry : _entries) {
- if (entry->descriptor() == desc) {
- return entry;
- }
- }
- return {};
-}
-
-IndexCatalogEntry* IndexCatalogEntryContainer::find(const std::string& name) {
- for (iterator i = begin(); i != end(); ++i) {
- IndexCatalogEntry* e = i->get();
- if (e->descriptor()->indexName() == name)
- return e;
- }
- return nullptr;
-}
-
std::shared_ptr<IndexCatalogEntry> IndexCatalogEntryContainer::release(
const IndexDescriptor* desc) {
for (auto i = _entries.begin(); i != _entries.end(); ++i) {
diff --git a/src/mongo/db/catalog/index_catalog_entry.h b/src/mongo/db/catalog/index_catalog_entry.h
index f65194fe62b..97c3b919adb 100644
--- a/src/mongo/db/catalog/index_catalog_entry.h
+++ b/src/mongo/db/catalog/index_catalog_entry.h
@@ -45,6 +45,7 @@
namespace mongo {
class CollatorInterface;
+class Collection;
class CollectionCatalogEntry;
class IndexAccessMethod;
class IndexBuildInterceptor;
@@ -52,7 +53,7 @@ class IndexDescriptor;
class MatchExpression;
class OperationContext;
-class IndexCatalogEntry {
+class IndexCatalogEntry : public std::enable_shared_from_this<IndexCatalogEntry> {
public:
IndexCatalogEntry() = default;
virtual ~IndexCatalogEntry() = default;
@@ -60,8 +61,6 @@ public:
inline IndexCatalogEntry(IndexCatalogEntry&&) = delete;
inline IndexCatalogEntry& operator=(IndexCatalogEntry&&) = delete;
- virtual const NamespaceString& ns() const = 0;
-
virtual void init(std::unique_ptr<IndexAccessMethod> accessMethod) = 0;
virtual const std::string& getIdent() const = 0;
@@ -88,6 +87,11 @@ public:
virtual const CollatorInterface* getCollator() const = 0;
+ /**
+ * Looks up the namespace name in the durable catalog. May do I/O.
+ */
+ virtual NamespaceString getNSSFromCatalog(OperationContext* opCtx) const = 0;
+
/// ---------------------
virtual void setIsReady(const bool newIsReady) = 0;
@@ -128,7 +132,9 @@ public:
* namespace, index name, and multikey paths on the OperationContext rather than set the index
* as multikey here.
*/
- virtual void setMultikey(OperationContext* const opCtx, const MultikeyPaths& multikeyPaths) = 0;
+ virtual void setMultikey(OperationContext* const opCtx,
+ const Collection* coll,
+ const MultikeyPaths& multikeyPaths) = 0;
// if this ready is ready for queries
virtual bool isReady(OperationContext* const opCtx) const = 0;
@@ -171,19 +177,6 @@ public:
return _entries.end();
}
- // TODO: these have to be SUPER SUPER FAST
- // maybe even some pointer trickery is in order
- const IndexCatalogEntry* find(const IndexDescriptor* desc) const;
- IndexCatalogEntry* find(const IndexDescriptor* desc);
-
- IndexCatalogEntry* find(const std::string& name);
-
- /**
- * Returns a pointer to the IndexCatalogEntry corresponding to 'desc', where the caller assumes
- * shared ownership of the catalog object. Returns null if the entry does not exist.
- */
- std::shared_ptr<IndexCatalogEntry> findShared(const IndexDescriptor* desc) const;
-
unsigned size() const {
return _entries.size();
}
diff --git a/src/mongo/db/catalog/index_catalog_entry_impl.cpp b/src/mongo/db/catalog/index_catalog_entry_impl.cpp
index e12c74d55e5..5bf30200176 100644
--- a/src/mongo/db/catalog/index_catalog_entry_impl.cpp
+++ b/src/mongo/db/catalog/index_catalog_entry_impl.cpp
@@ -62,6 +62,7 @@ namespace mongo {
using std::string;
IndexCatalogEntryImpl::IndexCatalogEntryImpl(OperationContext* const opCtx,
+ RecordId catalogId,
const std::string& ident,
std::unique_ptr<IndexDescriptor> descriptor,
CollectionQueryInfo* const queryInfo,
@@ -69,14 +70,15 @@ IndexCatalogEntryImpl::IndexCatalogEntryImpl(OperationContext* const opCtx,
: _ident(ident),
_descriptor(std::move(descriptor)),
_queryInfo(queryInfo),
+ _catalogId(catalogId),
_ordering(Ordering::make(_descriptor->keyPattern())),
_isReady(false),
_isFrozen(isFrozen),
_isDropped(false),
- _prefix(DurableCatalog::get(opCtx)->getIndexPrefix(
- opCtx, _descriptor->getCollection()->getCatalogId(), _descriptor->indexName())) {
- _descriptor->_cachedEntry = this;
+ _prefix(
+ DurableCatalog::get(opCtx)->getIndexPrefix(opCtx, _catalogId, _descriptor->indexName())) {
+ _descriptor->_entry = this;
_isReady = _catalogIsReady(opCtx);
{
@@ -87,6 +89,7 @@ IndexCatalogEntryImpl::IndexCatalogEntryImpl(OperationContext* const opCtx,
_indexTracksPathLevelMultikeyInfo = !_indexMultikeyPaths.empty();
}
+ auto nss = DurableCatalog::get(opCtx)->getEntry(_catalogId).nss;
const BSONObj& collation = _descriptor->collation();
if (!collation.isEmpty()) {
auto statusWithCollator =
@@ -102,7 +105,7 @@ IndexCatalogEntryImpl::IndexCatalogEntryImpl(OperationContext* const opCtx,
const BSONObj& filter = _descriptor->partialFilterExpression();
_expCtxForFilter = make_intrusive<ExpressionContext>(
- opCtx, CollatorInterface::cloneCollator(_collator.get()), ns());
+ opCtx, CollatorInterface::cloneCollator(_collator.get()), nss);
// Parsing the partial filter expression is not expected to fail here since the
// expression would have been successfully parsed upstream during index creation.
@@ -113,23 +116,13 @@ IndexCatalogEntryImpl::IndexCatalogEntryImpl(OperationContext* const opCtx,
MatchExpressionParser::kBanAllSpecialFeatures);
LOGV2_DEBUG(20350,
2,
- "have filter expression for {ns} {descriptor_indexName} {filter}",
- "ns"_attr = ns(),
- "descriptor_indexName"_attr = _descriptor->indexName(),
+ "have filter expression for {namespace} {indexName} {filter}",
+ "namespace"_attr = nss,
+ "indexName"_attr = _descriptor->indexName(),
"filter"_attr = redact(filter));
}
}
-IndexCatalogEntryImpl::~IndexCatalogEntryImpl() {
- _descriptor->_cachedEntry = nullptr; // defensive
-
- _descriptor.reset();
-}
-
-const NamespaceString& IndexCatalogEntryImpl::ns() const {
- return _descriptor->parentNS();
-}
-
void IndexCatalogEntryImpl::init(std::unique_ptr<IndexAccessMethod> accessMethod) {
invariant(!_accessMethod);
_accessMethod = std::move(accessMethod);
@@ -180,6 +173,7 @@ void IndexCatalogEntryImpl::setIsReady(bool newIsReady) {
}
void IndexCatalogEntryImpl::setMultikey(OperationContext* opCtx,
+ const Collection* collection,
const MultikeyPaths& multikeyPaths) {
if (!_indexTracksPathLevelMultikeyInfo && _isMultikeyForWrite.load()) {
// If the index is already set as multikey and we don't have any path-level information to
@@ -235,7 +229,7 @@ void IndexCatalogEntryImpl::setMultikey(OperationContext* opCtx,
// OperationContext and we can safely defer that write to the end of the batch.
if (MultikeyPathTracker::get(opCtx).isTrackingMultikeyPathInfo()) {
MultikeyPathInfo info;
- info.nss = ns();
+ info.nss = collection->ns();
info.indexName = _descriptor->indexName();
info.multikeyPaths = paths;
MultikeyPathTracker::get(opCtx).addMultikeyPathInfo(info);
@@ -243,18 +237,18 @@ void IndexCatalogEntryImpl::setMultikey(OperationContext* opCtx,
}
if (opCtx->inMultiDocumentTransaction()) {
- auto status = _setMultikeyInMultiDocumentTransaction(opCtx, paths);
+ auto status = _setMultikeyInMultiDocumentTransaction(opCtx, collection, paths);
// Retry without side transaction.
if (!status.isOK()) {
- _catalogSetMultikey(opCtx, paths);
+ _catalogSetMultikey(opCtx, collection, paths);
}
} else {
- _catalogSetMultikey(opCtx, paths);
+ _catalogSetMultikey(opCtx, collection, paths);
}
}
Status IndexCatalogEntryImpl::_setMultikeyInMultiDocumentTransaction(
- OperationContext* opCtx, const MultikeyPaths& multikeyPaths) {
+ OperationContext* opCtx, const Collection* collection, const MultikeyPaths& multikeyPaths) {
// If we are inside a multi-document transaction, we write the on-disk multikey update in a
// separate transaction so that it will not generate prepare conflicts with other operations
// that try to set the multikey flag. In general, it should always be safe to update the
@@ -273,7 +267,7 @@ Status IndexCatalogEntryImpl::_setMultikeyInMultiDocumentTransaction(
}
writeConflictRetry(
- opCtx, "set index multikey", ns().ns(), [&] {
+ opCtx, "set index multikey", collection->ns().ns(), [&] {
WriteUnitOfWork wuow(opCtx);
// If we have a prepare optime for recovery, then we always use that. This is safe since
@@ -300,11 +294,12 @@ Status IndexCatalogEntryImpl::_setMultikeyInMultiDocumentTransaction(
auto msg = BSON("msg"
<< "Setting index to multikey"
- << "coll" << ns().ns() << "index" << _descriptor->indexName());
+ << "coll" << collection->ns().ns() << "index"
+ << _descriptor->indexName());
opCtx->getClient()->getServiceContext()->getOpObserver()->onOpMessage(opCtx, msg);
}
- _catalogSetMultikey(opCtx, multikeyPaths);
+ _catalogSetMultikey(opCtx, collection, multikeyPaths);
wuow.commit();
});
@@ -314,36 +309,34 @@ Status IndexCatalogEntryImpl::_setMultikeyInMultiDocumentTransaction(
// ----
+NamespaceString IndexCatalogEntryImpl::getNSSFromCatalog(OperationContext* opCtx) const {
+ return DurableCatalog::get(opCtx)->getEntry(_catalogId).nss;
+}
+
bool IndexCatalogEntryImpl::_catalogIsReady(OperationContext* opCtx) const {
- return DurableCatalog::get(opCtx)->isIndexReady(
- opCtx, _descriptor->getCollection()->getCatalogId(), _descriptor->indexName());
+ return DurableCatalog::get(opCtx)->isIndexReady(opCtx, _catalogId, _descriptor->indexName());
}
bool IndexCatalogEntryImpl::_catalogIsPresent(OperationContext* opCtx) const {
- return DurableCatalog::get(opCtx)->isIndexPresent(
- opCtx, _descriptor->getCollection()->getCatalogId(), _descriptor->indexName());
+ return DurableCatalog::get(opCtx)->isIndexPresent(opCtx, _catalogId, _descriptor->indexName());
}
bool IndexCatalogEntryImpl::_catalogIsMultikey(OperationContext* opCtx,
MultikeyPaths* multikeyPaths) const {
- return DurableCatalog::get(opCtx)->isIndexMultikey(opCtx,
- _descriptor->getCollection()->getCatalogId(),
- _descriptor->indexName(),
- multikeyPaths);
+ return DurableCatalog::get(opCtx)->isIndexMultikey(
+ opCtx, _catalogId, _descriptor->indexName(), multikeyPaths);
}
void IndexCatalogEntryImpl::_catalogSetMultikey(OperationContext* opCtx,
+ const Collection* collection,
const MultikeyPaths& multikeyPaths) {
// It's possible that the index type (e.g. ascending/descending index) supports tracking
// path-level multikey information, but this particular index doesn't.
// CollectionCatalogEntry::setIndexIsMultikey() requires that we discard the path-level
// multikey information in order to avoid unintentionally setting path-level multikey
// information on an index created before 3.4.
- auto indexMetadataHasChanged =
- DurableCatalog::get(opCtx)->setIndexIsMultikey(opCtx,
- _descriptor->getCollection()->getCatalogId(),
- _descriptor->indexName(),
- multikeyPaths);
+ auto indexMetadataHasChanged = DurableCatalog::get(opCtx)->setIndexIsMultikey(
+ opCtx, _catalogId, _descriptor->indexName(), multikeyPaths);
// In the absense of using the storage engine to read from the catalog, we must set multikey
// prior to the storage engine transaction committing.
@@ -363,9 +356,9 @@ void IndexCatalogEntryImpl::_catalogSetMultikey(OperationContext* opCtx,
LOGV2_DEBUG(47187005,
1,
"Index set to multi key, clearing query plan cache",
- "namespace"_attr = ns(),
+ "namespace"_attr = collection->ns(),
"keyPattern"_attr = _descriptor->keyPattern());
- _queryInfo->clearQueryCache();
+ _queryInfo->clearQueryCache(collection);
}
opCtx->recoveryUnit()->onCommit([this](boost::optional<Timestamp>) {
@@ -377,8 +370,7 @@ void IndexCatalogEntryImpl::_catalogSetMultikey(OperationContext* opCtx,
}
KVPrefix IndexCatalogEntryImpl::_catalogGetPrefix(OperationContext* opCtx) const {
- return DurableCatalog::get(opCtx)->getIndexPrefix(
- opCtx, _descriptor->getCollection()->getCatalogId(), _descriptor->indexName());
+ return DurableCatalog::get(opCtx)->getIndexPrefix(opCtx, _catalogId, _descriptor->indexName());
}
} // namespace mongo
diff --git a/src/mongo/db/catalog/index_catalog_entry_impl.h b/src/mongo/db/catalog/index_catalog_entry_impl.h
index 91d4762b826..c1f02c254c5 100644
--- a/src/mongo/db/catalog/index_catalog_entry_impl.h
+++ b/src/mongo/db/catalog/index_catalog_entry_impl.h
@@ -59,15 +59,12 @@ class IndexCatalogEntryImpl : public IndexCatalogEntry {
public:
IndexCatalogEntryImpl(OperationContext* opCtx,
+ RecordId catalogId,
const std::string& ident,
std::unique_ptr<IndexDescriptor> descriptor, // ownership passes to me
CollectionQueryInfo* queryInfo, // not owned, optional
bool isFrozen);
- ~IndexCatalogEntryImpl() final;
-
- const NamespaceString& ns() const final;
-
void init(std::unique_ptr<IndexAccessMethod> accessMethod) final;
const std::string& getIdent() const final {
@@ -116,6 +113,8 @@ public:
return _collator.get();
}
+ NamespaceString getNSSFromCatalog(OperationContext* opCtx) const final;
+
/// ---------------------
void setIsReady(bool newIsReady) final;
@@ -161,7 +160,9 @@ public:
* namespace, index name, and multikey paths on the OperationContext rather than set the index
* as multikey here.
*/
- void setMultikey(OperationContext* opCtx, const MultikeyPaths& multikeyPaths) final;
+ void setMultikey(OperationContext* opCtx,
+ const Collection* coll,
+ const MultikeyPaths& multikeyPaths) final;
// if this ready is ready for queries
bool isReady(OperationContext* opCtx) const final;
@@ -192,6 +193,7 @@ private:
* Used by setMultikey() only.
*/
Status _setMultikeyInMultiDocumentTransaction(OperationContext* opCtx,
+ const Collection* collection,
const MultikeyPaths& multikeyPaths);
bool _catalogIsReady(OperationContext* opCtx) const;
@@ -207,7 +209,9 @@ private:
/**
* Sets on-disk multikey flag for this index.
*/
- void _catalogSetMultikey(OperationContext* opCtx, const MultikeyPaths& multikeyPaths);
+ void _catalogSetMultikey(OperationContext* opCtx,
+ const Collection* collection,
+ const MultikeyPaths& multikeyPaths);
KVPrefix _catalogGetPrefix(OperationContext* opCtx) const;
@@ -230,8 +234,10 @@ private:
// cached stuff
- Ordering _ordering; // TODO: this might be b-tree specific
- bool _isReady; // cache of NamespaceDetails info
+ const RecordId _catalogId; // Location in the durable catalog of the collection entry
+ // containing this index entry.
+ Ordering _ordering; // TODO: this might be b-tree specific
+ bool _isReady; // cache of NamespaceDetails info
bool _isFrozen;
AtomicWord<bool> _isDropped; // Whether the index drop is committed.
diff --git a/src/mongo/db/catalog/index_catalog_impl.cpp b/src/mongo/db/catalog/index_catalog_impl.cpp
index 7ab4638e63e..bee5d150f86 100644
--- a/src/mongo/db/catalog/index_catalog_impl.cpp
+++ b/src/mongo/db/catalog/index_catalog_impl.cpp
@@ -154,7 +154,7 @@ Status IndexCatalogImpl::init(OperationContext* opCtx) {
}
}
- CollectionQueryInfo::get(_collection).init(opCtx);
+ CollectionQueryInfo::get(_collection).init(opCtx, _collection);
return Status::OK();
}
@@ -422,11 +422,14 @@ IndexCatalogEntry* IndexCatalogImpl::createIndexEntry(OperationContext* opCtx,
invariant(!frozen || !isReadyIndex);
auto* const descriptorPtr = descriptor.get();
- auto entry = std::make_shared<IndexCatalogEntryImpl>(
- opCtx, ident, std::move(descriptor), &CollectionQueryInfo::get(_collection), frozen);
+ auto entry = std::make_shared<IndexCatalogEntryImpl>(opCtx,
+ _collection->getCatalogId(),
+ ident,
+ std::move(descriptor),
+ &CollectionQueryInfo::get(_collection),
+ frozen);
IndexDescriptor* desc = entry->descriptor();
-
std::unique_ptr<SortedDataInterface> sdi =
engine->getEngine()->getGroupedSortedDataInterface(opCtx, ident, desc, entry->getPrefix());
@@ -444,8 +447,7 @@ IndexCatalogEntry* IndexCatalogImpl::createIndexEntry(OperationContext* opCtx,
}
bool initFromDisk = CreateIndexEntryFlags::kInitFromDisk & flags;
- if (!initFromDisk &&
- UncommittedCollections::getForTxn(opCtx, descriptorPtr->parentNS()) == nullptr) {
+ if (!initFromDisk && UncommittedCollections::getForTxn(opCtx, _collection->ns()) == nullptr) {
opCtx->recoveryUnit()->onRollback([this, opCtx, isReadyIndex, descriptor = descriptorPtr] {
// Need to preserve indexName as descriptor no longer exists after remove().
const std::string indexName = descriptor->indexName();
@@ -454,7 +456,7 @@ IndexCatalogEntry* IndexCatalogImpl::createIndexEntry(OperationContext* opCtx,
} else {
_buildingIndexes.remove(descriptor);
}
- CollectionQueryInfo::get(_collection).droppedIndex(opCtx, indexName);
+ CollectionQueryInfo::get(_collection).droppedIndex(opCtx, _collection, indexName);
});
}
@@ -489,7 +491,6 @@ StatusWith<BSONObj> IndexCatalogImpl::createIndexOnEmptyCollection(OperationCont
invariant(entry);
IndexDescriptor* descriptor = entry->descriptor();
invariant(descriptor);
- invariant(entry == _buildingIndexes.find(descriptor));
status = entry->accessMethod()->initializeAsEmpty(opCtx);
if (!status.isOK())
@@ -784,7 +785,8 @@ Status IndexCatalogImpl::_doesSpecConflictWithExisting(OperationContext* opCtx,
if (desc) {
// Index already exists with same name. Check whether the options are the same as well.
IndexDescriptor candidate(_collection, _getAccessMethodName(key), spec);
- auto indexComparison = candidate.compareIndexOptions(opCtx, getEntry(desc));
+ auto indexComparison =
+ candidate.compareIndexOptions(opCtx, _collection->ns(), getEntry(desc));
// Key pattern or another uniquely-identifying option differs. We can build this index,
// but not with the specified (duplicate) name. User must specify another index name.
@@ -842,7 +844,8 @@ Status IndexCatalogImpl::_doesSpecConflictWithExisting(OperationContext* opCtx,
// We will return an error in either case, but this check allows us to generate a more
// informative error message.
IndexDescriptor candidate(_collection, _getAccessMethodName(key), spec);
- auto indexComparison = candidate.compareIndexOptions(opCtx, getEntry(desc));
+ auto indexComparison =
+ candidate.compareIndexOptions(opCtx, _collection->ns(), getEntry(desc));
// The candidate's key and uniquely-identifying options are equivalent to an existing
// index, but some other options are not identical. Return a message to that effect.
@@ -938,7 +941,7 @@ void IndexCatalogImpl::dropAllIndexes(OperationContext* opCtx,
const IndexDescriptor* desc = findIndexByName(opCtx, indexName, true);
invariant(desc);
LOGV2_DEBUG(20355, 1, "\t dropAllIndexes dropping: {desc}", "desc"_attr = *desc);
- IndexCatalogEntry* entry = _readyIndexes.find(desc);
+ IndexCatalogEntry* entry = desc->getEntry();
invariant(entry);
// If the onDrop function creates an oplog entry, it should run first so that the drop is
@@ -977,7 +980,7 @@ void IndexCatalogImpl::dropAllIndexes(OperationContext* opCtx, bool includingIdI
Status IndexCatalogImpl::dropIndex(OperationContext* opCtx, const IndexDescriptor* desc) {
invariant(opCtx->lockState()->isCollectionLockedForMode(_collection->ns(), MODE_X));
- IndexCatalogEntry* entry = _readyIndexes.find(desc);
+ IndexCatalogEntry* entry = desc->getEntry();
if (!entry)
return Status(ErrorCodes::InternalError, "cannot find index to delete");
@@ -991,7 +994,7 @@ Status IndexCatalogImpl::dropIndex(OperationContext* opCtx, const IndexDescripto
Status IndexCatalogImpl::dropUnfinishedIndex(OperationContext* opCtx, const IndexDescriptor* desc) {
invariant(opCtx->lockState()->isCollectionLockedForMode(_collection->ns(), MODE_X));
- IndexCatalogEntry* entry = _buildingIndexes.find(desc);
+ IndexCatalogEntry* entry = desc->getEntry();
if (!entry)
return Status(ErrorCodes::InternalError, "cannot find index to delete");
@@ -1031,7 +1034,7 @@ public:
// Refresh the CollectionQueryInfo's knowledge of what indices are present. This must be
// done after re-adding our IndexCatalogEntry to the '_entries' list, since 'addedIndex()'
// refreshes its knowledge by iterating the list of indices currently in the catalog.
- CollectionQueryInfo::get(_collection).addedIndex(_opCtx, indexDescriptor);
+ CollectionQueryInfo::get(_collection).addedIndex(_opCtx, _collection, indexDescriptor);
}
private:
@@ -1063,7 +1066,7 @@ Status IndexCatalogImpl::dropIndexEntry(OperationContext* opCtx, IndexCatalogEnt
opCtx, _collection, &_buildingIndexes, std::move(released)));
}
- CollectionQueryInfo::get(_collection).droppedIndex(opCtx, indexName);
+ CollectionQueryInfo::get(_collection).droppedIndex(opCtx, _collection, indexName);
entry = nullptr;
deleteIndexFromDisk(opCtx, indexName);
@@ -1089,28 +1092,13 @@ void IndexCatalogImpl::deleteIndexFromDisk(OperationContext* opCtx, const string
}
}
-bool IndexCatalogImpl::isMultikey(const IndexDescriptor* const idx) {
- IndexCatalogEntry* entry = _readyIndexes.find(idx);
- invariant(entry);
- return entry->isMultikey();
-}
-
-MultikeyPaths IndexCatalogImpl::getMultikeyPaths(OperationContext* opCtx,
- const IndexDescriptor* idx) {
- IndexCatalogEntry* entry = _readyIndexes.find(idx);
- invariant(entry);
- return entry->getMultikeyPaths(opCtx);
-}
-
void IndexCatalogImpl::setMultikeyPaths(OperationContext* const opCtx,
+ Collection* coll,
const IndexDescriptor* desc,
const MultikeyPaths& multikeyPaths) {
- IndexCatalogEntry* entry = _readyIndexes.find(desc);
- if (!entry) {
- entry = _buildingIndexes.find(desc);
- }
+ IndexCatalogEntry* entry = desc->getEntry();
invariant(entry);
- entry->setMultikey(opCtx, multikeyPaths);
+ entry->setMultikey(opCtx, coll, multikeyPaths);
};
// ---------------------------
@@ -1215,12 +1203,13 @@ const IndexDescriptor* IndexCatalogImpl::findIndexByKeyPatternAndOptions(
IndexDescriptor needle(_collection, _getAccessMethodName(key), indexSpec);
while (ii->more()) {
const auto* entry = ii->next();
- if (needle.compareIndexOptions(opCtx, entry) != IndexDescriptor::Comparison::kDifferent) {
+ if (needle.compareIndexOptions(opCtx, {}, entry) !=
+ IndexDescriptor::Comparison::kDifferent) {
return entry->descriptor();
}
}
return nullptr;
-}
+} // namespace mongo
void IndexCatalogImpl::findIndexesByKeyPattern(OperationContext* opCtx,
const BSONObj& key,
@@ -1243,7 +1232,8 @@ const IndexDescriptor* IndexCatalogImpl::findShardKeyPrefixedIndex(OperationCont
std::unique_ptr<IndexIterator> ii = getIndexIterator(opCtx, false);
while (ii->more()) {
- const IndexDescriptor* desc = ii->next()->descriptor();
+ const IndexCatalogEntry* entry = ii->next();
+ const IndexDescriptor* desc = entry->descriptor();
bool hasSimpleCollation = desc->collation().isEmpty();
if (desc->isPartial() || desc->isSparse())
@@ -1252,7 +1242,7 @@ const IndexDescriptor* IndexCatalogImpl::findShardKeyPrefixedIndex(OperationCont
if (!shardKey.isPrefixOf(desc->keyPattern(), SimpleBSONElementComparator::kInstance))
continue;
- if (!desc->isMultikey() && hasSimpleCollation)
+ if (!entry->isMultikey() && hasSimpleCollation)
return desc;
if (!requireSingleKey && hasSimpleCollation)
@@ -1276,22 +1266,14 @@ void IndexCatalogImpl::findIndexByType(OperationContext* opCtx,
}
const IndexCatalogEntry* IndexCatalogImpl::getEntry(const IndexDescriptor* desc) const {
- const IndexCatalogEntry* entry = _readyIndexes.find(desc);
- if (!entry) {
- entry = _buildingIndexes.find(desc);
- }
-
+ const IndexCatalogEntry* entry = desc->getEntry();
massert(17357, "cannot find index entry", entry);
return entry;
}
std::shared_ptr<const IndexCatalogEntry> IndexCatalogImpl::getEntryShared(
const IndexDescriptor* indexDescriptor) const {
- auto entry = _readyIndexes.findShared(indexDescriptor);
- if (entry) {
- return entry;
- }
- return _buildingIndexes.findShared(indexDescriptor);
+ return indexDescriptor->getEntry()->shared_from_this();
}
std::vector<std::shared_ptr<const IndexCatalogEntry>> IndexCatalogImpl::getAllReadyEntriesShared()
@@ -1315,7 +1297,7 @@ const IndexDescriptor* IndexCatalogImpl::refreshEntry(OperationContext* opCtx,
invariant(oldEntry);
opCtx->recoveryUnit()->registerChange(std::make_unique<IndexRemoveChange>(
opCtx, _collection, &_readyIndexes, std::move(oldEntry)));
- CollectionQueryInfo::get(_collection).droppedIndex(opCtx, indexName);
+ CollectionQueryInfo::get(_collection).droppedIndex(opCtx, _collection, indexName);
// Ask the CollectionCatalogEntry for the new index spec.
BSONObj spec =
@@ -1329,7 +1311,7 @@ const IndexDescriptor* IndexCatalogImpl::refreshEntry(OperationContext* opCtx,
const IndexCatalogEntry* newEntry =
createIndexEntry(opCtx, std::move(newDesc), CreateIndexEntryFlags::kIsReady);
invariant(newEntry->isReady(opCtx));
- CollectionQueryInfo::get(_collection).addedIndex(opCtx, newEntry->descriptor());
+ CollectionQueryInfo::get(_collection).addedIndex(opCtx, _collection, newEntry->descriptor());
// Return the new descriptor.
return newEntry->descriptor();
@@ -1338,6 +1320,7 @@ const IndexDescriptor* IndexCatalogImpl::refreshEntry(OperationContext* opCtx,
// ---------------------------
Status IndexCatalogImpl::_indexKeys(OperationContext* opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const KeyStringSet& keys,
const KeyStringSet& multikeyMetadataKeys,
@@ -1373,6 +1356,7 @@ Status IndexCatalogImpl::_indexKeys(OperationContext* opCtx,
InsertResult result;
status = index->accessMethod()->insertKeys(
opCtx,
+ coll,
keys,
{multikeyMetadataKeys.begin(), multikeyMetadataKeys.end()},
multikeyPaths,
@@ -1388,13 +1372,14 @@ Status IndexCatalogImpl::_indexKeys(OperationContext* opCtx,
}
Status IndexCatalogImpl::_indexFilteredRecords(OperationContext* opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const std::vector<BsonRecord>& bsonRecords,
int64_t* keysInsertedOut) {
auto& executionCtx = StorageExecutionContext::get(opCtx);
InsertDeleteOptions options;
- prepareInsertDeleteOptions(opCtx, index->descriptor(), &options);
+ prepareInsertDeleteOptions(opCtx, coll->ns(), index->descriptor(), &options);
for (auto bsonRecord : bsonRecords) {
invariant(bsonRecord.id != RecordId());
@@ -1420,6 +1405,7 @@ Status IndexCatalogImpl::_indexFilteredRecords(OperationContext* opCtx,
IndexAccessMethod::kNoopOnSuppressedErrorFn);
Status status = _indexKeys(opCtx,
+ coll,
index,
*keys,
*multikeyMetadataKeys,
@@ -1437,6 +1423,7 @@ Status IndexCatalogImpl::_indexFilteredRecords(OperationContext* opCtx,
}
Status IndexCatalogImpl::_indexRecords(OperationContext* opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const std::vector<BsonRecord>& bsonRecords,
int64_t* keysInsertedOut) {
@@ -1446,7 +1433,7 @@ Status IndexCatalogImpl::_indexRecords(OperationContext* opCtx,
const MatchExpression* filter = index->getFilterExpression();
if (!filter)
- return _indexFilteredRecords(opCtx, index, bsonRecords, keysInsertedOut);
+ return _indexFilteredRecords(opCtx, coll, index, bsonRecords, keysInsertedOut);
std::vector<BsonRecord> filteredBsonRecords;
for (auto bsonRecord : bsonRecords) {
@@ -1454,10 +1441,11 @@ Status IndexCatalogImpl::_indexRecords(OperationContext* opCtx,
filteredBsonRecords.push_back(bsonRecord);
}
- return _indexFilteredRecords(opCtx, index, filteredBsonRecords, keysInsertedOut);
+ return _indexFilteredRecords(opCtx, coll, index, filteredBsonRecords, keysInsertedOut);
}
Status IndexCatalogImpl::_updateRecord(OperationContext* const opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const BSONObj& oldDoc,
const BSONObj& newDoc,
@@ -1467,7 +1455,7 @@ Status IndexCatalogImpl::_updateRecord(OperationContext* const opCtx,
IndexAccessMethod* iam = index->accessMethod();
InsertDeleteOptions options;
- prepareInsertDeleteOptions(opCtx, index->descriptor(), &options);
+ prepareInsertDeleteOptions(opCtx, coll->ns(), index->descriptor(), &options);
UpdateTicket updateTicket;
@@ -1482,6 +1470,7 @@ Status IndexCatalogImpl::_updateRecord(OperationContext* const opCtx,
_unindexKeys(
opCtx, index, updateTicket.removed, oldDoc, recordId, logIfError, &keysDeleted);
status = _indexKeys(opCtx,
+ coll,
index,
updateTicket.added,
updateTicket.newMultikeyMetadataKeys,
@@ -1491,7 +1480,7 @@ Status IndexCatalogImpl::_updateRecord(OperationContext* const opCtx,
options,
&keysInserted);
} else {
- status = iam->update(opCtx, updateTicket, &keysInserted, &keysDeleted);
+ status = iam->update(opCtx, coll, updateTicket, &keysInserted, &keysDeleted);
}
if (!status.isOK())
@@ -1511,7 +1500,7 @@ void IndexCatalogImpl::_unindexKeys(OperationContext* opCtx,
bool logIfError,
int64_t* const keysDeletedOut) {
InsertDeleteOptions options;
- prepareInsertDeleteOptions(opCtx, index->descriptor(), &options);
+ prepareInsertDeleteOptions(opCtx, _collection->ns(), index->descriptor(), &options);
options.logIfError = logIfError;
if (index->isHybridBuilding()) {
@@ -1597,6 +1586,7 @@ void IndexCatalogImpl::_unindexRecord(OperationContext* opCtx,
}
Status IndexCatalogImpl::indexRecords(OperationContext* opCtx,
+ Collection* coll,
const std::vector<BsonRecord>& bsonRecords,
int64_t* keysInsertedOut) {
if (keysInsertedOut) {
@@ -1604,13 +1594,13 @@ Status IndexCatalogImpl::indexRecords(OperationContext* opCtx,
}
for (auto&& it : _readyIndexes) {
- Status s = _indexRecords(opCtx, it.get(), bsonRecords, keysInsertedOut);
+ Status s = _indexRecords(opCtx, coll, it.get(), bsonRecords, keysInsertedOut);
if (!s.isOK())
return s;
}
for (auto&& it : _buildingIndexes) {
- Status s = _indexRecords(opCtx, it.get(), bsonRecords, keysInsertedOut);
+ Status s = _indexRecords(opCtx, coll, it.get(), bsonRecords, keysInsertedOut);
if (!s.isOK())
return s;
}
@@ -1619,6 +1609,7 @@ Status IndexCatalogImpl::indexRecords(OperationContext* opCtx,
}
Status IndexCatalogImpl::updateRecord(OperationContext* const opCtx,
+ Collection* coll,
const BSONObj& oldDoc,
const BSONObj& newDoc,
const RecordId& recordId,
@@ -1632,8 +1623,8 @@ Status IndexCatalogImpl::updateRecord(OperationContext* const opCtx,
it != _readyIndexes.end();
++it) {
IndexCatalogEntry* entry = it->get();
- auto status =
- _updateRecord(opCtx, entry, oldDoc, newDoc, recordId, keysInsertedOut, keysDeletedOut);
+ auto status = _updateRecord(
+ opCtx, coll, entry, oldDoc, newDoc, recordId, keysInsertedOut, keysDeletedOut);
if (!status.isOK())
return status;
}
@@ -1643,8 +1634,8 @@ Status IndexCatalogImpl::updateRecord(OperationContext* const opCtx,
it != _buildingIndexes.end();
++it) {
IndexCatalogEntry* entry = it->get();
- auto status =
- _updateRecord(opCtx, entry, oldDoc, newDoc, recordId, keysInsertedOut, keysDeletedOut);
+ auto status = _updateRecord(
+ opCtx, coll, entry, oldDoc, newDoc, recordId, keysInsertedOut, keysDeletedOut);
if (!status.isOK())
return status;
}
@@ -1724,10 +1715,11 @@ BSONObj IndexCatalogImpl::fixIndexKey(const BSONObj& key) const {
}
void IndexCatalogImpl::prepareInsertDeleteOptions(OperationContext* opCtx,
+ const NamespaceString& ns,
const IndexDescriptor* desc,
InsertDeleteOptions* options) const {
auto replCoord = repl::ReplicationCoordinator::get(opCtx);
- if (replCoord->shouldRelaxIndexConstraints(opCtx, desc->parentNS())) {
+ if (replCoord->shouldRelaxIndexConstraints(opCtx, ns)) {
options->getKeysMode = IndexAccessMethod::GetKeysMode::kRelaxConstraints;
} else {
options->getKeysMode = IndexAccessMethod::GetKeysMode::kEnforceConstraints;
@@ -1742,7 +1734,9 @@ void IndexCatalogImpl::prepareInsertDeleteOptions(OperationContext* opCtx,
}
}
-void IndexCatalogImpl::indexBuildSuccess(OperationContext* opCtx, IndexCatalogEntry* index) {
+void IndexCatalogImpl::indexBuildSuccess(OperationContext* opCtx,
+ const Collection* coll,
+ IndexCatalogEntry* index) {
auto releasedEntry = _buildingIndexes.release(index->descriptor());
invariant(releasedEntry.get() == index);
_readyIndexes.add(std::move(releasedEntry));
@@ -1752,7 +1746,7 @@ void IndexCatalogImpl::indexBuildSuccess(OperationContext* opCtx, IndexCatalogEn
index->setIsReady(true);
// Only roll back index changes that are part of pre-existing collections.
- if (UncommittedCollections::getForTxn(opCtx, index->descriptor()->parentNS()) == nullptr) {
+ if (UncommittedCollections::getForTxn(opCtx, coll->ns()) == nullptr) {
opCtx->recoveryUnit()->onRollback([this, index, interceptor]() {
auto releasedEntry = _readyIndexes.release(index->descriptor());
invariant(releasedEntry.get() == index);
diff --git a/src/mongo/db/catalog/index_catalog_impl.h b/src/mongo/db/catalog/index_catalog_impl.h
index fa195004756..2b5ff4f5f4a 100644
--- a/src/mongo/db/catalog/index_catalog_impl.h
+++ b/src/mongo/db/catalog/index_catalog_impl.h
@@ -218,23 +218,8 @@ public:
// ---- modify single index
- /**
- * Returns true if the index 'idx' is multikey, and returns false otherwise.
- */
- bool isMultikey(const IndexDescriptor* const idx) override;
-
- /**
- * Returns the path components that cause the index 'idx' to be multikey if the index supports
- * path-level multikey tracking, and returns an empty vector if path-level multikey tracking
- * isn't supported.
- *
- * If the index supports path-level multikey tracking but isn't multikey, then this function
- * returns a vector with size equal to the number of elements in the index key pattern where
- * each element in the vector is an empty set.
- */
- MultikeyPaths getMultikeyPaths(OperationContext* opCtx, const IndexDescriptor* idx) override;
-
void setMultikeyPaths(OperationContext* const opCtx,
+ Collection* coll,
const IndexDescriptor* desc,
const MultikeyPaths& multikeyPaths) override;
@@ -247,6 +232,7 @@ public:
* This method may throw.
*/
Status indexRecords(OperationContext* opCtx,
+ Collection* coll,
const std::vector<BsonRecord>& bsonRecords,
int64_t* keysInsertedOut) override;
@@ -254,6 +240,7 @@ public:
* See IndexCatalog::updateRecord
*/
Status updateRecord(OperationContext* const opCtx,
+ Collection* coll,
const BSONObj& oldDoc,
const BSONObj& newDoc,
const RecordId& recordId,
@@ -286,10 +273,13 @@ public:
* index constraints, as needed by replication.
*/
void prepareInsertDeleteOptions(OperationContext* opCtx,
+ const NamespaceString&,
const IndexDescriptor* desc,
InsertDeleteOptions* options) const override;
- void indexBuildSuccess(OperationContext* opCtx, IndexCatalogEntry* index) override;
+ void indexBuildSuccess(OperationContext* opCtx,
+ const Collection* collection,
+ IndexCatalogEntry* index) override;
private:
static const BSONObj _idObj; // { _id : 1 }
@@ -303,6 +293,7 @@ private:
std::string _getAccessMethodName(const BSONObj& keyPattern) const;
Status _indexKeys(OperationContext* opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const KeyStringSet& keys,
const KeyStringSet& multikeyMetadataKeys,
@@ -313,16 +304,19 @@ private:
int64_t* keysInsertedOut);
Status _indexFilteredRecords(OperationContext* opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const std::vector<BsonRecord>& bsonRecords,
int64_t* keysInsertedOut);
Status _indexRecords(OperationContext* opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const std::vector<BsonRecord>& bsonRecords,
int64_t* keysInsertedOut);
Status _updateRecord(OperationContext* const opCtx,
+ Collection* coll,
IndexCatalogEntry* index,
const BSONObj& oldDoc,
const BSONObj& newDoc,
diff --git a/src/mongo/db/catalog/index_catalog_noop.h b/src/mongo/db/catalog/index_catalog_noop.h
index 3c66fcd90b1..12b97c208f8 100644
--- a/src/mongo/db/catalog/index_catalog_noop.h
+++ b/src/mongo/db/catalog/index_catalog_noop.h
@@ -192,16 +192,19 @@ public:
}
void setMultikeyPaths(OperationContext* const opCtx,
+ Collection* coll,
const IndexDescriptor* const desc,
const MultikeyPaths& multikeyPaths) override {}
Status indexRecords(OperationContext* const opCtx,
+ Collection* coll,
const std::vector<BsonRecord>& bsonRecords,
int64_t* const keysInsertedOut) override {
return Status::OK();
}
Status updateRecord(OperationContext* const opCtx,
+ Collection* coll,
const BSONObj& oldDoc,
const BSONObj& newDoc,
const RecordId& recordId,
@@ -233,10 +236,13 @@ public:
}
void prepareInsertDeleteOptions(OperationContext* opCtx,
+ const NamespaceString& ns,
const IndexDescriptor* desc,
InsertDeleteOptions* options) const override {}
- void indexBuildSuccess(OperationContext* opCtx, IndexCatalogEntry* index) override {}
+ void indexBuildSuccess(OperationContext* opCtx,
+ const Collection* coll,
+ IndexCatalogEntry* index) override {}
};
} // namespace mongo
diff --git a/src/mongo/db/catalog/index_signature_test.cpp b/src/mongo/db/catalog/index_signature_test.cpp
index b7f50fe72bc..842093f40e5 100644
--- a/src/mongo/db/catalog/index_signature_test.cpp
+++ b/src/mongo/db/catalog/index_signature_test.cpp
@@ -103,7 +103,7 @@ TEST_F(IndexSignatureTest, CanCreateMultipleIndexesOnSameKeyPatternWithDifferent
// the parsed collator rather than the static collation object from the BSON index specs.
auto collationDesc = makeIndexDescriptor(
indexSpec.addFields(fromjson("{collation: {locale: 'en_US', strength: 3}}")));
- ASSERT(collationDesc->compareIndexOptions(opCtx(), basicIndex) ==
+ ASSERT(collationDesc->compareIndexOptions(opCtx(), coll()->ns(), basicIndex) ==
IndexDescriptor::Comparison::kIdentical);
// Confirm that attempting to build this index will result in ErrorCodes::IndexAlreadyExists.
@@ -113,7 +113,7 @@ TEST_F(IndexSignatureTest, CanCreateMultipleIndexesOnSameKeyPatternWithDifferent
// that all signature fields which uniquely identify the index match, but other fields differ.
auto collationUniqueDesc =
makeIndexDescriptor(collationDesc->infoObj().addFields(fromjson("{unique: true}")));
- ASSERT(collationUniqueDesc->compareIndexOptions(opCtx(), basicIndex) ==
+ ASSERT(collationUniqueDesc->compareIndexOptions(opCtx(), coll()->ns(), basicIndex) ==
IndexDescriptor::Comparison::kEquivalent);
// Attempting to build the index, whether with the same name or a different name, now throws
@@ -128,7 +128,7 @@ TEST_F(IndexSignatureTest, CanCreateMultipleIndexesOnSameKeyPatternWithDifferent
// being kDifferent; this means that both of these indexes can co-exist together.
auto differentCollationDesc = makeIndexDescriptor(
collationDesc->infoObj().addFields(fromjson("{collation: {locale: 'fr'}}")));
- ASSERT(differentCollationDesc->compareIndexOptions(opCtx(), basicIndex) ==
+ ASSERT(differentCollationDesc->compareIndexOptions(opCtx(), coll()->ns(), basicIndex) ==
IndexDescriptor::Comparison::kDifferent);
// Verify that we can build this index alongside the existing indexes.
@@ -147,7 +147,7 @@ TEST_F(IndexSignatureTest,
// the index's signature.
auto partialFilterDesc = makeIndexDescriptor(indexSpec.addFields(
fromjson("{partialFilterExpression: {a: {$gt: 5, $lt: 10}, b: 'blah'}}")));
- ASSERT(partialFilterDesc->compareIndexOptions(opCtx(), basicIndex) ==
+ ASSERT(partialFilterDesc->compareIndexOptions(opCtx(), coll()->ns(), basicIndex) ==
IndexDescriptor::Comparison::kDifferent);
// Verify that we can build an index with this spec alongside the original index.
@@ -159,7 +159,7 @@ TEST_F(IndexSignatureTest,
auto partialFilterDupeDesc = makeIndexDescriptor(indexSpec.addFields(
fromjson("{name: 'partialFilter', partialFilterExpression: {$and: [{b: 'blah'}, "
"{a: {$lt: 10}}, {a: {$gt: 5}}]}}")));
- ASSERT(partialFilterDupeDesc->compareIndexOptions(opCtx(), partialFilterIndex) ==
+ ASSERT(partialFilterDupeDesc->compareIndexOptions(opCtx(), coll()->ns(), partialFilterIndex) ==
IndexDescriptor::Comparison::kIdentical);
// Confirm that attempting to build this index will result in ErrorCodes::IndexAlreadyExists.
@@ -169,8 +169,9 @@ TEST_F(IndexSignatureTest,
// that all signature fields which uniquely identify the index match, but other fields differ.
auto partialFilterUniqueDesc =
makeIndexDescriptor(partialFilterDesc->infoObj().addFields(fromjson("{unique: true}")));
- ASSERT(partialFilterUniqueDesc->compareIndexOptions(opCtx(), partialFilterIndex) ==
- IndexDescriptor::Comparison::kEquivalent);
+ ASSERT(
+ partialFilterUniqueDesc->compareIndexOptions(opCtx(), coll()->ns(), partialFilterIndex) ==
+ IndexDescriptor::Comparison::kEquivalent);
// Attempting to build the index, whether with the same name or a different name, now throws
// IndexOptionsConflict. The error message returned with the exception specifies whether the
@@ -186,7 +187,8 @@ TEST_F(IndexSignatureTest,
// compare as kDifferent; this means that both of these indexes can co-exist together.
auto differentPartialFilterDesc = makeIndexDescriptor(partialFilterDesc->infoObj().addFields(
fromjson("{partialFilterExpression: {a: {$gt: 0, $lt: 10}, b: 'blah'}}")));
- ASSERT(differentPartialFilterDesc->compareIndexOptions(opCtx(), partialFilterIndex) ==
+ ASSERT(differentPartialFilterDesc->compareIndexOptions(
+ opCtx(), coll()->ns(), partialFilterIndex) ==
IndexDescriptor::Comparison::kDifferent);
// Verify that we can build this index alongside the existing indexes.
@@ -209,7 +211,7 @@ TEST_F(IndexSignatureTest, CannotCreateMultipleIndexesOnSameKeyPatternIfNonSigna
// the existing index. The two are considered equivalent, and we cannot build the new index.
for (auto&& nonSigOpt : nonSigOptions) {
auto nonSigDesc = makeIndexDescriptor(indexSpec.addFields(nonSigOpt));
- ASSERT(nonSigDesc->compareIndexOptions(opCtx(), basicIndex) ==
+ ASSERT(nonSigDesc->compareIndexOptions(opCtx(), coll()->ns(), basicIndex) ==
IndexDescriptor::Comparison::kEquivalent);
ASSERT_EQ(createIndex(nonSigDesc->infoObj()), ErrorCodes::IndexOptionsConflict);
}
@@ -220,7 +222,7 @@ TEST_F(IndexSignatureTest, CannotCreateMultipleIndexesOnSameKeyPatternIfNonSigna
unittest::assertGet(createIndex(fromjson("{v: 2, name: '$**_1', key: {'$**': 1}}")));
auto nonSigWildcardDesc = makeIndexDescriptor(
wildcardIndex->descriptor()->infoObj().addFields(fromjson("{wildcardProjection: {a: 1}}")));
- ASSERT(nonSigWildcardDesc->compareIndexOptions(opCtx(), wildcardIndex) ==
+ ASSERT(nonSigWildcardDesc->compareIndexOptions(opCtx(), coll()->ns(), wildcardIndex) ==
IndexDescriptor::Comparison::kEquivalent);
ASSERT_EQ(
createIndex(nonSigWildcardDesc->infoObj().addFields(fromjson("{name: 'nonSigWildcard'}"))),
diff --git a/src/mongo/db/catalog/multi_index_block.cpp b/src/mongo/db/catalog/multi_index_block.cpp
index 68f3ea85fb2..3b280b728b0 100644
--- a/src/mongo/db/catalog/multi_index_block.cpp
+++ b/src/mongo/db/catalog/multi_index_block.cpp
@@ -38,6 +38,7 @@
#include "mongo/base/error_codes.h"
#include "mongo/db/audit.h"
#include "mongo/db/catalog/collection.h"
+#include "mongo/db/catalog/collection_catalog.h"
#include "mongo/db/catalog/index_timestamp_helper.h"
#include "mongo/db/catalog/multi_index_block_gen.h"
#include "mongo/db/catalog/uncommitted_collections.h"
@@ -262,7 +263,7 @@ StatusWith<std::vector<BSONObj>> MultiIndexBlock::init(OperationContext* opCtx,
const IndexDescriptor* descriptor = index.block->getEntry()->descriptor();
collection->getIndexCatalog()->prepareInsertDeleteOptions(
- opCtx, descriptor, &index.options);
+ opCtx, collection->ns(), descriptor, &index.options);
// Index builds always relax constraints and check for violations at commit-time.
index.options.getKeysMode = IndexAccessMethod::GetKeysMode::kRelaxConstraints;
@@ -571,6 +572,8 @@ Status MultiIndexBlock::drainBackgroundWrites(
IndexBuildInterceptor::DrainYieldPolicy drainYieldPolicy) {
invariant(!_buildIsCleanedUp);
invariant(!opCtx->lockState()->inAWriteUnitOfWork());
+ const Collection* coll =
+ CollectionCatalog::get(opCtx).lookupCollectionByUUID(opCtx, _collectionUUID.get());
// Drain side-writes table for each index. This only drains what is visible. Assuming intent
// locks are held on the user collection, more writes can come in after this drain completes.
@@ -586,7 +589,7 @@ Status MultiIndexBlock::drainBackgroundWrites(
auto trackDups = !_ignoreUnique ? IndexBuildInterceptor::TrackDuplicates::kTrack
: IndexBuildInterceptor::TrackDuplicates::kNoTrack;
auto status = interceptor->drainWritesIntoIndex(
- opCtx, _indexes[i].options, trackDups, drainYieldPolicy);
+ opCtx, coll, _indexes[i].options, trackDups, drainYieldPolicy);
if (!status.isOK()) {
return status;
}
@@ -670,7 +673,7 @@ Status MultiIndexBlock::commit(OperationContext* opCtx,
if (interceptor) {
auto multikeyPaths = interceptor->getMultikeyPaths();
if (multikeyPaths) {
- _indexes[i].block->getEntry()->setMultikey(opCtx, multikeyPaths.get());
+ _indexes[i].block->getEntry()->setMultikey(opCtx, collection, multikeyPaths.get());
}
}
@@ -679,7 +682,8 @@ Status MultiIndexBlock::commit(OperationContext* opCtx,
// The bulk builder will track multikey information itself.
const auto& bulkBuilder = _indexes[i].bulk;
if (bulkBuilder->isMultikey()) {
- _indexes[i].block->getEntry()->setMultikey(opCtx, bulkBuilder->getMultikeyPaths());
+ _indexes[i].block->getEntry()->setMultikey(
+ opCtx, collection, bulkBuilder->getMultikeyPaths());
}
// The commit() function can be called multiple times on write conflict errors. Dropping the
@@ -693,7 +697,7 @@ Status MultiIndexBlock::commit(OperationContext* opCtx,
onCommit();
opCtx->recoveryUnit()->onCommit([collection, this](boost::optional<Timestamp> commitTime) {
- CollectionQueryInfo::get(collection).clearQueryCache();
+ CollectionQueryInfo::get(collection).clearQueryCache(collection);
_buildIsCleanedUp = true;
});
diff --git a/src/mongo/db/catalog/validate_adaptor.cpp b/src/mongo/db/catalog/validate_adaptor.cpp
index d0be3920043..5ac5b244f53 100644
--- a/src/mongo/db/catalog/validate_adaptor.cpp
+++ b/src/mongo/db/catalog/validate_adaptor.cpp
@@ -97,12 +97,8 @@ Status ValidateAdaptor::validateRecord(OperationContext* opCtx,
const IndexDescriptor* descriptor = index->descriptor();
const IndexAccessMethod* iam = index->accessMethod();
- if (descriptor->isPartial()) {
- const IndexCatalogEntry* ice = indexCatalog->getEntry(descriptor);
- if (!ice->getFilterExpression()->matchesBSON(recordBson)) {
- continue;
- }
- }
+ if (descriptor->isPartial() && !index->getFilterExpression()->matchesBSON(recordBson))
+ continue;
auto documentKeySet = executionCtx.keys();
auto multikeyMetadataKeys = executionCtx.multikeyMetadataKeys();
@@ -118,7 +114,7 @@ Status ValidateAdaptor::validateRecord(OperationContext* opCtx,
recordId,
IndexAccessMethod::kNoopOnSuppressedErrorFn);
- if (!descriptor->isMultikey() &&
+ if (!index->isMultikey() &&
iam->shouldMarkIndexAsMultikey(
documentKeySet->size(),
{multikeyMetadataKeys->begin(), multikeyMetadataKeys->end()},
@@ -134,8 +130,8 @@ Status ValidateAdaptor::validateRecord(OperationContext* opCtx,
}
}
- if (descriptor->isMultikey()) {
- const MultikeyPaths& indexPaths = descriptor->getMultikeyPaths(opCtx);
+ if (index->isMultikey()) {
+ const MultikeyPaths& indexPaths = index->getMultikeyPaths(opCtx);
if (!MultikeyPathTracker::covers(indexPaths, *documentMultikeyPaths.get())) {
std::string msg = str::stream()
<< "Index " << descriptor->indexName()
@@ -409,9 +405,11 @@ void ValidateAdaptor::traverseRecordStore(OperationContext* opCtx,
output->appendNumber("nrecords", _numRecords);
}
-void ValidateAdaptor::validateIndexKeyCount(const IndexDescriptor* idx, ValidateResults& results) {
+void ValidateAdaptor::validateIndexKeyCount(const IndexCatalogEntry* index,
+ ValidateResults& results) {
// Fetch the total number of index entries we previously found traversing the index.
- const std::string indexName = idx->indexName();
+ const IndexDescriptor* desc = index->descriptor();
+ const std::string indexName = desc->indexName();
IndexInfo* indexInfo = &_indexConsistency->getIndexInfo(indexName);
auto numTotalKeys = indexInfo->numKeys;
@@ -419,7 +417,7 @@ void ValidateAdaptor::validateIndexKeyCount(const IndexDescriptor* idx, Validate
bool hasTooFewKeys = false;
bool noErrorOnTooFewKeys = !_validateState->isFullIndexValidation();
- if (idx->isIdIndex() && numTotalKeys != _numRecords) {
+ if (desc->isIdIndex() && numTotalKeys != _numRecords) {
hasTooFewKeys = (numTotalKeys < _numRecords);
std::string msg = str::stream()
<< "number of _id index entries (" << numTotalKeys
@@ -436,10 +434,10 @@ void ValidateAdaptor::validateIndexKeyCount(const IndexDescriptor* idx, Validate
// collection. This check is only valid for indexes that are not multikey (indexed arrays
// produce an index key per array entry) and not $** indexes which can produce index keys for
// multiple paths within a single document.
- if (results.valid && !idx->isMultikey() && idx->getIndexType() != IndexType::INDEX_WILDCARD &&
- numTotalKeys > _numRecords) {
+ if (results.valid && !index->isMultikey() &&
+ desc->getIndexType() != IndexType::INDEX_WILDCARD && numTotalKeys > _numRecords) {
std::string err = str::stream()
- << "index " << idx->indexName() << " is not multi-key, but has more entries ("
+ << "index " << desc->indexName() << " is not multi-key, but has more entries ("
<< numTotalKeys << ") than documents in the index (" << _numRecords << ")";
results.errors.push_back(err);
results.valid = false;
@@ -447,11 +445,11 @@ void ValidateAdaptor::validateIndexKeyCount(const IndexDescriptor* idx, Validate
// Ignore any indexes with a special access method. If an access method name is given, the
// index may be a full text, geo or special index plugin with different semantics.
- if (results.valid && !idx->isSparse() && !idx->isPartial() && !idx->isIdIndex() &&
- idx->getAccessMethodName() == "" && numTotalKeys < _numRecords) {
+ if (results.valid && !desc->isSparse() && !desc->isPartial() && !desc->isIdIndex() &&
+ desc->getAccessMethodName() == "" && numTotalKeys < _numRecords) {
hasTooFewKeys = true;
std::string msg = str::stream()
- << "index " << idx->indexName() << " is not sparse or partial, but has fewer entries ("
+ << "index " << desc->indexName() << " is not sparse or partial, but has fewer entries ("
<< numTotalKeys << ") than documents in the index (" << _numRecords << ")";
if (noErrorOnTooFewKeys) {
results.warnings.push_back(msg);
@@ -463,7 +461,7 @@ void ValidateAdaptor::validateIndexKeyCount(const IndexDescriptor* idx, Validate
if (!_validateState->isFullIndexValidation() && hasTooFewKeys) {
std::string warning = str::stream()
- << "index " << idx->indexName() << " has fewer keys than records."
+ << "index " << desc->indexName() << " has fewer keys than records."
<< " Please re-run the validate command with {full: true}";
results.warnings.push_back(warning);
}
diff --git a/src/mongo/db/catalog/validate_adaptor.h b/src/mongo/db/catalog/validate_adaptor.h
index 87f843db129..89efab68f28 100644
--- a/src/mongo/db/catalog/validate_adaptor.h
+++ b/src/mongo/db/catalog/validate_adaptor.h
@@ -84,7 +84,7 @@ public:
* Validates that the number of document keys matches the number of index keys previously
* traversed in traverseIndex().
*/
- void validateIndexKeyCount(const IndexDescriptor* idx, ValidateResults& results);
+ void validateIndexKeyCount(const IndexCatalogEntry* index, ValidateResults& results);
private:
IndexConsistency* _indexConsistency;
diff --git a/src/mongo/db/commands/count_cmd.cpp b/src/mongo/db/commands/count_cmd.cpp
index 56a07deb865..54b8a808781 100644
--- a/src/mongo/db/commands/count_cmd.cpp
+++ b/src/mongo/db/commands/count_cmd.cpp
@@ -264,7 +264,7 @@ public:
PlanSummaryStats summaryStats;
Explain::getSummaryStats(*exec, &summaryStats);
if (collection) {
- CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, summaryStats);
+ CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, collection, summaryStats);
}
curOp->debug().setPlanSummaryMetrics(summaryStats);
diff --git a/src/mongo/db/commands/distinct.cpp b/src/mongo/db/commands/distinct.cpp
index 52432518498..d614be3de7e 100644
--- a/src/mongo/db/commands/distinct.cpp
+++ b/src/mongo/db/commands/distinct.cpp
@@ -296,7 +296,7 @@ public:
PlanSummaryStats stats;
Explain::getSummaryStats(*executor.getValue(), &stats);
if (collection) {
- CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, stats);
+ CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, collection, stats);
}
curOp->debug().setPlanSummaryMetrics(stats);
diff --git a/src/mongo/db/commands/find_and_modify.cpp b/src/mongo/db/commands/find_and_modify.cpp
index 1b30ce9d5a5..69f418b5137 100644
--- a/src/mongo/db/commands/find_and_modify.cpp
+++ b/src/mongo/db/commands/find_and_modify.cpp
@@ -473,7 +473,7 @@ public:
PlanSummaryStats summaryStats;
Explain::getSummaryStats(*exec, &summaryStats);
if (collection) {
- CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, summaryStats);
+ CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, collection, summaryStats);
}
opDebug->setPlanSummaryMetrics(summaryStats);
@@ -554,7 +554,7 @@ public:
PlanSummaryStats summaryStats;
Explain::getSummaryStats(*exec, &summaryStats);
if (collection) {
- CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, summaryStats);
+ CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, collection, summaryStats);
}
UpdateStage::recordUpdateStatsInOpDebug(UpdateStage::getUpdateStats(exec.get()), opDebug);
opDebug->setPlanSummaryMetrics(summaryStats);
diff --git a/src/mongo/db/commands/run_aggregate.cpp b/src/mongo/db/commands/run_aggregate.cpp
index 242846a3c03..3816d31ae52 100644
--- a/src/mongo/db/commands/run_aggregate.cpp
+++ b/src/mongo/db/commands/run_aggregate.cpp
@@ -780,7 +780,8 @@ Status runAggregate(OperationContext* opCtx,
// For an optimized away pipeline, signal the cache that a query operation has completed.
// For normal pipelines this is done in DocumentSourceCursor.
if (ctx && ctx->getCollection()) {
- CollectionQueryInfo::get(ctx->getCollection()).notifyOfQuery(opCtx, stats);
+ Collection* coll = ctx->getCollection();
+ CollectionQueryInfo::get(coll).notifyOfQuery(opCtx, coll, stats);
}
}
diff --git a/src/mongo/db/exec/count_scan.cpp b/src/mongo/db/exec/count_scan.cpp
index 9e9ec9ab490..fea2fa70c4d 100644
--- a/src/mongo/db/exec/count_scan.cpp
+++ b/src/mongo/db/exec/count_scan.cpp
@@ -73,8 +73,11 @@ const char* CountScan::kStageType = "COUNT_SCAN";
// When building the CountScan stage we take the keyPattern, index name, and multikey details from
// the CountScanParams rather than resolving them via the IndexDescriptor, since these may differ
// from the descriptor's contents.
-CountScan::CountScan(ExpressionContext* expCtx, CountScanParams params, WorkingSet* workingSet)
- : RequiresIndexStage(kStageType, expCtx, params.indexDescriptor, workingSet),
+CountScan::CountScan(ExpressionContext* expCtx,
+ const Collection* collection,
+ CountScanParams params,
+ WorkingSet* workingSet)
+ : RequiresIndexStage(kStageType, expCtx, collection, params.indexDescriptor, workingSet),
_workingSet(workingSet),
_keyPattern(std::move(params.keyPattern)),
_shouldDedup(params.isMultiKey),
diff --git a/src/mongo/db/exec/count_scan.h b/src/mongo/db/exec/count_scan.h
index c8105c713ff..44a092c19b8 100644
--- a/src/mongo/db/exec/count_scan.h
+++ b/src/mongo/db/exec/count_scan.h
@@ -58,8 +58,8 @@ struct CountScanParams {
: CountScanParams(descriptor,
descriptor->indexName(),
descriptor->keyPattern(),
- descriptor->getMultikeyPaths(opCtx),
- descriptor->isMultikey()) {}
+ descriptor->getEntry()->getMultikeyPaths(opCtx),
+ descriptor->getEntry()->isMultikey()) {}
const IndexDescriptor* indexDescriptor;
std::string name;
@@ -87,7 +87,10 @@ struct CountScanParams {
*/
class CountScan final : public RequiresIndexStage {
public:
- CountScan(ExpressionContext* expCtx, CountScanParams params, WorkingSet* workingSet);
+ CountScan(ExpressionContext* expCtx,
+ const Collection* collection,
+ CountScanParams params,
+ WorkingSet* workingSet);
StageState doWork(WorkingSetID* out) final;
bool isEOF() final;
diff --git a/src/mongo/db/exec/distinct_scan.cpp b/src/mongo/db/exec/distinct_scan.cpp
index 6df30d00432..b0644a44c82 100644
--- a/src/mongo/db/exec/distinct_scan.cpp
+++ b/src/mongo/db/exec/distinct_scan.cpp
@@ -46,8 +46,11 @@ using std::vector;
// static
const char* DistinctScan::kStageType = "DISTINCT_SCAN";
-DistinctScan::DistinctScan(ExpressionContext* expCtx, DistinctParams params, WorkingSet* workingSet)
- : RequiresIndexStage(kStageType, expCtx, params.indexDescriptor, workingSet),
+DistinctScan::DistinctScan(ExpressionContext* expCtx,
+ const Collection* collection,
+ DistinctParams params,
+ WorkingSet* workingSet)
+ : RequiresIndexStage(kStageType, expCtx, collection, params.indexDescriptor, workingSet),
_workingSet(workingSet),
_keyPattern(std::move(params.keyPattern)),
_scanDirection(params.scanDirection),
diff --git a/src/mongo/db/exec/distinct_scan.h b/src/mongo/db/exec/distinct_scan.h
index e9c5c5c3852..b4d73e7b407 100644
--- a/src/mongo/db/exec/distinct_scan.h
+++ b/src/mongo/db/exec/distinct_scan.h
@@ -60,8 +60,8 @@ struct DistinctParams {
: DistinctParams(descriptor,
descriptor->indexName(),
descriptor->keyPattern(),
- descriptor->getMultikeyPaths(opCtx),
- descriptor->isMultikey()) {}
+ descriptor->getEntry()->getMultikeyPaths(opCtx),
+ descriptor->getEntry()->isMultikey()) {}
const IndexDescriptor* indexDescriptor;
std::string name;
@@ -95,7 +95,10 @@ struct DistinctParams {
*/
class DistinctScan final : public RequiresIndexStage {
public:
- DistinctScan(ExpressionContext* expCtx, DistinctParams params, WorkingSet* workingSet);
+ DistinctScan(ExpressionContext* expCtx,
+ const Collection* collection,
+ DistinctParams params,
+ WorkingSet* workingSet);
StageState doWork(WorkingSetID* out) final;
bool isEOF() final;
diff --git a/src/mongo/db/exec/geo_near.cpp b/src/mongo/db/exec/geo_near.cpp
index 6d363303963..10f1ff66888 100644
--- a/src/mongo/db/exec/geo_near.cpp
+++ b/src/mongo/db/exec/geo_near.cpp
@@ -266,11 +266,16 @@ static R2Annulus twoDDistanceBounds(const GeoNearParams& nearParams,
return fullBounds;
}
-GeoNear2DStage::DensityEstimator::DensityEstimator(PlanStage::Children* children,
+GeoNear2DStage::DensityEstimator::DensityEstimator(const Collection* collection,
+ PlanStage::Children* children,
BSONObj infoObj,
const GeoNearParams* nearParams,
const R2Annulus& fullBounds)
- : _children(children), _nearParams(nearParams), _fullBounds(fullBounds), _currentLevel(0) {
+ : _collection(collection),
+ _children(children),
+ _nearParams(nearParams),
+ _fullBounds(fullBounds),
+ _currentLevel(0) {
// The index status should always be valid.
auto result = invariantStatusOK(GeoHashConverter::createFromDoc(std::move(infoObj)));
@@ -320,7 +325,7 @@ void GeoNear2DStage::DensityEstimator::buildIndexScan(ExpressionContext* expCtx,
IndexBoundsBuilder::intersectize(oil, &scanParams.bounds.fields[twoDFieldPosition]);
invariant(!_indexScan);
- _indexScan = new IndexScan(expCtx, scanParams, workingSet, nullptr);
+ _indexScan = new IndexScan(expCtx, _collection, scanParams, workingSet, nullptr);
_children->emplace_back(_indexScan);
}
@@ -406,7 +411,7 @@ PlanStage::StageState GeoNear2DStage::initialize(OperationContext* opCtx,
WorkingSetID* out) {
if (!_densityEstimator) {
_densityEstimator.reset(new DensityEstimator(
- &_children, indexDescriptor()->infoObj(), &_nearParams, _fullBounds));
+ collection(), &_children, indexDescriptor()->infoObj(), &_nearParams, _fullBounds));
}
double estimatedDistance;
@@ -449,8 +454,14 @@ static const string kTwoDIndexNearStage("GEO_NEAR_2D");
GeoNear2DStage::GeoNear2DStage(const GeoNearParams& nearParams,
ExpressionContext* expCtx,
WorkingSet* workingSet,
+ const Collection* collection,
const IndexDescriptor* twoDIndex)
- : NearStage(expCtx, kTwoDIndexNearStage.c_str(), STAGE_GEO_NEAR_2D, workingSet, twoDIndex),
+ : NearStage(expCtx,
+ kTwoDIndexNearStage.c_str(),
+ STAGE_GEO_NEAR_2D,
+ workingSet,
+ collection,
+ twoDIndex),
_nearParams(nearParams),
_fullBounds(twoDDistanceBounds(nearParams, twoDIndex)),
_currBounds(_fullBounds.center(), -1, _fullBounds.getInner()),
@@ -693,7 +704,8 @@ std::unique_ptr<NearStage::CoveredInterval> GeoNear2DStage::nextInterval(
invariantStatusOK(GeoHashConverter::createFromDoc(indexDescriptor()->infoObj()));
// 2D indexes support covered search over additional fields they contain
- auto scan = std::make_unique<IndexScan>(expCtx(), scanParams, workingSet, _nearParams.filter);
+ auto scan = std::make_unique<IndexScan>(
+ expCtx(), collection, scanParams, workingSet, _nearParams.filter);
MatchExpression* docMatcher = nullptr;
@@ -741,8 +753,14 @@ static const string kS2IndexNearStage("GEO_NEAR_2DSPHERE");
GeoNear2DSphereStage::GeoNear2DSphereStage(const GeoNearParams& nearParams,
ExpressionContext* expCtx,
WorkingSet* workingSet,
+ const Collection* collection,
const IndexDescriptor* s2Index)
- : NearStage(expCtx, kS2IndexNearStage.c_str(), STAGE_GEO_NEAR_2DSPHERE, workingSet, s2Index),
+ : NearStage(expCtx,
+ kS2IndexNearStage.c_str(),
+ STAGE_GEO_NEAR_2DSPHERE,
+ workingSet,
+ collection,
+ s2Index),
_nearParams(nearParams),
_fullBounds(geoNearDistanceBounds(*nearParams.nearQuery)),
_currBounds(_fullBounds.center(), -1, _fullBounds.getInner()),
@@ -759,8 +777,6 @@ GeoNear2DSphereStage::GeoNear2DSphereStage(const GeoNearParams& nearParams,
ExpressionParams::initialize2dsphereParams(s2Index->infoObj(), collator, &_indexParams);
}
-GeoNear2DSphereStage::~GeoNear2DSphereStage() {}
-
namespace {
S2Region* buildS2Region(const R2Annulus& sphereBounds) {
@@ -802,11 +818,13 @@ S2Region* buildS2Region(const R2Annulus& sphereBounds) {
}
} // namespace
-GeoNear2DSphereStage::DensityEstimator::DensityEstimator(PlanStage::Children* children,
+GeoNear2DSphereStage::DensityEstimator::DensityEstimator(const Collection* collection,
+ PlanStage::Children* children,
const GeoNearParams* nearParams,
const S2IndexingParams& indexParams,
const R2Annulus& fullBounds)
- : _children(children),
+ : _collection(collection),
+ _children(children),
_nearParams(nearParams),
_indexParams(indexParams),
_fullBounds(fullBounds),
@@ -844,7 +862,7 @@ void GeoNear2DSphereStage::DensityEstimator::buildIndexScan(ExpressionContext* e
// Index scan
invariant(!_indexScan);
- _indexScan = new IndexScan(expCtx, scanParams, workingSet, nullptr);
+ _indexScan = new IndexScan(expCtx, _collection, scanParams, workingSet, nullptr);
_children->emplace_back(_indexScan);
}
@@ -930,8 +948,8 @@ PlanStage::StageState GeoNear2DSphereStage::initialize(OperationContext* opCtx,
WorkingSet* workingSet,
WorkingSetID* out) {
if (!_densityEstimator) {
- _densityEstimator.reset(
- new DensityEstimator(&_children, &_nearParams, _indexParams, _fullBounds));
+ _densityEstimator.reset(new DensityEstimator(
+ collection(), &_children, &_nearParams, _indexParams, _fullBounds));
}
double estimatedDistance;
@@ -1022,7 +1040,7 @@ std::unique_ptr<NearStage::CoveredInterval> GeoNear2DSphereStage::nextInterval(
OrderedIntervalList* coveredIntervals = &scanParams.bounds.fields[s2FieldPosition];
ExpressionMapping::S2CellIdsToIntervalsWithParents(cover, _indexParams, coveredIntervals);
- auto scan = std::make_unique<IndexScan>(expCtx(), scanParams, workingSet, nullptr);
+ auto scan = std::make_unique<IndexScan>(expCtx(), collection, scanParams, workingSet, nullptr);
// FetchStage owns index scan
_children.emplace_back(std::make_unique<FetchStage>(
diff --git a/src/mongo/db/exec/geo_near.h b/src/mongo/db/exec/geo_near.h
index dd3d33be97d..08001ecc77c 100644
--- a/src/mongo/db/exec/geo_near.h
+++ b/src/mongo/db/exec/geo_near.h
@@ -71,6 +71,7 @@ public:
GeoNear2DStage(const GeoNearParams& nearParams,
ExpressionContext* expCtx,
WorkingSet* workingSet,
+ const Collection* collection,
const IndexDescriptor* twoDIndex);
protected:
@@ -87,7 +88,8 @@ protected:
private:
class DensityEstimator {
public:
- DensityEstimator(PlanStage::Children* children,
+ DensityEstimator(const Collection* collection,
+ PlanStage::Children* children,
BSONObj infoObj,
const GeoNearParams* nearParams,
const R2Annulus& fullBounds);
@@ -103,6 +105,7 @@ private:
WorkingSet* workingSet,
const IndexDescriptor* twoDIndex);
+ const Collection* _collection;
PlanStage::Children* _children; // Points to PlanStage::_children in the NearStage.
const GeoNearParams* _nearParams; // Not owned here.
const R2Annulus& _fullBounds;
@@ -137,10 +140,9 @@ public:
GeoNear2DSphereStage(const GeoNearParams& nearParams,
ExpressionContext* expCtx,
WorkingSet* workingSet,
+ const Collection* collection,
const IndexDescriptor* s2Index);
- ~GeoNear2DSphereStage();
-
protected:
std::unique_ptr<CoveredInterval> nextInterval(OperationContext* opCtx,
WorkingSet* workingSet,
@@ -156,7 +158,8 @@ private:
// Estimate the density of data by search the nearest cells level by level around center.
class DensityEstimator {
public:
- DensityEstimator(PlanStage::Children* children,
+ DensityEstimator(const Collection* collection,
+ PlanStage::Children* children,
const GeoNearParams* nearParams,
const S2IndexingParams& indexParams,
const R2Annulus& fullBounds);
@@ -174,6 +177,7 @@ private:
WorkingSet* workingSet,
const IndexDescriptor* s2Index);
+ const Collection* _collection;
PlanStage::Children* _children; // Points to PlanStage::_children in the NearStage.
const GeoNearParams* _nearParams; // Not owned here.
const S2IndexingParams _indexParams;
diff --git a/src/mongo/db/exec/idhack.cpp b/src/mongo/db/exec/idhack.cpp
index d8b9400daba..f8992a49ff5 100644
--- a/src/mongo/db/exec/idhack.cpp
+++ b/src/mongo/db/exec/idhack.cpp
@@ -52,8 +52,9 @@ const char* IDHackStage::kStageType = "IDHACK";
IDHackStage::IDHackStage(ExpressionContext* expCtx,
CanonicalQuery* query,
WorkingSet* ws,
+ const Collection* collection,
const IndexDescriptor* descriptor)
- : RequiresIndexStage(kStageType, expCtx, descriptor, ws),
+ : RequiresIndexStage(kStageType, expCtx, collection, descriptor, ws),
_workingSet(ws),
_key(query->getQueryObj()["_id"].wrap()) {
_specificStats.indexName = descriptor->indexName();
@@ -63,8 +64,11 @@ IDHackStage::IDHackStage(ExpressionContext* expCtx,
IDHackStage::IDHackStage(ExpressionContext* expCtx,
const BSONObj& key,
WorkingSet* ws,
+ const Collection* collection,
const IndexDescriptor* descriptor)
- : RequiresIndexStage(kStageType, expCtx, descriptor, ws), _workingSet(ws), _key(key) {
+ : RequiresIndexStage(kStageType, expCtx, collection, descriptor, ws),
+ _workingSet(ws),
+ _key(key) {
_specificStats.indexName = descriptor->indexName();
}
diff --git a/src/mongo/db/exec/idhack.h b/src/mongo/db/exec/idhack.h
index 5c2ac9f894a..edb5b88e449 100644
--- a/src/mongo/db/exec/idhack.h
+++ b/src/mongo/db/exec/idhack.h
@@ -51,11 +51,13 @@ public:
IDHackStage(ExpressionContext* expCtx,
CanonicalQuery* query,
WorkingSet* ws,
+ const Collection* collection,
const IndexDescriptor* descriptor);
IDHackStage(ExpressionContext* expCtx,
const BSONObj& key,
WorkingSet* ws,
+ const Collection* collection,
const IndexDescriptor* descriptor);
~IDHackStage();
diff --git a/src/mongo/db/exec/index_scan.cpp b/src/mongo/db/exec/index_scan.cpp
index 975be60f241..62cc788b790 100644
--- a/src/mongo/db/exec/index_scan.cpp
+++ b/src/mongo/db/exec/index_scan.cpp
@@ -60,10 +60,11 @@ namespace mongo {
const char* IndexScan::kStageType = "IXSCAN";
IndexScan::IndexScan(ExpressionContext* expCtx,
+ const Collection* collection,
IndexScanParams params,
WorkingSet* workingSet,
const MatchExpression* filter)
- : RequiresIndexStage(kStageType, expCtx, params.indexDescriptor, workingSet),
+ : RequiresIndexStage(kStageType, expCtx, collection, params.indexDescriptor, workingSet),
_workingSet(workingSet),
_keyPattern(params.keyPattern.getOwned()),
_bounds(std::move(params.bounds)),
diff --git a/src/mongo/db/exec/index_scan.h b/src/mongo/db/exec/index_scan.h
index d36f99d9f9b..cad8df9b9a0 100644
--- a/src/mongo/db/exec/index_scan.h
+++ b/src/mongo/db/exec/index_scan.h
@@ -59,8 +59,8 @@ struct IndexScanParams {
: IndexScanParams(descriptor,
descriptor->indexName(),
descriptor->keyPattern(),
- descriptor->getMultikeyPaths(opCtx),
- descriptor->isMultikey()) {}
+ descriptor->getEntry()->getMultikeyPaths(opCtx),
+ descriptor->getEntry()->isMultikey()) {}
const IndexDescriptor* indexDescriptor;
@@ -109,6 +109,7 @@ public:
};
IndexScan(ExpressionContext* expCtx,
+ const Collection* collection,
IndexScanParams params,
WorkingSet* workingSet,
const MatchExpression* filter);
diff --git a/src/mongo/db/exec/near.cpp b/src/mongo/db/exec/near.cpp
index f5bd5a7c157..94af9d88f74 100644
--- a/src/mongo/db/exec/near.cpp
+++ b/src/mongo/db/exec/near.cpp
@@ -46,8 +46,9 @@ NearStage::NearStage(ExpressionContext* expCtx,
const char* typeName,
StageType type,
WorkingSet* workingSet,
+ const Collection* collection,
const IndexDescriptor* indexDescriptor)
- : RequiresIndexStage(typeName, expCtx, indexDescriptor, workingSet),
+ : RequiresIndexStage(typeName, expCtx, collection, indexDescriptor, workingSet),
_workingSet(workingSet),
_searchState(SearchState_Initializing),
_nextIntervalStats(nullptr),
diff --git a/src/mongo/db/exec/near.h b/src/mongo/db/exec/near.h
index bbbbee686e3..fef83f58640 100644
--- a/src/mongo/db/exec/near.h
+++ b/src/mongo/db/exec/near.h
@@ -108,6 +108,7 @@ protected:
const char* typeName,
StageType type,
WorkingSet* workingSet,
+ const Collection* collection,
const IndexDescriptor* indexDescriptor);
//
diff --git a/src/mongo/db/exec/requires_index_stage.cpp b/src/mongo/db/exec/requires_index_stage.cpp
index 1073330fcf0..f1b43466822 100644
--- a/src/mongo/db/exec/requires_index_stage.cpp
+++ b/src/mongo/db/exec/requires_index_stage.cpp
@@ -35,10 +35,11 @@ namespace mongo {
RequiresIndexStage::RequiresIndexStage(const char* stageType,
ExpressionContext* expCtx,
+ const Collection* collection,
const IndexDescriptor* indexDescriptor,
WorkingSet* workingSet)
- : RequiresCollectionStage(stageType, expCtx, indexDescriptor->getCollection()),
- _weakIndexCatalogEntry(collection()->getIndexCatalog()->getEntryShared(indexDescriptor)) {
+ : RequiresCollectionStage(stageType, expCtx, collection),
+ _weakIndexCatalogEntry(indexDescriptor->getEntry()->shared_from_this()) {
auto indexCatalogEntry = _weakIndexCatalogEntry.lock();
_indexDescriptor = indexCatalogEntry->descriptor();
_indexAccessMethod = indexCatalogEntry->accessMethod();
diff --git a/src/mongo/db/exec/requires_index_stage.h b/src/mongo/db/exec/requires_index_stage.h
index 374dc5f0615..f8e1e7a3b88 100644
--- a/src/mongo/db/exec/requires_index_stage.h
+++ b/src/mongo/db/exec/requires_index_stage.h
@@ -49,6 +49,7 @@ class RequiresIndexStage : public RequiresCollectionStage {
public:
RequiresIndexStage(const char* stageType,
ExpressionContext* expCtx,
+ const Collection* collection,
const IndexDescriptor* indexDescriptor,
WorkingSet* workingSet);
diff --git a/src/mongo/db/exec/stagedebug_cmd.cpp b/src/mongo/db/exec/stagedebug_cmd.cpp
index 6d1ffe1ea11..31556bc0217 100644
--- a/src/mongo/db/exec/stagedebug_cmd.cpp
+++ b/src/mongo/db/exec/stagedebug_cmd.cpp
@@ -287,9 +287,9 @@ public:
params.bounds.boundInclusion = IndexBounds::makeBoundInclusionFromBoundBools(
nodeArgs["startKeyInclusive"].Bool(), nodeArgs["endKeyInclusive"].Bool());
params.direction = nodeArgs["direction"].numberInt();
- params.shouldDedup = desc->isMultikey();
+ params.shouldDedup = desc->getEntry()->isMultikey();
- return new IndexScan(expCtx.get(), params, workingSet, matcher);
+ return new IndexScan(expCtx.get(), collection, params, workingSet, matcher);
} else if ("andHash" == nodeName) {
uassert(
16921, "Nodes argument must be provided to AND", nodeArgs["nodes"].isABSONObj());
@@ -466,7 +466,7 @@ public:
return nullptr;
}
- return new TextStage(expCtx.get(), params, workingSet, matcher);
+ return new TextStage(expCtx.get(), collection, params, workingSet, matcher);
} else if ("delete" == nodeName) {
uassert(18636,
"Delete stage doesn't have a filter (put it on the child)",
diff --git a/src/mongo/db/exec/text.cpp b/src/mongo/db/exec/text.cpp
index 7b7e4598e43..2657556af7e 100644
--- a/src/mongo/db/exec/text.cpp
+++ b/src/mongo/db/exec/text.cpp
@@ -57,11 +57,13 @@ using fts::MAX_WEIGHT;
const char* TextStage::kStageType = "TEXT";
TextStage::TextStage(ExpressionContext* expCtx,
+ const Collection* collection,
const TextStageParams& params,
WorkingSet* ws,
const MatchExpression* filter)
: PlanStage(kStageType, expCtx), _params(params) {
- _children.emplace_back(buildTextTree(expCtx->opCtx, ws, filter, params.wantTextScore));
+ _children.emplace_back(
+ buildTextTree(expCtx->opCtx, collection, ws, filter, params.wantTextScore));
_specificStats.indexPrefix = _params.indexPrefix;
_specificStats.indexName = _params.index->indexName();
_specificStats.parsedTextQuery = _params.query.toBSON();
@@ -94,11 +96,10 @@ const SpecificStats* TextStage::getSpecificStats() const {
}
unique_ptr<PlanStage> TextStage::buildTextTree(OperationContext* opCtx,
+ const Collection* collection,
WorkingSet* ws,
const MatchExpression* filter,
bool wantTextScore) const {
- const auto* collection = _params.index->getCollection();
-
// Get all the index scans for each term in our query.
std::vector<std::unique_ptr<PlanStage>> indexScanList;
for (const auto& term : _params.query.getTermsForBounds()) {
@@ -110,9 +111,10 @@ unique_ptr<PlanStage> TextStage::buildTextTree(OperationContext* opCtx,
ixparams.bounds.boundInclusion = BoundInclusion::kIncludeBothStartAndEndKeys;
ixparams.bounds.isSimpleRange = true;
ixparams.direction = -1;
- ixparams.shouldDedup = _params.index->isMultikey();
+ ixparams.shouldDedup = _params.index->getEntry()->isMultikey();
- indexScanList.push_back(std::make_unique<IndexScan>(expCtx(), ixparams, ws, nullptr));
+ indexScanList.push_back(
+ std::make_unique<IndexScan>(expCtx(), collection, ixparams, ws, nullptr));
}
// Build the union of the index scans as a TEXT_OR or an OR stage, depending on whether the
diff --git a/src/mongo/db/exec/text.h b/src/mongo/db/exec/text.h
index 1e601d769de..9c0c54baadd 100644
--- a/src/mongo/db/exec/text.h
+++ b/src/mongo/db/exec/text.h
@@ -74,6 +74,7 @@ struct TextStageParams {
class TextStage final : public PlanStage {
public:
TextStage(ExpressionContext* expCtx,
+ const Collection* collection,
const TextStageParams& params,
WorkingSet* ws,
const MatchExpression* filter);
@@ -96,6 +97,7 @@ private:
* Helper method to built the query execution plan for the text stage.
*/
std::unique_ptr<PlanStage> buildTextTree(OperationContext* opCtx,
+ const Collection* collection,
WorkingSet* ws,
const MatchExpression* filter,
bool wantTextScore) const;
diff --git a/src/mongo/db/index/btree_access_method.cpp b/src/mongo/db/index/btree_access_method.cpp
index 6d262346dc9..9d2f8bfd755 100644
--- a/src/mongo/db/index/btree_access_method.cpp
+++ b/src/mongo/db/index/btree_access_method.cpp
@@ -72,8 +72,8 @@ void BtreeAccessMethod::doGetKeys(SharedBufferFragmentBuilder& pooledBufferBuild
KeyStringSet* multikeyMetadataKeys,
MultikeyPaths* multikeyPaths,
boost::optional<RecordId> id) const {
- const auto skipMultikey =
- context == IndexAccessMethod::GetKeysContext::kValidatingKeys && !_descriptor->isMultikey();
+ const auto skipMultikey = context == IndexAccessMethod::GetKeysContext::kValidatingKeys &&
+ !_descriptor->getEntry()->isMultikey();
_keyGenerator->getKeys(pooledBufferBuilder, obj, skipMultikey, keys, multikeyPaths, id);
}
diff --git a/src/mongo/db/index/index_access_method.cpp b/src/mongo/db/index/index_access_method.cpp
index 3543901be12..00980d9905f 100644
--- a/src/mongo/db/index/index_access_method.cpp
+++ b/src/mongo/db/index/index_access_method.cpp
@@ -121,6 +121,7 @@ bool AbstractIndexAccessMethod::isFatalError(OperationContext* opCtx,
// Find the keys for obj, put them in the tree pointing to loc.
Status AbstractIndexAccessMethod::insert(OperationContext* opCtx,
+ const Collection* coll,
const BSONObj& obj,
const RecordId& loc,
const InsertDeleteOptions& options,
@@ -143,10 +144,12 @@ Status AbstractIndexAccessMethod::insert(OperationContext* opCtx,
loc,
kNoopOnSuppressedErrorFn);
- return insertKeys(opCtx, *keys, *multikeyMetadataKeys, *multikeyPaths, loc, options, result);
+ return insertKeys(
+ opCtx, coll, *keys, *multikeyMetadataKeys, *multikeyPaths, loc, options, result);
}
Status AbstractIndexAccessMethod::insertKeys(OperationContext* opCtx,
+ const Collection* coll,
const KeyStringSet& keys,
const KeyStringSet& multikeyMetadataKeys,
const MultikeyPaths& multikeyPaths,
@@ -184,7 +187,7 @@ Status AbstractIndexAccessMethod::insertKeys(OperationContext* opCtx,
}
if (shouldMarkIndexAsMultikey(keys.size(), multikeyMetadataKeys, multikeyPaths)) {
- _indexCatalogEntry->setMultikey(opCtx, multikeyPaths);
+ _indexCatalogEntry->setMultikey(opCtx, coll, multikeyPaths);
}
return Status::OK();
}
@@ -197,15 +200,16 @@ void AbstractIndexAccessMethod::removeOneKey(OperationContext* opCtx,
try {
_newInterface->unindex(opCtx, keyString, dupsAllowed);
} catch (AssertionException& e) {
+ NamespaceString ns = _indexCatalogEntry->getNSSFromCatalog(opCtx);
LOGV2(20683,
- "Assertion failure: _unindex failed on: {descriptorParentNamespace} for index: "
- "{descriptorIndexName}. {error} KeyString:{keyString} dl:{recordId}",
+ "Assertion failure: _unindex failed on: {namespace} for index: {indexName}. "
+ "{error} KeyString:{keyString} dl:{recordId}",
"Assertion failure: _unindex failed",
"error"_attr = redact(e),
"keyString"_attr = keyString,
"recordId"_attr = loc,
- "descriptorParentNamespace"_attr = _descriptor->parentNS(),
- "descriptorIndexName"_attr = _descriptor->indexName());
+ "namespace"_attr = ns,
+ "indexName"_attr = _descriptor->indexName());
printStackTrace();
}
}
@@ -396,6 +400,7 @@ void AbstractIndexAccessMethod::prepareUpdate(OperationContext* opCtx,
}
Status AbstractIndexAccessMethod::update(OperationContext* opCtx,
+ Collection* coll,
const UpdateTicket& ticket,
int64_t* numInserted,
int64_t* numDeleted) {
@@ -430,7 +435,7 @@ Status AbstractIndexAccessMethod::update(OperationContext* opCtx,
if (shouldMarkIndexAsMultikey(
ticket.newKeys.size(), ticket.newMultikeyMetadataKeys, ticket.newMultikeyPaths)) {
- _indexCatalogEntry->setMultikey(opCtx, ticket.newMultikeyPaths);
+ _indexCatalogEntry->setMultikey(opCtx, coll, ticket.newMultikeyPaths);
}
*numDeleted = ticket.removed.size();
@@ -650,7 +655,7 @@ Status AbstractIndexAccessMethod::commitBulk(OperationContext* opCtx,
auto dupKey =
KeyString::toBson(data.first, getSortedDataInterface()->getOrdering());
return buildDupKeyErrorStatus(dupKey.getOwned(),
- _descriptor->parentNS(),
+ _indexCatalogEntry->getNSSFromCatalog(opCtx),
_descriptor->indexName(),
_descriptor->keyPattern(),
_descriptor->collation());
@@ -683,7 +688,7 @@ Status AbstractIndexAccessMethod::commitBulk(OperationContext* opCtx,
"Index build: inserted {bulk_getKeysInserted} keys from external sorter into index in "
"{timer_seconds} seconds",
"Index build: inserted keys from external sorter into index",
- "namespace"_attr = _descriptor->parentNS(),
+ "namespace"_attr = _indexCatalogEntry->getNSSFromCatalog(opCtx),
"index"_attr = _descriptor->indexName(),
"keysInserted"_attr = bulk->getKeysInserted(),
"duration"_attr = Milliseconds(Seconds(timer.seconds())));
@@ -694,8 +699,10 @@ Status AbstractIndexAccessMethod::commitBulk(OperationContext* opCtx,
return Status::OK();
}
-void AbstractIndexAccessMethod::setIndexIsMultikey(OperationContext* opCtx, MultikeyPaths paths) {
- _indexCatalogEntry->setMultikey(opCtx, paths);
+void AbstractIndexAccessMethod::setIndexIsMultikey(OperationContext* opCtx,
+ Collection* collection,
+ MultikeyPaths paths) {
+ _indexCatalogEntry->setMultikey(opCtx, collection, paths);
}
IndexAccessMethod::OnSuppressedErrorFn IndexAccessMethod::kNoopOnSuppressedErrorFn =
diff --git a/src/mongo/db/index/index_access_method.h b/src/mongo/db/index/index_access_method.h
index e2e125c0f7b..bf41334ce1f 100644
--- a/src/mongo/db/index/index_access_method.h
+++ b/src/mongo/db/index/index_access_method.h
@@ -85,12 +85,14 @@ public:
* The behavior of the insertion can be specified through 'options'.
*/
virtual Status insert(OperationContext* opCtx,
+ const Collection* coll,
const BSONObj& obj,
const RecordId& loc,
const InsertDeleteOptions& options,
InsertResult* result) = 0;
virtual Status insertKeys(OperationContext* opCtx,
+ const Collection* coll,
const KeyStringSet& keys,
const KeyStringSet& multikeyMetadataKeys,
const MultikeyPaths& multikeyPaths,
@@ -132,6 +134,7 @@ public:
* 'numDeleted' will be set to the number of keys removed from the index for the document.
*/
virtual Status update(OperationContext* opCtx,
+ Collection* coll,
const UpdateTicket& ticket,
int64_t* numInserted,
int64_t* numDeleted) = 0;
@@ -190,7 +193,9 @@ public:
/**
* Sets this index as multikey with the provided paths.
*/
- virtual void setIndexIsMultikey(OperationContext* opCtx, MultikeyPaths paths) = 0;
+ virtual void setIndexIsMultikey(OperationContext* opCtx,
+ Collection* collection,
+ MultikeyPaths paths) = 0;
//
// Bulk operations support
@@ -450,12 +455,14 @@ public:
std::unique_ptr<SortedDataInterface> btree);
Status insert(OperationContext* opCtx,
+ const Collection* coll,
const BSONObj& obj,
const RecordId& loc,
const InsertDeleteOptions& options,
InsertResult* result) final;
Status insertKeys(OperationContext* opCtx,
+ const Collection* coll,
const KeyStringSet& keys,
const KeyStringSet& multikeyMetadataKeys,
const MultikeyPaths& multikeyPaths,
@@ -478,6 +485,7 @@ public:
UpdateTicket* ticket) const final;
Status update(OperationContext* opCtx,
+ Collection* coll,
const UpdateTicket& ticket,
int64_t* numInserted,
int64_t* numDeleted) final;
@@ -502,7 +510,9 @@ public:
Status compact(OperationContext* opCtx) final;
- void setIndexIsMultikey(OperationContext* opCtx, MultikeyPaths paths) final;
+ void setIndexIsMultikey(OperationContext* opCtx,
+ Collection* collection,
+ MultikeyPaths paths) final;
std::unique_ptr<BulkBuilder> initiateBulk(size_t maxMemoryUsageBytes) final;
diff --git a/src/mongo/db/index/index_build_interceptor.cpp b/src/mongo/db/index/index_build_interceptor.cpp
index db881539abd..1c93f6dadc4 100644
--- a/src/mongo/db/index/index_build_interceptor.cpp
+++ b/src/mongo/db/index/index_build_interceptor.cpp
@@ -111,6 +111,7 @@ Status IndexBuildInterceptor::checkDuplicateKeyConstraints(OperationContext* opC
}
Status IndexBuildInterceptor::drainWritesIntoIndex(OperationContext* opCtx,
+ const Collection* coll,
const InsertDeleteOptions& options,
TrackDuplicates trackDuplicates,
DrainYieldPolicy drainYieldPolicy) {
@@ -189,8 +190,13 @@ Status IndexBuildInterceptor::drainWritesIntoIndex(OperationContext* opCtx,
batchSize += 1;
batchSizeBytes += objSize;
- if (auto status = _applyWrite(
- opCtx, unownedDoc, options, trackDuplicates, &totalInserted, &totalDeleted);
+ if (auto status = _applyWrite(opCtx,
+ coll,
+ unownedDoc,
+ options,
+ trackDuplicates,
+ &totalInserted,
+ &totalDeleted);
!status.isOK()) {
return status;
}
@@ -234,8 +240,8 @@ Status IndexBuildInterceptor::drainWritesIntoIndex(OperationContext* opCtx,
// Apply batches of side writes until the last record in the table is seen.
while (!atEof) {
- if (auto status = writeConflictRetry(
- opCtx, "index build drain", _indexCatalogEntry->ns().ns(), applySingleBatch);
+ if (auto status =
+ writeConflictRetry(opCtx, "index build drain", coll->ns().ns(), applySingleBatch);
!status.isOK()) {
return status;
}
@@ -257,6 +263,7 @@ Status IndexBuildInterceptor::drainWritesIntoIndex(OperationContext* opCtx,
}
Status IndexBuildInterceptor::_applyWrite(OperationContext* opCtx,
+ const Collection* coll,
const BSONObj& operation,
const InsertDeleteOptions& options,
TrackDuplicates trackDups,
@@ -281,6 +288,7 @@ Status IndexBuildInterceptor::_applyWrite(OperationContext* opCtx,
if (opType == Op::kInsert) {
InsertResult result;
auto status = accessMethod->insertKeys(opCtx,
+ coll,
{keySet.begin(), keySet.end()},
{},
MultikeyPaths{},
@@ -339,7 +347,8 @@ void IndexBuildInterceptor::_yield(OperationContext* opCtx) {
hangDuringIndexBuildDrainYield.pauseWhileSet();
},
[&](auto&& config) {
- return config.getStringField("namespace") == _indexCatalogEntry->ns().ns();
+ return config.getStringField("namespace") ==
+ _indexCatalogEntry->getNSSFromCatalog(opCtx).ns();
});
locker->restoreLockState(opCtx, snapshot);
diff --git a/src/mongo/db/index/index_build_interceptor.h b/src/mongo/db/index/index_build_interceptor.h
index 563237df411..c6d282a3f61 100644
--- a/src/mongo/db/index/index_build_interceptor.h
+++ b/src/mongo/db/index/index_build_interceptor.h
@@ -118,6 +118,7 @@ public:
* following the last inserted record from a previous call to drainWritesIntoIndex.
*/
Status drainWritesIntoIndex(OperationContext* opCtx,
+ const Collection* coll,
const InsertDeleteOptions& options,
TrackDuplicates trackDups,
DrainYieldPolicy drainYieldPolicy);
@@ -162,6 +163,7 @@ private:
using SideWriteRecord = std::pair<RecordId, BSONObj>;
Status _applyWrite(OperationContext* opCtx,
+ const Collection* coll,
const BSONObj& doc,
const InsertDeleteOptions& options,
TrackDuplicates trackDups,
diff --git a/src/mongo/db/index/index_descriptor.cpp b/src/mongo/db/index/index_descriptor.cpp
index ea08ed5094f..7c16ae58040 100644
--- a/src/mongo/db/index/index_descriptor.cpp
+++ b/src/mongo/db/index/index_descriptor.cpp
@@ -102,8 +102,7 @@ constexpr StringData IndexDescriptor::kWeightsFieldName;
IndexDescriptor::IndexDescriptor(Collection* collection,
const std::string& accessMethodName,
BSONObj infoObj)
- : _collection(collection),
- _accessMethodName(accessMethodName),
+ : _accessMethodName(accessMethodName),
_indexType(IndexNames::nameToType(accessMethodName)),
_infoObj(infoObj.getOwned()),
_numFields(infoObj.getObjectField(IndexDescriptor::kKeyPatternFieldName).nFields()),
@@ -114,8 +113,7 @@ IndexDescriptor::IndexDescriptor(Collection* collection,
_sparse(infoObj[IndexDescriptor::kSparseFieldName].trueValue()),
_unique(_isIdIndex || infoObj[kUniqueFieldName].trueValue()),
_hidden(infoObj[kHiddenFieldName].trueValue()),
- _partial(!infoObj[kPartialFilterExprFieldName].eoo()),
- _cachedEntry(nullptr) {
+ _partial(!infoObj[kPartialFilterExprFieldName].eoo()) {
BSONElement e = _infoObj[IndexDescriptor::kIndexVersionFieldName];
fassert(50942, e.isNumber());
_version = static_cast<IndexVersion>(e.numberInt());
@@ -162,24 +160,8 @@ IndexVersion IndexDescriptor::getDefaultIndexVersion() {
return IndexVersion::kV2;
}
-bool IndexDescriptor::isMultikey() const {
- return _collection->getIndexCatalog()->isMultikey(this);
-}
-
-MultikeyPaths IndexDescriptor::getMultikeyPaths(OperationContext* opCtx) const {
- return _collection->getIndexCatalog()->getMultikeyPaths(opCtx, this);
-}
-
-const IndexCatalog* IndexDescriptor::getIndexCatalog() const {
- return _collection->getIndexCatalog();
-}
-
-const NamespaceString& IndexDescriptor::parentNS() const {
- return _collection->ns();
-}
-
IndexDescriptor::Comparison IndexDescriptor::compareIndexOptions(
- OperationContext* opCtx, const IndexCatalogEntry* other) const {
+ OperationContext* opCtx, const NamespaceString& ns, const IndexCatalogEntry* other) const {
// We first check whether the key pattern is identical for both indexes.
if (SimpleBSONObjComparator::kInstance.evaluate(keyPattern() !=
other->descriptor()->keyPattern())) {
@@ -212,8 +194,7 @@ IndexDescriptor::Comparison IndexDescriptor::compareIndexOptions(
// would match the same set of documents, but these are not currently considered equivalent.
// TODO SERVER-47664: take collation into account while comparing string predicates.
if (isFCV46 && other->getFilterExpression()) {
- auto expCtx =
- make_intrusive<ExpressionContext>(opCtx, std::move(collator), _collection->ns());
+ auto expCtx = make_intrusive<ExpressionContext>(opCtx, std::move(collator), ns);
auto filter = MatchExpressionParser::parseAndNormalize(partialFilterExpression(), expCtx);
if (!filter->equivalent(other->getFilterExpression())) {
return Comparison::kDifferent;
diff --git a/src/mongo/db/index/index_descriptor.h b/src/mongo/db/index/index_descriptor.h
index 56913b05de2..fee07610272 100644
--- a/src/mongo/db/index/index_descriptor.h
+++ b/src/mongo/db/index/index_descriptor.h
@@ -155,9 +155,6 @@ public:
return _indexName;
}
- // Return the name of the indexed collection.
- const NamespaceString& parentNS() const;
-
// Return the name of the access method we must use to access this index's data.
const std::string& getAccessMethodName() const {
return _accessMethodName;
@@ -168,6 +165,13 @@ public:
return _indexType;
}
+ /**
+ * Return a pointer to the IndexCatalogEntry that owns this descriptor, or null if orphaned.
+ */
+ IndexCatalogEntry* getEntry() const {
+ return _entry;
+ }
+
//
// Properties every index has
//
@@ -196,11 +200,6 @@ public:
return _partial;
}
- // Is this index multikey?
- bool isMultikey() const;
-
- MultikeyPaths getMultikeyPaths(OperationContext* opCtx) const;
-
bool isIdIndex() const {
return _isIdIndex;
}
@@ -219,18 +218,14 @@ public:
return _infoObj;
}
- // Both the collection and the catalog must outlive the IndexDescriptor
- const Collection* getCollection() const {
- return _collection;
- }
- const IndexCatalog* getIndexCatalog() const;
-
/**
* Compares the current IndexDescriptor against the given index entry. Returns kIdentical if all
* index options are logically identical, kEquivalent if all options which uniquely identify an
* index are logically identical, and kDifferent otherwise.
*/
- Comparison compareIndexOptions(OperationContext* opCtx, const IndexCatalogEntry* other) const;
+ Comparison compareIndexOptions(OperationContext* opCtx,
+ const NamespaceString& ns,
+ const IndexCatalogEntry* other) const;
const BSONObj& collation() const {
return _collation;
@@ -252,9 +247,6 @@ public:
}
private:
- // Related catalog information of the parent collection
- Collection* _collection;
-
// What access method should we use for this index?
std::string _accessMethodName;
@@ -278,9 +270,9 @@ private:
BSONObj _collation;
BSONObj _partialFilterExpression;
- // only used by IndexCatalogEntryContainer to do caching for perf
- // users not allowed to touch, and not part of API
- IndexCatalogEntry* _cachedEntry;
+ // Many query stages require going from an IndexDescriptor to its IndexCatalogEntry, so for
+ // now we need this.
+ IndexCatalogEntry* _entry = nullptr;
friend class IndexCatalog;
friend class IndexCatalogEntryImpl;
diff --git a/src/mongo/db/index/skipped_record_tracker.cpp b/src/mongo/db/index/skipped_record_tracker.cpp
index 7151cb8741f..5c6acd5a4f2 100644
--- a/src/mongo/db/index/skipped_record_tracker.cpp
+++ b/src/mongo/db/index/skipped_record_tracker.cpp
@@ -89,7 +89,10 @@ Status SkippedRecordTracker::retrySkippedRecords(OperationContext* opCtx,
InsertDeleteOptions options;
collection->getIndexCatalog()->prepareInsertDeleteOptions(
- opCtx, _indexCatalogEntry->descriptor(), &options);
+ opCtx,
+ _indexCatalogEntry->getNSSFromCatalog(opCtx),
+ _indexCatalogEntry->descriptor(),
+ &options);
options.fromIndexBuilder = true;
// This should only be called when constraints are being enforced, on a primary. It does not
@@ -132,7 +135,7 @@ Status SkippedRecordTracker::retrySkippedRecords(OperationContext* opCtx,
// normally happen if constraints were relaxed.
InsertResult result;
auto status = _indexCatalogEntry->accessMethod()->insert(
- opCtx, skippedDoc, skippedRecordId, options, &result);
+ opCtx, collection, skippedDoc, skippedRecordId, options, &result);
if (!status.isOK()) {
return status;
}
diff --git a/src/mongo/db/index/wildcard_access_method.cpp b/src/mongo/db/index/wildcard_access_method.cpp
index 0c0f4cbf9be..d03dd33f175 100644
--- a/src/mongo/db/index/wildcard_access_method.cpp
+++ b/src/mongo/db/index/wildcard_access_method.cpp
@@ -92,10 +92,7 @@ std::set<FieldRef> WildcardAccessMethod::_getMultikeyPathSet(
const IndexBounds& indexBounds,
MultikeyMetadataAccessStats* stats) const {
return writeConflictRetry(
- opCtx,
- "wildcard multikey path retrieval",
- _descriptor->parentNS().ns(),
- [&]() -> std::set<FieldRef> {
+ opCtx, "wildcard multikey path retrieval", "", [&]() -> std::set<FieldRef> {
stats->numSeeks = 0;
stats->keysExamined = 0;
auto cursor = newCursor(opCtx);
@@ -224,42 +221,40 @@ std::set<FieldRef> WildcardAccessMethod::getMultikeyPathSet(
std::set<FieldRef> WildcardAccessMethod::getMultikeyPathSet(
OperationContext* opCtx, MultikeyMetadataAccessStats* stats) const {
- return writeConflictRetry(
- opCtx, "wildcard multikey path retrieval", _descriptor->parentNS().ns(), [&]() {
- invariant(stats);
- stats->numSeeks = 0;
- stats->keysExamined = 0;
-
- auto cursor = newCursor(opCtx);
-
- // All of the keys storing multikeyness metadata are prefixed by a value of 1. Establish
- // an index cursor which will scan this range.
- const BSONObj metadataKeyRangeBegin = BSON("" << 1 << "" << MINKEY);
- const BSONObj metadataKeyRangeEnd = BSON("" << 1 << "" << MAXKEY);
-
- constexpr bool inclusive = true;
- cursor->setEndPosition(metadataKeyRangeEnd, inclusive);
-
- auto keyStringForSeek = IndexEntryComparison::makeKeyStringFromBSONKeyForSeek(
- metadataKeyRangeBegin,
- getSortedDataInterface()->getKeyStringVersion(),
- getSortedDataInterface()->getOrdering(),
- true, /* forward */
- inclusive);
- auto entry = cursor->seek(keyStringForSeek);
- ++stats->numSeeks;
-
- // Iterate the cursor, copying the multikey paths into an in-memory set.
- std::set<FieldRef> multikeyPaths{};
- while (entry) {
- ++stats->keysExamined;
- multikeyPaths.emplace(
- WildcardAccessMethod::extractMultikeyPathFromIndexKey(*entry));
-
- entry = cursor->next();
- }
-
- return multikeyPaths;
- });
+ return writeConflictRetry(opCtx, "wildcard multikey path retrieval", "", [&]() {
+ invariant(stats);
+ stats->numSeeks = 0;
+ stats->keysExamined = 0;
+
+ auto cursor = newCursor(opCtx);
+
+ // All of the keys storing multikeyness metadata are prefixed by a value of 1. Establish
+ // an index cursor which will scan this range.
+ const BSONObj metadataKeyRangeBegin = BSON("" << 1 << "" << MINKEY);
+ const BSONObj metadataKeyRangeEnd = BSON("" << 1 << "" << MAXKEY);
+
+ constexpr bool inclusive = true;
+ cursor->setEndPosition(metadataKeyRangeEnd, inclusive);
+
+ auto keyStringForSeek = IndexEntryComparison::makeKeyStringFromBSONKeyForSeek(
+ metadataKeyRangeBegin,
+ getSortedDataInterface()->getKeyStringVersion(),
+ getSortedDataInterface()->getOrdering(),
+ true, /* forward */
+ inclusive);
+ auto entry = cursor->seek(keyStringForSeek);
+ ++stats->numSeeks;
+
+ // Iterate the cursor, copying the multikey paths into an in-memory set.
+ std::set<FieldRef> multikeyPaths{};
+ while (entry) {
+ ++stats->keysExamined;
+ multikeyPaths.emplace(WildcardAccessMethod::extractMultikeyPathFromIndexKey(*entry));
+
+ entry = cursor->next();
+ }
+
+ return multikeyPaths;
+ });
}
} // namespace mongo
diff --git a/src/mongo/db/namespace_string.h b/src/mongo/db/namespace_string.h
index 85062ff2f35..3b15575292b 100644
--- a/src/mongo/db/namespace_string.h
+++ b/src/mongo/db/namespace_string.h
@@ -424,7 +424,7 @@ public:
private:
std::string _ns;
- size_t _dotIndex;
+ size_t _dotIndex = 0;
};
/**
diff --git a/src/mongo/db/ops/write_ops_exec.cpp b/src/mongo/db/ops/write_ops_exec.cpp
index 5c99ac9ebbe..36c08b8efd5 100644
--- a/src/mongo/db/ops/write_ops_exec.cpp
+++ b/src/mongo/db/ops/write_ops_exec.cpp
@@ -679,8 +679,8 @@ static SingleWriteResult performSingleUpdateOp(OperationContext* opCtx,
PlanSummaryStats summary;
Explain::getSummaryStats(*exec, &summary);
- if (collection->getCollection()) {
- CollectionQueryInfo::get(collection->getCollection()).notifyOfQuery(opCtx, summary);
+ if (auto coll = collection->getCollection()) {
+ CollectionQueryInfo::get(coll).notifyOfQuery(opCtx, coll, summary);
}
if (curOp.shouldDBProfile()) {
@@ -919,8 +919,8 @@ static SingleWriteResult performSingleDeleteOp(OperationContext* opCtx,
PlanSummaryStats summary;
Explain::getSummaryStats(*exec, &summary);
- if (collection.getCollection()) {
- CollectionQueryInfo::get(collection.getCollection()).notifyOfQuery(opCtx, summary);
+ if (auto coll = collection.getCollection()) {
+ CollectionQueryInfo::get(coll).notifyOfQuery(opCtx, coll, summary);
}
curOp.debug().setPlanSummaryMetrics(summary);
diff --git a/src/mongo/db/pipeline/document_source_cursor.cpp b/src/mongo/db/pipeline/document_source_cursor.cpp
index d4e75b1d2b3..f26604ce3ca 100644
--- a/src/mongo/db/pipeline/document_source_cursor.cpp
+++ b/src/mongo/db/pipeline/document_source_cursor.cpp
@@ -306,7 +306,8 @@ DocumentSourceCursor::DocumentSourceCursor(
}
if (collection) {
- CollectionQueryInfo::get(collection).notifyOfQuery(pExpCtx->opCtx, _planSummaryStats);
+ CollectionQueryInfo::get(collection)
+ .notifyOfQuery(pExpCtx->opCtx, collection, _planSummaryStats);
}
}
diff --git a/src/mongo/db/query/collection_query_info.cpp b/src/mongo/db/query/collection_query_info.cpp
index 646f30eafad..63a029aca03 100644
--- a/src/mongo/db/query/collection_query_info.cpp
+++ b/src/mongo/db/query/collection_query_info.cpp
@@ -86,10 +86,9 @@ const UpdateIndexData& CollectionQueryInfo::getIndexKeys(OperationContext* opCtx
return _indexedPaths;
}
-void CollectionQueryInfo::computeIndexKeys(OperationContext* opCtx) {
+void CollectionQueryInfo::computeIndexKeys(OperationContext* opCtx, Collection* coll) {
_indexedPaths.clear();
- const Collection* coll = get.owner(this);
std::unique_ptr<IndexCatalog::IndexIterator> it =
coll->getIndexCatalog()->getIndexIterator(opCtx, true);
while (it->more()) {
@@ -160,12 +159,12 @@ void CollectionQueryInfo::computeIndexKeys(OperationContext* opCtx) {
}
void CollectionQueryInfo::notifyOfQuery(OperationContext* opCtx,
+ Collection* coll,
const PlanSummaryStats& summaryStats) {
_indexUsageTracker.recordCollectionScans(summaryStats.collectionScans);
_indexUsageTracker.recordCollectionScansNonTailable(summaryStats.collectionScansNonTailable);
const auto& indexesUsed = summaryStats.indexesUsed;
- const Collection* coll = get.owner(this);
// Record indexes used to fulfill query.
for (auto it = indexesUsed.begin(); it != indexesUsed.end(); ++it) {
// This index should still exist, since the PlanExecutor would have been killed if the
@@ -177,8 +176,7 @@ void CollectionQueryInfo::notifyOfQuery(OperationContext* opCtx,
}
}
-void CollectionQueryInfo::clearQueryCache() {
- const Collection* coll = get.owner(this);
+void CollectionQueryInfo::clearQueryCache(const Collection* coll) {
LOGV2_DEBUG(20907,
1,
"{namespace}: clearing plan cache - collection info cache reset",
@@ -197,13 +195,12 @@ QuerySettings* CollectionQueryInfo::getQuerySettings() const {
return _querySettings.get();
}
-void CollectionQueryInfo::updatePlanCacheIndexEntries(OperationContext* opCtx) {
+void CollectionQueryInfo::updatePlanCacheIndexEntries(OperationContext* opCtx, Collection* coll) {
std::vector<CoreIndexInfo> indexCores;
// TODO We shouldn't need to include unfinished indexes, but we must here because the index
// catalog may be in an inconsistent state. SERVER-18346.
const bool includeUnfinishedIndexes = true;
- const Collection* coll = get.owner(this);
std::unique_ptr<IndexCatalog::IndexIterator> ii =
coll->getIndexCatalog()->getIndexIterator(opCtx, includeUnfinishedIndexes);
while (ii->more()) {
@@ -214,8 +211,7 @@ void CollectionQueryInfo::updatePlanCacheIndexEntries(OperationContext* opCtx) {
_planCache->notifyOfIndexUpdates(indexCores);
}
-void CollectionQueryInfo::init(OperationContext* opCtx) {
- const Collection* coll = get.owner(this);
+void CollectionQueryInfo::init(OperationContext* opCtx, Collection* coll) {
const bool includeUnfinishedIndexes = false;
std::unique_ptr<IndexCatalog::IndexIterator> ii =
@@ -225,27 +221,31 @@ void CollectionQueryInfo::init(OperationContext* opCtx) {
_indexUsageTracker.registerIndex(desc->indexName(), desc->keyPattern());
}
- rebuildIndexData(opCtx);
+ rebuildIndexData(opCtx, coll);
}
-void CollectionQueryInfo::addedIndex(OperationContext* opCtx, const IndexDescriptor* desc) {
+void CollectionQueryInfo::addedIndex(OperationContext* opCtx,
+ Collection* coll,
+ const IndexDescriptor* desc) {
invariant(desc);
- rebuildIndexData(opCtx);
+ rebuildIndexData(opCtx, coll);
_indexUsageTracker.registerIndex(desc->indexName(), desc->keyPattern());
}
-void CollectionQueryInfo::droppedIndex(OperationContext* opCtx, StringData indexName) {
- rebuildIndexData(opCtx);
+void CollectionQueryInfo::droppedIndex(OperationContext* opCtx,
+ Collection* coll,
+ StringData indexName) {
+ rebuildIndexData(opCtx, coll);
_indexUsageTracker.unregisterIndex(indexName);
}
-void CollectionQueryInfo::rebuildIndexData(OperationContext* opCtx) {
- clearQueryCache();
+void CollectionQueryInfo::rebuildIndexData(OperationContext* opCtx, Collection* coll) {
+ clearQueryCache(coll);
_keysComputed = false;
- computeIndexKeys(opCtx);
- updatePlanCacheIndexEntries(opCtx);
+ computeIndexKeys(opCtx, coll);
+ updatePlanCacheIndexEntries(opCtx, coll);
}
CollectionIndexUsageMap CollectionQueryInfo::getIndexUsageStats() const {
diff --git a/src/mongo/db/query/collection_query_info.h b/src/mongo/db/query/collection_query_info.h
index 12b7ceecda7..61009072880 100644
--- a/src/mongo/db/query/collection_query_info.h
+++ b/src/mongo/db/query/collection_query_info.h
@@ -82,7 +82,7 @@ public:
/**
* Builds internal cache state based on the current state of the Collection's IndexCatalog.
*/
- void init(OperationContext* opCtx);
+ void init(OperationContext* opCtx, Collection* coll);
/**
* Register a newly-created index with the cache. Must be called whenever an index is
@@ -90,7 +90,7 @@ public:
*
* Must be called under exclusive collection lock.
*/
- void addedIndex(OperationContext* opCtx, const IndexDescriptor* desc);
+ void addedIndex(OperationContext* opCtx, Collection* coll, const IndexDescriptor* desc);
/**
* Deregister a newly-dropped index with the cache. Must be called whenever an index is
@@ -98,24 +98,26 @@ public:
*
* Must be called under exclusive collection lock.
*/
- void droppedIndex(OperationContext* opCtx, StringData indexName);
+ void droppedIndex(OperationContext* opCtx, Collection* coll, StringData indexName);
/**
* Removes all cached query plans.
*/
- void clearQueryCache();
+ void clearQueryCache(const Collection* coll);
- void notifyOfQuery(OperationContext* opCtx, const PlanSummaryStats& summaryStats);
+ void notifyOfQuery(OperationContext* opCtx,
+ Collection* coll,
+ const PlanSummaryStats& summaryStats);
private:
- void computeIndexKeys(OperationContext* opCtx);
- void updatePlanCacheIndexEntries(OperationContext* opCtx);
+ void computeIndexKeys(OperationContext* opCtx, Collection* coll);
+ void updatePlanCacheIndexEntries(OperationContext* opCtx, Collection* coll);
/**
* Rebuilds cached information that is dependent on index composition. Must be called
* when index composition changes.
*/
- void rebuildIndexData(OperationContext* opCtx);
+ void rebuildIndexData(OperationContext* opCtx, Collection* coll);
// --- index keys cache
bool _keysComputed;
diff --git a/src/mongo/db/query/find.cpp b/src/mongo/db/query/find.cpp
index 2da05379213..602e525cb9c 100644
--- a/src/mongo/db/query/find.cpp
+++ b/src/mongo/db/query/find.cpp
@@ -137,7 +137,7 @@ void endQueryOp(OperationContext* opCtx,
curOp->debug().setPlanSummaryMetrics(summaryStats);
if (collection) {
- CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, summaryStats);
+ CollectionQueryInfo::get(collection).notifyOfQuery(opCtx, collection, summaryStats);
}
if (curOp->shouldDBProfile()) {
diff --git a/src/mongo/db/query/get_executor.cpp b/src/mongo/db/query/get_executor.cpp
index 5d64f46527a..42314ec67aa 100644
--- a/src/mongo/db/query/get_executor.cpp
+++ b/src/mongo/db/query/get_executor.cpp
@@ -177,7 +177,7 @@ IndexEntry indexEntryFromIndexCatalogEntry(OperationContext* opCtx,
auto accessMethod = ice.accessMethod();
invariant(accessMethod);
- const bool isMultikey = desc->isMultikey();
+ const bool isMultikey = ice.isMultikey();
const WildcardProjection* wildcardProjection = nullptr;
std::set<FieldRef> multikeyPathSet;
@@ -412,7 +412,7 @@ StatusWith<PrepareExecutionResult> prepareExecution(OperationContext* opCtx,
"canonicalQuery_Short"_attr = redact(canonicalQuery->toStringShort()));
root = std::make_unique<IDHackStage>(
- canonicalQuery->getExpCtxRaw(), canonicalQuery.get(), ws, descriptor);
+ canonicalQuery->getExpCtxRaw(), canonicalQuery.get(), ws, collection, descriptor);
// Might have to filter out orphaned docs.
if (plannerParams.options & QueryPlannerParams::INCLUDE_SHARD_FILTER) {
@@ -819,7 +819,7 @@ StatusWith<unique_ptr<PlanExecutor, PlanExecutor::Deleter>> getExecutorDelete(
"query"_attr = redact(unparsedQuery));
auto idHackStage = std::make_unique<IDHackStage>(
- expCtx.get(), unparsedQuery["_id"].wrap(), ws.get(), descriptor);
+ expCtx.get(), unparsedQuery["_id"].wrap(), ws.get(), collection, descriptor);
unique_ptr<DeleteStage> root =
std::make_unique<DeleteStage>(expCtx.get(),
std::move(deleteStageParams),
@@ -1489,8 +1489,8 @@ QueryPlannerParams fillOutPlannerParamsForDistinct(OperationContext* opCtx,
if (desc->keyPattern().hasField(parsedDistinct.getKey())) {
if (!mayUnwindArrays &&
isAnyComponentOfPathMultikey(desc->keyPattern(),
- desc->isMultikey(),
- desc->getMultikeyPaths(opCtx),
+ ice->isMultikey(),
+ ice->getMultikeyPaths(opCtx),
parsedDistinct.getKey())) {
// If the caller requested "strict" distinct that does not "pre-unwind" arrays,
// then an index which is multikey on the distinct field may not be used. This is
diff --git a/src/mongo/db/query/internal_plans.cpp b/src/mongo/db/query/internal_plans.cpp
index ed03783f598..b826311614f 100644
--- a/src/mongo/db/query/internal_plans.cpp
+++ b/src/mongo/db/query/internal_plans.cpp
@@ -181,7 +181,8 @@ std::unique_ptr<PlanExecutor, PlanExecutor::Deleter> InternalPlanner::updateWith
auto expCtx = make_intrusive<ExpressionContext>(
opCtx, std::unique_ptr<CollatorInterface>(nullptr), collection->ns());
- auto idHackStage = std::make_unique<IDHackStage>(expCtx.get(), key, ws.get(), descriptor);
+ auto idHackStage =
+ std::make_unique<IDHackStage>(expCtx.get(), key, ws.get(), collection, descriptor);
const bool isUpsert = params.request->isUpsert();
auto root = (isUpsert ? std::make_unique<UpsertStage>(
@@ -234,10 +235,10 @@ std::unique_ptr<PlanStage> InternalPlanner::_indexScan(
params.bounds.startKey = startKey;
params.bounds.endKey = endKey;
params.bounds.boundInclusion = boundInclusion;
- params.shouldDedup = descriptor->isMultikey();
+ params.shouldDedup = descriptor->getEntry()->isMultikey();
std::unique_ptr<PlanStage> root =
- std::make_unique<IndexScan>(expCtx.get(), std::move(params), ws, nullptr);
+ std::make_unique<IndexScan>(expCtx.get(), collection, std::move(params), ws, nullptr);
if (InternalPlanner::IXSCAN_FETCH & options) {
root = std::make_unique<FetchStage>(expCtx.get(), ws, std::move(root), nullptr, collection);
diff --git a/src/mongo/db/query/stage_builder.cpp b/src/mongo/db/query/stage_builder.cpp
index 7858456be63..87a1987aa2e 100644
--- a/src/mongo/db/query/stage_builder.cpp
+++ b/src/mongo/db/query/stage_builder.cpp
@@ -114,7 +114,8 @@ std::unique_ptr<PlanStage> buildStages(OperationContext* opCtx,
params.direction = ixn->direction;
params.addKeyMetadata = ixn->addKeyMetadata;
params.shouldDedup = ixn->shouldDedup;
- return std::make_unique<IndexScan>(expCtx, std::move(params), ws, ixn->filter.get());
+ return std::make_unique<IndexScan>(
+ expCtx, collection, std::move(params), ws, ixn->filter.get());
}
case STAGE_FETCH: {
const FetchNode* fn = static_cast<const FetchNode*>(root);
@@ -252,7 +253,7 @@ std::unique_ptr<PlanStage> buildStages(OperationContext* opCtx,
opCtx, node->index.identifier.catalogName);
invariant(twoDIndex);
- return std::make_unique<GeoNear2DStage>(params, expCtx, ws, twoDIndex);
+ return std::make_unique<GeoNear2DStage>(params, expCtx, ws, collection, twoDIndex);
}
case STAGE_GEO_NEAR_2DSPHERE: {
const GeoNear2DSphereNode* node = static_cast<const GeoNear2DSphereNode*>(root);
@@ -269,7 +270,7 @@ std::unique_ptr<PlanStage> buildStages(OperationContext* opCtx,
opCtx, node->index.identifier.catalogName);
invariant(s2Index);
- return std::make_unique<GeoNear2DSphereStage>(params, expCtx, ws, s2Index);
+ return std::make_unique<GeoNear2DSphereStage>(params, expCtx, ws, collection, s2Index);
}
case STAGE_TEXT: {
const TextNode* node = static_cast<const TextNode*>(root);
@@ -289,7 +290,7 @@ std::unique_ptr<PlanStage> buildStages(OperationContext* opCtx,
// created by planning a query that contains "no-op" expressions.
params.query = static_cast<FTSQueryImpl&>(*node->ftsQuery);
params.wantTextScore = cq.metadataDeps()[DocumentMetadataFields::kTextScore];
- return std::make_unique<TextStage>(expCtx, params, ws, node->filter.get());
+ return std::make_unique<TextStage>(expCtx, collection, params, ws, node->filter.get());
}
case STAGE_SHARDING_FILTER: {
const ShardingFilterNode* fn = static_cast<const ShardingFilterNode*>(root);
@@ -322,7 +323,7 @@ std::unique_ptr<PlanStage> buildStages(OperationContext* opCtx,
params.scanDirection = dn->direction;
params.bounds = dn->bounds;
params.fieldNo = dn->fieldNo;
- return std::make_unique<DistinctScan>(expCtx, std::move(params), ws);
+ return std::make_unique<DistinctScan>(expCtx, collection, std::move(params), ws);
}
case STAGE_COUNT_SCAN: {
const CountScanNode* csn = static_cast<const CountScanNode*>(root);
@@ -344,7 +345,7 @@ std::unique_ptr<PlanStage> buildStages(OperationContext* opCtx,
params.startKeyInclusive = csn->startKeyInclusive;
params.endKey = csn->endKey;
params.endKeyInclusive = csn->endKeyInclusive;
- return std::make_unique<CountScan>(expCtx, std::move(params), ws);
+ return std::make_unique<CountScan>(expCtx, collection, std::move(params), ws);
}
case STAGE_ENSURE_SORTED: {
const EnsureSortedNode* esn = static_cast<const EnsureSortedNode*>(root);
diff --git a/src/mongo/db/repl/storage_interface_impl.cpp b/src/mongo/db/repl/storage_interface_impl.cpp
index 4d3e2cc9e81..ec9bdd75924 100644
--- a/src/mongo/db/repl/storage_interface_impl.cpp
+++ b/src/mongo/db/repl/storage_interface_impl.cpp
@@ -590,7 +590,7 @@ Status StorageInterfaceImpl::setIndexIsMultikey(OperationContext* opCtx,
str::stream() << "Could not find index " << indexName << " in "
<< nss.ns() << " to set to multikey.");
}
- collection->getIndexCatalog()->setMultikeyPaths(opCtx, idx, paths);
+ collection->getIndexCatalog()->setMultikeyPaths(opCtx, collection, idx, paths);
wunit.commit();
return Status::OK();
});
diff --git a/src/mongo/db/repl/storage_interface_impl_test.cpp b/src/mongo/db/repl/storage_interface_impl_test.cpp
index 9656b154a88..9a121099170 100644
--- a/src/mongo/db/repl/storage_interface_impl_test.cpp
+++ b/src/mongo/db/repl/storage_interface_impl_test.cpp
@@ -2654,9 +2654,9 @@ TEST_F(StorageInterfaceImplTest, SetIndexIsMultikeySucceeds) {
AutoGetCollectionForReadCommand autoColl(opCtx, nss);
ASSERT_TRUE(autoColl.getCollection());
auto indexCatalog = autoColl.getCollection()->getIndexCatalog();
- ASSERT(indexCatalog->isMultikey(indexCatalog->findIndexByName(opCtx, indexName)));
- ASSERT(paths ==
- indexCatalog->getMultikeyPaths(opCtx, indexCatalog->findIndexByName(opCtx, indexName)));
+ auto entry = indexCatalog->findIndexByName(opCtx, indexName)->getEntry();
+ ASSERT(entry->isMultikey());
+ ASSERT(paths == entry->getMultikeyPaths(opCtx));
}
} // namespace
diff --git a/src/mongo/db/storage/biggie/biggie_init.cpp b/src/mongo/db/storage/biggie/biggie_init.cpp
index 659798ac6b7..f54a76a863d 100644
--- a/src/mongo/db/storage/biggie/biggie_init.cpp
+++ b/src/mongo/db/storage/biggie/biggie_init.cpp
@@ -42,12 +42,12 @@ namespace biggie {
namespace {
class BiggieStorageEngineFactory : public StorageEngine::Factory {
public:
- virtual StorageEngine* create(const StorageGlobalParams& params,
- const StorageEngineLockFile* lockFile) const {
+ virtual std::unique_ptr<StorageEngine> create(const StorageGlobalParams& params,
+ const StorageEngineLockFile* lockFile) const {
StorageEngineOptions options;
options.directoryPerDB = params.directoryperdb;
options.forRepair = params.repair;
- return new StorageEngineImpl(new KVEngine(), options);
+ return std::make_unique<StorageEngineImpl>(std::make_unique<KVEngine>(), options);
}
virtual StringData getCanonicalName() const {
diff --git a/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp b/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp
index 53a8e7211ea..d5150f064c3 100644
--- a/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp
+++ b/src/mongo/db/storage/biggie/biggie_sorted_impl.cpp
@@ -193,7 +193,7 @@ SortedDataBuilderInterface::SortedDataBuilderInterface(OperationContext* opCtx,
Ordering order,
const std::string& prefix,
const std::string& identEnd,
- const NamespaceString& collectionNamespace,
+ const IndexDescriptor* desc,
const std::string& indexName,
const BSONObj& keyPattern,
const BSONObj& collation)
@@ -203,7 +203,7 @@ SortedDataBuilderInterface::SortedDataBuilderInterface(OperationContext* opCtx,
_order(order),
_prefix(prefix),
_identEnd(identEnd),
- _collectionNamespace(collectionNamespace),
+ _desc(desc),
_indexName(indexName),
_keyPattern(keyPattern),
_collation(collation),
@@ -242,11 +242,9 @@ Status SortedDataBuilderInterface::addKey(const KeyString::Value& keyString) {
createKeyString(keyString, sizeWithoutRecordId, loc, _prefix, /* isUnique */ _unique);
if (twoKeyCmp == 0 && twoRIDCmp != 0) {
- if (!_dupsAllowed) {
- auto key = KeyString::toBson(keyString, _order);
- return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation);
- }
+ if (!_dupsAllowed)
+ return buildDupKeyErrorStatus(_opCtx, keyString, _order, _desc);
+
// Duplicate index entries are allowed on this unique index, so we put the RecordId in the
// KeyString until the unique constraint is resolved.
workingCopyInsertKey =
@@ -280,7 +278,7 @@ SortedDataBuilderInterface* SortedDataInterface::getBulkBuilder(OperationContext
_ordering,
_prefix,
_identEnd,
- _collectionNamespace,
+ _desc,
_indexName,
_keyPattern,
_collation);
@@ -297,7 +295,7 @@ SortedDataInterface::SortedDataInterface(OperationContext* opCtx,
_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')),
- _collectionNamespace(desc->getCollection()->ns()),
+ _desc(desc),
_indexName(desc->indexName()),
_keyPattern(desc->keyPattern()),
_collation(desc->collation()),
@@ -338,6 +336,7 @@ Status SortedDataInterface::insert(OperationContext* opCtx,
// - 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.
@@ -346,30 +345,19 @@ Status SortedDataInterface::insert(OperationContext* opCtx,
IndexKeyEntry entry =
keyStringToIndexKeyEntry(workingCopyIt->first, workingCopyIt->second, _ordering);
- if (entry.loc != loc) {
- if (dupsAllowed) {
- // 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);
- } else {
- // There was an attempt to create an index entry with a different RecordId while
- // dups were not allowed.
- auto key = KeyString::toBson(keyString, _ordering);
- return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation);
- }
- } else {
+ 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);
}
- } else {
- invariant(dupsAllowed);
}
-
if (workingCopy->find(insertKeyString) != workingCopy->end())
return Status::OK();
@@ -496,8 +484,7 @@ Status SortedDataInterface::dupKeyCheck(OperationContext* opCtx, const KeyString
KeyString::sizeWithoutRecordIdAtEnd(next->keyString.getBuffer(),
next->keyString.getSize()),
key.getSize()) == 0) {
- return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation, _ordering);
+ return buildDupKeyErrorStatus(opCtx, key, _ordering, _desc);
}
return Status::OK();
diff --git a/src/mongo/db/storage/biggie/biggie_sorted_impl.h b/src/mongo/db/storage/biggie/biggie_sorted_impl.h
index 96f0229197c..adf0c2f1890 100644
--- a/src/mongo/db/storage/biggie/biggie_sorted_impl.h
+++ b/src/mongo/db/storage/biggie/biggie_sorted_impl.h
@@ -44,7 +44,7 @@ public:
Ordering order,
const std::string& prefix,
const std::string& identEnd,
- const NamespaceString& collectionNamespace,
+ const IndexDescriptor* _desc,
const std::string& indexName,
const BSONObj& keyPattern,
const BSONObj& collation);
@@ -61,7 +61,7 @@ private:
std::string _prefix;
std::string _identEnd;
// Index metadata.
- const NamespaceString _collectionNamespace;
+ const IndexDescriptor* _desc;
const std::string _indexName;
const BSONObj _keyPattern;
const BSONObj _collation;
@@ -194,7 +194,7 @@ private:
std::string _prefix;
std::string _identEnd;
// Index metadata.
- const NamespaceString _collectionNamespace;
+ const IndexDescriptor* _desc;
const std::string _indexName;
const BSONObj _keyPattern;
const BSONObj _collation;
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 f2beff65e08..8bedba822cc 100644
--- a/src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp
+++ b/src/mongo/db/storage/biggie/biggie_sorted_impl_test.cpp
@@ -48,10 +48,6 @@ namespace {
class BiggieSortedDataInterfaceTestHarnessHelper final
: public virtual SortedDataInterfaceHarnessHelper {
-private:
- KVEngine _kvEngine{};
- Ordering _order;
-
public:
BiggieSortedDataInterfaceTestHarnessHelper() : _order(Ordering::make(BSONObj())) {}
@@ -88,14 +84,18 @@ public:
}
auto collection = std::make_unique<CollectionMock>(NamespaceString(ns));
- IndexDescriptor desc(collection.get(), "", spec);
-
- return std::make_unique<SortedDataInterface>(&opCtx, "ident"_sd, &desc);
+ _descs.emplace_back(collection.get(), "", spec);
+ return std::make_unique<SortedDataInterface>(&opCtx, "ident"_sd, &_descs.back());
}
std::unique_ptr<mongo::RecoveryUnit> newRecoveryUnit() final {
return std::make_unique<RecoveryUnit>(&_kvEngine);
}
+
+private:
+ KVEngine _kvEngine{};
+ Ordering _order;
+ std::list<IndexDescriptor> _descs;
};
std::unique_ptr<mongo::SortedDataInterfaceHarnessHelper>
diff --git a/src/mongo/db/storage/devnull/devnull_init.cpp b/src/mongo/db/storage/devnull/devnull_init.cpp
index 2105395b38b..dc39346f42c 100644
--- a/src/mongo/db/storage/devnull/devnull_init.cpp
+++ b/src/mongo/db/storage/devnull/devnull_init.cpp
@@ -41,12 +41,12 @@ namespace mongo {
namespace {
class DevNullStorageEngineFactory : public StorageEngine::Factory {
public:
- virtual StorageEngine* create(const StorageGlobalParams& params,
- const StorageEngineLockFile* lockFile) const {
+ virtual std::unique_ptr<StorageEngine> create(const StorageGlobalParams& params,
+ const StorageEngineLockFile* lockFile) const {
StorageEngineOptions options;
options.directoryPerDB = params.directoryperdb;
options.forRepair = params.repair;
- return new StorageEngineImpl(new DevNullKVEngine(), options);
+ return std::make_unique<StorageEngineImpl>(std::make_unique<DevNullKVEngine>(), options);
}
virtual StringData getCanonicalName() const {
diff --git a/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_engine.cpp b/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_engine.cpp
index ee9bc57d945..dbe6a6f0b36 100644
--- a/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_engine.cpp
+++ b/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_engine.cpp
@@ -101,9 +101,13 @@ Status EphemeralForTestEngine::createSortedDataInterface(OperationContext* opCtx
std::unique_ptr<SortedDataInterface> EphemeralForTestEngine::getSortedDataInterface(
OperationContext* opCtx, StringData ident, const IndexDescriptor* desc) {
stdx::lock_guard<Latch> lk(_mutex);
+ NamespaceString collNss;
+ // Some unit tests don't have actual index entries.
+ if (auto entry = desc->getEntry())
+ collNss = entry->getNSSFromCatalog(opCtx);
return getEphemeralForTestBtreeImpl(Ordering::make(desc->keyPattern()),
desc->unique(),
- desc->parentNS(),
+ collNss,
desc->indexName(),
desc->keyPattern(),
desc->collation(),
diff --git a/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_init.cpp b/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_init.cpp
index 3ff09e24588..c7780eefed7 100644
--- a/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_init.cpp
+++ b/src/mongo/db/storage/ephemeral_for_test/ephemeral_for_test_init.cpp
@@ -42,8 +42,8 @@ namespace {
class EphemeralForTestFactory : public StorageEngine::Factory {
public:
virtual ~EphemeralForTestFactory() {}
- virtual StorageEngine* create(const StorageGlobalParams& params,
- const StorageEngineLockFile* lockFile) const {
+ virtual std::unique_ptr<StorageEngine> create(const StorageGlobalParams& params,
+ const StorageEngineLockFile* lockFile) const {
uassert(ErrorCodes::InvalidOptions,
"ephemeralForTest does not support --groupCollections",
!params.groupCollections);
@@ -51,7 +51,8 @@ public:
StorageEngineOptions options;
options.directoryPerDB = params.directoryperdb;
options.forRepair = params.repair;
- return new StorageEngineImpl(new EphemeralForTestEngine(), options);
+ return std::make_unique<StorageEngineImpl>(std::make_unique<EphemeralForTestEngine>(),
+ options);
}
virtual StringData getCanonicalName() const {
diff --git a/src/mongo/db/storage/index_entry_comparison.cpp b/src/mongo/db/storage/index_entry_comparison.cpp
index 86e0e502290..face7000937 100644
--- a/src/mongo/db/storage/index_entry_comparison.cpp
+++ b/src/mongo/db/storage/index_entry_comparison.cpp
@@ -255,4 +255,23 @@ Status buildDupKeyErrorStatus(const KeyString::Value& keyString,
return buildDupKeyErrorStatus(key, collectionNamespace, indexName, keyPattern, indexCollation);
}
+Status buildDupKeyErrorStatus(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ const Ordering& ordering,
+ const IndexDescriptor* desc) {
+ const BSONObj key = KeyString::toBson(
+ keyString.getBuffer(), keyString.getSize(), ordering, keyString.getTypeBits());
+ return buildDupKeyErrorStatus(opCtx, key, desc);
+}
+
+Status buildDupKeyErrorStatus(OperationContext* opCtx,
+ const BSONObj& key,
+ const IndexDescriptor* desc) {
+ NamespaceString nss;
+ // In testing these may be nullptr, and being a bit more lenient during error handling is OK.
+ if (desc && desc->getEntry())
+ nss = desc->getEntry()->getNSSFromCatalog(opCtx);
+ return buildDupKeyErrorStatus(
+ key, nss, desc->indexName(), desc->keyPattern(), desc->collation());
+}
} // namespace mongo
diff --git a/src/mongo/db/storage/index_entry_comparison.h b/src/mongo/db/storage/index_entry_comparison.h
index ad4bbbc8aa6..082e7a6b385 100644
--- a/src/mongo/db/storage/index_entry_comparison.h
+++ b/src/mongo/db/storage/index_entry_comparison.h
@@ -34,6 +34,7 @@
#include <vector>
#include "mongo/bson/simple_bsonobj_comparator.h"
+#include "mongo/db/index/index_descriptor.h"
#include "mongo/db/jsobj.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/record_id.h"
@@ -261,9 +262,6 @@ Status buildDupKeyErrorStatus(const BSONObj& key,
const BSONObj& keyPattern,
const BSONObj& indexCollation);
-/**
- * Returns the formatted error status about the duplicate KeyString.
- */
Status buildDupKeyErrorStatus(const KeyString::Value& keyString,
const NamespaceString& collectionNamespace,
const std::string& indexName,
@@ -271,4 +269,13 @@ Status buildDupKeyErrorStatus(const KeyString::Value& keyString,
const BSONObj& indexCollation,
const Ordering& ordering);
+Status buildDupKeyErrorStatus(OperationContext* opCtx,
+ const BSONObj& key,
+ const IndexDescriptor* desc);
+
+Status buildDupKeyErrorStatus(OperationContext* opCtx,
+ const KeyString::Value& keyString,
+ const Ordering& ordering,
+ const IndexDescriptor* desc);
+
} // namespace mongo
diff --git a/src/mongo/db/storage/kv/durable_catalog_test.cpp b/src/mongo/db/storage/kv/durable_catalog_test.cpp
index 187952be655..5710f1a191c 100644
--- a/src/mongo/db/storage/kv/durable_catalog_test.cpp
+++ b/src/mongo/db/storage/kv/durable_catalog_test.cpp
@@ -60,7 +60,7 @@ class DurableCatalogTest : public ServiceContextTest {
public:
DurableCatalogTest()
: _nss("unittests.durable_catalog"),
- _storageEngine(new DevNullKVEngine(), StorageEngineOptions()) {
+ _storageEngine(std::make_unique<DevNullKVEngine>(), StorageEngineOptions()) {
_storageEngine.finishInit();
}
diff --git a/src/mongo/db/storage/kv/kv_engine_test_harness.cpp b/src/mongo/db/storage/kv/kv_engine_test_harness.cpp
index 9ffb9a6da0d..de95507fc6c 100644
--- a/src/mongo/db/storage/kv/kv_engine_test_harness.cpp
+++ b/src/mongo/db/storage/kv/kv_engine_test_harness.cpp
@@ -185,7 +185,7 @@ TEST(KVEngineTestHarness, SimpleSorted1) {
}
- std::unique_ptr<CollectionImpl> collection;
+ std::unique_ptr<Collection> collection;
{
MyOperationContext opCtx(engine);
WriteUnitOfWork uow(&opCtx);
diff --git a/src/mongo/db/storage/kv/storage_engine_test.cpp b/src/mongo/db/storage/kv/storage_engine_test.cpp
index 5a77a9a68e1..e7e7bb43a79 100644
--- a/src/mongo/db/storage/kv/storage_engine_test.cpp
+++ b/src/mongo/db/storage/kv/storage_engine_test.cpp
@@ -429,7 +429,8 @@ public:
TimestampKVEngineTest() {
StorageEngineOptions options{
/*directoryPerDB=*/false, /*directoryForIndexes=*/false, /*forRepair=*/false};
- _storageEngine = std::make_unique<StorageEngineImpl>(new TimestampMockKVEngine, options);
+ _storageEngine =
+ std::make_unique<StorageEngineImpl>(std::make_unique<TimestampMockKVEngine>(), options);
_storageEngine->finishInit();
}
diff --git a/src/mongo/db/storage/storage_engine.h b/src/mongo/db/storage/storage_engine.h
index dcdb4347782..b600cc8ab56 100644
--- a/src/mongo/db/storage/storage_engine.h
+++ b/src/mongo/db/storage/storage_engine.h
@@ -29,6 +29,7 @@
#pragma once
+#include <memory>
#include <string>
#include <vector>
@@ -91,8 +92,8 @@ public:
* Return a new instance of the StorageEngine. The lockFile parameter may be null if
* params.readOnly is set. Caller owns the returned pointer.
*/
- virtual StorageEngine* create(const StorageGlobalParams& params,
- const StorageEngineLockFile* lockFile) const = 0;
+ virtual std::unique_ptr<StorageEngine> create(
+ const StorageGlobalParams& params, const StorageEngineLockFile* lockFile) const = 0;
/**
* Returns the name of the storage engine.
diff --git a/src/mongo/db/storage/storage_engine_impl.cpp b/src/mongo/db/storage/storage_engine_impl.cpp
index 8a8a1219d8e..4074e4b651b 100644
--- a/src/mongo/db/storage/storage_engine_impl.cpp
+++ b/src/mongo/db/storage/storage_engine_impl.cpp
@@ -67,10 +67,10 @@ const std::string catalogInfo = "_mdb_catalog";
const auto kCatalogLogLevel = logv2::LogSeverity::Debug(2);
} // namespace
-StorageEngineImpl::StorageEngineImpl(KVEngine* engine, StorageEngineOptions options)
- : _engine(engine),
+StorageEngineImpl::StorageEngineImpl(std::unique_ptr<KVEngine> engine, StorageEngineOptions options)
+ : _engine(std::move(engine)),
_options(std::move(options)),
- _dropPendingIdentReaper(engine),
+ _dropPendingIdentReaper(_engine.get()),
_minOfCheckpointAndOldestTimestampListener(
TimestampMonitor::TimestampType::kMinOfCheckpointAndOldest,
[this](Timestamp timestamp) { _onMinOfCheckpointAndOldestTimestampChanged(timestamp); }),
@@ -79,7 +79,7 @@ StorageEngineImpl::StorageEngineImpl(KVEngine* engine, StorageEngineOptions opti
_supportsCappedCollections(_engine->supportsCappedCollections()) {
uassert(28601,
"Storage engine does not support --directoryperdb",
- !(options.directoryPerDB && !engine->supportsDirectoryPerDB()));
+ !(options.directoryPerDB && !_engine->supportsDirectoryPerDB()));
OperationContextNoop opCtx(_engine->newRecoveryUnit());
loadCatalog(&opCtx);
diff --git a/src/mongo/db/storage/storage_engine_impl.h b/src/mongo/db/storage/storage_engine_impl.h
index a645a572e5c..e62104f31f2 100644
--- a/src/mongo/db/storage/storage_engine_impl.h
+++ b/src/mongo/db/storage/storage_engine_impl.h
@@ -62,10 +62,8 @@ struct StorageEngineOptions {
class StorageEngineImpl final : public StorageEngineInterface, public StorageEngine {
public:
- /**
- * @param engine - ownership passes to me.
- */
- StorageEngineImpl(KVEngine* engine, StorageEngineOptions options = StorageEngineOptions());
+ StorageEngineImpl(std::unique_ptr<KVEngine> engine,
+ StorageEngineOptions options = StorageEngineOptions());
virtual ~StorageEngineImpl();
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_index.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_index.cpp
index f8c7e2a4f22..537e083208b 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_index.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_index.cpp
@@ -158,11 +158,13 @@ std::string WiredTigerIndex::generateAppMetadataString(const IndexDescriptor& de
}
// static
-StatusWith<std::string> WiredTigerIndex::generateCreateString(const std::string& engineName,
- const std::string& sysIndexConfig,
- const std::string& collIndexConfig,
- const IndexDescriptor& desc,
- bool isPrefixed) {
+StatusWith<std::string> WiredTigerIndex::generateCreateString(
+ const std::string& engineName,
+ const std::string& sysIndexConfig,
+ const std::string& collIndexConfig,
+ const NamespaceString& collectionNamespace,
+ const IndexDescriptor& desc,
+ bool isPrefixed) {
str::stream ss;
// Separate out a prefix and suffix in the default string. User configuration will override
@@ -176,7 +178,7 @@ StatusWith<std::string> WiredTigerIndex::generateCreateString(const std::string&
ss << "block_compressor=" << wiredTigerGlobalOptions.indexBlockCompressor << ",";
ss << WiredTigerCustomizationHooks::get(getGlobalServiceContext())
- ->getTableCreateConfig(desc.parentNS().ns());
+ ->getTableCreateConfig(collectionNamespace.ns());
ss << sysIndexConfig << ",";
ss << collIndexConfig << ",";
@@ -213,7 +215,7 @@ StatusWith<std::string> WiredTigerIndex::generateCreateString(const std::string&
bool replicatedWrites = getGlobalReplSettings().usingReplSets() ||
repl::ReplSettings::shouldRecoverFromOplogAsStandalone();
- if (WiredTigerUtil::useTableLogging(NamespaceString(desc.parentNS()), replicatedWrites)) {
+ if (WiredTigerUtil::useTableLogging(collectionNamespace, replicatedWrites)) {
ss << "log=(enabled=true)";
} else {
ss << "log=(enabled=false)";
@@ -243,13 +245,17 @@ WiredTigerIndex::WiredTigerIndex(OperationContext* ctx,
Ordering::make(desc->keyPattern())),
_uri(uri),
_tableId(WiredTigerSession::genTableId()),
- _collectionNamespace(desc->parentNS()),
+ _desc(desc),
_indexName(desc->indexName()),
_keyPattern(desc->keyPattern()),
_collation(desc->collation()),
_prefix(prefix),
_isIdIndex(desc->isIdIndex()) {}
+NamespaceString WiredTigerIndex::getCollectionNamespace(OperationContext* opCtx) const {
+ return _desc->getEntry()->getNSSFromCatalog(opCtx);
+}
+
Status WiredTigerIndex::insert(OperationContext* opCtx,
const KeyString::Value& keyString,
bool dupsAllowed) {
@@ -380,9 +386,11 @@ Status WiredTigerIndex::dupKeyCheck(OperationContext* opCtx, const KeyString::Va
WiredTigerCursor curwrap(_uri, _tableId, false, opCtx);
WT_CURSOR* c = curwrap.get();
- if (isDup(opCtx, c, key))
- return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation, _ordering);
+ if (isDup(opCtx, c, key)) {
+ auto entry = _desc->getEntry();
+ auto nss = entry ? entry->getNSSFromCatalog(opCtx) : NamespaceString();
+ return buildDupKeyErrorStatus(key, nss, _indexName, _keyPattern, _collation, _ordering);
+ }
return Status::OK();
}
@@ -505,34 +513,36 @@ KeyString::Version WiredTigerIndex::_handleVersionInfo(OperationContext* ctx,
auto version = WiredTigerUtil::checkApplicationMetadataFormatVersion(
ctx, uri, kMinimumIndexVersion, kMaximumIndexVersion);
if (!version.isOK()) {
+ auto collectionNamespace = desc->getEntry()->getNSSFromCatalog(ctx);
Status versionStatus = version.getStatus();
Status indexVersionStatus(ErrorCodes::UnsupportedFormat,
str::stream()
<< versionStatus.reason() << " Index: {name: "
- << desc->indexName() << ", ns: " << desc->parentNS()
+ << desc->indexName() << ", ns: " << collectionNamespace
<< "} - version either too old or too new for this mongod.");
fassertFailedWithStatusNoTrace(28579, indexVersionStatus);
}
_dataFormatVersion = version.getValue();
- if (!desc->isIdIndex() && desc->unique()) {
- Status versionStatus = _dataFormatVersion == kDataFormatV3KeyStringV0UniqueIndexVersionV1 ||
- _dataFormatVersion == kDataFormatV4KeyStringV1UniqueIndexVersionV2
- ? Status::OK()
- : Status(ErrorCodes::UnsupportedFormat,
- str::stream()
- << "Index: {name: " << desc->indexName() << ", ns: " << desc->parentNS()
- << "} has incompatible format version: " << _dataFormatVersion);
- fassertNoTrace(31179, versionStatus);
+ if (!desc->isIdIndex() && desc->unique() &&
+ _dataFormatVersion != kDataFormatV3KeyStringV0UniqueIndexVersionV1 &&
+ _dataFormatVersion != kDataFormatV4KeyStringV1UniqueIndexVersionV2) {
+ auto collectionNamespace = desc->getEntry()->getNSSFromCatalog(ctx);
+ Status versionStatus(ErrorCodes::UnsupportedFormat,
+ str::stream()
+ << "Index: {name: " << desc->indexName()
+ << ", ns: " << collectionNamespace
+ << "} has incompatible format version: " << _dataFormatVersion);
+ fassertFailedWithStatusNoTrace(31179, versionStatus);
}
if (!isReadOnly) {
bool replicatedWrites = getGlobalReplSettings().usingReplSets() ||
repl::ReplSettings::shouldRecoverFromOplogAsStandalone();
- uassertStatusOK(WiredTigerUtil::setTableLogging(
- ctx,
- uri,
- WiredTigerUtil::useTableLogging(NamespaceString(desc->parentNS()), replicatedWrites)));
+ bool useTableLogging = !replicatedWrites ||
+ WiredTigerUtil::useTableLogging(desc->getEntry()->getNSSFromCatalog(ctx),
+ replicatedWrites);
+ uassertStatusOK(WiredTigerUtil::setTableLogging(ctx, uri, useTableLogging));
}
/*
@@ -692,8 +702,10 @@ private:
if (cmp == 0) {
// Duplicate found!
auto newKey = KeyString::toBson(newKeyString, _idx->_ordering);
+ auto entry = _idx->_desc->getEntry();
return buildDupKeyErrorStatus(newKey,
- _idx->collectionNamespace(),
+ entry ? entry->getNSSFromCatalog(_opCtx)
+ : NamespaceString(),
_idx->indexName(),
_idx->keyPattern(),
_idx->_collation);
@@ -741,7 +753,7 @@ private:
if (!_dupsAllowed) {
auto newKey = KeyString::toBson(newKeyString, _idx->_ordering);
return buildDupKeyErrorStatus(newKey,
- _idx->collectionNamespace(),
+ _idx->_desc->getEntry()->getNSSFromCatalog(_opCtx),
_idx->indexName(),
_idx->keyPattern(),
_idx->_collation);
@@ -1345,7 +1357,7 @@ private:
"key"_attr = redact(curr(kWantKey)->key),
"index"_attr = _idx.indexName(),
"uri"_attr = _idx.uri(),
- "collection"_attr = _idx.collectionNamespace());
+ "collection"_attr = _idx.getCollectionNamespace(_opCtx));
}
}
};
@@ -1528,7 +1540,7 @@ Status WiredTigerIndexUnique::_insertTimestampUnsafe(OperationContext* opCtx,
if (!dupsAllowed) {
auto key = KeyString::toBson(keyString, _ordering);
return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation);
+ key, _desc->getEntry()->getNSSFromCatalog(opCtx), _indexName, _keyPattern, _collation);
}
if (!insertedId) {
@@ -1574,8 +1586,11 @@ Status WiredTigerIndexUnique::_insertTimestampSafe(OperationContext* opCtx,
if (ret == WT_DUPLICATE_KEY) {
auto key = KeyString::toBson(
keyString.getBuffer(), sizeWithoutRecordId, _ordering, keyString.getTypeBits());
- return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation);
+ return buildDupKeyErrorStatus(key,
+ _desc->getEntry()->getNSSFromCatalog(opCtx),
+ _indexName,
+ _keyPattern,
+ _collation);
}
invariantWTOK(ret);
@@ -1590,8 +1605,13 @@ Status WiredTigerIndexUnique::_insertTimestampSafe(OperationContext* opCtx,
if (_keyExists(opCtx, c, keyString.getBuffer(), sizeWithoutRecordId)) {
auto key = KeyString::toBson(
keyString.getBuffer(), sizeWithoutRecordId, _ordering, keyString.getTypeBits());
- return buildDupKeyErrorStatus(
- key, _collectionNamespace, _indexName, _keyPattern, _collation);
+ auto entry = _desc->getEntry();
+ return buildDupKeyErrorStatus(key,
+ entry ? entry->getNSSFromCatalog(opCtx)
+ : NamespaceString(),
+ _indexName,
+ _keyPattern,
+ _collation);
}
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_index.h b/src/mongo/db/storage/wiredtiger/wiredtiger_index.h
index dddf3dc054c..a89dfe1572c 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_index.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_index.h
@@ -76,6 +76,7 @@ public:
static StatusWith<std::string> generateCreateString(const std::string& engineName,
const std::string& sysIndexConfig,
const std::string& collIndexConfig,
+ const NamespaceString& collectionNamespace,
const IndexDescriptor& desc,
bool isPrefixed);
@@ -127,14 +128,12 @@ public:
return _tableId;
}
- const NamespaceString& collectionNamespace() const {
- return _collectionNamespace;
- }
-
std::string indexName() const {
return _indexName;
}
+ NamespaceString getCollectionNamespace(OperationContext* opCtx) const;
+
const BSONObj& keyPattern() const {
return _keyPattern;
}
@@ -180,7 +179,7 @@ protected:
int _dataFormatVersion;
std::string _uri;
uint64_t _tableId;
- const NamespaceString _collectionNamespace;
+ const IndexDescriptor* _desc;
const std::string _indexName;
const BSONObj _keyPattern;
const BSONObj _collation;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_init.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_init.cpp
index 4edee81f671..46043f9da4c 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_init.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_init.cpp
@@ -64,8 +64,8 @@ namespace {
class WiredTigerFactory : public StorageEngine::Factory {
public:
virtual ~WiredTigerFactory() {}
- virtual StorageEngine* create(const StorageGlobalParams& params,
- const StorageEngineLockFile* lockFile) const {
+ virtual std::unique_ptr<StorageEngine> create(const StorageGlobalParams& params,
+ const StorageEngineLockFile* lockFile) const {
if (lockFile && lockFile->createdByUncleanShutdown()) {
LOGV2_WARNING(22302, "Recovering data from the last clean checkpoint.");
}
@@ -100,17 +100,17 @@ public:
}
}
const bool ephemeral = false;
- WiredTigerKVEngine* kv =
- new WiredTigerKVEngine(getCanonicalName().toString(),
- params.dbpath,
- getGlobalServiceContext()->getFastClockSource(),
- wiredTigerGlobalOptions.engineConfig,
- cacheMB,
- wiredTigerGlobalOptions.getMaxHistoryFileSizeMB(),
- params.dur,
- ephemeral,
- params.repair,
- params.readOnly);
+ auto kv =
+ std::make_unique<WiredTigerKVEngine>(getCanonicalName().toString(),
+ params.dbpath,
+ getGlobalServiceContext()->getFastClockSource(),
+ wiredTigerGlobalOptions.engineConfig,
+ cacheMB,
+ wiredTigerGlobalOptions.getMaxHistoryFileSizeMB(),
+ params.dur,
+ ephemeral,
+ params.repair,
+ params.readOnly);
kv->setRecordStoreExtraOptions(wiredTigerGlobalOptions.collectionConfig);
kv->setSortedDataInterfaceExtraOptions(wiredTigerGlobalOptions.indexConfig);
@@ -121,7 +121,7 @@ public:
// Intentionally leaked.
MONGO_COMPILER_VARIABLE_UNUSED auto leakedSection =
- new WiredTigerServerStatusSection(kv);
+ new WiredTigerServerStatusSection(kv.get());
// This allows unit tests to run this code without encountering memory leaks
#if __has_feature(address_sanitizer)
@@ -133,7 +133,7 @@ public:
options.directoryPerDB = params.directoryperdb;
options.directoryForIndexes = wiredTigerGlobalOptions.directoryForIndexes;
options.forRepair = params.repair;
- return new StorageEngineImpl(kv, options);
+ return std::make_unique<StorageEngineImpl>(std::move(kv), options);
}
virtual StringData getCanonicalName() const {
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
index 53f7a3124a7..98070942338 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
@@ -58,6 +58,7 @@
#include "mongo/bson/bsonobjbuilder.h"
#include "mongo/db/bson/dotted_path_support.h"
#include "mongo/db/catalog/collection.h"
+#include "mongo/db/catalog/collection_catalog.h"
#include "mongo/db/client.h"
#include "mongo/db/commands/server_status_metric.h"
#include "mongo/db/concurrency/locker.h"
@@ -1662,34 +1663,34 @@ Status WiredTigerKVEngine::createGroupedSortedDataInterface(OperationContext* op
_ensureIdentPath(ident);
std::string collIndexOptions;
- const Collection* collection = desc->getCollection();
-
- // Treat 'collIndexOptions' as an empty string when the collection member of 'desc' is NULL in
- // order to allow for unit testing WiredTigerKVEngine::createSortedDataInterface().
- if (collection) {
- if (!collOptions.indexOptionDefaults["storageEngine"].eoo()) {
- BSONObj storageEngineOptions = collOptions.indexOptionDefaults["storageEngine"].Obj();
- collIndexOptions =
- dps::extractElementAtPath(storageEngineOptions, _canonicalName + ".configString")
- .valuestrsafe();
- }
+
+ if (!collOptions.indexOptionDefaults["storageEngine"].eoo()) {
+ BSONObj storageEngineOptions = collOptions.indexOptionDefaults["storageEngine"].Obj();
+ collIndexOptions =
+ dps::extractElementAtPath(storageEngineOptions, _canonicalName + ".configString")
+ .valuestrsafe();
}
+ // Some unittests use a OperationContextNoop that can't support such lookups.
+ auto ns = collOptions.uuid
+ ? *CollectionCatalog::get(opCtx).lookupNSSByUUID(opCtx, *collOptions.uuid)
+ : NamespaceString();
StatusWith<std::string> result = WiredTigerIndex::generateCreateString(
- _canonicalName, _indexOptions, collIndexOptions, *desc, prefix.isPrefixed());
+ _canonicalName, _indexOptions, collIndexOptions, ns, *desc, prefix.isPrefixed());
if (!result.isOK()) {
return result.getStatus();
}
std::string config = result.getValue();
- LOGV2_DEBUG(22336,
- 2,
- "WiredTigerKVEngine::createSortedDataInterface ns: {collection_ns} ident: {ident} "
- "config: {config}",
- "collection_ns"_attr = collection->ns(),
- "ident"_attr = ident,
- "config"_attr = config);
+ LOGV2_DEBUG(
+ 22336,
+ 2,
+ "WiredTigerKVEngine::createSortedDataInterface uuid: {collection_uuid} ident: {ident} "
+ "config: {config}",
+ "collection_uuid"_attr = collOptions.uuid,
+ "ident"_attr = ident,
+ "config"_attr = config);
return wtRCToStatus(WiredTigerIndex::Create(opCtx, _uri(ident), config));
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine_test.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine_test.cpp
index 7160532f38d..a92f75c0c1f 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine_test.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine_test.cpp
@@ -59,18 +59,16 @@ namespace {
class WiredTigerKVHarnessHelper : public KVHarnessHelper, public ScopedGlobalServiceContextForTest {
public:
WiredTigerKVHarnessHelper(bool forRepair = false)
- : _dbpath("wt-kv-harness"), _forRepair(forRepair) {
- invariant(hasGlobalServiceContext());
- _engine.reset(makeEngine());
+ : _dbpath("wt-kv-harness"), _forRepair(forRepair), _engine(makeEngine()) {
+ auto context = getGlobalServiceContext();
repl::ReplicationCoordinator::set(
- getGlobalServiceContext(),
- std::unique_ptr<repl::ReplicationCoordinator>(new repl::ReplicationCoordinatorMock(
- getGlobalServiceContext(), repl::ReplSettings())));
+ context,
+ std::make_unique<repl::ReplicationCoordinatorMock>(context, repl::ReplSettings()));
}
virtual KVEngine* restartEngine() override {
_engine.reset(nullptr);
- _engine.reset(makeEngine());
+ _engine = makeEngine();
return _engine.get();
}
@@ -83,17 +81,17 @@ public:
}
private:
- WiredTigerKVEngine* makeEngine() {
- auto engine = new WiredTigerKVEngine(kWiredTigerEngineName,
- _dbpath.path(),
- _cs.get(),
- "",
- 1,
- 0,
- false,
- false,
- _forRepair,
- false);
+ std::unique_ptr<WiredTigerKVEngine> makeEngine() {
+ auto engine = std::make_unique<WiredTigerKVEngine>(kWiredTigerEngineName,
+ _dbpath.path(),
+ _cs.get(),
+ "",
+ 1,
+ 0,
+ false,
+ false,
+ _forRepair,
+ false);
// There are unit tests expecting checkpoints to occur asynchronously.
engine->startAsyncThreads();
return engine;
@@ -101,8 +99,8 @@ private:
const std::unique_ptr<ClockSource> _cs = std::make_unique<ClockSourceMock>();
unittest::TempDir _dbpath;
- std::unique_ptr<WiredTigerKVEngine> _engine;
bool _forRepair;
+ std::unique_ptr<WiredTigerKVEngine> _engine;
};
class WiredTigerKVEngineTest : public unittest::Test, public ScopedGlobalServiceContextForTest {
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_prefixed_index_test.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_prefixed_index_test.cpp
index d069c12ee13..9f7e391a709 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_prefixed_index_test.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_prefixed_index_test.cpp
@@ -71,23 +71,23 @@ public:
}
std::unique_ptr<SortedDataInterface> newIdIndexSortedDataInterface() final {
- std::string ns = "test.wt";
+ NamespaceString nss = {"test", "wt"};
OperationContextNoop opCtx(newRecoveryUnit().release());
BSONObj spec = BSON("key" << BSON("_id" << 1) << "name"
<< "_id_"
<< "v" << static_cast<int>(IndexDescriptor::kLatestIndexVersion)
<< "unique" << true);
- auto collection = std::make_unique<CollectionMock>(NamespaceString(ns));
+ auto collection = std::make_unique<CollectionMock>(nss);
IndexDescriptor desc(collection.get(), "", spec);
invariant(desc.isIdIndex());
KVPrefix prefix = KVPrefix::generateNextPrefix();
StatusWith<std::string> result = WiredTigerIndex::generateCreateString(
- kWiredTigerEngineName, "", "", desc, prefix.isPrefixed());
+ kWiredTigerEngineName, "", "", nss, desc, prefix.isPrefixed());
ASSERT_OK(result.getStatus());
- string uri = "table:" + ns;
+ string uri = "table:" + nss.ns();
invariantWTOK(WiredTigerIndex::Create(&opCtx, uri, result.getValue()));
return std::make_unique<WiredTigerIndexUnique>(&opCtx, uri, &desc, prefix);
@@ -95,7 +95,7 @@ public:
std::unique_ptr<SortedDataInterface> newSortedDataInterface(bool unique, bool partial) final {
- std::string ns = "test.wt";
+ NamespaceString nss = {"test", "wt"};
OperationContextNoop opCtx(newRecoveryUnit().release());
BSONObj spec = BSON("key" << BSON("a" << 1) << "name"
@@ -110,15 +110,15 @@ public:
spec = spec.addField(partialBSON.firstElement());
}
- auto collection = std::make_unique<CollectionMock>(NamespaceString(ns));
- IndexDescriptor desc(collection.get(), "", spec);
+ auto collection = std::make_unique<CollectionMock>(nss);
+ IndexDescriptor& desc = _descriptors.emplace_back(collection.get(), "", spec);
KVPrefix prefix = KVPrefix::generateNextPrefix();
StatusWith<std::string> result = WiredTigerIndex::generateCreateString(
- kWiredTigerEngineName, "", "", desc, prefix.isPrefixed());
+ kWiredTigerEngineName, "", "", nss, desc, prefix.isPrefixed());
ASSERT_OK(result.getStatus());
- string uri = "table:" + ns;
+ string uri = "table:" + nss.ns();
invariantWTOK(WiredTigerIndex::Create(&opCtx, uri, result.getValue()));
if (unique)
@@ -133,6 +133,7 @@ public:
private:
unittest::TempDir _dbpath;
std::unique_ptr<ClockSource> _fastClockSource;
+ std::vector<IndexDescriptor> _descriptors;
WT_CONNECTION* _conn;
WiredTigerSessionCache* _sessionCache;
WiredTigerOplogManager _oplogManager;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_standard_index_test.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_standard_index_test.cpp
index 0d736c6a3b8..abf9e7a500d 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_standard_index_test.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_standard_index_test.cpp
@@ -85,7 +85,7 @@ public:
KVPrefix prefix = KVPrefix::kNotPrefixed;
StatusWith<std::string> result = WiredTigerIndex::generateCreateString(
- kWiredTigerEngineName, "", "", desc, prefix.isPrefixed());
+ kWiredTigerEngineName, "", "", NamespaceString(ns), desc, prefix.isPrefixed());
ASSERT_OK(result.getStatus());
string uri = "table:" + ns;
@@ -111,11 +111,12 @@ public:
}
auto collection = std::make_unique<CollectionMock>(NamespaceString(ns));
- IndexDescriptor desc(collection.get(), "", spec);
+
+ IndexDescriptor& desc = _descriptors.emplace_back(collection.get(), "", spec);
KVPrefix prefix = KVPrefix::kNotPrefixed;
StatusWith<std::string> result = WiredTigerIndex::generateCreateString(
- kWiredTigerEngineName, "", "", desc, prefix.isPrefixed());
+ kWiredTigerEngineName, "", "", NamespaceString(ns), desc, prefix.isPrefixed());
ASSERT_OK(result.getStatus());
string uri = "table:" + ns;
@@ -133,6 +134,7 @@ public:
private:
unittest::TempDir _dbpath;
std::unique_ptr<ClockSource> _fastClockSource;
+ std::vector<IndexDescriptor> _descriptors;
WT_CONNECTION* _conn;
WiredTigerSessionCache* _sessionCache;
WiredTigerOplogManager _oplogManager;
diff --git a/src/mongo/db/ttl.cpp b/src/mongo/db/ttl.cpp
index 711bafcc8d4..987b66976b7 100644
--- a/src/mongo/db/ttl.cpp
+++ b/src/mongo/db/ttl.cpp
@@ -381,11 +381,11 @@ private:
try {
exec->executePlan();
} catch (const DBException& exception) {
- LOGV2_ERROR(22543,
- "ttl query execution for index {index} failed with status: {error}",
- "TTL query execution failed",
- "index"_attr = idx,
- "error"_attr = redact(exception.toStatus()));
+ LOGV2_WARNING(22543,
+ "ttl query execution for index {index} failed with status: {error}",
+ "TTL query execution failed",
+ "index"_attr = idx,
+ "error"_attr = redact(exception.toStatus()));
return;
}
diff --git a/src/mongo/dbtests/query_plan_executor.cpp b/src/mongo/dbtests/query_plan_executor.cpp
index 32c61a80eb5..529a572097f 100644
--- a/src/mongo/dbtests/query_plan_executor.cpp
+++ b/src/mongo/dbtests/query_plan_executor.cpp
@@ -153,7 +153,7 @@ public:
unique_ptr<WorkingSet> ws(new WorkingSet());
- auto ixscan = std::make_unique<IndexScan>(_expCtx.get(), ixparams, ws.get(), nullptr);
+ auto ixscan = std::make_unique<IndexScan>(_expCtx.get(), coll, ixparams, ws.get(), nullptr);
unique_ptr<PlanStage> root =
std::make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ixscan), nullptr, coll);
diff --git a/src/mongo/dbtests/query_stage_and.cpp b/src/mongo/dbtests/query_stage_and.cpp
index 9070b5f5621..96dac1ea2e5 100644
--- a/src/mongo/dbtests/query_stage_and.cpp
+++ b/src/mongo/dbtests/query_stage_and.cpp
@@ -199,12 +199,12 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10.
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 10);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// 'ah' reads the first child into its hash table: foo=20, foo=19, ..., foo=0
// in that order. Read half of them.
@@ -286,13 +286,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar <= 19 (descending).
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 19);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// First call to work reads the first result from the children. The first result for the
// first scan over foo is {foo: 20, bar: 20, baz: 20}. The first result for the second scan
@@ -366,13 +366,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 10);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// foo == bar == baz, and foo<=20, bar>=10, so our values are:
// foo == 10, 11, 12, 13, 14, 15. 16, 17, 18, 19, 20
@@ -415,13 +415,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1 << "big" << 1), coll));
params.bounds.startKey = BSON("" << 20 << "" << big);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 10);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
ASSERT_THROWS_CODE(countResults(ah.get()),
DBException,
@@ -463,13 +463,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1 << "big" << 1), coll));
params.bounds.startKey = BSON("" << 10 << "" << big);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// foo == bar == baz, and foo<=20, bar>=10, so our values are:
// foo == 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20.
@@ -505,18 +505,18 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 10);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// 5 <= baz <= 15
params = makeIndexScanParams(&_opCtx, getIndex(BSON("baz" << 1), coll));
params.bounds.startKey = BSON("" << 5);
params.bounds.endKey = BSON("" << 15);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// foo == bar == baz, and foo<=20, bar>=10, 5<=baz<=15, so our values are:
// foo == 10, 11, 12, 13, 14, 15.
@@ -563,18 +563,18 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1 << "big" << 1), coll));
params.bounds.startKey = BSON("" << 10 << "" << big);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// 5 <= baz <= 15
params = makeIndexScanParams(&_opCtx, getIndex(BSON("baz" << 1), coll));
params.bounds.startKey = BSON("" << 5);
params.bounds.endKey = BSON("" << 15);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Stage execution should fail.
ASSERT_THROWS_CODE(countResults(ah.get()),
@@ -610,13 +610,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar == 5. Index scan should be eof.
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 5);
params.bounds.endKey = BSON("" << 5);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
int count = 0;
int works = 0;
@@ -666,7 +666,7 @@ public:
// Foo >= 100
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 100);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar <= 100
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
@@ -678,7 +678,7 @@ public:
<< "");
params.bounds.boundInclusion = BoundInclusion::kIncludeStartKeyOnly;
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
ASSERT_EQUALS(0, countResults(ah.get()));
}
@@ -714,7 +714,7 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- auto firstScan = std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr);
+ auto firstScan = std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr);
// First child of the AND_HASH stage is a Fetch. The NULL in the
// constructor means there is no filter.
@@ -725,7 +725,7 @@ public:
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 10);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Check that the AndHash stage returns docs {foo: 10, bar: 10}
// through {foo: 20, bar: 20}.
@@ -767,12 +767,12 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.direction = -1;
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar >= 10
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 10);
- auto secondScan = std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr);
+ auto secondScan = std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr);
// Second child of the AND_HASH stage is a Fetch. The NULL in the
// constructor means there is no filter.
@@ -939,13 +939,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Scan over bar == 1.
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Get the set of RecordIds in our collection to use later.
set<RecordId> data;
@@ -1056,19 +1056,19 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// bar == 1
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// baz == 1
params = makeIndexScanParams(&_opCtx, getIndex(BSON("baz" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
ASSERT_EQUALS(50, countResults(ah.get()));
}
@@ -1101,13 +1101,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 7);
params.bounds.endKey = BSON("" << 7);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Bar == 20, not EOF.
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.bounds.endKey = BSON("" << 20);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
ASSERT_EQUALS(0, countResults(ah.get()));
}
@@ -1144,13 +1144,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 7);
params.bounds.endKey = BSON("" << 7);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// bar == 20.
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 20);
params.bounds.endKey = BSON("" << 20);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
ASSERT_EQUALS(0, countResults(ah.get()));
}
@@ -1183,13 +1183,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// Intersect with 7 <= bar < 10000
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 7);
params.bounds.endKey = BSON("" << 10000);
- ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ah->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
WorkingSetID lastId = WorkingSet::INVALID_ID;
@@ -1245,7 +1245,7 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- auto firstScan = std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr);
+ auto firstScan = std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr);
// First child of the AND_SORTED stage is a Fetch. The NULL in the
// constructor means there is no filter.
@@ -1257,7 +1257,7 @@ public:
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- as->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ as->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
for (int i = 0; i < 50; i++) {
BSONObj obj = getNext(as.get(), &ws);
@@ -1298,13 +1298,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(BSON("foo" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- as->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ as->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
// bar == 1
params = makeIndexScanParams(&_opCtx, getIndex(BSON("bar" << 1), coll));
params.bounds.startKey = BSON("" << 1);
params.bounds.endKey = BSON("" << 1);
- auto secondScan = std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr);
+ auto secondScan = std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr);
// Second child of the AND_SORTED stage is a Fetch. The NULL in the
// constructor means there is no filter.
diff --git a/src/mongo/dbtests/query_stage_count.cpp b/src/mongo/dbtests/query_stage_count.cpp
index 495105bad9c..a43346e849b 100644
--- a/src/mongo/dbtests/query_stage_count.cpp
+++ b/src/mongo/dbtests/query_stage_count.cpp
@@ -212,7 +212,7 @@ public:
params.direction = 1;
// This child stage gets owned and freed by its parent CountStage
- return new IndexScan(_expCtx.get(), params, ws, expr);
+ return new IndexScan(_expCtx.get(), _coll, params, ws, expr);
}
CollectionScan* createCollScan(MatchExpression* expr, WorkingSet* ws) {
diff --git a/src/mongo/dbtests/query_stage_count_scan.cpp b/src/mongo/dbtests/query_stage_count_scan.cpp
index d89316948fc..422d37161e0 100644
--- a/src/mongo/dbtests/query_stage_count_scan.cpp
+++ b/src/mongo/dbtests/query_stage_count_scan.cpp
@@ -54,20 +54,20 @@ public:
CountBase() : _client(&_opCtx) {}
virtual ~CountBase() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
- _client.dropCollection(ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
+ _client.dropCollection(ns().ns());
}
void addIndex(const BSONObj& obj) {
- ASSERT_OK(dbtests::createIndex(&_opCtx, ns(), obj));
+ ASSERT_OK(dbtests::createIndex(&_opCtx, ns().ns(), obj));
}
void insert(const BSONObj& obj) {
- _client.insert(ns(), obj);
+ _client.insert(ns().ns(), obj);
}
void remove(const BSONObj& obj) {
- _client.remove(ns(), obj);
+ _client.remove(ns().ns(), obj);
}
/*
@@ -89,11 +89,13 @@ public:
return countWorks;
}
+ const Collection* getCollection() {
+ return CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, ns());
+ }
+
const IndexDescriptor* getIndex(Database* db, const BSONObj& obj) {
- Collection* collection = CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(
- &_opCtx, NamespaceString(ns()));
std::vector<const IndexDescriptor*> indexes;
- collection->getIndexCatalog()->findIndexesByKeyPattern(&_opCtx, obj, false, &indexes);
+ getCollection()->getIndexCatalog()->findIndexesByKeyPattern(&_opCtx, obj, false, &indexes);
return indexes.empty() ? nullptr : indexes[0];
}
@@ -102,8 +104,8 @@ public:
return {opCtx, descriptor};
}
- static const char* ns() {
- return "unittests.QueryStageCountScanScan";
+ static NamespaceString ns() {
+ return {"unittests", "QueryStageCountScanScan"};
}
protected:
@@ -111,7 +113,7 @@ protected:
OperationContext& _opCtx = *_txnPtr;
boost::intrusive_ptr<ExpressionContext> _expCtx =
- make_intrusive<ExpressionContext>(&_opCtx, nullptr, NamespaceString(ns()));
+ make_intrusive<ExpressionContext>(&_opCtx, nullptr, ns());
private:
DBDirectClient _client;
@@ -124,7 +126,7 @@ private:
class QueryStageCountScanDups : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert some docs
insert(BSON("a" << BSON_ARRAY(5 << 7)));
@@ -141,7 +143,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(2, numCounted);
@@ -154,7 +156,7 @@ public:
class QueryStageCountScanInclusiveBounds : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert some docs
for (int i = 0; i < 10; ++i) {
@@ -172,7 +174,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(5, numCounted);
@@ -185,7 +187,7 @@ public:
class QueryStageCountScanExclusiveBounds : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert some docs
for (int i = 0; i < 10; ++i) {
@@ -203,7 +205,7 @@ public:
params.endKeyInclusive = false;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(3, numCounted);
@@ -216,7 +218,7 @@ public:
class QueryStageCountScanLowerBound : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert doc, add index
insert(BSON("a" << 2));
@@ -230,7 +232,7 @@ public:
params.endKeyInclusive = false;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(0, numCounted);
@@ -243,7 +245,7 @@ public:
class QueryStageCountScanNothingInInterval : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert documents, add index
insert(BSON("a" << 2));
@@ -258,7 +260,7 @@ public:
params.endKeyInclusive = false;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(0, numCounted);
@@ -272,7 +274,7 @@ public:
class QueryStageCountScanNothingInIntervalFirstMatchTooHigh : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert some documents, add index
insert(BSON("a" << 2));
@@ -287,7 +289,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(0, numCounted);
@@ -301,7 +303,7 @@ public:
class QueryStageCountScanNoChangeDuringYield : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert documents, add index
for (int i = 0; i < 10; ++i) {
@@ -317,7 +319,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
WorkingSetID wsid;
int numCounted = 0;
@@ -353,7 +355,7 @@ public:
class QueryStageCountScanDeleteDuringYield : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert documents, add index
for (int i = 0; i < 10; ++i) {
@@ -369,7 +371,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
WorkingSetID wsid;
int numCounted = 0;
@@ -408,7 +410,7 @@ public:
class QueryStageCountScanInsertNewDocsDuringYield : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert documents, add index
for (int i = 0; i < 10; ++i) {
@@ -424,7 +426,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
WorkingSetID wsid;
int numCounted = 0;
@@ -465,7 +467,7 @@ public:
class QueryStageCountScanUnusedKeys : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert docs, add index
for (int i = 0; i < 10; ++i) {
@@ -486,7 +488,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(7, numCounted);
@@ -499,7 +501,7 @@ public:
class QueryStageCountScanUnusedEndKey : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert docs, add index
for (int i = 0; i < 10; ++i) {
@@ -518,7 +520,7 @@ public:
params.endKeyInclusive = true; // yes?
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
int numCounted = runCount(&count);
ASSERT_EQUALS(9, numCounted);
@@ -531,7 +533,7 @@ public:
class QueryStageCountScanKeyBecomesUnusedDuringYield : public CountBase {
public:
void run() {
- dbtests::WriteContextForTests ctx(&_opCtx, ns());
+ dbtests::WriteContextForTests ctx(&_opCtx, ns().ns());
// Insert documents, add index
for (int i = 0; i < 10; ++i) {
@@ -547,7 +549,7 @@ public:
params.endKeyInclusive = true;
WorkingSet ws;
- CountScan count(_expCtx.get(), params, &ws);
+ CountScan count(_expCtx.get(), getCollection(), params, &ws);
WorkingSetID wsid;
int numCounted = 0;
diff --git a/src/mongo/dbtests/query_stage_distinct.cpp b/src/mongo/dbtests/query_stage_distinct.cpp
index ee369386069..92a7bd2644e 100644
--- a/src/mongo/dbtests/query_stage_distinct.cpp
+++ b/src/mongo/dbtests/query_stage_distinct.cpp
@@ -144,7 +144,7 @@ public:
params.bounds.fields.push_back(oil);
WorkingSet ws;
- DistinctScan distinct(_expCtx.get(), std::move(params), &ws);
+ DistinctScan distinct(_expCtx.get(), coll, std::move(params), &ws);
WorkingSetID wsid;
// Get our first result.
@@ -212,7 +212,7 @@ public:
params.bounds.fields.push_back(oil);
WorkingSet ws;
- DistinctScan distinct(_expCtx.get(), std::move(params), &ws);
+ DistinctScan distinct(_expCtx.get(), coll, std::move(params), &ws);
// We should see each number in the range [1, 6] exactly once.
std::set<int> seen;
@@ -281,7 +281,7 @@ public:
params.bounds.fields.push_back(bOil);
WorkingSet ws;
- DistinctScan distinct(_expCtx.get(), std::move(params), &ws);
+ DistinctScan distinct(_expCtx.get(), coll, std::move(params), &ws);
WorkingSetID wsid;
PlanStage::StageState state;
diff --git a/src/mongo/dbtests/query_stage_ixscan.cpp b/src/mongo/dbtests/query_stage_ixscan.cpp
index 0a46f2a34ae..d74a212b865 100644
--- a/src/mongo/dbtests/query_stage_ixscan.cpp
+++ b/src/mongo/dbtests/query_stage_ixscan.cpp
@@ -108,7 +108,7 @@ public:
// This child stage gets owned and freed by the caller.
MatchExpression* filter = nullptr;
- return new IndexScan(_expCtx.get(), params, &_ws, filter);
+ return new IndexScan(_expCtx.get(), _coll, params, &_ws, filter);
}
IndexScan* createIndexScan(BSONObj startKey,
@@ -132,7 +132,7 @@ public:
params.bounds.fields.push_back(oil);
MatchExpression* filter = nullptr;
- return new IndexScan(_expCtx.get(), params, &_ws, filter);
+ return new IndexScan(_expCtx.get(), _coll, params, &_ws, filter);
}
static const char* ns() {
diff --git a/src/mongo/dbtests/query_stage_merge_sort.cpp b/src/mongo/dbtests/query_stage_merge_sort.cpp
index 74b6d6d1843..3ad17b1cf52 100644
--- a/src/mongo/dbtests/query_stage_merge_sort.cpp
+++ b/src/mongo/dbtests/query_stage_merge_sort.cpp
@@ -177,11 +177,11 @@ public:
// a:1
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:1
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -243,11 +243,11 @@ public:
// a:1
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:1
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -308,11 +308,11 @@ public:
// a:1
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:1
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -377,13 +377,13 @@ public:
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
params.bounds.startKey = objWithMaxKey(1);
params.bounds.endKey = objWithMinKey(1);
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:1
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
params.bounds.startKey = objWithMaxKey(1);
params.bounds.endKey = objWithMinKey(1);
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -444,13 +444,13 @@ public:
// a:1
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:51 (EOF)
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
params.bounds.startKey = BSON("" << 51 << "" << MinKey);
params.bounds.endKey = BSON("" << 51 << "" << MaxKey);
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -502,7 +502,8 @@ public:
BSONObj indexSpec = BSON(index << 1 << "foo" << 1);
addIndex(indexSpec);
auto params = makeIndexScanParams(&_opCtx, getIndex(indexSpec, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(
+ std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
}
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -557,7 +558,7 @@ public:
BSONObj indexSpec = BSON(index << 1 << "foo" << 1);
addIndex(indexSpec);
auto params = makeIndexScanParams(&_opCtx, getIndex(indexSpec, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr));
}
set<RecordId> recordIds;
@@ -685,7 +686,7 @@ public:
auto fetchStage = std::make_unique<FetchStage>(
_expCtx.get(),
&ws,
- std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr),
+ std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr),
nullptr,
coll);
ms->addChild(std::move(fetchStage));
@@ -699,7 +700,7 @@ public:
auto fetchStage = std::make_unique<FetchStage>(
_expCtx.get(),
&ws,
- std::make_unique<IndexScan>(_expCtx.get(), params, &ws, nullptr),
+ std::make_unique<IndexScan>(_expCtx.get(), coll, params, &ws, nullptr),
nullptr,
coll);
ms->addChild(std::move(fetchStage));
@@ -787,11 +788,11 @@ public:
// a:1
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:1
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
auto fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
@@ -856,11 +857,11 @@ public:
// a:1
auto params = makeIndexScanParams(&_opCtx, getIndex(firstIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
// b:1
params = makeIndexScanParams(&_opCtx, getIndex(secondIndex, coll));
- ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), nullptr));
+ ms->addChild(std::make_unique<IndexScan>(_expCtx.get(), coll, params, ws.get(), nullptr));
unique_ptr<FetchStage> fetchStage =
make_unique<FetchStage>(_expCtx.get(), ws.get(), std::move(ms), nullptr, coll);
diff --git a/src/mongo/dbtests/query_stage_multiplan.cpp b/src/mongo/dbtests/query_stage_multiplan.cpp
index 2236b34ac35..1e12492f4e5 100644
--- a/src/mongo/dbtests/query_stage_multiplan.cpp
+++ b/src/mongo/dbtests/query_stage_multiplan.cpp
@@ -152,7 +152,7 @@ unique_ptr<PlanStage> getIxScanPlan(ExpressionContext* expCtx,
ixparams.bounds.boundInclusion = BoundInclusion::kIncludeBothStartAndEndKeys;
ixparams.direction = 1;
- auto ixscan = std::make_unique<IndexScan>(expCtx, ixparams, sharedWs, nullptr);
+ auto ixscan = std::make_unique<IndexScan>(expCtx, coll, ixparams, sharedWs, nullptr);
return std::make_unique<FetchStage>(expCtx, sharedWs, std::move(ixscan), nullptr, coll);
}
diff --git a/src/mongo/dbtests/query_stage_near.cpp b/src/mongo/dbtests/query_stage_near.cpp
index b9c372c1dd2..ca9fe26c240 100644
--- a/src/mongo/dbtests/query_stage_near.cpp
+++ b/src/mongo/dbtests/query_stage_near.cpp
@@ -74,6 +74,10 @@ public:
ASSERT(_mockGeoIndex);
}
+ const Collection* getCollection() const {
+ return _autoColl->getCollection();
+ }
+
protected:
BSONObj _makeMinimalIndexSpec(BSONObj keyPattern) {
return BSON(IndexDescriptor::kKeyPatternFieldName
@@ -108,11 +112,13 @@ public:
MockNearStage(const boost::intrusive_ptr<ExpressionContext>& expCtx,
WorkingSet* workingSet,
+ const Collection* coll,
const IndexDescriptor* indexDescriptor)
: NearStage(expCtx.get(),
"MOCK_DISTANCE_SEARCH_STAGE",
STAGE_UNKNOWN,
workingSet,
+ coll,
indexDescriptor),
_pos(0) {}
@@ -192,7 +198,7 @@ TEST_F(QueryStageNearTest, Basic) {
vector<BSONObj> mockData;
WorkingSet workingSet;
- MockNearStage nearStage(_expCtx.get(), &workingSet, _mockGeoIndex);
+ MockNearStage nearStage(_expCtx.get(), &workingSet, getCollection(), _mockGeoIndex);
// First set of results
mockData.clear();
@@ -231,7 +237,7 @@ TEST_F(QueryStageNearTest, EmptyResults) {
auto* coll = autoColl.getCollection();
ASSERT(coll);
- MockNearStage nearStage(_expCtx.get(), &workingSet, _mockGeoIndex);
+ MockNearStage nearStage(_expCtx.get(), &workingSet, coll, _mockGeoIndex);
// Empty set of results
mockData.clear();
diff --git a/src/mongo/dbtests/query_stage_tests.cpp b/src/mongo/dbtests/query_stage_tests.cpp
index b3003d56213..1c705c3ff12 100644
--- a/src/mongo/dbtests/query_stage_tests.cpp
+++ b/src/mongo/dbtests/query_stage_tests.cpp
@@ -89,8 +89,8 @@ public:
unique_ptr<MatchExpression> filterExpr = std::move(statusWithMatcher.getValue());
unique_ptr<WorkingSet> ws = std::make_unique<WorkingSet>();
- unique_ptr<IndexScan> ix =
- std::make_unique<IndexScan>(_expCtx.get(), params, ws.get(), filterExpr.get());
+ unique_ptr<IndexScan> ix = std::make_unique<IndexScan>(
+ _expCtx.get(), ctx.getCollection(), params, ws.get(), filterExpr.get());
auto statusWithPlanExecutor = PlanExecutor::make(
_expCtx, std::move(ws), std::move(ix), ctx.getCollection(), PlanExecutor::NO_YIELD);
diff --git a/src/mongo/dbtests/validate_tests.cpp b/src/mongo/dbtests/validate_tests.cpp
index 17538a57289..c98fcd22aea 100644
--- a/src/mongo/dbtests/validate_tests.cpp
+++ b/src/mongo/dbtests/validate_tests.cpp
@@ -903,7 +903,7 @@ public:
auto removeStatus =
iam->removeKeys(&_opCtx, {keys.begin(), keys.end()}, id1, options, &numDeleted);
- auto insertStatus = iam->insert(&_opCtx, badKey, id1, options, &insertResult);
+ auto insertStatus = iam->insert(&_opCtx, coll, badKey, id1, options, &insertResult);
ASSERT_EQUALS(numDeleted, 1);
ASSERT_EQUALS(insertResult.numInserted, 1);
@@ -1710,6 +1710,7 @@ public:
InsertResult result;
auto insertStatus = iam->insertKeys(&_opCtx,
+ coll,
{keys.begin(), keys.end()},
{},
MultikeyPaths{},
@@ -1741,6 +1742,7 @@ public:
IndexAccessMethod::kNoopOnSuppressedErrorFn);
ASSERT_EQ(1, keys.size());
auto insertStatus = iam->insertKeys(&_opCtx,
+ coll,
{keys.begin(), keys.end()},
{},
MultikeyPaths{},
diff --git a/src/mongo/dbtests/wildcard_multikey_persistence_test.cpp b/src/mongo/dbtests/wildcard_multikey_persistence_test.cpp
index 2dd3de7f0c0..6b3ceac8ad2 100644
--- a/src/mongo/dbtests/wildcard_multikey_persistence_test.cpp
+++ b/src/mongo/dbtests/wildcard_multikey_persistence_test.cpp
@@ -97,7 +97,8 @@ protected:
auto collection = autoColl.getCollection();
// Verify whether or not the index has been marked as multikey.
- ASSERT_EQ(expectIndexIsMultikey, getIndexDesc(collection, indexName)->isMultikey());
+ ASSERT_EQ(expectIndexIsMultikey,
+ getIndexDesc(collection, indexName)->getEntry()->isMultikey());
// Obtain a cursor over the index, and confirm that the keys are present in order.
auto indexCursor = getIndexCursor(collection, indexName);