summaryrefslogtreecommitdiff
path: root/src/mongo/db
diff options
context:
space:
mode:
Diffstat (limited to 'src/mongo/db')
-rw-r--r--src/mongo/db/commands/pipeline_command.cpp63
-rw-r--r--src/mongo/db/pipeline/SConscript1
-rw-r--r--src/mongo/db/pipeline/document_source.h94
-rw-r--r--src/mongo/db/pipeline/document_source_add_fields.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_bucket.cpp11
-rw-r--r--src/mongo/db/pipeline/document_source_bucket_auto.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_coll_stats.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_count.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_facet.cpp117
-rw-r--r--src/mongo/db/pipeline/document_source_facet.h18
-rw-r--r--src/mongo/db/pipeline/document_source_geo_near.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_graph_lookup.cpp28
-rw-r--r--src/mongo/db/pipeline/document_source_group.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_index_stats.cpp6
-rw-r--r--src/mongo/db/pipeline/document_source_limit.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_lookup.cpp26
-rw-r--r--src/mongo/db/pipeline/document_source_match.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_merge_cursors.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_out.cpp18
-rw-r--r--src/mongo/db/pipeline/document_source_project.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_redact.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_replace_root.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_sample.cpp5
-rw-r--r--src/mongo/db/pipeline/document_source_skip.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_sort.cpp6
-rw-r--r--src/mongo/db/pipeline/document_source_sort_by_count.cpp8
-rw-r--r--src/mongo/db/pipeline/document_source_unwind.cpp8
-rw-r--r--src/mongo/db/pipeline/lite_parsed_document_source.cpp63
-rw-r--r--src/mongo/db/pipeline/lite_parsed_document_source.h128
-rw-r--r--src/mongo/db/pipeline/lite_parsed_pipeline.h85
-rw-r--r--src/mongo/db/pipeline/pipeline.h4
-rw-r--r--src/mongo/db/views/view_catalog.cpp4
32 files changed, 638 insertions, 138 deletions
diff --git a/src/mongo/db/commands/pipeline_command.cpp b/src/mongo/db/commands/pipeline_command.cpp
index 46cc4e8a40d..061c76b4883 100644
--- a/src/mongo/db/commands/pipeline_command.cpp
+++ b/src/mongo/db/commands/pipeline_command.cpp
@@ -51,6 +51,7 @@
#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_pipeline.h"
#include "mongo/db/pipeline/pipeline.h"
#include "mongo/db/pipeline/pipeline_d.h"
#include "mongo/db/query/collation/collator_factory_interface.h"
@@ -170,18 +171,17 @@ bool handleCursorCommand(OperationContext* txn,
}
StatusWith<StringMap<ExpressionContext::ResolvedNamespace>> resolveInvolvedNamespaces(
- OperationContext* txn,
- const boost::intrusive_ptr<Pipeline>& pipeline,
- const boost::intrusive_ptr<ExpressionContext>& expCtx) {
+ OperationContext* txn, const AggregationRequest& request) {
// We intentionally do not drop and reacquire our DB lock after resolving the view definition in
// order to prevent the definition for any view namespaces we've already resolved from changing.
// This is necessary to prevent a cycle from being formed among the view definitions cached in
// 'resolvedNamespaces' because we won't re-resolve a view namespace we've already encountered.
- AutoGetDb autoDb(txn, expCtx->ns.db(), MODE_IS);
+ AutoGetDb autoDb(txn, request.getNamespaceString().db(), MODE_IS);
Database* const db = autoDb.getDb();
ViewCatalog* viewCatalog = db ? db->getViewCatalog() : nullptr;
- const auto& pipelineInvolvedNamespaces = pipeline->getInvolvedCollections();
+ const LiteParsedPipeline liteParsedPipeline(request);
+ const auto& pipelineInvolvedNamespaces = liteParsedPipeline.getInvolvedNamespaces();
std::deque<NamespaceString> involvedNamespacesQueue(pipelineInvolvedNamespaces.begin(),
pipelineInvolvedNamespaces.end());
StringMap<ExpressionContext::ResolvedNamespace> resolvedNamespaces;
@@ -215,17 +215,11 @@ StatusWith<StringMap<ExpressionContext::ResolvedNamespace>> resolveInvolvedNames
// We parse the pipeline corresponding to the resolved view in case we must resolve
// other view namespaces that are also involved.
- auto resolvedViewPipeline =
- Pipeline::parse(resolvedView.getValue().getPipeline(), expCtx);
- if (!resolvedViewPipeline.isOK()) {
- return {ErrorCodes::FailedToParse,
- str::stream() << "Failed to parse definition for view '" << involvedNs.ns()
- << "': "
- << resolvedViewPipeline.getStatus().toString()};
- }
+ LiteParsedPipeline resolvedViewLitePipeline(
+ {resolvedView.getValue().getNamespace(), resolvedView.getValue().getPipeline()});
const auto& resolvedViewInvolvedNamespaces =
- resolvedViewPipeline.getValue()->getInvolvedCollections();
+ resolvedViewLitePipeline.getInvolvedNamespaces();
involvedNamespacesQueue.insert(involvedNamespacesQueue.end(),
resolvedViewInvolvedNamespaces.begin(),
resolvedViewInvolvedNamespaces.end());
@@ -361,14 +355,7 @@ public:
intrusive_ptr<ExpressionContext> expCtx = new ExpressionContext(txn, request);
expCtx->tempDir = storageGlobalParams.dbpath + "/_tmp";
- // Parse the pipeline.
- auto statusWithPipeline = Pipeline::parse(request.getPipeline(), expCtx);
- if (!statusWithPipeline.isOK()) {
- return appendCommandStatus(result, statusWithPipeline.getStatus());
- }
- auto pipeline = std::move(statusWithPipeline.getValue());
-
- auto resolvedNamespaces = resolveInvolvedNamespaces(txn, pipeline, expCtx);
+ auto resolvedNamespaces = resolveInvolvedNamespaces(txn, request);
if (!resolvedNamespaces.isOK()) {
return appendCommandStatus(result, resolvedNamespaces.getStatus());
}
@@ -376,6 +363,7 @@ public:
unique_ptr<ClientCursorPin> pin; // either this OR the exec will be non-null
unique_ptr<PlanExecutor> exec;
+ boost::intrusive_ptr<Pipeline> pipeline;
auto curOp = CurOp::get(txn);
{
// This will throw if the sharding version for this connection is out of date. If the
@@ -388,31 +376,19 @@ public:
AutoGetCollectionOrViewForRead ctx(txn, nss);
Collection* collection = ctx.getCollection();
- // If running $collStats on a view, we do not resolve the view since we want stats
- // on this view namespace.
- auto startsWithCollStats = [&pipeline]() {
- const Pipeline::SourceContainer& sources = pipeline->getSources();
- return !sources.empty() &&
- dynamic_cast<DocumentSourceCollStats*>(sources.front().get());
- };
-
// If this is a view, resolve it by finding the underlying collection and stitching view
// pipelines and this request's pipeline together. We then release our locks before
// recursively calling run, which will re-acquire locks on the underlying collection.
// (The lock must be released because recursively acquiring locks on the database will
// prohibit yielding.)
- if (ctx.getView() && !startsWithCollStats()) {
- // Check that the default collation of 'view' is compatible with the
- // operation's collation. The check is skipped if the 'request' has the empty
- // collation, which means that no collation was specified.
+ const LiteParsedPipeline liteParsedPipeline(request);
+ if (ctx.getView() && !liteParsedPipeline.startsWithCollStats()) {
+ // Check that the default collation of 'view' is compatible with the operation's
+ // collation. The check is skipped if the 'request' has the empty collation, which
+ // means that no collation was specified.
if (!request.getCollation().isEmpty()) {
- auto operationCollator = CollatorFactoryInterface::get(txn->getServiceContext())
- ->makeFromBSON(request.getCollation());
- if (!operationCollator.isOK()) {
- return appendCommandStatus(result, operationCollator.getStatus());
- }
if (!CollatorInterface::collatorsMatch(ctx.getView()->defaultCollator(),
- operationCollator.getValue().get())) {
+ expCtx->getCollator())) {
return appendCommandStatus(result,
{ErrorCodes::OptionNotSupportedOnView,
"Cannot override a view's default collation"});
@@ -473,6 +449,13 @@ public:
expCtx->setCollator(collection->getDefaultCollator()->clone());
}
+ // Parse the pipeline.
+ auto statusWithPipeline = Pipeline::parse(request.getPipeline(), expCtx);
+ if (!statusWithPipeline.isOK()) {
+ return appendCommandStatus(result, statusWithPipeline.getStatus());
+ }
+ pipeline = std::move(statusWithPipeline.getValue());
+
// Check that the view's collation matches the collation of any views involved
// in the pipeline.
auto pipelineCollationStatus =
diff --git a/src/mongo/db/pipeline/SConscript b/src/mongo/db/pipeline/SConscript
index 8a4b39f9477..b55d164ec4c 100644
--- a/src/mongo/db/pipeline/SConscript
+++ b/src/mongo/db/pipeline/SConscript
@@ -236,6 +236,7 @@ docSourceEnv.Library(
'document_source_sort.cpp',
'document_source_sort_by_count.cpp',
'document_source_unwind.cpp',
+ 'lite_parsed_document_source.cpp',
],
LIBDEPS=[
'accumulator',
diff --git a/src/mongo/db/pipeline/document_source.h b/src/mongo/db/pipeline/document_source.h
index 53ddc54332b..2d780df649c 100644
--- a/src/mongo/db/pipeline/document_source.h
+++ b/src/mongo/db/pipeline/document_source.h
@@ -44,6 +44,7 @@
#include "mongo/db/collection_index_usage_tracker.h"
#include "mongo/db/jsobj.h"
#include "mongo/db/matcher/matcher.h"
+#include "mongo/db/namespace_string.h"
#include "mongo/db/pipeline/accumulation_statement.h"
#include "mongo/db/pipeline/accumulator.h"
#include "mongo/db/pipeline/dependencies.h"
@@ -51,6 +52,7 @@
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
#include "mongo/db/pipeline/granularity_rounder.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/lookup_set_cache.h"
#include "mongo/db/pipeline/pipeline.h"
#include "mongo/db/pipeline/value.h"
@@ -63,6 +65,7 @@
namespace mongo {
+class AggregationRequest;
class Document;
class Expression;
class ExpressionFieldPath;
@@ -73,38 +76,57 @@ class PlanExecutor;
class RecordCursor;
/**
- * Registers a DocumentSource to have the name 'key'. When a stage with name '$key' is found,
- * 'parser' will be called to construct a DocumentSource.
+ * Registers a DocumentSource to have the name 'key'.
*
- * This can also be used for stages like $project and $addFields which share common functionality
- * in the unregistered DocumentSourceSingleDocumentTransformation, or for any future single-stage
- * aliases.
+ * 'liteParser' takes an AggregationRequest and a BSONElement and returns a
+ * LiteParsedDocumentSource. This is used for checks that need to happen before a full parse,
+ * such as checks about which namespaces are referenced by this aggregation.
*
- * As an example, if your document source looks like {"$foo": <args>}, with a parsing function
- * 'createFromBson', you would add this line:
- * REGISTER_DOCUMENT_SOURCE(foo, DocumentSourceFoo::createFromBson);
+ * 'fullParser' takes a BSONElement and an ExpressionContext and returns a fully-executable
+ * DocumentSource. This will be used for optimization and execution.
+ *
+ * Stages that do not require any special pre-parse checks can use
+ * LiteParsedDocumentSourceDefault::parse as their 'liteParser'.
+ *
+ * As an example, if your stage DocumentSourceFoo looks like {$foo: <args>} and does *not* require
+ * any special pre-parse checks, you should implement a static parser like
+ * DocumentSourceFoo::createFromBson(), and register it like so:
+ * REGISTER_DOCUMENT_SOURCE(foo,
+ * LiteParsedDocumentSourceDefault::parse,
+ * DocumentSourceFoo::createFromBson);
+ *
+ * If your stage is actually an alias which needs to return more than one stage (such as
+ * $sortByCount), you should use the REGISTER_MULTI_STAGE_ALIAS macro instead.
*/
-#define REGISTER_DOCUMENT_SOURCE(key, parser) \
- MONGO_INITIALIZER(addToDocSourceParserMap_##key)(InitializerContext*) { \
- auto parserWrapper = [](BSONElement stageSpec, \
- const boost::intrusive_ptr<ExpressionContext>& expCtx) { \
- return std::vector<boost::intrusive_ptr<DocumentSource>>{(parser)(stageSpec, expCtx)}; \
- }; \
- DocumentSource::registerParser("$" #key, parserWrapper); \
- return Status::OK(); \
+#define REGISTER_DOCUMENT_SOURCE(key, liteParser, fullParser) \
+ MONGO_INITIALIZER(addToDocSourceParserMap_##key)(InitializerContext*) { \
+ auto fullParserWrapper = [](BSONElement stageSpec, \
+ const boost::intrusive_ptr<ExpressionContext>& expCtx) { \
+ return std::vector<boost::intrusive_ptr<DocumentSource>>{ \
+ (fullParser)(stageSpec, expCtx)}; \
+ }; \
+ LiteParsedDocumentSource::registerParser("$" #key, liteParser); \
+ DocumentSource::registerParser("$" #key, fullParserWrapper); \
+ return Status::OK(); \
}
/**
- * Registers a multi-stage alias to have the single name 'key'. When a stage with name '$key' is
- * found, 'parser' will be called to construct a vector of DocumentSources.
+ * Registers a multi-stage alias (such as $sortByCount) to have the single name 'key'. When a stage
+ * with name '$key' is found, 'liteParser' will be used to produce a LiteParsedDocumentSource,
+ * while 'fullParser' will be called to construct a vector of DocumentSources. See the comments on
+ * REGISTER_DOCUMENT_SOURCE for more information.
*
- * As an example, if your document source looks like {"$foo": <args>}, with a parsing function
- * 'createFromBson', you would add this line:
- * REGISTER_MULTI_STAGE_ALIAS(foo, DocumentSourceFoo::createFromBson);
+ * As an example, if your stage alias looks like {$foo: <args>} and does *not* require any special
+ * pre-parse checks, you should implement a static parser like DocumentSourceFoo::createFromBson(),
+ * and register it like so:
+ * REGISTER_MULTI_STAGE_ALIAS(foo,
+ * LiteParsedDocumentSourceDefault::parse,
+ * DocumentSourceFoo::createFromBson);
*/
-#define REGISTER_MULTI_STAGE_ALIAS(key, parser) \
+#define REGISTER_MULTI_STAGE_ALIAS(key, liteParser, fullParser) \
MONGO_INITIALIZER(addAliasToDocSourceParserMap_##key)(InitializerContext*) { \
- DocumentSource::registerParser("$" #key, (parser)); \
+ LiteParsedDocumentSource::registerParser("$" #key, (liteParser)); \
+ DocumentSource::registerParser("$" #key, (fullParser)); \
return Status::OK(); \
}
@@ -1149,6 +1171,9 @@ private:
class DocumentSourceOut final : public DocumentSourceNeedsMongod, public SplittableDocumentSource {
public:
+ static std::unique_ptr<LiteParsedDocumentSourceOneForeignCollection> liteParse(
+ const AggregationRequest& request, const BSONElement& spec);
+
// virtuals from DocumentSource
~DocumentSourceOut() final;
GetNextResult getNext() final;
@@ -1764,9 +1789,13 @@ private:
class DocumentSourceLookUp final : public DocumentSourceNeedsMongod,
public SplittableDocumentSource {
public:
+ static std::unique_ptr<LiteParsedDocumentSourceOneForeignCollection> liteParse(
+ const AggregationRequest& request, const BSONElement& spec);
+
GetNextResult getNext() final;
const char* getSourceName() const final;
void serializeToArray(std::vector<Value>& array, bool explain = false) const final;
+
/**
* Attempts to combine with a subsequent $unwind stage, setting the internal '_unwindSrc'
* field.
@@ -1867,6 +1896,9 @@ private:
class DocumentSourceGraphLookUp final : public DocumentSourceNeedsMongod {
public:
+ static std::unique_ptr<LiteParsedDocumentSourceOneForeignCollection> liteParse(
+ const AggregationRequest& request, const BSONElement& spec);
+
GetNextResult getNext() final;
const char* getSourceName() const final;
void dispose() final;
@@ -2112,6 +2144,22 @@ private:
*/
class DocumentSourceCollStats : public DocumentSourceNeedsMongod {
public:
+ class LiteParsed final : public LiteParsedDocumentSource {
+ public:
+ static std::unique_ptr<LiteParsed> parse(const AggregationRequest& request,
+ const BSONElement& spec) {
+ return stdx::make_unique<LiteParsed>();
+ }
+
+ bool isCollStats() const final {
+ return true;
+ }
+
+ stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const final {
+ return stdx::unordered_set<NamespaceString>();
+ }
+ };
+
DocumentSourceCollStats(const boost::intrusive_ptr<ExpressionContext>& pExpCtx)
: DocumentSourceNeedsMongod(pExpCtx) {}
diff --git a/src/mongo/db/pipeline/document_source_add_fields.cpp b/src/mongo/db/pipeline/document_source_add_fields.cpp
index df2003f2610..7f9c2383b58 100644
--- a/src/mongo/db/pipeline/document_source_add_fields.cpp
+++ b/src/mongo/db/pipeline/document_source_add_fields.cpp
@@ -33,6 +33,7 @@
#include <boost/optional.hpp>
#include <boost/smart_ptr/intrusive_ptr.hpp>
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/parsed_add_fields.h"
namespace mongo {
@@ -40,7 +41,9 @@ namespace mongo {
using boost::intrusive_ptr;
using parsed_aggregation_projection::ParsedAddFields;
-REGISTER_DOCUMENT_SOURCE(addFields, DocumentSourceAddFields::createFromBson);
+REGISTER_DOCUMENT_SOURCE(addFields,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceAddFields::createFromBson);
intrusive_ptr<DocumentSource> DocumentSourceAddFields::create(
BSONObj addFieldsSpec, const intrusive_ptr<ExpressionContext>& expCtx) {
diff --git a/src/mongo/db/pipeline/document_source_bucket.cpp b/src/mongo/db/pipeline/document_source_bucket.cpp
index f3eebe2860c..484f4b9a62f 100644
--- a/src/mongo/db/pipeline/document_source_bucket.cpp
+++ b/src/mongo/db/pipeline/document_source_bucket.cpp
@@ -28,12 +28,16 @@
#include "mongo/db/pipeline/document_source.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
+
namespace mongo {
using boost::intrusive_ptr;
using std::vector;
-REGISTER_MULTI_STAGE_ALIAS(bucket, DocumentSourceBucket::createFromBson);
+REGISTER_MULTI_STAGE_ALIAS(bucket,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceBucket::createFromBson);
namespace {
intrusive_ptr<ExpressionConstant> getExpressionConstant(BSONElement expressionElem,
@@ -121,8 +125,6 @@ vector<intrusive_ptr<DocumentSource>> DocumentSourceBucket::createFromBson(
<< typeName(upper.getType())
<< ".",
lowerCanonicalType == upperCanonicalType);
- // TODO SERVER-25038: This check must be deferred so that it respects the final
- // collator, which is not necessarily the same as the collator at parse time.
uassert(40194,
str::stream()
<< "The 'boundaries' option to $bucket must be sorted, but elements "
@@ -178,9 +180,6 @@ vector<intrusive_ptr<DocumentSource>> DocumentSourceBucket::createFromBson(
// If the default has the same canonical type as the bucket's boundaries, then make sure the
// default is less than the lowest boundary or greater than or equal to the highest
// boundary.
- //
- // TODO SERVER-25038: This check must be deferred so that it respects the final collator,
- // which is not necessarily the same as the collator at parse time.
const auto& valueCmp = pExpCtx->getValueComparator();
const bool hasValidDefault = valueCmp.evaluate(defaultValue < lowerValue) ||
valueCmp.evaluate(defaultValue >= upperValue);
diff --git a/src/mongo/db/pipeline/document_source_bucket_auto.cpp b/src/mongo/db/pipeline/document_source_bucket_auto.cpp
index 3d89da9d790..31374548175 100644
--- a/src/mongo/db/pipeline/document_source_bucket_auto.cpp
+++ b/src/mongo/db/pipeline/document_source_bucket_auto.cpp
@@ -31,6 +31,7 @@
#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/accumulation_statement.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
namespace mongo {
@@ -39,7 +40,9 @@ using std::pair;
using std::string;
using std::vector;
-REGISTER_DOCUMENT_SOURCE(bucketAuto, DocumentSourceBucketAuto::createFromBson);
+REGISTER_DOCUMENT_SOURCE(bucketAuto,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceBucketAuto::createFromBson);
const char* DocumentSourceBucketAuto::getSourceName() const {
return "$bucketAuto";
diff --git a/src/mongo/db/pipeline/document_source_coll_stats.cpp b/src/mongo/db/pipeline/document_source_coll_stats.cpp
index 7ae2e1d0e7b..4c81626942b 100644
--- a/src/mongo/db/pipeline/document_source_coll_stats.cpp
+++ b/src/mongo/db/pipeline/document_source_coll_stats.cpp
@@ -31,6 +31,7 @@
#include "mongo/db/pipeline/document_source.h"
#include "mongo/bson/bsonobj.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/stats/top.h"
#include "mongo/util/time_support.h"
@@ -38,7 +39,9 @@ using boost::intrusive_ptr;
namespace mongo {
-REGISTER_DOCUMENT_SOURCE(collStats, DocumentSourceCollStats::createFromBson);
+REGISTER_DOCUMENT_SOURCE(collStats,
+ DocumentSourceCollStats::LiteParsed::parse,
+ DocumentSourceCollStats::createFromBson);
const char* DocumentSourceCollStats::getSourceName() const {
return "$collStats";
diff --git a/src/mongo/db/pipeline/document_source_count.cpp b/src/mongo/db/pipeline/document_source_count.cpp
index 6447f0af08a..269126b219b 100644
--- a/src/mongo/db/pipeline/document_source_count.cpp
+++ b/src/mongo/db/pipeline/document_source_count.cpp
@@ -28,9 +28,11 @@
#include "mongo/platform/basic.h"
-#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document_source.h"
+
+#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
namespace mongo {
@@ -38,7 +40,9 @@ using boost::intrusive_ptr;
using std::vector;
using std::string;
-REGISTER_MULTI_STAGE_ALIAS(count, DocumentSourceCount::createFromBson);
+REGISTER_MULTI_STAGE_ALIAS(count,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceCount::createFromBson);
vector<intrusive_ptr<DocumentSource>> DocumentSourceCount::createFromBson(
BSONElement elem, const intrusive_ptr<ExpressionContext>& pExpCtx) {
diff --git a/src/mongo/db/pipeline/document_source_facet.cpp b/src/mongo/db/pipeline/document_source_facet.cpp
index 3761fb24e81..55fd916e02c 100644
--- a/src/mongo/db/pipeline/document_source_facet.cpp
+++ b/src/mongo/db/pipeline/document_source_facet.cpp
@@ -30,6 +30,7 @@
#include "mongo/db/pipeline/document_source_facet.h"
+#include <memory>
#include <vector>
#include "mongo/base/string_data.h"
@@ -42,12 +43,14 @@
#include "mongo/db/pipeline/pipeline.h"
#include "mongo/db/pipeline/tee_buffer.h"
#include "mongo/db/pipeline/value.h"
+#include "mongo/stdx/memory.h"
#include "mongo/util/assert_util.h"
#include "mongo/util/mongoutils/str.h"
namespace mongo {
using boost::intrusive_ptr;
+using std::pair;
using std::string;
using std::vector;
@@ -63,7 +66,78 @@ DocumentSourceFacet::DocumentSourceFacet(std::vector<FacetPipeline> facetPipelin
}
}
-REGISTER_DOCUMENT_SOURCE(facet, DocumentSourceFacet::createFromBson);
+namespace {
+/**
+ * Extracts the names of the facets and the vectors of raw BSONObjs representing the stages within
+ * that facet's pipeline.
+ *
+ * Throws a UserException if it fails to parse for any reason.
+ */
+vector<pair<string, vector<BSONObj>>> extractRawPipelines(const BSONElement& elem) {
+ uassert(40169,
+ str::stream() << "the $facet specification must be a non-empty object, but found: "
+ << elem,
+ elem.type() == BSONType::Object && !elem.embeddedObject().isEmpty());
+
+ vector<pair<string, vector<BSONObj>>> rawFacetPipelines;
+ for (auto&& facetElem : elem.embeddedObject()) {
+ const auto facetName = facetElem.fieldNameStringData();
+ FieldPath::uassertValidFieldName(facetName);
+ uassert(40170,
+ str::stream() << "arguments to $facet must be arrays, " << facetName << " is type "
+ << typeName(facetElem.type()),
+ facetElem.type() == BSONType::Array);
+
+ vector<BSONObj> rawPipeline;
+ for (auto&& subPipeElem : facetElem.Obj()) {
+ uassert(40171,
+ str::stream() << "elements of arrays in $facet spec must be non-empty objects, "
+ << facetName
+ << " argument contained an element of type "
+ << typeName(subPipeElem.type())
+ << ": "
+ << subPipeElem,
+ subPipeElem.type() == BSONType::Object);
+ auto stageName = subPipeElem.Obj().firstElementFieldName();
+ uassert(
+ 40331,
+ str::stream() << "specified stage is not allowed to be used within a $facet stage: "
+ << subPipeElem,
+ !str::equals(stageName, "$out") && !str::equals(stageName, "$facet"));
+
+ rawPipeline.push_back(subPipeElem.embeddedObject());
+ }
+
+ rawFacetPipelines.emplace_back(facetName.toString(), std::move(rawPipeline));
+ }
+ return rawFacetPipelines;
+}
+} // namespace
+
+std::unique_ptr<DocumentSourceFacet::LiteParsed> DocumentSourceFacet::LiteParsed::parse(
+ const AggregationRequest& request, const BSONElement& spec) {
+ std::vector<LiteParsedPipeline> liteParsedPipelines;
+ for (auto&& rawPipeline : extractRawPipelines(spec)) {
+ liteParsedPipelines.emplace_back(
+ AggregationRequest(request.getNamespaceString(), rawPipeline.second));
+ }
+ return std::unique_ptr<DocumentSourceFacet::LiteParsed>(
+ new DocumentSourceFacet::LiteParsed(std::move(liteParsedPipelines)));
+}
+
+stdx::unordered_set<NamespaceString> DocumentSourceFacet::LiteParsed::getInvolvedNamespaces()
+ const {
+ stdx::unordered_set<NamespaceString> involvedNamespaces;
+ for (auto&& liteParsedPipeline : _liteParsedPipelines) {
+ auto involvedInSubPipe = liteParsedPipeline.getInvolvedNamespaces();
+ involvedNamespaces.insert(involvedInSubPipe.begin(), involvedInSubPipe.end());
+ }
+ return involvedNamespaces;
+}
+
+REGISTER_DOCUMENT_SOURCE(facet,
+ DocumentSourceFacet::LiteParsed::parse,
+ DocumentSourceFacet::createFromBson);
intrusive_ptr<DocumentSourceFacet> DocumentSourceFacet::create(
std::vector<FacetPipeline> facetPipelines, const intrusive_ptr<ExpressionContext>& expCtx) {
@@ -189,53 +263,28 @@ DocumentSource::GetDepsReturn DocumentSourceFacet::getDependencies(DepsTracker*
intrusive_ptr<DocumentSource> DocumentSourceFacet::createFromBson(
BSONElement elem, const intrusive_ptr<ExpressionContext>& expCtx) {
- uassert(40169,
- str::stream() << "the $facet specification must be a non-empty object, but found: "
- << elem,
- elem.type() == BSONType::Object && !elem.embeddedObject().isEmpty());
std::vector<FacetPipeline> facetPipelines;
- for (auto&& facetElem : elem.embeddedObject()) {
- const auto facetName = facetElem.fieldNameStringData();
- FieldPath::uassertValidFieldName(facetName);
- uassert(40170,
- str::stream() << "arguments to $facet must be arrays, " << facetName << " is type "
- << typeName(facetElem.type()),
- facetElem.type() == BSONType::Array);
-
- vector<BSONObj> rawPipeline;
- for (auto&& subPipeElem : facetElem.Obj()) {
- uassert(40171,
- str::stream() << "elements of arrays in $facet spec must be objects, "
- << facetName
- << " argument contained an element of type "
- << typeName(subPipeElem.type()),
- subPipeElem.type() == BSONType::Object);
-
- rawPipeline.push_back(subPipeElem.embeddedObject());
- }
+ for (auto&& rawFacet : extractRawPipelines(elem)) {
+ const auto facetName = rawFacet.first;
- auto pipeline = uassertStatusOK(Pipeline::parse(rawPipeline, expCtx));
+ auto pipeline = uassertStatusOK(Pipeline::parse(rawFacet.second, expCtx));
uassert(40172,
- str::stream() << "sub-pipelines in $facet stage cannot be empty: "
- << facetElem.toString(),
+ str::stream() << "sub-pipeline in $facet stage cannot be empty: " << facetName,
!pipeline->getSources().empty());
- // Disallow $out stages, $facet stages, and any stages that need to be the first stage in
- // the pipeline.
+ // Disallow any stages that need to be the first stage in the pipeline.
for (auto&& stage : pipeline->getSources()) {
- if ((dynamic_cast<DocumentSourceOut*>(stage.get())) ||
- (dynamic_cast<DocumentSourceFacet*>(stage.get())) ||
- (stage->isValidInitialSource())) {
+ if (stage->isValidInitialSource()) {
uasserted(40173,
str::stream() << stage->getSourceName()
<< " is not allowed to be used within a $facet stage: "
- << facetElem.toString());
+ << elem.toString());
}
}
- facetPipelines.emplace_back(facetName.toString(), std::move(pipeline));
+ facetPipelines.emplace_back(facetName, std::move(pipeline));
}
return new DocumentSourceFacet(std::move(facetPipelines), expCtx);
diff --git a/src/mongo/db/pipeline/document_source_facet.h b/src/mongo/db/pipeline/document_source_facet.h
index 8f30f10f887..b0a9a0867ee 100644
--- a/src/mongo/db/pipeline/document_source_facet.h
+++ b/src/mongo/db/pipeline/document_source_facet.h
@@ -34,6 +34,8 @@
#include <vector>
#include "mongo/db/pipeline/document_source.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
+#include "mongo/db/pipeline/lite_parsed_pipeline.h"
#include "mongo/db/pipeline/pipeline.h"
namespace mongo {
@@ -64,6 +66,20 @@ public:
boost::intrusive_ptr<Pipeline> pipeline;
};
+ class LiteParsed : public LiteParsedDocumentSource {
+ public:
+ static std::unique_ptr<LiteParsed> parse(const AggregationRequest& request,
+ const BSONElement& spec);
+
+ stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const final;
+
+ private:
+ LiteParsed(std::vector<LiteParsedPipeline> liteParsedPipelines)
+ : _liteParsedPipelines(std::move(liteParsedPipelines)) {}
+
+ const std::vector<LiteParsedPipeline> _liteParsedPipelines;
+ };
+
static boost::intrusive_ptr<DocumentSource> createFromBson(
BSONElement elem, const boost::intrusive_ptr<ExpressionContext>& pExpCtx);
@@ -122,7 +138,7 @@ public:
private:
DocumentSourceFacet(std::vector<FacetPipeline> facetPipelines,
- const boost::intrusive_ptr<ExpressionContext>& pExpCtx);
+ const boost::intrusive_ptr<ExpressionContext>& expCtx);
Value serialize(bool explain = false) const final;
diff --git a/src/mongo/db/pipeline/document_source_geo_near.cpp b/src/mongo/db/pipeline/document_source_geo_near.cpp
index 4ec43f59d5d..62914105716 100644
--- a/src/mongo/db/pipeline/document_source_geo_near.cpp
+++ b/src/mongo/db/pipeline/document_source_geo_near.cpp
@@ -30,15 +30,19 @@
#include "mongo/platform/basic.h"
-#include "mongo/db/pipeline/document.h"
#include "mongo/db/pipeline/document_source.h"
+
+#include "mongo/db/pipeline/document.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/util/log.h"
namespace mongo {
using boost::intrusive_ptr;
-REGISTER_DOCUMENT_SOURCE(geoNear, DocumentSourceGeoNear::createFromBson);
+REGISTER_DOCUMENT_SOURCE(geoNear,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceGeoNear::createFromBson);
const long long DocumentSourceGeoNear::kDefaultLimit = 100;
diff --git a/src/mongo/db/pipeline/document_source_graph_lookup.cpp b/src/mongo/db/pipeline/document_source_graph_lookup.cpp
index daa51ad8294..2e8b5131d01 100644
--- a/src/mongo/db/pipeline/document_source_graph_lookup.cpp
+++ b/src/mongo/db/pipeline/document_source_graph_lookup.cpp
@@ -44,11 +44,35 @@
namespace mongo {
using boost::intrusive_ptr;
-using std::unique_ptr;
namespace dps = ::mongo::dotted_path_support;
-REGISTER_DOCUMENT_SOURCE(graphLookup, DocumentSourceGraphLookUp::createFromBson);
+std::unique_ptr<LiteParsedDocumentSourceOneForeignCollection> DocumentSourceGraphLookUp::liteParse(
+ const AggregationRequest& request, const BSONElement& spec) {
+ uassert(40327,
+ str::stream() << "the $graphLookup stage specification must be an object, but found "
+ << typeName(spec.type()),
+ spec.type() == BSONType::Object);
+
+ auto specObj = spec.Obj();
+ auto fromElement = specObj["from"];
+ uassert(40328,
+ str::stream() << "missing 'from' option to $graphLookup stage specification: "
+ << specObj,
+ fromElement);
+ uassert(40329,
+ str::stream() << "'from' option to $graphLookup must be a string, but was type "
+ << typeName(specObj["from"].type()),
+ fromElement.type() == BSONType::String);
+
+ NamespaceString nss(request.getNamespaceString().db(), fromElement.valueStringData());
+ uassert(40330, str::stream() << "invalid $graphLookup namespace: " << nss.ns(), nss.isValid());
+ return stdx::make_unique<LiteParsedDocumentSourceOneForeignCollection>(std::move(nss));
+}
+
+REGISTER_DOCUMENT_SOURCE(graphLookup,
+ DocumentSourceGraphLookUp::liteParse,
+ DocumentSourceGraphLookUp::createFromBson);
const char* DocumentSourceGraphLookUp::getSourceName() const {
return "$graphLookup";
diff --git a/src/mongo/db/pipeline/document_source_group.cpp b/src/mongo/db/pipeline/document_source_group.cpp
index 56844530c26..31012cdc476 100644
--- a/src/mongo/db/pipeline/document_source_group.cpp
+++ b/src/mongo/db/pipeline/document_source_group.cpp
@@ -28,13 +28,15 @@
#include "mongo/platform/basic.h"
+#include "mongo/db/pipeline/document_source.h"
+
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/accumulation_statement.h"
#include "mongo/db/pipeline/accumulator.h"
#include "mongo/db/pipeline/document.h"
-#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
#include "mongo/db/pipeline/value_comparator.h"
#include "mongo/stdx/memory.h"
@@ -46,7 +48,9 @@ using std::shared_ptr;
using std::pair;
using std::vector;
-REGISTER_DOCUMENT_SOURCE(group, DocumentSourceGroup::createFromBson);
+REGISTER_DOCUMENT_SOURCE(group,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceGroup::createFromBson);
const char* DocumentSourceGroup::getSourceName() const {
return "$group";
diff --git a/src/mongo/db/pipeline/document_source_index_stats.cpp b/src/mongo/db/pipeline/document_source_index_stats.cpp
index b3b003bb89f..b6feff69eb7 100644
--- a/src/mongo/db/pipeline/document_source_index_stats.cpp
+++ b/src/mongo/db/pipeline/document_source_index_stats.cpp
@@ -30,11 +30,15 @@
#include "mongo/db/pipeline/document_source.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
+
namespace mongo {
using boost::intrusive_ptr;
-REGISTER_DOCUMENT_SOURCE(indexStats, DocumentSourceIndexStats::createFromBson);
+REGISTER_DOCUMENT_SOURCE(indexStats,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceIndexStats::createFromBson);
const char* DocumentSourceIndexStats::getSourceName() const {
return "$indexStats";
diff --git a/src/mongo/db/pipeline/document_source_limit.cpp b/src/mongo/db/pipeline/document_source_limit.cpp
index c358efa9441..cdc3ed2d0a6 100644
--- a/src/mongo/db/pipeline/document_source_limit.cpp
+++ b/src/mongo/db/pipeline/document_source_limit.cpp
@@ -28,11 +28,13 @@
#include "mongo/platform/basic.h"
+#include "mongo/db/pipeline/document_source.h"
+
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document.h"
-#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -43,7 +45,9 @@ DocumentSourceLimit::DocumentSourceLimit(const intrusive_ptr<ExpressionContext>&
long long limit)
: DocumentSource(pExpCtx), _limit(limit) {}
-REGISTER_DOCUMENT_SOURCE(limit, DocumentSourceLimit::createFromBson);
+REGISTER_DOCUMENT_SOURCE(limit,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceLimit::createFromBson);
const char* DocumentSourceLimit::getSourceName() const {
return "$limit";
diff --git a/src/mongo/db/pipeline/document_source_lookup.cpp b/src/mongo/db/pipeline/document_source_lookup.cpp
index 5c9856f93ab..4607d9a28e8 100644
--- a/src/mongo/db/pipeline/document_source_lookup.cpp
+++ b/src/mongo/db/pipeline/document_source_lookup.cpp
@@ -56,7 +56,31 @@ DocumentSourceLookUp::DocumentSourceLookUp(NamespaceString fromNs,
_foreignField(foreignField),
_foreignFieldFieldName(std::move(foreignField)) {}
-REGISTER_DOCUMENT_SOURCE(lookup, DocumentSourceLookUp::createFromBson);
+std::unique_ptr<LiteParsedDocumentSourceOneForeignCollection> DocumentSourceLookUp::liteParse(
+ const AggregationRequest& request, const BSONElement& spec) {
+ uassert(40319,
+ str::stream() << "the $lookup stage specification must be an object, but found "
+ << typeName(spec.type()),
+ spec.type() == BSONType::Object);
+
+ auto specObj = spec.Obj();
+ auto fromElement = specObj["from"];
+ uassert(40320,
+ str::stream() << "missing 'from' option to $lookup stage specification: " << specObj,
+ fromElement);
+ uassert(40321,
+ str::stream() << "'from' option to $lookup must be a string, but was type "
+ << typeName(specObj["from"].type()),
+ fromElement.type() == BSONType::String);
+
+ NamespaceString nss(request.getNamespaceString().db(), fromElement.valueStringData());
+ uassert(40322, str::stream() << "invalid $lookup namespace: " << nss.ns(), nss.isValid());
+ return stdx::make_unique<LiteParsedDocumentSourceOneForeignCollection>(std::move(nss));
+}
+
+REGISTER_DOCUMENT_SOURCE(lookup,
+ DocumentSourceLookUp::liteParse,
+ DocumentSourceLookUp::createFromBson);
const char* DocumentSourceLookUp::getSourceName() const {
return "$lookup";
diff --git a/src/mongo/db/pipeline/document_source_match.cpp b/src/mongo/db/pipeline/document_source_match.cpp
index f5c4432bc31..8476c82a67d 100644
--- a/src/mongo/db/pipeline/document_source_match.cpp
+++ b/src/mongo/db/pipeline/document_source_match.cpp
@@ -28,7 +28,7 @@
#include "mongo/platform/basic.h"
-#include <cctype>
+#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/jsobj.h"
#include "mongo/db/matcher/expression_algo.h"
@@ -36,8 +36,8 @@
#include "mongo/db/matcher/expression_leaf.h"
#include "mongo/db/matcher/extensions_callback_noop.h"
#include "mongo/db/pipeline/document.h"
-#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/expression.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/stringutils.h"
@@ -49,7 +49,9 @@ using std::unique_ptr;
using std::string;
using std::vector;
-REGISTER_DOCUMENT_SOURCE(match, DocumentSourceMatch::createFromBson);
+REGISTER_DOCUMENT_SOURCE(match,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceMatch::createFromBson);
const char* DocumentSourceMatch::getSourceName() const {
return "$match";
diff --git a/src/mongo/db/pipeline/document_source_merge_cursors.cpp b/src/mongo/db/pipeline/document_source_merge_cursors.cpp
index aec8c73facf..87a5700f206 100644
--- a/src/mongo/db/pipeline/document_source_merge_cursors.cpp
+++ b/src/mongo/db/pipeline/document_source_merge_cursors.cpp
@@ -30,6 +30,7 @@
#include "mongo/db/pipeline/document_source.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
namespace mongo {
@@ -43,7 +44,9 @@ DocumentSourceMergeCursors::DocumentSourceMergeCursors(
const intrusive_ptr<ExpressionContext>& pExpCtx)
: DocumentSource(pExpCtx), _cursorDescriptors(std::move(cursorDescriptors)), _unstarted(true) {}
-REGISTER_DOCUMENT_SOURCE(mergeCursors, DocumentSourceMergeCursors::createFromBson);
+REGISTER_DOCUMENT_SOURCE(mergeCursors,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceMergeCursors::createFromBson);
const char* DocumentSourceMergeCursors::getSourceName() const {
return "$mergeCursors";
diff --git a/src/mongo/db/pipeline/document_source_out.cpp b/src/mongo/db/pipeline/document_source_out.cpp
index 8e5218e552a..be9f76d2207 100644
--- a/src/mongo/db/pipeline/document_source_out.cpp
+++ b/src/mongo/db/pipeline/document_source_out.cpp
@@ -29,6 +29,8 @@
#include "mongo/platform/basic.h"
#include "mongo/db/pipeline/document_source.h"
+
+#include "mongo/stdx/memory.h"
#include "mongo/util/destructor_guard.h"
namespace mongo {
@@ -44,7 +46,21 @@ DocumentSourceOut::~DocumentSourceOut() {
if (_mongod && _tempNs.size()) _mongod->directClient()->dropCollection(_tempNs.ns());)
}
-REGISTER_DOCUMENT_SOURCE(out, DocumentSourceOut::createFromBson);
+std::unique_ptr<LiteParsedDocumentSourceOneForeignCollection> DocumentSourceOut::liteParse(
+ const AggregationRequest& request, const BSONElement& spec) {
+ uassert(40325,
+ str::stream() << "$out stage requires a string argument, but found "
+ << typeName(spec.type()),
+ spec.type() == BSONType::String);
+
+ NamespaceString targetNss(request.getNamespaceString().db(), spec.valueStringData());
+ uassert(40326,
+ str::stream() << "Invalid $out target namespace, " << targetNss.ns(),
+ targetNss.isValid());
+ return stdx::make_unique<LiteParsedDocumentSourceOneForeignCollection>(std::move(targetNss));
+}
+
+REGISTER_DOCUMENT_SOURCE(out, DocumentSourceOut::liteParse, DocumentSourceOut::createFromBson);
const char* DocumentSourceOut::getSourceName() const {
return "$out";
diff --git a/src/mongo/db/pipeline/document_source_project.cpp b/src/mongo/db/pipeline/document_source_project.cpp
index aa6a1529a26..8ea144d0d70 100644
--- a/src/mongo/db/pipeline/document_source_project.cpp
+++ b/src/mongo/db/pipeline/document_source_project.cpp
@@ -33,6 +33,7 @@
#include <boost/optional.hpp>
#include <boost/smart_ptr/intrusive_ptr.hpp>
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/parsed_aggregation_projection.h"
namespace mongo {
@@ -41,7 +42,9 @@ using boost::intrusive_ptr;
using parsed_aggregation_projection::ParsedAggregationProjection;
using parsed_aggregation_projection::ProjectionType;
-REGISTER_DOCUMENT_SOURCE(project, DocumentSourceProject::createFromBson);
+REGISTER_DOCUMENT_SOURCE(project,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceProject::createFromBson);
intrusive_ptr<DocumentSource> DocumentSourceProject::create(
BSONObj projectSpec, const intrusive_ptr<ExpressionContext>& expCtx) {
diff --git a/src/mongo/db/pipeline/document_source_redact.cpp b/src/mongo/db/pipeline/document_source_redact.cpp
index 9ad79eca661..f7570c8c733 100644
--- a/src/mongo/db/pipeline/document_source_redact.cpp
+++ b/src/mongo/db/pipeline/document_source_redact.cpp
@@ -35,6 +35,7 @@
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document.h"
#include "mongo/db/pipeline/expression.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -46,7 +47,9 @@ DocumentSourceRedact::DocumentSourceRedact(const intrusive_ptr<ExpressionContext
const intrusive_ptr<Expression>& expression)
: DocumentSource(expCtx), _expression(expression) {}
-REGISTER_DOCUMENT_SOURCE(redact, DocumentSourceRedact::createFromBson);
+REGISTER_DOCUMENT_SOURCE(redact,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceRedact::createFromBson);
const char* DocumentSourceRedact::getSourceName() const {
return "$redact";
diff --git a/src/mongo/db/pipeline/document_source_replace_root.cpp b/src/mongo/db/pipeline/document_source_replace_root.cpp
index 8d29a5d7c4a..0c1c1a38e8c 100644
--- a/src/mongo/db/pipeline/document_source_replace_root.cpp
+++ b/src/mongo/db/pipeline/document_source_replace_root.cpp
@@ -35,6 +35,7 @@
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document.h"
#include "mongo/db/pipeline/expression.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -145,7 +146,9 @@ private:
boost::intrusive_ptr<Expression> _newRoot;
};
-REGISTER_DOCUMENT_SOURCE(replaceRoot, DocumentSourceReplaceRoot::createFromBson);
+REGISTER_DOCUMENT_SOURCE(replaceRoot,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceReplaceRoot::createFromBson);
intrusive_ptr<DocumentSource> DocumentSourceReplaceRoot::createFromBson(
BSONElement elem, const intrusive_ptr<ExpressionContext>& pExpCtx) {
diff --git a/src/mongo/db/pipeline/document_source_sample.cpp b/src/mongo/db/pipeline/document_source_sample.cpp
index 4e7fdbc3364..53c3040de7d 100644
--- a/src/mongo/db/pipeline/document_source_sample.cpp
+++ b/src/mongo/db/pipeline/document_source_sample.cpp
@@ -34,6 +34,7 @@
#include "mongo/db/pipeline/document.h"
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -42,7 +43,9 @@ using boost::intrusive_ptr;
DocumentSourceSample::DocumentSourceSample(const intrusive_ptr<ExpressionContext>& pExpCtx)
: DocumentSource(pExpCtx), _size(0) {}
-REGISTER_DOCUMENT_SOURCE(sample, DocumentSourceSample::createFromBson);
+REGISTER_DOCUMENT_SOURCE(sample,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceSample::createFromBson);
const char* DocumentSourceSample::getSourceName() const {
return "$sample";
diff --git a/src/mongo/db/pipeline/document_source_skip.cpp b/src/mongo/db/pipeline/document_source_skip.cpp
index 0b751506619..137b9ac9483 100644
--- a/src/mongo/db/pipeline/document_source_skip.cpp
+++ b/src/mongo/db/pipeline/document_source_skip.cpp
@@ -28,11 +28,13 @@
#include "mongo/platform/basic.h"
+#include "mongo/db/pipeline/document_source.h"
+
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document.h"
-#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -43,7 +45,9 @@ DocumentSourceSkip::DocumentSourceSkip(const intrusive_ptr<ExpressionContext>& p
long long nToSkip)
: DocumentSource(pExpCtx), _nToSkip(nToSkip) {}
-REGISTER_DOCUMENT_SOURCE(skip, DocumentSourceSkip::createFromBson);
+REGISTER_DOCUMENT_SOURCE(skip,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceSkip::createFromBson);
const char* DocumentSourceSkip::getSourceName() const {
return "$skip";
diff --git a/src/mongo/db/pipeline/document_source_sort.cpp b/src/mongo/db/pipeline/document_source_sort.cpp
index 2091b6a5cf4..e25867f476b 100644
--- a/src/mongo/db/pipeline/document_source_sort.cpp
+++ b/src/mongo/db/pipeline/document_source_sort.cpp
@@ -30,11 +30,11 @@
#include "mongo/db/pipeline/document_source.h"
-
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document.h"
#include "mongo/db/pipeline/expression.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -48,7 +48,9 @@ using std::vector;
DocumentSourceSort::DocumentSourceSort(const intrusive_ptr<ExpressionContext>& pExpCtx)
: DocumentSource(pExpCtx), _mergingPresorted(false) {}
-REGISTER_DOCUMENT_SOURCE(sort, DocumentSourceSort::createFromBson);
+REGISTER_DOCUMENT_SOURCE(sort,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceSort::createFromBson);
const char* DocumentSourceSort::getSourceName() const {
return "$sort";
diff --git a/src/mongo/db/pipeline/document_source_sort_by_count.cpp b/src/mongo/db/pipeline/document_source_sort_by_count.cpp
index 38fb151976d..021843b3e7f 100644
--- a/src/mongo/db/pipeline/document_source_sort_by_count.cpp
+++ b/src/mongo/db/pipeline/document_source_sort_by_count.cpp
@@ -28,16 +28,20 @@
#include "mongo/platform/basic.h"
-#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document_source.h"
+
+#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/expression_context.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
namespace mongo {
using boost::intrusive_ptr;
using std::vector;
-REGISTER_MULTI_STAGE_ALIAS(sortByCount, DocumentSourceSortByCount::createFromBson);
+REGISTER_MULTI_STAGE_ALIAS(sortByCount,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceSortByCount::createFromBson);
vector<intrusive_ptr<DocumentSource>> DocumentSourceSortByCount::createFromBson(
BSONElement elem, const intrusive_ptr<ExpressionContext>& pExpCtx) {
diff --git a/src/mongo/db/pipeline/document_source_unwind.cpp b/src/mongo/db/pipeline/document_source_unwind.cpp
index 86c6aa2c41d..a03bd1e8034 100644
--- a/src/mongo/db/pipeline/document_source_unwind.cpp
+++ b/src/mongo/db/pipeline/document_source_unwind.cpp
@@ -28,10 +28,12 @@
#include "mongo/platform/basic.h"
+#include "mongo/db/pipeline/document_source.h"
+
#include "mongo/db/jsobj.h"
#include "mongo/db/pipeline/document.h"
-#include "mongo/db/pipeline/document_source.h"
#include "mongo/db/pipeline/expression.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
#include "mongo/db/pipeline/value.h"
namespace mongo {
@@ -163,7 +165,9 @@ DocumentSourceUnwind::DocumentSourceUnwind(const intrusive_ptr<ExpressionContext
_indexPath(indexPath),
_unwinder(new Unwinder(fieldPath, preserveNullAndEmptyArrays, indexPath)) {}
-REGISTER_DOCUMENT_SOURCE(unwind, DocumentSourceUnwind::createFromBson);
+REGISTER_DOCUMENT_SOURCE(unwind,
+ LiteParsedDocumentSourceDefault::parse,
+ DocumentSourceUnwind::createFromBson);
const char* DocumentSourceUnwind::getSourceName() const {
return "$unwind";
diff --git a/src/mongo/db/pipeline/lite_parsed_document_source.cpp b/src/mongo/db/pipeline/lite_parsed_document_source.cpp
new file mode 100644
index 00000000000..d25c24290ed
--- /dev/null
+++ b/src/mongo/db/pipeline/lite_parsed_document_source.cpp
@@ -0,0 +1,63 @@
+/**
+ * Copyright (C) 2016 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/lite_parsed_document_source.h"
+
+#include "mongo/util/string_map.h"
+
+namespace mongo {
+
+using Parser = LiteParsedDocumentSource::Parser;
+
+namespace {
+StringMap<Parser> parserMap;
+} // namespace
+
+void LiteParsedDocumentSource::registerParser(const std::string& name, Parser parser) {
+ parserMap[name] = parser;
+}
+
+std::unique_ptr<LiteParsedDocumentSource> LiteParsedDocumentSource::parse(
+ const AggregationRequest& request, const BSONObj& spec) {
+ uassert(40323,
+ "A pipeline stage specification object must contain exactly one field.",
+ spec.nFields() == 1);
+ BSONElement specElem = spec.firstElement();
+
+ auto stageName = specElem.fieldNameStringData();
+ auto it = parserMap.find(stageName);
+
+ uassert(40324,
+ str::stream() << "Unrecognized pipeline stage name: '" << stageName << "'",
+ it != parserMap.end());
+
+ return it->second(request, specElem);
+}
+}
diff --git a/src/mongo/db/pipeline/lite_parsed_document_source.h b/src/mongo/db/pipeline/lite_parsed_document_source.h
new file mode 100644
index 00000000000..d7a33a1776e
--- /dev/null
+++ b/src/mongo/db/pipeline/lite_parsed_document_source.h
@@ -0,0 +1,128 @@
+/**
+ * Copyright (C) 2016 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 <boost/optional.hpp>
+#include <memory>
+#include <vector>
+
+#include "mongo/db/namespace_string.h"
+#include "mongo/db/pipeline/aggregation_request.h"
+#include "mongo/stdx/functional.h"
+#include "mongo/stdx/memory.h"
+#include "mongo/stdx/unordered_set.h"
+
+namespace mongo {
+
+/**
+ * A lightly parsed version of a DocumentSource. It is not executable and not guaranteed to return a
+ * parse error when encountering an invalid specification. Instead, the purpose of this class is to
+ * make certain DocumentSource properties available before full parsing (e.g., getting the involved
+ * foreign collections).
+ */
+class LiteParsedDocumentSource {
+public:
+ /*
+ * This is the type of parser you should register using REGISTER_DOCUMENT_SOURCE. It need not
+ * do any validation of options, only enough parsing to be able to implement the interface.
+ *
+ * The AggregationRequest can be used to determine related information like the namespace on
+ * which this aggregation is being performed, and the BSONElement will be the element whose
+ * field name is the name of this stage (e.g. the first and only element in {$limit: 1}).
+ */
+ using Parser = stdx::function<std::unique_ptr<LiteParsedDocumentSource>(
+ const AggregationRequest&, const BSONElement&)>;
+
+ /**
+ * Registers a DocumentSource with a spec parsing function, so that when a stage with the given
+ * name is encountered, it will call 'parser' to construct that stage's specification object.
+ *
+ * DO NOT call this method directly. Instead, use the REGISTER_DOCUMENT_SOURCE macro defined in
+ * document_source.h.
+ */
+ static void registerParser(const std::string& name, Parser parser);
+
+ /**
+ * Constructs a LiteParsedDocumentSource from the user-supplied BSON, or throws a UserException.
+ *
+ * Extracts the first field name from 'spec', and delegates to the parser that was registered
+ * with that field name using registerParser() above.
+ */
+ static std::unique_ptr<LiteParsedDocumentSource> parse(const AggregationRequest& request,
+ const BSONObj& spec);
+
+ /**
+ * Returns the foreign collection(s) referenced by this stage, if any.
+ */
+ virtual stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const = 0;
+
+ /**
+ * Returns true if this is a $collStats stage.
+ */
+ virtual bool isCollStats() const {
+ return false;
+ }
+};
+
+class LiteParsedDocumentSourceDefault final : public LiteParsedDocumentSource {
+public:
+ /**
+ * Creates the default LiteParsedDocumentSource. This should be used with caution. Make sure
+ * your stage doesn't need to communicate any special behavior before registering a
+ * DocumentSource using this parser.
+ */
+ static std::unique_ptr<LiteParsedDocumentSourceDefault> parse(const AggregationRequest& request,
+ const BSONElement& spec) {
+ return stdx::make_unique<LiteParsedDocumentSourceDefault>();
+ }
+
+ LiteParsedDocumentSourceDefault() = default;
+
+ stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const final {
+ return stdx::unordered_set<NamespaceString>();
+ }
+};
+
+/**
+ * Helper class for DocumentSources which work with exactly one foreign collection to register as
+ * their lite parser.
+ */
+class LiteParsedDocumentSourceOneForeignCollection : public LiteParsedDocumentSource {
+public:
+ explicit LiteParsedDocumentSourceOneForeignCollection(NamespaceString foreignNss)
+ : _foreignNss(std::move(foreignNss)) {}
+
+ stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const final {
+ return {_foreignNss};
+ }
+
+private:
+ NamespaceString _foreignNss;
+};
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/lite_parsed_pipeline.h b/src/mongo/db/pipeline/lite_parsed_pipeline.h
new file mode 100644
index 00000000000..c39d9f1031e
--- /dev/null
+++ b/src/mongo/db/pipeline/lite_parsed_pipeline.h
@@ -0,0 +1,85 @@
+/**
+ * Copyright (C) 2016 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 <memory>
+#include <vector>
+
+#include "mongo/bson/bsonobj.h"
+#include "mongo/db/namespace_string.h"
+#include "mongo/db/pipeline/aggregation_request.h"
+#include "mongo/db/pipeline/lite_parsed_document_source.h"
+
+namespace mongo {
+
+/**
+ * A semi-parsed version of a Pipeline, parsed just enough to determine information like what
+ * foreign collections are involved.
+ */
+class LiteParsedPipeline {
+public:
+ /**
+ * Constructs a LiteParsedPipeline from the raw BSON stages given in 'request'.
+ *
+ * May throw a UserException if there is an invalid stage specification, although full
+ * validation happens later, during Pipeline construction.
+ */
+ LiteParsedPipeline(const AggregationRequest& request) {
+ _stageSpecs.reserve(request.getPipeline().size());
+
+ for (auto&& rawStage : request.getPipeline()) {
+ _stageSpecs.push_back(LiteParsedDocumentSource::parse(request, rawStage));
+ }
+ }
+
+ /**
+ * Returns all foreign namespaces referenced by stages within this pipeline, if any.
+ */
+ stdx::unordered_set<NamespaceString> getInvolvedNamespaces() const {
+ stdx::unordered_set<NamespaceString> involvedNamespaces;
+ for (auto&& spec : _stageSpecs) {
+ auto stagesInvolvedNamespaces = spec->getInvolvedNamespaces();
+ involvedNamespaces.insert(stagesInvolvedNamespaces.begin(),
+ stagesInvolvedNamespaces.end());
+ }
+ return involvedNamespaces;
+ }
+
+ /**
+ * Returns true if the pipeline begins with a $collStats stage.
+ */
+ bool startsWithCollStats() const {
+ return !_stageSpecs.empty() && _stageSpecs.front()->isCollStats();
+ }
+
+private:
+ std::vector<std::unique_ptr<LiteParsedDocumentSource>> _stageSpecs;
+};
+
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/pipeline.h b/src/mongo/db/pipeline/pipeline.h
index 9ea9cb07df7..aff72fa40bc 100644
--- a/src/mongo/db/pipeline/pipeline.h
+++ b/src/mongo/db/pipeline/pipeline.h
@@ -59,6 +59,10 @@ public:
* Parses a Pipeline from a BSONElement representing a list of DocumentSources. Returns a non-OK
* status if it failed to parse. The returned pipeline is not optimized, but the caller may
* convert it to an optimized pipeline by calling optimizePipeline().
+ *
+ * It is illegal to create a pipeline using an ExpressionContext which contains a collation that
+ * will not be used during execution of the pipeline. Doing so may cause comparisons made during
+ * parse-time to return the wrong results.
*/
static StatusWith<boost::intrusive_ptr<Pipeline>> parse(
const std::vector<BSONObj>& rawPipeline,
diff --git a/src/mongo/db/views/view_catalog.cpp b/src/mongo/db/views/view_catalog.cpp
index 237414b4882..37f7e3950f4 100644
--- a/src/mongo/db/views/view_catalog.cpp
+++ b/src/mongo/db/views/view_catalog.cpp
@@ -158,11 +158,11 @@ Status ViewCatalog::_upsertIntoGraph(OperationContext* txn, const ViewDefinition
// Parse the pipeline for this view to get the namespaces it references.
AggregationRequest request(viewDef.viewOn(), viewDef.pipeline());
boost::intrusive_ptr<ExpressionContext> expCtx = new ExpressionContext(txn, request);
+ expCtx->setCollator(CollatorInterface::cloneCollator(viewDef.defaultCollator()));
auto pipelineStatus = Pipeline::parse(viewDef.pipeline(), expCtx);
if (!pipelineStatus.isOK()) {
uassert(40255,
- str::stream() << "Invalid pipeline for existing view " << viewDef.name().ns()
- << "; "
+ str::stream() << "Invalid pipeline for view " << viewDef.name().ns() << "; "
<< pipelineStatus.getStatus().reason(),
!needsValidation);
return pipelineStatus.getStatus();