summaryrefslogtreecommitdiff
path: root/src/mongo/db/ops/update.cpp
diff options
context:
space:
mode:
authorDavid Storch <david.storch@10gen.com>2014-08-01 13:15:36 -0400
committerDavid Storch <david.storch@10gen.com>2014-08-11 16:51:47 -0400
commitc184143fa4d8a4fdf4fdc684404d4aad3e55794b (patch)
tree2948d8aa90ccff4528a7a441ec6ad0e8df6e07cf /src/mongo/db/ops/update.cpp
parent2a90d1e20230800f8dc5a01ea066873354fcd938 (diff)
downloadmongo-c184143fa4d8a4fdf4fdc684404d4aad3e55794b.tar.gz
SERVER-14497 UpdateStage
Diffstat (limited to 'src/mongo/db/ops/update.cpp')
-rw-r--r--src/mongo/db/ops/update.cpp734
1 files changed, 40 insertions, 694 deletions
diff --git a/src/mongo/db/ops/update.cpp b/src/mongo/db/ops/update.cpp
index 921f7fd02e6..f1af1023d86 100644
--- a/src/mongo/db/ops/update.cpp
+++ b/src/mongo/db/ops/update.cpp
@@ -32,39 +32,26 @@
#include "mongo/db/ops/update.h"
-#include <cstring> // for memcpy
-
-#include "mongo/bson/mutable/algorithm.h"
-#include "mongo/bson/mutable/document.h"
#include "mongo/client/dbclientinterface.h"
#include "mongo/db/catalog/collection.h"
#include "mongo/db/catalog/database_holder.h"
#include "mongo/db/clientcursor.h"
+#include "mongo/db/exec/update.h"
#include "mongo/db/operation_context_impl.h"
#include "mongo/db/ops/update_driver.h"
#include "mongo/db/ops/update_executor.h"
#include "mongo/db/ops/update_lifecycle.h"
#include "mongo/db/query/explain.h"
#include "mongo/db/query/get_executor.h"
-#include "mongo/db/query/lite_parsed_query.h"
-#include "mongo/db/query/query_planner_common.h"
-#include "mongo/db/repl/oplog.h"
-#include "mongo/db/repl/repl_coordinator_global.h"
#include "mongo/db/update_index_data.h"
-#include "mongo/platform/unordered_set.h"
#include "mongo/util/log.h"
namespace mongo {
MONGO_LOG_DEFAULT_COMPONENT_FILE(::mongo::logger::LogComponent::kQuery);
- namespace mb = mutablebson;
-
namespace {
- const char idFieldName[] = "_id";
- const FieldRef idFieldRef(idFieldName);
-
// TODO: Make this a function on NamespaceString, or make it cleaner.
inline void validateUpdate(const char* ns ,
const BSONObj& updateobj,
@@ -80,345 +67,6 @@ namespace mongo {
}
}
- Status storageValid(const mb::Document&, const bool);
- Status storageValid(const mb::ConstElement&, const bool);
- Status storageValidChildren(const mb::ConstElement&, const bool);
-
- /**
- * mutable::document storageValid check -- like BSONObj::_okForStorage
- */
- Status storageValid(const mb::Document& doc, const bool deep = true) {
- mb::ConstElement currElem = doc.root().leftChild();
- while (currElem.ok()) {
- if (currElem.getFieldName() == idFieldName) {
- switch (currElem.getType()) {
- case RegEx:
- case Array:
- case Undefined:
- return Status(ErrorCodes::InvalidIdField,
- str::stream() << "The '_id' value cannot be of type "
- << typeName(currElem.getType()));
- default:
- break;
- }
- }
- Status s = storageValid(currElem, deep);
- if (!s.isOK())
- return s;
- currElem = currElem.rightSibling();
- }
-
- return Status::OK();
- }
-
- /**
- * Validates an element that has a field name which starts with a dollar sign ($).
- * In the case of a DBRef field ($id, $ref, [$db]) these fields may be valid in
- * the correct order/context only.
- */
- Status validateDollarPrefixElement(const mb::ConstElement elem, const bool deep) {
- mb::ConstElement curr = elem;
- StringData currName = elem.getFieldName();
-
- // Found a $db field
- if (currName == "$db") {
- if (curr.getType() != String) {
- return Status(ErrorCodes::InvalidDBRef,
- str::stream() << "The DBRef $db field must be a String, not a "
- << typeName(curr.getType()));
- }
- curr = curr.leftSibling();
-
- if (!curr.ok() || (curr.getFieldName() != "$id"))
- return Status(ErrorCodes::InvalidDBRef,
- "Found $db field without a $id before it, which is invalid.");
-
- currName = curr.getFieldName();
- }
-
- // Found a $id field
- if (currName == "$id") {
- Status s = storageValidChildren(curr, deep);
- if (!s.isOK())
- return s;
-
- curr = curr.leftSibling();
- if (!curr.ok() || (curr.getFieldName() != "$ref")) {
- return Status(ErrorCodes::InvalidDBRef,
- "Found $id field without a $ref before it, which is invalid.");
- }
-
- currName = curr.getFieldName();
- }
-
- if (currName == "$ref") {
- if (curr.getType() != String) {
- return Status(ErrorCodes::InvalidDBRef,
- str::stream() << "The DBRef $ref field must be a String, not a "
- << typeName(curr.getType()));
- }
-
- if (!curr.rightSibling().ok() || curr.rightSibling().getFieldName() != "$id")
- return Status(ErrorCodes::InvalidDBRef,
- str::stream() << "The DBRef $ref field must be "
- "following by a $id field");
- }
- else {
- // not an okay, $ prefixed field name.
- return Status(ErrorCodes::DollarPrefixedFieldName,
- str::stream() << "The dollar ($) prefixed field '"
- << elem.getFieldName() << "' in '"
- << mb::getFullName(elem)
- << "' is not valid for storage.");
-
- }
-
- return Status::OK();
- }
-
- Status storageValid(const mb::ConstElement& elem, const bool deep = true) {
- if (!elem.ok())
- return Status(ErrorCodes::BadValue, "Invalid elements cannot be stored.");
-
- // Field names of elements inside arrays are not meaningful in mutable bson,
- // so we do not want to validate them.
- //
- // TODO: Revisit how mutable handles array field names. We going to need to make
- // this better if we ever want to support ordered updates that can alter the same
- // element repeatedly; see SERVER-12848.
- const bool childOfArray = elem.parent().ok() ?
- (elem.parent().getType() == mongo::Array) : false;
-
- if (!childOfArray) {
- StringData fieldName = elem.getFieldName();
- // Cannot start with "$", unless dbref
- if (fieldName[0] == '$') {
- Status status = validateDollarPrefixElement(elem, deep);
- if (!status.isOK())
- return status;
- }
- else if (fieldName.find(".") != string::npos) {
- // Field name cannot have a "." in it.
- return Status(ErrorCodes::DottedFieldName,
- str::stream() << "The dotted field '"
- << elem.getFieldName() << "' in '"
- << mb::getFullName(elem)
- << "' is not valid for storage.");
- }
- }
-
- // Check children if there are any.
- Status s = storageValidChildren(elem, deep);
- if (!s.isOK())
- return s;
-
- return Status::OK();
- }
-
- Status storageValidChildren(const mb::ConstElement& elem, const bool deep) {
- if (!elem.hasChildren())
- return Status::OK();
-
- mb::ConstElement curr = elem.leftChild();
- while (curr.ok()) {
- Status s = storageValid(curr, deep);
- if (!s.isOK())
- return s;
- curr = curr.rightSibling();
- }
-
- return Status::OK();
- }
-
- /**
- * This will verify that all updated fields are
- * 1.) Valid for storage (checking parent to make sure things like DBRefs are valid)
- * 2.) Compare updated immutable fields do not change values
- *
- * If updateFields is empty then it was replacement and/or we need to check all fields
- */
- inline Status validate(const BSONObj& original,
- const FieldRefSet& updatedFields,
- const mb::Document& updated,
- const std::vector<FieldRef*>* immutableAndSingleValueFields,
- const ModifierInterface::Options& opts) {
-
- LOG(3) << "update validate options -- "
- << " updatedFields: " << updatedFields
- << " immutableAndSingleValueFields.size:"
- << (immutableAndSingleValueFields ? immutableAndSingleValueFields->size() : 0)
- << " fromRepl: " << opts.fromReplication
- << " validate:" << opts.enforceOkForStorage;
-
- // 1.) Loop through each updated field and validate for storage
- // and detect immutable field updates
-
- // The set of possibly changed immutable fields -- we will need to check their vals
- FieldRefSet changedImmutableFields;
-
- // Check to see if there were no fields specified or if we are not validating
- // The case if a range query, or query that didn't result in saved fields
- if (updatedFields.empty() || !opts.enforceOkForStorage) {
- if (opts.enforceOkForStorage) {
- // No specific fields were updated so the whole doc must be checked
- Status s = storageValid(updated, true);
- if (!s.isOK())
- return s;
- }
-
- // Check all immutable fields
- if (immutableAndSingleValueFields)
- changedImmutableFields.fillFrom(*immutableAndSingleValueFields);
- }
- else {
-
- // TODO: Change impl so we don't need to create a new FieldRefSet
- // -- move all conflict logic into static function on FieldRefSet?
- FieldRefSet immutableFieldRef;
- if (immutableAndSingleValueFields)
- immutableFieldRef.fillFrom(*immutableAndSingleValueFields);
-
- FieldRefSet::const_iterator where = updatedFields.begin();
- const FieldRefSet::const_iterator end = updatedFields.end();
- for( ; where != end; ++where) {
- const FieldRef& current = **where;
-
- // Find the updated field in the updated document.
- mutablebson::ConstElement newElem = updated.root();
- size_t currentPart = 0;
- while (newElem.ok() && currentPart < current.numParts())
- newElem = newElem[current.getPart(currentPart++)];
-
- // newElem might be missing if $unset/$renamed-away
- if (newElem.ok()) {
- Status s = storageValid(newElem, true);
- if (!s.isOK())
- return s;
- }
- // Check if the updated field conflicts with immutable fields
- immutableFieldRef.findConflicts(&current, &changedImmutableFields);
- }
- }
-
- const bool checkIdField = (updatedFields.empty() && !original.isEmpty()) ||
- updatedFields.findConflicts(&idFieldRef, NULL);
-
- // Add _id to fields to check since it too is immutable
- if (checkIdField)
- changedImmutableFields.keepShortest(&idFieldRef);
- else if (changedImmutableFields.empty()) {
- // Return early if nothing changed which is immutable
- return Status::OK();
- }
-
- LOG(4) << "Changed immutable fields: " << changedImmutableFields;
- // 2.) Now compare values of the changed immutable fields (to make sure they haven't)
-
- const mutablebson::ConstElement newIdElem = updated.root()[idFieldName];
-
- FieldRefSet::const_iterator where = changedImmutableFields.begin();
- const FieldRefSet::const_iterator end = changedImmutableFields.end();
- for( ; where != end; ++where ) {
- const FieldRef& current = **where;
-
- // Find the updated field in the updated document.
- mutablebson::ConstElement newElem = updated.root();
- size_t currentPart = 0;
- while (newElem.ok() && currentPart < current.numParts())
- newElem = newElem[current.getPart(currentPart++)];
-
- if (!newElem.ok()) {
- if (original.isEmpty()) {
- // If the _id is missing and not required, then skip this check
- if (!(current.dottedField() == idFieldName))
- return Status(ErrorCodes::NoSuchKey,
- mongoutils::str::stream()
- << "After applying the update, the new"
- << " document was missing the '"
- << current.dottedField()
- << "' (required and immutable) field.");
-
- }
- else {
- if (current.dottedField() != idFieldName)
- return Status(ErrorCodes::ImmutableField,
- mongoutils::str::stream()
- << "After applying the update to the document with "
- << newIdElem.toString()
- << ", the '" << current.dottedField()
- << "' (required and immutable) field was "
- "found to have been removed --"
- << original);
- }
- }
- else {
-
- // Find the potentially affected field in the original document.
- const BSONElement oldElem = original.getFieldDotted(current.dottedField());
- const BSONElement oldIdElem = original.getField(idFieldName);
-
- // Ensure no arrays since neither _id nor shard keys can be in an array, or one.
- mb::ConstElement currElem = newElem;
- while (currElem.ok()) {
- if (currElem.getType() == Array) {
- return Status(ErrorCodes::NotSingleValueField,
- mongoutils::str::stream()
- << "After applying the update to the document {"
- << (oldIdElem.ok() ? oldIdElem.toString() :
- newIdElem.toString())
- << " , ...}, the (immutable) field '"
- << current.dottedField()
- << "' was found to be an array or array descendant.");
- }
- currElem = currElem.parent();
- }
-
- // If we have both (old and new), compare them. If we just have new we are good
- if (oldElem.ok() && newElem.compareWithBSONElement(oldElem, false) != 0) {
- return Status(ErrorCodes::ImmutableField,
- mongoutils::str::stream()
- << "After applying the update to the document {"
- << (oldIdElem.ok() ? oldIdElem.toString() :
- newIdElem.toString())
- << " , ...}, the (immutable) field '" << current.dottedField()
- << "' was found to have been altered to "
- << newElem.toString());
- }
- }
- }
-
- return Status::OK();
- }
-
- Status ensureIdAndFirst(mb::Document& doc) {
- mb::Element idElem = mb::findFirstChildNamed(doc.root(), idFieldName);
-
- // Move _id as first element if it exists
- if (idElem.ok()) {
- if (idElem.leftSibling().ok()) {
- Status s = idElem.remove();
- if (!s.isOK())
- return s;
- s = doc.root().pushFront(idElem);
- if (!s.isOK())
- return s;
- }
- }
- else {
- // Create _id if the document does not currently have one.
- idElem = doc.makeElementNewOID(idFieldName);
- if (!idElem.ok())
- return Status(ErrorCodes::BadValue,
- "Could not create new _id ObjectId element.",
- 17268);
- Status s = doc.root().pushFront(idElem);
- if (!s.isOK())
- return s;
- }
-
- return Status::OK();
-
- }
} // namespace
UpdateResult update(Database* db,
@@ -455,245 +103,53 @@ namespace mongo {
}
PlanExecutor* rawExec;
- Status status = cq ?
- getExecutor(request.getOpCtx(), collection, cqHolder.release(), &rawExec) :
- getExecutor(request.getOpCtx(), collection, nsString.ns(), request.getQuery(),
- &rawExec);
+ Status status = Status::OK();
+ if (cq) {
+ // This is the regular path for when we have a CanonicalQuery.
+ status = getExecutorUpdate(request.getOpCtx(), db, cqHolder.release(), &request, driver,
+ opDebug, &rawExec);
+ }
+ else {
+ // This is the idhack fast-path for getting a PlanExecutor without doing the work
+ // to create a CanonicalQuery.
+ status = getExecutorUpdate(request.getOpCtx(), db, nsString.ns(), &request, driver,
+ opDebug, &rawExec);
+ }
uassert(17243,
"could not get executor" + request.getQuery().toString() + "; " + causedBy(status),
status.isOK());
// Create the plan executor and setup all deps.
- auto_ptr<PlanExecutor> exec(rawExec);
+ scoped_ptr<PlanExecutor> exec(rawExec);
// Register executor with the collection cursor cache.
const ScopedExecutorRegistration safety(exec.get());
- // Get the canonical query which the underlying executor is using. This may be NULL in
- // the case of idhack updates.
- cq = exec->getCanonicalQuery();
-
- //
- // We'll start assuming we have one or more documents for this update. (Otherwise,
- // we'll fall-back to insert case (if upsert is true).)
- //
-
- // We are an update until we fall into the insert case below.
- driver->setContext(ModifierInterface::ExecInfo::UPDATE_CONTEXT);
-
- int numMatched = 0;
-
- // 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.
- //
- // For example: If we're scanning an index {x:1} and performing {$inc:{x:5}}, we'll keep
- // moving the document forward and it will continue to reappear in our index scan.
- // Unless the index is multikey, the underlying query machinery won't de-dup.
- //
- // If the update wasn't in-place we may see it again. Our query may return the new
- // document and we wouldn't want to update that.
- //
- // So, no matter what, we keep track of where the doc wound up.
- typedef unordered_set<DiskLoc, DiskLoc::Hasher> DiskLocSet;
- const scoped_ptr<DiskLocSet> updatedLocs(request.isMulti() ? new DiskLocSet : NULL);
-
- // Reset these counters on each call. We might re-enter this function to retry this
- // update if we throw a page fault exception below, and we rely on these counters
- // reflecting only the actions taken locally. In particlar, we must have the no-op
- // counter reset so that we can meaningfully comapre it with numMatched above.
- opDebug->nModified = 0;
-
- // -1 for these fields means that we don't have a value. Once the update completes
- // we request these values from the plan executor.
- opDebug->nscanned = -1;
- opDebug->nscannedObjects = -1;
-
- // Get the cached document from the update driver.
- mutablebson::Document& doc = driver->getDocument();
- mutablebson::DamageVector damages;
-
- // Used during iteration of docs
- BSONObj oldObj;
-
- // Get first doc, and location
- PlanExecutor::ExecState state = PlanExecutor::ADVANCED;
-
- uassert(ErrorCodes::NotMaster,
- mongoutils::str::stream() << "Not primary while updating " << nsString.ns(),
- !request.shouldCallLogOp()
- || repl::getGlobalReplicationCoordinator()->canAcceptWritesForDatabase(
- nsString.db()));
-
- while (true) {
- // Get next doc, and location
- DiskLoc loc;
- state = exec->getNext(&oldObj, &loc);
-
- if (state != PlanExecutor::ADVANCED) {
- if (state == PlanExecutor::IS_EOF) {
- // We have reached the logical end of the loop, so do yielding recovery
- break;
- }
- else {
- uassertStatusOK(Status(ErrorCodes::InternalError,
- str::stream() << " Update query failed -- "
- << PlanExecutor::statestr(state)));
- }
- }
-
- // We fill this with the new locs of moved doc so we don't double-update.
- if (updatedLocs && updatedLocs->count(loc) > 0) {
- continue;
- }
-
- numMatched++;
-
- // Ask the driver to apply the mods. It may be that the driver can apply those "in
- // place", that is, some values of the old document just get adjusted without any
- // change to the binary layout on the bson layer. It may be that a whole new
- // document is needed to accomodate the new bson layout of the resulting document.
- doc.reset(oldObj, mutablebson::Document::kInPlaceEnabled);
- BSONObj logObj;
-
-
- FieldRefSet updatedFields;
-
- Status status = Status::OK();
- if (!driver->needMatchDetails()) {
- // If we don't need match details, avoid doing the rematch
- status = driver->update(StringData(), &doc, &logObj, &updatedFields);
- }
- else {
- // If there was a matched field, obtain it.
- MatchDetails matchDetails;
- matchDetails.requestElemMatchKey();
-
- dassert(cq);
- verify(cq->root()->matchesBSON(oldObj, &matchDetails));
-
- string matchedField;
- if (matchDetails.hasElemMatchKey())
- matchedField = matchDetails.elemMatchKey();
-
- // TODO: Right now, each mod checks in 'prepare' that if it needs positional
- // data, that a non-empty StringData() was provided. In principle, we could do
- // that check here in an else clause to the above conditional and remove the
- // checks from the mods.
-
- status = driver->update(matchedField, &doc, &logObj, &updatedFields);
- }
+ // Run the plan (don't need to collect results because UpdateStage always returns
+ // NEED_TIME).
+ uassertStatusOK(exec->executePlan());
- if (!status.isOK()) {
- uasserted(16837, status.reason());
- }
-
- // Ensure _id exists and is first
- uassertStatusOK(ensureIdAndFirst(doc));
-
- // If the driver applied the mods in place, we can ask the mutable for what
- // changed. We call those changes "damages". :) We use the damages to inform the
- // journal what was changed, and then apply them to the original document
- // ourselves. If, however, the driver applied the mods out of place, we ask it to
- // generate a new, modified document for us. In that case, the file manager will
- // take care of the journaling details for us.
- //
- // This code flow is admittedly odd. But, right now, journaling is baked in the file
- // manager. And if we aren't using the file manager, we have to do jounaling
- // ourselves.
- bool docWasModified = false;
- BSONObj newObj;
- const char* source = NULL;
- bool inPlace = doc.getInPlaceUpdates(&damages, &source);
-
- // If something changed in the document, verify that no immutable fields were changed
- // and data is valid for storage.
- if ((!inPlace || !damages.empty()) ) {
- if (!(request.isFromReplication() || request.isFromMigration())) {
- const std::vector<FieldRef*>* immutableFields = NULL;
- if (lifecycle)
- immutableFields = lifecycle->getImmutableFields();
-
- uassertStatusOK(validate(oldObj,
- updatedFields,
- doc,
- immutableFields,
- driver->modOptions()) );
- }
- }
-
- WriteUnitOfWork wunit(request.getOpCtx()->recoveryUnit());
-
- // Save state before making changes
- exec->saveState();
-
- if (inPlace && !driver->modsAffectIndices()) {
- // If a set of modifiers were all no-ops, we are still 'in place', but there is
- // no work to do, in which case we want to consider the object unchanged.
- if (!damages.empty() ) {
- collection->updateDocumentWithDamages(request.getOpCtx(), loc, source, damages);
- docWasModified = true;
- opDebug->fastmod = true;
- }
-
- newObj = oldObj;
- }
- else {
- // The updates were not in place. Apply them through the file manager.
-
- // XXX: With experimental document-level locking, we do not hold the sufficient
- // locks, so this would cause corruption.
- fassert(18516, !useExperimentalDocLocking);
-
- newObj = doc.getObject();
- uassert(17419,
- str::stream() << "Resulting document after update is larger than "
- << BSONObjMaxUserSize,
- newObj.objsize() <= BSONObjMaxUserSize);
- StatusWith<DiskLoc> res = collection->updateDocument(request.getOpCtx(),
- loc,
- newObj,
- true,
- opDebug);
- uassertStatusOK(res.getStatus());
- DiskLoc newLoc = res.getValue();
- docWasModified = true;
-
- // If the document moved, we might see it again in a collection scan (maybe it's
- // a document after our current document).
- //
- // If the document is indexed and the mod changes an indexed value, we might see it
- // again. For an example, see the comment above near declaration of updatedLocs.
- if (updatedLocs && (newLoc != loc || driver->modsAffectIndices())) {
- updatedLocs->insert(newLoc);
- }
- }
-
- // Restore state after modification
- uassert(17278,
- "Update could not restore plan executor state after updating a document.",
- exec->restoreState(request.getOpCtx()));
-
- // Call logOp if requested.
- if (request.shouldCallLogOp() && !logObj.isEmpty()) {
- BSONObj idQuery = driver->makeOplogEntryQuery(newObj, request.isMulti());
- repl::logOp(request.getOpCtx(), "u", nsString.ns().c_str(), logObj, &idQuery,
- NULL, request.isFromMigration());
- }
+ // Get stats from the root stage.
+ invariant(exec->getRootStage()->stageType() == STAGE_UPDATE);
+ UpdateStage* updateStage = static_cast<UpdateStage*>(exec->getRootStage());
+ const UpdateStats* updateStats =
+ static_cast<const UpdateStats*>(updateStage->getSpecificStats());
- wunit.commit();
+ // Use stats from the root stage to fill out opDebug.
+ opDebug->nMatched = updateStats->nMatched;
+ opDebug->nModified = updateStats->nModified;
+ opDebug->upsert = updateStats->inserted;
+ opDebug->fastmodinsert = updateStats->fastmodinsert;
+ opDebug->fastmod = updateStats->fastmod;
- // Only record doc modifications if they wrote (exclude no-ops)
- if (docWasModified)
- opDebug->nModified++;
-
- if (!request.isMulti()) {
- break;
- }
-
- // Opportunity for journaling to write during the update.
- request.getOpCtx()->recoveryUnit()->commitIfNeeded();
+ // Historically, 'opDebug' considers 'nMatched' and 'nModified' to be 1 (rather than 0) if
+ // there is an upsert that inserts a document. The UpdateStage does not participate in this
+ // madness in order to have saner stats reporting for explain. This means that we have to
+ // set these values "manually" in the case of an insert.
+ if (updateStats->inserted) {
+ opDebug->nMatched = 1;
+ opDebug->nModified = 1;
}
// Get summary information about the plan.
@@ -702,121 +158,11 @@ namespace mongo {
opDebug->nscanned = stats.totalKeysExamined;
opDebug->nscannedObjects = stats.totalDocsExamined;
- // TODO: Can this be simplified?
- if ((numMatched > 0) || (numMatched == 0 && !request.isUpsert()) ) {
- opDebug->nMatched = numMatched;
- return UpdateResult(numMatched > 0 /* updated existing object(s) */,
- !driver->isDocReplacement() /* $mod or obj replacement */,
- opDebug->nModified /* number of modified docs, no no-ops */,
- numMatched /* # of docs matched/updated, even no-ops */,
- BSONObj());
- }
-
- //
- // We haven't found any existing document so an insert is done
- // (upsert is true).
- //
- opDebug->upsert = true;
-
- // 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);
- driver->setContext(ModifierInterface::ExecInfo::INSERT_CONTEXT);
-
- // Reset the document we will be writing to
- doc.reset();
-
- // This remains the empty object in the case of an object replacement, but in the case
- // of an upsert where we are creating a base object from the query and applying mods,
- // we capture the query as the original so that we can detect immutable field mutations.
- BSONObj original = BSONObj();
-
- // Calling createFromQuery will populate the 'doc' with fields from the query which
- // creates the base of the update for the inserterd doc (because upsert was true)
- if (cq) {
- uassertStatusOK(driver->populateDocumentWithQueryFields(cq, doc));
- if (!driver->isDocReplacement()) {
- opDebug->fastmodinsert = true;
- // We need all the fields from the query to compare against for validation below.
- original = doc.getObject();
- }
- else {
- original = request.getQuery();
- }
- }
- else {
- fassert(17354, CanonicalQuery::isSimpleIdQuery(request.getQuery()));
- BSONElement idElt = request.getQuery()["_id"];
- original = idElt.wrap();
- fassert(17352, doc.root().appendElement(idElt));
- }
-
- // Apply the update modifications and then log the update as an insert manually.
- status = driver->update(StringData(), &doc);
- if (!status.isOK()) {
- uasserted(16836, status.reason());
- }
-
- // Ensure _id exists and is first
- uassertStatusOK(ensureIdAndFirst(doc));
-
- // Validate that the object replacement or modifiers resulted in a document
- // that contains all the immutable keys and can be stored if it isn't coming
- // from a migration or via replication.
- if (!(request.isFromReplication() || request.isFromMigration())){
- const std::vector<FieldRef*>* immutableFields = NULL;
- if (lifecycle)
- immutableFields = lifecycle->getImmutableFields();
-
- FieldRefSet noFields;
- // This will only validate the modified fields if not a replacement.
- uassertStatusOK(validate(original,
- noFields,
- doc,
- immutableFields,
- driver->modOptions()) );
- }
-
- // Insert the doc
- BSONObj newObj = doc.getObject();
- uassert(17420,
- str::stream() << "Document to upsert is larger than " << BSONObjMaxUserSize,
- newObj.objsize() <= BSONObjMaxUserSize);
-
- WriteUnitOfWork wunit(request.getOpCtx()->recoveryUnit());
- // Only create the collection if the doc will be inserted.
- if (!collection) {
- collection = db->getCollection(request.getOpCtx(), request.getNamespaceString().ns());
- if (!collection) {
- collection = db->createCollection(request.getOpCtx(), request.getNamespaceString().ns());
- }
- }
-
-
- StatusWith<DiskLoc> newLoc = collection->insertDocument(request.getOpCtx(),
- newObj,
- !request.isGod() /*enforceQuota*/);
- uassertStatusOK(newLoc.getStatus());
- if (request.shouldCallLogOp()) {
- repl::logOp(request.getOpCtx(),
- "i",
- nsString.ns().c_str(),
- newObj,
- NULL,
- NULL,
- request.isFromMigration());
- }
-
- wunit.commit();
-
- opDebug->nMatched = 1;
- return UpdateResult(false /* updated a non existing document */,
- !driver->isDocReplacement() /* $mod or obj replacement? */,
- 1 /* docs written*/,
- 1 /* count of updated documents */,
- newObj /* object that was upserted */ );
+ return UpdateResult(updateStats->nMatched > 0 /* Did we update at least one obj? */,
+ !driver->isDocReplacement() /* $mod or obj replacement */,
+ opDebug->nModified /* number of modified docs, no no-ops */,
+ opDebug->nMatched /* # of docs matched/updated, even no-ops */,
+ updateStats->objInserted);
}
BSONObj applyUpdateOperators(const BSONObj& from, const BSONObj& operators) {