summaryrefslogtreecommitdiff
path: root/src/mongo/s
diff options
context:
space:
mode:
Diffstat (limited to 'src/mongo/s')
-rw-r--r--src/mongo/s/client/shard.h4
-rw-r--r--src/mongo/s/client/shard_remote.cpp11
-rw-r--r--src/mongo/s/client/shard_remote.h2
-rw-r--r--src/mongo/s/commands/cluster_aggregate_test.cpp18
-rw-r--r--src/mongo/s/commands/cluster_count_cmd.cpp10
-rw-r--r--src/mongo/s/commands/cluster_current_op.cpp6
-rw-r--r--src/mongo/s/commands/cluster_distinct_cmd.cpp9
-rw-r--r--src/mongo/s/commands/cluster_find_cmd.cpp12
-rw-r--r--src/mongo/s/commands/cluster_map_reduce_agg.cpp18
-rw-r--r--src/mongo/s/commands/cluster_pipeline_cmd.cpp19
-rw-r--r--src/mongo/s/query/cluster_aggregate.cpp30
-rw-r--r--src/mongo/s/query/cluster_aggregate.h9
-rw-r--r--src/mongo/s/query/cluster_aggregation_planner.cpp4
13 files changed, 85 insertions, 67 deletions
diff --git a/src/mongo/s/client/shard.h b/src/mongo/s/client/shard.h
index 3ef339cdbd7..d4ea6cedfc4 100644
--- a/src/mongo/s/client/shard.h
+++ b/src/mongo/s/client/shard.h
@@ -36,7 +36,7 @@
#include "mongo/client/read_preference.h"
#include "mongo/db/logical_time.h"
#include "mongo/db/namespace_string.h"
-#include "mongo/db/pipeline/aggregation_request.h"
+#include "mongo/db/pipeline/aggregation_request_helper.h"
#include "mongo/db/repl/optime.h"
#include "mongo/db/repl/read_concern_args.h"
#include "mongo/executor/remote_command_response.h"
@@ -217,7 +217,7 @@ public:
*/
virtual Status runAggregation(
OperationContext* opCtx,
- const AggregationRequest& aggRequest,
+ const AggregateCommand& aggRequest,
std::function<bool(const std::vector<BSONObj>& batch)> callback) = 0;
/**
diff --git a/src/mongo/s/client/shard_remote.cpp b/src/mongo/s/client/shard_remote.cpp
index 177aa227074..eaea181f039 100644
--- a/src/mongo/s/client/shard_remote.cpp
+++ b/src/mongo/s/client/shard_remote.cpp
@@ -418,14 +418,15 @@ void ShardRemote::runFireAndForgetCommand(OperationContext* opCtx,
Status ShardRemote::runAggregation(
OperationContext* opCtx,
- const AggregationRequest& aggRequest,
+ const AggregateCommand& aggRequest,
std::function<bool(const std::vector<BSONObj>& batch)> callback) {
BSONObj readPrefMetadata;
ReadPreferenceSetting readPreference =
uassertStatusOK(ReadPreferenceSetting::fromContainingBSON(
- aggRequest.getUnwrappedReadPref(), ReadPreference::SecondaryPreferred));
+ aggRequest.getUnwrappedReadPref().value_or(BSONObj()),
+ ReadPreference::SecondaryPreferred));
auto swHost = _targeter->findHost(opCtx, readPreference);
if (!swHost.isOK()) {
@@ -480,14 +481,14 @@ Status ShardRemote::runAggregation(
Milliseconds requestTimeout(-1);
if (aggRequest.getMaxTimeMS()) {
- requestTimeout = Milliseconds(aggRequest.getMaxTimeMS());
+ requestTimeout = Milliseconds(aggRequest.getMaxTimeMS().value_or(0));
}
auto executor = Grid::get(opCtx)->getExecutorPool()->getFixedExecutor();
Fetcher fetcher(executor.get(),
host,
- aggRequest.getNamespaceString().db().toString(),
- aggRequest.serializeToCommandObj().toBson(),
+ aggRequest.getNamespace().db().toString(),
+ aggregation_request_helper::serializeToCommandObj(aggRequest),
fetcherCallback,
readPrefMetadata,
requestTimeout, /* command network timeout */
diff --git a/src/mongo/s/client/shard_remote.h b/src/mongo/s/client/shard_remote.h
index e7ca9004112..da84f2b633b 100644
--- a/src/mongo/s/client/shard_remote.h
+++ b/src/mongo/s/client/shard_remote.h
@@ -86,7 +86,7 @@ public:
const BSONObj& cmdObj) final;
Status runAggregation(OperationContext* opCtx,
- const AggregationRequest& aggRequest,
+ const AggregateCommand& aggRequest,
std::function<bool(const std::vector<BSONObj>& batch)> callback);
private:
diff --git a/src/mongo/s/commands/cluster_aggregate_test.cpp b/src/mongo/s/commands/cluster_aggregate_test.cpp
index 08bcc8081e9..17700666352 100644
--- a/src/mongo/s/commands/cluster_aggregate_test.cpp
+++ b/src/mongo/s/commands/cluster_aggregate_test.cpp
@@ -95,7 +95,7 @@ protected:
NamespaceString nss{"a.collection"};
auto client = getServiceContext()->makeClient("ClusterCmdClient");
auto opCtx = client->makeOperationContext();
- auto request = AggregationRequest::parseFromBSON(nss, inputBson);
+ auto request = aggregation_request_helper::parseFromBSON(nss, inputBson);
if (request.getStatus() != Status::OK()) {
return request.getStatus();
}
@@ -130,25 +130,29 @@ TEST_F(ClusterAggregateTest, SnapshotReadConcernWithAfterClusterTime) {
}
TEST_F(ClusterAggregateTest, ShouldFailWhenFromMongosIsTrue) {
- const BSONObj inputBson = fromjson("{pipeline: [], cursor: {}, fromMongos: true}");
+ const BSONObj inputBson =
+ fromjson("{aggregate: 'coll', pipeline: [], cursor: {}, fromMongos: true, $db: 'test'}");
ASSERT_THROWS_CODE(testRunAggregateEarlyExit(inputBson), AssertionException, 51089);
}
TEST_F(ClusterAggregateTest, ShouldFailWhenNeedsMergeIstrueAndFromMongosIsFalse) {
- const BSONObj inputBson =
- fromjson("{pipeline: [], cursor: {}, needsMerge: true, fromMongos: false}");
+ const BSONObj inputBson = fromjson(
+ "{aggregate: 'coll', pipeline: [], cursor: {}, needsMerge: true, fromMongos: false, $db: "
+ "'test'}");
ASSERT_THROWS_CODE(testRunAggregateEarlyExit(inputBson), AssertionException, 51089);
}
TEST_F(ClusterAggregateTest, ShouldFailWhenNeedsMergeIstrueAndFromMongosIsTrue) {
- const BSONObj inputBson =
- fromjson("{pipeline: [], cursor: {}, needsMerge: true, fromMongos: true}");
+ const BSONObj inputBson = fromjson(
+ "{aggregate: 'coll', pipeline: [], cursor: {}, needsMerge: true, fromMongos: true, $db: "
+ "'test'}");
ASSERT_THROWS_CODE(testRunAggregateEarlyExit(inputBson), AssertionException, 51089);
}
TEST_F(ClusterAggregateTest, ShouldFailWhenExchengeIsPresent) {
const BSONObj inputBson = fromjson(
- "{pipeline: [], cursor: {}, exchange: {policy: 'roundrobin', consumers: NumberInt(2)}}");
+ "{aggregate: 'coll', pipeline: [], cursor: {}, exchange: {policy: 'roundrobin', consumers: "
+ "NumberInt(2)}, $db: 'test'}");
ASSERT_THROWS_CODE(testRunAggregateEarlyExit(inputBson), AssertionException, 51028);
}
diff --git a/src/mongo/s/commands/cluster_count_cmd.cpp b/src/mongo/s/commands/cluster_count_cmd.cpp
index 48eb683ad7b..3ade68c8bd7 100644
--- a/src/mongo/s/commands/cluster_count_cmd.cpp
+++ b/src/mongo/s/commands/cluster_count_cmd.cpp
@@ -128,11 +128,13 @@ public:
auto countRequest = CountCommand::parse(IDLParserErrorContext("count"), cmdObj);
auto aggCmdOnView =
uassertStatusOK(countCommandAsAggregationCommand(countRequest, nss));
+ auto aggCmdOnViewObj = OpMsgRequest::fromDBAndBody(nss.db(), aggCmdOnView).body;
auto aggRequestOnView =
- uassertStatusOK(AggregationRequest::parseFromBSON(nss, aggCmdOnView));
+ uassertStatusOK(aggregation_request_helper::parseFromBSON(nss, aggCmdOnViewObj));
auto resolvedAggRequest = ex->asExpandedViewAggregation(aggRequestOnView);
- auto resolvedAggCmd = resolvedAggRequest.serializeToCommandObj().toBson();
+ auto resolvedAggCmd =
+ aggregation_request_helper::serializeToCommandObj(resolvedAggRequest);
BSONObj aggResult = CommandHelpers::runCommandDirectly(
opCtx, OpMsgRequest::fromDBAndBody(dbname, std::move(resolvedAggCmd)));
@@ -236,8 +238,10 @@ public:
return aggCmdOnView.getStatus();
}
+ auto aggCmdOnViewObj =
+ OpMsgRequest::fromDBAndBody(nss.db(), aggCmdOnView.getValue()).body;
auto aggRequestOnView =
- AggregationRequest::parseFromBSON(nss, aggCmdOnView.getValue(), verbosity);
+ aggregation_request_helper::parseFromBSON(nss, aggCmdOnViewObj, verbosity);
if (!aggRequestOnView.isOK()) {
return aggRequestOnView.getStatus();
}
diff --git a/src/mongo/s/commands/cluster_current_op.cpp b/src/mongo/s/commands/cluster_current_op.cpp
index 772f4c8ea0f..71849c56a9f 100644
--- a/src/mongo/s/commands/cluster_current_op.cpp
+++ b/src/mongo/s/commands/cluster_current_op.cpp
@@ -70,9 +70,9 @@ private:
pipeline->push_back(sortBuilder.obj());
}
- virtual StatusWith<CursorResponse> runAggregation(
- OperationContext* opCtx, const AggregationRequest& request) const final {
- auto nss = request.getNamespaceString();
+ virtual StatusWith<CursorResponse> runAggregation(OperationContext* opCtx,
+ const AggregateCommand& request) const final {
+ auto nss = request.getNamespace();
BSONObjBuilder responseBuilder;
diff --git a/src/mongo/s/commands/cluster_distinct_cmd.cpp b/src/mongo/s/commands/cluster_distinct_cmd.cpp
index 41475a8cec2..e6167129a67 100644
--- a/src/mongo/s/commands/cluster_distinct_cmd.cpp
+++ b/src/mongo/s/commands/cluster_distinct_cmd.cpp
@@ -132,8 +132,9 @@ public:
return aggCmdOnView.getStatus();
}
+ auto viewAggCmd = OpMsgRequest::fromDBAndBody(nss.db(), aggCmdOnView.getValue()).body;
auto aggRequestOnView =
- AggregationRequest::parseFromBSON(nss, aggCmdOnView.getValue(), verbosity);
+ aggregation_request_helper::parseFromBSON(nss, viewAggCmd, verbosity);
if (!aggRequestOnView.isOK()) {
return aggRequestOnView.getStatus();
}
@@ -206,11 +207,13 @@ public:
auto aggCmdOnView = parsedDistinct.getValue().asAggregationCommand();
uassertStatusOK(aggCmdOnView.getStatus());
- auto aggRequestOnView = AggregationRequest::parseFromBSON(nss, aggCmdOnView.getValue());
+ auto viewAggCmd = OpMsgRequest::fromDBAndBody(nss.db(), aggCmdOnView.getValue()).body;
+ auto aggRequestOnView = aggregation_request_helper::parseFromBSON(nss, viewAggCmd);
uassertStatusOK(aggRequestOnView.getStatus());
auto resolvedAggRequest = ex->asExpandedViewAggregation(aggRequestOnView.getValue());
- auto resolvedAggCmd = resolvedAggRequest.serializeToCommandObj().toBson();
+ auto resolvedAggCmd =
+ aggregation_request_helper::serializeToCommandObj(resolvedAggRequest);
if (auto txnRouter = TransactionRouter::get(opCtx)) {
txnRouter.onViewResolutionError(opCtx, nss);
diff --git a/src/mongo/s/commands/cluster_find_cmd.cpp b/src/mongo/s/commands/cluster_find_cmd.cpp
index 6607bd99144..05ed5c43e7a 100644
--- a/src/mongo/s/commands/cluster_find_cmd.cpp
+++ b/src/mongo/s/commands/cluster_find_cmd.cpp
@@ -192,9 +192,11 @@ public:
bodyBuilder.resetToEmpty();
auto aggCmdOnView = uassertStatusOK(qr->asAggregationCommand());
+ auto viewAggregationCommand =
+ OpMsgRequest::fromDBAndBody(_dbName, aggCmdOnView).body;
- auto aggRequestOnView = uassertStatusOK(
- AggregationRequest::parseFromBSON(ns(), aggCmdOnView, verbosity));
+ auto aggRequestOnView = uassertStatusOK(aggregation_request_helper::parseFromBSON(
+ ns(), viewAggregationCommand, verbosity));
// An empty PrivilegeVector is acceptable because these privileges are only checked
// on getMore and explain will not open a cursor.
@@ -253,9 +255,11 @@ public:
result->reset();
auto aggCmdOnView = uassertStatusOK(cq->getQueryRequest().asAggregationCommand());
+ auto viewAggregationCommand =
+ OpMsgRequest::fromDBAndBody(_dbName, aggCmdOnView).body;
- auto aggRequestOnView =
- uassertStatusOK(AggregationRequest::parseFromBSON(ns(), aggCmdOnView));
+ auto aggRequestOnView = uassertStatusOK(
+ aggregation_request_helper::parseFromBSON(ns(), viewAggregationCommand));
auto bodyBuilder = result->getBodyBuilder();
uassertStatusOK(ClusterAggregate::retryOnViewError(
diff --git a/src/mongo/s/commands/cluster_map_reduce_agg.cpp b/src/mongo/s/commands/cluster_map_reduce_agg.cpp
index 63cd6dfbd39..2af202ec39b 100644
--- a/src/mongo/s/commands/cluster_map_reduce_agg.cpp
+++ b/src/mongo/s/commands/cluster_map_reduce_agg.cpp
@@ -113,22 +113,22 @@ Document serializeToCommand(BSONObj originalCmd, const MapReduce& parsedMr, Pipe
MutableDocument translatedCmd;
translatedCmd["aggregate"] = Value(parsedMr.getNamespace().coll());
- translatedCmd[AggregationRequest::kPipelineName] = Value(pipeline->serialize());
- translatedCmd[AggregationRequest::kCursorName] =
+ translatedCmd[AggregateCommand::kPipelineFieldName] = Value(pipeline->serialize());
+ translatedCmd[AggregateCommand::kBatchSizeFieldName] =
Value(Document{{"batchSize", std::numeric_limits<long long>::max()}});
- translatedCmd[AggregationRequest::kAllowDiskUseName] = Value(true);
- translatedCmd[AggregationRequest::kFromMongosName] = Value(true);
- translatedCmd[AggregationRequest::kLetName] = Value(
+ translatedCmd[AggregateCommand::kAllowDiskUseFieldName] = Value(true);
+ translatedCmd[AggregateCommand::kFromMongosFieldName] = Value(true);
+ translatedCmd[AggregateCommand::kLetFieldName] = Value(
pipeline->getContext()->variablesParseState.serialize(pipeline->getContext()->variables));
- translatedCmd[AggregationRequest::kIsMapReduceCommandName] = Value(true);
+ translatedCmd[AggregateCommand::kIsMapReduceCommandFieldName] = Value(true);
if (shouldBypassDocumentValidationForCommand(originalCmd)) {
translatedCmd[bypassDocumentValidationCommandOption()] = Value(true);
}
- if (originalCmd[AggregationRequest::kCollationName]) {
- translatedCmd[AggregationRequest::kCollationName] =
- Value(originalCmd[AggregationRequest::kCollationName]);
+ if (originalCmd[AggregateCommand::kCollationFieldName]) {
+ translatedCmd[AggregateCommand::kCollationFieldName] =
+ Value(originalCmd[AggregateCommand::kCollationFieldName]);
}
// Append generic command options.
diff --git a/src/mongo/s/commands/cluster_pipeline_cmd.cpp b/src/mongo/s/commands/cluster_pipeline_cmd.cpp
index cbeb9022c43..631a5dabcc0 100644
--- a/src/mongo/s/commands/cluster_pipeline_cmd.cpp
+++ b/src/mongo/s/commands/cluster_pipeline_cmd.cpp
@@ -34,6 +34,7 @@
#include "mongo/base/status.h"
#include "mongo/db/auth/authorization_session.h"
#include "mongo/db/commands.h"
+#include "mongo/db/pipeline/aggregate_command_gen.h"
#include "mongo/db/pipeline/lite_parsed_pipeline.h"
#include "mongo/db/views/resolved_view.h"
#include "mongo/s/query/cluster_aggregate.h"
@@ -72,13 +73,13 @@ public:
OperationContext* opCtx,
const OpMsgRequest& opMsgRequest,
boost::optional<ExplainOptions::Verbosity> explainVerbosity) override {
- const auto aggregationRequest = uassertStatusOK(AggregationRequest::parseFromBSON(
+ const auto aggregationRequest = uassertStatusOK(aggregation_request_helper::parseFromBSON(
opMsgRequest.getDatabase().toString(), opMsgRequest.body, explainVerbosity));
- auto privileges = uassertStatusOK(
- AuthorizationSession::get(opCtx->getClient())
- ->getPrivilegesForAggregate(
- aggregationRequest.getNamespaceString(), aggregationRequest, true));
+ auto privileges =
+ uassertStatusOK(AuthorizationSession::get(opCtx->getClient())
+ ->getPrivilegesForAggregate(
+ aggregationRequest.getNamespace(), aggregationRequest, true));
return std::make_unique<Invocation>(
this, opMsgRequest, std::move(aggregationRequest), std::move(privileges));
@@ -88,7 +89,7 @@ public:
public:
Invocation(Command* cmd,
const OpMsgRequest& request,
- const AggregationRequest aggregationRequest,
+ const AggregateCommand aggregationRequest,
PrivilegeVector privileges)
: CommandInvocation(cmd),
_request(request),
@@ -113,7 +114,7 @@ public:
const std::string& dbname,
const BSONObj& cmdObj,
BSONObjBuilder* result) {
- const auto& nss = _aggregationRequest.getNamespaceString();
+ const auto& nss = _aggregationRequest.getNamespace();
try {
uassertStatusOK(
@@ -158,12 +159,12 @@ public:
}
NamespaceString ns() const override {
- return _aggregationRequest.getNamespaceString();
+ return _aggregationRequest.getNamespace();
}
const OpMsgRequest& _request;
const std::string _dbName;
- const AggregationRequest _aggregationRequest;
+ const AggregateCommand _aggregationRequest;
const LiteParsedPipeline _liteParsedPipeline;
const PrivilegeVector _privileges;
};
diff --git a/src/mongo/s/query/cluster_aggregate.cpp b/src/mongo/s/query/cluster_aggregate.cpp
index e266f40fb5e..4c80fb988d0 100644
--- a/src/mongo/s/query/cluster_aggregate.cpp
+++ b/src/mongo/s/query/cluster_aggregate.cpp
@@ -101,7 +101,7 @@ auto resolveInvolvedNamespaces(stdx::unordered_set<NamespaceString> involvedName
// collection UUID if provided.
boost::intrusive_ptr<ExpressionContext> makeExpressionContext(
OperationContext* opCtx,
- const AggregationRequest& request,
+ const AggregateCommand& request,
BSONObj collationObj,
boost::optional<UUID> uuid,
StringMap<ExpressionContext::ResolvedNamespace> resolvedNamespaces) {
@@ -191,7 +191,7 @@ void updateHostsTargetedMetrics(OperationContext* opCtx,
Status ClusterAggregate::runAggregate(OperationContext* opCtx,
const Namespaces& namespaces,
- const AggregationRequest& request,
+ const AggregateCommand& request,
const PrivilegeVector& privileges,
BSONObjBuilder* result) {
return runAggregate(opCtx, namespaces, request, {request}, privileges, result);
@@ -199,21 +199,21 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
Status ClusterAggregate::runAggregate(OperationContext* opCtx,
const Namespaces& namespaces,
- const AggregationRequest& request,
+ const AggregateCommand& request,
const LiteParsedPipeline& liteParsedPipeline,
const PrivilegeVector& privileges,
BSONObjBuilder* result) {
- uassert(51028, "Cannot specify exchange option to a mongos", !request.getExchangeSpec());
+ uassert(51028, "Cannot specify exchange option to a mongos", !request.getExchange());
uassert(51143,
"Cannot specify runtime constants option to a mongos",
!request.getLegacyRuntimeConstants());
uassert(51089,
- str::stream() << "Internal parameter(s) [" << AggregationRequest::kNeedsMergeName
- << ", " << AggregationRequest::kFromMongosName
+ str::stream() << "Internal parameter(s) [" << AggregateCommand::kNeedsMergeFieldName
+ << ", " << AggregateCommand::kFromMongosFieldName
<< "] cannot be set to 'true' when sent to mongos",
- !request.needsMerge() && !request.isFromMongos());
+ !request.getNeedsMerge() && !request.getFromMongos());
uassert(4928902,
- str::stream() << AggregationRequest::kCollectionUUIDName
+ str::stream() << AggregateCommand::kCollectionUUIDFieldName
<< " is not supported on a mongos",
!request.getCollectionUUID());
@@ -263,11 +263,11 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
// database, the standard logic would attempt to resolve its non-existent UUID and
// collation by sending a specious 'listCollections' command to the config servers.
if (hasChangeStream) {
- return {request.getCollation(), boost::none};
+ return {request.getCollation().value_or(BSONObj()), boost::none};
}
return cluster_aggregation_planner::getCollationAndUUID(
- opCtx, cm, namespaces.executionNss, request.getCollation());
+ opCtx, cm, namespaces.executionNss, request.getCollation().value_or(BSONObj()));
}();
// Build an ExpressionContext for the pipeline. This instantiates an appropriate collator,
@@ -298,7 +298,7 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
invariant(targeter.policy ==
cluster_aggregation_planner::AggregationTargeter::kPassthrough);
expCtx = make_intrusive<ExpressionContext>(
- opCtx, nullptr, namespaces.executionNss, boost::none, request.getLetParameters());
+ opCtx, nullptr, namespaces.executionNss, boost::none, request.getLet());
}
if (request.getExplain()) {
@@ -315,7 +315,7 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
namespaces,
*targeter.cm,
request.getExplain(),
- request.serializeToCommandObj(),
+ aggregation_request_helper::serializeToCommandDoc(request),
privileges,
result);
}
@@ -344,7 +344,7 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
return cluster_aggregation_planner::dispatchPipelineAndMerge(
opCtx,
std::move(targeter),
- request.serializeToCommandObj(),
+ aggregation_request_helper::serializeToCommandDoc(request),
request.getBatchSize(),
namespaces,
privileges,
@@ -365,14 +365,14 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
// Add 'command' object to explain output.
if (expCtx->explain) {
explain_common::appendIfRoom(
- request.serializeToCommandObj().toBson(), "command", result);
+ aggregation_request_helper::serializeToCommandObj(request), "command", result);
}
}
return status;
}
Status ClusterAggregate::retryOnViewError(OperationContext* opCtx,
- const AggregationRequest& request,
+ const AggregateCommand& request,
const ResolvedView& resolvedView,
const NamespaceString& requestedNss,
const PrivilegeVector& privileges,
diff --git a/src/mongo/s/query/cluster_aggregate.h b/src/mongo/s/query/cluster_aggregate.h
index 2c74736235a..b4239e47891 100644
--- a/src/mongo/s/query/cluster_aggregate.h
+++ b/src/mongo/s/query/cluster_aggregate.h
@@ -32,7 +32,8 @@
#include "mongo/base/status.h"
#include "mongo/bson/bsonobj.h"
#include "mongo/db/namespace_string.h"
-#include "mongo/db/pipeline/aggregation_request.h"
+#include "mongo/db/pipeline/aggregate_command_gen.h"
+#include "mongo/db/pipeline/aggregation_request_helper.h"
#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/lite_parsed_pipeline.h"
#include "mongo/s/query/cluster_client_cursor_params.h"
@@ -78,7 +79,7 @@ public:
*/
static Status runAggregate(OperationContext* opCtx,
const Namespaces& namespaces,
- const AggregationRequest& request,
+ const AggregateCommand& request,
const LiteParsedPipeline& liteParsedPipeline,
const PrivilegeVector& privileges,
BSONObjBuilder* result);
@@ -88,7 +89,7 @@ public:
*/
static Status runAggregate(OperationContext* opCtx,
const Namespaces& namespaces,
- const AggregationRequest& request,
+ const AggregateCommand& request,
const PrivilegeVector& privileges,
BSONObjBuilder* result);
@@ -102,7 +103,7 @@ public:
* On success, populates 'result' with the command response.
*/
static Status retryOnViewError(OperationContext* opCtx,
- const AggregationRequest& request,
+ const AggregateCommand& request,
const ResolvedView& resolvedView,
const NamespaceString& requestedNss,
const PrivilegeVector& privileges,
diff --git a/src/mongo/s/query/cluster_aggregation_planner.cpp b/src/mongo/s/query/cluster_aggregation_planner.cpp
index 75c8f142b68..272d13aa6c2 100644
--- a/src/mongo/s/query/cluster_aggregation_planner.cpp
+++ b/src/mongo/s/query/cluster_aggregation_planner.cpp
@@ -124,9 +124,9 @@ BSONObj createCommandForMergingShard(Document serializedCommand,
MutableDocument mergeCmd(serializedCommand);
mergeCmd["pipeline"] = Value(pipelineForMerging->serialize());
- mergeCmd[AggregationRequest::kFromMongosName] = Value(true);
+ mergeCmd[AggregateCommand::kFromMongosFieldName] = Value(true);
- mergeCmd[AggregationRequest::kLetName] =
+ mergeCmd[AggregateCommand::kLetFieldName] =
Value(mergeCtx->variablesParseState.serialize(mergeCtx->variables));
// If the user didn't specify a collation already, make sure there's a collation attached to