summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorEric Cox <eric.cox@mongodb.com>2020-01-21 20:31:02 +0000
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2020-02-13 14:58:45 +0000
commite289b78c61033674f6440d9ddc402b50903717ac (patch)
treef468a4ea2a6c68e3bbf3e23522926e05e4f92aa4
parent173e12f195cf456c6f08d82131256d625c1f583f (diff)
downloadmongo-e289b78c61033674f6440d9ddc402b50903717ac.tar.gz
SERVER-45454 Desugar where to expr + function
-rw-r--r--buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_kill_primary.yml5
-rw-r--r--buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_terminate_primary.yml5
-rw-r--r--jstests/disk/killall.js4
-rw-r--r--jstests/noPassthrough/socket_disconnect_kills.js3
-rw-r--r--src/mongo/db/auth/authz_manager_external_state_mock.cpp6
-rw-r--r--src/mongo/db/auth/role_graph_update.cpp5
-rw-r--r--src/mongo/db/catalog/collection_impl.cpp3
-rw-r--r--src/mongo/db/catalog/database_impl.cpp2
-rw-r--r--src/mongo/db/catalog/index_catalog_entry_impl.cpp2
-rw-r--r--src/mongo/db/catalog/index_catalog_impl.cpp2
-rw-r--r--src/mongo/db/catalog/index_key_validate.cpp6
-rw-r--r--src/mongo/db/commands/list_collections.cpp3
-rw-r--r--src/mongo/db/commands/list_databases.cpp4
-rw-r--r--src/mongo/db/exec/sort_test.cpp5
-rw-r--r--src/mongo/db/exec/stagedebug_cmd.cpp22
-rw-r--r--src/mongo/db/index/wildcard_key_generator.cpp7
-rw-r--r--src/mongo/db/matcher/SConscript1
-rw-r--r--src/mongo/db/matcher/expression_parser.cpp6
-rw-r--r--src/mongo/db/matcher/expression_where.cpp125
-rw-r--r--src/mongo/db/matcher/expression_where.h56
-rw-r--r--src/mongo/db/matcher/extensions_callback.h4
-rw-r--r--src/mongo/db/matcher/extensions_callback_noop.cpp3
-rw-r--r--src/mongo/db/matcher/extensions_callback_noop.h3
-rw-r--r--src/mongo/db/matcher/extensions_callback_real.cpp29
-rw-r--r--src/mongo/db/matcher/extensions_callback_real.h3
-rw-r--r--src/mongo/db/ops/parsed_update.cpp12
-rw-r--r--src/mongo/db/ops/parsed_update.h3
-rw-r--r--src/mongo/db/ops/update.cpp3
-rw-r--r--src/mongo/db/ops/update.h2
-rw-r--r--src/mongo/db/pipeline/SConscript2
-rw-r--r--src/mongo/db/pipeline/aggregation_context_fixture.h2
-rw-r--r--src/mongo/db/pipeline/document_source_exchange_test.cpp17
-rw-r--r--src/mongo/db/pipeline/document_source_merge_cursors_test.cpp3
-rw-r--r--src/mongo/db/pipeline/expression_context.cpp4
-rw-r--r--src/mongo/db/pipeline/expression_context.h6
-rw-r--r--src/mongo/db/pipeline/expression_function.cpp46
-rw-r--r--src/mongo/db/pipeline/expression_function.h20
-rw-r--r--src/mongo/db/pipeline/expression_js_emit.cpp1
-rw-r--r--src/mongo/db/pipeline/expression_js_emit.h1
-rw-r--r--src/mongo/db/pipeline/javascript_execution.cpp17
-rw-r--r--src/mongo/db/pipeline/javascript_execution.h17
-rw-r--r--src/mongo/db/query/canonical_query.cpp3
-rw-r--r--src/mongo/db/query/find.cpp2
-rw-r--r--src/mongo/db/query/projection_test.cpp8
-rw-r--r--src/mongo/db/query/query_knobs.idl8
-rw-r--r--src/mongo/db/query/query_planner_test_lib.cpp4
-rw-r--r--src/mongo/db/query/query_request.h1
-rw-r--r--src/mongo/db/query/query_solution_test.cpp3
-rw-r--r--src/mongo/db/repl/apply_ops.cpp2
-rw-r--r--src/mongo/db/repl/oplog_fetcher.cpp4
-rw-r--r--src/mongo/db/transaction_participant.cpp3
-rw-r--r--src/mongo/db/update/update_driver_test.cpp6
-rw-r--r--src/mongo/dbtests/extensions_callback_real_test.cpp73
-rw-r--r--src/mongo/dbtests/matchertests.cpp4
-rw-r--r--src/mongo/dbtests/query_stage_collscan.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_count.cpp3
-rw-r--r--src/mongo/dbtests/query_stage_ensure_sorted.cpp7
-rw-r--r--src/mongo/dbtests/query_stage_fetch.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_multiplan.cpp3
-rw-r--r--src/mongo/dbtests/query_stage_sort.cpp3
-rw-r--r--src/mongo/dbtests/query_stage_sort_key_generator.cpp8
-rw-r--r--src/mongo/dbtests/query_stage_tests.cpp2
-rw-r--r--src/mongo/dbtests/query_stage_update.cpp8
-rw-r--r--src/mongo/embedded/stitch_support/stitch_support.cpp20
-rw-r--r--src/mongo/s/commands/cluster_clear_jumbo_flag_cmd.cpp5
-rw-r--r--src/mongo/s/commands/cluster_find_and_modify_cmd.cpp11
-rw-r--r--src/mongo/s/commands/cluster_move_chunk_cmd.cpp4
-rw-r--r--src/mongo/s/commands/cluster_split_cmd.cpp4
-rw-r--r--src/mongo/s/query/cluster_aggregate.cpp2
-rw-r--r--src/mongo/s/shard_key_pattern.cpp3
-rw-r--r--src/mongo/s/shard_key_pattern.h1
-rw-r--r--src/mongo/s/shard_key_pattern_test.cpp3
-rw-r--r--src/mongo/s/write_ops/chunk_manager_targeter.cpp14
-rw-r--r--src/mongo/scripting/mozjs/implscope.cpp1
74 files changed, 325 insertions, 372 deletions
diff --git a/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_kill_primary.yml b/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_kill_primary.yml
index f67056c0324..f6ee99674d0 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_kill_primary.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_kill_primary.yml
@@ -42,6 +42,11 @@ selector:
- jstests/concurrency/fsm_workloads/map_reduce_replace_remove.js
- jstests/concurrency/fsm_workloads/map_reduce_with_chunk_migrations.js
+ # Disabled due to thread local scope used in desugaring $where doesn't handle
+ # maxTimeMS timeouts correctly.
+ # TODO SERVER-46098: Remove when MaxTimeMs handling has been fixed for find with $where.
+ - jstests/concurrency/fsm_workloads/server_status_with_time_out_cursors.js
+
# Disabled due to MongoDB restrictions and/or workload restrictions
# These workloads sometimes trigger 'Could not lock auth data update lock'
diff --git a/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_terminate_primary.yml b/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_terminate_primary.yml
index 99a7b5cca00..6cb509bae6a 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_terminate_primary.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_terminate_primary.yml
@@ -42,6 +42,11 @@ selector:
- jstests/concurrency/fsm_workloads/map_reduce_replace_remove.js
- jstests/concurrency/fsm_workloads/map_reduce_with_chunk_migrations.js
+ # Disabled due to thread local scope used in desugaring $where doesn't handle
+ # maxTimeMS timeouts correctly.
+ # TODO SERVER-46098: Remove when MaxTimeMs handling has been fixed for find with $where.
+ - jstests/concurrency/fsm_workloads/server_status_with_time_out_cursors.js
+
# Disabled due to MongoDB restrictions and/or workload restrictions
# These workloads sometimes trigger 'Could not lock auth data update lock'
diff --git a/jstests/disk/killall.js b/jstests/disk/killall.js
index e8bb7ee71b5..5744c15c23b 100644
--- a/jstests/disk/killall.js
+++ b/jstests/disk/killall.js
@@ -14,6 +14,10 @@ var db = mongod.getDB("test");
var collection = db.getCollection(baseName);
assert.commandWorked(collection.insert({}));
+// set timeout for js function execution to 100 ms to speed up the test.
+assert.commandWorked(
+ db.adminCommand({setParameter: 1, internalQueryJavaScriptFnTimeoutMillis: 100}));
+
var awaitShell = startParallelShell(
"db." + baseName + ".count( { $where: function() { while( 1 ) { ; } } } )", mongod.port);
sleep(1000);
diff --git a/jstests/noPassthrough/socket_disconnect_kills.js b/jstests/noPassthrough/socket_disconnect_kills.js
index ec6c613cd1c..7876a73ccd0 100644
--- a/jstests/noPassthrough/socket_disconnect_kills.js
+++ b/jstests/noPassthrough/socket_disconnect_kills.js
@@ -129,6 +129,9 @@ function runCommand(cmd) {
function runTests(client) {
let admin = client.getDB("admin");
+ // set timeout for js function execution to 100 ms to speed up tests that run inf loop.
+ assert.commandWorked(client.getDB(testName).adminCommand(
+ {setParameter: 1, internalQueryJavaScriptFnTimeoutMillis: 100}));
assert.commandWorked(client.getDB(testName).test.insert({x: 1}));
assert.commandWorked(client.getDB(testName).test.insert({x: 2}));
assert.commandWorked(client.getDB(testName).test.insert({x: 3}));
diff --git a/src/mongo/db/auth/authz_manager_external_state_mock.cpp b/src/mongo/db/auth/authz_manager_external_state_mock.cpp
index 575985de12c..521ba897fda 100644
--- a/src/mongo/db/auth/authz_manager_external_state_mock.cpp
+++ b/src/mongo/db/auth/authz_manager_external_state_mock.cpp
@@ -187,7 +187,8 @@ Status AuthzManagerExternalStateMock::updateOne(OperationContext* opCtx,
const BSONObj& writeConcern) {
namespace mmb = mutablebson;
const CollatorInterface* collator = nullptr;
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, collator, collectionName));
UpdateDriver driver(std::move(expCtx));
std::map<StringData, std::unique_ptr<ExpressionWithPlaceholder>> arrayFilters;
driver.parse(updatePattern, arrayFilters);
@@ -299,7 +300,8 @@ Status AuthzManagerExternalStateMock::_queryVector(
const BSONObj& query,
std::vector<BSONObjCollection::iterator>* result) {
const CollatorInterface* collator = nullptr;
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, collator, collectionName));
StatusWithMatchExpression parseResult = MatchExpressionParser::parse(query, std::move(expCtx));
if (!parseResult.isOK()) {
return parseResult.getStatus();
diff --git a/src/mongo/db/auth/role_graph_update.cpp b/src/mongo/db/auth/role_graph_update.cpp
index 59d4f5dad7e..0307f4ef187 100644
--- a/src/mongo/db/auth/role_graph_update.cpp
+++ b/src/mongo/db/auth/role_graph_update.cpp
@@ -194,6 +194,7 @@ Status handleOplogInsert(RoleGraph* roleGraph, const BSONObj& insertedObj) {
*/
Status handleOplogUpdate(OperationContext* opCtx,
RoleGraph* roleGraph,
+ const NamespaceString& nss,
const BSONObj& updatePattern,
const BSONObj& queryPattern) {
RoleName roleToUpdate;
@@ -201,7 +202,7 @@ Status handleOplogUpdate(OperationContext* opCtx,
if (!status.isOK())
return status;
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, nullptr));
+ boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, nullptr, nss));
UpdateDriver driver(std::move(expCtx));
driver.setFromOplogApplication(true);
@@ -391,7 +392,7 @@ Status RoleGraph::handleLogOp(OperationContext* opCtx,
return Status(ErrorCodes::InternalError,
"Missing query pattern in update oplog entry.");
}
- return handleOplogUpdate(opCtx, this, o, *o2);
+ return handleOplogUpdate(opCtx, this, ns, o, *o2);
case 'd':
return handleOplogDelete(this, o);
case 'n':
diff --git a/src/mongo/db/catalog/collection_impl.cpp b/src/mongo/db/catalog/collection_impl.cpp
index 9d845cb28da..ff5a81d6f48 100644
--- a/src/mongo/db/catalog/collection_impl.cpp
+++ b/src/mongo/db/catalog/collection_impl.cpp
@@ -396,7 +396,8 @@ StatusWithMatchExpression CollectionImpl::parseValidator(
return canUseValidatorInThisContext;
}
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, _collator.get()));
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, _collator.get(), ns()));
// The MatchExpression and contained ExpressionContext created as part of the validator are
// owned by the Collection and will outlive the OperationContext they were created under.
diff --git a/src/mongo/db/catalog/database_impl.cpp b/src/mongo/db/catalog/database_impl.cpp
index c90c6c8fc37..78ce8314d82 100644
--- a/src/mongo/db/catalog/database_impl.cpp
+++ b/src/mongo/db/catalog/database_impl.cpp
@@ -838,7 +838,7 @@ Status DatabaseImpl::userCreateNS(OperationContext* opCtx,
if (!collectionOptions.validator.isEmpty()) {
boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx, collator.get()));
+ new ExpressionContext(opCtx, collator.get(), nss));
// Save this to a variable to avoid reading the atomic variable multiple times.
const auto currentFCV = serverGlobalParams.featureCompatibility.getVersion();
diff --git a/src/mongo/db/catalog/index_catalog_entry_impl.cpp b/src/mongo/db/catalog/index_catalog_entry_impl.cpp
index d3084459f69..3c208009adc 100644
--- a/src/mongo/db/catalog/index_catalog_entry_impl.cpp
+++ b/src/mongo/db/catalog/index_catalog_entry_impl.cpp
@@ -98,7 +98,7 @@ IndexCatalogEntryImpl::IndexCatalogEntryImpl(OperationContext* const opCtx,
const BSONObj& filter = _descriptor->partialFilterExpression();
boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx, _collator.get()));
+ new ExpressionContext(opCtx, _collator.get(), ns()));
// Parsing the partial filter expression is not expected to fail here since the
// expression would have been successfully parsed upstream during index creation.
diff --git a/src/mongo/db/catalog/index_catalog_impl.cpp b/src/mongo/db/catalog/index_catalog_impl.cpp
index 8ca12e22356..d1ac7fea2d0 100644
--- a/src/mongo/db/catalog/index_catalog_impl.cpp
+++ b/src/mongo/db/catalog/index_catalog_impl.cpp
@@ -655,7 +655,7 @@ Status IndexCatalogImpl::_isSpecOk(OperationContext* opCtx, const BSONObj& spec)
// The collator must outlive the constructed MatchExpression.
boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx, collator.get()));
+ new ExpressionContext(opCtx, collator.get(), nss));
// Parsing the partial filter expression is not expected to fail here since the
// expression would have been successfully parsed upstream during index creation.
diff --git a/src/mongo/db/catalog/index_key_validate.cpp b/src/mongo/db/catalog/index_key_validate.cpp
index 2c45fc02389..e107e000196 100644
--- a/src/mongo/db/catalog/index_key_validate.cpp
+++ b/src/mongo/db/catalog/index_key_validate.cpp
@@ -383,10 +383,12 @@ StatusWith<BSONObj> validateIndexSpec(
// specified or may inherit the default collation from the collection. It's legal to
// parse with the wrong collation, since the collation can be set on a MatchExpression
// after the fact. Here, we don't bother checking the collation after the fact, since
- // this invocation of the parser is just for validity checking.
+ // this invocation of the parser is just for validity checking. It's also legal to parse
+ // with an empty namespace string, because we are only doing validity checking and not
+ // resolving the expression against a given namespace.
auto simpleCollator = nullptr;
boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx, simpleCollator));
+ new ExpressionContext(opCtx, simpleCollator, NamespaceString()));
// Special match expression features (e.g. $jsonSchema, $expr, ...) are not allowed in a
// partialFilterExpression on index creation.
diff --git a/src/mongo/db/commands/list_collections.cpp b/src/mongo/db/commands/list_collections.cpp
index 7e2e90c8d86..d3841cbaed8 100644
--- a/src/mongo/db/commands/list_collections.cpp
+++ b/src/mongo/db/commands/list_collections.cpp
@@ -260,7 +260,8 @@ public:
}
// The collator is null because collection objects are compared using binary comparison.
const CollatorInterface* collator = nullptr;
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, collator, NamespaceString(StringData(dbname))));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(filterElt.Obj(), std::move(expCtx));
uassertStatusOK(statusWithMatcher.getStatus());
diff --git a/src/mongo/db/commands/list_databases.cpp b/src/mongo/db/commands/list_databases.cpp
index 2c34bb715c7..19ebc60b933 100644
--- a/src/mongo/db/commands/list_databases.cpp
+++ b/src/mongo/db/commands/list_databases.cpp
@@ -37,6 +37,7 @@
#include "mongo/db/concurrency/write_conflict_exception.h"
#include "mongo/db/curop_failpoint_helpers.h"
#include "mongo/db/matcher/expression.h"
+#include "mongo/db/namespace_string.h"
#include "mongo/db/operation_context.h"
#include "mongo/db/service_context.h"
#include "mongo/db/storage/storage_engine.h"
@@ -123,7 +124,8 @@ public:
// The collator is null because database metadata objects are compared using simple
// binary comparison.
const CollatorInterface* collator = nullptr;
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, collator, NamespaceString(dbname)));
auto matcher =
uassertStatusOK(MatchExpressionParser::parse(filterObj.get(), std::move(expCtx)));
filter = std::move(matcher);
diff --git a/src/mongo/db/exec/sort_test.cpp b/src/mongo/db/exec/sort_test.cpp
index 963f21f0b04..455b038dc3a 100644
--- a/src/mongo/db/exec/sort_test.cpp
+++ b/src/mongo/db/exec/sort_test.cpp
@@ -104,7 +104,8 @@ public:
auto sortPattern = fromjson(patternStr);
// Create an ExpressionContext for the SortKeyGeneratorStage.
- auto expCtx = make_intrusive<ExpressionContext>(getOpCtx(), collator);
+ auto expCtx =
+ make_intrusive<ExpressionContext>(getOpCtx(), collator, NamespaceString("foo"));
auto sortKeyGen = std::make_unique<SortKeyGeneratorStage>(
expCtx, std::move(queuedDataStage), &ws, sortPattern);
@@ -168,7 +169,7 @@ TEST_F(SortStageDefaultTest, SortEmptyWorkingSet) {
WorkingSet ws;
// Create an ExpressionContext for the SortKeyGeneratorStage.
- auto expCtx = make_intrusive<ExpressionContext>(getOpCtx(), nullptr);
+ auto expCtx = make_intrusive<ExpressionContext>(getOpCtx(), nullptr, NamespaceString("foo"));
// QueuedDataStage will be owned by SortStageDefault.
auto queuedDataStage = std::make_unique<QueuedDataStage>(getOpCtx(), &ws);
diff --git a/src/mongo/db/exec/stagedebug_cmd.cpp b/src/mongo/db/exec/stagedebug_cmd.cpp
index 50bbb7e624b..dfa5052b1d3 100644
--- a/src/mongo/db/exec/stagedebug_cmd.cpp
+++ b/src/mongo/db/exec/stagedebug_cmd.cpp
@@ -61,6 +61,7 @@
#include "mongo/db/matcher/expression_parser.h"
#include "mongo/db/matcher/expression_text_base.h"
#include "mongo/db/matcher/extensions_callback_real.h"
+#include "mongo/db/namespace_string.h"
#include "mongo/db/query/plan_executor.h"
#include "mongo/util/log.h"
@@ -171,7 +172,7 @@ public:
unique_ptr<WorkingSet> ws(new WorkingSet());
std::unique_ptr<PlanStage> userRoot{
- parseQuery(opCtx, collection, planObj, ws.get(), &exprs)};
+ parseQuery(opCtx, collection, planObj, ws.get(), nss, &exprs)};
uassert(16911, "Couldn't parse plan from " + cmdObj.toString(), nullptr != userRoot);
// Add a fetch at the top for the user so we can get obj back for sure.
@@ -208,6 +209,7 @@ public:
Collection* collection,
BSONObj obj,
WorkingSet* workingSet,
+ const NamespaceString& nss,
std::vector<std::unique_ptr<MatchExpression>>* exprs) {
BSONElement firstElt = obj.firstElement();
if (!firstElt.isABSONObj()) {
@@ -232,7 +234,7 @@ public:
if (filterTag == e.fieldName()) {
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx, collator));
+ new ExpressionContext(opCtx, collator, nss));
auto statusWithMatcher =
MatchExpressionParser::parse(argObj,
expCtx,
@@ -308,7 +310,7 @@ public:
uassert(16922, "node of AND isn't an obj?: " + e.toString(), e.isABSONObj());
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, e.Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, e.Obj(), workingSet, nss, exprs)};
uassert(16923,
"Can't parse sub-node of AND: " + e.Obj().toString(),
nullptr != subNode);
@@ -332,7 +334,7 @@ public:
uassert(16925, "node of AND isn't an obj?: " + e.toString(), e.isABSONObj());
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, e.Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, e.Obj(), workingSet, nss, exprs)};
uassert(16926,
"Can't parse sub-node of AND: " + e.Obj().toString(),
nullptr != subNode);
@@ -356,7 +358,7 @@ public:
return nullptr;
}
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, e.Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, e.Obj(), workingSet, nss, exprs)};
uassert(
16936, "Can't parse sub-node of OR: " + e.Obj().toString(), nullptr != subNode);
orStage->addChild(std::move(subNode));
@@ -367,7 +369,7 @@ public:
uassert(
16929, "Node argument must be provided to fetch", nodeArgs["node"].isABSONObj());
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, nss, exprs)};
uassert(28731,
"Can't parse sub-node of FETCH: " + nodeArgs["node"].Obj().toString(),
nullptr != subNode);
@@ -380,7 +382,7 @@ public:
16930, "Node argument must be provided to limit", nodeArgs["node"].isABSONObj());
uassert(16931, "Num argument must be provided to limit", nodeArgs["num"].isNumber());
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, nss, exprs)};
uassert(28732,
"Can't parse sub-node of LIMIT: " + nodeArgs["node"].Obj().toString(),
nullptr != subNode);
@@ -393,7 +395,7 @@ public:
uassert(16932, "Node argument must be provided to skip", nodeArgs["node"].isABSONObj());
uassert(16933, "Num argument must be provided to skip", nodeArgs["num"].isNumber());
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, nss, exprs)};
uassert(28733,
"Can't parse sub-node of SKIP: " + nodeArgs["node"].Obj().toString(),
nullptr != subNode);
@@ -432,7 +434,7 @@ public:
uassert(16973, "node of mergeSort isn't an obj?: " + e.toString(), e.isABSONObj());
std::unique_ptr<PlanStage> subNode{
- parseQuery(opCtx, collection, e.Obj(), workingSet, exprs)};
+ parseQuery(opCtx, collection, e.Obj(), workingSet, nss, exprs)};
uassert(16974,
"Can't parse sub-node of mergeSort: " + e.Obj().toString(),
nullptr != subNode);
@@ -481,7 +483,7 @@ public:
"isMulti argument must be provided to delete",
nodeArgs["isMulti"].type() == Bool);
PlanStage* subNode =
- parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, exprs);
+ parseQuery(opCtx, collection, nodeArgs["node"].Obj(), workingSet, nss, exprs);
uassert(28734,
"Can't parse sub-node of DELETE: " + nodeArgs["node"].Obj().toString(),
nullptr != subNode);
diff --git a/src/mongo/db/index/wildcard_key_generator.cpp b/src/mongo/db/index/wildcard_key_generator.cpp
index 831cd776a15..70ab60805c3 100644
--- a/src/mongo/db/index/wildcard_key_generator.cpp
+++ b/src/mongo/db/index/wildcard_key_generator.cpp
@@ -86,9 +86,10 @@ WildcardProjection WildcardKeyGenerator::createProjectionExecutor(BSONObj keyPat
: pathProjection.isEmpty() ? kDefaultProjection : pathProjection);
// Construct a dummy ExpressionContext for ProjectionExecutor. It's OK to set the
- // ExpressionContext's OperationContext and CollatorInterface to 'nullptr' here; since we
- // ban computed fields from the projection, the ExpressionContext will never be used.
- auto expCtx = make_intrusive<ExpressionContext>(nullptr, nullptr);
+ // ExpressionContext's OperationContext and CollatorInterface to 'nullptr' and the namespace
+ // string to '' here; since we ban computed fields from the projection, the ExpressionContext
+ // will never be used.
+ auto expCtx = make_intrusive<ExpressionContext>(nullptr, nullptr, NamespaceString());
auto policies = ProjectionPolicies::wildcardIndexSpecProjectionPolicies();
auto projection = projection_ast::parse(expCtx, projSpec, policies);
return WildcardProjection{projection_executor::buildProjectionExecutor(
diff --git a/src/mongo/db/matcher/SConscript b/src/mongo/db/matcher/SConscript
index 5e6d5c6a360..07ce6a58bcd 100644
--- a/src/mongo/db/matcher/SConscript
+++ b/src/mongo/db/matcher/SConscript
@@ -80,7 +80,6 @@ env.Library(
source=[
'extensions_callback_real.cpp',
'expression_text.cpp',
- 'expression_where.cpp',
],
LIBDEPS=[
'$BUILD_DIR/mongo/db/concurrency/lock_manager',
diff --git a/src/mongo/db/matcher/expression_parser.cpp b/src/mongo/db/matcher/expression_parser.cpp
index 327d00262fe..a06e07b82d7 100644
--- a/src/mongo/db/matcher/expression_parser.cpp
+++ b/src/mongo/db/matcher/expression_parser.cpp
@@ -338,8 +338,12 @@ StatusWithMatchExpression parseWhere(StringData name,
if ((allowedFeatures & MatchExpressionParser::AllowedFeatures::kJavascript) == 0u) {
return {Status(ErrorCodes::BadValue, "$where is not allowed in this context")};
}
+ if (currentLevel == DocumentParseLevel::kUserSubDocument) {
+ return {
+ Status(ErrorCodes::BadValue, "$where can only be applied to the top-level document")};
+ }
- return extensionsCallback->parseWhere(elem);
+ return extensionsCallback->parseWhere(expCtx, elem);
}
StatusWithMatchExpression parseText(StringData name,
diff --git a/src/mongo/db/matcher/expression_where.cpp b/src/mongo/db/matcher/expression_where.cpp
deleted file mode 100644
index 20a2c4bda9d..00000000000
--- a/src/mongo/db/matcher/expression_where.cpp
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Copyright (C) 2018-present MongoDB, Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the Server Side Public License, version 1,
- * as published by MongoDB, Inc.
- *
- * 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
- * Server Side Public License for more details.
- *
- * You should have received a copy of the Server Side Public License
- * along with this program. If not, see
- * <http://www.mongodb.com/licensing/server-side-public-license>.
- *
- * 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 Server Side 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/matcher/expression_where.h"
-
-#include <memory>
-
-#include "mongo/base/init.h"
-#include "mongo/db/auth/authorization_session.h"
-#include "mongo/db/client.h"
-#include "mongo/db/jsobj.h"
-#include "mongo/db/matcher/expression.h"
-#include "mongo/db/matcher/expression_parser.h"
-#include "mongo/db/namespace_string.h"
-#include "mongo/scripting/engine.h"
-#include "mongo/util/scopeguard.h"
-
-
-namespace mongo {
-
-using std::string;
-using std::stringstream;
-using std::unique_ptr;
-
-namespace {
-std::string getAuthenticatedUserNamesToken(Client* client) {
- StringBuilder sb;
-
- auto as = AuthorizationSession::get(client);
- for (auto nameIter = as->getAuthenticatedUserNames(); nameIter.more(); nameIter.next()) {
- // Using a NUL byte which isn't valid in usernames to separate them.
- sb << '\0' << nameIter->getUnambiguousName();
- }
-
- return sb.str();
-}
-} // namespace
-
-WhereMatchExpression::WhereMatchExpression(OperationContext* opCtx,
- WhereParams params,
- StringData dbName)
- : WhereMatchExpressionBase(std::move(params)), _dbName(dbName.toString()), _opCtx(opCtx) {
- invariant(_opCtx != nullptr);
-
- uassert(
- ErrorCodes::BadValue, "no globalScriptEngine in $where parsing", getGlobalScriptEngine());
-
- uassert(ErrorCodes::BadValue, "ns for $where cannot be empty", dbName.size() != 0);
-
- const auto userToken = getAuthenticatedUserNamesToken(opCtx->getClient());
- _scope = getGlobalScriptEngine()->getPooledScope(_opCtx, _dbName, "where" + userToken);
- const auto guard = makeGuard([&] { _scope->unregisterOperation(); });
-
- _func = _scope->createFunction(getCode().c_str());
-
- uassert(ErrorCodes::BadValue, "$where compile error", _func);
-}
-
-bool WhereMatchExpression::matches(const MatchableDocument* doc, MatchDetails* details) const {
- uassert(28692, "$where compile error", _func);
- BSONObj obj = doc->toBSON();
-
- _scope->registerOperation(Client::getCurrent()->getOperationContext());
- const auto guard = makeGuard([&] { _scope->unregisterOperation(); });
-
- if (!getScope().isEmpty()) {
- _scope->init(&getScope());
- }
-
- _scope->advanceGeneration();
- _scope->setObject("obj", const_cast<BSONObj&>(obj));
- _scope->setBoolean("fullObject", true); // this is a hack b/c fullObject used to be relevant
-
- int err = _scope->invoke(_func, nullptr, &obj, 1000 * 60, false);
- if (err == -3) { // INVOKE_ERROR
- stringstream ss;
- ss << "error on invocation of $where function:\n" << _scope->getError();
- uassert(16812, ss.str(), false);
- } else if (err != 0) { // ! INVOKE_SUCCESS
- uassert(16813, "unknown error in invocation of $where function", false);
- }
-
- return _scope->getBoolean("__returnValue") != 0;
-}
-
-unique_ptr<MatchExpression> WhereMatchExpression::shallowClone() const {
- WhereParams params;
- params.code = getCode();
- params.scope = getScope();
- unique_ptr<WhereMatchExpression> e =
- std::make_unique<WhereMatchExpression>(_opCtx, std::move(params), _dbName);
- if (getTag()) {
- e->setTag(getTag()->clone());
- }
- return std::move(e);
-}
-} // namespace mongo
diff --git a/src/mongo/db/matcher/expression_where.h b/src/mongo/db/matcher/expression_where.h
deleted file mode 100644
index dfe7ee809ea..00000000000
--- a/src/mongo/db/matcher/expression_where.h
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Copyright (C) 2018-present MongoDB, Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the Server Side Public License, version 1,
- * as published by MongoDB, Inc.
- *
- * 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
- * Server Side Public License for more details.
- *
- * You should have received a copy of the Server Side Public License
- * along with this program. If not, see
- * <http://www.mongodb.com/licensing/server-side-public-license>.
- *
- * 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 Server Side 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/matcher/expression_where_base.h"
-#include "mongo/scripting/engine.h"
-
-namespace mongo {
-
-class OperationContext;
-
-class WhereMatchExpression final : public WhereMatchExpressionBase {
-public:
- WhereMatchExpression(OperationContext* opCtx, WhereParams params, StringData dbName);
-
- bool matches(const MatchableDocument* doc, MatchDetails* details = nullptr) const final;
-
- std::unique_ptr<MatchExpression> shallowClone() const final;
-
-private:
- std::string _dbName;
-
- std::unique_ptr<Scope> _scope;
- ScriptingFunction _func;
-
- OperationContext* const _opCtx;
-};
-
-} // namespace mongo
diff --git a/src/mongo/db/matcher/extensions_callback.h b/src/mongo/db/matcher/extensions_callback.h
index 17b1583e181..8bef3703234 100644
--- a/src/mongo/db/matcher/extensions_callback.h
+++ b/src/mongo/db/matcher/extensions_callback.h
@@ -32,6 +32,7 @@
#include "mongo/db/matcher/expression.h"
#include "mongo/db/matcher/expression_text_base.h"
#include "mongo/db/matcher/expression_where_base.h"
+#include "mongo/db/pipeline/expression_context.h"
namespace mongo {
@@ -45,7 +46,8 @@ public:
virtual StatusWithMatchExpression parseText(BSONElement text) const = 0;
- virtual StatusWithMatchExpression parseWhere(BSONElement where) const = 0;
+ virtual StatusWithMatchExpression parseWhere(
+ const boost::intrusive_ptr<ExpressionContext>& expCtx, BSONElement where) const = 0;
/**
* Returns true if extensions (e.g. $text and $where) are allowed but are converted into no-ops.
diff --git a/src/mongo/db/matcher/extensions_callback_noop.cpp b/src/mongo/db/matcher/extensions_callback_noop.cpp
index c4ce0f8f9b2..3380144736a 100644
--- a/src/mongo/db/matcher/extensions_callback_noop.cpp
+++ b/src/mongo/db/matcher/extensions_callback_noop.cpp
@@ -47,7 +47,8 @@ StatusWithMatchExpression ExtensionsCallbackNoop::parseText(BSONElement text) co
return {std::move(expr)};
}
-StatusWithMatchExpression ExtensionsCallbackNoop::parseWhere(BSONElement where) const {
+StatusWithMatchExpression ExtensionsCallbackNoop::parseWhere(
+ const boost::intrusive_ptr<ExpressionContext>& expCtx, BSONElement where) const {
auto whereParams = extractWhereMatchExpressionParams(where);
if (!whereParams.isOK()) {
return whereParams.getStatus();
diff --git a/src/mongo/db/matcher/extensions_callback_noop.h b/src/mongo/db/matcher/extensions_callback_noop.h
index 358674d472f..cecacb45c0d 100644
--- a/src/mongo/db/matcher/extensions_callback_noop.h
+++ b/src/mongo/db/matcher/extensions_callback_noop.h
@@ -48,7 +48,8 @@ public:
/**
* Returns a WhereNoOpMatchExpression, or an error Status if parsing fails.
*/
- StatusWithMatchExpression parseWhere(BSONElement where) const final;
+ StatusWithMatchExpression parseWhere(const boost::intrusive_ptr<ExpressionContext>& expCtx,
+ BSONElement where) const final;
bool hasNoopExtensions() const final {
return true;
diff --git a/src/mongo/db/matcher/extensions_callback_real.cpp b/src/mongo/db/matcher/extensions_callback_real.cpp
index 313602e4752..796715bdbca 100644
--- a/src/mongo/db/matcher/extensions_callback_real.cpp
+++ b/src/mongo/db/matcher/extensions_callback_real.cpp
@@ -31,9 +31,12 @@
#include "mongo/db/matcher/extensions_callback_real.h"
+#include "mongo/db/matcher/expression_expr.h"
#include "mongo/db/matcher/expression_text.h"
-#include "mongo/db/matcher/expression_where.h"
#include "mongo/db/namespace_string.h"
+#include "mongo/db/pipeline/expression_function.h"
+#include "mongo/db/query/util/make_data_structure.h"
+#include "mongo/scripting/engine.h"
namespace mongo {
@@ -52,15 +55,29 @@ StatusWithMatchExpression ExtensionsCallbackReal::parseText(BSONElement text) co
return {std::move(exp)};
}
-StatusWithMatchExpression ExtensionsCallbackReal::parseWhere(BSONElement where) const {
+StatusWithMatchExpression ExtensionsCallbackReal::parseWhere(
+ const boost::intrusive_ptr<ExpressionContext>& expCtx, BSONElement where) const {
+
auto whereParams = extractWhereMatchExpressionParams(where);
if (!whereParams.isOK()) {
return whereParams.getStatus();
}
- auto exp = std::make_unique<WhereMatchExpression>(
- _opCtx, std::move(whereParams.getValue()), _nss->db());
- return {std::move(exp)};
-}
+ uassert(ErrorCodes::BadValue, "ns for $where cannot be empty", expCtx->ns.db().size() != 0);
+ auto code = whereParams.getValue().code;
+
+ // Desugar $where to $expr. The $where function is invoked through an $_internalJs expression by
+ // passing the document as $$CURRENT.
+ auto fnExpression = ExpressionFunction::createForWhere(
+ expCtx,
+ ExpressionArray::create(
+ expCtx,
+ make_vector<boost::intrusive_ptr<Expression>>(
+ ExpressionFieldPath::parse(expCtx, "$$CURRENT", expCtx->variablesParseState))),
+ code,
+ ExpressionFunction::kJavaScript);
+
+ return {std::make_unique<ExprMatchExpression>(fnExpression, expCtx)};
+}
} // namespace mongo
diff --git a/src/mongo/db/matcher/extensions_callback_real.h b/src/mongo/db/matcher/extensions_callback_real.h
index 11e12ec2f54..18da858718d 100644
--- a/src/mongo/db/matcher/extensions_callback_real.h
+++ b/src/mongo/db/matcher/extensions_callback_real.h
@@ -60,7 +60,8 @@ public:
/**
* Returns a WhereMatchExpression, or an error Status if parsing fails.
*/
- StatusWithMatchExpression parseWhere(BSONElement where) const final;
+ StatusWithMatchExpression parseWhere(const boost::intrusive_ptr<ExpressionContext>& expCtx,
+ BSONElement where) const final;
private:
OperationContext* const _opCtx;
diff --git a/src/mongo/db/ops/parsed_update.cpp b/src/mongo/db/ops/parsed_update.cpp
index 6c68a621d82..6ea2c5ddfa6 100644
--- a/src/mongo/db/ops/parsed_update.cpp
+++ b/src/mongo/db/ops/parsed_update.cpp
@@ -45,7 +45,8 @@ ParsedUpdate::ParsedUpdate(OperationContext* opCtx,
const ExtensionsCallback& extensionsCallback)
: _opCtx(opCtx),
_request(request),
- _driver(new ExpressionContext(opCtx, nullptr, _request->getRuntimeConstants())),
+ _driver(new ExpressionContext(
+ opCtx, nullptr, _request->getNamespaceString(), _request->getRuntimeConstants())),
_canonicalQuery(),
_extensionsCallback(extensionsCallback) {}
@@ -83,8 +84,8 @@ Status ParsedUpdate::parseRequest() {
_collator = std::move(collator.getValue());
}
- auto statusWithArrayFilters =
- parseArrayFilters(_request->getArrayFilters(), _opCtx, _collator.get());
+ auto statusWithArrayFilters = parseArrayFilters(
+ _request->getArrayFilters(), _opCtx, _collator.get(), _request->getNamespaceString());
if (!statusWithArrayFilters.isOK()) {
return statusWithArrayFilters.getStatus();
}
@@ -177,10 +178,11 @@ void ParsedUpdate::parseUpdate() {
StatusWith<std::map<StringData, std::unique_ptr<ExpressionWithPlaceholder>>>
ParsedUpdate::parseArrayFilters(const std::vector<BSONObj>& rawArrayFiltersIn,
OperationContext* opCtx,
- CollatorInterface* collator) {
+ CollatorInterface* collator,
+ const NamespaceString& nss) {
std::map<StringData, std::unique_ptr<ExpressionWithPlaceholder>> arrayFiltersOut;
for (auto rawArrayFilter : rawArrayFiltersIn) {
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator, nss));
auto parsedArrayFilter =
MatchExpressionParser::parse(rawArrayFilter,
std::move(expCtx),
diff --git a/src/mongo/db/ops/parsed_update.h b/src/mongo/db/ops/parsed_update.h
index 3764b5f85d2..06176336b17 100644
--- a/src/mongo/db/ops/parsed_update.h
+++ b/src/mongo/db/ops/parsed_update.h
@@ -66,7 +66,8 @@ public:
static StatusWith<std::map<StringData, std::unique_ptr<ExpressionWithPlaceholder>>>
parseArrayFilters(const std::vector<BSONObj>& rawArrayFiltersIn,
OperationContext* opCtx,
- CollatorInterface* collator);
+ CollatorInterface* collator,
+ const NamespaceString& nss);
/**
* Constructs a parsed update.
diff --git a/src/mongo/db/ops/update.cpp b/src/mongo/db/ops/update.cpp
index 38cf8eb8c12..e4367d04ac5 100644
--- a/src/mongo/db/ops/update.cpp
+++ b/src/mongo/db/ops/update.cpp
@@ -109,10 +109,11 @@ UpdateResult update(OperationContext* opCtx, Database* db, const UpdateRequest&
}
BSONObj applyUpdateOperators(OperationContext* opCtx,
+ const NamespaceString& nss,
const BSONObj& from,
const BSONObj& operators) {
const CollatorInterface* collator = nullptr;
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator, nss));
UpdateDriver driver(std::move(expCtx));
std::map<StringData, std::unique_ptr<ExpressionWithPlaceholder>> arrayFilters;
driver.parse(operators, arrayFilters);
diff --git a/src/mongo/db/ops/update.h b/src/mongo/db/ops/update.h
index 6dc7601cb8d..3819ba6dc30 100644
--- a/src/mongo/db/ops/update.h
+++ b/src/mongo/db/ops/update.h
@@ -31,6 +31,7 @@
#include "mongo/db/curop.h"
#include "mongo/db/jsobj.h"
+#include "mongo/db/namespace_string.h"
#include "mongo/db/ops/update_request.h"
#include "mongo/db/ops/update_result.h"
@@ -56,6 +57,7 @@ UpdateResult update(OperationContext* opCtx, Database* db, const UpdateRequest&
* returns: { x : 2 }
*/
BSONObj applyUpdateOperators(OperationContext* opCtx,
+ const NamespaceString& ns,
const BSONObj& from,
const BSONObj& operators);
} // namespace mongo
diff --git a/src/mongo/db/pipeline/SConscript b/src/mongo/db/pipeline/SConscript
index a2f6fdd8544..0d8a9ecd390 100644
--- a/src/mongo/db/pipeline/SConscript
+++ b/src/mongo/db/pipeline/SConscript
@@ -64,7 +64,7 @@ env.Library(
LIBDEPS=[
'aggregation_request',
'$BUILD_DIR/mongo/db/query/collation/collator_factory_interface',
- '$BUILD_DIR/mongo/db/query/query_knobs',
+ '$BUILD_DIR/mongo/db/query/query_knobs',
'$BUILD_DIR/mongo/db/service_context',
'$BUILD_DIR/mongo/scripting/scripting',
'$BUILD_DIR/mongo/util/intrusive_counter',
diff --git a/src/mongo/db/pipeline/aggregation_context_fixture.h b/src/mongo/db/pipeline/aggregation_context_fixture.h
index 58a947117cc..df667e631fd 100644
--- a/src/mongo/db/pipeline/aggregation_context_fixture.h
+++ b/src/mongo/db/pipeline/aggregation_context_fixture.h
@@ -51,7 +51,7 @@ public:
TimeZoneDatabase::set(getServiceContext(), std::make_unique<TimeZoneDatabase>());
// Must instantiate ExpressionContext _after_ setting the TZ database on the service
// context.
- _expCtx = new ExpressionContext(_opCtx.get(), nullptr);
+ _expCtx = new ExpressionContext(_opCtx.get(), nullptr, nss);
_expCtx->ns = std::move(nss);
unittest::TempDir tempDir("AggregationContextFixture");
_expCtx->tempDir = tempDir.path();
diff --git a/src/mongo/db/pipeline/document_source_exchange_test.cpp b/src/mongo/db/pipeline/document_source_exchange_test.cpp
index ff517ad4f9c..65ab3f8b4a4 100644
--- a/src/mongo/db/pipeline/document_source_exchange_test.cpp
+++ b/src/mongo/db/pipeline/document_source_exchange_test.cpp
@@ -83,6 +83,8 @@ struct ThreadInfo {
};
} // namespace
+const NamespaceString kTestNss = NamespaceString("test.docSourceExchange"_sd);
+
class DocumentSourceExchangeTest : public AggregationContextFixture {
protected:
std::unique_ptr<executor::TaskExecutor> _executor;
@@ -148,7 +150,8 @@ protected:
threads.emplace_back(ThreadInfo{
std::move(client),
std::move(opCtxOwned),
- new DocumentSourceExchange(new ExpressionContext(opCtx, nullptr), ex, idx, nullptr),
+ new DocumentSourceExchange(
+ new ExpressionContext(opCtx, nullptr, kTestNss), ex, idx, nullptr),
});
}
return threads;
@@ -526,12 +529,12 @@ TEST_F(DocumentSourceExchangeTest, RandomExchangeNConsumerResourceYielding) {
auto yielder = std::make_unique<MutexYielder>(&artificalGlobalMutex);
auto yielderRaw = yielder.get();
- threads.push_back(
- ThreadInfo{std::move(client),
- std::move(opCtxOwned),
- new DocumentSourceExchange(
- new ExpressionContext(opCtx, nullptr), ex, idx, std::move(yielder)),
- yielderRaw});
+ threads.push_back(ThreadInfo{
+ std::move(client),
+ std::move(opCtxOwned),
+ new DocumentSourceExchange(
+ new ExpressionContext(opCtx, nullptr, kTestNss), ex, idx, std::move(yielder)),
+ yielderRaw});
}
std::vector<executor::TaskExecutor::CallbackHandle> handles;
diff --git a/src/mongo/db/pipeline/document_source_merge_cursors_test.cpp b/src/mongo/db/pipeline/document_source_merge_cursors_test.cpp
index 544748319a8..ef6d685983c 100644
--- a/src/mongo/db/pipeline/document_source_merge_cursors_test.cpp
+++ b/src/mongo/db/pipeline/document_source_merge_cursors_test.cpp
@@ -79,8 +79,7 @@ class DocumentSourceMergeCursorsTest : public ShardingTestFixture {
public:
DocumentSourceMergeCursorsTest() {
TimeZoneDatabase::set(getServiceContext(), std::make_unique<TimeZoneDatabase>());
- _expCtx = new ExpressionContext(operationContext(), nullptr);
- _expCtx->ns = kTestNss;
+ _expCtx = new ExpressionContext(operationContext(), nullptr, kTestNss);
}
void setUp() override {
diff --git a/src/mongo/db/pipeline/expression_context.cpp b/src/mongo/db/pipeline/expression_context.cpp
index 2e4908d8f7e..f135b105c35 100644
--- a/src/mongo/db/pipeline/expression_context.cpp
+++ b/src/mongo/db/pipeline/expression_context.cpp
@@ -128,8 +128,10 @@ ExpressionContext::ExpressionContext(
ExpressionContext::ExpressionContext(OperationContext* opCtx,
const CollatorInterface* collator,
+ const NamespaceString& nss,
const boost::optional<RuntimeConstants>& runtimeConstants)
- : opCtx(opCtx),
+ : ns(nss),
+ opCtx(opCtx),
mongoProcessInterface(std::make_shared<StubMongoProcessInterface>()),
timeZoneDatabase(opCtx && opCtx->getServiceContext()
? TimeZoneDatabase::get(opCtx->getServiceContext())
diff --git a/src/mongo/db/pipeline/expression_context.h b/src/mongo/db/pipeline/expression_context.h
index 11d0337c482..7013b813018 100644
--- a/src/mongo/db/pipeline/expression_context.h
+++ b/src/mongo/db/pipeline/expression_context.h
@@ -49,7 +49,6 @@
#include "mongo/db/query/collation/collator_interface.h"
#include "mongo/db/query/datetime/date_time_support.h"
#include "mongo/db/query/explain_options.h"
-#include "mongo/db/query/query_knobs_gen.h"
#include "mongo/db/query/tailable_mode.h"
#include "mongo/db/server_options.h"
#include "mongo/util/intrusive_counter.h"
@@ -138,6 +137,7 @@ public:
*/
ExpressionContext(OperationContext* opCtx,
const CollatorInterface* collator,
+ const NamespaceString& ns,
const boost::optional<RuntimeConstants>& runtimeConstants = boost::none);
/**
@@ -286,6 +286,10 @@ public:
// 'jsHeapLimitMB' server parameter.
boost::optional<int> jsHeapLimitMB;
+ // When set this timeout limits the allowed execution time for a JavaScript function invocation
+ // under any Scope returned by getJsExecWithScope().
+ int jsFnTimeoutMillis;
+
// An interface for accessing information or performing operations that have different
// implementations on mongod and mongos, or that only make sense on one of the two.
// Additionally, putting some of this functionality behind an interface prevents aggregation
diff --git a/src/mongo/db/pipeline/expression_function.cpp b/src/mongo/db/pipeline/expression_function.cpp
index 399a8847e03..c7c64184c8a 100644
--- a/src/mongo/db/pipeline/expression_function.cpp
+++ b/src/mongo/db/pipeline/expression_function.cpp
@@ -38,18 +38,25 @@ REGISTER_EXPRESSION_WITH_MIN_VERSION(
ExpressionFunction::ExpressionFunction(const boost::intrusive_ptr<ExpressionContext>& expCtx,
boost::intrusive_ptr<Expression> passedArgs,
+ bool assignFirstArgToThis,
std::string funcSource,
std::string lang)
: Expression(expCtx, {std::move(passedArgs)}),
_passedArgs(_children[0]),
+ _assignFirstArgToThis(assignFirstArgToThis),
_funcSource(std::move(funcSource)),
_lang(std::move(lang)) {}
Value ExpressionFunction::serialize(bool explain) const {
- return Value(Document{{kExpressionName,
- Document{{"body", _funcSource},
- {"args", _passedArgs->serialize(explain)},
- {"lang", _lang}}}});
+ MutableDocument d;
+ d["body"] = Value(_funcSource);
+ d["args"] = Value(_passedArgs->serialize(explain));
+ d["lang"] = Value(_lang);
+ // This field will only be seralized when desugaring $where in $expr + $_internalJs
+ if (_assignFirstArgToThis) {
+ d["_internalSetObjToThis"] = Value(_assignFirstArgToThis);
+ }
+ return Value(Document{{kExpressionName, d.freezeToValue()}});
}
void ExpressionFunction::_doAddDependencies(mongo::DepsTracker* deps) const {
@@ -84,17 +91,25 @@ boost::intrusive_ptr<Expression> ExpressionFunction::parse(
uassert(31263, "The args field must be specified.", argsField);
boost::intrusive_ptr<Expression> argsExpr = parseOperand(expCtx, argsField, vps);
+ // This element will be present when desugaring $where, only.
+ BSONElement assignFirstArgToThis = expr["_internalSetObjToThis"];
+
BSONElement langField = expr["lang"];
uassert(31418, "The lang field must be specified.", langField);
uassert(31419,
"Currently the only supported language specifier is 'js'.",
langField.type() == BSONType::String && langField.str() == kJavaScript);
- return new ExpressionFunction(expCtx, argsExpr, bodyValue.coerceToString(), langField.str());
+ return new ExpressionFunction(expCtx,
+ argsExpr,
+ assignFirstArgToThis.trueValue(),
+ bodyValue.coerceToString(),
+ langField.str());
}
Value ExpressionFunction::evaluate(const Document& root, Variables* variables) const {
auto jsExec = getExpressionContext()->getJsExecWithScope();
+ auto scope = jsExec->getScope();
ScriptingFunction func = jsExec->getScope()->createFunction(_funcSource.c_str());
uassert(31265, "The body function did not evaluate", func);
@@ -102,11 +117,28 @@ Value ExpressionFunction::evaluate(const Document& root, Variables* variables) c
auto argValue = _passedArgs->evaluate(root, variables);
uassert(31266, "The args field must be of type array", argValue.getType() == BSONType::Array);
- int argNum = 0;
+ // This logic exists to desugar $where into $expr + $function. In this case set the global obj
+ // to this, to handle cases where the $where function references the current document through
+ // obj.
BSONObjBuilder bob;
+ if (_assignFirstArgToThis) {
+ // For defense-in-depth, The $where case will pass a field path expr carrying $$CURRENT as
+ // the only element of the array.
+ auto args = argValue.getArray();
+ uassert(31422,
+ "field path $$CURRENT must be the only element in args",
+ argValue.getArrayLength() == 1);
+
+ BSONObj thisBSON = args[0].getDocument().toBson();
+ scope->setObject("obj", thisBSON);
+
+ return jsExec->callFunction(func, bob.done(), thisBSON);
+ }
+
+ int argNum = 0;
for (const auto& arg : argValue.getArray()) {
arg.addToBsonObj(&bob, "arg" + std::to_string(argNum++));
}
return jsExec->callFunction(func, bob.done(), {});
};
-} // namespace mongo \ No newline at end of file
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/expression_function.h b/src/mongo/db/pipeline/expression_function.h
index 3d8d5e2df7b..d42b0723802 100644
--- a/src/mongo/db/pipeline/expression_function.h
+++ b/src/mongo/db/pipeline/expression_function.h
@@ -50,8 +50,22 @@ public:
boost::intrusive_ptr<Expression> passedArgs,
std::string funcSourceString,
std::string lang) {
+ return new ExpressionFunction{expCtx,
+ passedArgs,
+ false /* don't assign first argument to 'this' */,
+ std::move(funcSourceString),
+ std::move(lang)};
+ }
+
+ // This method is intended for use when you want to bind obj to an argument for desugaring
+ // $where.
+ static boost::intrusive_ptr<ExpressionFunction> createForWhere(
+ const boost::intrusive_ptr<ExpressionContext>& expCtx,
+ boost::intrusive_ptr<Expression> passedArgs,
+ std::string funcSourceString,
+ std::string lang) {
return new ExpressionFunction{
- expCtx, passedArgs, std::move(funcSourceString), std::move(lang)};
+ expCtx, passedArgs, true, std::move(funcSourceString), std::move(lang)};
}
Value evaluate(const Document& root, Variables* variables) const final;
@@ -68,12 +82,14 @@ public:
private:
ExpressionFunction(const boost::intrusive_ptr<ExpressionContext>& expCtx,
boost::intrusive_ptr<Expression> passedArgs,
+ bool assignFirstArgToThis,
std::string funcSourceString,
std::string lang);
void _doAddDependencies(DepsTracker* deps) const final override;
const boost::intrusive_ptr<Expression>& _passedArgs;
+ bool _assignFirstArgToThis;
std::string _funcSource;
std::string _lang;
};
-} // namespace mongo \ No newline at end of file
+} // namespace mongo
diff --git a/src/mongo/db/pipeline/expression_js_emit.cpp b/src/mongo/db/pipeline/expression_js_emit.cpp
index 4072ecee993..1ea8a20db9c 100644
--- a/src/mongo/db/pipeline/expression_js_emit.cpp
+++ b/src/mongo/db/pipeline/expression_js_emit.cpp
@@ -89,7 +89,6 @@ boost::intrusive_ptr<Expression> ExpressionInternalJsEmit::parse(
uassert(
31223, str::stream() << kExpressionName << " requires 'this' to be specified", thisField);
boost::intrusive_ptr<Expression> thisRef = parseOperand(expCtx, thisField, vps);
-
return new ExpressionInternalJsEmit(expCtx, std::move(thisRef), std::move(funcSourceString));
}
diff --git a/src/mongo/db/pipeline/expression_js_emit.h b/src/mongo/db/pipeline/expression_js_emit.h
index b81c73baa75..61298e50d3f 100644
--- a/src/mongo/db/pipeline/expression_js_emit.h
+++ b/src/mongo/db/pipeline/expression_js_emit.h
@@ -94,5 +94,4 @@ private:
const boost::intrusive_ptr<Expression>& _thisRef;
std::string _funcSource;
};
-
} // namespace mongo
diff --git a/src/mongo/db/pipeline/javascript_execution.cpp b/src/mongo/db/pipeline/javascript_execution.cpp
index 7b00a6e4d7e..2f6255ac660 100644
--- a/src/mongo/db/pipeline/javascript_execution.cpp
+++ b/src/mongo/db/pipeline/javascript_execution.cpp
@@ -28,7 +28,9 @@
*/
#include "mongo/platform/basic.h"
+#include <iostream>
+#include "mongo/base/status_with.h"
#include "mongo/db/pipeline/javascript_execution.h"
namespace mongo {
@@ -56,4 +58,19 @@ JsExecution* JsExecution::get(OperationContext* opCtx,
return exec.get();
}
+Value JsExecution::doCallFunction(ScriptingFunction func,
+ const BSONObj& params,
+ const BSONObj& thisObj,
+ bool noReturnVal) {
+
+ int err = _scope->invoke(func, &params, &thisObj, _fnCallTimeoutMillis, noReturnVal);
+
+ uassert(
+ 31439, str::stream() << "js function failed to execute: " << _scope->getError(), err == 0);
+
+ BSONObjBuilder returnValue;
+ _scope->append(returnValue, "", "__returnValue");
+ return Value(returnValue.done().firstElement());
+}
+
} // namespace mongo
diff --git a/src/mongo/db/pipeline/javascript_execution.h b/src/mongo/db/pipeline/javascript_execution.h
index 544c2eae554..ea02f19699d 100644
--- a/src/mongo/db/pipeline/javascript_execution.h
+++ b/src/mongo/db/pipeline/javascript_execution.h
@@ -33,7 +33,9 @@
#include "mongo/db/client.h"
#include "mongo/db/exec/document_value/value.h"
#include "mongo/db/operation_context.h"
+#include "mongo/db/query/query_knobs_gen.h"
#include "mongo/scripting/engine.h"
+#include "mongo/util/str.h"
namespace mongo {
@@ -65,6 +67,8 @@ public:
_scopeVars = scopeVars.getOwned();
_scope->init(&_scopeVars);
_scope->registerOperation(Client::getCurrent()->getOperationContext());
+
+ _fnCallTimeoutMillis = internalQueryJavaScriptFnTimeoutMillis.load();
}
~JsExecution() {
@@ -80,7 +84,7 @@ public:
void callFunctionWithoutReturn(ScriptingFunction func,
const BSONObj& params,
const BSONObj& thisObj) {
- _scope->invoke(func, &params, &thisObj, 0, true);
+ doCallFunction(func, params, thisObj, true);
}
/**
@@ -90,10 +94,7 @@ public:
* Returns the value returned by the function.
*/
Value callFunction(ScriptingFunction func, const BSONObj& params, const BSONObj& thisObj) {
- _scope->invoke(func, &params, &thisObj, 0, false);
- BSONObjBuilder returnValue;
- _scope->append(returnValue, "", "__returnValue");
- return Value(returnValue.done().firstElement());
+ return doCallFunction(func, params, thisObj, false);
}
/**
@@ -115,5 +116,11 @@ private:
BSONObj _scopeVars;
std::unique_ptr<Scope> _scope;
bool _emitCreated = false;
+ int _fnCallTimeoutMillis;
+
+ Value doCallFunction(ScriptingFunction func,
+ const BSONObj& params,
+ const BSONObj& thisObj,
+ bool noReturnVal);
};
} // namespace mongo
diff --git a/src/mongo/db/query/canonical_query.cpp b/src/mongo/db/query/canonical_query.cpp
index dee4eca06cc..a3d04ae167b 100644
--- a/src/mongo/db/query/canonical_query.cpp
+++ b/src/mongo/db/query/canonical_query.cpp
@@ -153,7 +153,8 @@ StatusWith<std::unique_ptr<CanonicalQuery>> CanonicalQuery::canonicalize(
// Make MatchExpression.
boost::intrusive_ptr<ExpressionContext> newExpCtx;
if (!expCtx.get()) {
- newExpCtx.reset(new ExpressionContext(opCtx, collator.get(), qr->getRuntimeConstants()));
+ newExpCtx.reset(
+ new ExpressionContext(opCtx, collator.get(), qr->nss(), qr->getRuntimeConstants()));
} else {
newExpCtx = expCtx;
invariant(CollatorInterface::collatorsMatch(collator.get(), expCtx->getCollator()));
diff --git a/src/mongo/db/query/find.cpp b/src/mongo/db/query/find.cpp
index 87725fca4d1..a87df84a6df 100644
--- a/src/mongo/db/query/find.cpp
+++ b/src/mongo/db/query/find.cpp
@@ -595,7 +595,7 @@ bool runQuery(OperationContext* opCtx,
// Parse the qm into a CanonicalQuery.
const boost::intrusive_ptr<ExpressionContext> expCtx =
- make_intrusive<ExpressionContext>(opCtx, nullptr /* collator */);
+ make_intrusive<ExpressionContext>(opCtx, nullptr /* collator */, nss);
auto cq = uassertStatusOKWithContext(
CanonicalQuery::canonicalize(opCtx,
q,
diff --git a/src/mongo/db/query/projection_test.cpp b/src/mongo/db/query/projection_test.cpp
index 5618f31c134..b52d27bfb80 100644
--- a/src/mongo/db/query/projection_test.cpp
+++ b/src/mongo/db/query/projection_test.cpp
@@ -45,6 +45,8 @@ using namespace mongo;
using projection_ast::Projection;
+const NamespaceString kTestNss = NamespaceString("db.projection_test");
+
/**
* Helper for creating projections.
*/
@@ -55,7 +57,7 @@ projection_ast::Projection createProjection(const BSONObj& query,
auto opCtx = serviceCtx.makeOperationContext();
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx.get(), collator));
+ new ExpressionContext(opCtx.get(), collator, kTestNss));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(query, std::move(expCtx));
ASSERT_OK(statusWithMatcher.getStatus());
@@ -85,7 +87,7 @@ void assertInvalidProjection(const char* queryStr, const char* projStr) {
auto opCtx = serviceCtx.makeOperationContext();
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx.get(), collator));
+ new ExpressionContext(opCtx.get(), collator, kTestNss));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(query, std::move(expCtx));
ASSERT_OK(statusWithMatcher.getStatus());
@@ -206,7 +208,7 @@ TEST(QueryProjectionTest, InvalidPositionalProjectionDefaultPathMatchExpression)
QueryTestServiceContext serviceCtx;
auto opCtx = serviceCtx.makeOperationContext();
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx.get(), nullptr));
+ new ExpressionContext(opCtx.get(), nullptr, kTestNss));
unique_ptr<MatchExpression> queryMatchExpr(new AlwaysFalseMatchExpression());
ASSERT_EQ(nullptr, queryMatchExpr->path().rawData());
diff --git a/src/mongo/db/query/query_knobs.idl b/src/mongo/db/query/query_knobs.idl
index 7516de7b407..9eaee626499 100644
--- a/src/mongo/db/query/query_knobs.idl
+++ b/src/mongo/db/query/query_knobs.idl
@@ -338,5 +338,13 @@ server_parameters:
cpp_vartype: AtomicWord<int>
default:
expr: 100
+
+ internalQueryJavaScriptFnTimeoutMillis:
+ description: "Limits the maximum allowed time a user-defined javascript function can run in a query."
+ set_at: [ startup, runtime ]
+ cpp_varname: "internalQueryJavaScriptFnTimeoutMillis"
+ cpp_vartype: AtomicWord<int>
+ default:
+ expr: 60 * 1000
validator:
gt: 0
diff --git a/src/mongo/db/query/query_planner_test_lib.cpp b/src/mongo/db/query/query_planner_test_lib.cpp
index f5eebc27e70..4643d3c1516 100644
--- a/src/mongo/db/query/query_planner_test_lib.cpp
+++ b/src/mongo/db/query/query_planner_test_lib.cpp
@@ -57,6 +57,7 @@ using namespace mongo;
using std::string;
+
bool filterMatches(const BSONObj& testFilter,
const BSONObj& testCollation,
const QuerySolutionNode* trueFilterNode) {
@@ -619,7 +620,8 @@ bool QueryPlannerTestLib::solutionMatches(const BSONObj& testSoln,
// Create an empty/dummy expression context without access to the operation context and
// collator. This should be sufficient to parse a projection.
- auto expCtx = make_intrusive<ExpressionContext>(nullptr, nullptr);
+ auto expCtx =
+ make_intrusive<ExpressionContext>(nullptr, nullptr, NamespaceString("test.dummy"));
auto projection =
projection_ast::parse(expCtx, spec.Obj(), ProjectionPolicies::findProjectionPolicies());
auto specProjObj = projection_ast::astToDebugBSON(projection.root());
diff --git a/src/mongo/db/query/query_request.h b/src/mongo/db/query/query_request.h
index 490317960ce..3593489cec8 100644
--- a/src/mongo/db/query/query_request.h
+++ b/src/mongo/db/query/query_request.h
@@ -165,7 +165,6 @@ public:
static const std::string kAllowDiskUseField;
const NamespaceString& nss() const {
- invariant(!_nss.isEmpty());
return _nss;
}
diff --git a/src/mongo/db/query/query_solution_test.cpp b/src/mongo/db/query/query_solution_test.cpp
index 97b14fec0ce..142ed16fd2a 100644
--- a/src/mongo/db/query/query_solution_test.cpp
+++ b/src/mongo/db/query/query_solution_test.cpp
@@ -46,7 +46,6 @@
namespace {
using namespace mongo;
-
/**
* Make a minimal IndexEntry from just a key pattern. A dummy name will be added.
*/
@@ -720,7 +719,7 @@ auto createMatchExprAndProjection(const BSONObj& query, const BSONObj& projObj)
auto opCtx = serviceCtx.makeOperationContext();
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx.get(), collator));
+ new ExpressionContext(opCtx.get(), collator, NamespaceString("test.dummy")));
StatusWithMatchExpression queryMatchExpr =
MatchExpressionParser::parse(query, std::move(expCtx));
ASSERT(queryMatchExpr.isOK());
diff --git a/src/mongo/db/repl/apply_ops.cpp b/src/mongo/db/repl/apply_ops.cpp
index a5960b2d0e6..2df9c7428be 100644
--- a/src/mongo/db/repl/apply_ops.cpp
+++ b/src/mongo/db/repl/apply_ops.cpp
@@ -321,7 +321,7 @@ Status _checkPrecondition(OperationContext* opCtx,
// applyOps does not allow any extensions, such as $text, $where, $geoNear, $near,
// $nearSphere, or $expr.
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator, nss));
Matcher matcher(preCondition["res"].Obj(), std::move(expCtx));
if (!matcher.matches(realres)) {
result->append("got", realres);
diff --git a/src/mongo/db/repl/oplog_fetcher.cpp b/src/mongo/db/repl/oplog_fetcher.cpp
index 5cd9e5c967c..afff88413c3 100644
--- a/src/mongo/db/repl/oplog_fetcher.cpp
+++ b/src/mongo/db/repl/oplog_fetcher.cpp
@@ -472,7 +472,7 @@ StatusWith<BSONObj> OplogFetcher::_onSuccessfulBatch(const Fetcher::QueryRespons
[&](const BSONObj& data) {
auto opCtx = cc().makeOperationContext();
boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx.get(), nullptr));
+ new ExpressionContext(opCtx.get(), nullptr, _getNamespace()));
Matcher m(data["document"].Obj(), expCtx);
return !queryResponse.documents.empty() &&
m.matches(queryResponse.documents.front()["o"].Obj());
@@ -1079,7 +1079,7 @@ Status NewOplogFetcher::_onSuccessfulBatch(const Documents& documents) {
[&](const BSONObj& data) {
auto opCtx = cc().makeOperationContext();
boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtx.get(), nullptr));
+ new ExpressionContext(opCtx.get(), nullptr, _nss));
Matcher m(data["document"].Obj(), expCtx);
return !documents.empty() && m.matches(documents.front()["o"].Obj());
});
diff --git a/src/mongo/db/transaction_participant.cpp b/src/mongo/db/transaction_participant.cpp
index 031d4ae6568..3c9dfa78395 100644
--- a/src/mongo/db/transaction_participant.cpp
+++ b/src/mongo/db/transaction_participant.cpp
@@ -236,7 +236,8 @@ void updateSessionEntry(OperationContext* opCtx, const UpdateRequest& updateRequ
auto originalDoc = originalRecordData.toBson();
invariant(collection->getDefaultCollator() == nullptr);
- boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, nullptr));
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, nullptr, updateRequest.getNamespaceString()));
auto matcher =
fassert(40673, MatchExpressionParser::parse(updateRequest.getQuery(), std::move(expCtx)));
diff --git a/src/mongo/db/update/update_driver_test.cpp b/src/mongo/db/update/update_driver_test.cpp
index dcfab0d74fb..18ac94213f7 100644
--- a/src/mongo/db/update/update_driver_test.cpp
+++ b/src/mongo/db/update/update_driver_test.cpp
@@ -202,8 +202,10 @@ class CreateFromQueryFixture : public mongo::unittest::Test {
public:
CreateFromQueryFixture()
: _opCtx(_serviceContext.makeOperationContext()),
- _driverOps(new UpdateDriver(new ExpressionContext(_opCtx.get(), nullptr))),
- _driverRepl(new UpdateDriver(new ExpressionContext(_opCtx.get(), nullptr))) {
+ _driverOps(new UpdateDriver(
+ new ExpressionContext(_opCtx.get(), nullptr, NamespaceString("foo")))),
+ _driverRepl(new UpdateDriver(
+ new ExpressionContext(_opCtx.get(), nullptr, NamespaceString("foo")))) {
_driverOps->parse(fromjson("{$set:{'_':1}}"), _arrayFilters);
_driverRepl->parse(fromjson("{}"), _arrayFilters);
}
diff --git a/src/mongo/dbtests/extensions_callback_real_test.cpp b/src/mongo/dbtests/extensions_callback_real_test.cpp
index bdc84eb61bd..ae2287ac77a 100644
--- a/src/mongo/dbtests/extensions_callback_real_test.cpp
+++ b/src/mongo/dbtests/extensions_callback_real_test.cpp
@@ -240,72 +240,23 @@ TEST_F(ExtensionsCallbackRealTest, TextDiacriticSensitiveAndCaseSensitiveTrue) {
//
// $where parsing tests.
//
+const NamespaceString kTestNss = NamespaceString("db.dummy");
-TEST_F(ExtensionsCallbackRealTest, WhereExpressionsWithSameScopeHaveSameBSONRepresentation) {
- const char code[] = "function(){ return a; }";
+TEST_F(ExtensionsCallbackRealTest, WhereExpressionDesugarsToExprAndInternalJs) {
+ auto query1 = fromjson("{$where: 'function() { return this.x == 10; }'}");
+ boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(&_opCtx, nullptr, kTestNss));
- BSONObj query1 = BSON("$where" << BSONCodeWScope(code, BSON("a" << true)));
auto expr1 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query1.firstElement()));
- BSONObjBuilder builder1;
- expr1->serialize(&builder1);
+ ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(expCtx, query1.firstElement()));
- BSONObj query2 = BSON("$where" << BSONCodeWScope(code, BSON("a" << true)));
- auto expr2 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query2.firstElement()));
- BSONObjBuilder builder2;
- expr2->serialize(&builder2);
+ BSONObjBuilder gotMatch;
+ expr1->serialize(&gotMatch);
- ASSERT_BSONOBJ_EQ(builder1.obj(), builder2.obj());
-}
-
-TEST_F(ExtensionsCallbackRealTest,
- WhereExpressionsWithDifferentScopesHaveDifferentBSONRepresentations) {
- const char code[] = "function(){ return a; }";
-
- BSONObj query1 = BSON("$where" << BSONCodeWScope(code, BSON("a" << true)));
- auto expr1 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query1.firstElement()));
- BSONObjBuilder builder1;
- expr1->serialize(&builder1);
-
- BSONObj query2 = BSON("$where" << BSONCodeWScope(code, BSON("a" << false)));
- auto expr2 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query2.firstElement()));
- BSONObjBuilder builder2;
- expr2->serialize(&builder2);
-
- ASSERT_BSONOBJ_NE(builder1.obj(), builder2.obj());
-}
-
-TEST_F(ExtensionsCallbackRealTest, WhereExpressionsWithSameScopeAreEquivalent) {
- const char code[] = "function(){ return a; }";
-
- BSONObj query1 = BSON("$where" << BSONCodeWScope(code, BSON("a" << true)));
- auto expr1 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query1.firstElement()));
-
- BSONObj query2 = BSON("$where" << BSONCodeWScope(code, BSON("a" << true)));
- auto expr2 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query2.firstElement()));
-
- ASSERT(expr1->equivalent(expr2.get()));
- ASSERT(expr2->equivalent(expr1.get()));
-}
-
-TEST_F(ExtensionsCallbackRealTest, WhereExpressionsWithDifferentScopesAreNotEquivalent) {
- const char code[] = "function(){ return a; }";
-
- BSONObj query1 = BSON("$where" << BSONCodeWScope(code, BSON("a" << true)));
- auto expr1 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query1.firstElement()));
-
- BSONObj query2 = BSON("$where" << BSONCodeWScope(code, BSON("a" << false)));
- auto expr2 = unittest::assertGet(
- ExtensionsCallbackReal(&_opCtx, &_nss).parseWhere(query2.firstElement()));
-
- ASSERT_FALSE(expr1->equivalent(expr2.get()));
- ASSERT_FALSE(expr2->equivalent(expr1.get()));
+ auto expectedMatch = fromjson(
+ "{$expr: {$function: {'body': 'function() { return this.x == 10; }', 'args': "
+ "['$$CURRENT'], 'lang': 'js', '_internalSetObjToThis': true}}}");
+ ASSERT_BSONOBJ_EQ(gotMatch.obj(), expectedMatch);
}
} // namespace
diff --git a/src/mongo/dbtests/matchertests.cpp b/src/mongo/dbtests/matchertests.cpp
index b0fb98ded31..bc3e329816c 100644
--- a/src/mongo/dbtests/matchertests.cpp
+++ b/src/mongo/dbtests/matchertests.cpp
@@ -54,6 +54,8 @@ public:
virtual ~CollectionBase() {}
};
+const NamespaceString kTestNss = NamespaceString("db.dummy");
+
template <typename M>
class Basic {
public:
@@ -229,7 +231,7 @@ public:
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(opCtxPtr.get(), collator));
+ new ExpressionContext(opCtxPtr.get(), collator, kTestNss));
M m(BSON("$where"
<< "function(){ return this.a == 1; }"),
expCtx,
diff --git a/src/mongo/dbtests/query_stage_collscan.cpp b/src/mongo/dbtests/query_stage_collscan.cpp
index 757bb50c393..5eb79bb6e80 100644
--- a/src/mongo/dbtests/query_stage_collscan.cpp
+++ b/src/mongo/dbtests/query_stage_collscan.cpp
@@ -96,7 +96,7 @@ public:
// Make the filter.
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(&_opCtx, collator));
+ new ExpressionContext(&_opCtx, collator, nss));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(filterObj, expCtx);
verify(statusWithMatcher.isOK());
diff --git a/src/mongo/dbtests/query_stage_count.cpp b/src/mongo/dbtests/query_stage_count.cpp
index e8bdc4a0dba..0c62e6636f4 100644
--- a/src/mongo/dbtests/query_stage_count.cpp
+++ b/src/mongo/dbtests/query_stage_count.cpp
@@ -53,6 +53,7 @@ using std::vector;
const int kDocuments = 100;
const int kInterjections = kDocuments;
+const NamespaceString kTestNss = NamespaceString("db.dummy");
class CountStageTest {
public:
@@ -147,7 +148,7 @@ public:
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(&_opCtx, collator));
+ new ExpressionContext(&_opCtx, collator, kTestNss));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(request.getQuery(), expCtx);
ASSERT(statusWithMatcher.isOK());
diff --git a/src/mongo/dbtests/query_stage_ensure_sorted.cpp b/src/mongo/dbtests/query_stage_ensure_sorted.cpp
index a3a81a84660..e956472d40d 100644
--- a/src/mongo/dbtests/query_stage_ensure_sorted.cpp
+++ b/src/mongo/dbtests/query_stage_ensure_sorted.cpp
@@ -43,6 +43,8 @@ namespace mongo {
namespace {
+const NamespaceString kTestNss = NamespaceString("db.dummy");
+
class QueryStageEnsureSortedTest : public unittest::Test {
public:
/**
@@ -80,7 +82,7 @@ public:
// Create a mock ExpressionContext.
boost::intrusive_ptr<ExpressionContext> pExpCtx(
- new ExpressionContext(opCtx.get(), collator));
+ new ExpressionContext(opCtx.get(), collator, kTestNss));
pExpCtx->setCollator(collator);
// Initialization.
@@ -121,7 +123,8 @@ TEST_F(QueryStageEnsureSortedTest, EnsureSortedEmptyWorkingSet) {
auto opCtx = _serviceContext.makeOperationContext();
// Create a mock ExpressionContext.
- boost::intrusive_ptr<ExpressionContext> pExpCtx(new ExpressionContext(opCtx.get(), nullptr));
+ boost::intrusive_ptr<ExpressionContext> pExpCtx(
+ new ExpressionContext(opCtx.get(), nullptr, kTestNss));
WorkingSet ws;
auto queuedDataStage = std::make_unique<QueuedDataStage>(opCtx.get(), &ws);
diff --git a/src/mongo/dbtests/query_stage_fetch.cpp b/src/mongo/dbtests/query_stage_fetch.cpp
index 13781d15b3a..53b4e1e0646 100644
--- a/src/mongo/dbtests/query_stage_fetch.cpp
+++ b/src/mongo/dbtests/query_stage_fetch.cpp
@@ -202,7 +202,7 @@ public:
BSONObj filterObj = BSON("foo" << 6);
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(&_opCtx, collator));
+ new ExpressionContext(&_opCtx, collator, nss()));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(filterObj, expCtx);
verify(statusWithMatcher.isOK());
diff --git a/src/mongo/dbtests/query_stage_multiplan.cpp b/src/mongo/dbtests/query_stage_multiplan.cpp
index 23e624a0465..a9b5a8375fe 100644
--- a/src/mongo/dbtests/query_stage_multiplan.cpp
+++ b/src/mongo/dbtests/query_stage_multiplan.cpp
@@ -155,7 +155,8 @@ unique_ptr<PlanStage> getIxScanPlan(OperationContext* opCtx,
unique_ptr<MatchExpression> makeMatchExpressionFromFilter(OperationContext* opCtx,
BSONObj filterObj) {
const CollatorInterface* collator = nullptr;
- const boost::intrusive_ptr<ExpressionContext> expCtx(new ExpressionContext(opCtx, collator));
+ const boost::intrusive_ptr<ExpressionContext> expCtx(
+ new ExpressionContext(opCtx, collator, nss));
StatusWithMatchExpression statusWithMatcher = MatchExpressionParser::parse(filterObj, expCtx);
ASSERT_OK(statusWithMatcher.getStatus());
unique_ptr<MatchExpression> filter = std::move(statusWithMatcher.getValue());
diff --git a/src/mongo/dbtests/query_stage_sort.cpp b/src/mongo/dbtests/query_stage_sort.cpp
index 9f521d7fc36..3e01a3c97f6 100644
--- a/src/mongo/dbtests/query_stage_sort.cpp
+++ b/src/mongo/dbtests/query_stage_sort.cpp
@@ -236,7 +236,8 @@ public:
protected:
const ServiceContext::UniqueOperationContext _txnPtr = cc().makeOperationContext();
OperationContext& _opCtx = *_txnPtr;
- boost::intrusive_ptr<ExpressionContext> _expCtx = new ExpressionContext(&_opCtx, nullptr);
+ boost::intrusive_ptr<ExpressionContext> _expCtx =
+ new ExpressionContext(&_opCtx, nullptr, nss());
DBDirectClient _client;
};
diff --git a/src/mongo/dbtests/query_stage_sort_key_generator.cpp b/src/mongo/dbtests/query_stage_sort_key_generator.cpp
index fe7ae6583c0..351ad1cca0c 100644
--- a/src/mongo/dbtests/query_stage_sort_key_generator.cpp
+++ b/src/mongo/dbtests/query_stage_sort_key_generator.cpp
@@ -55,6 +55,8 @@ Value extractKeyFromKeyGenStage(SortKeyGeneratorStage* sortKeyGen, WorkingSet* w
return wsm->metadata().getSortKey();
}
+const NamespaceString kTestNss = NamespaceString("db.dummy");
+
/**
* Given a JSON string 'sortSpec' representing a sort pattern, returns the corresponding sort key
* from 'doc', a JSON string representation of a user document. Does so using the SORT_KEY_GENERATOR
@@ -66,7 +68,8 @@ Value extractKeyFromKeyGenStage(SortKeyGeneratorStage* sortKeyGen, WorkingSet* w
Value extractSortKey(const char* sortSpec, const char* doc, const CollatorInterface* collator) {
QueryTestServiceContext serviceContext;
auto opCtx = serviceContext.makeOperationContext();
- boost::intrusive_ptr<ExpressionContext> pExpCtx(new ExpressionContext(opCtx.get(), collator));
+ boost::intrusive_ptr<ExpressionContext> pExpCtx(
+ new ExpressionContext(opCtx.get(), collator, kTestNss));
WorkingSet workingSet;
@@ -95,7 +98,8 @@ Value extractSortKeyCovered(const char* sortSpec,
const CollatorInterface* collator) {
QueryTestServiceContext serviceContext;
auto opCtx = serviceContext.makeOperationContext();
- boost::intrusive_ptr<ExpressionContext> pExpCtx(new ExpressionContext(opCtx.get(), collator));
+ boost::intrusive_ptr<ExpressionContext> pExpCtx(
+ new ExpressionContext(opCtx.get(), collator, kTestNss));
WorkingSet workingSet;
diff --git a/src/mongo/dbtests/query_stage_tests.cpp b/src/mongo/dbtests/query_stage_tests.cpp
index a81bee3b634..43a6c482131 100644
--- a/src/mongo/dbtests/query_stage_tests.cpp
+++ b/src/mongo/dbtests/query_stage_tests.cpp
@@ -85,7 +85,7 @@ public:
const CollatorInterface* collator = nullptr;
const boost::intrusive_ptr<ExpressionContext> expCtx(
- new ExpressionContext(&_opCtx, collator));
+ new ExpressionContext(&_opCtx, collator, NamespaceString(ns())));
StatusWithMatchExpression statusWithMatcher =
MatchExpressionParser::parse(filterObj, expCtx);
verify(statusWithMatcher.isOK());
diff --git a/src/mongo/dbtests/query_stage_update.cpp b/src/mongo/dbtests/query_stage_update.cpp
index 2f501af6690..98061f09523 100644
--- a/src/mongo/dbtests/query_stage_update.cpp
+++ b/src/mongo/dbtests/query_stage_update.cpp
@@ -201,7 +201,7 @@ public:
CurOp& curOp = *CurOp::get(_opCtx);
OpDebug* opDebug = &curOp.debug();
const CollatorInterface* collator = nullptr;
- UpdateDriver driver(new ExpressionContext(&_opCtx, collator));
+ UpdateDriver driver(new ExpressionContext(&_opCtx, collator, nss));
Collection* collection = ctx.getCollection();
ASSERT(collection);
@@ -272,7 +272,7 @@ public:
CurOp& curOp = *CurOp::get(_opCtx);
OpDebug* opDebug = &curOp.debug();
const CollatorInterface* collator = nullptr;
- UpdateDriver driver(new ExpressionContext(&_opCtx, collator));
+ UpdateDriver driver(new ExpressionContext(&_opCtx, collator, nss));
Collection* coll =
CollectionCatalog::get(&_opCtx).lookupCollectionByNamespace(&_opCtx, nss);
ASSERT(coll);
@@ -387,7 +387,7 @@ public:
ASSERT(coll);
UpdateRequest request(nss);
const CollatorInterface* collator = nullptr;
- UpdateDriver driver(new ExpressionContext(&_opCtx, collator));
+ UpdateDriver driver(new ExpressionContext(&_opCtx, collator, nss));
const int targetDocIndex = 0; // We'll be working with the first doc in the collection.
const BSONObj query = BSON("foo" << BSON("$gte" << targetDocIndex));
const auto ws = make_unique<WorkingSet>();
@@ -479,7 +479,7 @@ public:
ASSERT(coll);
UpdateRequest request(nss);
const CollatorInterface* collator = nullptr;
- UpdateDriver driver(new ExpressionContext(&_opCtx, collator));
+ UpdateDriver driver(new ExpressionContext(&_opCtx, collator, nss));
const int targetDocIndex = 10;
const BSONObj query = BSON("foo" << BSON("$gte" << targetDocIndex));
const auto ws = make_unique<WorkingSet>();
diff --git a/src/mongo/embedded/stitch_support/stitch_support.cpp b/src/mongo/embedded/stitch_support/stitch_support.cpp
index 9bd13132808..53010105fca 100644
--- a/src/mongo/embedded/stitch_support/stitch_support.cpp
+++ b/src/mongo/embedded/stitch_support/stitch_support.cpp
@@ -39,6 +39,7 @@
#include "mongo/db/exec/projection_executor_builder.h"
#include "mongo/db/logical_clock.h"
#include "mongo/db/matcher/matcher.h"
+#include "mongo/db/namespace_string.h"
#include "mongo/db/ops/parsed_update.h"
#include "mongo/db/query/collation/collator_factory_interface.h"
#include "mongo/db/query/projection.h"
@@ -61,6 +62,8 @@ namespace mongo {
using StitchSupportStatusImpl = StatusForAPI<stitch_support_v1_error>;
+const NamespaceString kDummyNamespaceStr = NamespaceString("");
+
/**
* C interfaces that use enterCXX() must provide a translateException() function that converts any
* possible exception into a StatusForAPI<> object.
@@ -171,7 +174,8 @@ struct stitch_support_v1_matcher {
opCtx(this->client->makeOperationContext()),
matcher(filterBSON.getOwned(),
new mongo::ExpressionContext(opCtx.get(),
- collator ? collator->collator.get() : nullptr)){};
+ collator ? collator->collator.get() : nullptr,
+ mongo::kDummyNamespaceStr)){};
mongo::ServiceContext::UniqueClient client;
mongo::ServiceContext::UniqueOperationContext opCtx;
@@ -186,7 +190,7 @@ struct stitch_support_v1_projection {
: client(std::move(client)), opCtx(this->client->makeOperationContext()), matcher(matcher) {
auto expCtx = mongo::make_intrusive<mongo::ExpressionContext>(
- opCtx.get(), collator ? collator->collator.get() : nullptr);
+ opCtx.get(), collator ? collator->collator.get() : nullptr, mongo::kDummyNamespaceStr);
const auto policies = mongo::ProjectionPolicies::findProjectionPolicies();
auto proj =
mongo::projection_ast::parse(expCtx,
@@ -228,14 +232,18 @@ struct stitch_support_v1_update {
updateExpr(updateExpr.getOwned()),
arrayFilters(arrayFilters.getOwned()),
matcher(matcher),
- updateDriver(new mongo::ExpressionContext(
- opCtx.get(), collator ? collator->collator.get() : nullptr)) {
+ updateDriver(new mongo::ExpressionContext(opCtx.get(),
+ collator ? collator->collator.get() : nullptr,
+ mongo::kDummyNamespaceStr)) {
std::vector<mongo::BSONObj> arrayFilterVector;
for (auto&& filter : this->arrayFilters) {
arrayFilterVector.push_back(filter.embeddedObject());
}
- this->parsedFilters = uassertStatusOK(mongo::ParsedUpdate::parseArrayFilters(
- arrayFilterVector, this->opCtx.get(), collator ? collator->collator.get() : nullptr));
+ this->parsedFilters = uassertStatusOK(
+ mongo::ParsedUpdate::parseArrayFilters(arrayFilterVector,
+ this->opCtx.get(),
+ collator ? collator->collator.get() : nullptr,
+ mongo::kDummyNamespaceStr));
updateDriver.parse(this->updateExpr, parsedFilters);
diff --git a/src/mongo/s/commands/cluster_clear_jumbo_flag_cmd.cpp b/src/mongo/s/commands/cluster_clear_jumbo_flag_cmd.cpp
index 38e1e063ad1..19cfde4a72c 100644
--- a/src/mongo/s/commands/cluster_clear_jumbo_flag_cmd.cpp
+++ b/src/mongo/s/commands/cluster_clear_jumbo_flag_cmd.cpp
@@ -94,8 +94,9 @@ public:
boost::optional<Chunk> chunk;
if (request().getFind()) {
- BSONObj shardKey = uassertStatusOK(
- cm->getShardKeyPattern().extractShardKeyFromQuery(opCtx, *request().getFind()));
+ BSONObj shardKey =
+ uassertStatusOK(cm->getShardKeyPattern().extractShardKeyFromQuery(
+ opCtx, ns(), *request().getFind()));
uassert(51260,
str::stream()
<< "no shard key found in chunk query " << *request().getFind(),
diff --git a/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp b/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp
index 3a6c019d041..f8760179bfe 100644
--- a/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp
+++ b/src/mongo/s/commands/cluster_find_and_modify_cmd.cpp
@@ -81,9 +81,12 @@ BSONObj getCollation(const BSONObj& cmdObj) {
return BSONObj();
}
-BSONObj getShardKey(OperationContext* opCtx, const ChunkManager& chunkMgr, const BSONObj& query) {
+BSONObj getShardKey(OperationContext* opCtx,
+ const ChunkManager& chunkMgr,
+ const NamespaceString& nss,
+ const BSONObj& query) {
BSONObj shardKey =
- uassertStatusOK(chunkMgr.getShardKeyPattern().extractShardKeyFromQuery(opCtx, query));
+ uassertStatusOK(chunkMgr.getShardKeyPattern().extractShardKeyFromQuery(opCtx, nss, query));
uassert(ErrorCodes::ShardKeyNotFound,
"Query for sharded findAndModify must contain the shard key",
!shardKey.isEmpty());
@@ -188,7 +191,7 @@ public:
const BSONObj query = cmdObj.getObjectField("query");
const BSONObj collation = getCollation(cmdObj);
- const BSONObj shardKey = getShardKey(opCtx, *chunkMgr, query);
+ const BSONObj shardKey = getShardKey(opCtx, *chunkMgr, nss, query);
const auto chunk = chunkMgr->findIntersectingChunk(shardKey, collation);
shard = uassertStatusOK(
@@ -264,7 +267,7 @@ public:
const BSONObj query = cmdObjForShard.getObjectField("query");
const BSONObj collation = getCollation(cmdObjForShard);
- const BSONObj shardKey = getShardKey(opCtx, *chunkMgr, query);
+ const BSONObj shardKey = getShardKey(opCtx, *chunkMgr, nss, query);
auto chunk = chunkMgr->findIntersectingChunk(shardKey, collation);
_runCommand(opCtx,
diff --git a/src/mongo/s/commands/cluster_move_chunk_cmd.cpp b/src/mongo/s/commands/cluster_move_chunk_cmd.cpp
index b31448b839d..89732fa2407 100644
--- a/src/mongo/s/commands/cluster_move_chunk_cmd.cpp
+++ b/src/mongo/s/commands/cluster_move_chunk_cmd.cpp
@@ -147,8 +147,8 @@ public:
if (!find.isEmpty()) {
// find
- BSONObj shardKey =
- uassertStatusOK(cm->getShardKeyPattern().extractShardKeyFromQuery(opCtx, find));
+ BSONObj shardKey = uassertStatusOK(
+ cm->getShardKeyPattern().extractShardKeyFromQuery(opCtx, nss, find));
if (shardKey.isEmpty()) {
errmsg = str::stream() << "no shard key found in chunk query " << find;
return false;
diff --git a/src/mongo/s/commands/cluster_split_cmd.cpp b/src/mongo/s/commands/cluster_split_cmd.cpp
index 7e79bc71e2e..0ea00d0138d 100644
--- a/src/mongo/s/commands/cluster_split_cmd.cpp
+++ b/src/mongo/s/commands/cluster_split_cmd.cpp
@@ -193,8 +193,8 @@ public:
if (!find.isEmpty()) {
// find
- BSONObj shardKey =
- uassertStatusOK(cm->getShardKeyPattern().extractShardKeyFromQuery(opCtx, find));
+ BSONObj shardKey = uassertStatusOK(
+ cm->getShardKeyPattern().extractShardKeyFromQuery(opCtx, nss, find));
if (shardKey.isEmpty()) {
errmsg = str::stream() << "no shard key found in chunk query " << find;
return false;
diff --git a/src/mongo/s/query/cluster_aggregate.cpp b/src/mongo/s/query/cluster_aggregate.cpp
index c91f3436d16..b19183096e6 100644
--- a/src/mongo/s/query/cluster_aggregate.cpp
+++ b/src/mongo/s/query/cluster_aggregate.cpp
@@ -286,7 +286,7 @@ Status ClusterAggregate::runAggregate(OperationContext* opCtx,
// passthrough, we only need a bare minimum expression context anyway.
invariant(targeter.policy ==
cluster_aggregation_planner::AggregationTargeter::kPassthrough);
- expCtx = make_intrusive<ExpressionContext>(opCtx, nullptr);
+ expCtx = make_intrusive<ExpressionContext>(opCtx, nullptr, namespaces.executionNss);
}
if (request.getExplain()) {
diff --git a/src/mongo/s/shard_key_pattern.cpp b/src/mongo/s/shard_key_pattern.cpp
index ab6db357b34..ec69ebf8a46 100644
--- a/src/mongo/s/shard_key_pattern.cpp
+++ b/src/mongo/s/shard_key_pattern.cpp
@@ -377,8 +377,9 @@ BSONObj ShardKeyPattern::emplaceMissingShardKeyValuesForDocument(const BSONObj d
}
StatusWith<BSONObj> ShardKeyPattern::extractShardKeyFromQuery(OperationContext* opCtx,
+ const NamespaceString& nss,
const BSONObj& basicQuery) const {
- auto qr = std::make_unique<QueryRequest>(NamespaceString(""));
+ auto qr = std::make_unique<QueryRequest>(nss);
qr->setFilter(basicQuery);
const boost::intrusive_ptr<ExpressionContext> expCtx;
diff --git a/src/mongo/s/shard_key_pattern.h b/src/mongo/s/shard_key_pattern.h
index b8052a81f67..ce54bf57c34 100644
--- a/src/mongo/s/shard_key_pattern.h
+++ b/src/mongo/s/shard_key_pattern.h
@@ -211,6 +211,7 @@ public:
* a : { $eq : { b : ... } }
*/
StatusWith<BSONObj> extractShardKeyFromQuery(OperationContext* opCtx,
+ const NamespaceString& nss,
const BSONObj& basicQuery) const;
BSONObj extractShardKeyFromQuery(const CanonicalQuery& query) const;
diff --git a/src/mongo/s/shard_key_pattern_test.cpp b/src/mongo/s/shard_key_pattern_test.cpp
index 0929ac2e511..a022324e329 100644
--- a/src/mongo/s/shard_key_pattern_test.cpp
+++ b/src/mongo/s/shard_key_pattern_test.cpp
@@ -293,8 +293,9 @@ TEST(ShardKeyPattern, ExtractDocShardKeyHashed) {
static BSONObj queryKey(const ShardKeyPattern& pattern, const BSONObj& query) {
QueryTestServiceContext serviceContext;
auto opCtx = serviceContext.makeOperationContext();
+ const NamespaceString& nss{NamespaceString("foo")};
- StatusWith<BSONObj> status = pattern.extractShardKeyFromQuery(opCtx.get(), query);
+ StatusWith<BSONObj> status = pattern.extractShardKeyFromQuery(opCtx.get(), nss, query);
if (!status.isOK())
return BSONObj();
return status.getValue();
diff --git a/src/mongo/s/write_ops/chunk_manager_targeter.cpp b/src/mongo/s/write_ops/chunk_manager_targeter.cpp
index d2450bec6ac..d5323ae09da 100644
--- a/src/mongo/s/write_ops/chunk_manager_targeter.cpp
+++ b/src/mongo/s/write_ops/chunk_manager_targeter.cpp
@@ -115,6 +115,7 @@ StatusWith<UpdateType> getUpdateExprType(const write_ops::UpdateOpEntry& updateD
*/
StatusWith<BSONObj> getUpdateExprForTargeting(OperationContext* opCtx,
const ShardKeyPattern& shardKeyPattern,
+ const NamespaceString& nss,
const UpdateType updateType,
const write_ops::UpdateOpEntry& updateDoc) {
// We should never see an invalid update type here.
@@ -144,7 +145,8 @@ StatusWith<BSONObj> getUpdateExprForTargeting(OperationContext* opCtx,
// We are missing _id, so attempt to extract it from an exact match in the update's query spec.
// This will guarantee that we can target a single shard, but it is not necessarily fatal if no
// exact _id can be found.
- const auto idFromQuery = kVirtualIdShardKey.extractShardKeyFromQuery(opCtx, updateDoc.getQ());
+ const auto idFromQuery =
+ kVirtualIdShardKey.extractShardKeyFromQuery(opCtx, nss, updateDoc.getQ());
if (!idFromQuery.isOK()) {
return idFromQuery;
} else if (auto idElt = idFromQuery.getValue()[kIdFieldName]) {
@@ -447,8 +449,8 @@ StatusWith<std::vector<ShardEndpoint>> ChunkManagerTargeter::targetUpdate(
const auto& shardKeyPattern = _routingInfo->cm()->getShardKeyPattern();
const auto collation = write_ops::collationOf(updateDoc);
- const auto updateExpr =
- getUpdateExprForTargeting(opCtx, shardKeyPattern, updateType.getValue(), updateDoc);
+ const auto updateExpr = getUpdateExprForTargeting(
+ opCtx, shardKeyPattern, getNS(), updateType.getValue(), updateDoc);
const bool isUpsert = updateDoc.getUpsert();
const auto query = updateDoc.getQ();
if (!updateExpr.isOK()) {
@@ -477,7 +479,7 @@ StatusWith<std::vector<ShardEndpoint>> ChunkManagerTargeter::targetUpdate(
// to target based on the replacement doc, it could result in an insertion even if a document
// matching the query exists on another shard.
if (isUpsert) {
- return targetByShardKey(shardKeyPattern.extractShardKeyFromQuery(opCtx, query),
+ return targetByShardKey(shardKeyPattern.extractShardKeyFromQuery(opCtx, getNS(), query),
"Failed to target upsert by query");
}
@@ -531,8 +533,8 @@ StatusWith<std::vector<ShardEndpoint>> ChunkManagerTargeter::targetDelete(
// Get the shard key
StatusWith<BSONObj> status =
- _routingInfo->cm()->getShardKeyPattern().extractShardKeyFromQuery(opCtx,
- deleteDoc.getQ());
+ _routingInfo->cm()->getShardKeyPattern().extractShardKeyFromQuery(
+ opCtx, getNS(), deleteDoc.getQ());
// Bad query
if (!status.isOK())
diff --git a/src/mongo/scripting/mozjs/implscope.cpp b/src/mongo/scripting/mozjs/implscope.cpp
index 6faa2467125..7c2f57a04ef 100644
--- a/src/mongo/scripting/mozjs/implscope.cpp
+++ b/src/mongo/scripting/mozjs/implscope.cpp
@@ -33,6 +33,7 @@
#include "mongo/scripting/mozjs/implscope.h"
+#include <iostream>
#include <memory>
#include <js/CharacterEncoding.h>