mirror of https://github.com/mongodb/mongo
SERVER-109626 Adjust behavior for emitting additional fields in change streams (#40479)
GitOrigin-RevId: 0af0d35b09f93c00f38b2eb2146409bfc5f62382
This commit is contained in:
parent
a7a214a082
commit
4cc027921a
|
|
@ -278,6 +278,7 @@ selector:
|
|||
- jstests/sharding/query/change_streams/change_stream_no_shards.js
|
||||
- jstests/sharding/query/change_streams/change_stream_on_system_collection.js
|
||||
- jstests/sharding/query/change_streams/change_stream_transaction_sharded.js
|
||||
- jstests/sharding/query/change_streams/change_stream_transaction_sharded_commit_timestamp.js
|
||||
- jstests/sharding/query/change_streams/change_stream_update_lookup_collation.js
|
||||
- jstests/sharding/query/change_streams/change_stream_update_lookup_read_concern.js
|
||||
- jstests/sharding/query/change_streams/change_streams.js
|
||||
|
|
|
|||
|
|
@ -1,126 +0,0 @@
|
|||
/**
|
||||
* Verifies that change streams in 8.2 or higher return additional fields by default, without having
|
||||
* to use '{showExpandedEvents: true}'.
|
||||
*
|
||||
* @tags: [
|
||||
* # The test assumes certain ordering of the events. The chunk migrations on a sharded collection
|
||||
* # could break the test.
|
||||
* assumes_unsharded_collection,
|
||||
* requires_fcv_82
|
||||
* ]
|
||||
*/
|
||||
import {assertDropAndRecreateCollection, assertDropCollection} from "jstests/libs/collection_drop_recreate.js";
|
||||
import {assertChangeStreamEventEq, ChangeStreamTest} from "jstests/libs/query/change_stream_util.js";
|
||||
|
||||
const kLargeStr = "x".repeat(128);
|
||||
const testDB = db.getSiblingDB(jsTestName());
|
||||
const dbName = testDB.getName();
|
||||
const collName = "expanded_events_fields";
|
||||
const ns = {
|
||||
db: dbName,
|
||||
coll: collName,
|
||||
};
|
||||
const renamedCollName = "enriched_events_renamed";
|
||||
const renamedNs = {
|
||||
db: dbName,
|
||||
coll: renamedCollName,
|
||||
};
|
||||
const coll = assertDropAndRecreateCollection(testDB, collName);
|
||||
|
||||
const pipeline = [{$changeStream: {}}];
|
||||
const test = new ChangeStreamTest(testDB);
|
||||
const cursor = test.startWatchingChanges({pipeline, collection: 1});
|
||||
|
||||
function assertNextChangeEvent(expectedEvent) {
|
||||
const event = test.getOneChange(cursor);
|
||||
assert(event.wallTime instanceof Date);
|
||||
assertChangeStreamEventEq(event, expectedEvent);
|
||||
}
|
||||
|
||||
function getCollectionUuid(coll) {
|
||||
const collInfo = testDB.getCollectionInfos({name: coll})[0];
|
||||
return collInfo.info.uuid;
|
||||
}
|
||||
|
||||
function assertChangeEvent(operation, expectedEvent) {
|
||||
operation();
|
||||
assertNextChangeEvent(expectedEvent);
|
||||
}
|
||||
|
||||
// Test change stream event for 'insert' operation.
|
||||
assertChangeEvent(() => assert.commandWorked(coll.insert({_id: 0, a: 1})), {
|
||||
ns,
|
||||
operationType: "insert",
|
||||
documentKey: {_id: 0},
|
||||
fullDocument: {_id: 0, a: 1},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
// Test change stream event for replacement-style 'update' operation.
|
||||
assertChangeEvent(() => assert.commandWorked(coll.update({_id: 0}, {_id: 0, a: 2})), {
|
||||
ns,
|
||||
operationType: "replace",
|
||||
documentKey: {_id: 0},
|
||||
fullDocument: {_id: 0, a: 2},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
// Test change stream event for modifier style 'update' operation.
|
||||
assertChangeEvent(() => assert.commandWorked(coll.update({_id: 0}, {$inc: {a: 1}})), {
|
||||
ns,
|
||||
operationType: "update",
|
||||
documentKey: {_id: 0},
|
||||
updateDescription: {removedFields: [], updatedFields: {a: 3}, truncatedArrays: [], disambiguatedPaths: {}},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
// Test change stream event for modifier style 'update' operation with disambiguatedPaths.
|
||||
// Set up initial value.
|
||||
const update1 = {
|
||||
"arrayWithNumericField": [[{"0": "numeric", a: {"b.c": 1}, field: kLargeStr}]],
|
||||
};
|
||||
assertChangeEvent(() => assert.commandWorked(coll.update({_id: 0}, {$set: update1})), {
|
||||
ns,
|
||||
operationType: "update",
|
||||
documentKey: {_id: 0},
|
||||
updateDescription: {removedFields: [], updatedFields: update1, truncatedArrays: [], disambiguatedPaths: {}},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
// Update document so that 'disambiguatedPaths' will be populated.
|
||||
const update2 = {
|
||||
"arrayWithNumericField.0.0.1": {"b.c": "z".repeat(30)},
|
||||
};
|
||||
assertChangeEvent(() => assert.commandWorked(coll.update({_id: 0}, {$set: update2})), {
|
||||
ns,
|
||||
operationType: "update",
|
||||
documentKey: {_id: 0},
|
||||
updateDescription: {
|
||||
removedFields: [],
|
||||
updatedFields: update2,
|
||||
truncatedArrays: [],
|
||||
disambiguatedPaths: {"arrayWithNumericField.0.0.1": ["arrayWithNumericField", 0, 0, "1"]},
|
||||
},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
// Test change stream event for 'remove' operation.
|
||||
assertChangeEvent(() => assert.commandWorked(coll.remove({_id: 0})), {
|
||||
ns,
|
||||
operationType: "delete",
|
||||
documentKey: {_id: 0},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
// Test change stream event for 'rename' operation with 'dropTarget: false'.
|
||||
assertChangeEvent(() => assert.commandWorked(coll.renameCollection(renamedCollName)), {
|
||||
ns,
|
||||
operationType: "rename",
|
||||
to: renamedNs,
|
||||
operationDescription: {
|
||||
to: renamedNs,
|
||||
},
|
||||
collectionUUID: getCollectionUuid(ns.coll),
|
||||
});
|
||||
|
||||
assertDropCollection(testDB, renamedCollName);
|
||||
|
|
@ -59,7 +59,7 @@ function verifyOnWholeCluster(
|
|||
) {
|
||||
verifyChangeStreamOnWholeCluster({
|
||||
st,
|
||||
changeStreamSpec: {resumeAfter: resumeAfterToken, showExpandedEvents: true},
|
||||
changeStreamSpec: {resumeAfter: resumeAfterToken},
|
||||
userMatchExpr,
|
||||
expectedResult,
|
||||
expectedOplogNReturnedPerShard: Array.isArray(expectedOplogRetDocsForEachShard)
|
||||
|
|
@ -100,7 +100,6 @@ const updateDesc = {
|
|||
updatedFields: {},
|
||||
removedFields: ["z"],
|
||||
truncatedArrays: [],
|
||||
disambiguatedPaths: {},
|
||||
};
|
||||
|
||||
// Test out a predicate on the full 'updateDescription' field.
|
||||
|
|
|
|||
|
|
@ -95,8 +95,8 @@ export function canonicalizeEventForTesting(event, expected) {
|
|||
}
|
||||
|
||||
// Exclude "updateDescription.disambiguatedPaths" if not explicitly specified in the expected
|
||||
// event. This is necessary because from 8.2 onwards, we expose this field by default, but in
|
||||
// previous versions it is only exposed when the change stream was opened with
|
||||
// event. This is necessary because in v8.2.0, we expose this field unconditionally, but in
|
||||
// versions before or after v8.2.0 the field is only exposed if the change stream is opened with
|
||||
// '{showExpandedEvents: true}'.
|
||||
if (
|
||||
expected.hasOwnProperty("updateDescription") &&
|
||||
|
|
|
|||
|
|
@ -27,8 +27,8 @@ function compareChanges(expectedChanges, observedChanges) {
|
|||
assert.eq(expectedChanges[i].fullDocument, observedChanges[i].fullDocument);
|
||||
}
|
||||
if (expectedChanges[i].hasOwnProperty("updateDescription")) {
|
||||
// Need to remove this field because it is only exposed by default from v8.2 onwards,
|
||||
// but in previous versions it is only exposed when the change stream is opened with
|
||||
// Need to remove this field because it is only exposed by default in v8.2.0,
|
||||
// but in previous versions and versions >= v8.2.1 it is only exposed when the change stream is opened with
|
||||
// '{showExpandedEvents: true}'.
|
||||
delete observedChanges[i].updateDescription.disambiguatedPaths;
|
||||
assert.eq(expectedChanges[i].updateDescription, observedChanges[i].updateDescription);
|
||||
|
|
|
|||
|
|
@ -107,7 +107,7 @@ assert.commandWorked(st.s0.adminCommand({moveChunk: kNsName, find: {shard: -1},
|
|||
assert.commandWorked(st.s0.adminCommand({moveChunk: kNsName, find: {shard: 1}, to: st["shard1"].shardName}));
|
||||
|
||||
// Create changestream on the target database.
|
||||
const cursor = db.watch([], {showExpandedEvents: true});
|
||||
const cursor = db.watch([], {showExpandedEvents: true, showCommitTimestamp: true});
|
||||
|
||||
// Start session.
|
||||
const sessionOptions = {
|
||||
|
|
|
|||
|
|
@ -94,7 +94,7 @@ function runTest(conn) {
|
|||
session3.startTransaction({readConcern: {level: "majority"}});
|
||||
|
||||
// Open a change stream on the test collection.
|
||||
const changeStreamCursor = coll.watch([], {showExpandedEvents: true});
|
||||
const changeStreamCursor = coll.watch([], {showExpandedEvents: true, showCommitTimestamp: true});
|
||||
const resumeToken = changeStreamCursor.getResumeToken();
|
||||
|
||||
// Insert a document and confirm that the change stream has it.
|
||||
|
|
|
|||
|
|
@ -89,7 +89,7 @@ for (let key of Object.keys(ChangeStreamWatchMode)) {
|
|||
assert.docEq({db: sDB.getName(), coll: coll.getName()}, change.ns);
|
||||
assert.eq(change.operationType, "insert");
|
||||
|
||||
// Needs to be removed here because of multiversion testing.
|
||||
// Needs to be removed here because of multiversion differences between v8.2.0 and previous/following versions.
|
||||
delete change.collectionUUID;
|
||||
docsFoundInOrder.push(change);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,4 +1,7 @@
|
|||
// Confirms that change streams only see committed operations for sharded transactions.
|
||||
// TODO SERVER-109890: The test 'change_stream_transaction_sharded_commit_timestamp.js'
|
||||
// is a clone of this file with additional testing for the 'commitTimestamp' field.
|
||||
// Once v9.0 becomes last LTS we can remove this file in favor of the other.
|
||||
// @tags: [
|
||||
// requires_sharding,
|
||||
// uses_change_streams,
|
||||
|
|
@ -50,11 +53,6 @@ assert.commandWorked(
|
|||
const db = mongosConn.getDB(dbName);
|
||||
const coll = db.getCollection(collName);
|
||||
|
||||
const fcvDoc = db.adminCommand({getParameter: 1, featureCompatibilityVersion: 1});
|
||||
// The 'commitTimestamp' field is exposed by default from v8.2 onwards.
|
||||
const alwaysExpectCommitTimestamp =
|
||||
MongoRunner.compareBinVersions(fcvDoc.featureCompatibilityVersion.version, "8.2") >= 0;
|
||||
|
||||
let changeListShard1 = [],
|
||||
changeListShard2 = [];
|
||||
|
||||
|
|
@ -93,7 +91,6 @@ session2.startTransaction({readConcern: {level: "majority"}});
|
|||
expectedChangesShard2,
|
||||
changeCaptureListShard1,
|
||||
changeCaptureListShard2,
|
||||
expectCommitTimestamp = false,
|
||||
) {
|
||||
function assertChangeEqualWithCapture(changeDoc, expectedChange, changeCaptureList) {
|
||||
assert.eq(expectedChange.operationType, changeDoc.operationType);
|
||||
|
|
@ -101,29 +98,6 @@ session2.startTransaction({readConcern: {level: "majority"}});
|
|||
changeCaptureList.push(changeDoc);
|
||||
}
|
||||
|
||||
// Verify that all commit timestamps are identical.
|
||||
let commitTimestamp = null;
|
||||
const assertCommitTimestamp = (changeDoc) => {
|
||||
if (expectCommitTimestamp) {
|
||||
assert(changeDoc.hasOwnProperty("commitTimestamp"), "expecting doc to have a 'commitTimestamp' field", {
|
||||
changeDoc,
|
||||
});
|
||||
assert(
|
||||
isTimestamp(changeDoc["commitTimestamp"]),
|
||||
"expecting 'commitTimestamp' field to be a timestamp",
|
||||
{changeDoc},
|
||||
);
|
||||
if (commitTimestamp === null) {
|
||||
commitTimestamp = changeDoc["commitTimestamp"];
|
||||
} else {
|
||||
assert.eq(commitTimestamp, changeDoc["commitTimestamp"], "expecting equal commitTimestamps", {
|
||||
commitTimestamp,
|
||||
changeDoc,
|
||||
});
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Cross-shard transaction, and "endOfTransaction" events are enabled.
|
||||
const expectEndOfTransaction =
|
||||
expectedChangesShard1.length &&
|
||||
|
|
@ -136,25 +110,14 @@ session2.startTransaction({readConcern: {level: "majority"}});
|
|||
|
||||
if (changeDoc.documentKey.shard === 1) {
|
||||
assert(expectedChangesShard1.length);
|
||||
assertChangeEqualWithCapture(
|
||||
changeDoc,
|
||||
expectedChangesShard1[0],
|
||||
changeCaptureListShard1,
|
||||
expectCommitTimestamp,
|
||||
);
|
||||
assertChangeEqualWithCapture(changeDoc, expectedChangesShard1[0], changeCaptureListShard1);
|
||||
expectedChangesShard1.shift();
|
||||
} else {
|
||||
assert.eq(2, changeDoc.documentKey.shard);
|
||||
assert(expectedChangesShard2.length);
|
||||
assertChangeEqualWithCapture(
|
||||
changeDoc,
|
||||
expectedChangesShard2[0],
|
||||
changeCaptureListShard2,
|
||||
expectCommitTimestamp,
|
||||
);
|
||||
assertChangeEqualWithCapture(changeDoc, expectedChangesShard2[0], changeCaptureListShard2);
|
||||
expectedChangesShard2.shift();
|
||||
}
|
||||
assertCommitTimestamp(changeDoc);
|
||||
}
|
||||
|
||||
if (expectEndOfTransaction) {
|
||||
|
|
@ -166,7 +129,6 @@ session2.startTransaction({readConcern: {level: "majority"}});
|
|||
assertNoChanges(cursor);
|
||||
}
|
||||
|
||||
// Open a change stream on the test collection.
|
||||
const changeStreamCursor = coll.watch([], {showExpandedEvents: true});
|
||||
|
||||
// Insert a document and confirm that the change stream has it.
|
||||
|
|
@ -240,7 +202,6 @@ session2.startTransaction({readConcern: {level: "majority"}});
|
|||
],
|
||||
changeListShard1,
|
||||
changeListShard2,
|
||||
alwaysExpectCommitTimestamp,
|
||||
);
|
||||
|
||||
// Perform a write outside of the transaction.
|
||||
|
|
@ -287,6 +248,10 @@ session2.startTransaction({readConcern: {level: "majority"}});
|
|||
assert(changeListIndex < shardChangeList.length);
|
||||
|
||||
const expectedChangeDoc = shardChangeList[changeListIndex];
|
||||
// Remove 'commitTimestamp' field from expected and actual events, as this field is only exposed by default in v8.2.0. Versions before v8.2.0 do not expose this field, and versions after v8.2.0 only expose this field when the internal flag 'showCommitTimestamp' is set when opening the change stream.
|
||||
delete expectedChangeDoc.commitTimestamp;
|
||||
delete changeDoc.commitTimestamp;
|
||||
|
||||
assert.eq(changeDoc, expectedChangeDoc);
|
||||
assert.eq(
|
||||
expectedChangeDoc.documentKey,
|
||||
|
|
|
|||
|
|
@ -0,0 +1,344 @@
|
|||
// This is a clone of 'change_stream_transaction_sharded.js' that also validates
|
||||
// that the 'commitTimestamp' field of qualifying change stream events is present.
|
||||
// The 'commitTimestamp' field for DML events in prepared transactions is only
|
||||
// emitted from v8.2.1 onwards if the change stream is opened with the
|
||||
// 'showCommitTimestamp' flag.
|
||||
// We cannot run this test in multiversion environments at the moment, as older
|
||||
// versions do not understand the 'showCommitTimestamp' flag and error out if set.
|
||||
// TODO SERVER-109890: enable multiversion testing for this file once v9.0 becomes
|
||||
// last LTS.
|
||||
// @tags: [
|
||||
// multiversion_incompatible,
|
||||
// requires_fcv_83,
|
||||
// requires_sharding,
|
||||
// uses_change_streams,
|
||||
// uses_multi_shard_transaction,
|
||||
// uses_transactions
|
||||
// ]
|
||||
import {FeatureFlagUtil} from "jstests/libs/feature_flag_util.js";
|
||||
import {assertNoChanges} from "jstests/libs/query/change_stream_util.js";
|
||||
import {ShardingTest} from "jstests/libs/shardingtest.js";
|
||||
import {isTimestamp} from "jstests/libs/timestamp_util.js";
|
||||
|
||||
const dbName = "test";
|
||||
const collName = "change_stream_transaction_sharded_commit_timestamp";
|
||||
const namespace = dbName + "." + collName;
|
||||
|
||||
const st = new ShardingTest({
|
||||
shards: 2,
|
||||
rs: {nodes: 1, setParameter: {writePeriodicNoops: true, periodicNoopIntervalSecs: 1}},
|
||||
});
|
||||
|
||||
const mongosConn = st.s;
|
||||
assert.commandWorked(mongosConn.adminCommand({enableSharding: dbName, primaryShard: st.shard0.shardName}));
|
||||
assert.commandWorked(mongosConn.getDB(dbName).getCollection(collName).createIndex({shard: 1}));
|
||||
|
||||
// Shard the test collection and split it into two chunks: one that contains all {shard: 1}
|
||||
// documents and one that contains all {shard: 2} documents.
|
||||
st.shardColl(
|
||||
collName,
|
||||
{shard: 1} /* shard key */,
|
||||
{shard: 2} /* split at */,
|
||||
{shard: 2} /* move the chunk containing {shard: 2} to its own shard */,
|
||||
dbName,
|
||||
true,
|
||||
);
|
||||
// Seed each chunk with an initial document.
|
||||
assert.commandWorked(
|
||||
mongosConn
|
||||
.getDB(dbName)
|
||||
.getCollection(collName)
|
||||
.insert({shard: 1}, {writeConcern: {w: "majority"}}),
|
||||
);
|
||||
assert.commandWorked(
|
||||
mongosConn
|
||||
.getDB(dbName)
|
||||
.getCollection(collName)
|
||||
.insert({shard: 2}, {writeConcern: {w: "majority"}}),
|
||||
);
|
||||
|
||||
const db = mongosConn.getDB(dbName);
|
||||
const coll = db.getCollection(collName);
|
||||
|
||||
let changeListShard1 = [],
|
||||
changeListShard2 = [];
|
||||
|
||||
//
|
||||
// Start transaction 1.
|
||||
//
|
||||
const session1 = db.getMongo().startSession({causalConsistency: true});
|
||||
const sessionDb1 = session1.getDatabase(dbName);
|
||||
const sessionColl1 = sessionDb1[collName];
|
||||
session1.startTransaction({readConcern: {level: "majority"}});
|
||||
|
||||
//
|
||||
// Start transaction 2.
|
||||
//
|
||||
const session2 = db.getMongo().startSession({causalConsistency: true});
|
||||
const sessionDb2 = session2.getDatabase(dbName);
|
||||
const sessionColl2 = sessionDb2[collName];
|
||||
session2.startTransaction({readConcern: {level: "majority"}});
|
||||
|
||||
//
|
||||
// Perform writes both in and outside of transactions and confirm that the changes expected are
|
||||
// returned by the change stream.
|
||||
//
|
||||
(function () {
|
||||
/**
|
||||
* Asserts that the expected changes are found on the change stream cursor. Pushes the
|
||||
* corresponding change stream document (with resume token) to an array. When expected
|
||||
* changes are provided for both shards, we must assume that either shard's changes could
|
||||
* come first or that they are interleaved via applyOps index. This is because a cross shard
|
||||
* transaction may commit at a different cluster time on each shard, which impacts the
|
||||
* ordering of the change stream.
|
||||
*/
|
||||
function assertWritesVisibleWithCapture(
|
||||
cursor,
|
||||
expectedChangesShard1,
|
||||
expectedChangesShard2,
|
||||
changeCaptureListShard1,
|
||||
changeCaptureListShard2,
|
||||
expectCommitTimestamp = false,
|
||||
) {
|
||||
function assertChangeEqualWithCapture(changeDoc, expectedChange, changeCaptureList) {
|
||||
assert.eq(expectedChange.operationType, changeDoc.operationType);
|
||||
assert.eq(expectedChange._id, changeDoc.documentKey._id);
|
||||
changeCaptureList.push(changeDoc);
|
||||
}
|
||||
|
||||
// Verify that all commit timestamps are identical.
|
||||
let commitTimestamp = null;
|
||||
const assertCommitTimestamp = (changeDoc) => {
|
||||
if (expectCommitTimestamp) {
|
||||
assert(changeDoc.hasOwnProperty("commitTimestamp"), "expecting doc to have a 'commitTimestamp' field", {
|
||||
changeDoc,
|
||||
});
|
||||
assert(
|
||||
isTimestamp(changeDoc["commitTimestamp"]),
|
||||
"expecting 'commitTimestamp' field to be a timestamp",
|
||||
{changeDoc},
|
||||
);
|
||||
if (commitTimestamp === null) {
|
||||
commitTimestamp = changeDoc["commitTimestamp"];
|
||||
} else {
|
||||
assert.eq(commitTimestamp, changeDoc["commitTimestamp"], "expecting equal commitTimestamps", {
|
||||
commitTimestamp,
|
||||
changeDoc,
|
||||
});
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Cross-shard transaction, and "endOfTransaction" events are enabled.
|
||||
const expectEndOfTransaction =
|
||||
expectedChangesShard1.length &&
|
||||
expectedChangesShard2.length &&
|
||||
FeatureFlagUtil.isEnabled(db, "EndOfTransactionChangeEvent");
|
||||
|
||||
while (expectedChangesShard1.length || expectedChangesShard2.length) {
|
||||
assert.soon(() => cursor.hasNext());
|
||||
const changeDoc = cursor.next();
|
||||
|
||||
if (changeDoc.documentKey.shard === 1) {
|
||||
assert(expectedChangesShard1.length);
|
||||
assertChangeEqualWithCapture(
|
||||
changeDoc,
|
||||
expectedChangesShard1[0],
|
||||
changeCaptureListShard1,
|
||||
expectCommitTimestamp,
|
||||
);
|
||||
expectedChangesShard1.shift();
|
||||
} else {
|
||||
assert.eq(2, changeDoc.documentKey.shard);
|
||||
assert(expectedChangesShard2.length);
|
||||
assertChangeEqualWithCapture(
|
||||
changeDoc,
|
||||
expectedChangesShard2[0],
|
||||
changeCaptureListShard2,
|
||||
expectCommitTimestamp,
|
||||
);
|
||||
expectedChangesShard2.shift();
|
||||
}
|
||||
assertCommitTimestamp(changeDoc);
|
||||
}
|
||||
|
||||
if (expectEndOfTransaction) {
|
||||
assert.soon(() => cursor.hasNext());
|
||||
const changeDoc = cursor.next();
|
||||
assert.eq("endOfTransaction", changeDoc.operationType, changeDoc);
|
||||
}
|
||||
|
||||
assertNoChanges(cursor);
|
||||
}
|
||||
|
||||
const changeStreamCursor = coll.watch([], {showExpandedEvents: true, showCommitTimestamp: true});
|
||||
|
||||
// Insert a document and confirm that the change stream has it.
|
||||
assert.commandWorked(coll.insert({shard: 1, _id: "no-txn-doc-1"}, {writeConcern: {w: "majority"}}));
|
||||
assertWritesVisibleWithCapture(
|
||||
changeStreamCursor,
|
||||
[{operationType: "insert", _id: "no-txn-doc-1"}],
|
||||
[],
|
||||
changeListShard1,
|
||||
changeListShard2,
|
||||
);
|
||||
|
||||
// Insert two documents under each transaction and confirm no change stream updates.
|
||||
assert.commandWorked(
|
||||
sessionColl1.insert([
|
||||
{shard: 1, _id: "txn1-doc-1"},
|
||||
{shard: 2, _id: "txn1-doc-2"},
|
||||
]),
|
||||
);
|
||||
assert.commandWorked(
|
||||
sessionColl2.insert([
|
||||
{shard: 1, _id: "txn2-doc-1"},
|
||||
{shard: 2, _id: "txn2-doc-2"},
|
||||
]),
|
||||
);
|
||||
|
||||
// Update one document under each transaction and confirm no change stream updates.
|
||||
assert.commandWorked(sessionColl1.update({shard: 1, _id: "txn1-doc-1"}, {$set: {"updated": 1}}));
|
||||
assert.commandWorked(sessionColl2.update({shard: 2, _id: "txn2-doc-2"}, {$set: {"updated": 1}}));
|
||||
|
||||
// Update and then remove second doc under each transaction.
|
||||
assert.commandWorked(sessionColl1.update({shard: 2, _id: "txn1-doc-2"}, {$set: {"update-before-delete": 1}}));
|
||||
assert.commandWorked(sessionColl2.update({shard: 1, _id: "txn2-doc-1"}, {$set: {"update-before-delete": 1}}));
|
||||
assert.commandWorked(sessionColl1.remove({shard: 2, _id: "txn1-doc-2"}));
|
||||
assert.commandWorked(sessionColl2.remove({shard: 1, _id: "txn2-doc-2"}));
|
||||
|
||||
// Perform a write outside of a transaction and confirm that the change stream sees only
|
||||
// this write.
|
||||
assert.commandWorked(coll.insert({shard: 2, _id: "no-txn-doc-2"}, {writeConcern: {w: "majority"}}));
|
||||
assertWritesVisibleWithCapture(
|
||||
changeStreamCursor,
|
||||
[],
|
||||
[{operationType: "insert", _id: "no-txn-doc-2"}],
|
||||
changeListShard1,
|
||||
changeListShard2,
|
||||
);
|
||||
|
||||
// Perform a write outside of the transaction.
|
||||
assert.commandWorked(coll.insert({shard: 1, _id: "no-txn-doc-3"}, {writeConcern: {w: "majority"}}));
|
||||
assertWritesVisibleWithCapture(
|
||||
changeStreamCursor,
|
||||
[{operationType: "insert", _id: "no-txn-doc-3"}],
|
||||
[],
|
||||
changeListShard1,
|
||||
changeListShard2,
|
||||
);
|
||||
|
||||
// Commit first transaction and confirm that the change stream sees the changes expected
|
||||
// from each shard.
|
||||
assert.commandWorked(session1.commitTransaction_forTesting());
|
||||
assertWritesVisibleWithCapture(
|
||||
changeStreamCursor,
|
||||
[
|
||||
{operationType: "insert", _id: "txn1-doc-1"},
|
||||
{operationType: "update", _id: "txn1-doc-1"},
|
||||
],
|
||||
[
|
||||
{operationType: "insert", _id: "txn1-doc-2"},
|
||||
{operationType: "update", _id: "txn1-doc-2"},
|
||||
{operationType: "delete", _id: "txn1-doc-2"},
|
||||
],
|
||||
changeListShard1,
|
||||
changeListShard2,
|
||||
true /* expectCommitTimestamp */,
|
||||
);
|
||||
|
||||
// Perform a write outside of the transaction.
|
||||
assert.commandWorked(coll.insert({shard: 2, _id: "no-txn-doc-4"}, {writeConcern: {w: "majority"}}));
|
||||
|
||||
// Abort second transaction and confirm that the change stream sees only the previous
|
||||
// non-transaction write.
|
||||
assert.commandWorked(session2.abortTransaction_forTesting());
|
||||
assertWritesVisibleWithCapture(
|
||||
changeStreamCursor,
|
||||
[],
|
||||
[{operationType: "insert", _id: "no-txn-doc-4"}],
|
||||
changeListShard1,
|
||||
changeListShard2,
|
||||
);
|
||||
changeStreamCursor.close();
|
||||
})();
|
||||
|
||||
//
|
||||
// Open a change stream at each resume point captured for the previous writes. Confirm that the
|
||||
// documents returned match what was returned for the initial change stream.
|
||||
//
|
||||
(function () {
|
||||
/**
|
||||
* Iterates over a list of changes and returns the index of the change whose resume token is
|
||||
* higher than that of 'changeDoc'. It is expected that 'changeList' entries at this index
|
||||
* and beyond will be included in a change stream resumed at 'changeDoc._id'.
|
||||
*/
|
||||
function getPostTokenChangeIndex(changeDoc, changeList) {
|
||||
for (let i = 0; i < changeList.length; ++i) {
|
||||
if (changeDoc._id._data < changeList[i]._id._data) {
|
||||
return i;
|
||||
}
|
||||
}
|
||||
|
||||
return changeList.length;
|
||||
}
|
||||
|
||||
/**
|
||||
* Confirms that the change represented by 'changeDoc' exists in 'shardChangeList' at index
|
||||
* 'changeListIndex'.
|
||||
*/
|
||||
function shardHasDocumentAtChangeListIndex(changeDoc, shardChangeList, changeListIndex) {
|
||||
assert(changeListIndex < shardChangeList.length);
|
||||
|
||||
const expectedChangeDoc = shardChangeList[changeListIndex];
|
||||
assert.eq(changeDoc, expectedChangeDoc);
|
||||
assert.eq(
|
||||
expectedChangeDoc.documentKey,
|
||||
changeDoc.documentKey,
|
||||
tojson(changeDoc) + ", " + tojson(expectedChangeDoc),
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that change stream returns the expected set of documuments when resumed from each
|
||||
* point captured by 'changeList'.
|
||||
*/
|
||||
function confirmResumeForChangeList(changeList, changeListShard1, changeListShard2) {
|
||||
for (let i = 0; i < changeList.length; ++i) {
|
||||
const resumeDoc = changeList[i];
|
||||
let indexShard1 = getPostTokenChangeIndex(resumeDoc, changeListShard1);
|
||||
let indexShard2 = getPostTokenChangeIndex(resumeDoc, changeListShard2);
|
||||
const resumeCursor = coll.watch([], {
|
||||
startAfter: resumeDoc._id,
|
||||
showExpandedEvents: true,
|
||||
showCommitTimestamp: true,
|
||||
});
|
||||
|
||||
while (indexShard1 + indexShard2 < changeListShard1.length + changeListShard2.length) {
|
||||
assert.soon(() => resumeCursor.hasNext());
|
||||
const changeDoc = resumeCursor.next();
|
||||
|
||||
if (changeDoc.operationType === "endOfTransaction") {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (changeDoc.documentKey.shard === 1) {
|
||||
shardHasDocumentAtChangeListIndex(changeDoc, changeListShard1, indexShard1++);
|
||||
} else {
|
||||
assert.eq(2, changeDoc.documentKey.shard);
|
||||
shardHasDocumentAtChangeListIndex(changeDoc, changeListShard2, indexShard2++);
|
||||
}
|
||||
}
|
||||
|
||||
assertNoChanges(resumeCursor);
|
||||
resumeCursor.close();
|
||||
}
|
||||
}
|
||||
|
||||
// Confirm that the sequence of events returned by the stream is consistent when resuming
|
||||
// from any point in the stream on either shard.
|
||||
confirmResumeForChangeList(changeListShard1, changeListShard1, changeListShard2);
|
||||
confirmResumeForChangeList(changeListShard2, changeListShard1, changeListShard2);
|
||||
})();
|
||||
|
||||
st.stop();
|
||||
|
|
@ -347,11 +347,21 @@ Document ChangeStreamDefaultEventTransformation::applyTransformation(const Docum
|
|||
auto deltaDesc = change_stream_document_diff_parser::parseDiff(
|
||||
diffObj.getDocument().toBson());
|
||||
|
||||
// If the 'showExpandedEvents' flag is set, the update description will also
|
||||
// contain the 'disambiguatedPaths' sub-field. The field will not be emitted
|
||||
// otherwise.
|
||||
if (_changeStreamSpec.getShowExpandedEvents()) {
|
||||
updateDescription = Value(Document{
|
||||
{"updatedFields", std::move(deltaDesc.updatedFields)},
|
||||
{"removedFields", std::move(deltaDesc.removedFields)},
|
||||
{"truncatedArrays", std::move(deltaDesc.truncatedArrays)},
|
||||
{"disambiguatedPaths", Value(std::move(deltaDesc.disambiguatedPaths))}});
|
||||
{"disambiguatedPaths", std::move(deltaDesc.disambiguatedPaths)}});
|
||||
} else {
|
||||
updateDescription = Value(
|
||||
Document{{"updatedFields", std::move(deltaDesc.updatedFields)},
|
||||
{"removedFields", std::move(deltaDesc.removedFields)},
|
||||
{"truncatedArrays", std::move(deltaDesc.truncatedArrays)}});
|
||||
}
|
||||
}
|
||||
} else if (!oplogVersion.missing() || id.missing()) {
|
||||
// This is not a replacement op, and we did not see a valid update version number.
|
||||
|
|
@ -525,13 +535,15 @@ Document ChangeStreamDefaultEventTransformation::applyTransformation(const Docum
|
|||
doc.addField(DocumentSourceChangeStream::kOperationTypeField, Value(operationType));
|
||||
doc.addField(DocumentSourceChangeStream::kClusterTimeField, Value(resumeTokenData.clusterTime));
|
||||
|
||||
if (_changeStreamSpec.getShowCommitTimestamp()) {
|
||||
// Commit timestamp for CRUD events in prepared transactions.
|
||||
auto commitTimestamp = input[DocumentSourceChangeStream::kCommitTimestampField];
|
||||
if (!commitTimestamp.missing()) {
|
||||
doc.addField(DocumentSourceChangeStream::kCommitTimestampField, commitTimestamp);
|
||||
}
|
||||
}
|
||||
|
||||
if (!uuid.missing()) {
|
||||
if (_changeStreamSpec.getShowExpandedEvents() && !uuid.missing()) {
|
||||
doc.addField(DocumentSourceChangeStream::kCollectionUuidField, uuid);
|
||||
}
|
||||
|
||||
|
|
@ -569,7 +581,7 @@ Document ChangeStreamDefaultEventTransformation::applyTransformation(const Docum
|
|||
// The event may have a documentKey OR an operationDescription, but not both. We already
|
||||
// validated this while creating the resume token.
|
||||
doc.addField(DocumentSourceChangeStream::kDocumentKeyField, std::move(documentKey));
|
||||
if (!operationDescription.missing()) {
|
||||
if (_changeStreamSpec.getShowExpandedEvents() && !operationDescription.missing()) {
|
||||
doc.addField(DocumentSourceChangeStream::kOperationDescriptionField,
|
||||
std::move(operationDescription));
|
||||
}
|
||||
|
|
@ -658,14 +670,17 @@ Document ChangeStreamViewDefinitionEventTransformation::applyTransformation(
|
|||
Document opDesc = copyDocExceptFields(oField, {"_id"_sd});
|
||||
operationDescription = Value(opDesc);
|
||||
|
||||
if (_changeStreamSpec.getShowExpandedEvents()) {
|
||||
// Populate 'nsType' field with either "view" or "timeseries".
|
||||
auto collectionType = determineCollectionType(oField, nss.dbName());
|
||||
tassert(8814202,
|
||||
tassert(
|
||||
8814202,
|
||||
"'operationDescription.type' should always resolve to 'view' or 'timeseries' "
|
||||
"for view creation event",
|
||||
collectionType == CollectionType::kView ||
|
||||
collectionType == CollectionType::kTimeseries);
|
||||
nsType = Value(toString(collectionType));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case repl::OpTypeEnum::kUpdate: {
|
||||
|
|
|
|||
|
|
@ -315,7 +315,8 @@ std::vector<Document> ChangeStreamStageTest::getApplyOpsResults(
|
|||
Document ChangeStreamStageTest::makeExpectedUpdateEvent(Timestamp ts,
|
||||
const NamespaceString& nss,
|
||||
BSONObj documentKey,
|
||||
Document updateDescription) {
|
||||
Document updateDescription,
|
||||
bool expandedEvents) {
|
||||
return Document{
|
||||
{DocumentSourceChangeStream::kIdField,
|
||||
change_stream_test_helper::makeResumeToken(ts,
|
||||
|
|
@ -326,7 +327,7 @@ Document ChangeStreamStageTest::makeExpectedUpdateEvent(Timestamp ts,
|
|||
DocumentSourceChangeStream::kUpdateOpType},
|
||||
{DocumentSourceChangeStream::kClusterTimeField, ts},
|
||||
{DocumentSourceChangeStream::kCollectionUuidField,
|
||||
Value{change_stream_test_helper::testUuid()}},
|
||||
expandedEvents ? Value{change_stream_test_helper::testUuid()} : Value{}},
|
||||
{DocumentSourceChangeStream::kWallTimeField, Date_t()},
|
||||
{DocumentSourceChangeStream::kNamespaceField,
|
||||
Document{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
|
|||
|
|
@ -154,7 +154,8 @@ public:
|
|||
Document makeExpectedUpdateEvent(Timestamp ts,
|
||||
const NamespaceString& nss,
|
||||
BSONObj documentKey,
|
||||
Document updateDescription);
|
||||
Document updateDescription,
|
||||
bool expandedEvents = false);
|
||||
|
||||
/**
|
||||
* Helper function to do a $v:2 delta oplog test.
|
||||
|
|
|
|||
|
|
@ -61,6 +61,8 @@ static const NamespaceString nss =
|
|||
static const BSONObj kDefaultSpec = BSON("$changeStream" << BSONObj());
|
||||
static const BSONObj kShowExpandedEventsSpec =
|
||||
BSON("$changeStream" << BSON("showExpandedEvents" << true));
|
||||
static const BSONObj kShowCommitTimestampSpec =
|
||||
BSON("$changeStream" << BSON("showCommitTimestamp" << true));
|
||||
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -197,6 +197,14 @@ structs:
|
|||
expanded events and parameters that may not be part of the stable API.
|
||||
query_shape: literal
|
||||
|
||||
showCommitTimestamp:
|
||||
type: optionalBool
|
||||
description:
|
||||
A flag indicating whether the commit timestamp should be emitted for
|
||||
DML events inside prepared transactions.
|
||||
This is an internal fields and should not be documented for public use.
|
||||
query_shape: literal
|
||||
|
||||
showRawUpdateDescription:
|
||||
cpp_name: showRawUpdateDescription
|
||||
type: optionalBool
|
||||
|
|
|
|||
|
|
@ -107,8 +107,8 @@ using DSChangeStream = DocumentSourceChangeStream;
|
|||
// Deterministic values used for testing
|
||||
const UUID testConstUuid = UUID::parse("6948DF80-14BD-4E04-8842-7668D9C001F5").getValue();
|
||||
|
||||
void assertCommitTimestamp(bool expandedEvents, const Document& doc) {
|
||||
if (expandedEvents) {
|
||||
void assertCommitTimestamp(bool showCommitTimestamp, const Document& doc) {
|
||||
if (showCommitTimestamp) {
|
||||
ASSERT_EQ(kDefaultCommitTs, doc[DSChangeStream::kCommitTimestampField].getTimestamp());
|
||||
} else {
|
||||
ASSERT_TRUE(doc[DSChangeStream::kCommitTimestampField].missing());
|
||||
|
|
@ -570,7 +570,6 @@ TEST_F(ChangeStreamStageTest, TransformInsertDocKeyXAndId) {
|
|||
kDefaultTs, testUuid(), BSON("x" << 2 << "_id" << 1), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}, {"x", 2}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -601,7 +600,6 @@ TEST_F(ChangeStreamStageTest, TransformInsertDocKeyIdAndX) {
|
|||
kDefaultTs, testUuid(), BSON("_id" << 1 << "x" << 2), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"x", 2}, {"_id", 1}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -623,7 +621,6 @@ TEST_F(ChangeStreamStageTest, TransformInsertDocKeyJustId) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), BSON("_id" << 1), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}, {"x", 2}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -660,7 +657,6 @@ TEST_F(ChangeStreamStageTest, TransformInsertFromMigrateShowMigrations) {
|
|||
kDefaultTs, testUuid(), BSON("_id" << 1 << "x" << 2), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"x", 2}, {"_id", 1}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -686,8 +682,7 @@ TEST_F(ChangeStreamStageTest, TransformUpdateFields) {
|
|||
o2,
|
||||
D{{"updatedFields", D{{"y", 1}}},
|
||||
{"removedFields", std::vector<V>()},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>()}});
|
||||
|
||||
checkTransformation(updateField, expectedUpdateField);
|
||||
}
|
||||
|
|
@ -710,7 +705,8 @@ TEST_F(ChangeStreamStageTest, TransformUpdateFieldsShowExpandedEvents) {
|
|||
D{{"updatedFields", D{{"y", 1}}},
|
||||
{"removedFields", std::vector<V>()},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"disambiguatedPaths", D{}}},
|
||||
true);
|
||||
checkTransformation(updateField, expectedUpdateField, kShowExpandedEventsSpec);
|
||||
}
|
||||
|
||||
|
|
@ -721,8 +717,7 @@ TEST_F(ChangeStreamStageTest, TransformSimpleDeltaOplogUpdatedFields) {
|
|||
runUpdateV2OplogTest(diff,
|
||||
D{{"updatedFields", D{{"a", 1}, {"b", "updated"_sd}}},
|
||||
{"removedFields", std::vector<V>{}},
|
||||
{"truncatedArrays", std::vector<V>{}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformSimpleDeltaOplogInsertFields) {
|
||||
|
|
@ -732,8 +727,7 @@ TEST_F(ChangeStreamStageTest, TransformSimpleDeltaOplogInsertFields) {
|
|||
runUpdateV2OplogTest(diff,
|
||||
D{{"updatedFields", D{{"a", 1}, {"b", "updated"_sd}}},
|
||||
{"removedFields", std::vector<V>{}},
|
||||
{"truncatedArrays", std::vector<V>{}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformSimpleDeltaOplogRemovedFields) {
|
||||
|
|
@ -742,8 +736,7 @@ TEST_F(ChangeStreamStageTest, TransformSimpleDeltaOplogRemovedFields) {
|
|||
runUpdateV2OplogTest(diff,
|
||||
D{{"updatedFields", D{}},
|
||||
{"removedFields", std::vector<V>{V("a"_sd), V("b"_sd)}},
|
||||
{"truncatedArrays", std::vector<V>{}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformComplexDeltaOplog) {
|
||||
|
|
@ -757,8 +750,7 @@ TEST_F(ChangeStreamStageTest, TransformComplexDeltaOplog) {
|
|||
runUpdateV2OplogTest(diff,
|
||||
D{{"updatedFields", D{{"c", 1}, {"d", "updated"_sd}, {"e", 2}, {"f", 3}}},
|
||||
{"removedFields", std::vector<V>{V("a"_sd), V("b"_sd)}},
|
||||
{"truncatedArrays", std::vector<V>{}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformDeltaOplogSubObjectDiff) {
|
||||
|
|
@ -776,8 +768,7 @@ TEST_F(ChangeStreamStageTest, TransformDeltaOplogSubObjectDiff) {
|
|||
D{{"updatedFields",
|
||||
D{{"c", 1}, {"d", "updated"_sd}, {"subObj.c", 1}, {"subObj.d", "updated"_sd}}},
|
||||
{"removedFields", std::vector<V>{V("subObj.a"_sd), V("subObj.b"_sd)}},
|
||||
{"truncatedArrays", std::vector<V>{}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformDeltaOplogSubArrayDiff) {
|
||||
|
|
@ -794,8 +785,7 @@ TEST_F(ChangeStreamStageTest, TransformDeltaOplogSubArrayDiff) {
|
|||
{"removedFields", std::vector<V>{}},
|
||||
{"truncatedArrays",
|
||||
std::vector<V>{V{D{{"field", "arrField"_sd}, {"newSize", 10}}},
|
||||
V{D{{"field", "arrField2"_sd}, {"newSize", 20}}}}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
V{D{{"field", "arrField2"_sd}, {"newSize", 20}}}}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformDeltaOplogSubArrayDiffWithEmptyStringField) {
|
||||
|
|
@ -810,8 +800,7 @@ TEST_F(ChangeStreamStageTest, TransformDeltaOplogSubArrayDiffWithEmptyStringFiel
|
|||
diff,
|
||||
D{{"updatedFields", D{{".0", 1}, {".1", D{{"a", 1}}}}},
|
||||
{"removedFields", std::vector<V>{}},
|
||||
{"truncatedArrays", std::vector<V>{V{D{{"field", ""_sd}, {"newSize", 10}}}}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{V{D{{"field", ""_sd}, {"newSize", 10}}}}}});
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransformDeltaOplogNestedComplexSubDiffs) {
|
||||
|
|
@ -843,8 +832,7 @@ TEST_F(ChangeStreamStageTest, TransformDeltaOplogNestedComplexSubDiffs) {
|
|||
{"subObj.a", 1},
|
||||
}},
|
||||
{"removedFields", std::vector<V>{V("subObj.b"_sd)}},
|
||||
{"truncatedArrays", std::vector<V>{V{D{{"field", "arrField"_sd}, {"newSize", 10}}}}},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>{V{D{{"field", "arrField"_sd}, {"newSize", 10}}}}}});
|
||||
}
|
||||
|
||||
// Legacy documents might not have an _id field; then the document key is the full (post-update)
|
||||
|
|
@ -866,8 +854,7 @@ TEST_F(ChangeStreamStageTest, TransformUpdateFieldsLegacyNoId) {
|
|||
o2,
|
||||
D{{"updatedFields", D{{"y", 1}}},
|
||||
{"removedFields", std::vector<V>()},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>()}});
|
||||
checkTransformation(updateField, expectedUpdateField);
|
||||
}
|
||||
|
||||
|
|
@ -888,8 +875,7 @@ TEST_F(ChangeStreamStageTest, TransformRemoveFields) {
|
|||
o2,
|
||||
D{{"updatedFields", D{}},
|
||||
{"removedFields", {"y"_sd}},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>()}});
|
||||
checkTransformation(removeField, expectedUpdateField);
|
||||
} // namespace
|
||||
|
||||
|
|
@ -909,7 +895,6 @@ TEST_F(ChangeStreamStageTest, TransformReplace) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), o2, DSChangeStream::kReplaceOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kReplaceOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}, {"x", 2}, {"y", 1}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -958,7 +943,6 @@ TEST_F(ChangeStreamStageTest, TransformDelete) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), o, DSChangeStream::kDeleteOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDeleteOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kDocumentKeyField, D{{"_id", 1}, {"x", 2}}},
|
||||
|
|
@ -1037,7 +1021,6 @@ TEST_F(ChangeStreamStageTest, TransformDeleteFromMigrateShowMigrations) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), o, DSChangeStream::kDeleteOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDeleteOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kDocumentKeyField, D{{"_id", 1}}},
|
||||
|
|
@ -1054,7 +1037,6 @@ TEST_F(ChangeStreamStageTest, TransformDrop) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), Value(), DSChangeStream::kDropCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDropCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
};
|
||||
|
|
@ -1137,10 +1119,8 @@ TEST_F(ChangeStreamStageTest, TransformRename) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
Document expectedInvalidate{
|
||||
{DSChangeStream::kIdField,
|
||||
|
|
@ -1227,11 +1207,9 @@ TEST_F(ChangeStreamStageTest, TransformRenameTarget) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField,
|
||||
D{{"db", otherColl.db_forTest()}, {"coll", otherColl.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
Document expectedInvalidate{
|
||||
{DSChangeStream::kIdField,
|
||||
|
|
@ -1353,7 +1331,7 @@ TEST_F(ChangeStreamStageTest, TransformReshardBegin) {
|
|||
true, // fromMigrate
|
||||
o2Field.toBSON());
|
||||
|
||||
auto spec = fromjson("{$changeStream: {showMigrationEvents: true}}");
|
||||
auto spec = fromjson("{$changeStream: {showMigrationEvents: true, showExpandedEvents: true}}");
|
||||
|
||||
const auto opDesc = V{D{{"reshardingUUID", reshardingUuid}}};
|
||||
Document expectedReshardingBegin{
|
||||
|
|
@ -1383,7 +1361,7 @@ TEST_F(ChangeStreamStageTest, TransformReshardBlockingWrites) {
|
|||
false, // fromMigrate
|
||||
o2Field.toBSON());
|
||||
|
||||
auto spec = fromjson("{$changeStream: {showSystemEvents: true}}");
|
||||
auto spec = fromjson("{$changeStream: {showSystemEvents: true, showExpandedEvents: true}}");
|
||||
|
||||
const auto opDesc =
|
||||
D{{"reshardingUUID", reshardingUuid}, {"type", resharding::kReshardFinalOpLogType}};
|
||||
|
|
@ -1415,8 +1393,9 @@ TEST_F(ChangeStreamStageTest, TransformReshardDoneCatchUp) {
|
|||
true, // fromMigrate
|
||||
o2Field.toBSON());
|
||||
|
||||
auto spec =
|
||||
fromjson("{$changeStream: {showMigrationEvents: true, allowToRunOnSystemNS: true}}");
|
||||
auto spec = fromjson(
|
||||
"{$changeStream: {showMigrationEvents: true, allowToRunOnSystemNS: true, "
|
||||
"showExpandedEvents: true}}");
|
||||
auto expCtx = getExpCtx();
|
||||
expCtx->setNamespaceString(temporaryNs);
|
||||
|
||||
|
|
@ -1640,6 +1619,31 @@ TEST_F(ChangeStreamStageTest, CommitCommandReturnsOperationsFromPreparedTransact
|
|||
boost::none, // _id
|
||||
boost::none); // needsRetryImage
|
||||
|
||||
{
|
||||
// No expanded events.
|
||||
|
||||
// When the DocumentSourceChangeStreamTransform sees the "commitTransaction" oplog entry, we
|
||||
// expect it to return the insert op within our 'preparedApplyOps' oplog entry.
|
||||
Document expectedResult{
|
||||
{DSChangeStream::kTxnNumberField, static_cast<int>(*sessionInfo.getTxnNumber())},
|
||||
{DSChangeStream::kLsidField, Document{{sessionInfo.getSessionId()->toBSON()}}},
|
||||
{DSChangeStream::kIdField,
|
||||
makeResumeToken(kDefaultTs, testUuid(), BSONObj(), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 123}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kDocumentKeyField, D{}},
|
||||
};
|
||||
|
||||
checkTransformation(oplogEntry, expectedResult, kDefaultSpec, {}, {preparedTransaction});
|
||||
}
|
||||
|
||||
{
|
||||
// Expanded events: this will additionally emit the 'commitTimestamp' and 'collectionUUID'
|
||||
// fields.
|
||||
|
||||
// When the DocumentSourceChangeStreamTransform sees the "commitTransaction" oplog entry, we
|
||||
// expect it to return the insert op within our 'preparedApplyOps' oplog entry.
|
||||
Document expectedResult{
|
||||
|
|
@ -1650,14 +1654,15 @@ TEST_F(ChangeStreamStageTest, CommitCommandReturnsOperationsFromPreparedTransact
|
|||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCommitTimestampField, kDefaultCommitTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 123}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kDocumentKeyField, D{}},
|
||||
};
|
||||
|
||||
checkTransformation(oplogEntry, expectedResult, kDefaultSpec, {}, {preparedTransaction});
|
||||
checkTransformation(
|
||||
oplogEntry, expectedResult, kShowCommitTimestampSpec, {}, {preparedTransaction});
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest, TransactionWithMultipleOplogEntries) {
|
||||
|
|
@ -1739,7 +1744,7 @@ TEST_F(ChangeStreamStageTest, TransactionWithMultipleOplogEntries) {
|
|||
ASSERT_EQ(nextDoc[DSChangeStream::kFullDocumentField]["_id"].getInt(), 123);
|
||||
// Note that we never expect to see a 'commitTimestamp' event field because the events are
|
||||
// not from a prepared transaction.
|
||||
assertCommitTimestamp(false /* expandedEvents */, nextDoc);
|
||||
assertCommitTimestamp(false /* showCommitTimestamp */, nextDoc);
|
||||
ASSERT_EQ(
|
||||
nextDoc["lsid"].getDocument().toBson().woCompare(sessionInfo.getSessionId()->toBSON()), 0);
|
||||
auto resumeToken = ResumeToken::parse(nextDoc["_id"].getDocument()).toDocument();
|
||||
|
|
@ -1760,7 +1765,7 @@ TEST_F(ChangeStreamStageTest, TransactionWithMultipleOplogEntries) {
|
|||
ASSERT_EQ(nextDoc[DSChangeStream::kFullDocumentField]["_id"].getInt(), 456);
|
||||
// Note that we never expect to see a 'commitTimestamp' event field because the events are
|
||||
// not from a prepared transaction.
|
||||
assertCommitTimestamp(false /* expandedEvents */, nextDoc);
|
||||
assertCommitTimestamp(false /* showCommitTimestamp */, nextDoc);
|
||||
ASSERT_EQ(
|
||||
nextDoc["lsid"].getDocument().toBson().woCompare(sessionInfo.getSessionId()->toBSON()), 0);
|
||||
resumeToken = ResumeToken::parse(nextDoc["_id"].getDocument()).toDocument();
|
||||
|
|
@ -1781,7 +1786,7 @@ TEST_F(ChangeStreamStageTest, TransactionWithMultipleOplogEntries) {
|
|||
ASSERT_EQ(nextDoc[DSChangeStream::kFullDocumentField]["_id"].getInt(), 789);
|
||||
// Note that we never expect to see a 'commitTimestamp' event field because the events are
|
||||
// not from a prepared transaction.
|
||||
assertCommitTimestamp(false /* expandedEvents */, nextDoc);
|
||||
assertCommitTimestamp(false /* showCommitTimestamp */, nextDoc);
|
||||
ASSERT_EQ(
|
||||
nextDoc["lsid"].getDocument().toBson().woCompare(sessionInfo.getSessionId()->toBSON()), 0);
|
||||
resumeToken = ResumeToken::parse(nextDoc["_id"].getDocument()).toDocument();
|
||||
|
|
@ -2252,7 +2257,7 @@ TEST_F(ChangeStreamStageTest, PreparedTransactionEndingWithEmptyApplyOps) {
|
|||
ASSERT_EQ(nextDoc[DSChangeStream::kOperationTypeField].getString(),
|
||||
DSChangeStream::kInsertOpType);
|
||||
ASSERT_EQ(nextDoc[DSChangeStream::kFullDocumentField]["_id"].getInt(), 123);
|
||||
assertCommitTimestamp(true /* showExpandedEvents */, nextDoc);
|
||||
assertCommitTimestamp(false /* showCommitTimestamp */, nextDoc);
|
||||
ASSERT_EQ(
|
||||
nextDoc["lsid"].getDocument().toBson().woCompare(sessionInfo.getSessionId()->toBSON()), 0);
|
||||
auto resumeToken = ResumeToken::parse(nextDoc["_id"].getDocument()).toDocument();
|
||||
|
|
@ -2272,7 +2277,7 @@ TEST_F(ChangeStreamStageTest, PreparedTransactionEndingWithEmptyApplyOps) {
|
|||
ASSERT_EQ(nextDoc[DSChangeStream::kOperationTypeField].getString(),
|
||||
DSChangeStream::kInsertOpType);
|
||||
ASSERT_EQ(nextDoc[DSChangeStream::kFullDocumentField]["_id"].getInt(), 456);
|
||||
assertCommitTimestamp(true /* expandedEvents */, nextDoc);
|
||||
assertCommitTimestamp(false /* showCommitTimestamp */, nextDoc);
|
||||
ASSERT_EQ(
|
||||
nextDoc["lsid"].getDocument().toBson().woCompare(sessionInfo.getSessionId()->toBSON()), 0);
|
||||
resumeToken = ResumeToken::parse(nextDoc["_id"].getDocument()).toDocument();
|
||||
|
|
@ -2434,8 +2439,7 @@ TEST_F(ChangeStreamStageTest, ClusterTimeMatchesOplogEntry) {
|
|||
o2,
|
||||
D{{"updatedFields", D{{"y", 1}}},
|
||||
{"removedFields", std::vector<V>()},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>()}});
|
||||
checkTransformation(updateField, expectedUpdateField);
|
||||
|
||||
// Test the 'clusterTime' field is copied from the oplog entry for a collection drop.
|
||||
|
|
@ -2447,7 +2451,6 @@ TEST_F(ChangeStreamStageTest, ClusterTimeMatchesOplogEntry) {
|
|||
makeResumeToken(ts, testUuid(), Value(), DSChangeStream::kDropCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDropCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, ts},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
};
|
||||
|
|
@ -2470,10 +2473,8 @@ TEST_F(ChangeStreamStageTest, ClusterTimeMatchesOplogEntry) {
|
|||
makeResumeToken(ts, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, ts},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
checkTransformation(rename, expectedRename);
|
||||
}
|
||||
|
|
@ -2597,10 +2598,8 @@ TEST_F(ChangeStreamStageTest, DocumentSourceChangeStreamTransformTransformSingle
|
|||
makeResumeToken(kDefaultTs, testUuid(), operationDescription, "eventType1"_sd)},
|
||||
{DSChangeStream::kOperationTypeField, "eventType1"_sd},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, operationDescription}};
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}}};
|
||||
|
||||
auto stage =
|
||||
exec::agg::MockStage::createForTest({Document{entry.getEntry().toBSON()}}, getExpCtx());
|
||||
|
|
@ -2651,6 +2650,85 @@ TEST_F(ChangeStreamStageTest, DocumentSourceChangeStreamTransformTransformMultip
|
|||
false,
|
||||
BSON("eventType2" << true).addFields(operationDescriptionEvent2));
|
||||
|
||||
Document expectedDoc1{
|
||||
{DSChangeStream::kIdField,
|
||||
makeResumeToken(kDefaultTs, testUuid(), operationDescriptionEvent1, "eventType1"_sd)},
|
||||
{DSChangeStream::kOperationTypeField, "eventType1"_sd},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}}};
|
||||
|
||||
Document expectedDoc2{
|
||||
{DSChangeStream::kIdField,
|
||||
makeResumeToken(kDefaultTs, testUuid(), operationDescriptionEvent2, "eventType2"_sd)},
|
||||
{DSChangeStream::kOperationTypeField, "eventType2"_sd},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}}};
|
||||
|
||||
std::deque<exec::agg::GetNextResult> docs;
|
||||
docs.push_back(Document{entry1.getEntry().toBSON()});
|
||||
docs.push_back(Document{entry2.getEntry().toBSON()});
|
||||
docs.push_back(Document{entry1.getEntry().toBSON()});
|
||||
auto stage = exec::agg::MockStage::createForTest(std::move(docs), getExpCtx());
|
||||
auto transformDS =
|
||||
DocumentSourceChangeStreamTransform::createFromBson(spec.firstElement(), getExpCtx());
|
||||
auto transformStage = exec::agg::buildStage(transformDS);
|
||||
|
||||
transformStage->setSource(stage.get());
|
||||
|
||||
auto next = transformStage->getNext();
|
||||
ASSERT_TRUE(next.isAdvanced());
|
||||
ASSERT_DOCUMENT_EQ(next.releaseDocument(), expectedDoc1);
|
||||
|
||||
next = transformStage->getNext();
|
||||
ASSERT_TRUE(next.isAdvanced());
|
||||
ASSERT_DOCUMENT_EQ(next.releaseDocument(), expectedDoc2);
|
||||
|
||||
next = transformStage->getNext();
|
||||
ASSERT_TRUE(next.isAdvanced());
|
||||
ASSERT_DOCUMENT_EQ(next.releaseDocument(), expectedDoc1);
|
||||
|
||||
next = transformStage->getNext();
|
||||
ASSERT_TRUE(next.isEOF());
|
||||
}
|
||||
|
||||
TEST_F(ChangeStreamStageTest,
|
||||
DocumentSourceChangeStreamTransformTransformMultipleSupportedEventsExpandedEvents) {
|
||||
BSONObj spec =
|
||||
BSON(DocumentSourceChangeStreamTransform::kStageName
|
||||
<< BSON("resumeAfter" << makeResumeToken(kDefaultTs, Value(), Value(), "eventType1"_sd)
|
||||
<< "showExpandedEvents" << true << "supportedEvents"
|
||||
<< BSON_ARRAY("eventType1" << "eventType2")));
|
||||
|
||||
BSONObj operationDescriptionEvent1 =
|
||||
BSON("foo" << "bar"
|
||||
<< "baz"
|
||||
<< "qux"
|
||||
<< "sub" << BSON("sub1" << true << "sub2" << false));
|
||||
BSONObj operationDescriptionEvent2 = BSON("some" << BSON("that" << "will"
|
||||
<< "end"
|
||||
<< "up"
|
||||
<< "in"
|
||||
<< "result"));
|
||||
|
||||
auto entry1 = makeOplogEntry(OpTypeEnum::kNoop,
|
||||
nss,
|
||||
BSONObj(),
|
||||
testUuid(),
|
||||
false,
|
||||
BSON("eventType1" << BSON("will" << "be"
|
||||
<< "removed"
|
||||
<< "too"))
|
||||
.addFields(operationDescriptionEvent1));
|
||||
|
||||
auto entry2 = makeOplogEntry(OpTypeEnum::kNoop,
|
||||
nss,
|
||||
BSONObj(),
|
||||
testUuid(),
|
||||
false,
|
||||
BSON("eventType2" << true).addFields(operationDescriptionEvent2));
|
||||
|
||||
Document expectedDoc1{
|
||||
{DSChangeStream::kIdField,
|
||||
makeResumeToken(kDefaultTs, testUuid(), operationDescriptionEvent1, "eventType1"_sd)},
|
||||
|
|
@ -3173,7 +3251,6 @@ TEST_F(ChangeStreamStageTest, CloseCursorOnInvalidateEntries) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), Value(), DSChangeStream::kDropCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDropCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
};
|
||||
|
|
@ -3249,7 +3326,6 @@ TEST_F(ChangeStreamStageTest, DocumentKeyShouldNotIncludeShardKeyWhenNoO2FieldIn
|
|||
makeResumeToken(ts, uuid, D{{"_id", 2}}, DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, ts},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 2}, {"shardKey", 3}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -3298,7 +3374,6 @@ TEST_F(ChangeStreamStageTest, DocumentKeyShouldUseO2FieldInOplog) {
|
|||
makeResumeToken(ts, uuid, insertDoc, DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, ts},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 2}, {"shardKey", 3}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -3354,11 +3429,9 @@ TEST_F(ChangeStreamStageTest, RenameFromSystemToUserCollectionShouldIncludeNotif
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField,
|
||||
D{{"db", systemColl.db_forTest()}, {"coll", systemColl.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
checkTransformation(rename, expectedRename);
|
||||
}
|
||||
|
|
@ -3381,10 +3454,8 @@ TEST_F(ChangeStreamStageTest, RenameFromUserToSystemCollectionShouldIncludeNotif
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
checkTransformation(rename, expectedRename);
|
||||
}
|
||||
|
|
@ -3464,7 +3535,6 @@ TEST_F(ChangeStreamStageDBTest, TransformInsert) {
|
|||
kDefaultTs, testUuid(), BSON("x" << 2 << "_id" << 1), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}, {"x", 2}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -3513,7 +3583,6 @@ TEST_F(ChangeStreamStageDBTest, InsertOnOtherCollections) {
|
|||
kDefaultTs, testUuid(), BSON("x" << 2 << "_id" << 1), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}, {"x", 2}}},
|
||||
{DSChangeStream::kNamespaceField,
|
||||
|
|
@ -3576,7 +3645,6 @@ TEST_F(ChangeStreamStageDBTest, TransformsEntriesForLegalClientCollectionsWithSy
|
|||
kDefaultTs, testUuid(), BSON("_id" << 1), DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", ns.db_forTest()}, {"coll", ns.coll()}}},
|
||||
|
|
@ -3614,8 +3682,7 @@ TEST_F(ChangeStreamStageDBTest, TransformUpdateFields) {
|
|||
o2,
|
||||
D{{"updatedFields", D{{"y", 1}}},
|
||||
{"removedFields", std::vector<V>()},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>()}});
|
||||
checkTransformation(updateField, expectedUpdateField);
|
||||
}
|
||||
|
||||
|
|
@ -3636,8 +3703,7 @@ TEST_F(ChangeStreamStageDBTest, TransformRemoveFields) {
|
|||
o2,
|
||||
D{{"updatedFields", D{}},
|
||||
{"removedFields", {"y"_sd}},
|
||||
{"truncatedArrays", std::vector<V>()},
|
||||
{"disambiguatedPaths", D{}}});
|
||||
{"truncatedArrays", std::vector<V>()}});
|
||||
checkTransformation(removeField, expectedRemoveField);
|
||||
}
|
||||
|
||||
|
|
@ -3657,7 +3723,6 @@ TEST_F(ChangeStreamStageDBTest, TransformReplace) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), o2, DSChangeStream::kReplaceOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kReplaceOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 1}, {"x", 2}, {"y", 1}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -3681,7 +3746,6 @@ TEST_F(ChangeStreamStageDBTest, TransformDelete) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), o, DSChangeStream::kDeleteOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDeleteOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kDocumentKeyField, D{{"_id", 1}, {"x", 2}}},
|
||||
|
|
@ -3728,7 +3792,6 @@ TEST_F(ChangeStreamStageDBTest, TransformDeleteFromMigrateShowMigrations) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), o, DSChangeStream::kDeleteOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDeleteOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kDocumentKeyField, D{{"_id", 1}, {"x", 2}}},
|
||||
|
|
@ -3744,7 +3807,6 @@ TEST_F(ChangeStreamStageDBTest, TransformDrop) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), Value(), DSChangeStream::kDropCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kDropCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
};
|
||||
|
|
@ -3765,10 +3827,8 @@ TEST_F(ChangeStreamStageDBTest, TransformRename) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
checkTransformation(rename, expectedRename);
|
||||
}
|
||||
|
|
@ -3865,11 +3925,9 @@ TEST_F(ChangeStreamStageDBTest, RenameFromSystemToUserCollectionShouldIncludeNot
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField,
|
||||
D{{"db", systemColl.db_forTest()}, {"coll", systemColl.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
checkTransformation(rename, expectedRename);
|
||||
}
|
||||
|
|
@ -3892,10 +3950,8 @@ TEST_F(ChangeStreamStageDBTest, RenameFromUserToSystemCollectionShouldIncludeNot
|
|||
makeResumeToken(kDefaultTs, testUuid(), opDesc, DSChangeStream::kRenameCollectionOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kRenameCollectionOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
{DSChangeStream::kOperationDescriptionField, opDesc},
|
||||
};
|
||||
checkTransformation(rename, expectedRename);
|
||||
}
|
||||
|
|
@ -3941,7 +3997,6 @@ TEST_F(ChangeStreamStageDBTest, DocumentKeyShouldNotIncludeShardKeyWhenNoO2Field
|
|||
makeResumeToken(ts, uuid, D{{"_id", 2}}, DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, ts},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 2}, {"shardKey", 3}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -3985,7 +4040,6 @@ TEST_F(ChangeStreamStageDBTest, DocumentKeyShouldUseO2FieldInOplog) {
|
|||
makeResumeToken(ts, uuid, insertDoc, DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, ts},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 2}, {"shardKey", 3}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -4055,7 +4109,6 @@ TEST_F(ChangeStreamStageDBTest, ResumeAfterWithTokenFromDropDatabase) {
|
|||
makeResumeToken(kDefaultTs, testUuid(), insertDoc, DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 2}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
@ -4092,7 +4145,6 @@ TEST_F(ChangeStreamStageDBTest, StartAfterSucceedsEvenIfResumeTokenDoesNotContai
|
|||
makeResumeToken(kDefaultTs, uuid, insertDoc, DSChangeStream::kInsertOpType)},
|
||||
{DSChangeStream::kOperationTypeField, DSChangeStream::kInsertOpType},
|
||||
{DSChangeStream::kClusterTimeField, kDefaultTs},
|
||||
{DSChangeStream::kCollectionUuidField, testUuid()},
|
||||
{DSChangeStream::kWallTimeField, Date_t()},
|
||||
{DSChangeStream::kFullDocumentField, D{{"_id", 2}}},
|
||||
{DSChangeStream::kNamespaceField, D{{"db", nss.db_forTest()}, {"coll", nss.coll()}}},
|
||||
|
|
|
|||
|
|
@ -213,6 +213,11 @@ int64_t ReshardingChangeStreamsMonitor::numBatchesForTest() {
|
|||
|
||||
std::vector<BSONObj> ReshardingChangeStreamsMonitor::_makeAggregatePipeline() const {
|
||||
DocumentSourceChangeStreamSpec changeStreamSpec;
|
||||
|
||||
// This field must be enabled so that change streams return the 'commitTimestamp' field for
|
||||
// events that are part of a prepared transaction.
|
||||
changeStreamSpec.setShowCommitTimestamp(true);
|
||||
|
||||
changeStreamSpec.setAllowToRunOnSystemNS(_monitorNss.isSystem());
|
||||
// The monitor for a recipient needs to set 'showMigrationEvents' to true since the events
|
||||
// against the temporary resharding collection are only output when 'showMigrationEvents'
|
||||
|
|
|
|||
|
|
@ -1189,6 +1189,7 @@ TEST_F(ReshardingChangeStreamsMonitorTest, TestChangeStreamMonitorSettingsForDon
|
|||
ASSERT_TRUE(donorChangeStreamSpec.getShowSystemEvents());
|
||||
ASSERT_FALSE(donorChangeStreamSpec.getAllowToRunOnSystemNS());
|
||||
ASSERT_FALSE(donorChangeStreamSpec.getShowExpandedEvents());
|
||||
ASSERT_TRUE(donorChangeStreamSpec.getShowCommitTimestamp());
|
||||
}
|
||||
|
||||
TEST_F(ReshardingChangeStreamsMonitorTest, TestChangeStreamMonitorSettingsForDonorTimeseries) {
|
||||
|
|
@ -1216,6 +1217,7 @@ TEST_F(ReshardingChangeStreamsMonitorTest, TestChangeStreamMonitorSettingsForDon
|
|||
ASSERT_TRUE(donorChangeStreamSpec.getShowSystemEvents());
|
||||
ASSERT_TRUE(donorChangeStreamSpec.getAllowToRunOnSystemNS());
|
||||
ASSERT_FALSE(donorChangeStreamSpec.getShowExpandedEvents());
|
||||
ASSERT_TRUE(donorChangeStreamSpec.getShowCommitTimestamp());
|
||||
}
|
||||
|
||||
TEST_F(ReshardingChangeStreamsMonitorTest, TestChangeStreamMonitorSettingsForRecipient) {
|
||||
|
|
@ -1239,6 +1241,7 @@ TEST_F(ReshardingChangeStreamsMonitorTest, TestChangeStreamMonitorSettingsForRec
|
|||
ASSERT_FALSE(recipientChangeStreamSpec.getShowSystemEvents());
|
||||
ASSERT_TRUE(recipientChangeStreamSpec.getAllowToRunOnSystemNS());
|
||||
ASSERT_FALSE(recipientChangeStreamSpec.getShowExpandedEvents());
|
||||
ASSERT_TRUE(recipientChangeStreamSpec.getShowCommitTimestamp());
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
|
|
|||
|
|
@ -606,6 +606,11 @@ Mongo.prototype._extractChangeStreamOptions = function (options) {
|
|||
delete options.showRawUpdateDescription;
|
||||
}
|
||||
|
||||
if (options.hasOwnProperty("showCommitTimestamp")) {
|
||||
changeStreamOptions.showCommitTimestamp = options.showCommitTimestamp;
|
||||
delete options.showCommitTimestamp;
|
||||
}
|
||||
|
||||
// If no maxAwaitTimeMS is set in the options, we set a high wait timeout, so that there won't
|
||||
// be any issues with no data being available on the server side due to limited processing
|
||||
// resources during testing.
|
||||
|
|
|
|||
Loading…
Reference in New Issue