summaryrefslogtreecommitdiff
path: root/src/mongo/db/pipeline
diff options
context:
space:
mode:
authorDavid Storch <david.storch@10gen.com>2018-08-15 13:38:08 -0400
committerDavid Storch <david.storch@10gen.com>2018-08-21 18:54:22 -0400
commit21762f8b9b8c64fd32bed173eb74e2172088e21b (patch)
tree6ad4fef6ca4a2e303bfc6aacb55b03b77d268417 /src/mongo/db/pipeline
parente2f47a13f11d5a4aa2b7f23e1e0c157d90435e9e (diff)
downloadmongo-21762f8b9b8c64fd32bed173eb74e2172088e21b.tar.gz
SERVER-36528 Implement $planCacheStats agg source.
The $planCacheStats agg metadata source should typically be used instead of the older 'planCacheListPlans' and 'planCacheListQueryShapes' commands (although these plan cache commands are still fully supported). Lookup by 'queryHash' can be achieved via a $planCacheStats-$match pipeline.
Diffstat (limited to 'src/mongo/db/pipeline')
-rw-r--r--src/mongo/db/pipeline/SConscript2
-rw-r--r--src/mongo/db/pipeline/document_source_plan_cache_stats.cpp112
-rw-r--r--src/mongo/db/pipeline/document_source_plan_cache_stats.h144
-rw-r--r--src/mongo/db/pipeline/document_source_plan_cache_stats_test.cpp178
-rw-r--r--src/mongo/db/pipeline/mongo_process_interface.h10
-rw-r--r--src/mongo/db/pipeline/mongod_process_interface.cpp25
-rw-r--r--src/mongo/db/pipeline/mongod_process_interface.h4
-rw-r--r--src/mongo/db/pipeline/mongos_process_interface.h11
-rw-r--r--src/mongo/db/pipeline/stub_mongo_process_interface.h6
9 files changed, 492 insertions, 0 deletions
diff --git a/src/mongo/db/pipeline/SConscript b/src/mongo/db/pipeline/SConscript
index 952f7baf5f6..d26d429a29a 100644
--- a/src/mongo/db/pipeline/SConscript
+++ b/src/mongo/db/pipeline/SConscript
@@ -202,6 +202,7 @@ env.CppUnitTest(
'document_source_check_resume_token_test.cpp',
'document_source_count_test.cpp',
'document_source_current_op_test.cpp',
+ 'document_source_plan_cache_stats_test.cpp',
'document_source_exchange_test.cpp',
'document_source_geo_near_test.cpp',
'document_source_graph_lookup_test.cpp',
@@ -336,6 +337,7 @@ pipelineeEnv.Library(
'document_source_update_on_add_shard.cpp',
'document_source_out.cpp',
'document_source_out_replace_coll.cpp',
+ 'document_source_plan_cache_stats.cpp',
'document_source_project.cpp',
'document_source_redact.cpp',
'document_source_replace_root.cpp',
diff --git a/src/mongo/db/pipeline/document_source_plan_cache_stats.cpp b/src/mongo/db/pipeline/document_source_plan_cache_stats.cpp
new file mode 100644
index 00000000000..9bdc11ad80c
--- /dev/null
+++ b/src/mongo/db/pipeline/document_source_plan_cache_stats.cpp
@@ -0,0 +1,112 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#include "mongo/platform/basic.h"
+
+#include "mongo/db/pipeline/document_source_plan_cache_stats.h"
+
+namespace mongo {
+
+const char* DocumentSourcePlanCacheStats::kStageName = "$planCacheStats";
+
+REGISTER_DOCUMENT_SOURCE(planCacheStats,
+ DocumentSourcePlanCacheStats::LiteParsed::parse,
+ DocumentSourcePlanCacheStats::createFromBson);
+
+boost::intrusive_ptr<DocumentSource> DocumentSourcePlanCacheStats::createFromBson(
+ BSONElement spec, const boost::intrusive_ptr<ExpressionContext>& pExpCtx) {
+ uassert(
+ ErrorCodes::FailedToParse,
+ str::stream() << kStageName << " value must be an object. Found: " << typeName(spec.type()),
+ spec.type() == BSONType::Object);
+
+ uassert(ErrorCodes::FailedToParse,
+ str::stream() << kStageName << " parameters object must be empty. Found: "
+ << typeName(spec.type()),
+ spec.embeddedObject().isEmpty());
+
+ uassert(50932,
+ str::stream() << kStageName << " cannot be executed against a MongoS.",
+ !pExpCtx->inMongos && !pExpCtx->fromMongos && !pExpCtx->needsMerge);
+
+ return new DocumentSourcePlanCacheStats(pExpCtx);
+}
+
+DocumentSourcePlanCacheStats::DocumentSourcePlanCacheStats(
+ const boost::intrusive_ptr<ExpressionContext>& expCtx)
+ : DocumentSource(expCtx) {}
+
+void DocumentSourcePlanCacheStats::serializeToArray(
+ std::vector<Value>& array, boost::optional<ExplainOptions::Verbosity> explain) const {
+ if (explain) {
+ array.push_back(Value{
+ Document{{kStageName,
+ Document{{"match"_sd,
+ _absorbedMatch ? Value{_absorbedMatch->getQuery()} : Value{}}}}}});
+ } else {
+ array.push_back(Value{Document{{kStageName, Document{}}}});
+ if (_absorbedMatch) {
+ _absorbedMatch->serializeToArray(array);
+ }
+ }
+}
+
+Pipeline::SourceContainer::iterator DocumentSourcePlanCacheStats::doOptimizeAt(
+ Pipeline::SourceContainer::iterator itr, Pipeline::SourceContainer* container) {
+ auto itrToNext = std::next(itr);
+ if (itrToNext == container->end()) {
+ return itrToNext;
+ }
+
+ auto subsequentMatch = dynamic_cast<DocumentSourceMatch*>(itrToNext->get());
+ if (!subsequentMatch) {
+ return itrToNext;
+ }
+
+ _absorbedMatch = subsequentMatch;
+ return container->erase(itrToNext);
+}
+
+DocumentSource::GetNextResult DocumentSourcePlanCacheStats::getNext() {
+ if (!_haveRetrievedStats) {
+ const auto matchExpr = _absorbedMatch ? _absorbedMatch->getMatchExpression() : nullptr;
+ _results = pExpCtx->mongoProcessInterface->getMatchingPlanCacheEntryStats(
+ pExpCtx->opCtx, pExpCtx->ns, matchExpr);
+
+ _resultsIter = _results.begin();
+ _haveRetrievedStats = true;
+ }
+
+ if (_resultsIter == _results.end()) {
+ return GetNextResult::makeEOF();
+ }
+
+ return Document{*_resultsIter++};
+}
+
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/document_source_plan_cache_stats.h b/src/mongo/db/pipeline/document_source_plan_cache_stats.h
new file mode 100644
index 00000000000..f6d9f8dce0e
--- /dev/null
+++ b/src/mongo/db/pipeline/document_source_plan_cache_stats.h
@@ -0,0 +1,144 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#pragma once
+
+#include "mongo/db/pipeline/document_source.h"
+#include "mongo/db/pipeline/document_source_match.h"
+
+namespace mongo {
+
+class DocumentSourcePlanCacheStats final : public DocumentSource {
+public:
+ static const char* kStageName;
+
+ class LiteParsed final : public LiteParsedDocumentSource {
+ public:
+ static std::unique_ptr<LiteParsed> parse(const AggregationRequest& request,
+ const BSONElement& spec) {
+ return stdx::make_unique<LiteParsed>(request.getNamespaceString());
+ }
+
+ explicit LiteParsed(NamespaceString nss) : _nss(std::move(nss)) {}
+
+ stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const override {
+ // There are no foreign collections.
+ return stdx::unordered_set<NamespaceString>();
+ }
+
+ PrivilegeVector requiredPrivileges(bool isMongos) const override {
+ return {Privilege(ResourcePattern::forExactNamespace(_nss), ActionType::planCacheRead)};
+ }
+
+ bool isInitialSource() const final {
+ return true;
+ }
+
+ bool allowedToForwardFromMongos() const override {
+ // $planCacheStats must be run locally on a mongod.
+ return false;
+ }
+
+ bool allowedToPassthroughFromMongos() const override {
+ // $planCacheStats must be run locally on a mongod.
+ return false;
+ }
+
+ void assertSupportsReadConcern(const repl::ReadConcernArgs& readConcern) const {
+ uassert(ErrorCodes::InvalidOptions,
+ str::stream() << "Aggregation stage " << kStageName
+ << " requires read concern local but found "
+ << readConcern.toString(),
+ readConcern.getLevel() == repl::ReadConcernLevel::kLocalReadConcern);
+ }
+
+ private:
+ const NamespaceString _nss;
+ };
+
+ static boost::intrusive_ptr<DocumentSource> createFromBson(
+ BSONElement elem, const boost::intrusive_ptr<ExpressionContext>& pExpCtx);
+
+ virtual ~DocumentSourcePlanCacheStats() = default;
+
+ GetNextResult getNext() override;
+
+ StageConstraints constraints(
+ Pipeline::SplitState = Pipeline::SplitState::kUnsplit) const override {
+ StageConstraints constraints{StreamType::kStreaming,
+ PositionRequirement::kFirst,
+ // This stage must run on a mongod, and will fail at parse time
+ // if an attempt is made to run it on mongos.
+ HostTypeRequirement::kAnyShard,
+ DiskUseRequirement::kNoDiskUse,
+ FacetRequirement::kNotAllowed,
+ TransactionRequirement::kNotAllowed};
+
+ constraints.requiresInputDocSource = false;
+ return constraints;
+ }
+
+ const char* getSourceName() const override {
+ return kStageName;
+ }
+
+ /**
+ * Absorbs a subsequent $match, in order to avoid copying the entire contents of the plan cache
+ * prior to filtering.
+ */
+ Pipeline::SourceContainer::iterator doOptimizeAt(Pipeline::SourceContainer::iterator itr,
+ Pipeline::SourceContainer* container) override;
+
+ void serializeToArray(
+ std::vector<Value>& array,
+ boost::optional<ExplainOptions::Verbosity> explain = boost::none) const override;
+
+private:
+ DocumentSourcePlanCacheStats(const boost::intrusive_ptr<ExpressionContext>& expCtx);
+
+ Value serialize(
+ boost::optional<ExplainOptions::Verbosity> explain = boost::none) const override {
+ MONGO_UNREACHABLE; // Should call serializeToArray instead.
+ }
+
+ // The result set for this change is produced through the mongo process interface on the first
+ // call to getNext(), and then held by this data member.
+ std::vector<BSONObj> _results;
+
+ // Whether '_results' has been populated yet.
+ bool _haveRetrievedStats = false;
+
+ // Used to spool out '_results' as calls to getNext() are made.
+ std::vector<BSONObj>::iterator _resultsIter;
+
+ // $planCacheStats can push a match down into the plan cache layer, in order to avoid copying
+ // the entire contents of the cache.
+ boost::intrusive_ptr<DocumentSourceMatch> _absorbedMatch;
+};
+
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/document_source_plan_cache_stats_test.cpp b/src/mongo/db/pipeline/document_source_plan_cache_stats_test.cpp
new file mode 100644
index 00000000000..f89028b7add
--- /dev/null
+++ b/src/mongo/db/pipeline/document_source_plan_cache_stats_test.cpp
@@ -0,0 +1,178 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#include "mongo/platform/basic.h"
+
+#include <algorithm>
+
+#include "mongo/bson/json.h"
+#include "mongo/db/pipeline/aggregation_context_fixture.h"
+#include "mongo/db/pipeline/document_source_plan_cache_stats.h"
+#include "mongo/db/pipeline/stub_mongo_process_interface.h"
+#include "mongo/unittest/unittest.h"
+
+namespace mongo {
+
+using DocumentSourcePlanCacheStatsTest = AggregationContextFixture;
+
+/**
+ * A MongoProcessInterface used for testing which returns artificial plan cache stats.
+ */
+class PlanCacheStatsMongoProcessInterface final : public StubMongoProcessInterface {
+public:
+ PlanCacheStatsMongoProcessInterface(std::vector<BSONObj> planCacheStats)
+ : _planCacheStats(std::move(planCacheStats)) {}
+
+ std::vector<BSONObj> getMatchingPlanCacheEntryStats(
+ OperationContext* opCtx,
+ const NamespaceString& nss,
+ const MatchExpression* matchExpr) const override {
+ std::vector<BSONObj> filteredStats{};
+ std::copy_if(_planCacheStats.begin(),
+ _planCacheStats.end(),
+ std::back_inserter(filteredStats),
+ [&matchExpr](const BSONObj& obj) { return matchExpr->matchesBSON(obj); });
+ return filteredStats;
+ }
+
+private:
+ std::vector<BSONObj> _planCacheStats;
+};
+
+TEST_F(DocumentSourcePlanCacheStatsTest, ShouldFailToParseIfSpecIsNotObject) {
+ const auto specObj = fromjson("{$planCacheStats: 1}");
+ ASSERT_THROWS_CODE(
+ DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx()),
+ AssertionException,
+ ErrorCodes::FailedToParse);
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, ShouldFailToParseIfSpecIsANonEmptyObject) {
+ const auto specObj = fromjson("{$planCacheStats: {unknownOption: 1}}");
+ ASSERT_THROWS_CODE(
+ DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx()),
+ AssertionException,
+ ErrorCodes::FailedToParse);
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, CannotCreateWhenInMongos) {
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ getExpCtx()->inMongos = true;
+ ASSERT_THROWS_CODE(
+ DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx()),
+ AssertionException,
+ 50932);
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, CanParseAndSerializeSuccessfully) {
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ auto stage = DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx());
+ std::vector<Value> serialized;
+ stage->serializeToArray(serialized);
+ ASSERT_EQ(1u, serialized.size());
+ ASSERT_BSONOBJ_EQ(specObj, serialized[0].getDocument().toBson());
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, CanParseAndSerializeAsExplainSuccessfully) {
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ auto stage = DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx());
+ std::vector<Value> serialized;
+ stage->serializeToArray(serialized, ExplainOptions::Verbosity::kQueryPlanner);
+ ASSERT_EQ(1u, serialized.size());
+ ASSERT_BSONOBJ_EQ(specObj, serialized[0].getDocument().toBson());
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, SerializesSuccessfullyAfterAbsorbingMatch) {
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ auto planCacheStats =
+ DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx());
+ auto match = DocumentSourceMatch::create(fromjson("{foo: 'bar'}"), getExpCtx());
+ auto pipeline = unittest::assertGet(Pipeline::create({planCacheStats, match}, getExpCtx()));
+ ASSERT_EQ(2u, pipeline->getSources().size());
+
+ pipeline->optimizePipeline();
+ ASSERT_EQ(1u, pipeline->getSources().size());
+
+ auto serialized = pipeline->serialize();
+ ASSERT_EQ(2u, serialized.size());
+ ASSERT_BSONOBJ_EQ(specObj, serialized[0].getDocument().toBson());
+ ASSERT_BSONOBJ_EQ(fromjson("{$match: {foo: 'bar'}}"), serialized[1].getDocument().toBson());
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, SerializesSuccessfullyAfterAbsorbingMatchForExplain) {
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ auto planCacheStats =
+ DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx());
+ auto match = DocumentSourceMatch::create(fromjson("{foo: 'bar'}"), getExpCtx());
+ auto pipeline = unittest::assertGet(Pipeline::create({planCacheStats, match}, getExpCtx()));
+ ASSERT_EQ(2u, pipeline->getSources().size());
+
+ pipeline->optimizePipeline();
+ ASSERT_EQ(1u, pipeline->getSources().size());
+
+ auto serialized = pipeline->writeExplainOps(ExplainOptions::Verbosity::kQueryPlanner);
+ ASSERT_EQ(1u, serialized.size());
+ ASSERT_BSONOBJ_EQ(fromjson("{$planCacheStats: {match: {foo: 'bar'}}}"),
+ serialized[0].getDocument().toBson());
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, ReturnsImmediateEOFWithEmptyPlanCache) {
+ getExpCtx()->mongoProcessInterface =
+ std::make_shared<PlanCacheStatsMongoProcessInterface>(std::vector<BSONObj>{});
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ auto stage = DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx());
+ ASSERT(stage->getNext().isEOF());
+ ASSERT(stage->getNext().isEOF());
+}
+
+TEST_F(DocumentSourcePlanCacheStatsTest, ReturnsOnlyMatchingStatsAfterAbsorbingMatch) {
+ std::vector<BSONObj> stats{BSONObj(),
+ BSON("foo"
+ << "bar"),
+ BSON("foo"
+ << "baz"),
+ BSON("foo"
+ << "bar"
+ << "match"
+ << true)};
+ getExpCtx()->mongoProcessInterface =
+ std::make_shared<PlanCacheStatsMongoProcessInterface>(stats);
+
+ const auto specObj = fromjson("{$planCacheStats: {}}");
+ auto planCacheStats =
+ DocumentSourcePlanCacheStats::createFromBson(specObj.firstElement(), getExpCtx());
+ auto match = DocumentSourceMatch::create(fromjson("{foo: 'bar'}"), getExpCtx());
+ auto pipeline = unittest::assertGet(Pipeline::create({planCacheStats, match}, getExpCtx()));
+ pipeline->optimizePipeline();
+
+ ASSERT_BSONOBJ_EQ(pipeline->getNext()->toBson(), stats[1]);
+ ASSERT_BSONOBJ_EQ(pipeline->getNext()->toBson(), stats[3]);
+ ASSERT(!pipeline->getNext());
+}
+
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/mongo_process_interface.h b/src/mongo/db/pipeline/mongo_process_interface.h
index 13efd2928f3..3d5ce9a27f9 100644
--- a/src/mongo/db/pipeline/mongo_process_interface.h
+++ b/src/mongo/db/pipeline/mongo_process_interface.h
@@ -38,6 +38,7 @@
#include "mongo/client/dbclient_base.h"
#include "mongo/db/collection_index_usage_tracker.h"
#include "mongo/db/generic_cursor.h"
+#include "mongo/db/matcher/expression.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/pipeline/document.h"
#include "mongo/db/pipeline/field_path.h"
@@ -235,6 +236,15 @@ public:
virtual BackupCursorState openBackupCursor(OperationContext* opCtx) = 0;
virtual void closeBackupCursor(OperationContext* opCtx, std::uint64_t cursorId) = 0;
+
+ /**
+ * Returns a vector of BSON objects, where each entry in the vector describes a plan cache entry
+ * inside the cache for the given namespace. Only those entries which match the supplied
+ * MatchExpression are returned.
+ */
+ virtual std::vector<BSONObj> getMatchingPlanCacheEntryStats(OperationContext*,
+ const NamespaceString&,
+ const MatchExpression*) const = 0;
};
} // namespace mongo
diff --git a/src/mongo/db/pipeline/mongod_process_interface.cpp b/src/mongo/db/pipeline/mongod_process_interface.cpp
index 7f313a96a00..5db1e50083e 100644
--- a/src/mongo/db/pipeline/mongod_process_interface.cpp
+++ b/src/mongo/db/pipeline/mongod_process_interface.cpp
@@ -418,6 +418,31 @@ void MongoDInterface::closeBackupCursor(OperationContext* opCtx, std::uint64_t c
backupCursorService->closeBackupCursor(opCtx, cursorId);
}
+std::vector<BSONObj> MongoDInterface::getMatchingPlanCacheEntryStats(
+ OperationContext* opCtx, const NamespaceString& nss, const MatchExpression* matchExp) const {
+ const auto serializer = [](const PlanCacheEntry& entry) {
+ BSONObjBuilder out;
+ Explain::planCacheEntryToBSON(entry, &out);
+ return out.obj();
+ };
+
+ const auto predicate = [&matchExp](const BSONObj& obj) {
+ return !matchExp ? true : matchExp->matchesBSON(obj);
+ };
+
+ AutoGetCollection autoColl(opCtx, nss, MODE_IS);
+ const auto collection = autoColl.getCollection();
+ uassert(
+ 50933, str::stream() << "collection '" << nss.toString() << "' does not exist", collection);
+
+ const auto infoCache = collection->infoCache();
+ invariant(infoCache);
+ const auto planCache = infoCache->getPlanCache();
+ invariant(planCache);
+
+ return planCache->getMatchingStats(serializer, predicate);
+}
+
BSONObj MongoDInterface::_reportCurrentOpForClient(OperationContext* opCtx,
Client* client,
CurrentOpTruncateMode truncateOps) const {
diff --git a/src/mongo/db/pipeline/mongod_process_interface.h b/src/mongo/db/pipeline/mongod_process_interface.h
index 945ffb5ca91..e64d57761fc 100644
--- a/src/mongo/db/pipeline/mongod_process_interface.h
+++ b/src/mongo/db/pipeline/mongod_process_interface.h
@@ -98,6 +98,10 @@ public:
BackupCursorState openBackupCursor(OperationContext* opCtx) final;
void closeBackupCursor(OperationContext* opCtx, std::uint64_t cursorId) final;
+ std::vector<BSONObj> getMatchingPlanCacheEntryStats(OperationContext*,
+ const NamespaceString&,
+ const MatchExpression*) const final;
+
protected:
BSONObj _reportCurrentOpForClient(OperationContext* opCtx,
Client* client,
diff --git a/src/mongo/db/pipeline/mongos_process_interface.h b/src/mongo/db/pipeline/mongos_process_interface.h
index 41af61d9b1f..214d35a1c5d 100644
--- a/src/mongo/db/pipeline/mongos_process_interface.h
+++ b/src/mongo/db/pipeline/mongos_process_interface.h
@@ -154,6 +154,17 @@ public:
MONGO_UNREACHABLE;
}
+ /**
+ * Mongos does not have a plan cache, so this method should never be called on mongos. Upstream
+ * checks are responsible for generating an error if a user attempts to introspect the plan
+ * cache on mongos.
+ */
+ std::vector<BSONObj> getMatchingPlanCacheEntryStats(OperationContext*,
+ const NamespaceString&,
+ const MatchExpression*) const final {
+ MONGO_UNREACHABLE;
+ }
+
protected:
BSONObj _reportCurrentOpForClient(OperationContext* opCtx,
Client* client,
diff --git a/src/mongo/db/pipeline/stub_mongo_process_interface.h b/src/mongo/db/pipeline/stub_mongo_process_interface.h
index 7e3d05a9fce..c0c68e85f21 100644
--- a/src/mongo/db/pipeline/stub_mongo_process_interface.h
+++ b/src/mongo/db/pipeline/stub_mongo_process_interface.h
@@ -168,5 +168,11 @@ public:
void closeBackupCursor(OperationContext* opCtx, std::uint64_t cursorId) final {
MONGO_UNREACHABLE;
}
+
+ std::vector<BSONObj> getMatchingPlanCacheEntryStats(OperationContext*,
+ const NamespaceString&,
+ const MatchExpression*) const override {
+ MONGO_UNREACHABLE;
+ }
};
} // namespace mongo