mirror of https://github.com/mongodb/mongo
SERVER-90668 Convert invariants on Status.isOK() to invariants on the Status itself (#24122)
GitOrigin-RevId: 18e023571acf483b000f61497af430af99e5cad0
This commit is contained in:
parent
415174320f
commit
77263e2bf7
|
|
@ -81,6 +81,7 @@ Checks: '-*,
|
|||
mongo-cxx20-std-chrono-check,
|
||||
mongo-fcv-constant-check,
|
||||
mongo-header-bracket-check,
|
||||
mongo-invariant-status-is-ok-check,
|
||||
mongo-macro-definition-leaks-check,
|
||||
mongo-mutex-check,
|
||||
mongo-polyfill-check,
|
||||
|
|
|
|||
|
|
@ -219,7 +219,7 @@ void BM_validate(benchmark::State& state) {
|
|||
auto status = validateBSON(elem.objdata(), elem.objsize());
|
||||
if (!status.isOK())
|
||||
LOGV2(4440100, "Validate failed", "elem"_attr = elem, "status"_attr = status);
|
||||
invariant(status.isOK());
|
||||
invariant(status);
|
||||
|
||||
for (auto _ : state) {
|
||||
benchmark::ClobberMemory();
|
||||
|
|
@ -241,7 +241,7 @@ void BM_validate_contents(benchmark::State& state) {
|
|||
auto status = validateBSON(elem.objdata(), elem.objsize(), BSONValidateModeEnum::kFull);
|
||||
if (!status.isOK())
|
||||
LOGV2(6752100, "Validate failed", "elem"_attr = elem, "status"_attr = status);
|
||||
invariant(status.isOK());
|
||||
invariant(status);
|
||||
|
||||
for (auto _ : state) {
|
||||
benchmark::ClobberMemory();
|
||||
|
|
|
|||
|
|
@ -624,7 +624,7 @@ void DBConnectionPool::appendConnectionStats(executor::ConnectionPoolStats* stat
|
|||
// as our label for connPoolStats. Note that these stats will collide
|
||||
// with any existing stats for the chosen host.
|
||||
auto uri = ConnectionString::parse(i->first.ident);
|
||||
invariant(uri.isOK());
|
||||
invariant(uri.getStatus());
|
||||
HostAndPort host = uri.getValue().getServers().front();
|
||||
|
||||
executor::ConnectionStatsPer hostStats{static_cast<size_t>(i->second.numInUse()),
|
||||
|
|
|
|||
|
|
@ -717,7 +717,7 @@ void AuthorizationSessionImpl::_refreshUserInfoAsNeeded(OperationContext* opCtx)
|
|||
}
|
||||
|
||||
// !ok check above should never fallthrough.
|
||||
invariant(swUser.isOK());
|
||||
invariant(swUser.getStatus());
|
||||
|
||||
if (currentUser.isValid() && !currentUser->isInvalidated()) {
|
||||
// Current user may carry on, no need to update.
|
||||
|
|
|
|||
|
|
@ -209,9 +209,8 @@ void IndexBuildBlock::fail(OperationContext* opCtx, Collection* collection) {
|
|||
|
||||
auto indexCatalogEntry = getWritableEntry(opCtx, collection);
|
||||
if (indexCatalogEntry) {
|
||||
invariant(collection->getIndexCatalog()
|
||||
->dropIndexEntry(opCtx, collection, indexCatalogEntry)
|
||||
.isOK());
|
||||
invariant(
|
||||
collection->getIndexCatalog()->dropIndexEntry(opCtx, collection, indexCatalogEntry));
|
||||
if (_indexBuildInterceptor) {
|
||||
indexCatalogEntry->setIndexBuildInterceptor(nullptr);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1276,7 +1276,7 @@ void IndexCatalogImpl::dropIndexes(OperationContext* opCtx,
|
|||
if (onDropFn) {
|
||||
onDropFn(writableEntry->descriptor());
|
||||
}
|
||||
invariant(dropIndexEntry(opCtx, collection, writableEntry).isOK());
|
||||
invariant(dropIndexEntry(opCtx, collection, writableEntry));
|
||||
}
|
||||
|
||||
// verify state is sane post cleaning
|
||||
|
|
|
|||
|
|
@ -460,7 +460,7 @@ public:
|
|||
->makeFromBSON(collationElem.Obj());
|
||||
// validateIndexSpecCollation() should have checked that the _id index collation
|
||||
// spec is valid.
|
||||
invariant(collatorStatus.isOK());
|
||||
invariant(collatorStatus.getStatus());
|
||||
idIndexCollator = std::move(collatorStatus.getValue());
|
||||
}
|
||||
if (!CollatorInterface::collatorsMatch(defaultCollator.get(),
|
||||
|
|
|
|||
|
|
@ -138,7 +138,7 @@ protected:
|
|||
if (elem = elem[part]; !elem.ok())
|
||||
break;
|
||||
if (elem.ok())
|
||||
invariant(elem.remove().isOK());
|
||||
invariant(elem.remove());
|
||||
}
|
||||
return doc.getObject();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -144,7 +144,7 @@ public:
|
|||
std::unique_ptr<CollatorInterface> createCollator() {
|
||||
auto statusWithCollator = _collatorFactory.makeFromBSON(BSON("locale"
|
||||
<< "en_US"));
|
||||
invariant(statusWithCollator.isOK());
|
||||
invariant(statusWithCollator.getStatus());
|
||||
return std::move(statusWithCollator.getValue());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -458,7 +458,7 @@ repl::OpTime getLatestOplogOpTime(OperationContext* opCtx) {
|
|||
});
|
||||
|
||||
auto optime = repl::OpTime::parseFromOplogEntry(oplogEntryBSON);
|
||||
invariant(optime.isOK(),
|
||||
invariant(optime.getStatus(),
|
||||
str::stream() << "Found an invalid oplog entry: " << oplogEntryBSON
|
||||
<< ", error: " << optime.getStatus());
|
||||
return optime.getValue();
|
||||
|
|
|
|||
|
|
@ -408,7 +408,7 @@ void MirrorMaestroImpl::tryMirror(const std::shared_ptr<CommandInvocation>& invo
|
|||
}
|
||||
});
|
||||
if (!ErrorCodes::isShutdownError(status)) {
|
||||
invariant(status.isOK());
|
||||
invariant(status);
|
||||
requestState->mirror();
|
||||
}
|
||||
clientExecutorHandle.schedule([requestState = std::move(requestState)](
|
||||
|
|
|
|||
|
|
@ -95,7 +95,7 @@ OpMsgFuzzerFixture::OpMsgFuzzerFixture(bool skipGlobalInitializers)
|
|||
: _dir(kTempDirStem.toString()) {
|
||||
if (!skipGlobalInitializers) {
|
||||
auto ret = runGlobalInitializers(std::vector<std::string>{});
|
||||
invariant(ret.isOK());
|
||||
invariant(ret);
|
||||
}
|
||||
|
||||
setGlobalServiceContext(ServiceContext::make());
|
||||
|
|
|
|||
|
|
@ -284,7 +284,7 @@ protected:
|
|||
WriteUnitOfWork wunit(opCtx);
|
||||
AutoGetCollection collRaii(opCtx, nss, MODE_X);
|
||||
if (collRaii) {
|
||||
invariant(collRaii.getWritableCollection(opCtx)->truncate(opCtx).isOK());
|
||||
invariant(collRaii.getWritableCollection(opCtx)->truncate(opCtx));
|
||||
} else {
|
||||
auto db = collRaii.ensureDbExists(opCtx);
|
||||
CollectionOptions opts;
|
||||
|
|
|
|||
|
|
@ -254,7 +254,7 @@ void CanonicalQuery::initCq(boost::intrusive_ptr<ExpressionContext> expCtx,
|
|||
}
|
||||
}
|
||||
// The tree must always be valid after normalization.
|
||||
dassert(parsed_find_command::isValid(_primaryMatchExpression.get(), *_findCommand).isOK());
|
||||
dassert(parsed_find_command::isValid(_primaryMatchExpression.get(), *_findCommand).getStatus());
|
||||
if (auto status = isValidNormalized(_primaryMatchExpression.get()); !status.isOK()) {
|
||||
uasserted(status.code(), status.reason());
|
||||
}
|
||||
|
|
|
|||
|
|
@ -187,7 +187,7 @@ std::unique_ptr<PlanExecutor, PlanExecutor::Deleter> InternalPlanner::sampleColl
|
|||
auto statusWithPlanExecutor = plan_executor_factory::make(
|
||||
expCtx, std::move(ws), std::move(root), collection, yieldPolicy, false);
|
||||
|
||||
invariant(statusWithPlanExecutor.isOK());
|
||||
invariant(statusWithPlanExecutor.getStatus());
|
||||
return std::move(statusWithPlanExecutor.getValue());
|
||||
}
|
||||
|
||||
|
|
@ -229,7 +229,7 @@ std::unique_ptr<PlanExecutor, PlanExecutor::Deleter> InternalPlanner::collection
|
|||
collection,
|
||||
yieldPolicy,
|
||||
false /* whether owned BSON must be returned */);
|
||||
invariant(statusWithPlanExecutor.isOK());
|
||||
invariant(statusWithPlanExecutor.getStatus());
|
||||
return std::move(statusWithPlanExecutor.getValue());
|
||||
}
|
||||
|
||||
|
|
@ -255,7 +255,7 @@ std::unique_ptr<PlanExecutor, PlanExecutor::Deleter> InternalPlanner::collection
|
|||
&collection,
|
||||
yieldPolicy,
|
||||
false /* whether owned BSON must be returned */);
|
||||
invariant(statusWithPlanExecutor.isOK());
|
||||
invariant(statusWithPlanExecutor.getStatus());
|
||||
return std::move(statusWithPlanExecutor.getValue());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -158,7 +158,7 @@ executor::RemoteCommandRequest getRemoteCommandRequest(OperationContext* opCtx,
|
|||
doThrowIfNotRunningWithMongotHostConfigured();
|
||||
auto swHostAndPort = HostAndPort::parse(globalMongotParams.host);
|
||||
// This host and port string is configured and validated at startup.
|
||||
invariant(swHostAndPort.getStatus().isOK());
|
||||
invariant(swHostAndPort.getStatus());
|
||||
executor::RemoteCommandRequest rcr(
|
||||
executor::RemoteCommandRequest(swHostAndPort.getValue(), nss.dbName(), cmdObj, opCtx));
|
||||
rcr.sslMode = transport::ConnectSSLMode::kDisableSSL;
|
||||
|
|
|
|||
|
|
@ -48,7 +48,7 @@ executor::RemoteCommandRequest createManageSearchIndexRemoteCommandRequest(
|
|||
invariant(!globalSearchIndexParams.host.empty());
|
||||
auto swHostAndPort = HostAndPort::parse(globalSearchIndexParams.host);
|
||||
// This host and port string is configured and validated at startup.
|
||||
invariant(swHostAndPort.getStatus().isOK());
|
||||
invariant(swHostAndPort.getStatus());
|
||||
|
||||
// Format the command request.
|
||||
ManageSearchIndexRequest manageSearchIndexRequest;
|
||||
|
|
|
|||
|
|
@ -1027,7 +1027,7 @@ Status applyOplogEntryOrGroupedInserts(OperationContext* opCtx,
|
|||
auto op = entryOrGroupedInserts.getOp();
|
||||
if (op->getOpType() == OpTypeEnum::kNoop) {
|
||||
// No-ops should never fail application, since there's nothing to do.
|
||||
invariant(status.isOK());
|
||||
invariant(status);
|
||||
|
||||
auto opObj = op->getObject();
|
||||
if (opObj.hasField(ReplicationCoordinator::newPrimaryMsgField) &&
|
||||
|
|
|
|||
|
|
@ -252,8 +252,7 @@ void ReplicationConsistencyMarkersImpl::ensureFastCountOnOplogTruncateAfterPoint
|
|||
if (result.getStatus() == ErrorCodes::CollectionIsEmpty) {
|
||||
// The count is updated before successful commit of a write, so unclean shutdown can leave
|
||||
// the value incorrectly set to one.
|
||||
invariant(
|
||||
_storageInterface->setCollectionCount(opCtx, _oplogTruncateAfterPointNss, 0).isOK());
|
||||
invariant(_storageInterface->setCollectionCount(opCtx, _oplogTruncateAfterPointNss, 0));
|
||||
return;
|
||||
}
|
||||
|
||||
|
|
@ -270,7 +269,7 @@ void ReplicationConsistencyMarkersImpl::ensureFastCountOnOplogTruncateAfterPoint
|
|||
|
||||
// We can safely set a count of one. We know that we only ever write one document, and the
|
||||
// success of findSingleton above confirms only one document exists in the collection.
|
||||
invariant(_storageInterface->setCollectionCount(opCtx, _oplogTruncateAfterPointNss, 1).isOK());
|
||||
invariant(_storageInterface->setCollectionCount(opCtx, _oplogTruncateAfterPointNss, 1));
|
||||
}
|
||||
|
||||
Status ReplicationConsistencyMarkersImpl::_upsertOplogTruncateAfterPointDocument(
|
||||
|
|
|
|||
|
|
@ -469,7 +469,7 @@ stdx::unique_lock<Latch> ReplicationCoordinatorImpl::_handleHeartbeatResponseAct
|
|||
}
|
||||
break;
|
||||
case HeartbeatResponseAction::Reconfig:
|
||||
invariant(responseStatus.isOK());
|
||||
invariant(responseStatus.getStatus());
|
||||
_scheduleHeartbeatReconfig(lock, responseStatus.getValue().getConfig());
|
||||
break;
|
||||
case HeartbeatResponseAction::RetryReconfig:
|
||||
|
|
|
|||
|
|
@ -1357,13 +1357,13 @@ Timestamp StorageInterfaceImpl::getLatestOplogTimestamp(OperationContext* opCtx)
|
|||
invariant(Helpers::getLast(opCtx, NamespaceString::kRsOplogNamespace, oplogEntryBSON));
|
||||
|
||||
auto optime = OpTime::parseFromOplogEntry(oplogEntryBSON);
|
||||
invariant(optime.isOK(),
|
||||
invariant(optime.getStatus(),
|
||||
str::stream() << "Found an invalid oplog entry: " << oplogEntryBSON
|
||||
<< ", error: " << optime.getStatus());
|
||||
return optime.getValue().getTimestamp();
|
||||
}
|
||||
|
||||
invariant(statusWithTimestamp.isOK(),
|
||||
invariant(statusWithTimestamp.getStatus(),
|
||||
str::stream() << "Expected oplog entries to exist: "
|
||||
<< statusWithTimestamp.getStatus());
|
||||
|
||||
|
|
|
|||
|
|
@ -88,7 +88,7 @@ struct SyncSourceResolverResponse {
|
|||
}
|
||||
|
||||
HostAndPort getSyncSource() {
|
||||
invariant(syncSourceStatus.isOK());
|
||||
invariant(syncSourceStatus.getStatus());
|
||||
return syncSourceStatus.getValue();
|
||||
}
|
||||
};
|
||||
|
|
|
|||
|
|
@ -304,7 +304,7 @@ bool TenantOplogApplier::_shouldStopApplying(Status status) {
|
|||
return true;
|
||||
}
|
||||
|
||||
dassert(_finalStatus.isOK());
|
||||
dassert(_finalStatus);
|
||||
// Set the _finalStatus. This guarantees that the shutdown() called after releasing
|
||||
// the mutex will signal donor opTime waiters with the 'status' error code and not with
|
||||
// ErrorCodes::CallbackCanceled.
|
||||
|
|
|
|||
|
|
@ -2846,7 +2846,7 @@ bool TopologyCoordinator::isSafeToStepDown() {
|
|||
}
|
||||
|
||||
auto tagStatus = _rsConfig.findCustomWriteMode(ReplSetConfig::kMajorityWriteConcernModeName);
|
||||
invariant(tagStatus.isOK());
|
||||
invariant(tagStatus.getStatus());
|
||||
|
||||
// Check if a majority of nodes have reached the last applied optime.
|
||||
if (!haveTaggedNodesReachedOpTime(lastApplied, tagStatus.getValue(), false)) {
|
||||
|
|
|
|||
|
|
@ -317,8 +317,7 @@ CollectionType ConfigServerTestFixture::setupCollection(
|
|||
// create the db
|
||||
auto swShardDoc = findOneOnConfigCollection(
|
||||
operationContext(), NamespaceString::kConfigsvrShardsNamespace, BSONObj());
|
||||
invariant(swShardDoc.isOK(),
|
||||
"At least one shard should be setup when initializing a collection");
|
||||
invariant(swShardDoc, "At least one shard should be setup when initializing a collection");
|
||||
auto shard = uassertStatusOK(ShardType::fromBSON(swShardDoc.getValue()));
|
||||
setupDatabase(nss.dbName(), ShardId(shard.getName()));
|
||||
}
|
||||
|
|
|
|||
|
|
@ -133,7 +133,7 @@ void MigrationBatchInserter::run(Status status) const try {
|
|||
// already shutdown. If we were to schedule a task on a shutdown ThreadPool,
|
||||
// then there is a logic error in our code. Therefore, we assert that here.
|
||||
|
||||
invariant(status.isOK());
|
||||
invariant(status);
|
||||
auto arr = _batch["objects"].Obj();
|
||||
if (arr.isEmpty())
|
||||
return;
|
||||
|
|
|
|||
|
|
@ -46,7 +46,7 @@ public:
|
|||
// That happens if we schedule this task on a ThreadPool that is
|
||||
// already shutdown. We should never do that. Therefore,
|
||||
// we assert that here.
|
||||
invariant(status.isOK());
|
||||
invariant(status);
|
||||
}
|
||||
MigrationBatchMockInserter(OperationContext*,
|
||||
OperationContext*,
|
||||
|
|
|
|||
|
|
@ -88,7 +88,7 @@ protected:
|
|||
WriteUnitOfWork wunit(opCtx);
|
||||
AutoGetCollection collRaii(opCtx, nss, MODE_X);
|
||||
if (collRaii) {
|
||||
invariant(collRaii.getWritableCollection(opCtx)->truncate(opCtx).isOK());
|
||||
invariant(collRaii.getWritableCollection(opCtx)->truncate(opCtx));
|
||||
} else {
|
||||
auto db = collRaii.ensureDbExists(opCtx);
|
||||
CollectionOptions opts;
|
||||
|
|
|
|||
|
|
@ -64,7 +64,7 @@ public:
|
|||
ss << extraStrings;
|
||||
std::string config = ss.str();
|
||||
int ret = wiredtiger_open(dbpath.toString().c_str(), nullptr, config.c_str(), &_conn);
|
||||
invariant(wtRCToStatus(ret, nullptr).isOK());
|
||||
invariant(wtRCToStatus(ret, nullptr));
|
||||
}
|
||||
~WiredTigerConnection() {
|
||||
_conn->close(_conn, nullptr);
|
||||
|
|
@ -89,8 +89,8 @@ public:
|
|||
auto ru = WiredTigerRecoveryUnit::get(_opCtxHolder.get());
|
||||
_session = ru->getSession();
|
||||
auto wt_session = _session->getSession();
|
||||
invariant(wtRCToStatus(wt_session->create(wt_session, "table:mytable", nullptr), wt_session)
|
||||
.isOK());
|
||||
invariant(
|
||||
wtRCToStatus(wt_session->create(wt_session, "table:mytable", nullptr), wt_session));
|
||||
ru->abandonSnapshot();
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -2653,7 +2653,8 @@ StatusWith<Timestamp> WiredTigerKVEngine::pinOldestTimestamp(
|
|||
stdx::lock_guard<Latch> lock(_oldestTimestampPinRequestsMutex);
|
||||
// When a write is updating the value from an earlier pin to a later one, use
|
||||
// rounding to make a best effort to repin the earlier value.
|
||||
invariant(_pinOldestTimestamp(lock, svcName, previousTimestamp, true).isOK());
|
||||
invariant(
|
||||
_pinOldestTimestamp(lock, svcName, previousTimestamp, true).getStatus());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
|||
|
|
@ -47,7 +47,7 @@ BucketCompressionFailure::BucketCompressionFailure(const UUID& collectionUUID,
|
|||
|
||||
std::shared_ptr<const ErrorExtraInfo> BucketCompressionFailure::parse(const BSONObj& obj) {
|
||||
auto uuidSW = UUID::parse(obj[kUUIDFieldName]);
|
||||
invariant(uuidSW.isOK());
|
||||
invariant(uuidSW.getStatus());
|
||||
auto collectionUUID = uuidSW.getValue();
|
||||
return std::make_shared<BucketCompressionFailure>(
|
||||
collectionUUID,
|
||||
|
|
|
|||
|
|
@ -141,8 +141,8 @@ Status RenameNode::init(BSONElement modExpr,
|
|||
// Parsing {$rename: {'from': 'to'}} places nodes in the UpdateNode tree for both the "from" and
|
||||
// "to" paths via UpdateObjectNode::parseAndMerge(), which will enforce this isUpdatable
|
||||
// property.
|
||||
dassert(fieldchecker::isUpdatable(fromFieldRef).isOK());
|
||||
dassert(fieldchecker::isUpdatable(toFieldRef).isOK());
|
||||
dassert(fieldchecker::isUpdatable(fromFieldRef));
|
||||
dassert(fieldchecker::isUpdatable(toFieldRef));
|
||||
|
||||
// Though we could treat this as a no-op, it is illegal in the current implementation.
|
||||
if (fromFieldRef == toFieldRef) {
|
||||
|
|
|
|||
|
|
@ -177,7 +177,7 @@ public:
|
|||
&coll,
|
||||
PlanYieldPolicy::YieldPolicy::YIELD_AUTO,
|
||||
QueryPlannerParams::DEFAULT);
|
||||
invariant(statusWithPlanExecutor.isOK());
|
||||
invariant(statusWithPlanExecutor.getStatus());
|
||||
return std::move(statusWithPlanExecutor.getValue());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -431,7 +431,7 @@ auto NetworkInterfaceTL::CommandState::make(NetworkInterfaceTL* interface,
|
|||
.tapAll([state](const auto& swRequest) {
|
||||
// swRequest is either populated from the success path or the value returning
|
||||
// onError above. swRequest.isOK() should not be possible.
|
||||
invariant(swRequest.isOK());
|
||||
invariant(swRequest.getStatus());
|
||||
|
||||
// At this point, the command has either been sent and returned an RCRsp or
|
||||
// has received a local interruption that was wrapped in a RCRsp.
|
||||
|
|
@ -637,7 +637,7 @@ Status NetworkInterfaceTL::startCommand(const TaskExecutor::CallbackHandle& cbHa
|
|||
.thenRunOn(makeGuaranteedExecutor(baton, _reactor))
|
||||
.getAsync([cmdState = cmdState,
|
||||
onFinish = std::move(onFinish)](StatusWith<RemoteCommandOnAnyResponse> swr) {
|
||||
invariant(swr.isOK(),
|
||||
invariant(swr.getStatus(),
|
||||
"Remote command response failed with an error: {}"_format(
|
||||
swr.getStatus().toString()));
|
||||
auto rs = std::move(swr.getValue());
|
||||
|
|
@ -1288,7 +1288,7 @@ Status NetworkInterfaceTL::_killOperation(CommandStateBase* cmdStateToKill, size
|
|||
|
||||
std::move(future).getAsync(
|
||||
[this, operationKey, killOpRequest](StatusWith<RemoteCommandOnAnyResponse> swr) {
|
||||
invariant(swr.isOK());
|
||||
invariant(swr.getStatus());
|
||||
auto rs = std::move(swr.getValue());
|
||||
LOGV2_DEBUG(51813,
|
||||
2,
|
||||
|
|
|
|||
|
|
@ -140,7 +140,7 @@ LogDomainGlobal::Impl::Impl(LogDomainGlobal& parent) : _parent(parent) {
|
|||
_consoleSink->set_filter(ComponentSettingsFilter(_parent, _settings));
|
||||
|
||||
// Set default configuration
|
||||
invariant(configure({}).isOK());
|
||||
invariant(configure({}));
|
||||
|
||||
// Make a call to source() to make sure the internal thread_local is created as early as
|
||||
// possible and thus destroyed as late as possible.
|
||||
|
|
|
|||
|
|
@ -91,7 +91,7 @@ private:
|
|||
void setupAppender() {
|
||||
logv2::LogDomainGlobal::ConfigurationOptions config;
|
||||
config.makeDisabled();
|
||||
invariant(logv2::LogManager::global().getGlobalDomainInternal().configure(config).isOK());
|
||||
invariant(logv2::LogManager::global().getGlobalDomainInternal().configure(config));
|
||||
|
||||
auto backend = boost::make_shared<boost::log::sinks::text_ostream_backend>();
|
||||
backend->add_stream(makeNullStream());
|
||||
|
|
@ -108,7 +108,7 @@ private:
|
|||
|
||||
void tearDownAppender() {
|
||||
boost::log::core::get()->remove_sink(_sink);
|
||||
invariant(logv2::LogManager::global().getGlobalDomainInternal().configure({}).isOK());
|
||||
invariant(logv2::LogManager::global().getGlobalDomainInternal().configure({}));
|
||||
}
|
||||
|
||||
boost::shared_ptr<boost::log::sinks::synchronous_sink<boost::log::sinks::text_ostream_backend>>
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ Status ClusterIdentityLoader::loadClusterId(OperationContext* opCtx,
|
|||
const repl::ReadConcernLevel& readConcernLevel) {
|
||||
stdx::unique_lock<Latch> lk(_mutex);
|
||||
if (_initializationState == InitializationState::kInitialized) {
|
||||
invariant(_lastLoadResult.isOK());
|
||||
invariant(_lastLoadResult.getStatus());
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -78,7 +78,7 @@ void aggregateResults(const DBStatsCommand& cmd,
|
|||
double indexFreeStorageSize = 0;
|
||||
|
||||
for (const auto& response : responses) {
|
||||
invariant(response.swResponse.getStatus().isOK());
|
||||
invariant(response.swResponse.getStatus());
|
||||
const BSONObj& b = response.swResponse.getValue().data;
|
||||
auto resp = DBStats::parse(IDLParserContext{"dbstats"}, b);
|
||||
|
||||
|
|
|
|||
|
|
@ -429,7 +429,7 @@ ClusterQueryResult AsyncResultsMerger::_nextReadySorted(WithLock) {
|
|||
_mergeQueue.pop();
|
||||
|
||||
invariant(!_remotes[smallestRemote].docBuffer.empty());
|
||||
invariant(_remotes[smallestRemote].status.isOK());
|
||||
invariant(_remotes[smallestRemote].status);
|
||||
|
||||
ClusterQueryResult front = _remotes[smallestRemote].docBuffer.front();
|
||||
_remotes[smallestRemote].docBuffer.pop();
|
||||
|
|
@ -455,7 +455,7 @@ ClusterQueryResult AsyncResultsMerger::_nextReadyUnsorted(WithLock) {
|
|||
size_t remotesAttempted = 0;
|
||||
while (remotesAttempted < _remotes.size()) {
|
||||
// It is illegal to call this method if there is an error received from any shard.
|
||||
invariant(_remotes[_gettingFromRemote].status.isOK());
|
||||
invariant(_remotes[_gettingFromRemote].status);
|
||||
|
||||
if (_remotes[_gettingFromRemote].hasNext()) {
|
||||
ClusterQueryResult front = _remotes[_gettingFromRemote].docBuffer.front();
|
||||
|
|
|
|||
|
|
@ -668,7 +668,7 @@ Status setUpOperationContextStateForGetMore(OperationContext* opCtx,
|
|||
awaitDataState(opCtx).waitForInsertsDeadline =
|
||||
opCtx->getServiceContext()->getPreciseClockSource()->now() + timeout;
|
||||
awaitDataState(opCtx).shouldWaitForInserts = true;
|
||||
invariant(cursor->setAwaitDataTimeout(timeout).isOK());
|
||||
invariant(cursor->setAwaitDataTimeout(timeout));
|
||||
} else if (cmd.getMaxTimeMS()) {
|
||||
return {ErrorCodes::BadValue,
|
||||
"maxTimeMS can only be used with getMore for tailable, awaitData cursors"};
|
||||
|
|
|
|||
|
|
@ -1259,7 +1259,7 @@ void BulkWriteOp::processChildBatchResponseFromRemote(
|
|||
const TargetedWriteBatch& writeBatch,
|
||||
const AsyncRequestsSender::Response& response,
|
||||
boost::optional<stdx::unordered_map<NamespaceString, TrackedErrors>&> errorsPerNamespace) {
|
||||
invariant(response.swResponse.getStatus().isOK(), "Response status was unexpectedly not OK");
|
||||
invariant(response.swResponse.getStatus(), "Response status was unexpectedly not OK");
|
||||
|
||||
auto childBatchResponse = response.swResponse.getValue();
|
||||
LOGV2_DEBUG(7279200,
|
||||
|
|
|
|||
|
|
@ -0,0 +1,68 @@
|
|||
/**
|
||||
* Copyright (C) 2024-present MongoDB, Inc.
|
||||
*
|
||||
* This program is free software: you can redistribute it and/or modify
|
||||
* it under the terms of the Server Side Public License, version 1,
|
||||
* as published by MongoDB, Inc.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* Server Side Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the Server Side Public License
|
||||
* along with this program. If not, see
|
||||
* <http://www.mongodb.com/licensing/server-side-public-license>.
|
||||
*
|
||||
* As a special exception, the copyright holders give permission to link the
|
||||
* code of portions of this program with the OpenSSL library under certain
|
||||
* conditions as described in each individual source file and distribute
|
||||
* linked combinations including the program with the OpenSSL library. You
|
||||
* must comply with the Server Side Public License in all respects for
|
||||
* all of the code used other than as permitted herein. If you modify file(s)
|
||||
* with this exception, you may extend this exception to your version of the
|
||||
* file(s), but you are not obligated to do so. If you do not wish to do so,
|
||||
* delete this exception statement from your version. If you delete this
|
||||
* exception statement from all source files in the program, then also delete
|
||||
* it in the license file.
|
||||
*/
|
||||
|
||||
#include "MongoInvariantStatusIsOKCheck.h"
|
||||
#include "MongoTidyUtils.h"
|
||||
|
||||
#include <iostream>
|
||||
|
||||
namespace mongo::tidy {
|
||||
|
||||
using namespace clang;
|
||||
using namespace clang::ast_matchers;
|
||||
|
||||
MongoInvariantStatusIsOKCheck::MongoInvariantStatusIsOKCheck(StringRef Name,
|
||||
clang::tidy::ClangTidyContext* Context)
|
||||
: ClangTidyCheck(Name, Context) {}
|
||||
|
||||
void MongoInvariantStatusIsOKCheck::registerMatchers(ast_matchers::MatchFinder* Finder) {
|
||||
|
||||
Finder->addMatcher(
|
||||
callExpr(callee(functionDecl(anyOf(hasName("invariantWithContextAndLocation"),
|
||||
hasName("invariantWithLocation")))),
|
||||
hasArgument(0,
|
||||
cxxMemberCallExpr(on(hasType(cxxRecordDecl(anyOf(
|
||||
hasName("Status"), hasName("StatusWith"))))),
|
||||
callee(cxxMethodDecl(hasName("isOK"))))))
|
||||
.bind("invariant_call"),
|
||||
this);
|
||||
}
|
||||
|
||||
void MongoInvariantStatusIsOKCheck::check(const ast_matchers::MatchFinder::MatchResult& Result) {
|
||||
|
||||
const auto* invariantCall = Result.Nodes.getNodeAs<CallExpr>("invariant_call");
|
||||
|
||||
if (invariantCall) {
|
||||
diag(invariantCall->getBeginLoc(),
|
||||
"Found invariant(status.isOK()) or dassert(status.isOK()), use invariant(status) for "
|
||||
"better diagnostics");
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace mongo::tidy
|
||||
|
|
@ -0,0 +1,49 @@
|
|||
/**
|
||||
* Copyright (C) 2024-present MongoDB, Inc.
|
||||
*
|
||||
* This program is free software: you can redistribute it and/or modify
|
||||
* it under the terms of the Server Side Public License, version 1,
|
||||
* as published by MongoDB, Inc.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* Server Side Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the Server Side Public License
|
||||
* along with this program. If not, see
|
||||
* <http://www.mongodb.com/licensing/server-side-public-license>.
|
||||
*
|
||||
* As a special exception, the copyright holders give permission to link the
|
||||
* code of portions of this program with the OpenSSL library under certain
|
||||
* conditions as described in each individual source file and distribute
|
||||
* linked combinations including the program with the OpenSSL library. You
|
||||
* must comply with the Server Side Public License in all respects for
|
||||
* all of the code used other than as permitted herein. If you modify file(s)
|
||||
* with this exception, you may extend this exception to your version of the
|
||||
* file(s), but you are not obligated to do so. If you do not wish to do so,
|
||||
* delete this exception statement from your version. If you delete this
|
||||
* exception statement from all source files in the program, then also delete
|
||||
* it in the license file.
|
||||
*/
|
||||
#pragma once
|
||||
|
||||
#include <clang-tidy/ClangTidy.h>
|
||||
#include <clang-tidy/ClangTidyCheck.h>
|
||||
namespace mongo::tidy {
|
||||
|
||||
/**
|
||||
* Check for any instances of invariant(status.isOK()).
|
||||
* Overrides the default registerMatchers function to add matcher to match the
|
||||
* usage of invariant(status.isOK()). Overrides the default check function to
|
||||
* flag the uses of invariant(status.isOK()) to enforce usage of invariant(status) instead.
|
||||
*/
|
||||
class MongoInvariantStatusIsOKCheck : public clang::tidy::ClangTidyCheck {
|
||||
|
||||
public:
|
||||
MongoInvariantStatusIsOKCheck(clang::StringRef Name, clang::tidy::ClangTidyContext* Context);
|
||||
void registerMatchers(clang::ast_matchers::MatchFinder* Finder) override;
|
||||
void check(const clang::ast_matchers::MatchFinder::MatchResult& Result) override;
|
||||
};
|
||||
|
||||
} // namespace mongo::tidy
|
||||
|
|
@ -35,6 +35,7 @@
|
|||
#include "MongoCxx20StdChronoCheck.h"
|
||||
#include "MongoFCVConstantCheck.h"
|
||||
#include "MongoHeaderBracketCheck.h"
|
||||
#include "MongoInvariantStatusIsOKCheck.h"
|
||||
#include "MongoMacroDefinitionLeaksCheck.h"
|
||||
#include "MongoMutexCheck.h"
|
||||
#include "MongoNoUniqueAddressCheck.h"
|
||||
|
|
@ -86,6 +87,8 @@ public:
|
|||
CheckFactories.registerCheck<MongoStringDataConstRefCheck>(
|
||||
"mongo-stringdata-const-ref-check");
|
||||
CheckFactories.registerCheck<MongoRWMutexCheck>("mongo-rwmutex-check");
|
||||
CheckFactories.registerCheck<MongoInvariantStatusIsOKCheck>(
|
||||
"mongo-invariant-status-is-ok-check");
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
|||
|
|
@ -145,6 +145,7 @@ mongo_custom_check = env.SharedLibrary(
|
|||
"MongoRandCheck.cpp",
|
||||
"MongoRWMutexCheck.cpp",
|
||||
"MongoStringDataConstRefCheck.cpp",
|
||||
"MongoInvariantStatusIsOKCheck.cpp",
|
||||
],
|
||||
LIBDEPS_NO_INHERIT=[
|
||||
"$BUILD_DIR/third_party/shim_allocator",
|
||||
|
|
|
|||
|
|
@ -466,6 +466,20 @@ class MongoTidyTests(unittest.TestCase):
|
|||
|
||||
self.run_clang_tidy()
|
||||
|
||||
def test_MongoInvariantStatusIsOKCheck(self):
|
||||
self.write_config(
|
||||
textwrap.dedent("""\
|
||||
Checks: '-*,mongo-invariant-status-is-ok-check'
|
||||
WarningsAsErrors: '*'
|
||||
""")
|
||||
)
|
||||
|
||||
self.expected_output = [
|
||||
"Found invariant(status.isOK()) or dassert(status.isOK()), use invariant(status) for better diagnostics",
|
||||
]
|
||||
|
||||
self.run_clang_tidy()
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
parser = argparse.ArgumentParser()
|
||||
|
|
|
|||
|
|
@ -54,6 +54,7 @@ if env.GetOption("ninja") == "disabled":
|
|||
"test_MongoStringDataConstRefCheck1.cpp",
|
||||
"test_MongoStringDataConstRefCheck2.cpp",
|
||||
"test_MongoStringDataConstRefCheck3.cpp",
|
||||
"test_MongoInvariantStatusIsOKCheck.cpp",
|
||||
]
|
||||
|
||||
# So that we can do fast runs, we will generate a separate compilation database file for each
|
||||
|
|
|
|||
|
|
@ -0,0 +1,16 @@
|
|||
template <typename T>
|
||||
inline void invariantWithLocation(const T& testOK) {}
|
||||
|
||||
#define invariant(...) invariantWithLocation(__VA_ARGS__)
|
||||
|
||||
namespace mongo {
|
||||
class Status {
|
||||
public:
|
||||
bool isOK() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
void fun(Status status) {
|
||||
invariant(status.isOK());
|
||||
}
|
||||
} // namespace mongo
|
||||
|
|
@ -195,8 +195,8 @@ StatusWith<std::string> icuX509DNPrep(StringData str) try {
|
|||
*/
|
||||
MONGO_INITIALIZER_GENERAL(LoadIcuPrep, ("LoadICUData"), ("default"))(InitializerContext*) {
|
||||
// Force ICU to load its caches by calling each function.
|
||||
invariant(icuSaslPrep("a"_sd).isOK());
|
||||
invariant(icuX509DNPrep("a"_sd).isOK());
|
||||
invariant(icuSaslPrep("a"_sd).getStatus());
|
||||
invariant(icuX509DNPrep("a"_sd).getStatus());
|
||||
}
|
||||
|
||||
} // namespace mongo
|
||||
|
|
|
|||
|
|
@ -768,7 +768,7 @@ TEST(SSLManager, TransientSSLParamsStressTestWithTransport) {
|
|||
Timer timer;
|
||||
while (timer.elapsed() < Seconds(2)) {
|
||||
auto swContext = tla.createTransientSSLContext(transientSSLParams);
|
||||
invariant(swContext.getStatus().isOK());
|
||||
invariant(swContext.getStatus());
|
||||
std::shared_ptr<const transport::SSLConnectionContext> ctxToDelete;
|
||||
{
|
||||
auto lk = stdx::lock_guard(mutex);
|
||||
|
|
@ -815,11 +815,10 @@ TEST(SSLManager, TransientSSLParamsStressTestWithManager) {
|
|||
SSLManagerInterface::create(params, transientParams, true /* isSSLServer */);
|
||||
|
||||
auto egress = std::make_unique<asio::ssl::context>(asio::ssl::context::sslv23);
|
||||
invariant(manager
|
||||
->initSSLContext(egress->native_handle(),
|
||||
invariant(
|
||||
manager->initSSLContext(egress->native_handle(),
|
||||
params,
|
||||
SSLManagerInterface::ConnectionDirection::kOutgoing)
|
||||
.isOK());
|
||||
SSLManagerInterface::ConnectionDirection::kOutgoing));
|
||||
std::shared_ptr<SSLManagerInterface> managerToDelete;
|
||||
{
|
||||
auto lk = stdx::lock_guard(mutex);
|
||||
|
|
|
|||
Loading…
Reference in New Issue