From 23e55cb3d041236f399f7095df31cd3e3da491cc Mon Sep 17 00:00:00 2001 From: Bernard Gorman Date: Thu, 14 Nov 2019 21:59:35 +0000 Subject: SERVER-43860 Always upsert exact source document for pipeline-insert $merge --- src/mongo/db/SConscript | 1 + .../db/commands/write_commands/write_commands.cpp | 1 + src/mongo/db/exec/update_stage.cpp | 340 +++------------------ src/mongo/db/exec/update_stage.h | 103 +++---- src/mongo/db/exec/upsert_stage.cpp | 294 ++++++++++++++++++ src/mongo/db/exec/upsert_stage.h | 76 +++++ src/mongo/db/field_ref_set.cpp | 21 ++ src/mongo/db/field_ref_set.h | 22 +- src/mongo/db/ops/parsed_update.cpp | 17 ++ src/mongo/db/ops/update_request.h | 39 +-- src/mongo/db/ops/write_ops.idl | 5 + src/mongo/db/ops/write_ops_exec.cpp | 1 + src/mongo/db/pipeline/document_source_merge.cpp | 45 +-- .../db/pipeline/document_source_merge_test.cpp | 229 ++++++++------ src/mongo/db/pipeline/mongo_process_interface.h | 8 +- src/mongo/db/pipeline/mongos_process_interface.h | 2 +- .../db/pipeline/process_interface_shardsvr.cpp | 2 +- src/mongo/db/pipeline/process_interface_shardsvr.h | 2 +- .../db/pipeline/process_interface_standalone.cpp | 8 +- .../db/pipeline/process_interface_standalone.h | 4 +- .../db/pipeline/stub_mongo_process_interface.h | 2 +- src/mongo/db/query/get_executor.cpp | 8 +- src/mongo/db/query/internal_plans.cpp | 9 +- 23 files changed, 724 insertions(+), 515 deletions(-) create mode 100644 src/mongo/db/exec/upsert_stage.cpp create mode 100644 src/mongo/db/exec/upsert_stage.h (limited to 'src/mongo/db') 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,78 +73,12 @@ 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& 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& 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& 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; @@ -118,32 +118,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; @@ -172,24 +173,6 @@ private: */ BSONObj transformAndUpdate(const Snapshotted& 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& 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 RecordIdSet; const std::unique_ptr _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 + * . + * + * 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 + * . + * + * 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>& paths) { + fillFrom(paths); +} + +FieldRefSet::FieldRefSet(const vector& paths) { + _fieldSet.insert(paths.begin(), paths.end()); +} + FieldRefSet::FieldRefSet(const vector& paths) { fillFrom(paths); } @@ -105,6 +113,19 @@ void FieldRefSet::fillFrom(const std::vector& fields) { _fieldSet.insert(fields.begin(), fields.end()); } +void FieldRefSet::fillFrom(const std::vector>& 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>& paths); + FieldRefSet(const std::vector& paths); FieldRefSet(const std::vector& 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,12 +102,24 @@ 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& 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>& fields); + /** * Replace any existing conflicting FieldRef with the shortest (closest to root) one. */ 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; 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 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>; using BatchedObjects = std::vector; + 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 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) 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 MongoInterfaceShardServer::updat const NamespaceString& ns, BatchedObjects&& batch, const WriteConcernOptions& wc, - bool upsert, + UpsertType upsert, bool multi, boost::optional 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 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& 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 MongoInterfaceStandalone::update const NamespaceString& ns, BatchedObjects&& batch, const WriteConcernOptions& wc, - bool upsert, + UpsertType upsert, bool multi, boost::optional 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 targetEpoch) override; @@ -183,7 +183,7 @@ protected: Update buildUpdateOp(const boost::intrusive_ptr& 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) 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> getExecutorUpdate( invariant(root); updateStageParams.canonicalQuery = cq.get(); - root = std::make_unique( - opCtx, updateStageParams, ws.get(), collection, root.release()); + const bool isUpsert = updateStageParams.request->isUpsert(); + root = (isUpsert ? std::make_unique( + opCtx, updateStageParams, ws.get(), collection, root.release()) + : std::make_unique( + 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 InternalPlanner::updateWith auto ws = std::make_unique(); auto idHackStage = std::make_unique(opCtx, key, ws.get(), descriptor); - auto root = - std::make_unique(opCtx, params, ws.get(), collection, idHackStage.release()); + + const bool isUpsert = params.request->isUpsert(); + auto root = (isUpsert ? std::make_unique( + opCtx, params, ws.get(), collection, idHackStage.release()) + : std::make_unique( + opCtx, params, ws.get(), collection, idHackStage.release())); auto executor = PlanExecutor::make(opCtx, std::move(ws), std::move(root), collection, yieldPolicy); -- cgit v1.2.1