SERVER-90668 Convert invariants on Status.isOK() to invariants on the Status itself (#24122)

GitOrigin-RevId: 18e023571acf483b000f61497af430af99e5cad0
This commit is contained in:
Alex Li 2024-07-10 20:46:55 -04:00 committed by MongoDB Bot
parent 415174320f
commit 77263e2bf7
50 changed files with 214 additions and 64 deletions

View File

@ -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,

View File

@ -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();

View File

@ -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()),

View File

@ -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.

View File

@ -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);
}

View File

@ -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

View File

@ -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(),

View File

@ -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();
}

View File

@ -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());
}

View File

@ -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();

View File

@ -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)](

View File

@ -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());

View File

@ -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;

View File

@ -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());
}

View File

@ -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());
}

View File

@ -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;

View File

@ -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;

View File

@ -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) &&

View File

@ -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(

View File

@ -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:

View File

@ -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());

View File

@ -88,7 +88,7 @@ struct SyncSourceResolverResponse {
}
HostAndPort getSyncSource() {
invariant(syncSourceStatus.isOK());
invariant(syncSourceStatus.getStatus());
return syncSourceStatus.getValue();
}
};

View File

@ -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.

View File

@ -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)) {

View File

@ -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()));
}

View File

@ -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;

View File

@ -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*,

View File

@ -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;

View File

@ -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();
}

View File

@ -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());
}
});
}

View File

@ -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,

View File

@ -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) {

View File

@ -177,7 +177,7 @@ public:
&coll,
PlanYieldPolicy::YieldPolicy::YIELD_AUTO,
QueryPlannerParams::DEFAULT);
invariant(statusWithPlanExecutor.isOK());
invariant(statusWithPlanExecutor.getStatus());
return std::move(statusWithPlanExecutor.getValue());
}

View File

@ -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,

View File

@ -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.

View File

@ -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>>

View File

@ -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();
}

View File

@ -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);

View File

@ -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();

View File

@ -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"};

View File

@ -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,

View File

@ -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

View File

@ -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

View File

@ -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");
}
};

View File

@ -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",

View File

@ -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()

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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);