summaryrefslogtreecommitdiff
path: root/src/mongo/db
diff options
context:
space:
mode:
authorBernard Gorman <bernard.gorman@mongodb.com>2019-11-14 21:59:35 +0000
committerevergreen <evergreen@mongodb.com>2019-11-14 21:59:35 +0000
commit23e55cb3d041236f399f7095df31cd3e3da491cc (patch)
tree25bc309af51bc66dbd46922b0cf7560b3351478a /src/mongo/db
parentcdc44d95e169da75093f25c324aa9670e72743e8 (diff)
downloadmongo-23e55cb3d041236f399f7095df31cd3e3da491cc.tar.gz
SERVER-43860 Always upsert exact source document for pipeline-insert $merge
Diffstat (limited to 'src/mongo/db')
-rw-r--r--src/mongo/db/SConscript1
-rw-r--r--src/mongo/db/commands/write_commands/write_commands.cpp1
-rw-r--r--src/mongo/db/exec/update_stage.cpp340
-rw-r--r--src/mongo/db/exec/update_stage.h103
-rw-r--r--src/mongo/db/exec/upsert_stage.cpp294
-rw-r--r--src/mongo/db/exec/upsert_stage.h76
-rw-r--r--src/mongo/db/field_ref_set.cpp21
-rw-r--r--src/mongo/db/field_ref_set.h22
-rw-r--r--src/mongo/db/ops/parsed_update.cpp17
-rw-r--r--src/mongo/db/ops/update_request.h39
-rw-r--r--src/mongo/db/ops/write_ops.idl5
-rw-r--r--src/mongo/db/ops/write_ops_exec.cpp1
-rw-r--r--src/mongo/db/pipeline/document_source_merge.cpp45
-rw-r--r--src/mongo/db/pipeline/document_source_merge_test.cpp229
-rw-r--r--src/mongo/db/pipeline/mongo_process_interface.h8
-rw-r--r--src/mongo/db/pipeline/mongos_process_interface.h2
-rw-r--r--src/mongo/db/pipeline/process_interface_shardsvr.cpp2
-rw-r--r--src/mongo/db/pipeline/process_interface_shardsvr.h2
-rw-r--r--src/mongo/db/pipeline/process_interface_standalone.cpp8
-rw-r--r--src/mongo/db/pipeline/process_interface_standalone.h4
-rw-r--r--src/mongo/db/pipeline/stub_mongo_process_interface.h2
-rw-r--r--src/mongo/db/query/get_executor.cpp8
-rw-r--r--src/mongo/db/query/internal_plans.cpp9
23 files changed, 724 insertions, 515 deletions
diff --git a/src/mongo/db/SConscript b/src/mongo/db/SConscript
index d32fc244d1b..7dcdda1cbf1 100644
--- a/src/mongo/db/SConscript
+++ b/src/mongo/db/SConscript
@@ -1038,6 +1038,7 @@ env.Library(
'exec/text_or.cpp',
'exec/trial_stage.cpp',
'exec/update_stage.cpp',
+ 'exec/upsert_stage.cpp',
'exec/working_set_common.cpp',
'exec/write_stage_common.cpp',
'ops/parsed_delete.cpp',
diff --git a/src/mongo/db/commands/write_commands/write_commands.cpp b/src/mongo/db/commands/write_commands/write_commands.cpp
index 4c9671b9e34..0b1184238ae 100644
--- a/src/mongo/db/commands/write_commands/write_commands.cpp
+++ b/src/mongo/db/commands/write_commands/write_commands.cpp
@@ -364,6 +364,7 @@ private:
updateRequest.setArrayFilters(write_ops::arrayFiltersOf(_batch.getUpdates()[0]));
updateRequest.setMulti(_batch.getUpdates()[0].getMulti());
updateRequest.setUpsert(_batch.getUpdates()[0].getUpsert());
+ updateRequest.setUpsertSuppliedDocument(_batch.getUpdates()[0].getUpsertSupplied());
updateRequest.setYieldPolicy(PlanExecutor::YIELD_AUTO);
updateRequest.setHint(_batch.getUpdates()[0].getHint());
updateRequest.setExplain();
diff --git a/src/mongo/db/exec/update_stage.cpp b/src/mongo/db/exec/update_stage.cpp
index bd19f96ffd5..2ded1a54e58 100644
--- a/src/mongo/db/exec/update_stage.cpp
+++ b/src/mongo/db/exec/update_stage.cpp
@@ -40,7 +40,6 @@
#include "mongo/bson/bson_comparator_interface_base.h"
#include "mongo/bson/mutable/algorithm.h"
#include "mongo/db/concurrency/write_conflict_exception.h"
-#include "mongo/db/curop_failpoint_helpers.h"
#include "mongo/db/exec/scoped_timer.h"
#include "mongo/db/exec/working_set_common.h"
#include "mongo/db/exec/write_stage_common.h"
@@ -58,11 +57,9 @@
#include "mongo/util/assert_util.h"
#include "mongo/util/log.h"
#include "mongo/util/scopeguard.h"
-#include "mongo/util/transitional_tools_do_not_use/vector_spooling.h"
namespace mongo {
-MONGO_FAIL_POINT_DEFINE(hangBeforeUpsertPerformsInsert);
MONGO_FAIL_POINT_DEFINE(hangBeforeThrowWouldChangeOwningShard);
using std::string;
@@ -76,79 +73,13 @@ namespace {
const char idFieldName[] = "_id";
const FieldRef idFieldRef(idFieldName);
-Status ensureIdFieldIsFirst(mb::Document* doc) {
- mb::Element idElem = mb::findFirstChildNamed(doc->root(), idFieldName);
-
- if (!idElem.ok()) {
- return {ErrorCodes::InvalidIdField, "_id field is missing"};
- }
-
- if (idElem.leftSibling().ok()) {
- // Move '_id' to be the first element
- Status s = idElem.remove();
- if (!s.isOK())
- return s;
- s = doc->root().pushFront(idElem);
- if (!s.isOK())
- return s;
- }
-
- return Status::OK();
-}
-
void addObjectIDIdField(mb::Document* doc) {
const auto idElem = doc->makeElementNewOID(idFieldName);
- if (!idElem.ok())
- uasserted(17268, "Could not create new ObjectId '_id' field.");
-
+ uassert(17268, "Could not create new ObjectId '_id' field.", idElem.ok());
uassertStatusOK(doc->root().pushFront(idElem));
}
/**
- * Uasserts if any of the paths in 'requiredPaths' are not present in 'document', or if they are
- * arrays or array descendants.
- */
-void assertRequiredPathsPresent(const mb::Document& document, const FieldRefSet& requiredPaths) {
- for (const auto& path : requiredPaths) {
- auto elem = document.root();
- for (size_t i = 0; i < (*path).numParts(); ++i) {
- elem = elem[(*path).getPart(i)];
- uassert(ErrorCodes::NoSuchKey,
- str::stream() << "After applying the update, the new document was missing the "
- "required field '"
- << (*path).dottedField() << "'",
- elem.ok());
- uassert(
- ErrorCodes::NotSingleValueField,
- str::stream() << "After applying the update to the document, the required field '"
- << (*path).dottedField()
- << "' was found to be an array or array descendant.",
- elem.getType() != BSONType::Array);
- }
- }
-}
-
-/**
- * Uasserts if any of the paths in 'requiredPaths' are arrays or array descendants.
- */
-void assertPathsNotArray(const mb::Document& document, const FieldRefSet& requiredPaths) {
- for (const auto& path : requiredPaths) {
- auto elem = document.root();
- for (size_t i = 0; i < (*path).numParts(); ++i) {
- elem = elem[(*path).getPart(i)];
- if (!elem.ok()) {
- break;
- }
- uassert(ErrorCodes::NotSingleValueField,
- str::stream() << "After applying the update to the document, the field '"
- << (*path).dottedField()
- << "' was found to be an array or array descendant.",
- elem.getType() != BSONType::Array);
- }
- }
-}
-
-/**
* Returns true if we should throw a WriteConflictException in order to retry the operation in the
* case of a conflict. Returns false if we should skip the document and keep going.
*/
@@ -178,19 +109,30 @@ const char* UpdateStage::kStageType = "UPDATE";
const UpdateStats UpdateStage::kEmptyUpdateStats;
+// Public constructor.
UpdateStage::UpdateStage(OperationContext* opCtx,
const UpdateStageParams& params,
WorkingSet* ws,
Collection* collection,
PlanStage* child)
+ : UpdateStage(opCtx, params, ws, collection) {
+ // We should never reach here if the request is an upsert.
+ invariant(!_params.request->isUpsert());
+ _children.emplace_back(child);
+}
+
+// Protected constructor.
+UpdateStage::UpdateStage(OperationContext* opCtx,
+ const UpdateStageParams& params,
+ WorkingSet* ws,
+ Collection* collection)
: RequiresMutableCollectionStage(kStageType, opCtx, collection),
_params(params),
_ws(ws),
+ _doc(params.driver->getDocument()),
_idRetrying(WorkingSet::INVALID_ID),
_idReturning(WorkingSet::INVALID_ID),
- _updatedRecordIds(params.request->isMulti() ? new RecordIdSet() : nullptr),
- _doc(params.driver->getDocument()) {
- _children.emplace_back(child);
+ _updatedRecordIds(params.request->isMulti() ? new RecordIdSet() : nullptr) {
// Should the modifiers validate their embedded docs via storage_validation::storageValid()?
// Only user updates should be checked. Any system or replication stuff should pass through.
@@ -241,9 +183,7 @@ BSONObj UpdateStage::transformAndUpdate(const Snapshotted<BSONObj>& oldObj, Reco
FieldRefSet immutablePaths;
if (getOpCtx()->writesAreReplicated() && !request->isFromMigration()) {
if (metadata->isSharded() && !OperationShardingState::isOperationVersioned(getOpCtx())) {
- auto& immutablePathsVector = metadata->getKeyPatternFields();
- immutablePaths.fillFrom(
- transitional_tools_do_not_use::unspool_vector(immutablePathsVector));
+ immutablePaths.fillFrom(metadata->getKeyPatternFields());
}
immutablePaths.keepShortest(&idFieldRef);
}
@@ -285,16 +225,8 @@ BSONObj UpdateStage::transformAndUpdate(const Snapshotted<BSONObj>& oldObj, Reco
// neither grow nor shrink).
const auto createIdField = !collection()->isCapped();
- // Ensure if _id exists it is first
- status = ensureIdFieldIsFirst(&_doc);
- if (status.code() == ErrorCodes::InvalidIdField) {
- // Create ObjectId _id field if we are doing that
- if (createIdField) {
- addObjectIDIdField(&_doc);
- }
- } else {
- uassertStatusOK(status);
- }
+ // Ensure _id is first if it exists, and generate a new OID if appropriate.
+ _ensureIdFieldIsFirst(&_doc, createIdField);
// See if the changes were applied in place
const char* source = nullptr;
@@ -409,97 +341,19 @@ BSONObj UpdateStage::transformAndUpdate(const Snapshotted<BSONObj>& oldObj, Reco
return newObj;
}
-BSONObj UpdateStage::applyUpdateOpsForInsert(OperationContext* opCtx,
- const CanonicalQuery* cq,
- const BSONObj& query,
- UpdateDriver* driver,
- mutablebson::Document* doc,
- bool isInternalRequest,
- const NamespaceString& ns,
- bool enforceOkForStorage,
- UpdateStats* stats) {
- // Since this is an insert (no docs found and upsert:true), we will be logging it
- // as an insert in the oplog. We don't need the driver's help to build the
- // oplog record, then. We also set the context of the update driver to the INSERT_CONTEXT.
- // Some mods may only work in that context (e.g. $setOnInsert).
- driver->setLogOp(false);
-
- auto* const css = CollectionShardingState::get(opCtx, ns);
- auto metadata = css->getCurrentMetadata();
-
- FieldRefSet immutablePaths;
- if (metadata->isSharded() && !OperationShardingState::isOperationVersioned(opCtx)) {
- auto& immutablePathsVector = metadata->getKeyPatternFields();
- immutablePaths.fillFrom(
- transitional_tools_do_not_use::unspool_vector(immutablePathsVector));
- }
- immutablePaths.keepShortest(&idFieldRef);
-
- if (cq) {
- FieldRefSet requiredPaths;
- if (metadata->isSharded()) {
- const auto& shardKeyPathsVector = metadata->getKeyPatternFields();
- requiredPaths.fillFrom(
- transitional_tools_do_not_use::unspool_vector(shardKeyPathsVector));
- }
- requiredPaths.keepShortest(&idFieldRef);
- uassertStatusOK(driver->populateDocumentWithQueryFields(*cq, requiredPaths, *doc));
- } else {
- fassert(17354, CanonicalQuery::isSimpleIdQuery(query));
- BSONElement idElt = query[idFieldName];
- fassert(17352, doc->root().appendElement(idElt));
- }
-
- // Apply the update modifications here. Do not validate for storage, since we will validate the
- // entire document after the update. However, we ensure that no immutable fields are updated.
- const bool validateForStorage = false;
- const bool isInsert = true;
- if (isInternalRequest) {
- immutablePaths.clear();
- }
- Status updateStatus =
- driver->update(StringData(), doc, validateForStorage, immutablePaths, isInsert);
- if (!updateStatus.isOK()) {
- uasserted(16836, updateStatus.reason());
- }
-
- // Ensure _id exists and is first
- auto idAndFirstStatus = ensureIdFieldIsFirst(doc);
- if (idAndFirstStatus.code() == ErrorCodes::InvalidIdField) { // _id field is missing
- addObjectIDIdField(doc);
- } else {
- uassertStatusOK(idAndFirstStatus);
- }
-
- // Validate that the object replacement or modifiers resulted in a document
- // that contains all the required keys and can be stored if it isn't coming
- // from a migration or via replication.
- if (!isInternalRequest) {
- if (enforceOkForStorage) {
- storage_validation::storageValid(*doc);
- }
-
- // _id must be present and cannot be an array.
- FieldRefSet idFieldSet;
- idFieldSet.keepShortest(&idFieldRef);
- assertRequiredPathsPresent(*doc, idFieldSet);
-
- if (metadata->isSharded()) {
- // The shard key path cannot contain an array.
- FieldRefSet requiredPaths;
- requiredPaths.fillFrom(
- transitional_tools_do_not_use::unspool_vector(metadata->getKeyPatternFields()));
- assertPathsNotArray(*doc, requiredPaths);
+void UpdateStage::_assertPathsNotArray(const mb::Document& document, const FieldRefSet& paths) {
+ for (const auto& path : paths) {
+ auto elem = document.root();
+ // If any path component does not exist, we stop checking for arrays along the path.
+ for (size_t i = 0; elem.ok() && i < (*path).numParts(); ++i) {
+ elem = elem[(*path).getPart(i)];
+ uassert(ErrorCodes::NotSingleValueField,
+ str::stream() << "After applying the update to the document, the field '"
+ << (*path).dottedField()
+ << "' was found to be an array or array descendant.",
+ !elem.ok() || elem.getType() != BSONType::Array);
}
}
-
- BSONObj newObj = doc->getObject();
- if (newObj.objsize() > BSONObjMaxUserSize) {
- uasserted(17420,
- str::stream() << "Document to upsert is larger than " << BSONObjMaxUserSize);
- }
-
- return newObj;
}
bool UpdateStage::matchContainsOnlyAndedEqualityNodes(const MatchExpression& root) {
@@ -578,128 +432,18 @@ bool UpdateStage::shouldRetryDuplicateKeyException(const ParsedUpdate& parsedUpd
return true;
}
-void UpdateStage::doInsert() {
- _specificStats.inserted = true;
-
- const UpdateRequest* request = _params.request;
- bool isInternalRequest = !getOpCtx()->writesAreReplicated() || request->isFromMigration();
-
- // Reset the document we will be writing to.
- _doc.reset();
-
- BSONObj newObj = applyUpdateOpsForInsert(getOpCtx(),
- _params.canonicalQuery,
- request->getQuery(),
- _params.driver,
- &_doc,
- isInternalRequest,
- request->getNamespaceString(),
- _enforceOkForStorage,
- &_specificStats);
-
- _specificStats.objInserted = newObj;
-
- // If this is an explain, bail out now without doing the insert.
- if (request->isExplain()) {
- return;
- }
-
- // If in FCV 4.2 and this collection is sharded, check if the doc we plan to insert belongs to
- // this shard. MongoS uses the query field to target a shard, and it is possible the shard key
- // fields in the 'q' field belong to this shard, but those in the 'u' field do not. In this case
- // we need to throw so that MongoS can target the insert to the correct shard.
- if (_shouldCheckForShardKeyUpdate) {
- auto* const css = CollectionShardingState::get(getOpCtx(), collection()->ns());
- const auto& metadata = css->getCurrentMetadata();
-
- if (metadata->isSharded()) {
- const ShardKeyPattern shardKeyPattern(metadata->getKeyPattern());
- auto newShardKey = shardKeyPattern.extractShardKeyFromDoc(newObj);
-
- if (!metadata->keyBelongsToMe(newShardKey)) {
- // An attempt to upsert a document with a shard key value that belongs on another
- // shard must either be a retryable write or inside a transaction.
- uassert(ErrorCodes::IllegalOperation,
- "The upsert document could not be inserted onto the shard targeted by the "
- "query, since its shard key belongs on a different shard. Cross-shard "
- "upserts are only allowed when running in a transaction or with "
- "retryWrites: true.",
- getOpCtx()->getTxnNumber());
- uasserted(
- WouldChangeOwningShardInfo(request->getQuery(), newObj, true /* upsert */),
- "The document we are inserting belongs on a different shard");
- }
- }
- }
-
- if (MONGO_unlikely(hangBeforeUpsertPerformsInsert.shouldFail())) {
- CurOpFailpointHelpers::waitWhileFailPointEnabled(
- &hangBeforeUpsertPerformsInsert, getOpCtx(), "hangBeforeUpsertPerformsInsert");
- }
-
- writeConflictRetry(getOpCtx(), "upsert", collection()->ns().ns(), [&] {
- WriteUnitOfWork wunit(getOpCtx());
- uassertStatusOK(collection()->insertDocument(getOpCtx(),
- InsertStatement(request->getStmtId(), newObj),
- _params.opDebug,
- request->isFromMigration()));
-
- // Technically, we should save/restore state here, but since we are going to return
- // immediately after, it would just be wasted work.
- wunit.commit();
- });
-}
-
-bool UpdateStage::doneUpdating() {
+bool UpdateStage::isEOF() {
// We're done updating if either the child has no more results to give us, or we've
// already gotten a result back and we're not a multi-update.
return _idRetrying == WorkingSet::INVALID_ID && _idReturning == WorkingSet::INVALID_ID &&
(child()->isEOF() || (_specificStats.nMatched > 0 && !_params.request->isMulti()));
}
-bool UpdateStage::needInsert() {
- // We need to insert if
- // 1) we haven't inserted already,
- // 2) the child stage returned zero matches, and
- // 3) the user asked for an upsert.
- return !_specificStats.inserted && _specificStats.nMatched == 0 && _params.request->isUpsert();
-}
-
-bool UpdateStage::isEOF() {
- return doneUpdating() && !needInsert();
-}
-
PlanStage::StageState UpdateStage::doWork(WorkingSetID* out) {
if (isEOF()) {
return PlanStage::IS_EOF;
}
- if (doneUpdating()) {
- // Even if we're done updating, we may have some inserting left to do.
- if (needInsert()) {
-
- doInsert();
-
- invariant(isEOF());
- if (_params.request->shouldReturnNewDocs()) {
- // Want to return the document we just inserted, create it as a WorkingSetMember
- // so that we can return it.
- BSONObj newObj = _specificStats.objInserted;
- *out = _ws->allocate();
- WorkingSetMember* member = _ws->get(*out);
- member->resetDocument(getOpCtx()->recoveryUnit()->getSnapshotId(),
- newObj.getOwned());
- member->transitionToOwnedObj();
- return PlanStage::ADVANCED;
- }
- }
-
- // At this point either we're done updating and there was no insert to do,
- // or we're done updating and we're done inserting. Either way, we're EOF.
- invariant(isEOF());
- return PlanStage::IS_EOF;
- }
-
// It is possible that after an update was applied, a WriteConflictException
// occurred and prevented us from returning ADVANCED with the requested version
// of the document.
@@ -846,9 +590,8 @@ PlanStage::StageState UpdateStage::doWork(WorkingSetID* out) {
return PlanStage::NEED_TIME;
} else if (PlanStage::IS_EOF == status) {
- // The child is out of results, but we might not be done yet because we still might
- // have to do an insert.
- return PlanStage::NEED_TIME;
+ // The child is out of results, and therefore so are we.
+ return PlanStage::IS_EOF;
} else if (PlanStage::FAILURE == status) {
*out = id;
// If a stage fails, it may create a status WSM to indicate why it failed, in which case
@@ -867,6 +610,19 @@ PlanStage::StageState UpdateStage::doWork(WorkingSetID* out) {
return status;
}
+void UpdateStage::_ensureIdFieldIsFirst(mb::Document* doc, bool generateOIDIfMissing) {
+ mb::Element idElem = mb::findFirstChildNamed(doc->root(), idFieldName);
+
+ // If the document has no _id and the caller has requested that we generate one, do so.
+ if (!idElem.ok() && generateOIDIfMissing) {
+ addObjectIDIdField(doc);
+ } else if (idElem.ok() && idElem.leftSibling().ok()) {
+ // If the document does have an _id but it is not the first element, move it to the front.
+ uassertStatusOK(idElem.remove());
+ uassertStatusOK(doc->root().pushFront(idElem));
+ }
+}
+
void UpdateStage::doRestoreStateRequiresCollection() {
const UpdateRequest& request = *_params.request;
const NamespaceString& nsString(request.getNamespaceString());
@@ -959,12 +715,10 @@ bool UpdateStage::checkUpdateChangesShardKeyFields(ScopedCollectionMetadata meta
return false;
}
- FieldRefSet shardKeyPaths;
- const auto& shardKeyPathsVector = metadata->getKeyPatternFields();
- shardKeyPaths.fillFrom(transitional_tools_do_not_use::unspool_vector(shardKeyPathsVector));
+ FieldRefSet shardKeyPaths(metadata->getKeyPatternFields());
// Assert that the updated doc has no arrays or array descendants for the shard key fields.
- assertPathsNotArray(_doc, shardKeyPaths);
+ _assertPathsNotArray(_doc, shardKeyPaths);
// We do not allow updates to the shard key when 'multi' is true.
uassert(ErrorCodes::InvalidOptions,
diff --git a/src/mongo/db/exec/update_stage.h b/src/mongo/db/exec/update_stage.h
index 24beae7ffaa..cf167d603c0 100644
--- a/src/mongo/db/exec/update_stage.h
+++ b/src/mongo/db/exec/update_stage.h
@@ -69,14 +69,14 @@ private:
};
/**
- * Execution stage responsible for updates to documents and upserts. If the prior or
- * newly-updated version of the document was requested to be returned, then ADVANCED is
- * returned after updating or inserting a document. Otherwise, NEED_TIME is returned after
- * updating or inserting a document.
+ * Execution stage responsible for updates to documents. If the prior or newly-updated version of
+ * the document was requested to be returned, then ADVANCED is returned after updating a document.
+ * Otherwise, NEED_TIME is returned after updating a document if further updates are pending,
+ * and IS_EOF is returned if no documents were found or all updates have been performed.
*
* Callers of doWork() must be holding a write lock.
*/
-class UpdateStage final : public RequiresMutableCollectionStage {
+class UpdateStage : public RequiresMutableCollectionStage {
UpdateStage(const UpdateStage&) = delete;
UpdateStage& operator=(const UpdateStage&) = delete;
@@ -87,8 +87,8 @@ public:
Collection* collection,
PlanStage* child);
- bool isEOF() final;
- StageState doWork(WorkingSetID* out) final;
+ bool isEOF() override;
+ StageState doWork(WorkingSetID* out) override;
StageType stageType() const final {
return STAGE_UPDATE;
@@ -119,32 +119,6 @@ public:
static UpdateResult makeUpdateResult(const UpdateStats* updateStats);
/**
- * Computes the document to insert if the upsert flag is set to true and no matching
- * documents are found in the database. The document to upsert is computing using the
- * query 'cq' and the update mods contained in 'driver'.
- *
- * If 'cq' is NULL, which can happen for the idhack update fast path, then 'query' is
- * used to compute the doc to insert instead of 'cq'.
- *
- * 'doc' is the mutable BSON document which you would like the update driver to use
- * when computing the document to insert.
- *
- * Set 'isInternalRequest' to true if the upsert was issued by the replication or
- * sharding systems.
- *
- * Returns the document to insert.
- */
- static BSONObj applyUpdateOpsForInsert(OperationContext* opCtx,
- const CanonicalQuery* cq,
- const BSONObj& query,
- UpdateDriver* driver,
- mutablebson::Document* doc,
- bool isInternalRequest,
- const NamespaceString& ns,
- bool enforceOkForStorage,
- UpdateStats* stats);
-
- /**
* Returns true if an update failure due to a given DuplicateKey error is eligible for retry.
* Requires that parsedUpdate.hasParsedQuery() is true.
*/
@@ -152,10 +126,37 @@ public:
const DuplicateKeyErrorInfo& errorInfo);
protected:
+ UpdateStage(OperationContext* opCtx,
+ const UpdateStageParams& params,
+ WorkingSet* ws,
+ Collection* collection);
+
void doSaveStateRequiresCollection() final {}
void doRestoreStateRequiresCollection() final;
+ void _ensureIdFieldIsFirst(mutablebson::Document* doc, bool generateOIDIfMissing);
+
+ void _assertPathsNotArray(const mutablebson::Document& document, const FieldRefSet& paths);
+
+ UpdateStageParams _params;
+
+ // Not owned by us.
+ WorkingSet* _ws;
+
+ // Stats
+ UpdateStats _specificStats;
+
+ // True if the request should be checked for an update to the shard key.
+ bool _shouldCheckForShardKeyUpdate;
+
+ // True if updated documents should be validated with storage_validation::storageValid().
+ bool _enforceOkForStorage;
+
+ // These get reused for each update.
+ mutablebson::Document& _doc;
+ mutablebson::DamageVector _damages;
+
private:
static const UpdateStats kEmptyUpdateStats;
@@ -173,24 +174,6 @@ private:
BSONObj transformAndUpdate(const Snapshotted<BSONObj>& oldObj, RecordId& recordId);
/**
- * Computes the document to insert and inserts it into the collection. Used if the
- * user requested an upsert and no matching documents were found.
- */
- void doInsert();
-
- /**
- * Have we performed all necessary updates? Even if this is true, we might not be EOF,
- * as we might still have to do an insert.
- */
- bool doneUpdating();
-
- /**
- * Examines the stats / update request and returns whether there is still an insert left
- * to do. If so then this stage is not EOF yet.
- */
- bool needInsert();
-
- /**
* Stores 'idToRetry' in '_idRetrying' so the update can be retried during the next call to
* doWork(). Always returns NEED_YIELD and sets 'out' to WorkingSet::INVALID_ID.
*/
@@ -210,26 +193,12 @@ private:
bool checkUpdateChangesShardKeyFields(ScopedCollectionMetadata metadata,
const Snapshotted<BSONObj>& oldObj);
- UpdateStageParams _params;
-
- // Not owned by us.
- WorkingSet* _ws;
-
// If not WorkingSet::INVALID_ID, we use this rather than asking our child what to do next.
WorkingSetID _idRetrying;
// If not WorkingSet::INVALID_ID, we return this member to our caller.
WorkingSetID _idReturning;
- // Stats
- UpdateStats _specificStats;
-
- // True if updated documents should be validated with storage_validation::storageValid().
- bool _enforceOkForStorage;
-
- // True if the request should be checked for an update to the shard key.
- bool _shouldCheckForShardKeyUpdate;
-
// If the update was in-place, we may see it again. This only matters if we're doing
// a multi-update; if we're not doing a multi-update we stop after one update and we
// won't see any more docs.
@@ -244,10 +213,6 @@ private:
// So, no matter what, we keep track of where the doc wound up.
typedef stdx::unordered_set<RecordId, RecordId::Hasher> RecordIdSet;
const std::unique_ptr<RecordIdSet> _updatedRecordIds;
-
- // These get reused for each update.
- mutablebson::Document& _doc;
- mutablebson::DamageVector _damages;
};
} // namespace mongo
diff --git a/src/mongo/db/exec/upsert_stage.cpp b/src/mongo/db/exec/upsert_stage.cpp
new file mode 100644
index 00000000000..387613df82f
--- /dev/null
+++ b/src/mongo/db/exec/upsert_stage.cpp
@@ -0,0 +1,294 @@
+/**
+ * Copyright (C) 2019 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/db/exec/upsert_stage.h"
+
+#include "mongo/db/concurrency/write_conflict_exception.h"
+#include "mongo/db/curop_failpoint_helpers.h"
+#include "mongo/db/s/operation_sharding_state.h"
+#include "mongo/db/update/storage_validation.h"
+#include "mongo/s/would_change_owning_shard_exception.h"
+
+namespace mongo {
+
+MONGO_FAIL_POINT_DEFINE(hangBeforeUpsertPerformsInsert);
+
+namespace mb = mutablebson;
+
+namespace {
+
+const char idFieldName[] = "_id";
+const FieldRef idFieldRef(idFieldName);
+
+/**
+ * Populates the given FieldRefSets with the shard key paths (if applicable) and all paths which
+ * are not modifiable, respectively. The contents of these two sets may or may not be identical.
+ */
+void getShardKeyAndImmutablePaths(OperationContext* opCtx,
+ const ScopedCollectionMetadata& metadata,
+ bool isInternalRequest,
+ FieldRefSet* shardKeyPaths,
+ FieldRefSet* immutablePaths) {
+ // If the collection is sharded, add all fields from the shard key to the 'shardKeyPaths' set.
+ if (metadata->isSharded()) {
+ shardKeyPaths->fillFrom(metadata->getKeyPatternFields());
+ }
+ // If this is an internal request, no fields are immutable and we leave 'immutablePaths' empty.
+ if (!isInternalRequest) {
+ // An unversioned request cannot update the shard key, so all shardKey paths are immutable.
+ if (!OperationShardingState::isOperationVersioned(opCtx)) {
+ for (auto&& shardKeyPath : *shardKeyPaths) {
+ immutablePaths->insert(shardKeyPath);
+ }
+ }
+ // The _id field is always immutable to user requests, even if the shard key is mutable.
+ immutablePaths->keepShortest(&idFieldRef);
+ }
+}
+} // namespace
+
+UpsertStage::UpsertStage(OperationContext* opCtx,
+ const UpdateStageParams& params,
+ WorkingSet* ws,
+ Collection* collection,
+ PlanStage* child)
+ : UpdateStage(opCtx, params, ws, collection) {
+ // We should never create this stage for a non-upsert request.
+ invariant(_params.request->isUpsert());
+ _children.emplace_back(child);
+};
+
+// We're done when updating is finished and we have either matched or inserted.
+bool UpsertStage::isEOF() {
+ return UpdateStage::isEOF() && (_specificStats.nMatched > 0 || _specificStats.inserted);
+}
+
+PlanStage::StageState UpsertStage::doWork(WorkingSetID* out) {
+ if (isEOF()) {
+ return StageState::IS_EOF;
+ }
+
+ // First, attempt to perform the update on a matching document.
+ auto updateState = UpdateStage::doWork(out);
+
+ // If the update returned anything other than EOF, just forward it along. There's a chance we
+ // still may find a document to update and will not have to insert anything. If it did return
+ // EOF and we do not need to insert a new document, return EOF immediately here.
+ if (updateState != PlanStage::IS_EOF || isEOF()) {
+ return updateState;
+ }
+
+ // If the update resulted in EOF without matching anything, we must insert a new document.
+ invariant(updateState == PlanStage::IS_EOF && !isEOF());
+
+ // Since this is an insert, we will be logging it as such in the oplog. We don't need the
+ // driver's help to build the oplog record. We also set the 'inserted' stats flag here.
+ _params.driver->setLogOp(false);
+ _specificStats.inserted = true;
+
+ // Determine whether this is a user-initiated or internal request.
+ const bool isInternalRequest =
+ !getOpCtx()->writesAreReplicated() || _params.request->isFromMigration();
+
+ // Generate the new document to be inserted.
+ _specificStats.objInserted = _produceNewDocumentForInsert(isInternalRequest);
+
+ // If this is an explain, skip performing the actual insert.
+ if (!_params.request->isExplain()) {
+ _performInsert(_specificStats.objInserted);
+ }
+
+ // We should always be EOF at this point.
+ invariant(isEOF());
+
+ // If we want to return the document we just inserted, create it as a WorkingSetMember.
+ if (_params.request->shouldReturnNewDocs()) {
+ BSONObj newObj = _specificStats.objInserted;
+ *out = _ws->allocate();
+ WorkingSetMember* member = _ws->get(*out);
+ member->resetDocument(getOpCtx()->recoveryUnit()->getSnapshotId(), newObj.getOwned());
+ member->transitionToOwnedObj();
+ return PlanStage::ADVANCED;
+ }
+
+ // If we don't need to return the inserted document, we're done.
+ return PlanStage::IS_EOF;
+}
+
+void UpsertStage::_performInsert(BSONObj newDocument) {
+ // If this collection is sharded, check if the doc we plan to insert belongs to this shard. The
+ // mongoS uses the query field to target a shard, and it is possible the shard key fields in the
+ // 'q' field belong to this shard, but those in the 'u' field do not. In this case we need to
+ // throw so that MongoS can target the insert to the correct shard.
+ if (_shouldCheckForShardKeyUpdate) {
+ auto* const css = CollectionShardingState::get(getOpCtx(), collection()->ns());
+ const auto& metadata = css->getCurrentMetadata();
+
+ if (metadata->isSharded()) {
+ const ShardKeyPattern shardKeyPattern(metadata->getKeyPattern());
+ auto newShardKey = shardKeyPattern.extractShardKeyFromDoc(newDocument);
+
+ if (!metadata->keyBelongsToMe(newShardKey)) {
+ // An attempt to upsert a document with a shard key value that belongs on another
+ // shard must either be a retryable write or inside a transaction.
+ uassert(ErrorCodes::IllegalOperation,
+ "The upsert document could not be inserted onto the shard targeted by the "
+ "query, since its shard key belongs on a different shard. Cross-shard "
+ "upserts are only allowed when running in a transaction or with "
+ "retryWrites: true.",
+ getOpCtx()->getTxnNumber());
+ uasserted(WouldChangeOwningShardInfo(
+ _params.request->getQuery(), newDocument, true /* upsert */),
+ "The document we are inserting belongs on a different shard");
+ }
+ }
+ }
+
+ if (MONGO_unlikely(hangBeforeUpsertPerformsInsert.shouldFail())) {
+ CurOpFailpointHelpers::waitWhileFailPointEnabled(
+ &hangBeforeUpsertPerformsInsert, getOpCtx(), "hangBeforeUpsertPerformsInsert");
+ }
+
+ writeConflictRetry(getOpCtx(), "upsert", collection()->ns().ns(), [&] {
+ WriteUnitOfWork wunit(getOpCtx());
+ uassertStatusOK(
+ collection()->insertDocument(getOpCtx(),
+ InsertStatement(_params.request->getStmtId(), newDocument),
+ _params.opDebug,
+ _params.request->isFromMigration()));
+
+ // Technically, we should save/restore state here, but since we are going to return
+ // immediately after, it would just be wasted work.
+ wunit.commit();
+ });
+}
+
+BSONObj UpsertStage::_produceNewDocumentForInsert(bool isInternalRequest) {
+ // Obtain the sharding metadata. This will be needed to compute the shardKey paths. The metadata
+ // must remain in scope since it owns the pointers used by 'shardKeyPaths' and 'immutablePaths'.
+ auto* css = CollectionShardingState::get(getOpCtx(), _params.request->getNamespaceString());
+ auto metadata = css->getCurrentMetadata();
+
+ // Compute the set of shard key paths and the set of immutable paths. Either may be empty.
+ FieldRefSet shardKeyPaths, immutablePaths;
+ getShardKeyAndImmutablePaths(
+ getOpCtx(), metadata, isInternalRequest, &shardKeyPaths, &immutablePaths);
+
+ // Reset the document into which we will be writing.
+ _doc.reset();
+
+ // First: populate the document's immutable paths with equality predicate values from the query,
+ // if available. This generates the pre-image document that we will run the update against.
+ if (auto* cq = _params.canonicalQuery) {
+ uassertStatusOK(_params.driver->populateDocumentWithQueryFields(*cq, immutablePaths, _doc));
+ } else {
+ fassert(17354, CanonicalQuery::isSimpleIdQuery(_params.request->getQuery()));
+ fassert(17352, _doc.root().appendElement(_params.request->getQuery()[idFieldName]));
+ }
+
+ // Second: run the appropriate document generation strategy over the document to generate the
+ // post-image. If the update operation modifies any of the immutable paths, this will throw.
+ if (_params.request->shouldUpsertSuppliedDocument()) {
+ _generateNewDocumentFromSuppliedDoc(immutablePaths);
+ } else {
+ _generateNewDocumentFromUpdateOp(immutablePaths);
+ }
+
+ // Third: ensure _id is first if it exists, and generate a new OID otherwise.
+ _ensureIdFieldIsFirst(&_doc, true);
+
+ // Fourth: assert that the finished document has all required fields and is valid for storage.
+ _assertDocumentToBeInsertedIsValid(
+ _doc, shardKeyPaths, isInternalRequest, _enforceOkForStorage);
+
+ // Fifth: validate that the newly-produced document does not exceed the maximum BSON user size.
+ auto newDocument = _doc.getObject();
+ uassert(17420,
+ str::stream() << "Document to upsert is larger than " << BSONObjMaxUserSize,
+ newDocument.objsize() <= BSONObjMaxUserSize);
+
+ return newDocument;
+}
+
+void UpsertStage::_generateNewDocumentFromUpdateOp(const FieldRefSet& immutablePaths) {
+ // Use the UpdateModification from the original request to generate a new document by running
+ // the update over the empty (except for fields extracted from the query) document. We do not
+ // validate for storage until later, but we do ensure that no immutable fields are modified.
+ const bool validateForStorage = false;
+ const bool isInsert = true;
+ uassertStatusOK(
+ _params.driver->update({}, &_doc, validateForStorage, immutablePaths, isInsert));
+};
+
+void UpsertStage::_generateNewDocumentFromSuppliedDoc(const FieldRefSet& immutablePaths) {
+ // We should never call this method unless the request has a set of update constants.
+ invariant(_params.request->shouldUpsertSuppliedDocument());
+ invariant(_params.request->getUpdateConstants());
+
+ // Extract the supplied document from the constants and validate that it is an object.
+ auto suppliedDocElt = _params.request->getUpdateConstants()->getField("new"_sd);
+ invariant(suppliedDocElt.type() == BSONType::Object);
+ auto suppliedDoc = suppliedDocElt.embeddedObject();
+
+ // The supplied doc is functionally a replacement update. We need a new driver to apply it.
+ UpdateDriver replacementDriver(nullptr);
+
+ // Create a new replacement-style update from the supplied document.
+ replacementDriver.parse({suppliedDoc}, {});
+ replacementDriver.setLogOp(false);
+
+ // We do not validate for storage, as we will validate the full document before inserting.
+ // However, we ensure that no immutable fields are modified.
+ const bool validateForStorage = false;
+ const bool isInsert = true;
+ uassertStatusOK(
+ replacementDriver.update({}, &_doc, validateForStorage, immutablePaths, isInsert));
+}
+
+void UpsertStage::_assertDocumentToBeInsertedIsValid(const mb::Document& document,
+ const FieldRefSet& shardKeyPaths,
+ bool isInternalRequest,
+ bool enforceOkForStorage) {
+ // For a non-internal operation, we assert that the document contains all required paths, that
+ // no shard key fields have arrays at any point along their paths, and that the document is
+ // valid for storage. Skip all such checks for an internal operation.
+ if (!isInternalRequest) {
+ if (enforceOkForStorage) {
+ storage_validation::storageValid(document);
+ }
+ // Shard key values are permitted to be missing, and so the only required field is _id. We
+ // should always have an _id here, since we generated one earlier if not already present.
+ invariant(document.root().ok() && document.root()[idFieldName].ok());
+
+ // Neither _id nor the shard key fields may have arrays at any point along their paths.
+ _assertPathsNotArray(document, {{&idFieldRef}});
+ _assertPathsNotArray(document, shardKeyPaths);
+ }
+}
+} // namespace mongo
diff --git a/src/mongo/db/exec/upsert_stage.h b/src/mongo/db/exec/upsert_stage.h
new file mode 100644
index 00000000000..092439bcf3d
--- /dev/null
+++ b/src/mongo/db/exec/upsert_stage.h
@@ -0,0 +1,76 @@
+/**
+ * Copyright (C) 2019 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/exec/update_stage.h"
+
+namespace mongo {
+
+/**
+ * Execution stage for update requests with {upsert:true}. This is a specialized UpdateStage which,
+ * in the event that no documents match the update request's query, generates and inserts a new
+ * document into the collection. All logic related to the insertion phase is implemented by this
+ * class.
+ *
+ * If the prior or newly-updated version of the document was requested to be returned, then ADVANCED
+ * is returned after updating or inserting a document. Otherwise, NEED_TIME is returned after
+ * updating a document if further updates are pending, and IS_EOF is returned if all updates have
+ * been performed or if a document has been inserted.
+ *
+ * Callers of doWork() must be holding a write lock.
+ */
+class UpsertStage final : public UpdateStage {
+ UpsertStage(const UpsertStage&) = delete;
+ UpsertStage& operator=(const UpsertStage&) = delete;
+
+public:
+ UpsertStage(OperationContext* opCtx,
+ const UpdateStageParams& params,
+ WorkingSet* ws,
+ Collection* collection,
+ PlanStage* child);
+
+ bool isEOF() final;
+ StageState doWork(WorkingSetID* out) final;
+
+private:
+ BSONObj _produceNewDocumentForInsert(bool isInternalRequest);
+ void _performInsert(BSONObj newDocument);
+
+ void _generateNewDocumentFromSuppliedDoc(const FieldRefSet& immutablePaths);
+ void _generateNewDocumentFromUpdateOp(const FieldRefSet& immutablePaths);
+
+ void _assertDocumentToBeInsertedIsValid(const mutablebson::Document& document,
+ const FieldRefSet& shardKeyPaths,
+ bool isInternalRequest,
+ bool enforceOkForStorage);
+};
+
+} // namespace mongo
diff --git a/src/mongo/db/field_ref_set.cpp b/src/mongo/db/field_ref_set.cpp
index 79741af83d3..fccde6b2f09 100644
--- a/src/mongo/db/field_ref_set.cpp
+++ b/src/mongo/db/field_ref_set.cpp
@@ -60,6 +60,14 @@ bool FieldRefSet::FieldRefPtrLessThan::operator()(const FieldRef* l, const Field
FieldRefSet::FieldRefSet() {}
+FieldRefSet::FieldRefSet(const std::vector<std::unique_ptr<FieldRef>>& paths) {
+ fillFrom(paths);
+}
+
+FieldRefSet::FieldRefSet(const vector<const FieldRef*>& paths) {
+ _fieldSet.insert(paths.begin(), paths.end());
+}
+
FieldRefSet::FieldRefSet(const vector<FieldRef*>& paths) {
fillFrom(paths);
}
@@ -105,6 +113,19 @@ void FieldRefSet::fillFrom(const std::vector<FieldRef*>& fields) {
_fieldSet.insert(fields.begin(), fields.end());
}
+void FieldRefSet::fillFrom(const std::vector<std::unique_ptr<FieldRef>>& fields) {
+ dassert(_fieldSet.empty());
+ std::transform(fields.begin(),
+ fields.end(),
+ std::inserter(_fieldSet, _fieldSet.begin()),
+ [](const auto& field) { return field.get(); });
+}
+
+bool FieldRefSet::insertNoConflict(const FieldRef* toInsert) {
+ const FieldRef* conflict;
+ return insert(toInsert, &conflict);
+}
+
bool FieldRefSet::insert(const FieldRef* toInsert, const FieldRef** conflict) {
// We can determine if two fields conflict by checking their common prefix.
//
diff --git a/src/mongo/db/field_ref_set.h b/src/mongo/db/field_ref_set.h
index 63166acbc89..0019bf9b4ba 100644
--- a/src/mongo/db/field_ref_set.h
+++ b/src/mongo/db/field_ref_set.h
@@ -64,6 +64,8 @@ public:
FieldRefSet();
+ FieldRefSet(const std::vector<std::unique_ptr<FieldRef>>& paths);
+ FieldRefSet(const std::vector<const FieldRef*>& paths);
FieldRefSet(const std::vector<FieldRef*>& paths);
/** Returns 'true' if the set is empty */
@@ -89,9 +91,9 @@ public:
}
/**
- * Returns true if the field 'toInsert' can be added in the set without
- * conflicts. Otherwise returns false and fill in '*conflict' with the field 'toInsert'
- * clashed with.
+ * Returns true if the field 'toInsert' was added to the set without conflicts.
+ *
+ * Otherwise, returns false and fills '*conflict' with the field 'toInsert' clashed with.
*
* There is no ownership transfer of 'toInsert'. The caller is responsible for
* maintaining it alive for as long as the FieldRefSet is so. By the same token
@@ -100,13 +102,25 @@ public:
bool insert(const FieldRef* toInsert, const FieldRef** conflict);
/**
- * Fills the set with the supplied FieldRef*s
+ * Returns true if the field 'toInsert' was added to the set without conflicts.
+ */
+ bool insertNoConflict(const FieldRef* toInsert);
+
+ /**
+ * Fills the set with the supplied FieldRef pointers.
*
* Note that *no* conflict resolution occurs here.
*/
void fillFrom(const std::vector<FieldRef*>& fields);
/**
+ * Fills the set with the supplied FieldRefs. Does not take ownership of the managed pointers.
+ *
+ * Note that *no* conflict resolution occurs here.
+ */
+ void fillFrom(const std::vector<std::unique_ptr<FieldRef>>& fields);
+
+ /**
* Replace any existing conflicting FieldRef with the shortest (closest to root) one.
*/
void keepShortest(const FieldRef* toInsert);
diff --git a/src/mongo/db/ops/parsed_update.cpp b/src/mongo/db/ops/parsed_update.cpp
index 073ea2d4865..6c68a621d82 100644
--- a/src/mongo/db/ops/parsed_update.cpp
+++ b/src/mongo/db/ops/parsed_update.cpp
@@ -32,6 +32,7 @@
#include "mongo/db/ops/parsed_update.h"
#include "mongo/db/ops/update_request.h"
+#include "mongo/db/ops/write_ops_gen.h"
#include "mongo/db/query/canonical_query.h"
#include "mongo/db/query/collation/collator_factory_interface.h"
#include "mongo/db/query/query_planner_common.h"
@@ -53,6 +54,22 @@ Status ParsedUpdate::parseRequest() {
// of a document during a multi-update.
invariant(!(_request->shouldReturnAnyDocs() && _request->isMulti()));
+ // It is invalid to specify 'upsertSupplied:true' for a non-upsert operation, or if no upsert
+ // document was supplied with the request.
+ if (_request->shouldUpsertSuppliedDocument()) {
+ uassert(ErrorCodes::FailedToParse,
+ str::stream() << "cannot specify '"
+ << write_ops::UpdateOpEntry::kUpsertSuppliedFieldName
+ << ": true' for a non-upsert operation",
+ _request->isUpsert());
+ const auto& constants = _request->getUpdateConstants();
+ uassert(ErrorCodes::FailedToParse,
+ str::stream() << "the parameter '"
+ << write_ops::UpdateOpEntry::kUpsertSuppliedFieldName
+ << "' is set to 'true', but no document was supplied",
+ constants && (*constants)["new"_sd].type() == BSONType::Object);
+ }
+
// It is invalid to request that a ProjectionStage be applied to the UpdateStage if the
// UpdateStage would not return any document.
invariant(_request->getProj().isEmpty() || _request->shouldReturnAnyDocs());
diff --git a/src/mongo/db/ops/update_request.h b/src/mongo/db/ops/update_request.h
index f946302c6b9..5fa0a561b7e 100644
--- a/src/mongo/db/ops/update_request.h
+++ b/src/mongo/db/ops/update_request.h
@@ -56,16 +56,7 @@ public:
RETURN_NEW
};
- inline UpdateRequest(const NamespaceString& nsString)
- : _nsString(nsString),
- _god(false),
- _upsert(false),
- _multi(false),
- _fromMigration(false),
- _fromOplogApplication(false),
- _isExplain(false),
- _returnDocs(ReturnDocOption::RETURN_NONE),
- _yieldPolicy(PlanExecutor::NO_YIELD) {}
+ inline UpdateRequest(const NamespaceString& nsString) : _nsString(nsString) {}
const NamespaceString& getNamespaceString() const {
return _nsString;
@@ -154,6 +145,14 @@ public:
return _upsert;
}
+ inline void setUpsertSuppliedDocument(bool value = true) {
+ _upsertSuppliedDocument = value;
+ }
+
+ bool shouldUpsertSuppliedDocument() const {
+ return _upsertSuppliedDocument;
+ }
+
inline void setMulti(bool value = true) {
_multi = value;
}
@@ -306,22 +305,26 @@ private:
// God bypasses _id checking and index generation. It is only used on behalf of system
// updates, never user updates.
- bool _god;
+ bool _god = false;
// True if this should insert if no matching document is found.
- bool _upsert;
+ bool _upsert = false;
+
+ // True if this upsert operation should insert the document supplied as 'c.new' if the query
+ // does not match any documents.
+ bool _upsertSuppliedDocument = false;
// True if this update is allowed to affect more than one document.
- bool _multi;
+ bool _multi = false;
// True if this update is on behalf of a chunk migration.
- bool _fromMigration;
+ bool _fromMigration = false;
// True if this update was triggered by the application of an oplog entry.
- bool _fromOplogApplication;
+ bool _fromOplogApplication = false;
// Whether or not we are requesting an explained update. Explained updates are read-only.
- bool _isExplain;
+ bool _isExplain = false;
// Specifies which version of the documents to return, if any.
//
@@ -335,10 +338,10 @@ private:
//
// This allows findAndModify to execute an update and retrieve the resulting document
// without another query before or after the update.
- ReturnDocOption _returnDocs;
+ ReturnDocOption _returnDocs = ReturnDocOption::RETURN_NONE;
// Whether or not the update should yield. Defaults to NO_YIELD.
- PlanExecutor::YieldPolicy _yieldPolicy;
+ PlanExecutor::YieldPolicy _yieldPolicy = PlanExecutor::NO_YIELD;
};
} // namespace mongo
diff --git a/src/mongo/db/ops/write_ops.idl b/src/mongo/db/ops/write_ops.idl
index 9409aa73041..42f86b1f481 100644
--- a/src/mongo/db/ops/write_ops.idl
+++ b/src/mongo/db/ops/write_ops.idl
@@ -127,6 +127,11 @@ structs:
operation inserts only a single document."
type: bool
default: false
+ upsertSupplied:
+ description: "Only applicable when upsert is true. If set, and if no documents match
+ the query, the update subsystem will insert the document supplied as
+ 'c.new' rather than generating a new document from the update spec."
+ type: optionalBool
collation:
description: "Specifies the collation to use for the operation."
type: object
diff --git a/src/mongo/db/ops/write_ops_exec.cpp b/src/mongo/db/ops/write_ops_exec.cpp
index 38da11a7d0e..3b1c09b6a62 100644
--- a/src/mongo/db/ops/write_ops_exec.cpp
+++ b/src/mongo/db/ops/write_ops_exec.cpp
@@ -724,6 +724,7 @@ static SingleWriteResult performSingleUpdateOpWithDupKeyRetry(OperationContext*
request.setArrayFilters(write_ops::arrayFiltersOf(op));
request.setMulti(op.getMulti());
request.setUpsert(op.getUpsert());
+ request.setUpsertSuppliedDocument(op.getUpsertSupplied());
request.setHint(op.getHint());
request.setYieldPolicy(opCtx->inMultiDocumentTransaction() ? PlanExecutor::INTERRUPT_ONLY
diff --git a/src/mongo/db/pipeline/document_source_merge.cpp b/src/mongo/db/pipeline/document_source_merge.cpp
index a804dc921f9..9669fc49f22 100644
--- a/src/mongo/db/pipeline/document_source_merge.cpp
+++ b/src/mongo/db/pipeline/document_source_merge.cpp
@@ -59,6 +59,7 @@ using WhenMatched = MergeStrategyDescriptor::WhenMatched;
using WhenNotMatched = MergeStrategyDescriptor::WhenNotMatched;
using BatchTransform = std::function<void(DocumentSourceMerge::BatchedObjects&)>;
using UpdateModification = write_ops::UpdateModification;
+using UpsertType = MongoProcessInterface::UpsertType;
constexpr auto kStageName = DocumentSourceMerge::kStageName;
constexpr auto kDefaultWhenMatched = WhenMatched::kMerge;
@@ -76,12 +77,15 @@ constexpr auto kPipelineInsertMode = MergeMode{WhenMatched::kPipeline, WhenNotMa
constexpr auto kPipelineFailMode = MergeMode{WhenMatched::kPipeline, WhenNotMatched::kFail};
constexpr auto kPipelineDiscardMode = MergeMode{WhenMatched::kPipeline, WhenNotMatched::kDiscard};
+const auto kDefaultPipelineLet = BSON("new"
+ << "$$ROOT");
+
/**
* Creates a merge strategy which uses update semantics to perform a merge operation. If
* 'BatchTransform' function is provided, it will be called to transform batched objects before
* passing them to the 'update'.
*/
-MergeStrategy makeUpdateStrategy(bool upsert, BatchTransform transform) {
+MergeStrategy makeUpdateStrategy(UpsertType upsert, BatchTransform transform) {
return [upsert, transform](
const auto& expCtx, const auto& ns, const auto& wc, auto epoch, auto&& batch) {
if (transform) {
@@ -102,7 +106,7 @@ MergeStrategy makeUpdateStrategy(bool upsert, BatchTransform transform) {
* error. If 'BatchTransform' function is provided, it will be called to transform batched objects
* before passing them to the 'update'.
*/
-MergeStrategy makeStrictUpdateStrategy(bool upsert, BatchTransform transform) {
+MergeStrategy makeStrictUpdateStrategy(UpsertType upsert, BatchTransform transform) {
return [upsert, transform](
const auto& expCtx, const auto& ns, const auto& wc, auto epoch, auto&& batch) {
if (transform) {
@@ -169,44 +173,46 @@ const MergeStrategyDescriptorsMap& getDescriptors() {
{kReplaceInsertMode,
{kReplaceInsertMode,
{ActionType::insert, ActionType::update},
- makeUpdateStrategy(true, {})}},
+ makeUpdateStrategy(UpsertType::kGenerateNewDoc, {})}},
// whenMatched: replace, whenNotMatched: fail
{kReplaceFailMode,
- {kReplaceFailMode, {ActionType::update}, makeStrictUpdateStrategy(false, {})}},
+ {kReplaceFailMode, {ActionType::update}, makeStrictUpdateStrategy(UpsertType::kNone, {})}},
// whenMatched: replace, whenNotMatched: discard
{kReplaceDiscardMode,
- {kReplaceDiscardMode, {ActionType::update}, makeUpdateStrategy(false, {})}},
+ {kReplaceDiscardMode, {ActionType::update}, makeUpdateStrategy(UpsertType::kNone, {})}},
// whenMatched: merge, whenNotMatched: insert
{kMergeInsertMode,
{kMergeInsertMode,
{ActionType::insert, ActionType::update},
- makeUpdateStrategy(true, makeUpdateTransform("$set"))}},
+ makeUpdateStrategy(UpsertType::kGenerateNewDoc, makeUpdateTransform("$set"))}},
// whenMatched: merge, whenNotMatched: fail
{kMergeFailMode,
{kMergeFailMode,
{ActionType::update},
- makeStrictUpdateStrategy(false, makeUpdateTransform("$set"))}},
+ makeStrictUpdateStrategy(UpsertType::kNone, makeUpdateTransform("$set"))}},
// whenMatched: merge, whenNotMatched: discard
{kMergeDiscardMode,
{kMergeDiscardMode,
{ActionType::update},
- makeUpdateStrategy(false, makeUpdateTransform("$set"))}},
+ makeUpdateStrategy(UpsertType::kNone, makeUpdateTransform("$set"))}},
// whenMatched: keepExisting, whenNotMatched: insert
{kKeepExistingInsertMode,
{kKeepExistingInsertMode,
{ActionType::insert, ActionType::update},
- makeUpdateStrategy(true, makeUpdateTransform("$setOnInsert"))}},
+ makeUpdateStrategy(UpsertType::kGenerateNewDoc, makeUpdateTransform("$setOnInsert"))}},
// whenMatched: [pipeline], whenNotMatched: insert
{kPipelineInsertMode,
{kPipelineInsertMode,
{ActionType::insert, ActionType::update},
- makeUpdateStrategy(true, {})}},
+ makeUpdateStrategy(UpsertType::kInsertSuppliedDoc, {})}},
// whenMatched: [pipeline], whenNotMatched: fail
{kPipelineFailMode,
- {kPipelineFailMode, {ActionType::update}, makeStrictUpdateStrategy(false, {})}},
+ {kPipelineFailMode,
+ {ActionType::update},
+ makeStrictUpdateStrategy(UpsertType::kNone, {})}},
// whenMatched: [pipeline], whenNotMatched: discard
{kPipelineDiscardMode,
- {kPipelineDiscardMode, {ActionType::update}, makeUpdateStrategy(false, {})}},
+ {kPipelineDiscardMode, {ActionType::update}, makeUpdateStrategy(UpsertType::kNone, {})}},
// whenMatched: fail, whenNotMatched: insert
{kFailInsertMode, {kFailInsertMode, {ActionType::insert}, makeInsertStrategy()}}};
return mergeStrategyDescriptors;
@@ -386,11 +392,16 @@ boost::intrusive_ptr<DocumentSource> DocumentSourceMerge::create(
!outputNs.isOnInternalDb());
if (whenMatched == WhenMatched::kPipeline) {
- if (!letVariables) {
- // For custom pipeline-style updates, default the 'let' variables to {new: "$$ROOT"},
- // if the user has omitted the 'let' argument.
- letVariables = BSON("new"
- << "$$ROOT");
+ // If unspecified, 'letVariables' defaults to {new: "$$ROOT"}.
+ letVariables = letVariables.value_or(kDefaultPipelineLet);
+ auto newElt = letVariables->getField("new"_sd);
+ uassert(51273,
+ "'let' may not define a value for the reserved 'new' variable other than '$$ROOT'",
+ !newElt || newElt.valueStringDataSafe() == "$$ROOT"_sd);
+ // If the 'new' variable is missing and this is a {whenNotMatched: "insert"} merge, then the
+ // new document *must* be serialized with the update request. Add it to the let variables.
+ if (!newElt && whenNotMatched == WhenNotMatched::kInsert) {
+ letVariables = letVariables->addField(kDefaultPipelineLet.firstElement());
}
} else {
// Ensure the 'let' argument cannot be used with any other merge modes.
diff --git a/src/mongo/db/pipeline/document_source_merge_test.cpp b/src/mongo/db/pipeline/document_source_merge_test.cpp
index f46df8bd212..b05a30e1507 100644
--- a/src/mongo/db/pipeline/document_source_merge_test.cpp
+++ b/src/mongo/db/pipeline/document_source_merge_test.cpp
@@ -729,20 +729,23 @@ TEST_F(DocumentSourceMergeTest, LetVariablesCanOnlyBeUsedWithPipelineMode) {
ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, 51199);
}
+// We always serialize the default let variables as {new: "$$ROOT"} if omitted.
TEST_F(DocumentSourceMergeTest, SerializeDefaultLetVariable) {
- auto spec =
- BSON("$merge" << BSON("into"
- << "target_collection"
- << "whenMatched" << BSON_ARRAY(BSON("$project" << BSON("x" << 1)))
- << "whenNotMatched"
- << "insert"));
- auto mergeStage = createMergeStage(spec);
- auto serialized = mergeStage->serialize().getDocument();
- ASSERT_VALUE_EQ(serialized["$merge"]["let"],
- Value(BSON("new"
- << "$$ROOT")));
+ for (auto&& whenNotMatched : {"insert", "fail", "discard"}) {
+ auto spec =
+ BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "whenMatched" << BSON_ARRAY(BSON("$project" << BSON("x" << 1)))
+ << "whenNotMatched" << whenNotMatched));
+ auto mergeStage = createMergeStage(spec);
+ auto serialized = mergeStage->serialize().getDocument();
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"],
+ Value(BSON("new"
+ << "$$ROOT")));
+ }
}
+// Test the behaviour of 'let' serialization for each whenNotMatched mode.
TEST_F(DocumentSourceMergeTest, SerializeLetVariables) {
auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
<< "$$v1"
@@ -750,50 +753,70 @@ TEST_F(DocumentSourceMergeTest, SerializeLetVariables) {
<< "$$v2"
<< "z"
<< "$$v3")));
- auto spec = BSON("$merge" << BSON("into"
- << "target_collection"
- << "let"
- << BSON("v1" << 10 << "v2"
- << "foo"
- << "v3"
- << BSON("x" << 1 << "y"
- << BSON("z"
- << "bar")))
- << "whenMatched" << pipeline << "whenNotMatched"
- << "insert"));
- auto mergeStage = createMergeStage(spec);
- ASSERT(mergeStage);
- auto serialized = mergeStage->serialize().getDocument();
- ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v1"], Value(BSON("$const" << 10)));
- ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v2"],
- Value(BSON("$const"
- << "foo")));
- ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v3"],
- Value(BSON("x" << BSON("$const" << 1) << "y"
- << BSON("z" << BSON("$const"
- << "bar")))));
- ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+
+ const auto createAndSerializeMergeStage = [this, &pipeline](StringData whenNotMatched) {
+ auto spec = BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "let"
+ << BSON("v1" << 10 << "v2"
+ << "foo"
+ << "v3"
+ << BSON("x" << 1 << "y"
+ << BSON("z"
+ << "bar")))
+ << "whenMatched" << pipeline << "whenNotMatched"
+ << whenNotMatched));
+ auto mergeStage = createMergeStage(spec);
+ ASSERT(mergeStage);
+
+ return mergeStage->serialize().getDocument();
+ };
+
+ for (auto&& whenNotMatched : {"insert", "fail", "discard"}) {
+ const auto serialized = createAndSerializeMergeStage(whenNotMatched);
+
+ // For {whenNotMatched:insert}, we always attach the 'new' document even if the user has
+ // already specified a set of variables. This is because a {whenNotMatched: insert} merge
+ // generates an upsert, and if no documents in the target collection match the query we must
+ // insert the original document. For other 'whenNotMatched' modes, we do not serialize the
+ // new document, since neither 'fail' nor 'discard' can result in an upsert.
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"]["new"],
+ (whenNotMatched == "insert"_sd ? Value("$$ROOT"_sd) : Value()));
+
+ // The user's variables should be serialized in all cases.
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v1"], Value(BSON("$const" << 10)));
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v2"],
+ Value(BSON("$const"
+ << "foo")));
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v3"],
+ Value(BSON("x" << BSON("$const" << 1) << "y"
+ << BSON("z" << BSON("$const"
+ << "bar")))));
+ ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ }
}
TEST_F(DocumentSourceMergeTest, SerializeLetArrayVariable) {
- auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
- << "$$v1")));
- auto spec =
- BSON("$merge" << BSON("into"
- << "target_collection"
- << "let"
- << BSON("v1" << BSON_ARRAY(1 << "2" << BSON("x" << 1 << "y" << 2)))
- << "whenMatched" << pipeline << "whenNotMatched"
- << "insert"));
- auto mergeStage = createMergeStage(spec);
- ASSERT(mergeStage);
- auto serialized = mergeStage->serialize().getDocument();
- ASSERT_VALUE_EQ(serialized["$merge"]["let"]["v1"],
- Value(BSON_ARRAY(BSON("$const" << 1) << BSON("$const"
- << "2")
- << BSON("x" << BSON("$const" << 1) << "y"
- << BSON("$const" << 2)))));
- ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ for (auto&& whenNotMatched : {"insert", "fail", "discard"}) {
+ auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
+ << "$$v1")));
+ auto spec = BSON(
+ "$merge" << BSON("into"
+ << "target_collection"
+ << "let"
+ << BSON("v1" << BSON_ARRAY(1 << "2" << BSON("x" << 1 << "y" << 2)))
+ << "whenMatched" << pipeline << "whenNotMatched" << whenNotMatched));
+ auto mergeStage = createMergeStage(spec);
+ ASSERT(mergeStage);
+ auto serialized = mergeStage->serialize().getDocument();
+ ASSERT_VALUE_EQ(
+ serialized["$merge"]["let"]["v1"],
+ Value(BSON_ARRAY(BSON("$const" << 1)
+ << BSON("$const"
+ << "2")
+ << BSON("x" << BSON("$const" << 1) << "y" << BSON("$const" << 2)))));
+ ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ }
}
// This test verifies that when the 'let' argument is specified as 'null', the default 'new'
@@ -803,60 +826,66 @@ TEST_F(DocumentSourceMergeTest, SerializeLetArrayVariable) {
// this test ensures that we're aware of this limitation. Once the limitation is addressed in
// SERVER-41272, this test should be updated to accordingly.
TEST_F(DocumentSourceMergeTest, SerializeNullLetVariablesAsDefault) {
- auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
- << "1")));
- auto spec =
- BSON("$merge" << BSON("into"
- << "target_collection"
- << "let" << BSONNULL << "whenMatched" << pipeline << "whenNotMatched"
- << "insert"));
- auto mergeStage = createMergeStage(spec);
- ASSERT(mergeStage);
- auto serialized = mergeStage->serialize().getDocument();
- ASSERT_VALUE_EQ(serialized["$merge"]["let"],
- Value(BSON("new"
- << "$$ROOT")));
- ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ for (auto&& whenNotMatched : {"insert", "fail", "discard"}) {
+ auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
+ << "1")));
+ auto spec = BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "let" << BSONNULL << "whenMatched" << pipeline
+ << "whenNotMatched" << whenNotMatched));
+ auto mergeStage = createMergeStage(spec);
+ ASSERT(mergeStage);
+ auto serialized = mergeStage->serialize().getDocument();
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"],
+ Value(BSON("new"
+ << "$$ROOT")));
+ ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ }
}
TEST_F(DocumentSourceMergeTest, SerializeEmptyLetVariables) {
- auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
- << "1")));
- auto spec =
- BSON("$merge" << BSON("into"
- << "target_collection"
- << "let" << BSONObj() << "whenMatched" << pipeline << "whenNotMatched"
- << "insert"));
- auto mergeStage = createMergeStage(spec);
- ASSERT(mergeStage);
- auto serialized = mergeStage->serialize().getDocument();
- ASSERT_VALUE_EQ(serialized["$merge"]["let"], Value(BSONObj()));
- ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ for (auto&& whenNotMatched : {"insert", "fail", "discard"}) {
+ auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
+ << "1")));
+ auto spec = BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "let" << BSONObj() << "whenMatched" << pipeline
+ << "whenNotMatched" << whenNotMatched));
+ auto mergeStage = createMergeStage(spec);
+ ASSERT(mergeStage);
+ auto serialized = mergeStage->serialize().getDocument();
+ ASSERT_VALUE_EQ(serialized["$merge"]["let"],
+ (whenNotMatched == "insert"_sd ? Value(BSON("new"
+ << "$$ROOT"))
+ : Value(BSONObj())));
+ ASSERT_VALUE_EQ(serialized["$merge"]["whenMatched"], Value(pipeline));
+ }
}
TEST_F(DocumentSourceMergeTest, OnlyObjectCanBeUsedAsLetVariables) {
- auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
- << "1")));
- auto spec = BSON("$merge" << BSON("into"
- << "target_collection"
- << "let" << 1 << "whenMatched" << pipeline << "whenNotMatched"
- << "insert"));
- ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, ErrorCodes::TypeMismatch);
-
- spec = BSON("$merge" << BSON("into"
- << "target_collection"
- << "let"
- << "foo"
- << "whenMatched" << pipeline << "whenNotMatched"
- << "insert"));
- ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, ErrorCodes::TypeMismatch);
-
- spec = BSON("$merge" << BSON("into"
- << "target_collection"
- << "let" << BSON_ARRAY(1 << "2") << "whenMatched" << pipeline
- << "whenNotMatched"
- << "insert"));
- ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, ErrorCodes::TypeMismatch);
+ for (auto&& whenNotMatched : {"insert", "fail", "discard"}) {
+ auto pipeline = BSON_ARRAY(BSON("$project" << BSON("x"
+ << "1")));
+ auto spec = BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "let" << 1 << "whenMatched" << pipeline
+ << "whenNotMatched" << whenNotMatched));
+ ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, ErrorCodes::TypeMismatch);
+
+ spec = BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "let"
+ << "foo"
+ << "whenMatched" << pipeline << "whenNotMatched"
+ << whenNotMatched));
+ ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, ErrorCodes::TypeMismatch);
+
+ spec = BSON("$merge" << BSON("into"
+ << "target_collection"
+ << "let" << BSON_ARRAY(1 << "2") << "whenMatched" << pipeline
+ << "whenNotMatched" << whenNotMatched));
+ ASSERT_THROWS_CODE(createMergeStage(spec), AssertionException, ErrorCodes::TypeMismatch);
+ }
}
} // namespace
diff --git a/src/mongo/db/pipeline/mongo_process_interface.h b/src/mongo/db/pipeline/mongo_process_interface.h
index b61556bf560..cbe85447f19 100644
--- a/src/mongo/db/pipeline/mongo_process_interface.h
+++ b/src/mongo/db/pipeline/mongo_process_interface.h
@@ -85,6 +85,12 @@ public:
std::tuple<BSONObj, write_ops::UpdateModification, boost::optional<BSONObj>>;
using BatchedObjects = std::vector<BatchObject>;
+ enum class UpsertType {
+ kNone, // This operation is not an upsert.
+ kGenerateNewDoc, // If no documents match, generate a new document using the update spec.
+ kInsertSuppliedDoc // If no documents match, insert the document supplied in 'c.new' as-is.
+ };
+
enum class CurrentOpConnectionsMode { kIncludeIdle, kExcludeIdle };
enum class CurrentOpUserMode { kIncludeAll, kExcludeOthers };
enum class CurrentOpTruncateMode { kNoTruncation, kTruncateOps };
@@ -166,7 +172,7 @@ public:
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID> targetEpoch) = 0;
diff --git a/src/mongo/db/pipeline/mongos_process_interface.h b/src/mongo/db/pipeline/mongos_process_interface.h
index c896402dbb8..f94e286d6c0 100644
--- a/src/mongo/db/pipeline/mongos_process_interface.h
+++ b/src/mongo/db/pipeline/mongos_process_interface.h
@@ -85,7 +85,7 @@ public:
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID>) final {
MONGO_UNREACHABLE;
diff --git a/src/mongo/db/pipeline/process_interface_shardsvr.cpp b/src/mongo/db/pipeline/process_interface_shardsvr.cpp
index 896de887cb7..b8b2a31c793 100644
--- a/src/mongo/db/pipeline/process_interface_shardsvr.cpp
+++ b/src/mongo/db/pipeline/process_interface_shardsvr.cpp
@@ -135,7 +135,7 @@ StatusWith<MongoProcessInterface::UpdateResult> MongoInterfaceShardServer::updat
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID> targetEpoch) {
BatchedCommandResponse response;
diff --git a/src/mongo/db/pipeline/process_interface_shardsvr.h b/src/mongo/db/pipeline/process_interface_shardsvr.h
index 4dbc21b6ca2..8b4666c9b55 100644
--- a/src/mongo/db/pipeline/process_interface_shardsvr.h
+++ b/src/mongo/db/pipeline/process_interface_shardsvr.h
@@ -78,7 +78,7 @@ public:
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID> targetEpoch) final;
diff --git a/src/mongo/db/pipeline/process_interface_standalone.cpp b/src/mongo/db/pipeline/process_interface_standalone.cpp
index 5607f42279d..c26d1642b1d 100644
--- a/src/mongo/db/pipeline/process_interface_standalone.cpp
+++ b/src/mongo/db/pipeline/process_interface_standalone.cpp
@@ -187,7 +187,7 @@ Update MongoInterfaceStandalone::buildUpdateOp(
const boost::intrusive_ptr<ExpressionContext>& expCtx,
const NamespaceString& nss,
BatchedObjects&& batch,
- bool upsert,
+ UpsertType upsert,
bool multi) {
Update updateOp(nss);
updateOp.setUpdates([&] {
@@ -199,7 +199,9 @@ Update MongoInterfaceStandalone::buildUpdateOp(
entry.setQ(std::move(q));
entry.setU(std::move(u));
entry.setC(std::move(c));
- entry.setUpsert(upsert);
+ entry.setUpsert(upsert != UpsertType::kNone);
+ entry.setUpsertSupplied(
+ {{entry.getUpsert(), upsert == UpsertType::kInsertSuppliedDoc}});
entry.setMulti(multi);
return entry;
}());
@@ -238,7 +240,7 @@ StatusWith<MongoProcessInterface::UpdateResult> MongoInterfaceStandalone::update
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID> targetEpoch) {
auto writeResults =
diff --git a/src/mongo/db/pipeline/process_interface_standalone.h b/src/mongo/db/pipeline/process_interface_standalone.h
index a2943813813..c045260e3e0 100644
--- a/src/mongo/db/pipeline/process_interface_standalone.h
+++ b/src/mongo/db/pipeline/process_interface_standalone.h
@@ -73,7 +73,7 @@ public:
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID> targetEpoch) override;
@@ -183,7 +183,7 @@ protected:
Update buildUpdateOp(const boost::intrusive_ptr<ExpressionContext>& expCtx,
const NamespaceString& nss,
BatchedObjects&& batch,
- bool upsert,
+ UpsertType upsert,
bool multi);
private:
diff --git a/src/mongo/db/pipeline/stub_mongo_process_interface.h b/src/mongo/db/pipeline/stub_mongo_process_interface.h
index 269ed49f328..381da454c5d 100644
--- a/src/mongo/db/pipeline/stub_mongo_process_interface.h
+++ b/src/mongo/db/pipeline/stub_mongo_process_interface.h
@@ -75,7 +75,7 @@ public:
const NamespaceString& ns,
BatchedObjects&& batch,
const WriteConcernOptions& wc,
- bool upsert,
+ UpsertType upsert,
bool multi,
boost::optional<OID>) final {
MONGO_UNREACHABLE;
diff --git a/src/mongo/db/query/get_executor.cpp b/src/mongo/db/query/get_executor.cpp
index 070218d9efd..e7f0513fbbc 100644
--- a/src/mongo/db/query/get_executor.cpp
+++ b/src/mongo/db/query/get_executor.cpp
@@ -54,6 +54,7 @@
#include "mongo/db/exec/sort_key_generator.h"
#include "mongo/db/exec/subplan.h"
#include "mongo/db/exec/update_stage.h"
+#include "mongo/db/exec/upsert_stage.h"
#include "mongo/db/index/index_descriptor.h"
#include "mongo/db/index/wildcard_access_method.h"
#include "mongo/db/index_names.h"
@@ -938,8 +939,11 @@ StatusWith<unique_ptr<PlanExecutor, PlanExecutor::Deleter>> getExecutorUpdate(
invariant(root);
updateStageParams.canonicalQuery = cq.get();
- root = std::make_unique<UpdateStage>(
- opCtx, updateStageParams, ws.get(), collection, root.release());
+ const bool isUpsert = updateStageParams.request->isUpsert();
+ root = (isUpsert ? std::make_unique<UpsertStage>(
+ opCtx, updateStageParams, ws.get(), collection, root.release())
+ : std::make_unique<UpdateStage>(
+ opCtx, updateStageParams, ws.get(), collection, root.release()));
if (!request->getProj().isEmpty()) {
invariant(request->shouldReturnAnyDocs());
diff --git a/src/mongo/db/query/internal_plans.cpp b/src/mongo/db/query/internal_plans.cpp
index 53328901c88..a9bc1b5021a 100644
--- a/src/mongo/db/query/internal_plans.cpp
+++ b/src/mongo/db/query/internal_plans.cpp
@@ -42,6 +42,7 @@
#include "mongo/db/exec/idhack.h"
#include "mongo/db/exec/index_scan.h"
#include "mongo/db/exec/update_stage.h"
+#include "mongo/db/exec/upsert_stage.h"
#include "mongo/db/query/get_executor.h"
namespace mongo {
@@ -166,8 +167,12 @@ std::unique_ptr<PlanExecutor, PlanExecutor::Deleter> InternalPlanner::updateWith
auto ws = std::make_unique<WorkingSet>();
auto idHackStage = std::make_unique<IDHackStage>(opCtx, key, ws.get(), descriptor);
- auto root =
- std::make_unique<UpdateStage>(opCtx, params, ws.get(), collection, idHackStage.release());
+
+ const bool isUpsert = params.request->isUpsert();
+ auto root = (isUpsert ? std::make_unique<UpsertStage>(
+ opCtx, params, ws.get(), collection, idHackStage.release())
+ : std::make_unique<UpdateStage>(
+ opCtx, params, ws.get(), collection, idHackStage.release()));
auto executor =
PlanExecutor::make(opCtx, std::move(ws), std::move(root), collection, yieldPolicy);