From 1de37afd5253f0b598821a20003a3300e1162b89 Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Tue, 19 Jul 2022 14:15:51 -0600 Subject: [PATCH] Make TEST macros C++ only (#7558) * proof of concept * use code-probe instead of test * code probe working on gcc * code probe implemented * renamed TestProbe to CodeProbe * fixed refactoring typo * support filtered output * print probes at end of simulation * fix missed probes print * fix deduplication * Fix refactoring issues * revert bad refactor * make sure file paths are relative * fix more wrong refactor changes --- cmake/FlowCommands.cmake | 13 + contrib/sqlite/sqlite3.amalgamation.c | 2 +- fdbcli/KillCommand.actor.cpp | 1 + fdbcli/fdbcli.actor.cpp | 1 + fdbclient/DatabaseBackupAgent.actor.cpp | 6 +- fdbclient/FileBackupAgent.actor.cpp | 8 +- fdbclient/ManagementAPI.actor.cpp | 4 +- fdbclient/MonitorLeader.actor.cpp | 5 +- fdbclient/NativeAPI.actor.cpp | 116 +++---- fdbclient/PaxosConfigTransaction.actor.cpp | 2 +- fdbclient/ReadYourWrites.actor.cpp | 28 +- fdbclient/SpecialKeySpace.actor.cpp | 8 +- fdbclient/TaskBucket.actor.cpp | 18 +- fdbclient/WriteMap.cpp | 2 +- fdbclient/include/fdbclient/Atomic.h | 2 +- fdbrpc/AsyncFileCached.actor.cpp | 4 +- fdbrpc/FlowTests.actor.cpp | 2 +- fdbrpc/FlowTransport.actor.cpp | 10 +- fdbrpc/include/fdbrpc/AsyncFileEIO.actor.h | 2 +- .../fdbrpc/AsyncFileNonDurable.actor.h | 12 +- fdbrpc/include/fdbrpc/LoadBalance.actor.h | 12 +- fdbrpc/include/fdbrpc/genericactors.actor.h | 4 +- fdbrpc/sim2.actor.cpp | 128 ++++++-- fdbserver/ApplyMetadataMutation.cpp | 10 +- fdbserver/BackupWorker.actor.cpp | 2 +- fdbserver/BlobManager.actor.cpp | 64 ++-- fdbserver/BlobWorker.actor.cpp | 109 ++++--- fdbserver/ClusterController.actor.cpp | 19 +- fdbserver/ClusterRecovery.actor.cpp | 20 +- fdbserver/CommitProxyServer.actor.cpp | 17 +- fdbserver/ConfigNode.actor.cpp | 5 +- fdbserver/CoordinatedState.actor.cpp | 4 +- fdbserver/DDTeamCollection.actor.cpp | 47 +-- fdbserver/DDTxnProcessor.actor.cpp | 4 +- fdbserver/DataDistribution.actor.cpp | 8 +- fdbserver/DataDistributionQueue.actor.cpp | 6 +- fdbserver/DataDistributionTracker.actor.cpp | 14 +- fdbserver/DiskQueue.actor.cpp | 33 +- fdbserver/GlobalTagThrottler.actor.cpp | 2 +- fdbserver/GrvProxyServer.actor.cpp | 6 +- fdbserver/KeyValueStoreMemory.actor.cpp | 10 +- fdbserver/KeyValueStoreSQLite.actor.cpp | 30 +- fdbserver/LeaderElection.actor.cpp | 2 +- fdbserver/LocalConfiguration.actor.cpp | 10 +- fdbserver/LogRouter.actor.cpp | 2 +- fdbserver/LogSystem.cpp | 8 +- fdbserver/LogSystemDiskQueueAdapter.actor.cpp | 2 +- fdbserver/LogSystemPeekCursor.actor.cpp | 6 +- fdbserver/MoveKeys.actor.cpp | 29 +- fdbserver/MutationTracking.cpp | 2 +- fdbserver/OldTLogServer_4_6.actor.cpp | 28 +- fdbserver/OldTLogServer_6_0.actor.cpp | 30 +- fdbserver/OldTLogServer_6_2.actor.cpp | 32 +- fdbserver/PaxosConfigConsumer.actor.cpp | 2 +- fdbserver/Ratekeeper.actor.cpp | 2 +- fdbserver/ResolutionBalancer.actor.cpp | 2 +- fdbserver/Resolver.actor.cpp | 19 +- fdbserver/RkTagThrottleCollection.cpp | 32 +- fdbserver/SimpleConfigConsumer.actor.cpp | 2 +- fdbserver/SimulatedCluster.actor.cpp | 83 ++--- fdbserver/StorageCache.actor.cpp | 39 +-- fdbserver/TLogServer.actor.cpp | 32 +- fdbserver/TagPartitionedLogSystem.actor.cpp | 6 +- fdbserver/TagThrottler.actor.cpp | 12 +- fdbserver/TransactionTagCounter.cpp | 2 +- fdbserver/VersionedBTree.actor.cpp | 8 +- fdbserver/WaitFailure.actor.cpp | 2 +- fdbserver/fdbserver.actor.cpp | 7 +- .../include/fdbserver/ClusterRecovery.actor.h | 4 +- fdbserver/include/fdbserver/LogSystem.h | 2 +- fdbserver/include/fdbserver/StorageMetrics.h | 16 +- .../include/fdbserver/WorkerInterface.actor.h | 2 +- fdbserver/masterserver.actor.cpp | 26 +- fdbserver/storageserver.actor.cpp | 157 ++++----- fdbserver/worker.actor.cpp | 16 +- fdbserver/workloads/AtomicOps.actor.cpp | 22 +- .../AtomicOpsApiCorrectness.actor.cpp | 18 +- ...kupAndParallelRestoreCorrectness.actor.cpp | 5 +- .../workloads/BackupCorrectness.actor.cpp | 5 +- .../workloads/BackupToDBCorrectness.actor.cpp | 5 +- .../BlobGranuleCorrectnessWorkload.actor.cpp | 8 +- fdbserver/workloads/CommitBugCheck.actor.cpp | 4 +- .../workloads/ConfigureDatabase.actor.cpp | 24 +- .../workloads/ConsistencyCheck.actor.cpp | 6 +- fdbserver/workloads/Cycle.actor.cpp | 2 +- .../DataDistributionMetrics.actor.cpp | 2 +- .../DifferentClustersSameRV.actor.cpp | 2 +- .../workloads/EncryptKeyProxyTest.actor.cpp | 2 +- fdbserver/workloads/Increment.actor.cpp | 2 +- fdbserver/workloads/LockDatabase.actor.cpp | 2 +- .../workloads/MachineAttrition.actor.cpp | 6 +- .../SpecialKeySpaceCorrectness.actor.cpp | 14 +- fdbserver/workloads/Throttling.actor.cpp | 6 +- fdbserver/workloads/VersionStamp.actor.cpp | 4 +- fdbserver/workloads/WriteDuringRead.actor.cpp | 4 +- flow/BlobCipher.cpp | 6 +- flow/CodeProbe.cpp | 293 +++++++++++++++++ flow/StreamCipher.cpp | 6 +- flow/TDMetric.cpp | 2 +- flow/config.h.cmake | 2 + flow/flow.cpp | 2 +- flow/include/flow/Arena.h | 11 + flow/include/flow/CodeProbe.h | 305 ++++++++++++++++++ flow/include/flow/CodeProbeUtils.h | 32 ++ flow/include/flow/flow.h | 13 +- flow/include/flow/genericactors.actor.h | 3 +- 106 files changed, 1493 insertions(+), 749 deletions(-) create mode 100644 flow/CodeProbe.cpp create mode 100644 flow/include/flow/CodeProbe.h create mode 100644 flow/include/flow/CodeProbeUtils.h diff --git a/cmake/FlowCommands.cmake b/cmake/FlowCommands.cmake index 4d7fa4f2c2..81ea36892d 100644 --- a/cmake/FlowCommands.cmake +++ b/cmake/FlowCommands.cmake @@ -9,6 +9,14 @@ define_property(TARGET PROPERTY COVERAGE_FILTERS expression in this list will be ignored when the coverage.target.xml file is \ generated. This property is set through the add_flow_target function.") +if(WIN32) + set(compilation_unit_macro_default OFF) +else() + set(compilation_unit_macro_default ON) +endif() + +set(PASS_COMPILATION_UNIT "${compilation_unit_macro_default}" CACHE BOOL + "Pass path to compilation unit as macro to each compilation unit (useful for code probes)") function(generate_coverage_xml) if(NOT (${ARGC} EQUAL "1")) @@ -259,6 +267,11 @@ function(add_flow_target) endif() endif() endforeach() + if(PASS_COMPILATION_UNIT) + foreach(s IN LISTS sources) + set_source_files_properties("${s}" PROPERTIES COMPILE_DEFINITIONS "COMPILATION_UNIT=${s}") + endforeach() + endif() if(AFT_EXECUTABLE) set(strip_target ON) set(target_type exec) diff --git a/contrib/sqlite/sqlite3.amalgamation.c b/contrib/sqlite/sqlite3.amalgamation.c index 2b0058be91..2992e521e2 100644 --- a/contrib/sqlite/sqlite3.amalgamation.c +++ b/contrib/sqlite/sqlite3.amalgamation.c @@ -87009,7 +87009,7 @@ SQLITE_PRIVATE WhereInfo *sqlite3WhereBegin( } sqlite3_query_plan[nQPlan] = 0; nQPlan = 0; -#endif /* SQLITE_TEST // Testing and debugging use only */ +#endif /* SQLITE_TEST // Testing and debugging use only */"); /* Record the continuation address in the WhereInfo structure. Then ** clean up and return. diff --git a/fdbcli/KillCommand.actor.cpp b/fdbcli/KillCommand.actor.cpp index 391bdb3064..d025b10388 100644 --- a/fdbcli/KillCommand.actor.cpp +++ b/fdbcli/KillCommand.actor.cpp @@ -29,6 +29,7 @@ #include "flow/Arena.h" #include "flow/FastRef.h" #include "flow/ThreadHelper.actor.h" +#include "flow/CodeProbe.h" #include "flow/actorcompiler.h" // This must be the last #include. namespace fdb_cli { diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index ba01fa3c22..b762c48e0a 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -49,6 +49,7 @@ #include "flow/FastRef.h" #include "flow/Platform.h" #include "flow/SystemMonitor.h" +#include "flow/CodeProbe.h" #include "flow/TLSConfig.actor.h" #include "flow/ThreadHelper.actor.h" diff --git a/fdbclient/DatabaseBackupAgent.actor.cpp b/fdbclient/DatabaseBackupAgent.actor.cpp index f549205e25..80d4a16cc9 100644 --- a/fdbclient/DatabaseBackupAgent.actor.cpp +++ b/fdbclient/DatabaseBackupAgent.actor.cpp @@ -363,7 +363,7 @@ struct BackupRangeTaskFunc : TaskFuncBase { if ((!prevAdjacent || !nextAdjacent) && rangeCount > ((prevAdjacent || nextAdjacent) ? CLIENT_KNOBS->BACKUP_MAP_KEY_UPPER_LIMIT : CLIENT_KNOBS->BACKUP_MAP_KEY_LOWER_LIMIT)) { - TEST(true); // range insert delayed because too versionMap is too large + CODE_PROBE(true, "range insert delayed because too versionMap is too large"); if (rangeCount > CLIENT_KNOBS->BACKUP_MAP_KEY_UPPER_LIMIT) TraceEvent(SevWarnAlways, "DBA_KeyRangeMapTooLarge").log(); @@ -2780,7 +2780,7 @@ public: Version destVersion = wait(tr3.getReadVersion()); TraceEvent("DBA_SwitchoverVersionUpgrade").detail("Src", commitVersion).detail("Dest", destVersion); if (destVersion <= commitVersion) { - TEST(true); // Forcing dest backup cluster to higher version + CODE_PROBE(true, "Forcing dest backup cluster to higher version"); tr3.set(minRequiredCommitVersionKey, BinaryWriter::toValue(commitVersion + 1, Unversioned())); wait(tr3.commit()); } else { @@ -2933,7 +2933,7 @@ public: Version applied = BinaryReader::fromStringRef(lastApplied.get(), Unversioned()); TraceEvent("DBA_AbortVersionUpgrade").detail("Src", applied).detail("Dest", current); if (current <= applied) { - TEST(true); // Upgrading version of local database. + CODE_PROBE(true, "Upgrading version of local database."); // The +1 is because we want to make sure that a versionstamped operation can't reuse // the same version as an already-applied transaction. tr->set(minRequiredCommitVersionKey, BinaryWriter::toValue(applied + 1, Unversioned())); diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 333cff9a81..56fdf74688 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -822,7 +822,7 @@ struct AbortFiveZeroBackupTask : TaskFuncBase { state FileBackupAgent backupAgent; state std::string tagName = task->params[BackupAgentBase::keyConfigBackupTag].toString(); - TEST(true); // Canceling old backup task + CODE_PROBE(true, "Canceling old backup task"); TraceEvent(SevInfo, "FileBackupCancelOldTask") .detail("Task", task->params[Task::reservedTaskParamKeyType]) @@ -908,7 +908,7 @@ struct AbortFiveOneBackupTask : TaskFuncBase { state BackupConfig config(task); state std::string tagName = wait(config.tag().getOrThrow(tr)); - TEST(true); // Canceling 5.1 backup task + CODE_PROBE(true, "Canceling 5.1 backup task"); TraceEvent(SevInfo, "FileBackupCancelFiveOneTask") .detail("Task", task->params[Task::reservedTaskParamKeyType]) @@ -1245,7 +1245,7 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase { // If we've seen a new read version OR hit the end of the stream, then if we were writing a file finish it. if (values.second != outVersion || done) { if (outFile) { - TEST(outVersion != invalidVersion); // Backup range task wrote multiple versions + CODE_PROBE(outVersion != invalidVersion, "Backup range task wrote multiple versions"); state Key nextKey = done ? endKey : keyAfter(lastKey); wait(rangeFile.writeKey(nextKey)); @@ -4098,7 +4098,7 @@ struct StartFullRestoreTaskFunc : RestoreTaskFuncBase { .detail("RestoreVersion", restoreVersion) .detail("Dest", destVersion); if (destVersion <= restoreVersion) { - TEST(true); // Forcing restored cluster to higher version + CODE_PROBE(true, "Forcing restored cluster to higher version"); tr->set(minRequiredCommitVersionKey, BinaryWriter::toValue(restoreVersion + 1, Unversioned())); wait(tr->commit()); } else { diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 3ce529fdd8..067730ba5e 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -1003,8 +1003,8 @@ ACTOR Future changeQuorum(Database cx, Reference>> leaderServers; state ClientCoordinators coord(Reference( diff --git a/fdbclient/MonitorLeader.actor.cpp b/fdbclient/MonitorLeader.actor.cpp index f0215ea36d..f72913f7ee 100644 --- a/fdbclient/MonitorLeader.actor.cpp +++ b/fdbclient/MonitorLeader.actor.cpp @@ -987,8 +987,9 @@ ACTOR Future monitorProxiesOneGeneration( successIndex = index; allConnectionsFailed = false; } else { - TEST(rep.getError().code() == error_code_failed_to_progress); // Coordinator cant talk to cluster controller - TEST(rep.getError().code() == error_code_lookup_failed); // Coordinator hostname resolving failure + CODE_PROBE(rep.getError().code() == error_code_failed_to_progress, + "Coordinator cant talk to cluster controller"); + CODE_PROBE(rep.getError().code() == error_code_lookup_failed, "Coordinator hostname resolving failure"); TraceEvent("MonitorProxiesConnectFailed") .detail("Error", rep.getError().name()) .detail("Coordinator", clientLeaderServer.getAddressString()); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 89b0452489..a1dbaad2ed 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -170,7 +170,7 @@ void DatabaseContext::addTssMapping(StorageServerInterface const& ssi, StorageSe if (result->second.id() == tssi.id()) { metrics = tssMetrics[tssi.id()]; } else { - TEST(true); // SS now maps to new TSS! This will probably never happen in practice + CODE_PROBE(true, "SS now maps to new TSS! This will probably never happen in practice"); tssMetrics.erase(result->second.id()); metrics = makeReference(); tssMetrics[tssi.id()] = metrics; @@ -444,7 +444,7 @@ void DatabaseContext::validateVersion(Version version) const { throw client_invalid_operation(); } if (switchable && version < minAcceptableReadVersion) { - TEST(true); // Attempted to read a version lower than any this client has seen from the current cluster + CODE_PROBE(true, "Attempted to read a version lower than any this client has seen from the current cluster"); throw transaction_too_old(); } @@ -1114,8 +1114,8 @@ ACTOR static Future handleTssMismatches(DatabaseContext* cx) { state bool quarantine = CLIENT_KNOBS->QUARANTINE_TSS_ON_MISMATCH; TraceEvent(SevWarnAlways, quarantine ? "TSS_QuarantineMismatch" : "TSS_KillMismatch") .detail("TSSID", data.first.toString()); - TEST(quarantine); // Quarantining TSS because it got mismatch - TEST(!quarantine); // Killing TSS because it got mismatch + CODE_PROBE(quarantine, "Quarantining TSS because it got mismatch"); + CODE_PROBE(!quarantine, "Killing TSS because it got mismatch"); tr = makeReference(Database(Reference::addRef(cx))); state int tries = 0; @@ -1154,7 +1154,7 @@ ACTOR static Future handleTssMismatches(DatabaseContext* cx) { // clear out txn so that the extra DatabaseContext ref gets decref'd and we can free cx tr = makeReference(); } else { - TEST(true); // Not handling TSS with mismatch because it's already gone + CODE_PROBE(true, "Not handling TSS with mismatch because it's already gone"); } } } @@ -1860,7 +1860,7 @@ bool DatabaseContext::getCachedLocations(const Optional& tenantName, loop { auto r = reverse ? end : begin; if (!r->value()) { - TEST(result.size()); // had some but not all cached locations + CODE_PROBE(result.size(), "had some but not all cached locations"); result.clear(); return false; } @@ -1907,7 +1907,7 @@ Reference DatabaseContext::setCachedLocation(const Optional(serverRefs); while (locationCache.size() > locationCacheSize && attempts < maxEvictionAttempts) { - TEST(true); // NativeAPI storage server locationCache entry evicted + CODE_PROBE(true, "NativeAPI storage server locationCache entry evicted"); attempts++; auto r = locationCache.randomRange(); Key begin = r.begin(), end = r.end(); // insert invalidates r, so can't be passed a mere reference into it @@ -2091,7 +2091,7 @@ Future DatabaseContext::onConnected() { ACTOR static Future switchConnectionRecordImpl(Reference connRecord, DatabaseContext* self) { - TEST(true); // Switch connection file + CODE_PROBE(true, "Switch connection file"); TraceEvent("SwitchConnectionRecord") .detail("ClusterFile", connRecord->toString()) .detail("ConnectionString", connRecord->getConnectionString().toString()); @@ -2152,7 +2152,7 @@ void DatabaseContext::expireThrottles() { for (auto& priorityItr : throttledTags) { for (auto tagItr = priorityItr.second.begin(); tagItr != priorityItr.second.end();) { if (tagItr->second.expired()) { - TEST(true); // Expiring client throttle + CODE_PROBE(true, "Expiring client throttle"); tagItr = priorityItr.second.erase(tagItr); } else { ++tagItr; @@ -2638,7 +2638,7 @@ bool DatabaseContext::isCurrentGrvProxy(UID proxyId) const { if (proxy.id() == proxyId) return true; } - TEST(true); // stale GRV proxy detected + CODE_PROBE(true, "stale GRV proxy detected"); return false; } @@ -3630,13 +3630,13 @@ ACTOR Future watchValue(Database cx, Reference p wait(delay(CLIENT_KNOBS->UNKNOWN_TENANT_RETRY_DELAY, parameters->taskID)); } else if (e.code() == error_code_watch_cancelled || e.code() == error_code_process_behind) { // clang-format off - TEST(e.code() == error_code_watch_cancelled); // Too many watches on the storage server, poll for changes instead - TEST(e.code() == error_code_process_behind); // The storage servers are all behind + CODE_PROBE(e.code() == error_code_watch_cancelled, "Too many watches on the storage server, poll for changes instead"); + CODE_PROBE(e.code() == error_code_process_behind, "The storage servers are all behind"); // clang-format on wait(delay(CLIENT_KNOBS->WATCH_POLLING_TIME, parameters->taskID)); } else if (e.code() == error_code_timed_out) { // The storage server occasionally times out watches in case // it was cancelled - TEST(true); // A watch timed out + CODE_PROBE(true, "A watch timed out"); wait(delay(CLIENT_KNOBS->FUTURE_VERSION_RETRY_DELAY, parameters->taskID)); } else { state Error err = e; @@ -3668,7 +3668,8 @@ ACTOR Future watchStorageServerResp(int64_t tenantId, Key key, Database cx } // ABA happens else { - TEST(true); // ABA issue where the version returned from the server is less than the version in the map + CODE_PROBE(true, + "ABA issue where the version returned from the server is less than the version in the map"); // case 2: version_1 < version_2 and future_count == 1 if (metadata->watchPromise.getFutureReferenceCount() == 1) { @@ -3759,7 +3760,8 @@ Future getWatchFuture(Database cx, Reference parameters) // case 3: val_1 != val_2 && version_2 > version_1 (received watch with different value and a higher version so // recreate in SS) else if (parameters->version > metadata->parameters->version) { - TEST(true); // Setting a watch that has a different value than the one in the map but a higher version (newer) + CODE_PROBE(true, + "Setting a watch that has a different value than the one in the map but a higher version (newer)"); cx->deleteWatchMetadata(parameters->tenant.tenantId, parameters->key); metadata->watchPromise.send(parameters->version); @@ -3774,10 +3776,10 @@ Future getWatchFuture(Database cx, Reference parameters) } // case 5: val_1 != val_2 && version_1 == version_2 (received watch with different value but same version) else if (metadata->parameters->version == parameters->version) { - TEST(true); // Setting a watch which has a different value than the one in the map but the same version + CODE_PROBE(true, "Setting a watch which has a different value than the one in the map but the same version"); return sameVersionDiffValue(cx, parameters); } - TEST(true); // Setting a watch which has a different value than the one in the map but a lower version (older) + CODE_PROBE(true, "Setting a watch which has a different value than the one in the map but a lower version (older)"); // case 4: val_1 != val_2 && version_2 < version_1 return Void(); @@ -3971,7 +3973,7 @@ Future getExactRange(Reference trState, .detail("BlockBytes", rep.data.expectedSize()); ASSERT(false); } - TEST(true); // GetKeyValuesFamilyReply.more in getExactRange + CODE_PROBE(true, "GetKeyValuesFamilyReply.more in getExactRange"); // Make next request to the same shard with a beginning key just after the last key returned if (reverse) locations[shard].range = @@ -3982,7 +3984,7 @@ Future getExactRange(Reference trState, } if (!more || locations[shard].range.empty()) { - TEST(true); // getExactrange (!more || locations[shard].first.empty()) + CODE_PROBE(true, "getExactrange (!more || locations[shard].first.empty())"); if (shard == locations.size() - 1) { const KeyRangeRef& range = locations[shard].range; KeyRef begin = reverse ? keys.begin : range.end; @@ -3992,7 +3994,7 @@ Future getExactRange(Reference trState, output.more = false; return output; } - TEST(true); // Multiple requests of key locations + CODE_PROBE(true, "Multiple requests of key locations"); keys = KeyRangeRef(begin, end); break; @@ -4432,7 +4434,7 @@ Future getRange(Reference trState, if (!rep.more) { ASSERT(modifiedSelectors); - TEST(true); // !GetKeyValuesFamilyReply.more and modifiedSelectors in getRange + CODE_PROBE(true, "!GetKeyValuesFamilyReply.more and modifiedSelectors in getRange"); if (!rep.data.size()) { RangeResultFamily result = wait( @@ -4456,7 +4458,7 @@ Future getRange(Reference trState, else begin = firstGreaterOrEqual(shard.end); } else { - TEST(true); // GetKeyValuesFamilyReply.more in getRange + CODE_PROBE(true, "GetKeyValuesFamilyReply.more in getRange"); if (reverse) end = firstGreaterOrEqual(output[output.size() - 1].key); else @@ -4575,7 +4577,7 @@ static Future tssStreamComparison(Request request, } else { tssData.metrics->ssError(e.code()); } - TEST(e.code() != error_code_end_of_stream); // SS got error in TSS stream comparison + CODE_PROBE(e.code() != error_code_end_of_stream, "SS got error in TSS stream comparison"); } state double sleepTime = std::max(startTime + FLOW_KNOBS->LOAD_BALANCE_TSS_TIMEOUT - now(), 0.0); @@ -4587,7 +4589,7 @@ static Future tssStreamComparison(Request request, } when(wait(delay(sleepTime))) { ++tssData.metrics->tssTimeouts; - TEST(true); // Got TSS timeout in stream comparison + CODE_PROBE(true, "Got TSS timeout in stream comparison"); } } } catch (Error& e) { @@ -4602,7 +4604,7 @@ static Future tssStreamComparison(Request request, } else { tssData.metrics->tssError(e.code()); } - TEST(e.code() != error_code_end_of_stream); // TSS got error in TSS stream comparison + CODE_PROBE(e.code() != error_code_end_of_stream, "TSS got error in TSS stream comparison"); } if (!ssEndOfStream || !tssEndOfStream) { @@ -4615,11 +4617,11 @@ static Future tssStreamComparison(Request request, // FIXME: this code is pretty much identical to LoadBalance.h // TODO could add team check logic in if we added synchronous way to turn this into a fixed getRange request // and send it to the whole team and compare? I think it's fine to skip that for streaming though - TEST(ssEndOfStream != tssEndOfStream); // SS or TSS stream finished early! + CODE_PROBE(ssEndOfStream != tssEndOfStream, "SS or TSS stream finished early!"); // skip tss comparison if both are end of stream if ((!ssEndOfStream || !tssEndOfStream) && !TSS_doCompare(ssReply.get(), tssReply.get())) { - TEST(true); // TSS mismatch in stream comparison + CODE_PROBE(true, "TSS mismatch in stream comparison"); TraceEvent mismatchEvent( (g_network->isSimulated() && g_simulator.tssMode == ISimulator::TSSMode::EnabledDropMutations) ? SevWarnAlways @@ -4631,10 +4633,10 @@ static Future tssStreamComparison(Request request, if (tssData.metrics->shouldRecordDetailedMismatch()) { TSS_traceMismatch(mismatchEvent, request, ssReply.get(), tssReply.get()); - TEST(FLOW_KNOBS - ->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL); // Tracing Full TSS Mismatch in stream comparison - TEST(!FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL); // Tracing Partial TSS Mismatch in stream - // comparison and storing the rest in FDB + CODE_PROBE(FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL, + "Tracing Full TSS Mismatch in stream comparison"); + CODE_PROBE(!FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL, + "Tracing Partial TSS Mismatch in stream comparison and storing the rest in FDB"); if (!FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL) { mismatchEvent.disable(); @@ -4674,7 +4676,7 @@ maybeDuplicateTSSStreamFragment(Request& req, QueueModel* model, RequestStream tssData = model->getTssData(ssStream->getEndpoint().token.first()); if (tssData.present()) { - TEST(true); // duplicating stream to TSS + CODE_PROBE(true, "duplicating stream to TSS"); resetReply(req); // FIXME: optimize to avoid creating new netNotifiedQueueWithAcknowledgements for each stream duplication RequestStream tssRequestStream(tssData.get().endpoint); @@ -4874,7 +4876,7 @@ ACTOR Future getRangeStreamFragment(Reference trState, .detail("BlockBytes", rep.data.expectedSize()); ASSERT(false); } - TEST(true); // GetKeyValuesStreamReply.more in getRangeStream + CODE_PROBE(true, "GetKeyValuesStreamReply.more in getRangeStream"); // Make next request to the same shard with a beginning key just after the last key returned if (reverse) locations[shard].range = @@ -5272,7 +5274,7 @@ ACTOR Future watch(Reference watch, when(wait(watch->watchFuture)) { break; } when(wait(cx->connectionFileChanged())) { - TEST(true); // Recreated a watch after switch + CODE_PROBE(true, "Recreated a watch after switch"); cx->clearWatchMetadata(); watch->watchFuture = watchValueMap(cx->minAcceptableReadVersion, tenantInfo, @@ -5445,18 +5447,18 @@ Future Transaction::getRangeInternal(const KeySelector& begin KeySelector b = begin; if (b.orEqual) { - TEST(true); // Native begin orEqual==true + CODE_PROBE(true, "Native begin orEqual==true"); b.removeOrEqual(b.arena()); } KeySelector e = end; if (e.orEqual) { - TEST(true); // Native end orEqual==true + CODE_PROBE(true, "Native end orEqual==true"); e.removeOrEqual(e.arena()); } if (b.offset >= e.offset && b.getKey() >= e.getKey()) { - TEST(true); // Native range inverted + CODE_PROBE(true, "Native range inverted"); return RangeResultFamily(); } @@ -5519,18 +5521,18 @@ Future Transaction::getRangeStream(const PromiseStream& resul KeySelector b = begin; if (b.orEqual) { - TEST(true); // Native stream begin orEqual==true + CODE_PROBE(true, "Native stream begin orEqual==true"); b.removeOrEqual(b.arena()); } KeySelector e = end; if (e.orEqual) { - TEST(true); // Native stream end orEqual==true + CODE_PROBE(true, "Native stream end orEqual==true"); e.removeOrEqual(e.arena()); } if (b.offset >= e.offset && b.getKey() >= e.getKey()) { - TEST(true); // Native stream range inverted + CODE_PROBE(true, "Native stream range inverted"); results.sendError(end_of_stream()); return Void(); } @@ -5633,7 +5635,7 @@ void Transaction::atomicOp(const KeyRef& key, if (addConflictRange && operationType != MutationRef::SetVersionstampedKey) t.write_conflict_ranges.push_back(req.arena, r); - TEST(true); // NativeAPI atomic operation + CODE_PROBE(true, "NativeAPI atomic operation"); } void Transaction::clear(const KeyRangeRef& range, AddConflictRange addConflictRange) { @@ -5719,7 +5721,7 @@ double Transaction::getBackoff(int errCode) { if (priorityItr != trState->cx->throttledTags.end()) { auto tagItr = priorityItr->second.find(tag); if (tagItr != priorityItr->second.end()) { - TEST(true); // Returning throttle backoff + CODE_PROBE(true, "Returning throttle backoff"); returnedBackoff = std::max( returnedBackoff, std::min(CLIENT_KNOBS->TAG_THROTTLE_RECHECK_INTERVAL, tagItr->second.throttleDuration())); @@ -6250,7 +6252,7 @@ ACTOR static Future tryCommit(Reference trState, KeyRangeRef selfConflictingRange = intersects(req.transaction.write_conflict_ranges, req.transaction.read_conflict_ranges).get(); - TEST(true); // Waiting for dummy transaction to report commit_unknown_result + CODE_PROBE(true, "Waiting for dummy transaction to report commit_unknown_result"); wait(commitDummyTransaction(trState, singleKeyRange(selfConflictingRange.begin))); } @@ -6588,7 +6590,7 @@ void Transaction::setOption(FDBTransactionOptions::Option option, Optional(value.get(), IncludeVersion())); break; @@ -6668,10 +6670,10 @@ ACTOR Future getConsistentReadVersion(SpanContext parentSpa for (auto& tag : tags) { auto itr = v.tagThrottleInfo.find(tag.first); if (itr == v.tagThrottleInfo.end()) { - TEST(true); // Removing client throttle + CODE_PROBE(true, "Removing client throttle"); priorityThrottledTags.erase(tag.first); } else { - TEST(true); // Setting client throttle + CODE_PROBE(true, "Setting client throttle"); auto result = priorityThrottledTags.try_emplace(tag.first, itr->second); if (!result.second) { result.first->second.update(itr->second); @@ -6854,7 +6856,7 @@ ACTOR Future extractReadVersion(Reference trState, if (itr->second.expired()) { priorityThrottledTags.erase(itr); } else if (itr->second.throttleDuration() > 0) { - TEST(true); // throttling transaction after getting read version + CODE_PROBE(true, "throttling transaction after getting read version"); ++trState->cx->transactionReadVersionsThrottled; throw tag_throttled(); } @@ -6960,12 +6962,12 @@ Future Transaction::getReadVersion(uint32_t flags) { } if (maxThrottleDelay > 0.0 && !canRecheck) { // TODO: allow delaying? - TEST(true); // Throttling tag before GRV request + CODE_PROBE(true, "Throttling tag before GRV request"); ++trState->cx->transactionReadVersionsThrottled; readVersion = tag_throttled(); return readVersion; } else { - TEST(maxThrottleDelay > 0.0); // Rechecking throttle + CODE_PROBE(maxThrottleDelay > 0.0, "Rechecking throttle"); } for (auto& tag : trState->options.tags) { @@ -7344,10 +7346,10 @@ ACTOR Future>> getReadHotRanges(Da wait(waitForAll(fReplies)); if (nLocs == 1) { - TEST(true); // Single-shard read hot range request + CODE_PROBE(true, "Single-shard read hot range request"); return fReplies[0].get().readHotRanges; } else { - TEST(true); // Multi-shard read hot range request + CODE_PROBE(true, "Multi-shard read hot range request"); Standalone> results; for (int i = 0; i < nLocs; i++) { results.append(results.arena(), @@ -7856,7 +7858,7 @@ ACTOR Future>> readBlobGranulesActor( if (!results.empty() && results.back().keyRange.end != chunk.keyRange.begin) { ASSERT(results.back().keyRange.end > chunk.keyRange.begin); ASSERT(results.back().keyRange.end <= chunk.keyRange.end); - TEST(true); // Merge while reading granule range + CODE_PROBE(true, "Merge while reading granule range"); while (!results.empty() && results.back().keyRange.begin >= chunk.keyRange.begin) { // TODO: we can't easily un-depend the arenas for these guys, but that's ok as this // should be rare @@ -8981,8 +8983,8 @@ ACTOR Future mergeChangeFeedStream(Reference db, state std::vector> onErrors(interfs.size()); state std::vector streams(interfs.size()); - TEST(interfs.size() > 10); // Large change feed merge cursor - TEST(interfs.size() > 100); // Very large change feed merge cursor + CODE_PROBE(interfs.size() > 10, "Large change feed merge cursor"); + CODE_PROBE(interfs.size() > 100, "Very large change feed merge cursor"); state UID mergeCursorUID = UID(); state std::vector debugUIDs; @@ -9306,13 +9308,13 @@ ACTOR Future getChangeFeedStreamActor(Reference db, interfs.emplace_back(locations[i].locations->getInterface(chosenLocations[i]), locations[i].range & range); } - TEST(true); // Change feed merge cursor + CODE_PROBE(true, "Change feed merge cursor"); // TODO (jslocum): validate connectionFileChanged behavior wait( mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || cx->connectionFileChanged()); } else { - TEST(true); // Change feed single cursor + CODE_PROBE(true, "Change feed single cursor"); StorageServerInterface interf = locations[0].locations->getInterface(chosenLocations[0]); wait(singleChangeFeedStream( db, interf, range, results, rangeID, &begin, end, replyBufferSize, canReadPopped) || @@ -9328,7 +9330,7 @@ ACTOR Future getChangeFeedStreamActor(Reference db, results->streams.clear(); results->storageData.clear(); if (e.code() == error_code_change_feed_popped) { - TEST(true); // getChangeFeedStreamActor got popped + CODE_PROBE(true, "getChangeFeedStreamActor got popped"); results->mutations.sendError(e); results->refresh.sendError(e); } else { diff --git a/fdbclient/PaxosConfigTransaction.actor.cpp b/fdbclient/PaxosConfigTransaction.actor.cpp index 4b7c19c05a..3e69df7227 100644 --- a/fdbclient/PaxosConfigTransaction.actor.cpp +++ b/fdbclient/PaxosConfigTransaction.actor.cpp @@ -199,7 +199,7 @@ class GetGenerationQuorum { } } catch (Error& e) { if (e.code() == error_code_failed_to_reach_quorum) { - TEST(true); // Failed to reach quorum getting generation + CODE_PROBE(true, "Failed to reach quorum getting generation"); wait(delayJittered( std::clamp(0.005 * (1 << retries), 0.0, CLIENT_KNOBS->TIMEOUT_RETRY_UPPER_BOUND))); ++retries; diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index a64a65e58a..0635358402 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1213,7 +1213,7 @@ public: // isolation support. But it is not default and is rarely used. So we disallow it until we have thorough test // coverage for it.) if (snapshot) { - TEST(true); // getMappedRange not supported for snapshot. + CODE_PROBE(true, "getMappedRange not supported for snapshot."); throw unsupported_operation(); } // For now, getMappedRange requires read-your-writes being NOT disabled. But the support of RYW is limited @@ -1222,7 +1222,7 @@ public: // which returns the written value transparently. In another word, it makes sure not break RYW semantics without // actually implementing reading from the writes. if (ryw->options.readYourWritesDisabled) { - TEST(true); // getMappedRange not supported for read-your-writes disabled. + CODE_PROBE(true, "getMappedRange not supported for read-your-writes disabled."); throw unsupported_operation(); } @@ -1242,7 +1242,7 @@ public: ++it; ASSERT(itCopy->value.size()); - TEST(itCopy->value.size() > 1); // Multiple watches on the same key triggered by RYOW + CODE_PROBE(itCopy->value.size() > 1, "Multiple watches on the same key triggered by RYOW"); for (int i = 0; i < itCopy->value.size(); i++) { if (itCopy->value[i]->onChangeTrigger.isSet()) { @@ -1535,11 +1535,11 @@ ACTOR Future getWorkerInterfaces(Reference> ReadYourWritesTransaction::get(const Key& key, Snapshot snapshot) { - TEST(true); // ReadYourWritesTransaction::get + CODE_PROBE(true, "ReadYourWritesTransaction::get"); if (getDatabase()->apiVersionAtLeast(630)) { if (specialKeys.contains(key)) { - TEST(true); // Special keys get + CODE_PROBE(true, "Special keys get"); return getDatabase()->specialKeySpace->get(this, key); } } else { @@ -1622,7 +1622,7 @@ Future ReadYourWritesTransaction::getRange(KeySelector begin, if (getDatabase()->apiVersionAtLeast(630)) { if (specialKeys.contains(begin.getKey()) && specialKeys.begin <= end.getKey() && end.getKey() <= specialKeys.end) { - TEST(true); // Special key space get range + CODE_PROBE(true, "Special key space get range"); return getDatabase()->specialKeySpace->getRange(this, begin, end, limits, reverse); } } else { @@ -1648,7 +1648,7 @@ Future ReadYourWritesTransaction::getRange(KeySelector begin, // This optimization prevents nullptr operations from being added to the conflict range if (limits.isReached()) { - TEST(true); // RYW range read limit 0 + CODE_PROBE(true, "RYW range read limit 0"); return RangeResult(); } @@ -1662,7 +1662,7 @@ Future ReadYourWritesTransaction::getRange(KeySelector begin, end.removeOrEqual(end.arena()); if (begin.offset >= end.offset && begin.getKey() >= end.getKey()) { - TEST(true); // RYW range inverted + CODE_PROBE(true, "RYW range inverted"); return RangeResult(); } @@ -1692,7 +1692,7 @@ Future ReadYourWritesTransaction::getMappedRange(KeySelector if (getDatabase()->apiVersionAtLeast(630)) { if (specialKeys.contains(begin.getKey()) && specialKeys.begin <= end.getKey() && end.getKey() <= specialKeys.end) { - TEST(true); // Special key space get range (getMappedRange) + CODE_PROBE(true, "Special key space get range (getMappedRange)"); throw client_invalid_operation(); // Not support special keys. } } else { @@ -1714,7 +1714,7 @@ Future ReadYourWritesTransaction::getMappedRange(KeySelector // This optimization prevents nullptr operations from being added to the conflict range if (limits.isReached()) { - TEST(true); // RYW range read limit 0 (getMappedRange) + CODE_PROBE(true, "RYW range read limit 0 (getMappedRange)"); return MappedRangeResult(); } @@ -1728,7 +1728,7 @@ Future ReadYourWritesTransaction::getMappedRange(KeySelector end.removeOrEqual(end.arena()); if (begin.offset >= end.offset && begin.getKey() >= end.getKey()) { - TEST(true); // RYW range inverted (getMappedRange) + CODE_PROBE(true, "RYW range inverted (getMappedRange)"); return MappedRangeResult(); } @@ -1998,7 +1998,7 @@ void ReadYourWritesTransaction::setToken(uint64_t token) { } RangeResult ReadYourWritesTransaction::getReadConflictRangeIntersecting(KeyRangeRef kr) { - TEST(true); // Special keys read conflict range + CODE_PROBE(true, "Special keys read conflict range"); ASSERT(readConflictRangeKeysRange.contains(kr)); ASSERT(!tr.trState->options.checkWritesEnabled); RangeResult result; @@ -2040,7 +2040,7 @@ RangeResult ReadYourWritesTransaction::getReadConflictRangeIntersecting(KeyRange } RangeResult ReadYourWritesTransaction::getWriteConflictRangeIntersecting(KeyRangeRef kr) { - TEST(true); // Special keys write conflict range + CODE_PROBE(true, "Special keys write conflict range"); ASSERT(writeConflictRangeKeysRange.contains(kr)); RangeResult result; @@ -2145,7 +2145,7 @@ void ReadYourWritesTransaction::atomicOp(const KeyRef& key, const ValueRef& oper } if (operationType == MutationRef::SetVersionstampedKey) { - TEST(options.readYourWritesDisabled); // SetVersionstampedKey without ryw enabled + CODE_PROBE(options.readYourWritesDisabled, "SetVersionstampedKey without ryw enabled"); // this does validation of the key and needs to be performed before the readYourWritesDisabled path KeyRangeRef range = getVersionstampKeyRange(arena, k, tr.getCachedReadVersion().orDefault(0), getMaxReadKey()); versionStampKeys.push_back(arena, k); diff --git a/fdbclient/SpecialKeySpace.actor.cpp b/fdbclient/SpecialKeySpace.actor.cpp index 85f237cb6d..c692699642 100644 --- a/fdbclient/SpecialKeySpace.actor.cpp +++ b/fdbclient/SpecialKeySpace.actor.cpp @@ -364,12 +364,12 @@ ACTOR Future SpecialKeySpace::getRangeAggregationActor(SpecialKeySp // Handle all corner cases like what RYW does // return if range inverted if (actualBeginOffset >= actualEndOffset && begin.getKey() >= end.getKey()) { - TEST(true); // inverted range + CODE_PROBE(true, "inverted range"); return RangeResultRef(false, false); } // If touches begin or end, return with readToBegin and readThroughEnd flags if (begin.getKey() == moduleBoundary.end || end.getKey() == moduleBoundary.begin) { - TEST(true); // query touches begin or end + CODE_PROBE(true, "query touches begin or end"); return result; } state RangeMap::Ranges ranges = @@ -453,7 +453,7 @@ Future SpecialKeySpace::getRange(ReadYourWritesTransaction* ryw, if (!limits.isValid()) return range_limits_invalid(); if (limits.isReached()) { - TEST(true); // read limit 0 + CODE_PROBE(true, "read limit 0"); return RangeResult(); } // make sure orEqual == false @@ -461,7 +461,7 @@ Future SpecialKeySpace::getRange(ReadYourWritesTransaction* ryw, end.removeOrEqual(end.arena()); if (begin.offset >= end.offset && begin.getKey() >= end.getKey()) { - TEST(true); // range inverted + CODE_PROBE(true, "range inverted"); return RangeResult(); } diff --git a/fdbclient/TaskBucket.actor.cpp b/fdbclient/TaskBucket.actor.cpp index dc3d75259a..6ef6dcce66 100644 --- a/fdbclient/TaskBucket.actor.cpp +++ b/fdbclient/TaskBucket.actor.cpp @@ -199,7 +199,7 @@ public: // many other new tasks get added so that the timed out tasks never get chances to re-run if (deterministicRandom()->random01() < CLIENT_KNOBS->TASKBUCKET_CHECK_TIMEOUT_CHANCE) { bool anyTimeouts = wait(requeueTimedOutTasks(tr, taskBucket)); - TEST(anyTimeouts); // Found a task that timed out + CODE_PROBE(anyTimeouts, "Found a task that timed out"); } state std::vector>> taskKeyFutures(CLIENT_KNOBS->TASKBUCKET_MAX_PRIORITY + 1); @@ -233,7 +233,7 @@ public: bool anyTimeouts = wait(requeueTimedOutTasks(tr, taskBucket)); // If there were timeouts, try to get a task since there should now be one in one of the available spaces. if (anyTimeouts) { - TEST(true); // Try to get one task from timeouts subspace + CODE_PROBE(true, "Try to get one task from timeouts subspace"); Reference task = wait(getOne(tr, taskBucket)); return task; } @@ -707,7 +707,7 @@ public: wait(delay(CLIENT_KNOBS->TASKBUCKET_CHECK_ACTIVE_DELAY)); bool isActiveKey = wait(getActiveKey(tr, taskBucket, startingValue)); if (isActiveKey) { - TEST(true); // checkActive return true + CODE_PROBE(true, "checkActive return true"); return true; } break; @@ -717,7 +717,7 @@ public: } } - TEST(true); // checkActive return false + CODE_PROBE(true, "checkActive return false"); return false; } @@ -742,7 +742,7 @@ public: // Returns True if any tasks were affected. ACTOR static Future requeueTimedOutTasks(Reference tr, Reference taskBucket) { - TEST(true); // Looks for tasks that have timed out and returns them to be available tasks. + CODE_PROBE(true, "Looks for tasks that have timed out and returns them to be available tasks."); Version end = wait(tr->getReadVersion()); state KeyRange range( KeyRangeRef(taskBucket->timeouts.get(0).range().begin, taskBucket->timeouts.get(end).range().end)); @@ -849,12 +849,12 @@ public: // If we're updating the task params the clear the old space and write params to the new space if (updateParams) { - TEST(true); // Extended a task while updating parameters + CODE_PROBE(true, "Extended a task while updating parameters"); for (auto& p : task->params) { tr->set(newTimeoutSpace.pack(p.key), p.value); } } else { - TEST(true); // Extended a task without updating parameters + CODE_PROBE(true, "Extended a task without updating parameters"); // Otherwise, read and transplant the params from the old to new timeout spaces RangeResult params = wait(tr->getRange(oldTimeoutSpace.range(), CLIENT_KNOBS->TOO_MANY)); for (auto& kv : params) { @@ -1138,10 +1138,10 @@ public: bool is_set = wait(isSet(tr, taskFuture)); if (is_set) { - TEST(true); // is_set == true + CODE_PROBE(true, "is_set == true"); wait(performAction(tr, taskBucket, taskFuture, task)); } else { - TEST(true); // is_set == false + CODE_PROBE(true, "is_set == false"); Subspace callbackSpace = taskFuture->callbacks.get(StringRef(deterministicRandom()->randomUniqueID().toString())); for (auto& v : task->params) { diff --git a/fdbclient/WriteMap.cpp b/fdbclient/WriteMap.cpp index dcb6ee9581..56aedd8e8c 100644 --- a/fdbclient/WriteMap.cpp +++ b/fdbclient/WriteMap.cpp @@ -567,7 +567,7 @@ void WriteMap::clearNoConflict(KeyRangeRef keys) { bool end_conflict = it.is_conflict_range(); bool end_unreadable = it.is_unreadable(); - TEST(it.is_conflict_range() != lastConflicted); // not last conflicted + CODE_PROBE(it.is_conflict_range() != lastConflicted, "not last conflicted"); it.tree.clear(); diff --git a/fdbclient/include/fdbclient/Atomic.h b/fdbclient/include/fdbclient/Atomic.h index 6643bcdafc..1d19150b04 100644 --- a/fdbclient/include/fdbclient/Atomic.h +++ b/fdbclient/include/fdbclient/Atomic.h @@ -120,7 +120,7 @@ inline ValueRef doAppendIfFits(const Optional& existingValueOptional, if (!otherOperand.size()) return existingValue; if (existingValue.size() + otherOperand.size() > CLIENT_KNOBS->VALUE_SIZE_LIMIT) { - TEST(true) // AppendIfFIts resulted in truncation + CODE_PROBE(true, "AppendIfFIts resulted in truncation"); return existingValue; } diff --git a/fdbrpc/AsyncFileCached.actor.cpp b/fdbrpc/AsyncFileCached.actor.cpp index 93bf9cfc70..1d56974094 100644 --- a/fdbrpc/AsyncFileCached.actor.cpp +++ b/fdbrpc/AsyncFileCached.actor.cpp @@ -193,14 +193,14 @@ Future AsyncFileCached::changeFileSize(int64_t size) { prevLength = size; if (offsetInPage) { - TEST(true); // Truncating to the middle of a page + CODE_PROBE(true, "Truncating to the middle of a page"); auto p = pages.find(pageOffset); if (p != pages.end()) { auto f = p->second->flush(); if (!f.isReady() || f.isError()) actors.push_back(f); } else { - TEST(true); // Truncating to the middle of a page that isn't in cache + CODE_PROBE(true, "Truncating to the middle of a page that isn't in cache"); } pageOffset += pageCache->pageSize; diff --git a/fdbrpc/FlowTests.actor.cpp b/fdbrpc/FlowTests.actor.cpp index 2af6dbb4aa..b88f8b60e7 100644 --- a/fdbrpc/FlowTests.actor.cpp +++ b/fdbrpc/FlowTests.actor.cpp @@ -69,7 +69,7 @@ TEST_CASE("/flow/buggifiedDelay") { }); wait(f1 && f2); if (last == 1) { - TEST(true); // Delays can become ready out of order + CODE_PROBE(true, "Delays can become ready out of order"); return Void(); } } diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 2dddf2ab48..eb71616917 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -615,8 +615,8 @@ ACTOR Future connectionWriter(Reference self, Reference break; } - TEST(true); // We didn't write everything, so apparently the write buffer is full. Wait for it to be - // nonfull. + CODE_PROBE( + true, "We didn't write everything, so apparently the write buffer is full. Wait for it to be nonfull"); wait(conn->onWritable()); wait(yield(TaskPriority::WriteSocket)); } @@ -1462,7 +1462,7 @@ ACTOR static Future connectionIncoming(TransportData* self, Reference p = wait(onConnected.getFuture())) { p->onIncomingConnection(p, conn, reader); } when(wait(delayJittered(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT))) { - TEST(true); // Incoming connection timed out + CODE_PROBE(true, "Incoming connection timed out"); throw timed_out(); } } @@ -1703,7 +1703,7 @@ void FlowTransport::addWellKnownEndpoint(Endpoint& endpoint, NetworkMessageRecei } static void sendLocal(TransportData* self, ISerializeSource const& what, const Endpoint& destination) { - TEST(true); // "Loopback" delivery + CODE_PROBE(true, "\"Loopback\" delivery"); // SOMEDAY: Would it be better to avoid (de)serialization by doing this check in flow? Standalone copy; @@ -1742,7 +1742,7 @@ static ReliablePacket* sendPacket(TransportData* self, // If there isn't an open connection, a public address, or the peer isn't compatible, we can't send if (!peer || (peer->outgoingConnectionIdle && !destination.getPrimaryAddress().isPublic()) || (!peer->compatible && destination.token != Endpoint::wellKnownToken(WLTOKEN_PING_PACKET))) { - TEST(true); // Can't send to private address without a compatible open connection + CODE_PROBE(true, "Can't send to private address without a compatible open connection"); return nullptr; } diff --git a/fdbrpc/include/fdbrpc/AsyncFileEIO.actor.h b/fdbrpc/include/fdbrpc/AsyncFileEIO.actor.h index 1a4c37ed20..a41bbfadea 100644 --- a/fdbrpc/include/fdbrpc/AsyncFileEIO.actor.h +++ b/fdbrpc/include/fdbrpc/AsyncFileEIO.actor.h @@ -116,7 +116,7 @@ public: static Future deleteFile(std::string filename, bool mustBeDurable) { ::deleteFile(filename); if (mustBeDurable) { - TEST(true); // deleteFile and fsync parent dir + CODE_PROBE(true, "deleteFile and fsync parent dir"); return async_fsync_parent(filename); } else return Void(); diff --git a/fdbrpc/include/fdbrpc/AsyncFileNonDurable.actor.h b/fdbrpc/include/fdbrpc/AsyncFileNonDurable.actor.h index 071839bc15..4b548afbf6 100644 --- a/fdbrpc/include/fdbrpc/AsyncFileNonDurable.actor.h +++ b/fdbrpc/include/fdbrpc/AsyncFileNonDurable.actor.h @@ -360,7 +360,7 @@ public: //(e.g. to simulate power failure) Future kill() { TraceEvent("AsyncFileNonDurable_Kill", id).detail("Filename", filename); - TEST(true); // AsyncFileNonDurable was killed + CODE_PROBE(true, "AsyncFileNonDurable was killed"); return sync(this, false); } @@ -404,7 +404,7 @@ private: TraceEvent("AsyncFileNonDurable_KilledFileOperation", self->id) .detail("In", context) .detail("Filename", self->filename); - TEST(true); // AsyncFileNonDurable operation killed + CODE_PROBE(true, "AsyncFileNonDurable operation killed"); throw io_error().asInjectedFault(); } @@ -603,13 +603,13 @@ private: .detail("HasGarbage", garbage) .detail("Side", side) .detail("Filename", self->filename); - TEST(true); // AsyncFileNonDurable bad write + CODE_PROBE(true, "AsyncFileNonDurable bad write"); } else { TraceEvent("AsyncFileNonDurable_DroppedWrite", self->id) .detail("Offset", offset + writeOffset + pageOffset) .detail("Length", sectorLength) .detail("Filename", self->filename); - TEST(true); // AsyncFileNonDurable dropped write + CODE_PROBE(true, "AsyncFileNonDurable dropped write"); } pageOffset += sectorLength; @@ -689,7 +689,7 @@ private: wait(self->file->truncate(size)); else { TraceEvent("AsyncFileNonDurable_DroppedTruncate", self->id).detail("Size", size); - TEST(true); // AsyncFileNonDurable dropped truncate + CODE_PROBE(true, "AsyncFileNonDurable dropped truncate"); } return Void(); @@ -753,7 +753,7 @@ private: // temporary file and then renamed to the correct location once sync is called. By not calling sync, we // simulate a failure to fsync the directory storing the file if (self->hasBeenSynced && writeDurable && deterministicRandom()->random01() < 0.5) { - TEST(true); // AsyncFileNonDurable kill was durable and synced + CODE_PROBE(true, "AsyncFileNonDurable kill was durable and synced"); wait(success(errorOr(self->file->sync()))); } diff --git a/fdbrpc/include/fdbrpc/LoadBalance.actor.h b/fdbrpc/include/fdbrpc/LoadBalance.actor.h index cf46e0b73a..91ab0e3b6d 100644 --- a/fdbrpc/include/fdbrpc/LoadBalance.actor.h +++ b/fdbrpc/include/fdbrpc/LoadBalance.actor.h @@ -140,7 +140,7 @@ Future tssComparison(Req req, tssData.metrics->recordLatency(req, srcEndTime - startTime, tssEndTime - startTime); if (!TSS_doCompare(src.get(), tss.get().get())) { - TEST(true); // TSS Mismatch + CODE_PROBE(true, "TSS Mismatch"); state TraceEvent mismatchEvent( (g_network->isSimulated() && g_simulator.tssMode == ISimulator::TSSMode::EnabledDropMutations) ? SevWarnAlways @@ -150,7 +150,7 @@ Future tssComparison(Req req, mismatchEvent.detail("TSSID", tssData.tssId); if (FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_VERIFY_SS && ssTeam->size() > 1) { - TEST(true); // checking TSS mismatch against rest of storage team + CODE_PROBE(true, "checking TSS mismatch against rest of storage team"); // if there is more than 1 SS in the team, attempt to verify that the other SS servers have the same // data @@ -195,9 +195,9 @@ Future tssComparison(Req req, if (tssData.metrics->shouldRecordDetailedMismatch()) { TSS_traceMismatch(mismatchEvent, req, src.get(), tss.get().get()); - TEST(FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL); // Tracing Full TSS Mismatch - TEST(!FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL); // Tracing Partial TSS Mismatch and storing - // the rest in FDB + CODE_PROBE(FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL, "Tracing Full TSS Mismatch"); + CODE_PROBE(!FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL, + "Tracing Partial TSS Mismatch and storing the rest in FDB"); if (!FLOW_KNOBS->LOAD_BALANCE_TSS_MISMATCH_TRACE_FULL) { mismatchEvent.disable(); @@ -268,7 +268,7 @@ struct RequestData : NonCopyable { Optional tssData = model->getTssData(stream->getEndpoint().token.first()); if (tssData.present()) { - TEST(true); // duplicating request to TSS + CODE_PROBE(true, "duplicating request to TSS"); resetReply(request); // FIXME: optimize to avoid creating new netNotifiedQueue for each message RequestStream tssRequestStream(tssData.get().endpoint); diff --git a/fdbrpc/include/fdbrpc/genericactors.actor.h b/fdbrpc/include/fdbrpc/genericactors.actor.h index de253069fc..f36a6271fb 100644 --- a/fdbrpc/include/fdbrpc/genericactors.actor.h +++ b/fdbrpc/include/fdbrpc/genericactors.actor.h @@ -47,7 +47,7 @@ Future retryBrokenPromise(RequestStream to, Req request throw; resetReply(request); wait(delayJittered(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY)); - TEST(true); // retryBrokenPromise + CODE_PROBE(true, "retryBrokenPromise"); } } } @@ -67,7 +67,7 @@ Future retryBrokenPromise(RequestStream to, Req request throw; resetReply(request); wait(delayJittered(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY, taskID)); - TEST(true); // retryBrokenPromise with taskID + CODE_PROBE(true, "retryBrokenPromise with taskID"); } } } diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 98bc972b57..f7689c0b4e 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -75,10 +75,19 @@ bool simulator_should_inject_fault(const char* context, const char* file, int li uint32_t h1 = line + (p->fault_injection_r >> 32); if (h1 < p->fault_injection_p1 * std::numeric_limits::max()) { - TEST(true); // A fault was injected - TEST(error_code == error_code_io_timeout); // An io timeout was injected - TEST(error_code == error_code_io_error); // An io error was injected - TEST(error_code == error_code_platform_error); // A platform error was injected. + CODE_PROBE(true, "A fault was injected", probe::assert::simOnly, probe::context::sim2); + CODE_PROBE(error_code == error_code_io_timeout, + "An io timeout was injected", + probe::assert::simOnly, + probe::context::sim2); + CODE_PROBE(error_code == error_code_io_error, + "An io error was injected", + probe::assert::simOnly, + probe::context::sim2); + CODE_PROBE(error_code == error_code_platform_error, + "A platform error was injected.", + probe::assert::simOnly, + probe::context::sim2); TraceEvent(SevWarn, "FaultInjected") .detail("Context", context) .detail("File", file) @@ -426,7 +435,7 @@ private: deterministicRandom()->random01() < .00001) { g_simulator.lastConnectionFailure = now(); double a = deterministicRandom()->random01(), b = deterministicRandom()->random01(); - TEST(true); // Simulated connection failure + CODE_PROBE(true, "Simulated connection failure", probe::context::sim2, probe::assert::simOnly); TraceEvent("ConnectionFailure", dbgid) .detail("MyAddr", process->address) .detail("PeerAddr", peerProcess->address) @@ -1178,7 +1187,7 @@ public: auto f = IAsyncFileSystem::filesystem(self->net2)->deleteFile(filename, false); ASSERT(f.isReady()); wait(::delay(0.05 * deterministicRandom()->random01())); - TEST(true); // Simulated durable delete + CODE_PROBE(true, "Simulated durable delete", probe::context::sim2, probe::assert::simOnly); } wait(g_simulator.onProcess(currentProcess, currentTaskID)); return Void(); @@ -1191,7 +1200,7 @@ public: TraceEvent(SevDebug, "Sim2DeleteFileImplNonDurable") .detail("Filename", filename) .detail("Durable", mustBeDurable); - TEST(true); // Simulated non-durable delete + CODE_PROBE(true, "Simulated non-durable delete", probe::context::sim2, probe::assert::simOnly); return Void(); } } @@ -1587,10 +1596,20 @@ public: killProcess_internal(p, KillInstantly); } void killProcess_internal(ProcessInfo* machine, KillType kt) { - TEST(true); // Simulated machine was killed with any kill type - TEST(kt == KillInstantly); // Simulated machine was killed instantly - TEST(kt == InjectFaults); // Simulated machine was killed with faults - TEST(kt == FailDisk); // Simulated machine was killed with a failed disk + CODE_PROBE( + true, "Simulated machine was killed with any kill type", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE(kt == KillInstantly, + "Simulated machine was killed instantly", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE(kt == InjectFaults, + "Simulated machine was killed with faults", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE(kt == FailDisk, + "Simulated machine was killed with a failed disk", + probe::context::sim2, + probe::assert::simOnly); if (kt == KillInstantly) { TraceEvent(SevWarn, "FailMachine") @@ -1715,9 +1734,10 @@ public: KillType* ktFinal) override { auto ktOrig = kt; - TEST(true); // Trying to killing a machine - TEST(kt == KillInstantly); // Trying to kill instantly - TEST(kt == InjectFaults); // Trying to kill by injecting faults + CODE_PROBE(true, "Trying to killing a machine", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE(kt == KillInstantly, "Trying to kill instantly", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE( + kt == InjectFaults, "Trying to kill by injecting faults", probe::context::sim2, probe::assert::simOnly); if (speedUpSimulation && !forceKill) { TraceEvent(SevWarn, "AbortedKill") @@ -1851,11 +1871,17 @@ public: } } - TEST(originalKt != kt); // Kill type was changed from requested to reboot. + CODE_PROBE(originalKt != kt, + "Kill type was changed from requested to reboot.", + probe::context::sim2, + probe::assert::simOnly); // Check if any processes on machine are rebooting if (processesOnMachine != processesPerMachine && kt >= RebootAndDelete) { - TEST(true); // Attempted reboot, but the target did not have all of its processes running + CODE_PROBE(true, + "Attempted reboot, but the target did not have all of its processes running", + probe::context::sim2, + probe::assert::simOnly); TraceEvent(SevWarn, "AbortedKill") .detail("KillType", kt) .detail("MachineId", machineId) @@ -1870,7 +1896,10 @@ public: // Check if any processes on machine are rebooting if (processesOnMachine != processesPerMachine) { - TEST(true); // Attempted reboot and kill, but the target did not have all of its processes running + CODE_PROBE(true, + "Attempted reboot and kill, but the target did not have all of its processes running", + probe::context::sim2, + probe::assert::simOnly); TraceEvent(SevWarn, "AbortedKill") .detail("KillType", kt) .detail("MachineId", machineId) @@ -1920,10 +1949,12 @@ public: } } - TEST(kt == RebootAndDelete); // Resulted in a reboot and delete - TEST(kt == Reboot); // Resulted in a reboot - TEST(kt == KillInstantly); // Resulted in an instant kill - TEST(kt == InjectFaults); // Resulted in a kill by injecting faults + CODE_PROBE( + kt == RebootAndDelete, "Resulted in a reboot and delete", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE(kt == Reboot, "Resulted in a reboot", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE(kt == KillInstantly, "Resulted in an instant kill", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE( + kt == InjectFaults, "Resulted in a kill by injecting faults", probe::context::sim2, probe::assert::simOnly); if (ktFinal) *ktFinal = kt; @@ -2037,13 +2068,32 @@ public: .detail("KillTypeMin", ktMin) .detail("KilledDC", kt == ktMin); - TEST(kt != ktMin); // DataCenter kill was rejected by killMachine - TEST((kt == ktMin) && (kt == RebootAndDelete)); // Datacenter kill Resulted in a reboot and delete - TEST((kt == ktMin) && (kt == Reboot)); // Datacenter kill Resulted in a reboot - TEST((kt == ktMin) && (kt == KillInstantly)); // Datacenter kill Resulted in an instant kill - TEST((kt == ktMin) && (kt == InjectFaults)); // Datacenter kill Resulted in a kill by injecting faults - TEST((kt == ktMin) && (kt != ktOrig)); // Datacenter Kill request was downgraded - TEST((kt == ktMin) && (kt == ktOrig)); // Datacenter kill - Requested kill was done + CODE_PROBE( + kt != ktMin, "DataCenter kill was rejected by killMachine", probe::context::sim2, probe::assert::simOnly); + CODE_PROBE((kt == ktMin) && (kt == RebootAndDelete), + "Datacenter kill Resulted in a reboot and delete", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE((kt == ktMin) && (kt == Reboot), + "Datacenter kill Resulted in a reboot", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE((kt == ktMin) && (kt == KillInstantly), + "Datacenter kill Resulted in an instant kill", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE((kt == ktMin) && (kt == InjectFaults), + "Datacenter kill Resulted in a kill by injecting faults", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE((kt == ktMin) && (kt != ktOrig), + "Datacenter Kill request was downgraded", + probe::context::sim2, + probe::assert::simOnly); + CODE_PROBE((kt == ktMin) && (kt == ktOrig), + "Datacenter kill - Requested kill was done", + probe::context::sim2, + probe::assert::simOnly); if (ktFinal) *ktFinal = ktMin; @@ -2276,7 +2326,7 @@ class UDPSimSocket : public IUDPSocket, ReferenceCounted { NetworkAddress _localAddress; bool randomDropPacket() { auto res = deterministicRandom()->random01() < .000001; - TEST(res); // UDP packet drop + CODE_PROBE(res, "UDP packet drop", probe::context::sim2, probe::assert::simOnly); return res; } @@ -2485,12 +2535,20 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) { ASSERT(kt == ISimulator::RebootProcess || kt == ISimulator::Reboot || kt == ISimulator::RebootAndDelete || kt == ISimulator::RebootProcessAndDelete); - TEST(kt == ISimulator::RebootProcess); // Simulated process rebooted - TEST(kt == ISimulator::Reboot); // Simulated machine rebooted - TEST(kt == ISimulator::RebootAndDelete); // Simulated machine rebooted with data and coordination state deletion - TEST( - kt == - ISimulator::RebootProcessAndDelete); // Simulated process rebooted with data and coordination state deletion + CODE_PROBE(kt == ISimulator::RebootProcess, + "Simulated process rebooted", + probe::assert::simOnly, + probe::context::sim2); + CODE_PROBE( + kt == ISimulator::Reboot, "Simulated machine rebooted", probe::assert::simOnly, probe::context::sim2); + CODE_PROBE(kt == ISimulator::RebootAndDelete, + "Simulated machine rebooted with data and coordination state deletion", + probe::assert::simOnly, + probe::context::sim2); + CODE_PROBE(kt == ISimulator::RebootProcessAndDelete, + "Simulated process rebooted with data and coordination state deletion", + probe::assert::simOnly, + probe::context::sim2); if (p->rebooting || !p->isReliable()) { TraceEvent(SevDebug, "DoRebootFailed") diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index b02f091179..f3b0e4c5f4 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -624,7 +624,7 @@ private: if (!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); confChange = true; - TEST(true); // Recovering at a higher version. + CODE_PROBE(true, "Recovering at a higher version."); } void checkSetVersionEpochKey(MutationRef m) { @@ -636,7 +636,7 @@ private: if (!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); confChange = true; - TEST(true); // Setting version epoch + CODE_PROBE(true, "Setting version epoch"); } void checkSetWriteRecoverKey(MutationRef m) { @@ -646,7 +646,7 @@ private: TraceEvent("WriteRecoveryKeySet", dbgid).log(); if (!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); - TEST(true); // Snapshot created, setting writeRecoveryKey in txnStateStore + CODE_PROBE(true, "Snapshot created, setting writeRecoveryKey in txnStateStore"); } void checkSetTenantMapPrefix(MutationRef m) { @@ -680,7 +680,7 @@ private: writeMutation(privatized); } - TEST(true); // Tenant added to map + CODE_PROBE(true, "Tenant added to map"); } } @@ -1068,7 +1068,7 @@ private: writeMutation(privatized); } - TEST(true); // Tenant cleared from map + CODE_PROBE(true, "Tenant cleared from map"); } } diff --git a/fdbserver/BackupWorker.actor.cpp b/fdbserver/BackupWorker.actor.cpp index 91c9962577..515e52f682 100644 --- a/fdbserver/BackupWorker.actor.cpp +++ b/fdbserver/BackupWorker.actor.cpp @@ -72,7 +72,7 @@ struct VersionedMessage { if (reader.protocolVersion().hasSpanContext() && SpanContextMessage::isNextIn(reader)) return false; if (reader.protocolVersion().hasOTELSpanContext() && OTELSpanContextMessage::isNextIn(reader)) { - TEST(true); // Returning false for OTELSpanContextMessage + CODE_PROBE(true, "Returning false for OTELSpanContextMessage"); return false; } if (EncryptedMutationMessage::isNextIn(reader)) { diff --git a/fdbserver/BlobManager.actor.cpp b/fdbserver/BlobManager.actor.cpp index 605b95e91e..91af517d4b 100644 --- a/fdbserver/BlobManager.actor.cpp +++ b/fdbserver/BlobManager.actor.cpp @@ -393,7 +393,7 @@ ACTOR Future>> splitRange(Reference splitThreshold) { // only split on bytes and write rate state StorageMetrics splitMetrics; @@ -429,7 +429,7 @@ ACTOR Future>> splitRange(Reference>> splitRange(Reference pickWorkerForAssign(Reference bmData) { // wait until there are BWs to pick from while (bmData->workerStats.size() == 0) { - TEST(true); // BM wants to assign range, but no workers available + CODE_PROBE(true, "BM wants to assign range, but no workers available"); if (BM_DEBUG) { fmt::print("BM {0} waiting for blob workers before assigning granules\n", bmData->epoch); } @@ -619,7 +619,7 @@ ACTOR Future doRangeAssignment(Reference bmData, throw; } - TEST(true); // BM retrying range assign + CODE_PROBE(true, "BM retrying range assign"); // We use reliable delivery (getReply), so the broken_promise means the worker is dead, and we may need to retry // somewhere else @@ -683,7 +683,7 @@ static bool handleRangeIsAssign(Reference bmData, RangeAssignme if (assignment.assign.get().type == AssignRequestType::Continue) { ASSERT(assignment.worker.present()); if (i.range() != assignment.keyRange || i.cvalue() != assignment.worker.get()) { - TEST(true); // BM assignment out of date + CODE_PROBE(true, "BM assignment out of date"); if (BM_DEBUG) { fmt::print("Out of date re-assign for ({0}, {1}). Assignment must have changed while " "checking split.\n Reassign: [{2} - {3}): {4}\n Existing: [{5} - {6}): {7}\n", @@ -814,7 +814,7 @@ ACTOR Future writeInitialGranuleMapping(Reference bmData, state int i = 0; state int transactionChunkSize = BUGGIFY ? deterministicRandom()->randomInt(2, 5) : 1000; while (i < boundaries.size() - 1) { - TEST(i > 0); // multiple transactions for large granule split + CODE_PROBE(i > 0, "multiple transactions for large granule split"); tr->reset(); state int j = 0; loop { @@ -1110,7 +1110,7 @@ ACTOR Future maybeSplitRange(Reference bmData, // Enforce max split fanout for performance reasons. This mainly happens when a blob worker is behind. if (newRanges.size() >= SERVER_KNOBS->BG_MAX_SPLIT_FANOUT + 2) { // +2 because this is boundaries, so N keys would have N+1 bounaries. - TEST(true); // downsampling granule split because fanout too high + CODE_PROBE(true, "downsampling granule split because fanout too high"); Standalone> coalescedRanges; coalescedRanges.arena().dependsOn(newRanges.arena()); coalescedRanges.push_back(coalescedRanges.arena(), newRanges.front()); @@ -1184,7 +1184,7 @@ ACTOR Future maybeSplitRange(Reference bmData, if (!existingState.empty()) { // Something was previously committed, we must go with that decision. // Read its boundaries and override our planned split boundaries - TEST(true); // Overriding split ranges with existing ones from DB + CODE_PROBE(true, "Overriding split ranges with existing ones from DB"); RangeResult existingBoundaries = wait(tr->getRange(KeyRangeRef(granuleRange.begin.withPrefix(blobGranuleMappingKeys.begin), keyAfter(granuleRange.end).withPrefix(blobGranuleMappingKeys.begin)), @@ -1561,7 +1561,7 @@ ACTOR Future persistMergeGranulesDone(Reference bmData, } } if (tmpWorkerId == UID()) { - TEST(true); // All workers dead right now + CODE_PROBE(true, "All workers dead right now"); while (bmData->workersById.empty()) { wait(bmData->recruitingStream.onChange() || bmData->foundBlobWorkers.getFuture()); } @@ -1631,7 +1631,7 @@ ACTOR Future persistMergeGranulesDone(Reference bmData, mergeVersion, tr->getCommittedVersion()); } - TEST(true); // Granule merge complete + CODE_PROBE(true, "Granule merge complete"); return Void(); } catch (Error& e) { wait(tr->onError(e)); @@ -1753,7 +1753,7 @@ ACTOR Future maybeMergeRange(Reference bmData, } if (beforeCandidates.empty() && afterCandidates.empty()) { - TEST(true); // no consecutive merge candidates + CODE_PROBE(true, "no consecutive merge candidates"); if (BM_DEBUG) { fmt::print("BM {0} maybe merge [{1} - {2}): No merge candidates\n", bmData->epoch, @@ -1763,7 +1763,7 @@ ACTOR Future maybeMergeRange(Reference bmData, return Void(); } - TEST(true); // consecutive granule merge candidates + CODE_PROBE(true, "consecutive granule merge candidates"); if (BM_DEBUG) { fmt::print("BM {0} maybe merge [{1} - {2}): Checking metrics for {3} candidates ({4} - {5})\n", @@ -1779,7 +1779,7 @@ ACTOR Future maybeMergeRange(Reference bmData, StorageMetrics targetGranuleMetrics = wait(bmData->db->getStorageMetrics(granuleRange, CLIENT_KNOBS->TOO_MANY)); if (targetGranuleMetrics.bytesPerKSecond >= SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC || targetGranuleMetrics.bytes >= SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES) { - TEST(true); // granule merge candidate no longer mergeable + CODE_PROBE(true, "granule merge candidate no longer mergeable"); return Void(); } @@ -1924,8 +1924,8 @@ ACTOR Future maybeMergeRange(Reference bmData, } } - TEST(bestGranuleIDs.size() == 1); // Cannot combine merge candidates into mergeable granule - TEST(bestGranuleIDs.size() > 1); // Granule ready for merge! + CODE_PROBE(bestGranuleIDs.size() == 1, "Cannot combine merge candidates into mergeable granule"); + CODE_PROBE(bestGranuleIDs.size() > 1, "Granule ready for merge!"); if (bestGranuleIDs.size() > 1) { if (BM_DEBUG) { @@ -1945,7 +1945,7 @@ ACTOR Future maybeMergeRange(Reference bmData, bool mergeStillOk = true; for (auto it : reCheckMergeCandidates) { if (!it->cvalue().present()) { - TEST(true); // granule no longer merge candidate after checking metrics, because of split eval + CODE_PROBE(true, "granule no longer merge candidate after checking metrics, because of split eval"); mergeStillOk = false; break; } @@ -1959,7 +1959,7 @@ ACTOR Future maybeMergeRange(Reference bmData, bestGranuleRange.begin.printable(), bestGranuleRange.end.printable(), bestGranuleIDs.size()); - TEST(true); // Doing granule merge! + CODE_PROBE(true, "Doing granule merge!"); bmData->activeGranuleMerges.insert(bestGranuleRange, 0); bmData->mergeCandidates.insert(bestGranuleRange, Optional>()); state std::pair persistMerge = wait(persistMergeGranulesStart( @@ -2173,7 +2173,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl } ignore = true; } else if (newEval < lastBoundaryEval.cvalue()) { - TEST(true); // BM got out-of-date split request + CODE_PROBE(true, "BM got out-of-date split request"); if (BM_DEBUG) { fmt::print("BM {0} ignoring status from BW {1} for granule [{2} - {3}) {4} since it " "already processed [{5} - {6}) {7}.\n", @@ -2249,7 +2249,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // suddenly gets a burst of writes after a decision to merge is made if (inProgressMergeVersion != invalidVersion) { if (rep.blockedVersion < inProgressMergeVersion) { - TEST(true); // merge blocking re-snapshot + CODE_PROBE(true, "merge blocking re-snapshot"); if (BM_DEBUG) { fmt::print("DBG: BM {0} MERGE @ {1} blocking re-snapshot [{2} - {3}) @ {4}, " "continuing snapshot\n", @@ -2328,7 +2328,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl } if (!bmData->isMergeActive(rep.granuleRange)) { ASSERT(rep.mergeCandidate); - TEST(true); // Granule merge candidate + CODE_PROBE(true, "Granule merge candidate"); bmData->mergeCandidates.insert(rep.granuleRange, std::pair(rep.granuleID, rep.startVersion)); newEval.inProgress = @@ -2363,7 +2363,7 @@ ACTOR Future monitorBlobWorkerStatus(Reference bmData, Bl // if it is permanent, the failure monitor will eventually trip. ASSERT(e.code() != error_code_end_of_stream); if (e.code() == error_code_request_maybe_delivered || e.code() == error_code_connection_failed) { - TEST(true); // BM retrying BW monitoring + CODE_PROBE(true, "BM retrying BW monitoring"); wait(delay(backoff)); backoff = std::min(backoff * SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_EXPONENT, SERVER_KNOBS->BLOB_MANAGER_STATUS_EXP_BACKOFF_MAX); @@ -2501,7 +2501,7 @@ static void addAssignment(KeyRangeMap>& map, if (oldEpoch > newEpoch || (oldEpoch == newEpoch && oldSeqno > newSeqno)) { newer.push_back(std::pair(old.range(), std::tuple(oldWorker, oldEpoch, oldSeqno))); if (old.range() != newRange) { - TEST(true); // BM Recovery: BWs disagree on range boundaries + CODE_PROBE(true, "BM Recovery: BWs disagree on range boundaries"); anyConflicts = true; } } else { @@ -2511,7 +2511,7 @@ static void addAssignment(KeyRangeMap>& map, ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno); } if (newEpoch == std::numeric_limits::max() && (oldWorker != newId || old.range() != newRange)) { - TEST(true); // BM Recovery: DB disagrees with workers + CODE_PROBE(true, "BM Recovery: DB disagrees with workers"); // new one is from DB (source of truth on boundaries) and existing mapping disagrees on boundary or // assignment, do explicit revoke and re-assign to converge anyConflicts = true; @@ -2535,7 +2535,7 @@ static void addAssignment(KeyRangeMap>& map, std::get<0>(old.value()) = UID(); } if (outOfDate.empty() || outOfDate.back() != std::pair(oldWorker, KeyRange(old.range()))) { - TEST(true); // BM Recovery: Two workers claim ownership of same granule + CODE_PROBE(true, "BM Recovery: Two workers claim ownership of same granule"); outOfDate.push_back(std::pair(oldWorker, old.range())); } } @@ -2575,7 +2575,7 @@ ACTOR Future resumeActiveMerges(Reference bmData) { RangeResult result = wait(tr->getRange(currentRange, rowLimit)); for (auto& it : result) { - TEST(true); // Blob Manager Recovery found merging granule + CODE_PROBE(true, "Blob Manager Recovery found merging granule"); UID mergeGranuleID = decodeBlobGranuleMergeKey(it.key); KeyRange mergeRange; std::vector parentGranuleIDs; @@ -2641,7 +2641,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { state Future resumeMergesFuture = resumeActiveMerges(bmData); - TEST(true); // BM doing recovery + CODE_PROBE(true, "BM doing recovery"); wait(delay(0)); @@ -2722,7 +2722,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { bmData->workerStats[workerId].numGranulesAssigned = reply.get().assignments.size(); } } else { - TEST(true); // BM Recovery: BW didn't respond to assignments request + CODE_PROBE(true, "BM Recovery: BW didn't respond to assignments request"); // SOMEDAY: mark as failed and kill it if (BM_DEBUG) { fmt::print(" Worker {}: failed\n", workerId.toString().substr(0, 5)); @@ -2826,7 +2826,7 @@ ACTOR Future recoverBlobManager(Reference bmData) { } // revoke assignments that are old and incorrect - TEST(!outOfDateAssignments.empty()); // BM resolved conflicting assignments on recovery + CODE_PROBE(!outOfDateAssignments.empty(), "BM resolved conflicting assignments on recovery"); for (auto& it : outOfDateAssignments) { if (BM_DEBUG) { fmt::print("BM {0} revoking out of date assignment [{1} - {2}): {3}:\n", @@ -2896,7 +2896,7 @@ ACTOR Future chaosRangeMover(Reference bmData) { // KeyRange isn't hashable and this is only for simulation, so just use toString of range state std::unordered_set alreadyMoved; ASSERT(g_network->isSimulated()); - TEST(true); // BM chaos range mover enabled + CODE_PROBE(true, "BM chaos range mover enabled"); loop { wait(delay(30.0)); @@ -3000,7 +3000,7 @@ ACTOR Future initializeBlobWorker(Reference self, Recruit // if it failed in an expected way, add some delay before we try to recruit again // on this worker if (newBlobWorker.isError()) { - TEST(true); // BM got error recruiting BW + CODE_PROBE(true, "BM got error recruiting BW"); TraceEvent(SevWarn, "BMRecruitmentError", self->id) .error(newBlobWorker.getError()) .detail("Epoch", self->epoch); @@ -3104,7 +3104,7 @@ ACTOR Future blobWorkerRecruiter( if (e.code() != error_code_timed_out) { throw; } - TEST(true); // Blob worker recruitment timed out + CODE_PROBE(true, "Blob worker recruitment timed out"); } } } @@ -3789,7 +3789,7 @@ ACTOR Future doLockChecks(Reference bmData) { wait(check.getFuture()); wait(delay(0.5)); // don't do this too often if a lot of conflict - TEST(true); // BM doing lock checks after getting conflicts + CODE_PROBE(true, "BM doing lock checks after getting conflicts"); state Reference tr = makeReference(bmData->db); diff --git a/fdbserver/BlobWorker.actor.cpp b/fdbserver/BlobWorker.actor.cpp index b723e40336..2d94e95eb4 100644 --- a/fdbserver/BlobWorker.actor.cpp +++ b/fdbserver/BlobWorker.actor.cpp @@ -539,7 +539,7 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, // tr->clear(singleKeyRange(oldGranuleLockKey)); tr->clear(currentRange); - TEST(true); // Granule split cleanup on last delta file persisted + CODE_PROBE(true, "Granule split cleanup on last delta file persisted"); } else { tr->atomicOp(myStateKey, blobGranuleSplitValueFor(newState), MutationRef::SetVersionstampedValue); if (newState == BlobGranuleSplitState::Assigned && currentState == BlobGranuleSplitState::Initialized && @@ -554,10 +554,10 @@ ACTOR Future updateGranuleSplitState(Transaction* tr, wait(updateChangeFeed( tr, KeyRef(granuleIDToCFKey(parentGranuleID)), ChangeFeedStatus::CHANGE_FEED_STOP)); } - TEST(true); // Granule split stopping change feed + CODE_PROBE(true, "Granule split stopping change feed"); } } else if (BW_DEBUG) { - TEST(true); // Out of order granule split state updates ignored + CODE_PROBE(true, "Out of order granule split state updates ignored"); fmt::print("Ignoring granule {0} split state from {1} {2} -> {3}\n", currentGranuleID.toString(), parentGranuleID.toString(), @@ -678,13 +678,13 @@ ACTOR Future writeDeltaFile(Reference bwData, // commit a transaction, we can and want to safely delete the file we wrote. Otherwise, we may have updated FDB // with file and cannot safely delete it. if (numIterations > 0) { - TEST(true); // Granule potentially leaving orphaned delta file + CODE_PROBE(true, "Granule potentially leaving orphaned delta file"); throw e; } if (BW_DEBUG) { fmt::print("deleting delta file {0} after error {1}\n", fname, e.name()); } - TEST(true); // Granule cleaning up delta file after error + CODE_PROBE(true, "Granule cleaning up delta file after error"); ++bwData->stats.s3DeleteReqs; bwData->addActor.send(writeBStore->deleteFile(fname)); throw e; @@ -811,13 +811,13 @@ ACTOR Future writeSnapshot(Reference bwData, // commit a transaction, we can and want to safely delete the file we wrote. Otherwise, we may have updated FDB // with file and cannot safely delete it. if (numIterations > 0) { - TEST(true); // Granule potentially leaving orphaned snapshot file + CODE_PROBE(true, "Granule potentially leaving orphaned snapshot file"); throw e; } if (BW_DEBUG) { fmt::print("deleting snapshot file {0} after error {1}\n", fname, e.name()); } - TEST(true); // Granule deleting snapshot file after error + CODE_PROBE(true, "Granule deleting snapshot file after error"); ++bwData->stats.s3DeleteReqs; bwData->addActor.send(writeBStore->deleteFile(fname)); throw e; @@ -907,7 +907,7 @@ ACTOR Future dumpInitialSnapshotFromFDB(Reference wait(tr->onError(e)); } retries++; - TEST(true); // Granule initial snapshot failed + CODE_PROBE(true, "Granule initial snapshot failed"); // FIXME: why can't we supress error event? TraceEvent(retries < 10 ? SevDebug : SevWarn, "BlobGranuleInitialSnapshotRetry", bwData->id) .error(err) @@ -1129,8 +1129,7 @@ ACTOR Future checkSplitAndReSnapshot(Reference bw if (e.code() == error_code_operation_cancelled) { throw e; } - TEST(true); // Blob worker re-sending split evaluation to manager after not error/not hearing - // back + CODE_PROBE(true, "Blob worker re-sending split evaluation to manager after not error/not hearing back"); // if we got broken promise while waiting, the old stream was killed, so we don't need to wait // on change, just retry if (e.code() == error_code_broken_promise) { @@ -1201,11 +1200,11 @@ ACTOR Future granuleCheckMergeCandidate(Reference bwData, if (currentMetrics.bytes >= SERVER_KNOBS->BG_SNAPSHOT_FILE_TARGET_BYTES / 2 || currentMetrics.bytesPerKSecond >= SERVER_KNOBS->SHARD_MIN_BYTES_PER_KSEC) { wait(delayJittered(SERVER_KNOBS->BG_MERGE_CANDIDATE_THRESHOLD_SECONDS / 2.0)); - TEST(true); // wait and check later to see if granule got smaller or colder + CODE_PROBE(true, "wait and check later to see if granule got smaller or colder"); continue; } - TEST(true); // Blob Worker identified merge candidate granule + CODE_PROBE(true, "Blob Worker identified merge candidate granule"); // if we are a merge candidate, send a message to the BM. Once successful, this actor is complete while (!bwData->statusStreamInitialized) { @@ -1226,7 +1225,7 @@ ACTOR Future granuleCheckMergeCandidate(Reference bwData, } if (now() >= sendTimeGiveUp) { - TEST(true); // Blob worker could not send merge candidate in time, re-checking status + CODE_PROBE(true, "Blob worker could not send merge candidate in time, re-checking status"); break; } @@ -1247,13 +1246,13 @@ ACTOR Future granuleCheckMergeCandidate(Reference bwData, wait(bwData->currentManagerStatusStream.onChange()); wait(delay(0)); } - TEST(true); // Blob worker re-sending merge candidate to new manager + CODE_PROBE(true, "Blob worker re-sending merge candidate to new manager"); } catch (Error& e) { if (e.code() == error_code_operation_cancelled) { throw e; } - TEST(true); // Blob worker re-sending merge candidate to manager after not error/not hearing back + CODE_PROBE(true, "Blob worker re-sending merge candidate to manager after not error/not hearing back"); // if we got broken promise while waiting, the old stream was killed, so we don't need to wait // on change, just retry @@ -1356,7 +1355,7 @@ Version doGranuleRollback(Reference metadata, for (auto& f : inFlightFiles) { if (f.snapshot) { if (f.version > rollbackVersion) { - TEST(true); // Granule rollback cancelling snapshot file + CODE_PROBE(true, "Granule rollback cancelling snapshot file"); if (BW_DEBUG) { fmt::print("[{0} - {1}) rollback cancelling snapshot file @ {2}\n", metadata->keyRange.begin.printable(), @@ -1377,7 +1376,7 @@ Version doGranuleRollback(Reference metadata, metadata->bytesInNewDeltaFiles -= f.bytes; } toPop++; - TEST(true); // Granule rollback cancelling delta file + CODE_PROBE(true, "Granule rollback cancelling delta file"); if (BW_DEBUG) { fmt::print("[{0} - {1}) rollback cancelling delta file @ {2}\n", metadata->keyRange.begin.printable(), @@ -1432,7 +1431,7 @@ Version doGranuleRollback(Reference metadata, } else { // No pending delta files to discard, just in-memory mutations - TEST(true); // Granule rollback discarding in memory mutations + CODE_PROBE(true, "Granule rollback discarding in memory mutations"); // FIXME: could binary search? int mIdx = metadata->currentDeltas.size() - 1; @@ -1637,7 +1636,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } if (!startState.doSnapshot) { - TEST(true); // Granule moved without split + CODE_PROBE(true, "Granule moved without split"); startVersion = startState.previousDurableVersion; ASSERT(!metadata->files.snapshotFiles.empty()); metadata->pendingSnapshotVersion = metadata->files.snapshotFiles.back().version; @@ -1798,7 +1797,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // popped up to V+1 is ok. Or in other words, if the last delta @ V, we only missed data // at V+1 onward if popVersion >= V+2 if (metadata->bufferedDeltaVersion < metadata->activeCFData.get()->popVersion - 1) { - TEST(true); // Blob Worker detected popped + CODE_PROBE(true, "Blob Worker detected popped"); TraceEvent("BlobWorkerChangeFeedPopped", bwData->id) .detail("Granule", metadata->keyRange) .detail("GranuleID", startState.granuleID) @@ -1896,7 +1895,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, if (metadata->pendingDeltaVersion <= rollbackVersion && (metadata->currentDeltas.empty() || metadata->currentDeltas.back().version <= rollbackVersion)) { - TEST(true); // Granule ignoring rollback + CODE_PROBE(true, "Granule ignoring rollback"); if (BW_DEBUG) { fmt::print("Granule [{0} - {1}) on BW {2} skipping rollback {3} -> {4} " @@ -1913,7 +1912,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // rollbackInProgress when we restart the stream. rollbacksCompleted.push_back(std::pair(rollbackVersion, deltas.version)); } else { - TEST(true); // Granule processing rollback + CODE_PROBE(true, "Granule processing rollback"); if (BW_DEBUG) { fmt::print("[{0} - {1}) on BW {2} ROLLBACK @ {3} -> {4}\n", metadata->keyRange.begin.printable(), @@ -1944,7 +1943,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // change feed ASSERT(cfRollbackVersion >= startState.previousDurableVersion); ASSERT(cfRollbackVersion >= metadata->durableDeltaVersion.get()); - TEST(true); // rollback crossed change feed boundaries + CODE_PROBE(true, "rollback crossed change feed boundaries"); readOldChangeFeed = true; oldChangeFeedDataComplete.reset(); } @@ -1990,7 +1989,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, } } else if (!rollbacksInProgress.empty() && rollbacksInProgress.front().first < deltas.version && rollbacksInProgress.front().second > deltas.version) { - TEST(true); // Granule skipping mutations b/c prior rollback + CODE_PROBE(true, "Granule skipping mutations b/c prior rollback"); if (BW_DEBUG) { fmt::print("Skipping mutations @ {} b/c prior rollback\n", deltas.version); } @@ -2033,7 +2032,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // The force flush contract is a version cannot be put in forceFlushVersion unless the change feed // is already whenAtLeast that version bool forceFlush = !forceFlushVersions.empty() && forceFlushVersions.back() > metadata->pendingDeltaVersion; - TEST(forceFlush); // Force flushing granule + CODE_PROBE(forceFlush, "Force flushing granule"); if (metadata->bufferedDeltaBytes >= SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES || forceFlush) { TraceEvent(SevDebug, "BlobGranuleDeltaFile", bwData->id) .detail("Granule", metadata->keyRange) @@ -2072,7 +2071,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // write/read a bunch of empty blob files ASSERT(forceFlush); ASSERT(!forceFlushVersions.empty()); - TEST(true); // Force flushing empty delta file! + CODE_PROBE(true, "Force flushing empty delta file!"); } if (BW_DEBUG) { @@ -2200,7 +2199,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, idx++; } while (waitIdx > 0) { - TEST(true); // Granule blocking on previous snapshot + CODE_PROBE(true, "Granule blocking on previous snapshot"); // TODO don't duplicate code BlobFileIndex completedFile = wait(inFlightFiles.front().future); if (inFlightFiles.front().snapshot) { @@ -2241,7 +2240,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, // queue too many files in parallel, and slow down change feed consuming to let file writing // catch up - TEST(true); // Granule processing long tail of old change feed + CODE_PROBE(true, "Granule processing long tail of old change feed"); if (inFlightFiles.size() > 10 && inFlightFiles.front().version <= metadata->knownCommittedVersion) { if (BW_DEBUG) { fmt::print("[{0} - {1}) Waiting on delta file b/c old change feed\n", @@ -2295,7 +2294,7 @@ ACTOR Future blobGranuleUpdateFiles(Reference bwData, ++bwData->stats.granuleUpdateErrors; if (granuleCanRetry(e)) { - TEST(true); // Granule close and re-open on error + CODE_PROBE(true, "Granule close and re-open on error"); TraceEvent("GranuleFileUpdaterRetriableError", bwData->id) .error(e) .detail("Granule", metadata->keyRange) @@ -2472,7 +2471,7 @@ ACTOR Future blobGranuleLoadHistory(Reference bwData, next.version); } } else { - TEST(true); // duplicate parent in granule history (split then merge) + CODE_PROBE(true, "duplicate parent in granule history (split then merge)"); if (BW_HISTORY_DEBUG) { fmt::print("HL {0} {1}) [{2} - {3}) @ {4}: duplicate parent [{5} - " "{6}) @ {7}\n", @@ -2820,7 +2819,7 @@ std::vector>> loadHistoryChunks(Referen ASSERT(!resultChunks.empty()); if (resultChunks.size() >= 2) { - TEST(true); // Multiple history chunks for time travel query + CODE_PROBE(true, "Multiple history chunks for time travel query"); std::sort(resultChunks.begin(), resultChunks.end(), sort_result_chunks()); // Assert contiguous for (int i = 0; i < resultChunks.size() - 1; i++) { @@ -2894,7 +2893,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v metadata->durableDeltaVersion.get() == metadata->pendingDeltaVersion) && (v <= metadata->durableSnapshotVersion.get() || metadata->durableSnapshotVersion.get() == metadata->pendingSnapshotVersion)) { - TEST(true); // Granule read not waiting + CODE_PROBE(true, "Granule read not waiting"); return Void(); } @@ -2911,7 +2910,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v // If there are mutations that are no longer buffered but have not been // persisted to a delta file that are necessary for the query, wait for them if (pendingDeltaV > metadata->durableDeltaVersion.get() && v > metadata->durableDeltaVersion.get()) { - TEST(true); // Granule read waiting for pending delta + CODE_PROBE(true, "Granule read waiting for pending delta"); wait(metadata->durableDeltaVersion.whenAtLeast(pendingDeltaV)); ASSERT(metadata->durableDeltaVersion.get() >= pendingDeltaV); } @@ -2919,7 +2918,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v // This isn't strictly needed, but if we're in the process of re-snapshotting, we'd likely rather // return that snapshot file than the previous snapshot file and all its delta files. if (pendingSnapshotV > metadata->durableSnapshotVersion.get() && v > metadata->durableSnapshotVersion.get()) { - TEST(true); // Granule read waiting for pending snapshot + CODE_PROBE(true, "Granule read waiting for pending snapshot"); wait(metadata->durableSnapshotVersion.whenAtLeast(pendingSnapshotV)); ASSERT(metadata->durableSnapshotVersion.get() >= pendingSnapshotV); } @@ -2929,7 +2928,7 @@ ACTOR Future waitForVersion(Reference metadata, Version v // file instead of in memory mutations, so we wait for that delta file to complete while (v > metadata->durableDeltaVersion.get() && metadata->pendingDeltaVersion > pendingDeltaV) { - TEST(true); // Granule mutations flushed while waiting for files to complete + CODE_PROBE(true, "Granule mutations flushed while waiting for files to complete"); Version waitVersion = std::min(v, metadata->pendingDeltaVersion); pendingDeltaV = metadata->pendingDeltaVersion; wait(metadata->durableDeltaVersion.whenAtLeast(waitVersion)); @@ -2960,7 +2959,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl ASSERT(tenantEntry.get().id == req.tenantInfo.tenantId); tenantPrefix = tenantEntry.get().prefix; } else { - TEST(true); // Blob worker unknown tenant + CODE_PROBE(true, "Blob worker unknown tenant"); // FIXME - better way. Wait on retry here, or just have better model for tenant metadata? // Just throw wrong_shard_server and make the client retry and assume we load it later TraceEvent(SevDebug, "BlobWorkerRequestUnknownTenant", bwData->id) @@ -3044,7 +3043,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl state std::vector> rangeGranulePair; if (req.readVersion < metadata->historyVersion) { - TEST(true); // Granule Time Travel Read + CODE_PROBE(true, "Granule Time Travel Read"); // this is a time travel query, find previous granule if (metadata->historyLoaded.canBeSet()) { choose { @@ -3083,7 +3082,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl throw blob_granule_transaction_too_old(); } - TEST(true); // Granule Active Read + CODE_PROBE(true, "Granule Active Read"); // this is an active granule query loop { if (!metadata->activeCFData.get().isValid() || !metadata->cancelled.canBeSet()) { @@ -3106,13 +3105,13 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl // We can get change feed cancelled from whenAtLeast. This means the change feed may // retry, or may be cancelled. Wait a bit and try again to see if (e.code() == error_code_change_feed_popped) { - TEST(true); // Change feed popped while read waiting + CODE_PROBE(true, "Change feed popped while read waiting"); throw wrong_shard_server(); } if (e.code() != error_code_change_feed_cancelled) { throw e; } - TEST(true); // Change feed switched while read waiting + CODE_PROBE(true, "Change feed switched while read waiting"); // wait 1ms and try again wait(delay(0.001)); } @@ -3143,7 +3142,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl Version granuleBeginVersion = req.beginVersion; // Right now we force a collapse if the version range crosses granule boundaries, for simplicity if (granuleBeginVersion > 0 && granuleBeginVersion <= item.second.snapshotFiles.front().version) { - TEST(true); // collapsed begin version request because of boundaries + CODE_PROBE(true, "collapsed begin version request because of boundaries"); didCollapse = true; granuleBeginVersion = 0; } @@ -3161,7 +3160,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl granuleBeginVersion, req.readVersion, req.canCollapseBegin, chunk, rep.arena, deltaBytes); bwData->stats.readReqDeltaBytesReturned += deltaBytes; if (granuleBeginVersion > 0 && chunk.snapshotFile.present()) { - TEST(true); // collapsed begin version request for efficiency + CODE_PROBE(true, "collapsed begin version request for efficiency"); didCollapse = true; } @@ -3193,11 +3192,11 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl rep.arena.dependsOn(metadata->currentDeltas.arena()); MutationsAndVersionRef* mutationIt = metadata->currentDeltas.begin(); if (granuleBeginVersion > metadata->currentDeltas.back().version) { - TEST(true); // beginVersion pruning all in-memory mutations + CODE_PROBE(true, "beginVersion pruning all in-memory mutations"); mutationIt = metadata->currentDeltas.end(); } else if (granuleBeginVersion > metadata->currentDeltas.front().version) { // binary search for beginVersion - TEST(true); // beginVersion pruning some in-memory mutations + CODE_PROBE(true, "beginVersion pruning some in-memory mutations"); mutationIt = std::lower_bound(metadata->currentDeltas.begin(), metadata->currentDeltas.end(), MutationsAndVersionRef(granuleBeginVersion, 0), @@ -3207,7 +3206,7 @@ ACTOR Future doBlobGranuleFileRequest(Reference bwData, Bl // add mutations to response while (mutationIt != metadata->currentDeltas.end()) { if (mutationIt->version > req.readVersion) { - TEST(true); // readVersion pruning some in-memory mutations + CODE_PROBE(true, "readVersion pruning some in-memory mutations"); break; } chunk.newDeltas.push_back_deep(rep.arena, *mutationIt); @@ -3262,7 +3261,7 @@ ACTOR Future handleBlobGranuleFileRequest(Reference bwData when(wait(doBlobGranuleFileRequest(bwData, req))) {} when(wait(delay(SERVER_KNOBS->BLOB_WORKER_REQUEST_TIMEOUT))) { if (!req.reply.isSet()) { - TEST(true); // Blob Worker request timeout hit + CODE_PROBE(true, "Blob Worker request timeout hit"); if (BW_DEBUG) { fmt::print("BW {0} request [{1} - {2}) @ {3} timed out, sending WSS\n", bwData->id.toString().substr(0, 5), @@ -3326,7 +3325,7 @@ ACTOR Future openGranule(Reference bwData, As state bool hasPrevOwner = prevLockValue.present(); state bool createChangeFeed = false; if (hasPrevOwner) { - TEST(true); // Granule open found previous owner + CODE_PROBE(true, "Granule open found previous owner"); std::tuple prevOwner = decodeBlobGranuleLockValue(prevLockValue.get()); info.granuleID = std::get<2>(prevOwner); @@ -3335,7 +3334,7 @@ ACTOR Future openGranule(Reference bwData, As // if it's the first snapshot of a new granule, history won't be present if (info.history.present()) { if (info.granuleID != info.history.get().value.granuleID) { - TEST(true); // Blob Worker re-opening granule after merge+resplit + CODE_PROBE(true, "Blob Worker re-opening granule after merge+resplit"); // The only case this can happen is when a granule was merged into a larger granule, // then split back out to the same one. Validate that this is a new granule that was // split previously. Just check lock based on epoch, since seqno is intentionally @@ -3412,7 +3411,7 @@ ACTOR Future openGranule(Reference bwData, As // ret.previousChangeFeedId, and the previous durable version will come from the previous // granules if (info.history.present() && info.history.get().value.parentGranules.size() > 0) { - TEST(true); // Granule open found parent + CODE_PROBE(true, "Granule open found parent"); if (info.history.get().value.parentGranules.size() == 1) { // split state Key parentHistoryKey = blobGranuleHistoryKeyFor(info.history.get().value.parentGranules[0].first, @@ -3437,12 +3436,12 @@ ACTOR Future openGranule(Reference bwData, As } if (granuleSplitState.first == BlobGranuleSplitState::Assigned) { - TEST(true); // Granule open found granule in assign state + CODE_PROBE(true, "Granule open found granule in assign state"); // was already assigned, use change feed start version ASSERT(granuleSplitState.second > 0); info.changeFeedStartVersion = granuleSplitState.second; } else if (granuleSplitState.first == BlobGranuleSplitState::Initialized) { - TEST(true); // Granule open found granule in initialized state + CODE_PROBE(true, "Granule open found granule in initialized state"); wait(updateGranuleSplitState(&tr, info.splitParentGranule.get().first, info.splitParentGranule.get().second, @@ -3451,7 +3450,7 @@ ACTOR Future openGranule(Reference bwData, As // change feed was created as part of this transaction, changeFeedStartVersion // will be set later } else { - TEST(true); // Granule open found granule in done state + CODE_PROBE(true, "Granule open found granule in done state"); // this sub-granule is done splitting, no need for split logic. info.splitParentGranule.reset(); } @@ -3470,7 +3469,7 @@ ACTOR Future openGranule(Reference bwData, As : info.blobFilesToSnapshot[0].deltaFiles.back().version; } } else if (info.doSnapshot) { - TEST(true); // merge needs to snapshot at start + CODE_PROBE(true, "merge needs to snapshot at start"); state std::vector> parentGranulesToSnapshot; ASSERT(info.previousDurableVersion == invalidVersion); // need first snapshot to be at history version so this granule can serve the full range @@ -3530,7 +3529,7 @@ ACTOR Future> loadBStoreForTenant(Referencebstore; } else { - TEST(true); // bstore for unknown tenant + CODE_PROBE(true, "bstore for unknown tenant"); // Assume not loaded yet, just wait a bit. Could do sophisticated mechanism but will redo tenant // loading to be versioned anyway. 10 retries means it's likely not a transient race with // loading tenants, and instead a persistent issue. @@ -3909,7 +3908,7 @@ ACTOR Future monitorRemoval(Reference bwData) { Optional val = wait(tr.get(blobWorkerListKey)); if (!val.present()) { - TEST(true); // Blob worker found out BM killed it from reading DB + CODE_PROBE(true, "Blob worker found out BM killed it from reading DB"); return Void(); } diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index f23fd5dce4..fb74b294cf 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -316,7 +316,7 @@ ACTOR Future clusterWatchDatabase(ClusterControllerData* cluster, wait(spinDelay); - TEST(true); // clusterWatchDatabase() master failed + CODE_PROBE(true, "clusterWatchDatabase() master failed"); TraceEvent(SevWarn, "DetectedFailedRecovery", cluster->id).detail("OldMaster", iMaster.id()); } catch (Error& e) { state Error err = e; @@ -328,13 +328,14 @@ ACTOR Future clusterWatchDatabase(ClusterControllerData* cluster, wait(cleanupRecoveryActorCollection(recoveryData, true /* exThrown */)); ASSERT(addActor.isEmpty()); - TEST(err.code() == error_code_tlog_failed); // Terminated due to tLog failure - TEST(err.code() == error_code_commit_proxy_failed); // Terminated due to commit proxy failure - TEST(err.code() == error_code_grv_proxy_failed); // Terminated due to GRV proxy failure - TEST(err.code() == error_code_resolver_failed); // Terminated due to resolver failure - TEST(err.code() == error_code_backup_worker_failed); // Terminated due to backup worker failure - TEST(err.code() == error_code_operation_failed); // Terminated due to failed operation - TEST(err.code() == error_code_restart_cluster_controller); // Terminated due to cluster-controller restart. + CODE_PROBE(err.code() == error_code_tlog_failed, "Terminated due to tLog failure"); + CODE_PROBE(err.code() == error_code_commit_proxy_failed, "Terminated due to commit proxy failure"); + CODE_PROBE(err.code() == error_code_grv_proxy_failed, "Terminated due to GRV proxy failure"); + CODE_PROBE(err.code() == error_code_resolver_failed, "Terminated due to resolver failure"); + CODE_PROBE(err.code() == error_code_backup_worker_failed, "Terminated due to backup worker failure"); + CODE_PROBE(err.code() == error_code_operation_failed, "Terminated due to failed operation"); + CODE_PROBE(err.code() == error_code_restart_cluster_controller, + "Terminated due to cluster-controller restart."); if (cluster->shouldCommitSuicide || err.code() == error_code_coordinators_changed) { TraceEvent("ClusterControllerTerminate", cluster->id).errorUnsuppressed(err); @@ -1244,7 +1245,7 @@ ACTOR Future registerWorker(RegisterWorkerRequest req, } checkOutstandingRequests(self); } else { - TEST(true); // Received an old worker registration request. + CODE_PROBE(true, "Received an old worker registration request."); } // For each singleton diff --git a/fdbserver/ClusterRecovery.actor.cpp b/fdbserver/ClusterRecovery.actor.cpp index 5cc4ce3743..9df53348f9 100644 --- a/fdbserver/ClusterRecovery.actor.cpp +++ b/fdbserver/ClusterRecovery.actor.cpp @@ -58,7 +58,7 @@ ACTOR Future recoveryTerminateOnConflict(UID dbgid, when(wait(onConflict)) { if (!fullyRecovered.isSet()) { TraceEvent("RecoveryTerminated", dbgid).detail("Reason", "Conflict"); - TEST(true); // Coordinated state conflict, recovery terminating + CODE_PROBE(true, "Coordinated state conflict, recovery terminating"); throw worker_removed(); } return Void(); @@ -110,7 +110,7 @@ ACTOR Future recruitNewMaster(ClusterControllerData* cluster, return Void(); } else { - TEST(true); // clusterWatchDatabase() !newMaster.present() + CODE_PROBE(true, "clusterWatchDatabase() !newMaster.present()"); wait(delay(SERVER_KNOBS->MASTER_SPIN_DELAY)); } } @@ -118,7 +118,7 @@ ACTOR Future recruitNewMaster(ClusterControllerData* cluster, ACTOR Future clusterRecruitFromConfiguration(ClusterControllerData* self, Reference req) { // At the moment this doesn't really need to be an actor (it always completes immediately) - TEST(true); // ClusterController RecruitTLogsRequest + CODE_PROBE(true, "ClusterController RecruitTLogsRequest"); loop { try { req->rep = self->findWorkersForConfiguration(req->req); @@ -150,7 +150,7 @@ ACTOR Future clusterRecruitRemoteFromConfig ClusterControllerData* self, Reference req) { // At the moment this doesn't really need to be an actor (it always completes immediately) - TEST(true); // ClusterController RecruitTLogsRequest Remote + CODE_PROBE(true, "ClusterController RecruitTLogsRequest Remote"); loop { try { auto rep = self->findRemoteWorkersForConfiguration(req->req); @@ -355,7 +355,7 @@ ACTOR Future newSeedServers(Reference self, !newServer.isError(error_code_request_maybe_delivered)) throw newServer.getError(); - TEST(true); // initial storage recuitment loop failed to get new server + CODE_PROBE(true, "initial storage recuitment loop failed to get new server"); wait(delay(SERVER_KNOBS->STORAGE_RECRUITMENT_DELAY)); } else { if (!dcId_tags.count(recruits.storageServers[idx].locality.dcId())) { @@ -736,7 +736,7 @@ ACTOR Future updateLogsValue(Reference self, Database } if (!found) { - TEST(true); // old master attempted to change logsKey + CODE_PROBE(true, "old master attempted to change logsKey"); return Void(); } @@ -815,7 +815,7 @@ ACTOR Future updateRegistration(Reference self, Refer std::vector())); } else { // The cluster should enter the accepting commits phase soon, and then we will register again - TEST(true); // cstate is updated but we aren't accepting commits yet + CODE_PROBE(true, "cstate is updated but we aren't accepting commits yet"); } } } @@ -1357,7 +1357,7 @@ ACTOR Future recoverFrom(Reference self, } when(Standalone _req = wait(provisional)) { state Standalone req = _req; // mutable - TEST(true); // Emergency transaction processing during recovery + CODE_PROBE(true, "Emergency transaction processing during recovery"); TraceEvent("EmergencyTransaction", self->dbgid).log(); for (auto m = req.mutations.begin(); m != req.mutations.end(); ++m) TraceEvent("EmergencyTransactionMutation", self->dbgid) @@ -1559,7 +1559,7 @@ ACTOR Future clusterRecoveryCore(Reference self) { .detail("SnapRecoveryFlag", snapRecoveryFlag.present() ? snapRecoveryFlag.get().toString() : "N/A") .detail("LastEpochEnd", self->lastEpochEnd); if (snapRecoveryFlag.present()) { - TEST(true); // Recovering from snapshot, writing to snapShotEndVersionKey + CODE_PROBE(true, "Recovering from snapshot, writing to snapShotEndVersionKey"); BinaryWriter bw(Unversioned()); tr.set(recoveryCommitRequest.arena, snapshotEndVersionKey, (bw << self->lastEpochEnd).toValue()); // Pause the backups that got restored in this snapshot to avoid data corruption @@ -1659,7 +1659,7 @@ ACTOR Future clusterRecoveryCore(Reference self) { // unless we want to change TLogs wait((success(recoveryCommit) && sendInitialCommitToResolvers(self))); if (recoveryCommit.isReady() && recoveryCommit.get().isError()) { - TEST(true); // Cluster recovery failed because of the initial commit failed + CODE_PROBE(true, "Cluster recovery failed because of the initial commit failed"); throw cluster_recovery_failed(); } diff --git a/fdbserver/CommitProxyServer.actor.cpp b/fdbserver/CommitProxyServer.actor.cpp index 8f0af9b200..237f888257 100644 --- a/fdbserver/CommitProxyServer.actor.cpp +++ b/fdbserver/CommitProxyServer.actor.cpp @@ -789,7 +789,7 @@ ACTOR Future preresolutionProcessing(CommitBatchContext* self) { } // Pre-resolution the commits - TEST(pProxyCommitData->latestLocalCommitBatchResolving.get() < localBatchNumber - 1); // Wait for local batch + CODE_PROBE(pProxyCommitData->latestLocalCommitBatchResolving.get() < localBatchNumber - 1, "Wait for local batch"); wait(pProxyCommitData->latestLocalCommitBatchResolving.whenAtLeast(localBatchNumber - 1)); pProxyCommitData->stats.computeLatency.addMeasurement(now() - timeStart); double queuingDelay = g_network->now() - timeStart; @@ -798,7 +798,7 @@ ACTOR Future preresolutionProcessing(CommitBatchContext* self) { (g_network->isSimulated() && BUGGIFY_WITH_PROB(0.01))) && SERVER_KNOBS->PROXY_REJECT_BATCH_QUEUED_TOO_LONG && canReject(trs)) { // Disabled for the recovery transaction. otherwise, recovery can't finish and keeps doing more recoveries. - TEST(true); // Reject transactions in the batch + CODE_PROBE(true, "Reject transactions in the batch"); TraceEvent(SevWarnAlways, "ProxyReject", pProxyCommitData->dbgid) .suppressFor(0.1) .detail("QDelay", queuingDelay) @@ -1152,7 +1152,7 @@ void writeMutation(CommitBatchContext* self, int64_t tenantId, const MutationRef bool isRawAccess = tenantId == TenantInfo::INVALID_TENANT && !isSystemKey(mutation.param1) && !(mutation.type == MutationRef::ClearRange && isSystemKey(mutation.param2)) && self->pProxyCommitData->db->get().client.tenantMode == TenantMode::REQUIRED; - TEST(isRawAccess); // Raw access to tenant key space + CODE_PROBE(isRawAccess, "Raw access to tenant key space"); self->toCommit.writeTypedMessage(mutation); } else { Arena arena; @@ -1259,7 +1259,7 @@ ACTOR Future assignMutationsToStorageServers(CommitBatchContext* self) { trCost->get().clearIdxCosts.pop_front(); } } else { - TEST(true); // A clear range extends past a shard boundary + CODE_PROBE(true, "A clear range extends past a shard boundary"); std::set allSources; for (auto r : ranges) { r.value().populateTags(); @@ -1347,7 +1347,7 @@ ACTOR Future postResolution(CommitBatchContext* self) { state Span span("MP:postResolution"_loc, self->span.context); bool queuedCommits = pProxyCommitData->latestLocalCommitBatchLogging.get() < localBatchNumber - 1; - TEST(queuedCommits); // Queuing post-resolution commit processing + CODE_PROBE(queuedCommits, "Queuing post-resolution commit processing"); wait(pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber - 1)); state double postResolutionQueuing = now(); pProxyCommitData->stats.postResolutionDist->sampleSeconds(postResolutionQueuing - postResolutionStart); @@ -1424,7 +1424,7 @@ ACTOR Future postResolution(CommitBatchContext* self) { self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { // This should be *extremely* rare in the real world, but knob buggification should make it happen in // simulation - TEST(true); // Semi-committed pipeline limited by MVCC window + CODE_PROBE(true, "Semi-committed pipeline limited by MVCC window"); //TraceEvent("ProxyWaitingForCommitted", pProxyCommitData->dbgid).detail("CommittedVersion", pProxyCommitData->committedVersion.get()).detail("NeedToCommit", commitVersion); waitVersionSpan = Span("MP:overMaxReadTransactionLifeVersions"_loc, span.context); choose { @@ -1617,7 +1617,8 @@ ACTOR Future reply(CommitBatchContext* self) { // client may get a commit version that the master is not aware of, and next GRV request may get a version less than // self->committedVersion. - TEST(pProxyCommitData->committedVersion.get() > self->commitVersion); // later version was reported committed first + CODE_PROBE(pProxyCommitData->committedVersion.get() > self->commitVersion, + "later version was reported committed first"); if (self->commitVersion >= pProxyCommitData->committedVersion.get()) { state Optional> writtenTags; @@ -2603,7 +2604,7 @@ ACTOR Future commitProxyServer(CommitProxyInterface proxy, e.code() != error_code_failed_to_progress) { throw; } - TEST(e.code() == error_code_failed_to_progress); // Commit proxy failed to progress + CODE_PROBE(e.code() == error_code_failed_to_progress, "Commit proxy failed to progress"); } return Void(); } diff --git a/fdbserver/ConfigNode.actor.cpp b/fdbserver/ConfigNode.actor.cpp index c76ee882cf..25c4cbc625 100644 --- a/fdbserver/ConfigNode.actor.cpp +++ b/fdbserver/ConfigNode.actor.cpp @@ -206,7 +206,7 @@ class ConfigNodeImpl { // Handle a very rare case where a ConfigNode loses data between // responding with a committed version and responding to the // subsequent get changes request. - TEST(true); // ConfigNode data loss occurred on a minority of coordinators + CODE_PROBE(true, "ConfigNode data loss occurred on a minority of coordinators"); req.reply.sendError(process_behind()); // Reuse the process_behind error return Void(); } @@ -230,7 +230,8 @@ class ConfigNodeImpl { state ConfigGeneration generation = wait(getGeneration(self)); ++generation.liveVersion; if (req.lastSeenLiveVersion.present()) { - TEST(req.lastSeenLiveVersion.get() >= generation.liveVersion); // Node is lagging behind some other node + CODE_PROBE(req.lastSeenLiveVersion.get() >= generation.liveVersion, + "Node is lagging behind some other node"); generation.liveVersion = std::max(generation.liveVersion, req.lastSeenLiveVersion.get() + 1); } self->kvStore->set(KeyValueRef(currentGenerationKey, BinaryWriter::toValue(generation, IncludeVersion()))); diff --git a/fdbserver/CoordinatedState.actor.cpp b/fdbserver/CoordinatedState.actor.cpp index 6101390d66..48efc195b4 100644 --- a/fdbserver/CoordinatedState.actor.cpp +++ b/fdbserver/CoordinatedState.actor.cpp @@ -273,7 +273,7 @@ struct MovableCoordinatedStateImpl { // SOMEDAY: If moveState.mode == MovingFrom, read (without locking) old state and assert that it corresponds // with our state and is ReallyTo(coordinators) if (moveState.mode == MovableValue::MaybeTo) { - TEST(true); // Maybe moveto state + CODE_PROBE(true, "Maybe moveto state"); ASSERT(moveState.other.present()); wait(self->moveTo( self, &self->cs, ClusterConnectionString(moveState.other.get().toString()), moveState.value)); @@ -322,7 +322,7 @@ struct MovableCoordinatedStateImpl { Value oldQuorumState = wait(cs.read()); if (oldQuorumState != self->lastCSValue.get()) { - TEST(true); // Quorum change aborted by concurrent write to old coordination state + CODE_PROBE(true, "Quorum change aborted by concurrent write to old coordination state"); TraceEvent("QuorumChangeAbortedByConcurrency").log(); throw coordinated_state_conflict(); } diff --git a/fdbserver/DDTeamCollection.actor.cpp b/fdbserver/DDTeamCollection.actor.cpp index b121056c35..303d13bd02 100644 --- a/fdbserver/DDTeamCollection.actor.cpp +++ b/fdbserver/DDTeamCollection.actor.cpp @@ -942,8 +942,9 @@ public: : SERVER_KNOBS->PRIORITY_TEAM_UNHEALTHY); } } else { - TEST(true); // A removed server is still associated with a team in - // ShardsAffectedByTeamFailure + CODE_PROBE(true, + "A removed server is still associated with a team in " + "ShardsAffectedByTeamFailure"); } } } @@ -1253,7 +1254,7 @@ public: server->updateLastKnown(newInterface.first, newInterface.second); if (localityChanged && !isTss) { - TEST(true); // Server locality changed + CODE_PROBE(true, "Server locality changed"); // The locality change of a server will affect machine teams related to the server if // the server's machine locality is changed @@ -1320,7 +1321,7 @@ public: } } if (addedNewBadTeam && self->badTeamRemover.isReady()) { - TEST(true); // Server locality change created bad teams + CODE_PROBE(true, "Server locality change created bad teams"); self->doBuildTeams = true; self->badTeamRemover = removeBadTeams(self); self->addActor.send(self->badTeamRemover); @@ -1724,7 +1725,7 @@ public: // in the serverTeams vector in the machine team. --teamIndex; self->addTeam(team->getServers(), IsInitialTeam::True, IsRedundantTeam::True); - TEST(true); // Removed machine team + CODE_PROBE(true, "Removed machine team"); } self->doBuildTeams = true; @@ -1808,7 +1809,7 @@ public: bool foundTeam = self->removeTeam(st); ASSERT(foundTeam); self->addTeam(st->getServers(), IsInitialTeam::True, IsRedundantTeam::True); - TEST(true); // Marked team as a bad team + CODE_PROBE(true, "Marked team as a bad team"); self->doBuildTeams = true; @@ -2052,7 +2053,7 @@ public: if (self->wigglingId.present()) { state UID id = self->wigglingId.get(); if (self->pauseWiggle->get()) { - TEST(true); // paused because cluster is unhealthy + CODE_PROBE(true, "paused because cluster is unhealthy"); moveFinishFuture = Never(); self->includeStorageServersForWiggle(); self->storageWiggler->setWiggleState(StorageWiggler::PAUSE); @@ -2068,7 +2069,7 @@ public: } else { choose { when(wait(self->waitUntilHealthy())) { - TEST(true); // start wiggling + CODE_PROBE(true, "start wiggling"); wait(self->storageWiggler->startWiggle()); auto fv = self->excludeStorageServersForWiggle(id); moveFinishFuture = fv; @@ -2431,10 +2432,10 @@ public: // SS and/or TSS recruitment failed at this point, update tssState if (recruitTss && tssState->tssRecruitFailed()) { tssState->markComplete(); - TEST(true); // TSS recruitment failed for some reason + CODE_PROBE(true, "TSS recruitment failed for some reason"); } if (!recruitTss && tssState->ssRecruitFailed()) { - TEST(true); // SS with pair TSS recruitment failed for some reason + CODE_PROBE(true, "SS with pair TSS recruitment failed for some reason"); } self->recruitingStream.set(self->recruitingStream.get() - 1); @@ -2575,7 +2576,7 @@ public: .detail("Addr", candidateSSAddr.toString()) .detail("Locality", candidateWorker.worker.locality.toString()); - TEST(true); // Starting TSS recruitment + CODE_PROBE(true, "Starting TSS recruitment"); self->isTssRecruiting = true; tssState = makeReference(candidateWorker.worker.locality); @@ -2585,7 +2586,7 @@ public: checkTss = self->initialFailureReactionDelay; } else { if (tssState->active && tssState->inDataZone(candidateWorker.worker.locality)) { - TEST(true); // TSS recruits pair in same dc/datahall + CODE_PROBE(true, "TSS recruits pair in same dc/datahall"); self->isTssRecruiting = false; TraceEvent("TSS_Recruit", self->distributorId) .detail("Stage", "PairSS") @@ -2596,8 +2597,9 @@ public: // successfully started recruitment of pair, reset tss recruitment state tssState = makeReference(); } else { - TEST(tssState->active); // TSS recruitment skipped potential pair because it's in a - // different dc/datahall + CODE_PROBE( + tssState->active, + "TSS recruitment skipped potential pair because it's in a different dc/datahall"); self->addActor.send(initializeStorage( self, candidateWorker, ddEnabledState, false, makeReference())); } @@ -2617,8 +2619,9 @@ public: int tssToKill = std::min((int)self->tss_info_by_pair.size(), std::max(-tssToRecruit, self->zeroHealthyTeams->get() ? 1 : 0)); if (cancelTss) { - TEST(tssToRecruit < 0); // tss recruitment cancelled due to too many TSS - TEST(self->zeroHealthyTeams->get()); // tss recruitment cancelled due zero healthy teams + CODE_PROBE(tssToRecruit < 0, "tss recruitment cancelled due to too many TSS"); + CODE_PROBE(self->zeroHealthyTeams->get(), + "tss recruitment cancelled due zero healthy teams"); TraceEvent(SevWarn, "TSS_RecruitCancelled", self->distributorId) .detail("Reason", tssToRecruit <= 0 ? "TooMany" : "ZeroHealthyTeams"); @@ -2637,8 +2640,8 @@ public: if (self->shouldHandleServer(tssi) && self->server_and_tss_info.count(tssId)) { Promise killPromise = itr->second->killTss; if (killPromise.canBeSet()) { - TEST(tssToRecruit < 0); // Killing TSS due to too many TSS - TEST(self->zeroHealthyTeams->get()); // Killing TSS due zero healthy teams + CODE_PROBE(tssToRecruit < 0, "Killing TSS due to too many TSS"); + CODE_PROBE(self->zeroHealthyTeams->get(), "Killing TSS due zero healthy teams"); TraceEvent(SevWarn, "TSS_DDKill", self->distributorId) .detail("TSSID", tssId) .detail("Reason", @@ -2672,7 +2675,7 @@ public: if (e.code() != error_code_timed_out) { throw; } - TEST(true); // Storage recruitment timed out + CODE_PROBE(true, "Storage recruitment timed out"); } } } @@ -2992,14 +2995,14 @@ public: loop choose { when(UID removedServer = waitNext(self->removedServers.getFuture())) { - TEST(true); // Storage server removed from database + CODE_PROBE(true, "Storage server removed from database"); self->removeServer(removedServer); serverRemoved.send(Void()); self->restartRecruiting.trigger(); } when(UID removedTSS = waitNext(self->removedTSS.getFuture())) { - TEST(true); // TSS removed from database + CODE_PROBE(true, "TSS removed from database"); self->removeTSS(removedTSS); serverRemoved.send(Void()); @@ -4808,7 +4811,7 @@ Reference DDTeamCollection::checkAndCreateMachine(Reference machineInfo; if (machine_info.find(machine_id) == machine_info.end()) { // uid is the first storage server process on the machine - TEST(true); // First storage server in process on the machine + CODE_PROBE(true, "First storage server in process on the machine"); // For each machine, store the first server's localityEntry into machineInfo for later use. LocalityEntry localityEntry = machineLocalityMap.add(locality, &server->getId()); machineInfo = makeReference(server, localityEntry); diff --git a/fdbserver/DDTxnProcessor.actor.cpp b/fdbserver/DDTxnProcessor.actor.cpp index 2d3279163c..a9be37ea86 100644 --- a/fdbserver/DDTxnProcessor.actor.cpp +++ b/fdbserver/DDTxnProcessor.actor.cpp @@ -250,7 +250,7 @@ class DDTxnProcessorImpl { // If keyServers is too large to read in a single transaction, then we will have to break this process up into // multiple transactions. In that case, each iteration should begin where the previous left off while (beginKey < allKeys.end) { - TEST(beginKey > allKeys.begin); // Multi-transactional getInitialDataDistribution + CODE_PROBE(beginKey > allKeys.begin, "Multi-transactional getInitialDataDistribution"); loop { succeeded = false; try { @@ -430,4 +430,4 @@ Future> DDTxnProcessor::getInitialDataDistrib Future DDTxnProcessor::waitForDataDistributionEnabled(const DDEnabledState* ddEnabledState) const { return DDTxnProcessorImpl::waitForDataDistributionEnabled(cx, ddEnabledState); -} \ No newline at end of file +} diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index e4682f44bd..0acdba7a00 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -881,7 +881,7 @@ ACTOR Future>> configuration.storageTeamSize - 1) - storageFailures; if (*storageFaultTolerance < 0) { - TEST(true); // Too many failed storage servers to complete snapshot + CODE_PROBE(true, "Too many failed storage servers to complete snapshot"); throw snap_storage_failed(); } // tlogs @@ -1327,14 +1327,14 @@ ACTOR Future dataDistributor(DataDistributorInterface di, Reference dataDistributionRelocator(DDQueueData* self, } if (anyDestOverloaded) { - TEST(true); // Destination overloaded throttled move + CODE_PROBE(true, "Destination overloaded throttled move"); destOverloadedCount++; TraceEvent(destOverloadedCount > 50 ? SevInfo : SevDebug, "DestSSBusy", distributorId) .suppressFor(1.0) @@ -1441,7 +1441,7 @@ ACTOR Future dataDistributionRelocator(DDQueueData* self, .detail("Servers", destServersString(bestTeams)); wait(delay(SERVER_KNOBS->DEST_OVERLOADED_DELAY, TaskPriority::DataDistributionLaunch)); } else { - TEST(true); // did not find a healthy destination team on the first attempt + CODE_PROBE(true, "did not find a healthy destination team on the first attempt"); stuckCount++; TraceEvent(stuckCount > 50 ? SevWarnAlways : SevWarn, "BestTeamStuck", distributorId) .suppressFor(1.0) @@ -1674,7 +1674,7 @@ ACTOR Future dataDistributionRelocator(DDQueueData* self, throw error; } } else { - TEST(true); // move to removed server + CODE_PROBE(true, "move to removed server"); healthyDestinations.addDataInFlightToTeam(-metrics.bytes); auto readLoad = metrics.bytesReadPerKSecond; auto& destinationRef = healthyDestinations; diff --git a/fdbserver/DataDistributionTracker.actor.cpp b/fdbserver/DataDistributionTracker.actor.cpp index 3976c76674..6b504a6f8a 100644 --- a/fdbserver/DataDistributionTracker.actor.cpp +++ b/fdbserver/DataDistributionTracker.actor.cpp @@ -110,7 +110,7 @@ struct DataDistributionTracker { DataDistributionTracker* operator()() { if (trackerCancelled) { - TEST(true); // Trying to access DataDistributionTracker after tracker has been cancelled + CODE_PROBE(true, "Trying to access DataDistributionTracker after tracker has been cancelled"); throw dd_tracker_cancelled(); } return &tracker; @@ -482,7 +482,7 @@ ACTOR Future shardSplitter(DataDistributionTracker* self, state BandwidthStatus bandwidthStatus = getBandwidthStatus(metrics); // Split - TEST(true); // shard to be split + CODE_PROBE(true, "shard to be split"); StorageMetrics splitMetrics; splitMetrics.bytes = shardBounds.max.bytes / 2; @@ -559,7 +559,7 @@ Future shardMerger(DataDistributionTracker* self, auto prevIter = self->shards.rangeContaining(keys.begin); auto nextIter = self->shards.rangeContaining(keys.begin); - TEST(true); // shard to be merged + CODE_PROBE(true, "shard to be merged"); ASSERT(keys.begin > allKeys.begin); // This will merge shards both before and after "this" shard in keyspace. @@ -604,7 +604,7 @@ Future shardMerger(DataDistributionTracker* self, // on the previous shard changing "size". if (!newMetrics.present() || shardCount + newMetrics.get().shardCount >= CLIENT_KNOBS->SHARD_COUNT_LIMIT) { if (shardsMerged == 1) { - TEST(true); // shardMerger cannot merge anything + CODE_PROBE(true, "shardMerger cannot merge anything"); return brokenPromiseToReady(prevIter->value().stats->onChange()); } @@ -797,7 +797,7 @@ void restartShardTrackers(DataDistributionTracker* self, KeyRangeRef keys, Optio .detail("Keys", keys) .detail("Size", startingMetrics.get().metrics.bytes) .detail("Merges", startingMetrics.get().merges);*/ - TEST(true); // shardTracker started with trackedBytes already set + CODE_PROBE(true, "shardTracker started with trackedBytes already set"); shardMetrics->set(startingMetrics); } @@ -903,7 +903,7 @@ ACTOR Future fetchTopKShardMetrics(DataDistributionTracker* self, GetTopKM choose { when(wait(fetchTopKShardMetrics_impl(self, req))) {} when(wait(delay(SERVER_KNOBS->DD_SHARD_METRICS_TIMEOUT))) { - TEST(true); // TopK DD_SHARD_METRICS_TIMEOUT + CODE_PROBE(true, "TopK DD_SHARD_METRICS_TIMEOUT"); req.reply.send(GetTopKMetricsReply()); } } @@ -942,7 +942,7 @@ ACTOR Future fetchShardMetrics(DataDistributionTracker* self, GetMetricsRe choose { when(wait(fetchShardMetrics_impl(self, req))) {} when(wait(delay(SERVER_KNOBS->DD_SHARD_METRICS_TIMEOUT, TaskPriority::DataDistribution))) { - TEST(true); // DD_SHARD_METRICS_TIMEOUT + CODE_PROBE(true, "DD_SHARD_METRICS_TIMEOUT"); StorageMetrics largeMetrics; largeMetrics.bytes = getMaxShardSize(self->dbSizeEstimate->get()); req.reply.send(largeMetrics); diff --git a/fdbserver/DiskQueue.actor.cpp b/fdbserver/DiskQueue.actor.cpp index 0b96f30f67..4ca2f6ca0e 100644 --- a/fdbserver/DiskQueue.actor.cpp +++ b/fdbserver/DiskQueue.actor.cpp @@ -379,7 +379,7 @@ public: pageFloor(std::max(self->files[1].size - desiredMaxFileSize, self->fileShrinkBytes)); if ((maxShrink > SERVER_KNOBS->DISK_QUEUE_MAX_TRUNCATE_BYTES) || (frivolouslyTruncate && deterministicRandom()->random01() < 0.3)) { - TEST(true); // Replacing DiskQueue file + CODE_PROBE(true, "Replacing DiskQueue file"); TraceEvent("DiskQueueReplaceFile", self->dbgid) .detail("Filename", self->files[1].f->getFilename()) .detail("OldFileSize", self->files[1].size) @@ -389,7 +389,7 @@ public: waitfor.push_back(self->files[1].f->truncate(self->fileExtensionBytes)); self->files[1].size = self->fileExtensionBytes; } else { - TEST(true); // Truncating DiskQueue file + CODE_PROBE(true, "Truncating DiskQueue file"); const int64_t startingSize = self->files[1].size; self->files[1].size -= std::min(maxShrink, self->files[1].size); self->files[1].size = std::max(self->files[1].size, self->fileExtensionBytes); @@ -460,12 +460,12 @@ public: wait(ready); - TEST(pageData.size() > sizeof(Page)); // push more than one page of data + CODE_PROBE(pageData.size() > sizeof(Page), "push more than one page of data"); Future pushed = wait(self->push(pageData, &syncFiles)); pushing.send(Void()); ASSERT(syncFiles.size() >= 1 && syncFiles.size() <= 2); - TEST(2 == syncFiles.size()); // push spans both files + CODE_PROBE(2 == syncFiles.size(), "push spans both files"); wait(pushed); delete pageMem; @@ -491,8 +491,8 @@ public: committed.send(Void()); } catch (Error& e) { delete pageMem; - TEST(true); // push error - TEST(2 == syncFiles.size()); // push spanning both files error + CODE_PROBE(true, "push error"); + CODE_PROBE(2 == syncFiles.size(), "push spanning both files error"); TraceEvent(SevError, "RDQPushAndCommitError", dbgid) .errorUnsuppressed(e) .detail("InitialFilename0", filename); @@ -805,7 +805,7 @@ public: Standalone result = self->readingBuffer.pop_front(sizeof(Page)); return result; } catch (Error& e) { - TEST(true); // Read next page error + CODE_PROBE(true, "Read next page error"); TraceEvent(SevError, "RDQReadNextPageError", self->dbgid) .errorUnsuppressed(e) .detail("File0Name", self->files[0].dbgFilename); @@ -840,8 +840,8 @@ public: state std::vector> commits; state bool swap = file == 0; - TEST(file == 0); // truncate before last read page on file 0 - TEST(file == 1 && pos != self->files[1].size); // truncate before last read page on file 1 + CODE_PROBE(file == 0, "truncate before last read page on file 0"); + CODE_PROBE(file == 1 && pos != self->files[1].size, "truncate before last read page on file 1"); self->readingFile = 2; self->readingBuffer.clear(); @@ -890,10 +890,10 @@ public: ASSERT(recovered); uint8_t const* begin = contents.begin(); uint8_t const* end = contents.end(); - TEST(contents.size() && pushedPageCount()); // More than one push between commits + CODE_PROBE(contents.size() && pushedPageCount(), "More than one push between commits"); bool pushAtEndOfPage = contents.size() >= 4 && pushedPageCount() && backPage().remainingCapacity() < 4; - TEST(pushAtEndOfPage); // Push right at the end of a page, possibly splitting size + CODE_PROBE(pushAtEndOfPage, "Push right at the end of a page, possibly splitting size"); while (begin != end) { if (!pushedPageCount() || !backPage().remainingCapacity()) addEmptyPage(); @@ -1391,7 +1391,7 @@ private: int f; int64_t p; bool poppedNotDurable = self->lastPoppedSeq / sizeof(Page) != self->poppedSeq / sizeof(Page); - TEST(poppedNotDurable); // DiskQueue: Recovery popped position not fully durable + CODE_PROBE(poppedNotDurable, "DiskQueue: Recovery popped position not fully durable"); self->findPhysicalLocation(self->lastPoppedSeq, &f, &p, "lastPoppedSeq"); wait(self->rawQueue->setPoppedPage(f, p, pageFloor(self->lastPoppedSeq))); @@ -1408,8 +1408,8 @@ private: self->recovered = true; ASSERT(self->poppedSeq <= self->endLocation()); - TEST(result.size() == 0); // End of queue at border between reads - TEST(result.size() != 0); // Partial read at end of queue + CODE_PROBE(result.size() == 0, "End of queue at border between reads"); + CODE_PROBE(result.size() != 0, "Partial read at end of queue"); // The next read location isn't necessarily the end of the last commit, but this is sufficient for helping us // check an ASSERTion @@ -1628,8 +1628,9 @@ public: // totally finished pop(popLocation); commitFuture = commitFuture && queue->commit(); - } else - TEST(true); // No uncommitted data was popped + } else { + CODE_PROBE(true, "No uncommitted data was popped"); + } return commitFuture; } diff --git a/fdbserver/GlobalTagThrottler.actor.cpp b/fdbserver/GlobalTagThrottler.actor.cpp index 5320f3671c..a93edcf176 100644 --- a/fdbserver/GlobalTagThrottler.actor.cpp +++ b/fdbserver/GlobalTagThrottler.actor.cpp @@ -166,7 +166,7 @@ class GlobalTagThrottlerImpl { // wait(tr.watch(tagThrottleSignalKey)); wait(delay(5.0)); TraceEvent("GlobalTagThrottler_ChangeSignaled"); - TEST(true); // Global tag throttler detected quota changes + CODE_PROBE(true, "Global tag throttler detected quota changes"); break; } catch (Error& e) { TraceEvent("GlobalTagThrottlerMonitoringChangesError", self->id).error(e); diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 30ad98bcf1..58a89d0431 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -661,14 +661,14 @@ ACTOR Future sendGrvReplies(Future replyFuture, if (tagItr != priorityThrottledTags.end()) { if (tagItr->second.expiration > now()) { if (tagItr->second.tpsRate == std::numeric_limits::max()) { - TEST(true); // Auto TPS rate is unlimited + CODE_PROBE(true, "Auto TPS rate is unlimited"); } else { - TEST(true); // GRV proxy returning tag throttle + CODE_PROBE(true, "GRV proxy returning tag throttle"); reply.tagThrottleInfo[tag.first] = tagItr->second; } } else { // This isn't required, but we might as well - TEST(true); // GRV proxy expiring tag throttle + CODE_PROBE(true, "GRV proxy expiring tag throttle"); priorityThrottledTags.erase(tagItr); } } diff --git a/fdbserver/KeyValueStoreMemory.actor.cpp b/fdbserver/KeyValueStoreMemory.actor.cpp index 0e31d3f85d..e5277c6862 100644 --- a/fdbserver/KeyValueStoreMemory.actor.cpp +++ b/fdbserver/KeyValueStoreMemory.actor.cpp @@ -104,9 +104,9 @@ public: TraceEvent("KVSMemSwitchingToLargeTransactionMode", id) .detail("TransactionSize", transactionSize) .detail("DataSize", committedDataSize); - TEST(true); // KeyValueStoreMemory switching to large transaction mode - TEST(committedDataSize > - 1e3); // KeyValueStoreMemory switching to large transaction mode with committed data + CODE_PROBE(true, "KeyValueStoreMemory switching to large transaction mode"); + CODE_PROBE(committedDataSize > 1e3, + "KeyValueStoreMemory switching to large transaction mode with committed data"); } int64_t bytesWritten = commit_queue(queue, true); @@ -574,7 +574,7 @@ private: Standalone data = wait(self->log->readNext(sizeof(OpHeader))); if (data.size() != sizeof(OpHeader)) { if (data.size()) { - TEST(true); // zero fill partial header in KeyValueStoreMemory + CODE_PROBE(true, "zero fill partial header in KeyValueStoreMemory"); memset(&h, 0, sizeof(OpHeader)); memcpy(&h, data.begin(), data.size()); zeroFillSize = sizeof(OpHeader) - data.size() + h.len1 + h.len2 + 1; @@ -705,7 +705,7 @@ private: ASSERT(false); } - TEST(true); // Fixing a partial commit at the end of the KeyValueStoreMemory log + CODE_PROBE(true, "Fixing a partial commit at the end of the KeyValueStoreMemory log"); for (int i = 0; i < zeroFillSize; i++) self->log->push(StringRef((const uint8_t*)"", 1)); } diff --git a/fdbserver/KeyValueStoreSQLite.actor.cpp b/fdbserver/KeyValueStoreSQLite.actor.cpp index f65ce27199..0df0d5eb15 100644 --- a/fdbserver/KeyValueStoreSQLite.actor.cpp +++ b/fdbserver/KeyValueStoreSQLite.actor.cpp @@ -117,7 +117,7 @@ struct PageChecksumCodec { crc32Sum.part1 = 0; crc32Sum.part2 = crc32c_append(0xfdbeefdb, static_cast(data), dataLen); if (crc32Sum == *pSumInPage) { - TEST(true); // Read CRC32 checksum + CODE_PROBE(true, "Read CRC32 checksum"); return true; } } @@ -133,7 +133,7 @@ struct PageChecksumCodec { xxHash3Sum.part1 = static_cast((xxHash3 >> 32) & 0x00ffffff); xxHash3Sum.part2 = static_cast(xxHash3 & 0xffffffff); if (xxHash3Sum == *pSumInPage) { - TEST(true); // Read xxHash3 checksum + CODE_PROBE(true, "Read xxHash3 checksum"); return true; } } @@ -144,7 +144,7 @@ struct PageChecksumCodec { hashLittle2Sum.part2 = 0x5ca1ab1e; hashlittle2(pData, dataLen, &hashLittle2Sum.part1, &hashLittle2Sum.part2); if (hashLittle2Sum == *pSumInPage) { - TEST(true); // Read HashLittle2 checksum + CODE_PROBE(true, "Read HashLittle2 checksum"); return true; } @@ -357,7 +357,7 @@ struct SQLiteDB : NonCopyable { lineStart = lineEnd; } } - TEST(true); // BTree integrity checked + CODE_PROBE(true, "BTree integrity checked"); } if (e) sqlite3_free(e); @@ -1423,7 +1423,7 @@ void SQLiteDB::open(bool writable) { renameFile(walpath, walpath + "-old-" + deterministicRandom()->randomUniqueID().toString()); ASSERT_WE_THINK(false); //< This code should not be hit in FoundationDB at the moment, because worker looks // for databases to open by listing .fdb files, not .fdb-wal files - // TEST(true); // Replace a partially constructed or destructed DB + // CODE_PROBE(true, "Replace a partially constructed or destructed DB"); } if (dbFile.isError() && walFile.isError() && writable && @@ -1942,8 +1942,8 @@ private: } if (canDelete && (!canVacuum || deterministicRandom()->random01() < lazyDeleteBatchProbability)) { - TEST(canVacuum); // SQLite lazy deletion when vacuuming is active - TEST(!canVacuum); // SQLite lazy deletion when vacuuming is inactive + CODE_PROBE(canVacuum, "SQLite lazy deletion when vacuuming is active"); + CODE_PROBE(!canVacuum, "SQLite lazy deletion when vacuuming is inactive"); int pagesToDelete = std::max( 1, @@ -1955,10 +1955,10 @@ private: lazyDeleteTime += now() - begin; } else { ASSERT(canVacuum); - TEST(canDelete); // SQLite vacuuming when lazy delete is active - TEST(!canDelete); // SQLite vacuuming when lazy delete is inactive - TEST(SERVER_KNOBS->SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE != - 0); // SQLite vacuuming with nonzero vacuums_per_lazy_delete_page + CODE_PROBE(canDelete, "SQLite vacuuming when lazy delete is active"); + CODE_PROBE(!canDelete, "SQLite vacuuming when lazy delete is inactive"); + CODE_PROBE(SERVER_KNOBS->SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE != 0, + "SQLite vacuuming with nonzero vacuums_per_lazy_delete_page"); vacuumFinished = conn.vacuum(); if (!vacuumFinished) { @@ -1973,10 +1973,10 @@ private: freeListPages = conn.freePages(); - TEST(workPerformed.lazyDeletePages > 0); // Pages lazily deleted - TEST(workPerformed.vacuumedPages > 0); // Pages vacuumed - TEST(vacuumTime > 0); // Time spent vacuuming - TEST(lazyDeleteTime > 0); // Time spent lazy deleting + CODE_PROBE(workPerformed.lazyDeletePages > 0, "Pages lazily deleted"); + CODE_PROBE(workPerformed.vacuumedPages > 0, "Pages vacuumed"); + CODE_PROBE(vacuumTime > 0, "Time spent vacuuming"); + CODE_PROBE(lazyDeleteTime > 0, "Time spent lazy deleting"); ++springCleaningStats.springCleaningCount; springCleaningStats.lazyDeletePages += workPerformed.lazyDeletePages; diff --git a/fdbserver/LeaderElection.actor.cpp b/fdbserver/LeaderElection.actor.cpp index d7eedd561f..40c093093e 100644 --- a/fdbserver/LeaderElection.actor.cpp +++ b/fdbserver/LeaderElection.actor.cpp @@ -206,7 +206,7 @@ ACTOR Future tryBecomeLeaderInternal(ServerCoordinators coordinators, choose { when(wait(nomineeChange.onTrigger())) {} when(wait(badCandidateTimeout.isValid() ? badCandidateTimeout : Never())) { - TEST(true); // Bad candidate timeout + CODE_PROBE(true, "Bad candidate timeout"); TraceEvent("LeaderBadCandidateTimeout", myInfo.changeID).log(); break; } diff --git a/fdbserver/LocalConfiguration.actor.cpp b/fdbserver/LocalConfiguration.actor.cpp index fbc3380f9e..b918fb2235 100644 --- a/fdbserver/LocalConfiguration.actor.cpp +++ b/fdbserver/LocalConfiguration.actor.cpp @@ -65,7 +65,7 @@ class LocalConfigurationImpl { configClassToKnobToValue[configPath.back()] = {}; } } else { - TEST(true); // Invalid configuration path + CODE_PROBE(true, "Invalid configuration path"); if (!g_network->isSimulated()) { fprintf(stderr, "WARNING: Invalid configuration path: `%s'\n", paramString.c_str()); } @@ -88,7 +88,7 @@ class LocalConfigurationImpl { knobCollection.setKnob(knobName.toString(), knobValue); } catch (Error& e) { if (e.code() == error_code_invalid_option_value) { - TEST(true); // invalid knob in configuration database + CODE_PROBE(true, "invalid knob in configuration database"); TraceEvent(SevWarnAlways, "InvalidKnobOptionValue") .detail("KnobName", knobName) .detail("KnobValue", knobValue.toString()); @@ -126,10 +126,10 @@ class LocalConfigurationImpl { this->overrides[stringToKeyRef(knobName)] = knobValue; } catch (Error& e) { if (e.code() == error_code_invalid_option) { - TEST(true); // Attempted to manually set invalid knob option + CODE_PROBE(true, "Attempted to manually set invalid knob option"); TraceEvent(SevWarnAlways, "UnrecognizedKnobOption").detail("Knob", printable(knobName)); } else if (e.code() == error_code_invalid_option_value) { - TEST(true); // Invalid manually set knob value + CODE_PROBE(true, "Invalid manually set knob value"); TraceEvent(SevWarnAlways, "InvalidKnobValue") .detail("Knob", printable(knobName)) .detail("Value", printable(knobValueString)); @@ -198,7 +198,7 @@ class LocalConfigurationImpl { state ConfigKnobOverrides storedConfigPath = BinaryReader::fromStringRef(storedConfigPathValue.get(), IncludeVersion()); if (!storedConfigPath.hasSameConfigPath(self->configKnobOverrides)) { - TEST(true); // All local information is outdated + CODE_PROBE(true, "All local information is outdated"); wait(clearKVStore(self)); wait(saveConfigPath(self)); self->updateInMemoryState(lastSeenVersion); diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index aa3e159be5..649fc429c9 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -592,7 +592,7 @@ Future logRouterPeekMessages(PromiseType replyPromise, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version + CODE_PROBE(true, "tlog peek second attempt ended at a different version"); replyPromise.sendError(operation_obsolete()); return Void(); } diff --git a/fdbserver/LogSystem.cpp b/fdbserver/LogSystem.cpp index 6444b23234..81ca7300f1 100644 --- a/fdbserver/LogSystem.cpp +++ b/fdbserver/LogSystem.cpp @@ -290,7 +290,7 @@ void LogPushData::addTxsTag() { } void LogPushData::addTransactionInfo(SpanContext const& context) { - TEST(!spanContext.isValid()); // addTransactionInfo with invalid SpanContext + CODE_PROBE(!spanContext.isValid(), "addTransactionInfo with invalid SpanContext"); spanContext = context; writtenLocations.clear(); } @@ -352,7 +352,7 @@ bool LogPushData::writeTransactionInfo(int location, uint32_t subseq) { return false; } - TEST(true); // Wrote SpanContextMessage to a transaction log + CODE_PROBE(true, "Wrote SpanContextMessage to a transaction log"); writtenLocations.insert(location); BinaryWriter& wr = messagesWriter[location]; @@ -375,10 +375,10 @@ bool LogPushData::writeTransactionInfo(int location, uint32_t subseq) { // parent->child. SpanContextMessage contextMessage; if (spanContext.isSampled()) { - TEST(true); // Converting OTELSpanContextMessage to traced SpanContextMessage + CODE_PROBE(true, "Converting OTELSpanContextMessage to traced SpanContextMessage"); contextMessage = SpanContextMessage(UID(spanContext.traceID.first(), spanContext.traceID.second())); } else { - TEST(true); // Converting OTELSpanContextMessage to untraced SpanContextMessage + CODE_PROBE(true, "Converting OTELSpanContextMessage to untraced SpanContextMessage"); contextMessage = SpanContextMessage(UID(0, 0)); } wr << contextMessage; diff --git a/fdbserver/LogSystemDiskQueueAdapter.actor.cpp b/fdbserver/LogSystemDiskQueueAdapter.actor.cpp index 029e551e7a..12e64a5f1e 100644 --- a/fdbserver/LogSystemDiskQueueAdapter.actor.cpp +++ b/fdbserver/LogSystemDiskQueueAdapter.actor.cpp @@ -101,7 +101,7 @@ public: TraceEvent(SevWarnAlways, "DiskQueueAdapterReset") .detail("Version", self->cursor->popped()) .detail("PeekTypeSwitch", self->peekTypeSwitches % 3); - TEST(true); // disk adapter reset + CODE_PROBE(true, "disk adapter reset"); if (self->cursor->popped() != 0) { self->recoveryLoc = self->cursor->popped(); } else { diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index a7bf08bdee..7d607e3044 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -317,7 +317,7 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, // // A cursor for a log router can be delayed indefinitely during a network partition, so only fail // simulation tests sufficiently far after we finish simulating network partitions. - TEST(e.code() == error_code_timed_out); // peek cursor timed out + CODE_PROBE(e.code() == error_code_timed_out, "peek cursor timed out"); if (now() >= FLOW_KNOBS->SIM_SPEEDUP_AFTER_SECONDS + SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME) { ASSERT_WE_THINK(e.code() == error_code_operation_obsolete || SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME < 10); @@ -653,7 +653,7 @@ void ILogSystem::MergedPeekCursor::updateMessage(bool usePolicy) { c->advanceTo(messageVersion); if (start <= messageVersion && messageVersion < c->version()) { advancedPast = true; - TEST(true); // Merge peek cursor advanced past desired sequence + CODE_PROBE(true, "Merge peek cursor advanced past desired sequence"); } } @@ -965,7 +965,7 @@ void ILogSystem::SetPeekCursor::updateMessage(int logIdx, bool usePolicy) { c->advanceTo(messageVersion); if (start <= messageVersion && messageVersion < c->version()) { advancedPast = true; - TEST(true); // Merge peek cursor with logIdx advanced past desired sequence + CODE_PROBE(true, "Merge peek cursor with logIdx advanced past desired sequence"); } } } diff --git a/fdbserver/MoveKeys.actor.cpp b/fdbserver/MoveKeys.actor.cpp index 68706d5293..232fdf9e42 100644 --- a/fdbserver/MoveKeys.actor.cpp +++ b/fdbserver/MoveKeys.actor.cpp @@ -217,7 +217,7 @@ ACTOR Future takeMoveKeysLock(Database cx, UID ddId) { return lock; } catch (Error& e) { wait(tr.onError(e)); - TEST(true); // takeMoveKeysLock retry + CODE_PROBE(true, "takeMoveKeysLock retry"); } } } @@ -239,7 +239,7 @@ ACTOR static Future checkMoveKeysLock(Transaction* tr, Optional readVal = wait(tr->get(moveKeysLockWriteKey)); UID lastWrite = readVal.present() ? BinaryReader::fromStringRef(readVal.get(), Unversioned()) : UID(); if (lastWrite != lock.prevWrite) { - TEST(true); // checkMoveKeysLock: Conflict with previous owner + CODE_PROBE(true, "checkMoveKeysLock: Conflict with previous owner"); throw movekeys_conflict(); } @@ -272,7 +272,7 @@ ACTOR static Future checkMoveKeysLock(Transaction* tr, return Void(); } else { - TEST(true); // checkMoveKeysLock: Conflict with new owner + CODE_PROBE(true, "checkMoveKeysLock: Conflict with new owner"); throw movekeys_conflict(); } } @@ -591,7 +591,7 @@ ACTOR static Future startMoveKeys(Database occ, // This process can be split up into multiple transactions if there are too many existing overlapping shards // In that case, each iteration of this loop will have begin set to the end of the last processed shard while (begin < keys.end) { - TEST(begin > keys.begin); // Multi-transactional startMoveKeys + CODE_PROBE(begin > keys.begin, "Multi-transactional startMoveKeys"); batches++; // RYW to optimize re-reading the same key ranges @@ -631,7 +631,7 @@ ACTOR static Future startMoveKeys(Database occ, // Attempt to move onto a server that isn't in serverList (removed or never added to the // database) This can happen (why?) and is handled by the data distribution algorithm // FIXME: Answer why this can happen? - TEST(true); // start move keys moving to a removed server + CODE_PROBE(true, "start move keys moving to a removed server"); throw move_to_removed_server(); } } @@ -825,7 +825,7 @@ ACTOR Future checkFetchingState(Database cx, for (int s = 0; s < serverListValues.size(); s++) { if (!serverListValues[s].present()) { // FIXME: Is this the right behavior? dataMovementComplete will never be sent! - TEST(true); // check fetching state moved to removed server + CODE_PROBE(true, "check fetching state moved to removed server"); throw move_to_removed_server(); } auto si = decodeServerListValue(serverListValues[s].get()); @@ -897,7 +897,7 @@ ACTOR static Future finishMoveKeys(Database occ, // This process can be split up into multiple transactions if there are too many existing overlapping shards // In that case, each iteration of this loop will have begin set to the end of the last processed shard while (begin < keys.end) { - TEST(begin > keys.begin); // Multi-transactional finishMoveKeys + CODE_PROBE(begin > keys.begin, "Multi-transactional finishMoveKeys"); state Transaction tr(occ); @@ -994,7 +994,8 @@ ACTOR static Future finishMoveKeys(Database occ, } else if (alreadyMoved) { dest.clear(); src.clear(); - TEST(true); // FinishMoveKeys first key in iteration sub-range has already been processed + CODE_PROBE(true, + "FinishMoveKeys first key in iteration sub-range has already been processed"); } } @@ -1029,8 +1030,9 @@ ACTOR static Future finishMoveKeys(Database occ, } } if (!dest.size()) { - TEST(true); // A previous finishMoveKeys for this range committed just as it was cancelled to - // start this one? + CODE_PROBE(true, + "A previous finishMoveKeys for this range committed just as it was cancelled to " + "start this one?"); TraceEvent("FinishMoveKeysNothingToDo", relocationIntervalId) .detail("KeyBegin", keys.begin) .detail("KeyEnd", keys.end) @@ -2020,8 +2022,9 @@ ACTOR Future removeStorageServer(Database cx, state bool canRemove = wait(canRemoveStorageServer(tr, serverID)); if (!canRemove) { - TEST(true); // The caller had a transaction in flight that assigned keys to the server. Wait for it to - // reverse its mistake. + CODE_PROBE(true, + "The caller had a transaction in flight that assigned keys to the server. Wait for it to " + "reverse its mistake."); TraceEvent(SevWarn, "NoCanRemove").detail("Count", noCanRemoveCount++).detail("ServerID", serverID); wait(delayJittered(SERVER_KNOBS->REMOVE_RETRY_DELAY, TaskPriority::DataDistributionLaunch)); tr->reset(); @@ -2038,7 +2041,7 @@ ACTOR Future removeStorageServer(Database cx, if (!fListKey.get().present()) { if (retry) { - TEST(true); // Storage server already removed after retrying transaction + CODE_PROBE(true, "Storage server already removed after retrying transaction"); return Void(); } TraceEvent(SevError, "RemoveInvalidServer").detail("ServerID", serverID); diff --git a/fdbserver/MutationTracking.cpp b/fdbserver/MutationTracking.cpp index b0db6b1ed5..5bb491100a 100644 --- a/fdbserver/MutationTracking.cpp +++ b/fdbserver/MutationTracking.cpp @@ -99,7 +99,7 @@ TraceEvent debugTagsAndMessageEnabled(const char* context, Version version, Stri SpanContextMessage scm; br >> scm; } else if (OTELSpanContextMessage::startsOTELSpanContextMessage(mutationType)) { - TEST(true); // MutationTracking reading OTELSpanContextMessage + CODE_PROBE(true, "MutationTracking reading OTELSpanContextMessage"); BinaryReader br(mutationData, AssumeVersion(rdr.protocolVersion())); OTELSpanContextMessage scm; br >> scm; diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 7fa6573a53..5d4afd0675 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -182,7 +182,7 @@ private: Standalone h = wait(self->queue->readNext(sizeof(uint32_t))); if (h.size() != sizeof(uint32_t)) { if (h.size()) { - TEST(true); // Zero fill within size field + CODE_PROBE(true, "Zero fill within size field"); int payloadSize = 0; memcpy(&payloadSize, h.begin(), h.size()); zeroFillSize = sizeof(uint32_t) - h.size(); // zero fill the size itself @@ -196,7 +196,7 @@ private: Standalone e = wait(self->queue->readNext(payloadSize + 1)); if (e.size() != payloadSize + 1) { - TEST(true); // Zero fill within payload + CODE_PROBE(true, "Zero fill within payload"); zeroFillSize = payloadSize + 1 - e.size(); break; } @@ -210,7 +210,7 @@ private: } } if (zeroFillSize) { - TEST(true); // Fixing a partial commit at the end of the tlog queue + CODE_PROBE(true, "Fixing a partial commit at the end of the tlog queue"); for (int i = 0; i < zeroFillSize; i++) self->queue->push(StringRef((const uint8_t*)"", 1)); } @@ -507,9 +507,9 @@ struct LogData : NonCopyable, public ReferenceCounted { ACTOR Future tLogLock(TLogData* self, ReplyPromise reply, Reference logData) { state Version stopVersion = logData->version.get(); - TEST(true); // TLog stopped by recovering master - TEST(logData->stopped); // LogData already stopped - TEST(!logData->stopped); // LogData not yet stopped + CODE_PROBE(true, "TLog stopped by recovering master"); + CODE_PROBE(logData->stopped, "LogData already stopped"); + CODE_PROBE(!logData->stopped, "LogData not yet stopped"); TraceEvent("TLogStop", logData->logId) .detail("Ver", stopVersion) @@ -611,7 +611,7 @@ ACTOR Future updatePersistentData(TLogData* self, Reference logDa // Now that the changes we made to persistentData are durable, erase the data we moved from memory and the queue, // increase bytesDurable accordingly, and update persistentDataDurableVersion. - TEST(anyData); // TLog moved data to persistentData + CODE_PROBE(anyData, "TLog moved data to persistentData"); logData->persistentDataDurableVersion = newPersistentDataVersion; for (tag = logData->tag_data.begin(); tag != logData->tag_data.end(); ++tag) { @@ -834,7 +834,7 @@ void commitMessages(Reference self, // Fill up the rest of this block int bytes = (uint8_t*)r.getLengthPtr() - messages.begin(); if (bytes) { - TEST(true); // Splitting commit messages across multiple blocks + CODE_PROBE(true, "Splitting commit messages across multiple blocks"); messages1 = StringRef(block.end(), bytes); block.append(block.arena(), messages.begin(), bytes); self->messageBlocks.emplace_back(version, block); @@ -1047,7 +1047,7 @@ Future tLogPeekMessages(PromiseType replyPromise, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get() != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version + CODE_PROBE(true, "tlog peek second attempt ended at a different version"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -1120,7 +1120,7 @@ Future tLogPeekMessages(PromiseType replyPromise, auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (sequenceData.isSet()) { if (sequenceData.getFuture().get() != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) + CODE_PROBE(true, "tlog peek second attempt ended at a different version (2)"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -1467,7 +1467,7 @@ ACTOR Future restorePersistentState(TLogData* self, LocalityData locality) if (!fFormat.get().present()) { RangeResult v = wait(self->persistentData->readRange(KeyRangeRef(StringRef(), LiteralStringRef("\xff")), 1)); if (!v.size()) { - TEST(true); // The DB is completely empty, so it was never initialized. Delete it. + CODE_PROBE(true, "The DB is completely empty, so it was never initialized. Delete it."); throw worker_removed(); } else { // This should never happen @@ -1553,7 +1553,7 @@ ACTOR Future restorePersistentState(TLogData* self, LocalityData locality) try { loop { if (allRemoved.isReady()) { - TEST(true); // all tlogs removed during queue recovery + CODE_PROBE(true, "all tlogs removed during queue recovery"); throw worker_removed(); } choose { @@ -1586,7 +1586,7 @@ ACTOR Future restorePersistentState(TLogData* self, LocalityData locality) logData->queueCommittedVersion.set(qe.version); while (self->bytesInput - self->bytesDurable >= recoverMemoryLimit) { - TEST(true); // Flush excess data during TLog queue recovery + CODE_PROBE(true, "Flush excess data during TLog queue recovery"); TraceEvent("FlushLargeQueueDuringRecovery", self->dbgid) .detail("BytesInput", self->bytesInput) .detail("BytesDurable", self->bytesDurable) @@ -1610,7 +1610,7 @@ ACTOR Future restorePersistentState(TLogData* self, LocalityData locality) } TraceEvent("TLogRestorePersistentStateDone", self->dbgid).detail("Took", now() - startt); - TEST(now() - startt >= 1.0); // TLog recovery took more than 1 second + CODE_PROBE(now() - startt >= 1.0, "TLog recovery took more than 1 second"); for (auto it : self->id_data) { if (it.second->queueCommittedVersion.get() == 0) { diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 2d016d4a6a..fbcf4d4dd1 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -148,7 +148,7 @@ private: Standalone h = wait(self->queue->readNext(sizeof(uint32_t))); if (h.size() != sizeof(uint32_t)) { if (h.size()) { - TEST(true); // Zero fill within size field + CODE_PROBE(true, "Zero fill within size field"); int payloadSize = 0; memcpy(&payloadSize, h.begin(), h.size()); zeroFillSize = sizeof(uint32_t) - h.size(); // zero fill the size itself @@ -162,7 +162,7 @@ private: Standalone e = wait(self->queue->readNext(payloadSize + 1)); if (e.size() != payloadSize + 1) { - TEST(true); // Zero fill within payload + CODE_PROBE(true, "Zero fill within payload"); zeroFillSize = payloadSize + 1 - e.size(); break; } @@ -176,7 +176,7 @@ private: } } if (zeroFillSize) { - TEST(true); // Fixing a partial commit at the end of the tlog queue + CODE_PROBE(true, "Fixing a partial commit at the end of the tlog queue"); for (int i = 0; i < zeroFillSize; i++) self->queue->push(StringRef((const uint8_t*)"", 1)); } @@ -653,9 +653,9 @@ void TLogQueue::updateVersionSizes(const TLogQueueEntry& result, TLogData* tLog) ACTOR Future tLogLock(TLogData* self, ReplyPromise reply, Reference logData) { state Version stopVersion = logData->version.get(); - TEST(true); // TLog stopped by recovering master - TEST(logData->stopped); // logData already stopped - TEST(!logData->stopped); // logData not yet stopped + CODE_PROBE(true, "TLog stopped by recovering master"); + CODE_PROBE(logData->stopped, "logData already stopped"); + CODE_PROBE(!logData->stopped, "logData not yet stopped"); TraceEvent("TLogStop", logData->logId) .detail("Ver", stopVersion) @@ -769,7 +769,7 @@ ACTOR Future updatePersistentData(TLogData* self, Reference logDa // Now that the changes we made to persistentData are durable, erase the data we moved from memory and the queue, // increase bytesDurable accordingly, and update persistentDataDurableVersion. - TEST(anyData); // TLog moved data to persistentData + CODE_PROBE(anyData, "TLog moved data to persistentData"); logData->persistentDataDurableVersion = newPersistentDataVersion; for (tagLocality = 0; tagLocality < logData->tag_data.size(); tagLocality++) { @@ -1341,7 +1341,7 @@ Future tLogPeekMessages(PromiseType replyPromise, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version + CODE_PROBE(true, "tlog peek second attempt ended at a different version"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -1439,7 +1439,7 @@ Future tLogPeekMessages(PromiseType replyPromise, if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) + CODE_PROBE(true, "tlog peek second attempt ended at a different version (2)"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -1546,7 +1546,7 @@ ACTOR Future doQueueCommit(TLogData* self, .detail("LogId", logData->logId) .detail("Version", it->version.get()) .detail("QueueVer", it->queueCommittedVersion.get()); - TEST(true); // A TLog was replaced before having a chance to commit its queue + CODE_PROBE(true, "A TLog was replaced before having a chance to commit its queue"); it->queueCommittedVersion.set(it->version.get()); } return Void(); @@ -2007,7 +2007,7 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogCommitRequest req = waitNext(tli.commit.getFuture())) { //TraceEvent("TLogCommitReq", logData->logId).detail("Ver", req.version).detail("PrevVer", req.prevVersion).detail("LogVer", logData->version.get()); ASSERT(logData->isPrimary); - TEST(logData->stopped); // TLogCommitRequest while stopped + CODE_PROBE(logData->stopped, "TLogCommitRequest while stopped"); if (!logData->stopped) logData->addActor.send(tLogCommit(self, req, logData, warningCollectorInput)); else @@ -2333,7 +2333,7 @@ ACTOR Future restorePersistentState(TLogData* self, if (!fFormat.get().present()) { RangeResult v = wait(self->persistentData->readRange(KeyRangeRef(StringRef(), LiteralStringRef("\xff")), 1)); if (!v.size()) { - TEST(true); // The DB is completely empty, so it was never initialized. Delete it. + CODE_PROBE(true, "The DB is completely empty, so it was never initialized. Delete it."); throw worker_removed(); } else { // This should never happen @@ -2473,7 +2473,7 @@ ACTOR Future restorePersistentState(TLogData* self, try { loop { if (allRemoved.isReady()) { - TEST(true); // all tlogs removed during queue recovery + CODE_PROBE(true, "all tlogs removed during queue recovery"); throw worker_removed(); } choose { @@ -2503,7 +2503,7 @@ ACTOR Future restorePersistentState(TLogData* self, logData->queueCommittedVersion.set(qe.version); while (self->bytesInput - self->bytesDurable >= recoverMemoryLimit) { - TEST(true); // Flush excess data during TLog queue recovery + CODE_PROBE(true, "Flush excess data during TLog queue recovery"); TraceEvent("FlushLargeQueueDuringRecovery", self->dbgid) .detail("BytesInput", self->bytesInput) .detail("BytesDurable", self->bytesDurable) @@ -2527,7 +2527,7 @@ ACTOR Future restorePersistentState(TLogData* self, } TraceEvent("TLogRestorePersistentStateDone", self->dbgid).detail("Took", now() - startt); - TEST(now() - startt >= 1.0); // TLog recovery took more than 1 second + CODE_PROBE(now() - startt >= 1.0, "TLog recovery took more than 1 second"); for (auto it : self->id_data) { if (it.second->queueCommittedVersion.get() == 0) { diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index ffb449aff5..a737c4994c 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -156,7 +156,7 @@ private: Standalone h = wait(self->queue->readNext(sizeof(uint32_t))); if (h.size() != sizeof(uint32_t)) { if (h.size()) { - TEST(true); // Zero fill within size field + CODE_PROBE(true, "Zero fill within size field"); int payloadSize = 0; memcpy(&payloadSize, h.begin(), h.size()); zeroFillSize = sizeof(uint32_t) - h.size(); // zero fill the size itself @@ -170,7 +170,7 @@ private: Standalone e = wait(self->queue->readNext(payloadSize + 1)); if (e.size() != payloadSize + 1) { - TEST(true); // Zero fill within payload + CODE_PROBE(true, "Zero fill within payload"); zeroFillSize = payloadSize + 1 - e.size(); break; } @@ -186,7 +186,7 @@ private: } } if (zeroFillSize) { - TEST(true); // Fixing a partial commit at the end of the tlog queue + CODE_PROBE(true, "Fixing a partial commit at the end of the tlog queue"); for (int i = 0; i < zeroFillSize; i++) self->queue->push(StringRef((const uint8_t*)"", 1)); } @@ -756,9 +756,9 @@ void TLogQueue::updateVersionSizes(const TLogQueueEntry& result, ACTOR Future tLogLock(TLogData* self, ReplyPromise reply, Reference logData) { state Version stopVersion = logData->version.get(); - TEST(true); // TLog stopped by recovering master - TEST(logData->stopped); // logData already stopped - TEST(!logData->stopped); // logData not yet stopped + CODE_PROBE(true, "TLog stopped by recovering master"); + CODE_PROBE(logData->stopped, "logData already stopped"); + CODE_PROBE(!logData->stopped, "logData not yet stopped"); TraceEvent("TLogStop", logData->logId) .detail("Ver", stopVersion) @@ -1042,7 +1042,7 @@ ACTOR Future updatePersistentData(TLogData* self, Reference logDa // Now that the changes we made to persistentData are durable, erase the data we moved from memory and the queue, // increase bytesDurable accordingly, and update persistentDataDurableVersion. - TEST(anyData); // TLog moved data to persistentData + CODE_PROBE(anyData, "TLog moved data to persistentData"); logData->persistentDataDurableVersion = newPersistentDataVersion; for (tagLocality = 0; tagLocality < logData->tag_data.size(); tagLocality++) { @@ -1680,7 +1680,7 @@ Future tLogPeekMessages(PromiseType replyPromise, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version + CODE_PROBE(true, "tlog peek second attempt ended at a different version"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -1868,7 +1868,7 @@ Future tLogPeekMessages(PromiseType replyPromise, if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) + CODE_PROBE(true, "tlog peek second attempt ended at a different version (2)"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -1930,7 +1930,7 @@ ACTOR Future watchDegraded(TLogData* self) { wait(lowPriorityDelay(SERVER_KNOBS->TLOG_DEGRADED_DURATION)); TraceEvent(SevWarnAlways, "TLogDegraded", self->dbgid).log(); - TEST(true); // TLog degraded + CODE_PROBE(true, "TLog degraded"); self->degraded->set(true); return Void(); } @@ -1988,7 +1988,7 @@ ACTOR Future doQueueCommit(TLogData* self, .detail("LogId", logData->logId) .detail("Version", it->version.get()) .detail("QueueVer", it->queueCommittedVersion.get()); - TEST(true); // A TLog was replaced before having a chance to commit its queue + CODE_PROBE(true, "A TLog was replaced before having a chance to commit its queue"); it->queueCommittedVersion.set(it->version.get()); } return Void(); @@ -2452,7 +2452,7 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogCommitRequest req = waitNext(tli.commit.getFuture())) { //TraceEvent("TLogCommitReq", logData->logId).detail("Ver", req.version).detail("PrevVer", req.prevVersion).detail("LogVer", logData->version.get()); ASSERT(logData->isPrimary); - TEST(logData->stopped); // TLogCommitRequest while stopped + CODE_PROBE(logData->stopped, "TLogCommitRequest while stopped"); if (!logData->stopped) logData->addActor.send(tLogCommit(self, req, logData, warningCollectorInput)); else @@ -2801,7 +2801,7 @@ ACTOR Future restorePersistentState(TLogData* self, if (!fFormat.get().present()) { RangeResult v = wait(self->persistentData->readRange(KeyRangeRef(StringRef(), LiteralStringRef("\xff")), 1)); if (!v.size()) { - TEST(true); // The DB is completely empty, so it was never initialized. Delete it. + CODE_PROBE(true, "The DB is completely empty, so it was never initialized. Delete it."); throw worker_removed(); } else { // This should never happen @@ -2949,7 +2949,7 @@ ACTOR Future restorePersistentState(TLogData* self, throw end_of_stream(); loop { if (allRemoved.isReady()) { - TEST(true); // all tlogs removed during queue recovery + CODE_PROBE(true, "all tlogs removed during queue recovery"); throw worker_removed(); } choose { @@ -2980,7 +2980,7 @@ ACTOR Future restorePersistentState(TLogData* self, logData->queueCommittedVersion.set(qe.version); while (self->bytesInput - self->bytesDurable >= recoverMemoryLimit) { - TEST(true); // Flush excess data during TLog queue recovery + CODE_PROBE(true, "Flush excess data during TLog queue recovery"); TraceEvent("FlushLargeQueueDuringRecovery", self->dbgid) .detail("LogId", logData->logId) .detail("BytesInput", self->bytesInput) @@ -3010,7 +3010,7 @@ ACTOR Future restorePersistentState(TLogData* self, } TraceEvent("TLogRestorePersistentStateDone", self->dbgid).detail("Took", now() - startt); - TEST(now() - startt >= 1.0); // TLog recovery took more than 1 second + CODE_PROBE(now() - startt >= 1.0, "TLog recovery took more than 1 second"); for (auto it : self->id_data) { if (it.second->queueCommittedVersion.get() == 0) { diff --git a/fdbserver/PaxosConfigConsumer.actor.cpp b/fdbserver/PaxosConfigConsumer.actor.cpp index bbe5fd7e8e..1c1d0d075e 100644 --- a/fdbserver/PaxosConfigConsumer.actor.cpp +++ b/fdbserver/PaxosConfigConsumer.actor.cpp @@ -449,7 +449,7 @@ class PaxosConfigConsumerImpl { if (e.code() == error_code_version_already_compacted || e.code() == error_code_timed_out || e.code() == error_code_failed_to_reach_quorum || e.code() == error_code_version_already_compacted || e.code() == error_code_process_behind) { - TEST(true); // PaxosConfigConsumer get version_already_compacted error + CODE_PROBE(true, "PaxosConfigConsumer get version_already_compacted error"); if (e.code() == error_code_failed_to_reach_quorum) { try { wait(self->getCommittedVersionQuorum.complete()); diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 73345b2e75..573feb6666 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -325,7 +325,7 @@ public: reply.throttledTags = self.tagThrottler->getClientRates(); bool returningTagsToProxy = reply.throttledTags.present() && reply.throttledTags.get().size() > 0; - TEST(returningTagsToProxy); // Returning tag throttles to a proxy + CODE_PROBE(returningTagsToProxy, "Returning tag throttles to a proxy"); } reply.healthMetrics.update(self.healthMetrics, true, req.detailed); diff --git a/fdbserver/ResolutionBalancer.actor.cpp b/fdbserver/ResolutionBalancer.actor.cpp index 5556bc5a1f..f56aec8e09 100644 --- a/fdbserver/ResolutionBalancer.actor.cpp +++ b/fdbserver/ResolutionBalancer.actor.cpp @@ -39,7 +39,7 @@ void ResolutionBalancer::setChangesInReply(UID requestingProxy, GetCommitVersion rep.resolverChangesVersion = resolverChangesVersion; resolverNeedingChanges.erase(requestingProxy); - TEST(!rep.resolverChanges.empty()); // resolution balancing moves keyranges + CODE_PROBE(!rep.resolverChanges.empty(), "resolution balancing moves keyranges"); if (resolverNeedingChanges.empty()) resolverChanges.set(Standalone>()); } diff --git a/fdbserver/Resolver.actor.cpp b/fdbserver/Resolver.actor.cpp index 5d9871aa9e..869872d061 100644 --- a/fdbserver/Resolver.actor.cpp +++ b/fdbserver/Resolver.actor.cpp @@ -350,7 +350,7 @@ ACTOR Future resolveBatch(Reference self, ResolveTransactionBatc applyMetadataMutations(spanContext, *resolverData, req.transactions[t].mutations); } - TEST(self->forceRecovery); // Resolver detects forced recovery + CODE_PROBE(self->forceRecovery, "Resolver detects forced recovery"); } self->resolvedStateTransactions += req.txnStateTransactions.size(); @@ -362,7 +362,7 @@ ACTOR Future resolveBatch(Reference self, ResolveTransactionBatc ASSERT(req.version >= firstUnseenVersion); ASSERT(firstUnseenVersion >= self->debugMinRecentStateVersion); - TEST(firstUnseenVersion == req.version); // Resolver first unseen version is current version + CODE_PROBE(firstUnseenVersion == req.version, "Resolver first unseen version is current version"); // If shardChanged at or before this commit version, the proxy may have computed // the wrong set of groups. Then we need to broadcast to all groups below. @@ -400,13 +400,14 @@ ACTOR Future resolveBatch(Reference self, ResolveTransactionBatc } } - TEST(oldestProxyVersion == req.version); // The proxy that sent this request has the oldest current version - TEST(oldestProxyVersion != - req.version); // The proxy that sent this request does not have the oldest current version + CODE_PROBE(oldestProxyVersion == req.version, + "The proxy that sent this request has the oldest current version"); + CODE_PROBE(oldestProxyVersion != req.version, + "The proxy that sent this request does not have the oldest current version"); bool anyPopped = false; if (firstUnseenVersion <= oldestProxyVersion && self->proxyInfoMap.size() == self->commitProxyCount + 1) { - TEST(true); // Deleting old state transactions + CODE_PROBE(true, "Deleting old state transactions"); int64_t erasedBytes = self->recentStateTransactionsInfo.eraseUpTo(oldestProxyVersion); self->debugMinRecentStateVersion = oldestProxyVersion + 1; anyPopped = erasedBytes > 0; @@ -445,7 +446,7 @@ ACTOR Future resolveBatch(Reference self, ResolveTransactionBatc if (req.debugID.present()) g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "Resolver.resolveBatch.After"); } else { - TEST(true); // Duplicate resolve batch request + CODE_PROBE(true, "Duplicate resolve batch request"); //TraceEvent("DupResolveBatchReq", self->dbgid).detail("From", proxyAddress); } @@ -456,13 +457,13 @@ ACTOR Future resolveBatch(Reference self, ResolveTransactionBatc if (batchItr != proxyInfoItr->second.outstandingBatches.end()) { req.reply.send(batchItr->second); } else { - TEST(true); // No outstanding batches for version on proxy + CODE_PROBE(true, "No outstanding batches for version on proxy"); req.reply.send(Never()); } } else { ASSERT_WE_THINK(false); // The first non-duplicate request with this proxyAddress, including this one, should // have inserted this item in the map! - // TEST(true); // No prior proxy requests + // CODE_PROBE(true, "No prior proxy requests"); req.reply.send(Never()); } diff --git a/fdbserver/RkTagThrottleCollection.cpp b/fdbserver/RkTagThrottleCollection.cpp index d0e8cb9892..83e6c1e1eb 100644 --- a/fdbserver/RkTagThrottleCollection.cpp +++ b/fdbserver/RkTagThrottleCollection.cpp @@ -48,7 +48,7 @@ Optional RkTagThrottleCollection::RkTagThrottleData::updateAndGetClientR ASSERT_GE(rate, 0); return rate; } else { - TEST(true); // Get throttle rate for expired throttle + CODE_PROBE(true, "Get throttle rate for expired throttle"); rateSet = false; return Optional(); } @@ -92,14 +92,14 @@ Optional RkTagThrottleCollection::autoThrottleTag(UID id, bool present = (itr != autoThrottledTags.end()); if (!present) { if (autoThrottledTags.size() >= SERVER_KNOBS->MAX_AUTO_THROTTLED_TRANSACTION_TAGS) { - TEST(true); // Reached auto-throttle limit + CODE_PROBE(true, "Reached auto-throttle limit"); return Optional(); } itr = autoThrottledTags.try_emplace(tag).first; initializeTag(tag); } else if (itr->second.limits.expiration <= now()) { - TEST(true); // Re-throttling expired tag that hasn't been cleaned up + CODE_PROBE(true, "Re-throttling expired tag that hasn't been cleaned up"); present = false; itr->second = RkTagThrottleData(); } @@ -113,7 +113,7 @@ Optional RkTagThrottleCollection::autoThrottleTag(UID id, return Optional(); } } else if (now() <= throttle.lastUpdated + SERVER_KNOBS->AUTO_TAG_THROTTLE_UPDATE_FREQUENCY) { - TEST(true); // Tag auto-throttled too quickly + CODE_PROBE(true, "Tag auto-throttled too quickly"); return Optional(); } else { tpsRate = computeTargetTpsRate(fractionalBusyness, @@ -121,7 +121,7 @@ Optional RkTagThrottleCollection::autoThrottleTag(UID id, tagData[tag].requestRate.smoothRate()); if (throttle.limits.expiration > now() && tpsRate.get() >= throttle.limits.tpsRate) { - TEST(true); // Tag auto-throttle rate increase attempt while active + CODE_PROBE(true, "Tag auto-throttle rate increase attempt while active"); return Optional(); } @@ -176,14 +176,14 @@ void RkTagThrottleCollection::manualThrottleTag(UID id, result.first->second.limits.expiration = expiration; if (!oldLimits.present()) { - TEST(true); // Transaction tag manually throttled + CODE_PROBE(true, "Transaction tag manually throttled"); TraceEvent("RatekeeperAddingManualThrottle", id) .detail("Tag", tag) .detail("Rate", tpsRate) .detail("Priority", transactionPriorityToString(priority)) .detail("SecondsToExpiration", expiration - now()); } else if (oldLimits.get().tpsRate != tpsRate || oldLimits.get().expiration != expiration) { - TEST(true); // Manual transaction tag throttle updated + CODE_PROBE(true, "Manual transaction tag throttle updated"); TraceEvent("RatekeeperUpdatingManualThrottle", id) .detail("Tag", tag) .detail("Rate", tpsRate) @@ -225,14 +225,14 @@ PrioritizedTransactionTagMap RkTagThrottleCollection::g if (priorityItr != manualItr->second.end()) { Optional priorityClientRate = priorityItr->second.updateAndGetClientRate(requestRate); if (!priorityClientRate.present()) { - TEST(true); // Manual priority throttle expired + CODE_PROBE(true, "Manual priority throttle expired"); priorityItr = manualItr->second.erase(priorityItr); } else { if (!manualClientRate.present() || manualClientRate.get().tpsRate > priorityClientRate.get()) { manualClientRate = ClientTagThrottleLimits(priorityClientRate.get(), priorityItr->second.limits.expiration); } else { - TEST(true); // Manual throttle overriden by higher priority + CODE_PROBE(true, "Manual throttle overriden by higher priority"); } ++priorityItr; @@ -241,13 +241,13 @@ PrioritizedTransactionTagMap RkTagThrottleCollection::g if (manualClientRate.present()) { tagPresent = true; - TEST(true); // Using manual throttle + CODE_PROBE(true, "Using manual throttle"); clientRates[*priority][tagItr->first] = manualClientRate.get(); } } if (manualItr->second.empty()) { - TEST(true); // All manual throttles expired + CODE_PROBE(true, "All manual throttles expired"); manualThrottledTags.erase(manualItr); break; } @@ -261,7 +261,7 @@ PrioritizedTransactionTagMap RkTagThrottleCollection::g double rampStartTime = autoItr->second.lastReduced + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION - SERVER_KNOBS->AUTO_TAG_THROTTLE_RAMP_UP_TIME; if (now() >= rampStartTime && adjustedRate != std::numeric_limits::max()) { - TEST(true); // Tag auto-throttle ramping up + CODE_PROBE(true, "Tag auto-throttle ramping up"); double targetBusyness = SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS; if (targetBusyness == 0) { @@ -280,14 +280,14 @@ PrioritizedTransactionTagMap RkTagThrottleCollection::g if (!result.second && result.first->second.tpsRate > adjustedRate) { result.first->second = ClientTagThrottleLimits(adjustedRate, autoItr->second.limits.expiration); } else { - TEST(true); // Auto throttle overriden by manual throttle + CODE_PROBE(true, "Auto throttle overriden by manual throttle"); } clientRates[TransactionPriority::BATCH][tagItr->first] = ClientTagThrottleLimits(0, autoItr->second.limits.expiration); } } else { ASSERT(autoItr->second.limits.expiration <= now()); - TEST(true); // Auto throttle expired + CODE_PROBE(true, "Auto throttle expired"); if (BUGGIFY) { // Temporarily extend the window between expiration and cleanup tagPresent = true; } else { @@ -297,7 +297,7 @@ PrioritizedTransactionTagMap RkTagThrottleCollection::g } if (!tagPresent) { - TEST(true); // All tag throttles expired + CODE_PROBE(true, "All tag throttles expired"); tagItr = tagData.erase(tagItr); } else { ++tagItr; @@ -309,7 +309,7 @@ PrioritizedTransactionTagMap RkTagThrottleCollection::g void RkTagThrottleCollection::addRequests(TransactionTag const& tag, int requests) { if (requests > 0) { - TEST(true); // Requests reported for throttled tag + CODE_PROBE(true, "Requests reported for throttled tag"); auto tagItr = tagData.try_emplace(tag); tagItr.first->second.requestRate.addDelta(requests); diff --git a/fdbserver/SimpleConfigConsumer.actor.cpp b/fdbserver/SimpleConfigConsumer.actor.cpp index ddad99c80d..619f211c11 100644 --- a/fdbserver/SimpleConfigConsumer.actor.cpp +++ b/fdbserver/SimpleConfigConsumer.actor.cpp @@ -109,7 +109,7 @@ class SimpleConfigConsumerImpl { } catch (Error& e) { ++self->failedChangeRequest; if (e.code() == error_code_version_already_compacted) { - TEST(true); // SimpleConfigConsumer get version_already_compacted error + CODE_PROBE(true, "SimpleConfigConsumer get version_already_compacted error"); wait(getSnapshotAndChanges(self, broadcaster)); } else { throw e; diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 90b5fea0fe..a3f0af0f63 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -46,6 +46,7 @@ #include "flow/network.h" #include "flow/TypeTraits.h" #include "flow/FaultInjection.h" +#include "flow/CodeProbeUtils.h" #include "flow/actorcompiler.h" // This must be the last #include. #undef max @@ -845,9 +846,9 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, .detail("Folder", myFolders[i]); } - TEST(bootCount >= 1); // Simulated machine rebooted - TEST(bootCount >= 2); // Simulated machine rebooted twice - TEST(bootCount >= 3); // Simulated machine rebooted three times + CODE_PROBE(bootCount >= 1, "Simulated machine rebooted"); + CODE_PROBE(bootCount >= 2, "Simulated machine rebooted twice"); + CODE_PROBE(bootCount >= 3, "Simulated machine rebooted three times"); ++bootCount; TraceEvent("SimulatedMachineStart", randomId) @@ -967,7 +968,7 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, for (int i = 1; i < ips.size(); i++) killType = std::max(processes[i].get(), killType); - TEST(true); // Simulated machine has been rebooted + CODE_PROBE(true, "Simulated machine has been rebooted"); state bool swap = killType == ISimulator::Reboot && BUGGIFY_WITH_PROB(0.75) && g_simulator.canSwapToMachine(localities.zoneId()); @@ -995,7 +996,7 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, avail.pop_back(); if (myFolders != toRebootFrom) { - TEST(true); // Simulated machine swapped data folders + CODE_PROBE(true, "Simulated machine swapped data folders"); TraceEvent("SimulatedMachineFolderSwap", randomId) .detail("OldFolder0", myFolders[0]) .detail("NewFolder0", toRebootFrom[0]) @@ -1020,7 +1021,7 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, } } - TEST(true); // Simulated machine rebooted with data loss + CODE_PROBE(true, "Simulated machine rebooted with data loss"); } // this machine is rebooting = false; @@ -1067,7 +1068,7 @@ ACTOR Future restartSimulatedSystem(std::vector>* systemActor // Randomly change data center id names to test that localities // can be modified on cluster restart bool renameZoneIds = testConfig.randomlyRenameZoneId ? deterministicRandom()->random01() < 0.1 : false; - TEST(renameZoneIds); // Zone ID names altered in restart test + CODE_PROBE(renameZoneIds, "Zone ID names altered in restart test"); // allows multiple ipAddr entries ini.SetMultiKey(); @@ -1397,27 +1398,27 @@ void SimulationConfig::setStorageEngine(const TestConfig& testConfig) { switch (storage_engine_type) { case 0: { - TEST(true); // Simulated cluster using ssd storage engine + CODE_PROBE(true, "Simulated cluster using ssd storage engine"); set_config("ssd"); break; } case 1: { - TEST(true); // Simulated cluster using default memory storage engine + CODE_PROBE(true, "Simulated cluster using default memory storage engine"); set_config("memory"); break; } case 2: { - TEST(true); // Simulated cluster using radix-tree storage engine + CODE_PROBE(true, "Simulated cluster using radix-tree storage engine"); set_config("memory-radixtree-beta"); break; } case 3: { - TEST(true); // Simulated cluster using redwood storage engine + CODE_PROBE(true, "Simulated cluster using redwood storage engine"); set_config("ssd-redwood-1-experimental"); break; } case 4: { - TEST(true); // Simulated cluster using RocksDB storage engine + CODE_PROBE(true, "Simulated cluster using RocksDB storage engine"); set_config("ssd-rocksdb-v1"); // Tests using the RocksDB engine are necessarily non-deterministic because of RocksDB // background threads. @@ -1427,7 +1428,7 @@ void SimulationConfig::setStorageEngine(const TestConfig& testConfig) { break; } case 5: { - TEST(true); // Simulated cluster using Sharded RocksDB storage engine + CODE_PROBE(true, "Simulated cluster using Sharded RocksDB storage engine"); set_config("ssd-sharded-rocksdb"); // Tests using the RocksDB engine are necessarily non-deterministic because of RocksDB // background threads. @@ -1453,7 +1454,7 @@ void SimulationConfig::setReplicationType(const TestConfig& testConfig) { } else { switch (replication_type) { case 0: { - TEST(true); // Simulated cluster using custom redundancy mode + CODE_PROBE(true, "Simulated cluster using custom redundancy mode"); int storage_servers = deterministicRandom()->randomInt(1, generateFearless ? 4 : 5); // FIXME: log replicas must be more than storage replicas because otherwise better master exists will not // recognize it needs to change dcs @@ -1472,21 +1473,21 @@ void SimulationConfig::setReplicationType(const TestConfig& testConfig) { break; } case 1: { - TEST(true); // Simulated cluster running in single redundancy mode + CODE_PROBE(true, "Simulated cluster running in single redundancy mode"); set_config("single"); break; } case 2: { - TEST(true); // Simulated cluster running in double redundancy mode + CODE_PROBE(true, "Simulated cluster running in double redundancy mode"); set_config("double"); break; } case 3: { if (datacenters <= 2 || generateFearless) { - TEST(true); // Simulated cluster running in triple redundancy mode + CODE_PROBE(true, "Simulated cluster running in triple redundancy mode"); set_config("triple"); } else if (datacenters == 3) { - TEST(true); // Simulated cluster running in 3 data-hall mode + CODE_PROBE(true, "Simulated cluster running in 3 data-hall mode"); set_config("three_data_hall"); } else { ASSERT(false); @@ -1537,17 +1538,17 @@ void SimulationConfig::setRegions(const TestConfig& testConfig) { int satellite_replication_type = deterministicRandom()->randomInt(0, 3); switch (satellite_replication_type) { case 0: { - TEST(true); // Simulated cluster using no satellite redundancy mode (>4 datacenters) + CODE_PROBE(true, "Simulated cluster using no satellite redundancy mode (>4 datacenters)"); break; } case 1: { - TEST(true); // Simulated cluster using two satellite fast redundancy mode + CODE_PROBE(true, "Simulated cluster using two satellite fast redundancy mode"); primaryObj["satellite_redundancy_mode"] = "two_satellite_fast"; remoteObj["satellite_redundancy_mode"] = "two_satellite_fast"; break; } case 2: { - TEST(true); // Simulated cluster using two satellite safe redundancy mode + CODE_PROBE(true, "Simulated cluster using two satellite safe redundancy mode"); primaryObj["satellite_redundancy_mode"] = "two_satellite_safe"; remoteObj["satellite_redundancy_mode"] = "two_satellite_safe"; break; @@ -1560,27 +1561,27 @@ void SimulationConfig::setRegions(const TestConfig& testConfig) { switch (satellite_replication_type) { case 0: { // FIXME: implement - TEST(true); // Simulated cluster using custom satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using custom satellite redundancy mode"); break; } case 1: { - TEST(true); // Simulated cluster using no satellite redundancy mode (<4 datacenters) + CODE_PROBE(true, "Simulated cluster using no satellite redundancy mode (<4 datacenters)"); break; } case 2: { - TEST(true); // Simulated cluster using single satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using single satellite redundancy mode"); primaryObj["satellite_redundancy_mode"] = "one_satellite_single"; remoteObj["satellite_redundancy_mode"] = "one_satellite_single"; break; } case 3: { - TEST(true); // Simulated cluster using double satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using double satellite redundancy mode"); primaryObj["satellite_redundancy_mode"] = "one_satellite_double"; remoteObj["satellite_redundancy_mode"] = "one_satellite_double"; break; } case 4: { - TEST(true); // Simulated cluster using triple satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using triple satellite redundancy mode"); primaryObj["satellite_redundancy_mode"] = "one_satellite_triple"; remoteObj["satellite_redundancy_mode"] = "one_satellite_triple"; break; @@ -1600,10 +1601,10 @@ void SimulationConfig::setRegions(const TestConfig& testConfig) { if (testConfig.minimumRegions <= 1 && (deterministicRandom()->random01() < 0.25 || SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS < SERVER_KNOBS->VERSIONS_PER_SECOND)) { - TEST(true); // Simulated cluster using one region + CODE_PROBE(true, "Simulated cluster using one region"); needsRemote = false; } else { - TEST(true); // Simulated cluster using two regions + CODE_PROBE(true, "Simulated cluster using two regions"); db.usableRegions = 2; } @@ -1611,25 +1612,25 @@ void SimulationConfig::setRegions(const TestConfig& testConfig) { switch (remote_replication_type) { case 0: { // FIXME: implement - TEST(true); // Simulated cluster using custom remote redundancy mode + CODE_PROBE(true, "Simulated cluster using custom remote redundancy mode"); break; } case 1: { - TEST(true); // Simulated cluster using default remote redundancy mode + CODE_PROBE(true, "Simulated cluster using default remote redundancy mode"); break; } case 2: { - TEST(true); // Simulated cluster using single remote redundancy mode + CODE_PROBE(true, "Simulated cluster using single remote redundancy mode"); set_config("remote_single"); break; } case 3: { - TEST(true); // Simulated cluster using double remote redundancy mode + CODE_PROBE(true, "Simulated cluster using double remote redundancy mode"); set_config("remote_double"); break; } case 4: { - TEST(true); // Simulated cluster using triple remote redundancy mode + CODE_PROBE(true, "Simulated cluster using triple remote redundancy mode"); set_config("remote_triple"); break; } @@ -1983,18 +1984,18 @@ void setupSimulatedSystem(std::vector>* systemActors, bool sslOnly = sslEnabled && deterministicRandom()->coinflip(); bool isTLS = sslEnabled && sslOnly; g_simulator.listenersPerProcess = sslEnabled && !sslOnly ? 2 : 1; - TEST(sslEnabled); // SSL enabled - TEST(!sslEnabled); // SSL disabled + CODE_PROBE(sslEnabled, "SSL enabled"); + CODE_PROBE(!sslEnabled, "SSL disabled"); // Use IPv6 25% of the time bool useIPv6 = deterministicRandom()->random01() < 0.25; - TEST(useIPv6); // Use IPv6 - TEST(!useIPv6); // Use IPv4 + CODE_PROBE(useIPv6, "Use IPv6"); + CODE_PROBE(!useIPv6, "Use IPv4"); // Use hostname 25% of the time, unless it is disabled bool useHostname = !testConfig.disableHostname && deterministicRandom()->random01() < 0.25; - TEST(useHostname); // Use hostname - TEST(!useHostname); // Use IP address + CODE_PROBE(useHostname, "Use hostname"); + CODE_PROBE(!useHostname, "Use IP address"); NetworkAddressFromHostname fromHostname = useHostname ? NetworkAddressFromHostname::True : NetworkAddressFromHostname::False; @@ -2430,7 +2431,7 @@ ACTOR void setupAndRun(std::string dataFolder, wait(g_simulator.onProcess(testSystem, TaskPriority::DefaultYield)); Sim2FileSystem::newFileSystem(); FlowTransport::createInstance(true, 1, WLTOKEN_RESERVED_COUNT, &allowList); - TEST(true); // Simulation start + CODE_PROBE(true, "Simulation start"); state Optional defaultTenant; state Standalone> tenantsToCreate; @@ -2507,6 +2508,8 @@ ACTOR void setupAndRun(std::string dataFolder, TraceEvent(SevError, "SetupAndRunError").error(e); } + TraceEvent("TracingMissingCodeProbes").log(); + probe::traceMissedProbes(probe::ExecutionContext::Simulation); TraceEvent("SimulatedSystemDestruct").log(); g_simulator.stop(); destructed = true; diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 1be4f63698..60bc889d6d 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -282,7 +282,7 @@ public: void checkChangeCounter(uint64_t oldCacheRangeChangeCounter, KeyRef const& key) { if (oldCacheRangeChangeCounter != cacheRangeChangeCounter && cachedRangeMap[key]->changeCounter > oldCacheRangeChangeCounter) { - TEST(true); // CacheRange change during getValueQ + CODE_PROBE(true, "CacheRange change during getValueQ"); // TODO: should we throw the cold_cache_server() error here instead? throw wrong_shard_server(); } @@ -293,7 +293,7 @@ public: auto sh = cachedRangeMap.intersectingRanges(keys); for (auto i = sh.begin(); i != sh.end(); ++i) if (i->value()->changeCounter > oldCacheRangeChangeCounter) { - TEST(true); // CacheRange change during range operation + CODE_PROBE(true, "CacheRange change during range operation"); // TODO: should we throw the cold_cache_server() error here instead? throw wrong_shard_server(); } @@ -665,7 +665,7 @@ Key findKey(StorageCacheData* data, KeySelectorRef sel, Version version, KeyRang // If we get only one result in the reverse direction as a result of the data being too large, we could get stuck in // a loop if (more && !forward && rep.data.size() == 1) { - TEST(true); // Reverse key selector returned only one result in range read + CODE_PROBE(true, "Reverse key selector returned only one result in range read"); maxBytes = std::numeric_limits::max(); GetKeyValuesReply rep2 = readRange(data, version, KeyRangeRef(range.begin, keyAfter(sel.getKey())), -2, &maxBytes); @@ -688,7 +688,7 @@ Key findKey(StorageCacheData* data, KeySelectorRef sel, Version version, KeyRang *pOffset = -*pOffset; if (more) { - TEST(true); // Key selector read range had more results + CODE_PROBE(true, "Key selector read range had more results"); ASSERT(rep.data.size()); Key returnKey = forward ? keyAfter(rep.data.back().key) : rep.data.back().key; @@ -781,7 +781,7 @@ ACTOR Future getKeyValues(StorageCacheData* data, GetKeyValuesRequest req) // cachedKeyRange is the end the last actual key returned must be from this cachedKeyRange. A begin offset of 1 // is also OK because then either begin is past end or equal to end (so the result is definitely empty) if ((offset1 && offset1 != 1) || (offset2 && offset2 != 1)) { - TEST(true); // wrong_cache_server due to offset + CODE_PROBE(true, "wrong_cache_server due to offset"); // We could detect when offset1 takes us off the beginning of the database or offset2 takes us off the end, // and return a clipped range rather than an error (since that is what the NativeAPI.getRange will do anyway // via its "slow path"), but we would have to add some flags to the response to encode whether we went off @@ -943,7 +943,7 @@ bool expandMutation(MutationRef& m, StorageCacheData::VersionedData const& data, if (it != data.atLatest().end() && it->isValue() && it.key() == m.param1) oldVal = it->getValue(); else if (it != data.atLatest().end() && it->isClearTo() && it->getEndKey() > m.param1) { - TEST(true); // Atomic op right after a clear. + CODE_PROBE(true, "Atomic op right after a clear."); } switch (m.type) { @@ -1073,7 +1073,7 @@ void splitMutation(StorageCacheData* data, KeyRangeMap& map, MutationRef cons } void rollback(StorageCacheData* data, Version rollbackVersion, Version nextVersion) { - TEST(true); // call to cacheRange rollback + CODE_PROBE(true, "call to cacheRange rollback"); // FIXME: enable when debugKeyRange is active // debugKeyRange("Rollback", rollbackVersion, allKeys); @@ -1279,7 +1279,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang lastAvailable = std::max(lastAvailable, r->value()); if (lastAvailable != invalidVersion && lastAvailable >= data->oldestVersion.get()) { - TEST(true); // wait for oldest version + CODE_PROBE(true, "wait for oldest version"); wait(data->oldestVersion.whenAtLeast(lastAvailable + 1)); } @@ -1318,7 +1318,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang loop { try { - TEST(true); // Fetching keys for transferred cacheRange + CODE_PROBE(true, "Fetching keys for transferred cacheRange"); state RangeResult this_block = wait(tryFetchRange(data->cx, @@ -1382,7 +1382,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang .suppressFor(1.0) .detail("FKID", interval.pairID); if (e.code() == error_code_transaction_too_old) { - TEST(true); // A storage server has forgotten the history data we are fetching + CODE_PROBE(true, "A storage server has forgotten the history data we are fetching"); Version lastFV = fetchVersion; fetchVersion = data->version.get(); isTooOld = false; @@ -1409,8 +1409,9 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang .detail("E", data->version.get()); } } else if (e.code() == error_code_future_version || e.code() == error_code_process_behind) { - TEST(true); // fetchKeys got future_version or process_behind, so there must be a huge storage lag - // somewhere. Keep trying. + CODE_PROBE(true, + "fetchKeys got future_version or process_behind, so there must be a huge storage lag " + "somewhere. Keep trying."); } else { throw; } @@ -1470,7 +1471,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang } int startSize = batch->changes.size(); - TEST(startSize); // Adding fetch data to a batch which already has changes + CODE_PROBE(startSize, "Adding fetch data to a batch which already has changes"); batch->changes.resize(batch->changes.size() + cacheRange->updates.size()); // FIXME: pass the deque back rather than copy the data @@ -1633,7 +1634,7 @@ void cacheWarmup(StorageCacheData* data, const KeyRangeRef& keys, bool nowAssign else { ASSERT(ranges[i].value->adding); data->addCacheRange(CacheRangeInfo::newAdding(data, ranges[i])); - TEST(true); // cacheWarmup reFetchKeys + CODE_PROBE(true, "cacheWarmup reFetchKeys"); } } @@ -1772,7 +1773,7 @@ private: br >> rollbackVersion; if (rollbackVersion < fromVersion && rollbackVersion > data->oldestVersion.get()) { - TEST(true); // CacheRangeApplyPrivateData cacheRange rollback + CODE_PROBE(true, "CacheRangeApplyPrivateData cacheRange rollback"); TraceEvent(SevWarn, "Rollback", data->thisServerID) .detail("FromVersion", fromVersion) .detail("ToVersion", rollbackVersion) @@ -1962,8 +1963,10 @@ ACTOR Future pullAsyncData(StorageCacheData* data) { } if (data->cacheRangeChangeCounter == changeCounter) break; - // TEST(true); // A fetchKeys completed while we were doing this, so eager might be outdated. Read - // it again. + // CODE_PROBE( + // true, + // "A fetchKeys completed while we were doing this, so eager might be outdated. Read it + // again."); } } @@ -2014,7 +2017,7 @@ ACTOR Future pullAsyncData(StorageCacheData* data) { SpanContextMessage scm; reader >> scm; } else if (reader.protocolVersion().hasOTELSpanContext() && OTELSpanContextMessage::isNextIn(reader)) { - TEST(true); // StorageCache reading OTELSpanContextMessage + CODE_PROBE(true, "StorageCache reading OTELSpanContextMessage"); OTELSpanContextMessage oscm; reader >> oscm; } else { diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 921000cad2..b2591fd20e 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -157,7 +157,7 @@ private: Standalone h = wait(self->queue->readNext(sizeof(uint32_t))); if (h.size() != sizeof(uint32_t)) { if (h.size()) { - TEST(true); // Zero fill within size field + CODE_PROBE(true, "Zero fill within size field"); int payloadSize = 0; memcpy(&payloadSize, h.begin(), h.size()); zeroFillSize = sizeof(uint32_t) - h.size(); // zero fill the size itself @@ -171,7 +171,7 @@ private: Standalone e = wait(self->queue->readNext(payloadSize + 1)); if (e.size() != payloadSize + 1) { - TEST(true); // Zero fill within payload + CODE_PROBE(true, "Zero fill within payload"); zeroFillSize = payloadSize + 1 - e.size(); break; } @@ -187,7 +187,7 @@ private: } } if (zeroFillSize) { - TEST(true); // Fixing a partial commit at the end of the tlog queue + CODE_PROBE(true, "Fixing a partial commit at the end of the tlog queue"); for (int i = 0; i < zeroFillSize; i++) self->queue->push(StringRef((const uint8_t*)"", 1)); } @@ -805,9 +805,9 @@ void TLogQueue::updateVersionSizes(const TLogQueueEntry& result, ACTOR Future tLogLock(TLogData* self, ReplyPromise reply, Reference logData) { state Version stopVersion = logData->version.get(); - TEST(true); // TLog stopped by recovering cluster-controller - TEST(logData->stopped); // logData already stopped - TEST(!logData->stopped); // logData not yet stopped + CODE_PROBE(true, "TLog stopped by recovering cluster-controller"); + CODE_PROBE(logData->stopped, "logData already stopped"); + CODE_PROBE(!logData->stopped, "logData not yet stopped"); TraceEvent("TLogStop", logData->logId) .detail("Ver", stopVersion) @@ -1097,7 +1097,7 @@ ACTOR Future updatePersistentData(TLogData* self, Reference logDa // Now that the changes we made to persistentData are durable, erase the data we moved from memory and the queue, // increase bytesDurable accordingly, and update persistentDataDurableVersion. - TEST(anyData); // TLog moved data to persistentData + CODE_PROBE(anyData, "TLog moved data to persistentData"); logData->persistentDataDurableVersion = newPersistentDataVersion; for (tagLocality = 0; tagLocality < logData->tag_data.size(); tagLocality++) { for (tagId = 0; tagId < logData->tag_data[tagLocality].size(); tagId++) { @@ -1248,7 +1248,7 @@ ACTOR Future processPopRequests(TLogData* self, Reference logData TraceEvent("PlayIgnoredPop", logData->logId).detail("Tag", tag.toString()).detail("Version", version); ignoredPops.push_back(tLogPopCore(self, tag, version, logData)); if (++ignoredPopsPlayed % SERVER_KNOBS->TLOG_POP_BATCH_SIZE == 0) { - TEST(true); // Yielding while processing pop requests + CODE_PROBE(true, "Yielding while processing pop requests"); wait(yield()); } } @@ -1836,7 +1836,7 @@ Future tLogPeekMessages(PromiseType replyPromise, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version + CODE_PROBE(true, "tlog peek second attempt ended at a different version"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -2069,7 +2069,7 @@ Future tLogPeekMessages(PromiseType replyPromise, if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) + CODE_PROBE(true, "tlog peek second attempt ended at a different version (2)"); replyPromise.sendError(operation_obsolete()); return Void(); } @@ -2177,7 +2177,7 @@ ACTOR Future doQueueCommit(TLogData* self, .detail("LogId", logData->logId) .detail("Version", it->version.get()) .detail("QueueVer", it->queueCommittedVersion.get()); - TEST(true); // A TLog was replaced before having a chance to commit its queue + CODE_PROBE(true, "A TLog was replaced before having a chance to commit its queue"); it->queueCommittedVersion.set(it->version.get()); } return Void(); @@ -2655,7 +2655,7 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogCommitRequest req = waitNext(tli.commit.getFuture())) { //TraceEvent("TLogCommitReq", logData->logId).detail("Ver", req.version).detail("PrevVer", req.prevVersion).detail("LogVer", logData->version.get()); ASSERT(logData->isPrimary); - TEST(logData->stopped); // TLogCommitRequest while stopped + CODE_PROBE(logData->stopped, "TLogCommitRequest while stopped"); if (!logData->stopped) logData->addActor.send(tLogCommit(self, req, logData, warningCollectorInput)); else @@ -3026,7 +3026,7 @@ ACTOR Future restorePersistentState(TLogData* self, if (!fFormat.get().present()) { RangeResult v = wait(self->persistentData->readRange(KeyRangeRef(StringRef(), LiteralStringRef("\xff")), 1)); if (!v.size()) { - TEST(true); // The DB is completely empty, so it was never initialized. Delete it. + CODE_PROBE(true, "The DB is completely empty, so it was never initialized. Delete it."); throw worker_removed(); } else { // This should never happen @@ -3183,7 +3183,7 @@ ACTOR Future restorePersistentState(TLogData* self, throw end_of_stream(); loop { if (allRemoved.isReady()) { - TEST(true); // all tlogs removed during queue recovery + CODE_PROBE(true, "all tlogs removed during queue recovery"); throw worker_removed(); } choose { @@ -3214,7 +3214,7 @@ ACTOR Future restorePersistentState(TLogData* self, logData->queueCommittedVersion.set(qe.version); while (self->bytesInput - self->bytesDurable >= recoverMemoryLimit) { - TEST(true); // Flush excess data during TLog queue recovery + CODE_PROBE(true, "Flush excess data during TLog queue recovery"); TraceEvent("FlushLargeQueueDuringRecovery", self->dbgid) .detail("LogId", logData->logId) .detail("BytesInput", self->bytesInput) @@ -3244,7 +3244,7 @@ ACTOR Future restorePersistentState(TLogData* self, } TraceEvent("TLogRestorePersistentStateDone", self->dbgid).detail("Took", now() - startt); - TEST(now() - startt >= 1.0); // TLog recovery took more than 1 second + CODE_PROBE(now() - startt >= 1.0, "TLog recovery took more than 1 second"); for (auto it : self->id_data) { if (it.second->queueCommittedVersion.get() == 0) { diff --git a/fdbserver/TagPartitionedLogSystem.actor.cpp b/fdbserver/TagPartitionedLogSystem.actor.cpp index 7f19ffdf4a..90bb2cc264 100644 --- a/fdbserver/TagPartitionedLogSystem.actor.cpp +++ b/fdbserver/TagPartitionedLogSystem.actor.cpp @@ -2167,7 +2167,7 @@ ACTOR Future TagPartitionedLogSystem::epochEnd(Reference TagPartitionedLogSystem::epochEnd(Referencelocality)) { TraceEvent("EpochEndLockExtra").detail("Locality", log->locality); - TEST(true); // locking old generations for version information + CODE_PROBE(true, "locking old generations for version information"); lockedLocalities.insert(log->locality); LogLockInfo lockResult; lockResult.epochEnd = old.epochEnd; @@ -2312,7 +2312,7 @@ ACTOR Future TagPartitionedLogSystem::epochEnd(Reference 0 && (!lastEnd.present() || maxEnd < lastEnd.get())) { - TEST(lastEnd.present()); // Restarting recovery at an earlier point + CODE_PROBE(lastEnd.present(), "Restarting recovery at an earlier point"); auto logSystem = makeReference(dbgid, locality, prevState.recoveryCount); diff --git a/fdbserver/TagThrottler.actor.cpp b/fdbserver/TagThrottler.actor.cpp index 432aa459b1..93332bb310 100644 --- a/fdbserver/TagThrottler.actor.cpp +++ b/fdbserver/TagThrottler.actor.cpp @@ -54,20 +54,20 @@ class TagThrottlerImpl { if (autoThrottlingEnabled.get().present() && autoThrottlingEnabled.get().get() == LiteralStringRef("0")) { - TEST(true); // Auto-throttling disabled + CODE_PROBE(true, "Auto-throttling disabled"); if (self->autoThrottlingEnabled) { TraceEvent("AutoTagThrottlingDisabled", self->id).log(); } self->autoThrottlingEnabled = false; } else if (autoThrottlingEnabled.get().present() && autoThrottlingEnabled.get().get() == LiteralStringRef("1")) { - TEST(true); // Auto-throttling enabled + CODE_PROBE(true, "Auto-throttling enabled"); if (!self->autoThrottlingEnabled) { TraceEvent("AutoTagThrottlingEnabled", self->id).log(); } self->autoThrottlingEnabled = true; } else { - TEST(true); // Auto-throttling unspecified + CODE_PROBE(true, "Auto-throttling unspecified"); if (autoThrottlingEnabled.get().present()) { TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue", self->id) .detail("Value", autoThrottlingEnabled.get().get()); @@ -90,7 +90,7 @@ class TagThrottlerImpl { if (tagValue.expirationTime == 0 || tagValue.expirationTime > now() + tagValue.initialDuration) { - TEST(true); // Converting tag throttle duration to absolute time + CODE_PROBE(true, "Converting tag throttle duration to absolute time"); tagValue.expirationTime = now() + tagValue.initialDuration; BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason())); wr << tagValue; @@ -128,7 +128,7 @@ class TagThrottlerImpl { wait(watchFuture); TraceEvent("RatekeeperThrottleSignaled", self->id).log(); - TEST(true); // Tag throttle changes detected + CODE_PROBE(true, "Tag throttle changes detected"); break; } catch (Error& e) { TraceEvent("RatekeeperMonitorThrottlingChangesError", self->id).error(e); @@ -142,7 +142,7 @@ class TagThrottlerImpl { // NOTE: before the comparison with MIN_TAG_COST, the busiest tag rate also compares with MIN_TAG_PAGES_RATE // currently MIN_TAG_PAGES_RATE > MIN_TAG_COST in our default knobs. if (busyness > SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS && rate > SERVER_KNOBS->MIN_TAG_COST) { - TEST(true); // Transaction tag auto-throttled + CODE_PROBE(true, "Transaction tag auto-throttled"); Optional clientRate = throttledTags.autoThrottleTag(id, tag, busyness); // TODO: Increment tag throttle counts here? if (clientRate.present()) { diff --git a/fdbserver/TransactionTagCounter.cpp b/fdbserver/TransactionTagCounter.cpp index 7b7829f676..bd09d19cd4 100644 --- a/fdbserver/TransactionTagCounter.cpp +++ b/fdbserver/TransactionTagCounter.cpp @@ -99,7 +99,7 @@ public: void addRequest(Optional const& tags, int64_t bytes) { if (tags.present()) { - TEST(true); // Tracking transaction tag in counter + CODE_PROBE(true, "Tracking transaction tag in counter"); double cost = costFunction(bytes); for (auto& tag : tags.get()) { int64_t& count = intervalCounts[TransactionTag(tag, tags.get().getArena())]; diff --git a/fdbserver/VersionedBTree.actor.cpp b/fdbserver/VersionedBTree.actor.cpp index 98bd0f684a..6e07cd9c50 100644 --- a/fdbserver/VersionedBTree.actor.cpp +++ b/fdbserver/VersionedBTree.actor.cpp @@ -7819,10 +7819,10 @@ public: if (rowLimit > 0) { f = cur.seekGTE(keys.begin); if (f.isReady()) { - TEST(true); // Cached forward range read seek + CODE_PROBE(true, "Cached forward range read seek"); f.get(); } else { - TEST(true); // Uncached forward range read seek + CODE_PROBE(true, "Uncached forward range read seek"); wait(store(lock, self->m_concurrentReads.lock())); wait(f); } @@ -7875,10 +7875,10 @@ public: } else { f = cur.seekLT(keys.end); if (f.isReady()) { - TEST(true); // Cached reverse range read seek + CODE_PROBE(true, "Cached reverse range read seek"); f.get(); } else { - TEST(true); // Uncached reverse range read seek + CODE_PROBE(true, "Uncached reverse range read seek"); wait(store(lock, self->m_concurrentReads.lock())); wait(f); } diff --git a/fdbserver/WaitFailure.actor.cpp b/fdbserver/WaitFailure.actor.cpp index d1ff0f0a4e..62e514ecab 100644 --- a/fdbserver/WaitFailure.actor.cpp +++ b/fdbserver/WaitFailure.actor.cpp @@ -30,7 +30,7 @@ ACTOR Future waitFailureServer(FutureStream> waitFailur ReplyPromise P = waitNext(waitFailure); queue.push_back(P); if (queue.size() > SERVER_KNOBS->MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS) { - TEST(true); // wait server queue full + CODE_PROBE(true, "wait server queue full"); queue.front().send(Void()); queue.pop_front(); } diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index b5331776ae..8ffa2deed8 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -108,7 +108,7 @@ enum { OPT_CONNFILE, OPT_SEEDCONNFILE, OPT_SEEDCONNSTRING, OPT_ROLE, OPT_LISTEN, OPT_PUBLICADDR, OPT_DATAFOLDER, OPT_LOGFOLDER, OPT_PARENTPID, OPT_TRACER, OPT_NEWCONSOLE, OPT_NOBOX, OPT_TESTFILE, OPT_RESTARTING, OPT_RESTORING, OPT_RANDOMSEED, OPT_KEY, OPT_MEMLIMIT, OPT_VMEMLIMIT, OPT_STORAGEMEMLIMIT, OPT_CACHEMEMLIMIT, OPT_MACHINEID, OPT_DCID, OPT_MACHINE_CLASS, OPT_BUGGIFY, OPT_VERSION, OPT_BUILD_FLAGS, OPT_CRASHONERROR, OPT_HELP, OPT_NETWORKIMPL, OPT_NOBUFSTDOUT, OPT_BUFSTDOUTERR, - OPT_TRACECLOCK, OPT_NUMTESTERS, OPT_DEVHELP, OPT_ROLLSIZE, OPT_MAXLOGS, OPT_MAXLOGSSIZE, OPT_KNOB, OPT_UNITTESTPARAM, OPT_TESTSERVERS, OPT_TEST_ON_SERVERS, OPT_METRICSCONNFILE, + OPT_TRACECLOCK, OPT_NUMTESTERS, OPT_DEVHELP, OPT_PRINT_CODE_PROBES, OPT_ROLLSIZE, OPT_MAXLOGS, OPT_MAXLOGSSIZE, OPT_KNOB, OPT_UNITTESTPARAM, OPT_TESTSERVERS, OPT_TEST_ON_SERVERS, OPT_METRICSCONNFILE, OPT_METRICSPREFIX, OPT_LOGGROUP, OPT_LOCALITY, OPT_IO_TRUST_SECONDS, OPT_IO_TRUST_WARN_ONLY, OPT_FILESYSTEM, OPT_PROFILER_RSS_SIZE, OPT_KVFILE, OPT_TRACE_FORMAT, OPT_WHITELIST_BINPATH, OPT_BLOB_CREDENTIAL_FILE, OPT_CONFIG_PATH, OPT_USE_TEST_CONFIG_DB, OPT_FAULT_INJECTION, OPT_PROFILER, OPT_PRINT_SIMTIME, OPT_FLOW_PROCESS_NAME, OPT_FLOW_PROCESS_ENDPOINT, OPT_IP_TRUSTED_MASK, OPT_KMS_CONN_DISCOVERY_URL_FILE, OPT_KMS_CONN_VALIDATION_TOKEN_DETAILS, OPT_KMS_CONN_GET_ENCRYPTION_KEYS_ENDPOINT @@ -183,6 +183,7 @@ CSimpleOpt::SOption g_rgOptions[] = { { OPT_HELP, "-h", SO_NONE }, { OPT_HELP, "--help", SO_NONE }, { OPT_DEVHELP, "--dev-help", SO_NONE }, + { OPT_PRINT_CODE_PROBES, "--code-probes", SO_REQ_SEP }, { OPT_KNOB, "--knob-", SO_REQ_SEP }, { OPT_UNITTESTPARAM, "--test-", SO_REQ_SEP }, { OPT_LOCALITY, "--locality-", SO_REQ_SEP }, @@ -1144,6 +1145,10 @@ private: printUsage(argv[0], true); flushAndExit(FDB_EXIT_SUCCESS); break; + case OPT_PRINT_CODE_PROBES: + probe::ICodeProbe::printProbesJSON({ std::string(args.OptionArg()) }); + flushAndExit(FDB_EXIT_SUCCESS); + break; case OPT_KNOB: { Optional knobName = extractPrefixedArgument("--knob", args.OptionSyntax()); if (!knobName.present()) { diff --git a/fdbserver/include/fdbserver/ClusterRecovery.actor.h b/fdbserver/include/fdbserver/ClusterRecovery.actor.h index 810bd35f7a..8a146b001e 100644 --- a/fdbserver/include/fdbserver/ClusterRecovery.actor.h +++ b/fdbserver/include/fdbserver/ClusterRecovery.actor.h @@ -132,7 +132,7 @@ private: wait(self->cstate.setExclusive( BinaryWriter::toValue(newState, IncludeVersion(ProtocolVersion::withDBCoreState())))); } catch (Error& e) { - TEST(true); // Master displaced during writeMasterState + CODE_PROBE(true, "Master displaced during writeMasterState"); throw; } @@ -148,7 +148,7 @@ private: if (readState != newState) { TraceEvent("RecoveryTerminated", self->dbgid).detail("Reason", "CStateChanged"); - TEST(true); // Coordinated state changed between writing and reading, recovery restarting + CODE_PROBE(true, "Coordinated state changed between writing and reading, recovery restarting"); throw worker_removed(); } self->switchedState = Promise(); diff --git a/fdbserver/include/fdbserver/LogSystem.h b/fdbserver/include/fdbserver/LogSystem.h index eb1e2d52cd..fd3f31ab6a 100644 --- a/fdbserver/include/fdbserver/LogSystem.h +++ b/fdbserver/include/fdbserver/LogSystem.h @@ -865,7 +865,7 @@ void LogPushData::writeTypedMessage(T const& item, bool metadataMessage, bool al // subsequence wasn't actually used and can be decremented. if (!updatedLocation) { this->subsequence--; - TEST(true); // No new SpanContextMessage written to transaction logs + CODE_PROBE(true, "No new SpanContextMessage written to transaction logs"); ASSERT(this->subsequence > 0); } } else { diff --git a/fdbserver/include/fdbserver/StorageMetrics.h b/fdbserver/include/fdbserver/StorageMetrics.h index 1ecf102e3e..9923bbf875 100644 --- a/fdbserver/include/fdbserver/StorageMetrics.h +++ b/fdbserver/include/fdbserver/StorageMetrics.h @@ -131,7 +131,7 @@ struct TransientStorageMetricSample : StorageMetricSample { StorageMetrics deltaM = m * delta; auto v = waitMap[key]; for (int i = 0; i < v.size(); i++) { - TEST(true); // TransientStorageMetricSample poll update + CODE_PROBE(true, "TransientStorageMetricSample poll update"); v[i].send(deltaM); } @@ -208,9 +208,9 @@ struct StorageServerMetrics { void notify(KeyRef key, StorageMetrics& metrics) { ASSERT(metrics.bytes == 0); // ShardNotifyMetrics if (g_network->isSimulated()) { - TEST(metrics.bytesPerKSecond != 0); // ShardNotifyMetrics bytes - TEST(metrics.iosPerKSecond != 0); // ShardNotifyMetrics ios - TEST(metrics.bytesReadPerKSecond != 0); // ShardNotifyMetrics bytesRead + CODE_PROBE(metrics.bytesPerKSecond != 0, "ShardNotifyMetrics bytes"); + CODE_PROBE(metrics.iosPerKSecond != 0, "ShardNotifyMetrics ios"); + CODE_PROBE(metrics.bytesReadPerKSecond != 0, "ShardNotifyMetrics bytesRead"); } double expire = now() + SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL; @@ -230,7 +230,7 @@ struct StorageServerMetrics { auto& v = waitMetricsMap[key]; for (int i = 0; i < v.size(); i++) { if (g_network->isSimulated()) { - TEST(true); // shard notify metrics + CODE_PROBE(true, "shard notify metrics"); } // ShardNotifyMetrics v[i].send(notifyMetrics); @@ -249,7 +249,7 @@ struct StorageServerMetrics { notifyMetrics.bytesReadPerKSecond = bytesReadPerKSecond; auto& v = waitMetricsMap[key]; for (int i = 0; i < v.size(); i++) { - TEST(true); // ShardNotifyMetrics + CODE_PROBE(true, "ShardNotifyMetrics"); v[i].send(notifyMetrics); } } @@ -264,7 +264,7 @@ struct StorageServerMetrics { StorageMetrics notifyMetrics; notifyMetrics.bytes = bytes; for (int i = 0; i < shard.value().size(); i++) { - TEST(true); // notifyBytes + CODE_PROBE(true, "notifyBytes"); shard.value()[i].send(notifyMetrics); } } @@ -284,7 +284,7 @@ struct StorageServerMetrics { auto rs = waitMetricsMap.intersectingRanges(keys); for (auto r = rs.begin(); r != rs.end(); ++r) { auto& v = r->value(); - TEST(v.size()); // notifyNotReadable() sending errors to intersecting ranges + CODE_PROBE(v.size(), "notifyNotReadable() sending errors to intersecting ranges"); for (int n = 0; n < v.size(); n++) v[n].sendError(wrong_shard_server()); } diff --git a/fdbserver/include/fdbserver/WorkerInterface.actor.h b/fdbserver/include/fdbserver/WorkerInterface.actor.h index dd73704f5f..1168632e1e 100644 --- a/fdbserver/include/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/include/fdbserver/WorkerInterface.actor.h @@ -1242,7 +1242,7 @@ Future ioDegradedOrTimeoutError(Future what, choose { when(T t = wait(what)) { return t; } when(wait(degradedEnd)) { - TEST(true); // TLog degraded + CODE_PROBE(true, "TLog degraded", probe::func::deduplicate); TraceEvent(SevWarnAlways, "IoDegraded").log(); degraded->set(true); } diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index b18ba6a047..1af905c71c 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -160,15 +160,16 @@ ACTOR Future getVersion(Reference self, GetCommitVersionReques return Void(); } - TEST(proxyItr->second.latestRequestNum.get() < req.requestNum - 1); // Commit version request queued up + CODE_PROBE(proxyItr->second.latestRequestNum.get() < req.requestNum - 1, "Commit version request queued up"); wait(proxyItr->second.latestRequestNum.whenAtLeast(req.requestNum - 1)); auto itr = proxyItr->second.replies.find(req.requestNum); if (itr != proxyItr->second.replies.end()) { - TEST(true); // Duplicate request for sequence + CODE_PROBE(true, "Duplicate request for sequence"); req.reply.send(itr->second); } else if (req.requestNum <= proxyItr->second.latestRequestNum.get()) { - TEST(true); // Old request for previously acknowledged sequence - may be impossible with current FlowTransport + CODE_PROBE(true, + "Old request for previously acknowledged sequence - may be impossible with current FlowTransport"); ASSERT(req.requestNum < proxyItr->second.latestRequestNum.get()); // The latest request can never be acknowledged req.reply.send(Never()); @@ -204,10 +205,10 @@ ACTOR Future getVersion(Reference self, GetCommitVersionReques self->version = self->version + toAdd; } - TEST(self->version - rep.prevVersion == 1); // Minimum possible version gap + CODE_PROBE(self->version - rep.prevVersion == 1, "Minimum possible version gap"); bool maxVersionGap = self->version - rep.prevVersion == SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS; - TEST(maxVersionGap); // Maximum possible version gap + CODE_PROBE(maxVersionGap, "Maximum possible version gap"); self->lastVersionTime = t1; self->resolutionBalancer.setChangesInReply(req.requestingProxy, rep); @@ -408,7 +409,8 @@ ACTOR Future masterServer(MasterInterface mi, addActor.send(serveLiveCommittedVersion(self)); addActor.send(updateRecoveryData(self)); - TEST(!lifetime.isStillValid(db->get().masterLifetime, mi.id() == db->get().master.id())); // Master born doomed + CODE_PROBE(!lifetime.isStillValid(db->get().masterLifetime, mi.id() == db->get().master.id()), + "Master born doomed"); TraceEvent("MasterLifetime", self->dbgid).detail("LifetimeToken", lifetime.toString()); try { @@ -420,7 +422,7 @@ ACTOR Future masterServer(MasterInterface mi, .detail("Reason", "LifetimeToken") .detail("MyToken", lifetime.toString()) .detail("CurrentToken", db->get().masterLifetime.toString()); - TEST(true); // Master replaced, dying + CODE_PROBE(true, "Master replaced, dying"); if (BUGGIFY) wait(delay(5)); throw worker_removed(); @@ -440,11 +442,11 @@ ACTOR Future masterServer(MasterInterface mi, addActor.getFuture().pop(); } - TEST(err.code() == error_code_tlog_failed); // Master: terminated due to tLog failure - TEST(err.code() == error_code_commit_proxy_failed); // Master: terminated due to commit proxy failure - TEST(err.code() == error_code_grv_proxy_failed); // Master: terminated due to GRV proxy failure - TEST(err.code() == error_code_resolver_failed); // Master: terminated due to resolver failure - TEST(err.code() == error_code_backup_worker_failed); // Master: terminated due to backup worker failure + CODE_PROBE(err.code() == error_code_tlog_failed, "Master: terminated due to tLog failure"); + CODE_PROBE(err.code() == error_code_commit_proxy_failed, "Master: terminated due to commit proxy failure"); + CODE_PROBE(err.code() == error_code_grv_proxy_failed, "Master: terminated due to GRV proxy failure"); + CODE_PROBE(err.code() == error_code_resolver_failed, "Master: terminated due to resolver failure"); + CODE_PROBE(err.code() == error_code_backup_worker_failed, "Master: terminated due to backup worker failure"); if (normalMasterErrors().count(err.code())) { TraceEvent("MasterTerminated", mi.id()).error(err); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 9a5ff1975a..c171c610df 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1136,7 +1136,7 @@ public: void checkChangeCounter(uint64_t oldShardChangeCounter, KeyRef const& key) { if (oldShardChangeCounter != shardChangeCounter && shards[key]->changeCounter > oldShardChangeCounter) { - TEST(true); // shard change during getValueQ + CODE_PROBE(true, "shard change during getValueQ"); throw wrong_shard_server(); } } @@ -1146,7 +1146,7 @@ public: auto sh = shards.intersectingRanges(keys); for (auto i = sh.begin(); i != sh.end(); ++i) if (i->value()->changeCounter > oldShardChangeCounter) { - TEST(true); // shard change during range operation + CODE_PROBE(true, "shard change during range operation"); throw wrong_shard_server(); } } @@ -1603,7 +1603,7 @@ ACTOR Future getValueQ(StorageServer* data, GetValueRequest req) { data->counters.kvGetBytes += vv.expectedSize(); // Validate that while we were reading the data we didn't lose the version or shard if (version < data->storageVersion()) { - TEST(true); // transaction_too_old after readValue + CODE_PROBE(true, "transaction_too_old after readValue"); throw transaction_too_old(); } data->checkChangeCounter(changeCounter, req.key); @@ -1708,8 +1708,8 @@ ACTOR Future watchWaitForValueChange(StorageServer* data, SpanContext p try { metadata = data->getWatchMetadata(key); state Version latest = data->version.get(); - TEST(latest >= minVersion && - latest < data->data().latestVersion); // Starting watch loop with latestVersion > data->version + CODE_PROBE(latest >= minVersion && latest < data->data().latestVersion, + "Starting watch loop with latestVersion > data->version"); GetValueRequest getReq( span.context, TenantInfo(), metadata->key, latest, metadata->tags, metadata->debugID, VersionVector()); state Future getValue = getValueQ( @@ -1744,7 +1744,7 @@ ACTOR Future watchWaitForValueChange(StorageServer* data, SpanContext p } if (data->watchBytes > SERVER_KNOBS->MAX_STORAGE_SERVER_WATCH_BYTES) { - TEST(true); // Too many watches, reverting to polling + CODE_PROBE(true, "Too many watches, reverting to polling"); throw watch_cancelled(); } @@ -1775,7 +1775,7 @@ ACTOR Future watchWaitForValueChange(StorageServer* data, SpanContext p throw e; } - TEST(true); // Reading a watched key failed with transaction_too_old + CODE_PROBE(true, "Reading a watched key failed with transaction_too_old"); } watchFuture = data->watches.onChange(metadata->key); @@ -2070,7 +2070,7 @@ ACTOR Future overlappingChangeFeedsQ(StorageServer* data, OverlappingChang // Make sure all of the metadata we are sending won't get rolled back if (metadataVersion != invalidVersion && metadataVersion > data->knownCommittedVersion.get()) { - TEST(true); // overlapping change feeds waiting for metadata version to be committed + CODE_PROBE(true, "overlapping change feeds waiting for metadata version to be committed"); wait(data->desiredOldestVersion.whenAtLeast(metadataVersion)); } req.reply.send(reply); @@ -2318,7 +2318,7 @@ ACTOR Future> getChangeFeedMutations(Stor // To not block fetchKeys making normal SS data readable on making change feed data written to storage, we // wait in here instead for all fetched data to become readable from the storage engine. ASSERT(req.begin <= feedInfo->fetchVersion); - TEST(true); // getChangeFeedMutations before fetched data durable + CODE_PROBE(true, "getChangeFeedMutations before fetched data durable"); // Wait for next commit to write pending feed data to storage wait(feedInfo->durableFetchVersion.whenAtLeast(feedInfo->fetchVersion)); @@ -2438,11 +2438,11 @@ ACTOR Future> getChangeFeedMutations(Stor // If still empty, that means disk results were filtered out, but skipped all memory results. Add an empty, // either the last version from disk if (reply.mutations.empty() && res.size()) { - TEST(true); // Change feed adding empty version after disk + memory filtered + CODE_PROBE(true, "Change feed adding empty version after disk + memory filtered"); reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else if (reply.mutations.empty() || reply.mutations.back().version < lastVersion) { - TEST(true); // Change feed adding empty version after disk filtered + CODE_PROBE(true, "Change feed adding empty version after disk filtered"); reply.mutations.push_back(reply.arena, MutationsAndVersionRef(lastVersion, lastKnownCommitted)); } } else { @@ -2453,7 +2453,7 @@ ACTOR Future> getChangeFeedMutations(Stor Version finalVersion = std::min(req.end - 1, dequeVersion); if ((reply.mutations.empty() || reply.mutations.back().version < finalVersion) && remainingLimitBytes > 0 && remainingDurableBytes > 0) { - TEST(true); // Change feed adding empty version after empty results + CODE_PROBE(true, "Change feed adding empty version after empty results"); reply.mutations.push_back( reply.arena, MutationsAndVersionRef(finalVersion, finalVersion == dequeVersion ? dequeKnownCommit : 0)); // if we add empty mutation after the last thing in memory, and didn't read from disk, gotAll is true @@ -2472,8 +2472,8 @@ ACTOR Future> getChangeFeedMutations(Stor // have happened between req.begin and minVersion. Version minVersion = data->minFeedVersionForAddress(req.reply.getEndpoint().getPrimaryAddress()); bool ok = atLatest && minVersion > feedInfo->emptyVersion; - TEST(ok); // feed popped while valid read waiting - TEST(!ok); // feed popped while invalid read waiting + CODE_PROBE(ok, "feed popped while valid read waiting"); + CODE_PROBE(!ok, "feed popped while invalid read waiting"); if (!ok) { TraceEvent("ChangeFeedMutationsPopped", data->thisServerID) .detail("FeedID", req.rangeID) @@ -2603,7 +2603,7 @@ ACTOR Future localChangeFeedStream(StorageServer* data, } } catch (Error& e) { if (e.code() == error_code_unknown_change_feed) { - TEST(true); // CF was moved away, no more local data to merge with + CODE_PROBE(true, "CF was moved away, no more local data to merge with"); // Send endVersion so local stream is effectively done. We couldn't have send that already, because that // would mean the stream would have finished without error results.send(MutationsAndVersionRef(end, invalidVersion)); @@ -2637,7 +2637,7 @@ ACTOR Future stopChangeFeedOnMove(StorageServer* data, ChangeFeedStreamReq } return Void(); } - TEST(true); // Change feed moved away cancelling queries + CODE_PROBE(true, "Change feed moved away cancelling queries"); // DO NOT call req.reply.onReady before sending - we need to propagate this error through regardless of how far // behind client is req.reply.sendError(wrong_shard_server()); @@ -3289,7 +3289,7 @@ ACTOR Future findKey(StorageServer* data, // If we get only one result in the reverse direction as a result of the data being too large, we could get stuck in // a loop if (more && !forward && rep.data.size() == 1) { - TEST(true); // Reverse key selector returned only one result in range read + CODE_PROBE(true, "Reverse key selector returned only one result in range read"); maxBytes = std::numeric_limits::max(); GetKeyValuesReply rep2 = wait(readRange(data, version, @@ -3330,7 +3330,7 @@ ACTOR Future findKey(StorageServer* data, *pOffset = -*pOffset; if (more) { - TEST(true); // Key selector read range had more results + CODE_PROBE(true, "Key selector read range had more results"); ASSERT(rep.data.size()); Key returnKey = forward ? keyAfter(rep.data.back().key) : rep.data.back().key; @@ -3442,7 +3442,7 @@ ACTOR Future getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req) // end the last actual key returned must be from this shard. A begin offset of 1 is also OK because then either // begin is past end or equal to end (so the result is definitely empty) if ((offset1 && offset1 != 1) || (offset2 && offset2 != 1)) { - TEST(true); // wrong_shard_server due to offset + CODE_PROBE(true, "wrong_shard_server due to offset"); // We could detect when offset1 takes us off the beginning of the database or offset2 takes us off the end, // and return a clipped range rather than an error (since that is what the NativeAPI.getRange will do anyway // via its "slow path"), but we would have to add some flags to the response to encode whether we went off @@ -4176,7 +4176,7 @@ ACTOR Future getMappedKeyValuesQ(StorageServer* data, GetMappedKeyValuesRe // end the last actual key returned must be from this shard. A begin offset of 1 is also OK because then either // begin is past end or equal to end (so the result is definitely empty) if ((offset1 && offset1 != 1) || (offset2 && offset2 != 1)) { - TEST(true); // wrong_shard_server due to offset in getMappedKeyValuesQ + CODE_PROBE(true, "wrong_shard_server due to offset in getMappedKeyValuesQ"); // We could detect when offset1 takes us off the beginning of the database or offset2 takes us off the end, // and return a clipped range rather than an error (since that is what the NativeAPI.getRange will do anyway // via its "slow path"), but we would have to add some flags to the response to encode whether we went off @@ -4369,7 +4369,7 @@ ACTOR Future getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe // end the last actual key returned must be from this shard. A begin offset of 1 is also OK because then either // begin is past end or equal to end (so the result is definitely empty) if ((offset1 && offset1 != 1) || (offset2 && offset2 != 1)) { - TEST(true); // wrong_shard_server due to offset in rangeStream + CODE_PROBE(true, "wrong_shard_server due to offset in rangeStream"); // We could detect when offset1 takes us off the beginning of the database or offset2 takes us off the end, // and return a clipped range rather than an error (since that is what the NativeAPI.getRange will do anyway // via its "slow path"), but we would have to add some flags to the response to encode whether we went off @@ -4744,7 +4744,7 @@ bool convertAtomicOp(MutationRef& m, StorageServer::VersionedData const& data, U if (it != data.atLatest().end() && it->isValue() && it.key() == m.param1) oldVal = it->getValue(); else if (it != data.atLatest().end() && it->isClearTo() && it->getEndKey() > m.param1) { - TEST(true); // Atomic op right after a clear. + CODE_PROBE(true, "Atomic op right after a clear."); } else { Optional& oldThing = eager->getValue(m.param1); if (oldThing.present()) @@ -4891,9 +4891,9 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version .detail("Range", it->range) .detail("ChangeFeedID", it->id); } else { - TEST(version <= it->emptyVersion); // Skip CF write because version <= emptyVersion - TEST(it->removing); // Skip CF write because removing - TEST(version >= it->stopVersion); // Skip CF write because stopped + CODE_PROBE(version <= it->emptyVersion, "Skip CF write because version <= emptyVersion"); + CODE_PROBE(it->removing, "Skip CF write because removing"); + CODE_PROBE(version >= it->stopVersion, "Skip CF write because stopped"); DEBUG_MUTATION("ChangeFeedWriteSetIgnore", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id) @@ -4916,9 +4916,9 @@ void applyChangeFeedMutation(StorageServer* self, MutationRef const& m, Version .detail("Range", it->range) .detail("ChangeFeedID", it->id); } else { - TEST(version <= it->emptyVersion); // Skip CF clear because version <= emptyVersion - TEST(it->removing); // Skip CF clear because removing - TEST(version >= it->stopVersion); // Skip CF clear because stopped + CODE_PROBE(version <= it->emptyVersion, "Skip CF clear because version <= emptyVersion"); + CODE_PROBE(it->removing, "Skip CF clear because removing"); + CODE_PROBE(version >= it->stopVersion, "Skip CF clear because stopped"); DEBUG_MUTATION("ChangeFeedWriteClearIgnore", version, m, self->thisServerID) .detail("Range", it->range) .detail("ChangeFeedID", it->id) @@ -5231,7 +5231,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, ASSERT(startVersion >= 0); if (startVersion >= endVersion || (changeFeedInfo->removing)) { - TEST(true); // Change Feed popped before fetch + CODE_PROBE(true, "Change Feed popped before fetch"); TraceEvent(SevDebug, "FetchChangeFeedNoOp", data->thisServerID) .detail("RangeID", rangeId) .detail("Range", range) @@ -5272,7 +5272,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, while (remoteLoc < remoteResult.size()) { if (feedResults->popVersion - 1 > changeFeedInfo->emptyVersion) { - TEST(true); // CF fetched updated popped version from src SS + CODE_PROBE(true, "CF fetched updated popped version from src SS"); changeFeedInfo->emptyVersion = feedResults->popVersion - 1; // pop mutations while (!changeFeedInfo->mutations.empty() && @@ -5340,7 +5340,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, lastVersion = remoteVersion; versionsFetched++; } else { - TEST(true); // Change feed ignoring write on move because it was popped concurrently + CODE_PROBE(true, "Change feed ignoring write on move because it was popped concurrently"); if (MUTATION_TRACKING_ENABLED) { for (auto& m : remoteResult[remoteLoc].mutations) { DEBUG_MUTATION("ChangeFeedWriteMoveIgnore", remoteVersion, m, data->thisServerID) @@ -5394,7 +5394,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, } if (feedResults->popVersion - 1 > changeFeedInfo->emptyVersion) { - TEST(true); // CF fetched updated popped version from src SS at end + CODE_PROBE(true, "CF fetched updated popped version from src SS at end"); changeFeedInfo->emptyVersion = feedResults->popVersion - 1; while (!changeFeedInfo->mutations.empty() && changeFeedInfo->mutations.front().version <= changeFeedInfo->emptyVersion) { @@ -5416,7 +5416,7 @@ ACTOR Future fetchChangeFeedApplier(StorageServer* data, // if we were popped or removed while fetching but it didn't pass the fetch version while writing, clean up here if (versionsFetched > 0 && startVersion < changeFeedInfo->emptyVersion) { - TEST(true); // Change feed cleaning up popped data after move + CODE_PROBE(true, "Change feed cleaning up popped data after move"); ASSERT(firstVersion != invalidVersion); ASSERT(lastVersion != invalidVersion); Version endClear = std::min(lastVersion + 1, changeFeedInfo->emptyVersion); @@ -5462,7 +5462,7 @@ ACTOR Future fetchChangeFeed(StorageServer* data, auto cleanupPending = data->changeFeedCleanupDurable.find(changeFeedInfo->id); if (cleanupPending != data->changeFeedCleanupDurable.end()) { - TEST(true); // Change feed waiting for dirty previous move to finish + CODE_PROBE(true, "Change feed waiting for dirty previous move to finish"); TraceEvent(SevDebug, "FetchChangeFeedWaitCleanup", data->thisServerID) .detail("RangeID", changeFeedInfo->id) .detail("Range", changeFeedInfo->range) @@ -5586,7 +5586,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, for (auto& cfInfo : r.value()) { auto feedCleanup = data->changeFeedCleanupDurable.find(cfInfo->id); if (feedCleanup != data->changeFeedCleanupDurable.end() && cfInfo->removing && !cfInfo->destroyed) { - TEST(true); // re-fetching feed scheduled for deletion! Un-mark it as removing + CODE_PROBE(true, "re-fetching feed scheduled for deletion! Un-mark it as removing"); destroyedFeedIds.insert(cfInfo->id); cfInfo->removing = false; @@ -5608,7 +5608,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, .detail("StopVersion", cfInfo->stopVersion) .detail("FKID", fetchKeysID); } else if (cfInfo->refreshInProgress) { - TEST(true); // Racing refreshes for same change feed in fetch + CODE_PROBE(true, "Racing refreshes for same change feed in fetch"); destroyedFeedIds.insert(cfInfo->id); } } @@ -5655,8 +5655,9 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, } if (!existing) { - TEST(cleanupPending); // Fetch change feed which is cleanup pending. This means there was a move away and a - // move back, this will remake the metadata + CODE_PROBE(cleanupPending, + "Fetch change feed which is cleanup pending. This means there was a move away and a move back, " + "this will remake the metadata"); changeFeedInfo = Reference(new ChangeFeedInfo()); changeFeedInfo->range = cfEntry.range; @@ -5685,14 +5686,14 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, // this actor would have been cancelled if a later remove happened ASSERT(!changeFeedInfo->removing); if (cfEntry.stopVersion < changeFeedInfo->stopVersion) { - TEST(true); // Change feed updated stop version from fetch metadata + CODE_PROBE(true, "Change feed updated stop version from fetch metadata"); changeFeedInfo->stopVersion = cfEntry.stopVersion; addMutationToLog = true; } // don't update empty version past SS version if SS is behind, it can cause issues if (cfEntry.emptyVersion < data->version.get() && cfEntry.emptyVersion > changeFeedInfo->emptyVersion) { - TEST(true); // Change feed updated empty version from fetch metadata + CODE_PROBE(true, "Change feed updated empty version from fetch metadata"); changeFeedInfo->emptyVersion = cfEntry.emptyVersion; addMutationToLog = true; } @@ -5714,13 +5715,13 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, } } - TEST(!refreshedFeedIds.empty()); // Feed refreshed between move away and move back - TEST(!destroyedFeedIds.empty()); // Feed destroyed between move away and move back + CODE_PROBE(!refreshedFeedIds.empty(), "Feed refreshed between move away and move back"); + CODE_PROBE(!destroyedFeedIds.empty(), "Feed destroyed between move away and move back"); for (auto& feedId : refreshedFeedIds) { auto existingEntry = data->uidChangeFeed.find(feedId); if (existingEntry == data->uidChangeFeed.end() || existingEntry->second->destroyed || !existingEntry->second->refreshInProgress) { - TEST(true); // feed refreshed + CODE_PROBE(true, "feed refreshed"); continue; } @@ -5753,7 +5754,7 @@ ACTOR Future> fetchChangeFeedMetadata(StorageServer* data, for (auto& feedId : destroyedFeedIds) { auto existingEntry = data->uidChangeFeed.find(feedId); if (existingEntry == data->uidChangeFeed.end() || existingEntry->second->destroyed) { - TEST(true); // feed refreshed but then destroyed elsewhere + CODE_PROBE(true, "feed refreshed but then destroyed elsewhere"); continue; } @@ -5927,7 +5928,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { lastAvailable = std::max(lastAvailable, r->value()); if (lastAvailable != invalidVersion && lastAvailable >= data->durableVersion.get()) { - TEST(true); // FetchKeys waits for previous available version to be durable + CODE_PROBE(true, "FetchKeys waits for previous available version to be durable"); wait(data->durableVersion.whenAtLeast(lastAvailable + 1)); } @@ -6005,7 +6006,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { try { loop { - TEST(true); // Fetching keys for transferred shard + CODE_PROBE(true, "Fetching keys for transferred shard"); while (data->fetchKeysBudgetUsed.get()) { wait(data->fetchKeysBudgetUsed.onChange()); } @@ -6113,8 +6114,8 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { splitMutations(data, data->shards, *u); } - TEST(true); // fetchkeys has more - TEST(shard->updates.size()); // Shard has updates + CODE_PROBE(true, "fetchkeys has more"); + CODE_PROBE(shard->updates.size(), "Shard has updates"); ASSERT(otherShard->updates.empty()); } break; @@ -6183,7 +6184,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { std::unordered_set newChangeFeeds; for (auto& r : ranges) { for (auto& cfInfo : r.value()) { - TEST(true); // SS fetching new change feed that didn't exist when fetch started + CODE_PROBE(true, "SS fetching new change feed that didn't exist when fetch started"); if (!cfInfo->removing) { newChangeFeeds.insert(cfInfo->id); } @@ -6225,7 +6226,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { } int startSize = batch->changes.size(); - TEST(startSize); // Adding fetch data to a batch which already has changes + CODE_PROBE(startSize, "Adding fetch data to a batch which already has changes"); batch->changes.resize(batch->changes.size() + shard->updates.size()); // FIXME: pass the deque back rather than copy the data @@ -6495,7 +6496,7 @@ void changeServerKeys(StorageServer* data, else { ASSERT(ranges[i].value->adding); data->addShard(ShardInfo::newAdding(data, ranges[i])); - TEST(true); // ChangeServerKeys reFetchKeys + CODE_PROBE(true, "ChangeServerKeys reFetchKeys"); } } @@ -6590,7 +6591,7 @@ void changeServerKeys(StorageServer* data, } void rollback(StorageServer* data, Version rollbackVersion, Version nextVersion) { - TEST(true); // call to shard rollback + CODE_PROBE(true, "call to shard rollback"); DEBUG_KEY_RANGE("Rollback", rollbackVersion, allKeys, data->thisServerID); // We used to do a complicated dance to roll back in MVCC history. It's much simpler, and more testable, @@ -6742,7 +6743,7 @@ private: br >> rollbackVersion; if (rollbackVersion < fromVersion && rollbackVersion > restoredVersion) { - TEST(true); // ShardApplyPrivateData shard rollback + CODE_PROBE(true, "ShardApplyPrivateData shard rollback"); TraceEvent(SevWarn, "Rollback", data->thisServerID) .detail("FromVersion", fromVersion) .detail("ToVersion", rollbackVersion) @@ -6835,7 +6836,7 @@ private: // for a create, the empty version should be now, otherwise it will be set in a later pop changeFeedInfo->emptyVersion = currentVersion - 1; } else { - TEST(true); // SS got non-create change feed private mutation before move created its metadata + CODE_PROBE(true, "SS got non-create change feed private mutation before move created its metadata"); changeFeedInfo->emptyVersion = invalidVersion; } changeFeedInfo->metadataCreateVersion = currentVersion; @@ -6975,7 +6976,7 @@ private: UID ssId = decodeTssQuarantineKey(m.param1.substr(1)); ASSERT(ssId == data->thisServerID); if (m.type == MutationRef::SetValue) { - TEST(true); // Putting TSS in quarantine + CODE_PROBE(true, "Putting TSS in quarantine"); TraceEvent(SevWarn, "TSSQuarantineStart", data->thisServerID).log(); data->startTssQuarantine(); } else { @@ -7291,8 +7292,9 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { wait(doEagerReads(data, &eager)); if (data->shardChangeCounter == changeCounter) break; - TEST(true); // A fetchKeys completed while we were doing this, so eager might be outdated. Read it - // again. + CODE_PROBE( + true, + "A fetchKeys completed while we were doing this, so eager might be outdated. Read it again."); // SOMEDAY: Theoretically we could check the change counters of individual shards and retry the reads // only selectively eager = UpdateEagerReadInfo(); @@ -7376,14 +7378,14 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { } else if (rd.protocolVersion().hasSpanContext() && SpanContextMessage::isNextIn(rd)) { SpanContextMessage scm; rd >> scm; - TEST(true); // storageserveractor converting SpanContextMessage into OTEL SpanContext + CODE_PROBE(true, "storageserveractor converting SpanContextMessage into OTEL SpanContext"); spanContext = SpanContext(UID(scm.spanContext.first(), scm.spanContext.second()), 0, scm.spanContext.first() != 0 && scm.spanContext.second() != 0 ? TraceFlags::sampled : TraceFlags::unsampled); } else if (rd.protocolVersion().hasOTELSpanContext() && OTELSpanContextMessage::isNextIn(rd)) { - TEST(true); // storageserveractor reading OTELSpanContextMessage + CODE_PROBE(true, "storageserveractor reading OTELSpanContextMessage"); OTELSpanContextMessage scm; rd >> scm; spanContext = scm.spanContext; @@ -7869,7 +7871,7 @@ ACTOR Future updateStorage(StorageServer* data) { data->uidChangeFeed.erase(feed); } else { - TEST(true); // Feed re-fetched after remove + CODE_PROBE(true, "Feed re-fetched after remove"); } cfCleanup = data->changeFeedCleanupDurable.erase(cfCleanup); } else { @@ -8294,7 +8296,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor data->clusterId.send(BinaryReader::fromStringRef(fClusterID.get().get(), Unversioned())); data->bytesRestored += fClusterID.get().expectedSize(); } else { - TEST(true); // storage server upgraded to version supporting cluster IDs + CODE_PROBE(true, "storage server upgraded to version supporting cluster IDs"); data->actors.add(persistClusterId(data)); } @@ -8302,7 +8304,7 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor // state means the storage engine already had a durability or correctness error, but it should get // re-quarantined very quickly because of a mismatch if it starts trying to do things again if (fTssQuarantine.get().present()) { - TEST(true); // TSS restarted while quarantined + CODE_PROBE(true, "TSS restarted while quarantined"); data->tssInQuarantine = true; data->bytesRestored += fTssQuarantine.get().expectedSize(); } @@ -8577,7 +8579,7 @@ ACTOR Future waitMetrics(StorageServerMetrics* self, WaitMetricsRequest re state bool timedout = false; if (!req.min.allLessOrEqual(metrics) || !metrics.allLessOrEqual(req.max)) { - TEST(true); // ShardWaitMetrics return case 1 (quickly) + CODE_PROBE(true, "ShardWaitMetrics return case 1 (quickly)"); req.reply.send(metrics); return Void(); } @@ -8618,7 +8620,7 @@ ACTOR Future waitMetrics(StorageServerMetrics* self, WaitMetricsRequest re } if (timedout) { - TEST(true); // ShardWaitMetrics return on timeout + CODE_PROBE(true, "ShardWaitMetrics return on timeout"); // FIXME: instead of using random chance, send wrong_shard_server when the call in from // waitMetricsMultiple (requires additional information in the request) if (deterministicRandom()->random01() < SERVER_KNOBS->WAIT_METRICS_WRONG_SHARD_CHANCE) { @@ -8630,7 +8632,7 @@ ACTOR Future waitMetrics(StorageServerMetrics* self, WaitMetricsRequest re } if (!req.min.allLessOrEqual(metrics) || !metrics.allLessOrEqual(req.max)) { - TEST(true); // ShardWaitMetrics return case 2 (delayed) + CODE_PROBE(true, "ShardWaitMetrics return case 2 (delayed)"); req.reply.send(metrics); break; } @@ -8654,7 +8656,7 @@ ACTOR Future waitMetrics(StorageServerMetrics* self, WaitMetricsRequest re if (error.code() != error_code_success) { if (error.code() != error_code_wrong_shard_server) throw error; - TEST(true); // ShardWaitMetrics delayed wrong_shard_server() + CODE_PROBE(true, "ShardWaitMetrics delayed wrong_shard_server()"); req.reply.sendError(error); } @@ -8712,7 +8714,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) choose { when(WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) { if (!self->isReadable(req.keys)) { - TEST(true); // waitMetrics immediate wrong_shard_server() + CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { self->actors.add( @@ -8721,7 +8723,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) } when(SplitMetricsRequest req = waitNext(ssi.splitMetrics.getFuture())) { if (!self->isReadable(req.keys)) { - TEST(true); // splitMetrics immediate wrong_shard_server() + CODE_PROBE(true, "splitMetrics immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { self->metrics.splitMetrics(req); @@ -8734,7 +8736,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) } when(ReadHotSubRangeRequest req = waitNext(ssi.getReadHotRanges.getFuture())) { if (!self->isReadable(req.keys)) { - TEST(true); // readHotSubRanges immediate wrong_shard_server() + CODE_PROBE(true, "readHotSubRanges immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { self->metrics.getReadHotRanges(req); @@ -8742,7 +8744,7 @@ ACTOR Future metricsCore(StorageServer* self, StorageServerInterface ssi) } when(SplitRangeRequest req = waitNext(ssi.getRangeSplitPoints.getFuture())) { if (!self->isReadable(req.keys)) { - TEST(true); // getSplitPoints immediate wrong_shard_server() + CODE_PROBE(true, "getSplitPoints immediate wrong_shard_server()"); self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty()); } else { self->getSplitPoints(req); @@ -8913,12 +8915,12 @@ ACTOR Future serveWatchValueRequestsImpl(StorageServer* self, FutureStream } // case 4: version in the map is higher so immediately trigger watch else if (req.version < metadata->version) { - TEST(true); // watch version in map is higher so trigger watch (case 4) + CODE_PROBE(true, "watch version in map is higher so trigger watch (case 4)"); req.reply.send(WatchValueReply{ metadata->version }); } // case 5: watch value differs but their versions are the same (rare case) so check with the SS else { - TEST(true); // watch version in the map is the same but value is different (case 5) + CODE_PROBE(true, "watch version in the map is the same but value is different (case 5)"); loop { try { state Version latest = self->version.get(); @@ -8958,7 +8960,7 @@ ACTOR Future serveWatchValueRequestsImpl(StorageServer* self, FutureStream self->sendErrorWithPenalty(req.reply, e, self->getPenalty()); break; } - TEST(true); // Reading a watched key failed with transaction_too_old case 5 + CODE_PROBE(true, "Reading a watched key failed with transaction_too_old case 5"); } } } @@ -9098,7 +9100,7 @@ ACTOR Future storageServerCore(StorageServer* self, StorageServerInterface } } when(wait(dbInfoChange)) { - TEST(self->logSystem); // shardServer dbInfo changed + CODE_PROBE(self->logSystem, "shardServer dbInfo changed"); dbInfoChange = self->db->onChange(); if (self->db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS) { self->logSystem = ILogSystem::fromServerDBInfo(self->thisServerID, self->db->get()); @@ -9223,8 +9225,9 @@ ACTOR Future memoryStoreRecover(IKeyValueStore* store, ReferenceREMOVE_RETRY_DELAY, TaskPriority::UpdateStorage)); tr->reset(); TraceEvent("RemoveStorageServerRetrying") @@ -9377,7 +9380,7 @@ ACTOR Future replaceTSSInterface(StorageServer* self, StorageServerInterfa Optional pairTagValue = wait(tr->get(serverTagKeyFor(self->tssPairID.get()))); if (!pairTagValue.present()) { - TEST(true); // Race where tss was down, pair was removed, tss starts back up + CODE_PROBE(true, "Race where tss was down, pair was removed, tss starts back up"); TraceEvent("StorageServerWorkerRemoved", self->thisServerID).detail("Reason", "TssPairMissing"); throw worker_removed(); } diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 506c5777de..1fc141bfdc 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -193,7 +193,7 @@ Error checkIOTimeout(Error const& e) { timeoutOccurred = g_pSimulator->getCurrentProcess()->machine->machineProcess->global(INetwork::enASIOTimedOut); if (timeoutOccurred) { - TEST(true); // Timeout occurred + CODE_PROBE(true, "Timeout occurred"); Error timeout = io_timeout(); // Preserve injectedness of error if (e.isInjectedFault()) @@ -244,7 +244,7 @@ ACTOR Future handleIOErrors(Future actor, IClosable* store, UID id, // file_not_found can occur due to attempting to open a partially deleted DiskQueue, which should not be // reported SevError. if (e.isError() && e.getError().code() == error_code_file_not_found) { - TEST(true); // Worker terminated with file_not_found error + CODE_PROBE(true, "Worker terminated with file_not_found error"); return Void(); } throw e.isError() ? e.getError() : actor_cancelled(); @@ -2044,7 +2044,7 @@ ACTOR Future workerServer(Reference connRecord, if (ddInterf->get().present()) { recruited = ddInterf->get().get(); - TEST(true); // Recruited while already a data distributor. + CODE_PROBE(true, "Recruited while already a data distributor."); } else { startRole(Role::DATA_DISTRIBUTOR, recruited.id(), interf.id()); DUMPTOKEN(recruited.waitFailure); @@ -2068,7 +2068,7 @@ ACTOR Future workerServer(Reference connRecord, if (rkInterf->get().present()) { recruited = rkInterf->get().get(); - TEST(true); // Recruited while already a ratekeeper. + CODE_PROBE(true, "Recruited while already a ratekeeper."); } else { startRole(Role::RATEKEEPER, recruited.id(), interf.id()); DUMPTOKEN(recruited.waitFailure); @@ -2096,7 +2096,7 @@ ACTOR Future workerServer(Reference connRecord, if (bmEpochAndInterf->get().present() && bmEpochAndInterf->get().get().first == req.epoch) { recruited = bmEpochAndInterf->get().get().second; - TEST(true); // Recruited while already a blob manager. + CODE_PROBE(true, "Recruited while already a blob manager."); } else { // TODO: it'd be more optimal to halt the last manager if present here, but it will figure it out // via the epoch check @@ -2151,7 +2151,7 @@ ACTOR Future workerServer(Reference connRecord, if (ekpInterf->get().present()) { recruited = ekpInterf->get().get(); - TEST(true); // Recruited while already a encryptKeyProxy server. + CODE_PROBE(true, "Recruited while already a encryptKeyProxy server."); } else { startRole(Role::ENCRYPT_KEY_PROXY, recruited.id(), interf.id()); DUMPTOKEN(recruited.waitFailure); @@ -2545,7 +2545,7 @@ ACTOR Future workerServer(Reference connRecord, when(state WorkerSnapRequest snapReq = waitNext(interf.workerSnapReq.getFuture())) { std::string snapUID = snapReq.snapUID.toString() + snapReq.role.toString(); if (snapReqResultMap.count(snapUID)) { - TEST(true); // Worker received a duplicate finished snap request + CODE_PROBE(true, "Worker received a duplicate finished snap request"); auto result = snapReqResultMap[snapUID]; result.isError() ? snapReq.reply.sendError(result.getError()) : snapReq.reply.send(result.get()); TraceEvent("RetryFinishedWorkerSnapRequest") @@ -2553,7 +2553,7 @@ ACTOR Future workerServer(Reference connRecord, .detail("Role", snapReq.role) .detail("Result", result.isError() ? result.getError().code() : 0); } else if (snapReqMap.count(snapUID)) { - TEST(true); // Worker received a duplicate ongoing snap request + CODE_PROBE(true, "Worker received a duplicate ongoing snap request"); TraceEvent("RetryOngoingWorkerSnapRequest").detail("SnapUID", snapUID).detail("Role", snapReq.role); ASSERT(snapReq.role == snapReqMap[snapUID].role); ASSERT(snapReq.snapPayload == snapReqMap[snapUID].snapPayload); diff --git a/fdbserver/workloads/AtomicOps.actor.cpp b/fdbserver/workloads/AtomicOps.actor.cpp index 2cc2f1020f..15b21d85ab 100644 --- a/fdbserver/workloads/AtomicOps.actor.cpp +++ b/fdbserver/workloads/AtomicOps.actor.cpp @@ -58,47 +58,47 @@ struct AtomicOpsWorkload : TestWorkload { switch (opType) { case 0: - TEST(true); // Testing atomic AddValue + CODE_PROBE(true, "Testing atomic AddValue"); opType = MutationRef::AddValue; break; case 1: - TEST(true); // Testing atomic And + CODE_PROBE(true, "Testing atomic And"); opType = MutationRef::And; break; case 2: - TEST(true); // Testing atomic Or + CODE_PROBE(true, "Testing atomic Or"); opType = MutationRef::Or; break; case 3: - TEST(true); // Testing atomic Xor + CODE_PROBE(true, "Testing atomic Xor"); opType = MutationRef::Xor; break; case 4: - TEST(true); // Testing atomic Max + CODE_PROBE(true, "Testing atomic Max"); opType = MutationRef::Max; break; case 5: - TEST(true); // Testing atomic Min + CODE_PROBE(true, "Testing atomic Min"); opType = MutationRef::Min; break; case 6: - TEST(true); // Testing atomic ByteMin + CODE_PROBE(true, "Testing atomic ByteMin"); opType = MutationRef::ByteMin; break; case 7: - TEST(true); // Testing atomic ByteMax + CODE_PROBE(true, "Testing atomic ByteMax"); opType = MutationRef::ByteMax; break; case 8: - TEST(true); // Testing atomic MinV2 + CODE_PROBE(true, "Testing atomic MinV2"); opType = MutationRef::MinV2; break; case 9: - TEST(true); // Testing atomic AndV2 + CODE_PROBE(true, "Testing atomic AndV2"); opType = MutationRef::AndV2; break; // case 10: - // TEST(true); // Testing atomic CompareAndClear Not supported yet + // CODE_PROBE(true, "Testing atomic CompareAndClear Not supported yet"); // opType = MutationRef::CompareAndClear // break; default: diff --git a/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp b/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp index 4acdb7eddc..e116ec6287 100644 --- a/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp +++ b/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp @@ -53,31 +53,31 @@ public: switch (opType) { case 0: - TEST(true); // Testing atomic Min + CODE_PROBE(true, "Testing atomic Min"); return testMin(cx->clone(), this); case 1: - TEST(true); // Testing atomic And + CODE_PROBE(true, "Testing atomic And"); return testAnd(cx->clone(), this); case 2: - TEST(true); // Testing atomic ByteMin + CODE_PROBE(true, "Testing atomic ByteMin"); return testByteMin(cx->clone(), this); case 3: - TEST(true); // Testing atomic ByteMax + CODE_PROBE(true, "Testing atomic ByteMax"); return testByteMax(cx->clone(), this); case 4: - TEST(true); // Testing atomic Or + CODE_PROBE(true, "Testing atomic Or"); return testOr(cx->clone(), this); case 5: - TEST(true); // Testing atomic Max + CODE_PROBE(true, "Testing atomic Max"); return testMax(cx->clone(), this); case 6: - TEST(true); // Testing atomic Xor + CODE_PROBE(true, "Testing atomic Xor"); return testXor(cx->clone(), this); case 7: - TEST(true); // Testing atomic Add + CODE_PROBE(true, "Testing atomic Add"); return testAdd(cx->clone(), this); case 8: - TEST(true); // Testing atomic CompareAndClear + CODE_PROBE(true, "Testing atomic CompareAndClear"); return testCompareAndClear(cx->clone(), this); default: ASSERT(false); diff --git a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp index c1bcd5c5d0..749a652907 100644 --- a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp @@ -239,7 +239,8 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { // Stop the differential backup, if enabled if (stopDifferentialDelay) { - TEST(!stopDifferentialFuture.isReady()); // Restore starts at specified time - stopDifferential not ready + CODE_PROBE(!stopDifferentialFuture.isReady(), + "Restore starts at specified time - stopDifferential not ready"); wait(stopDifferentialFuture); TraceEvent("BARW_DoBackupWaitToDiscontinue", randomID) .detail("Tag", printable(tag)) @@ -496,7 +497,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { } } - TEST(!startRestore.isReady()); // Restore starts at specified time + CODE_PROBE(!startRestore.isReady(), "Restore starts at specified time"); wait(startRestore); if (lastBackupContainer && self->performRestore) { diff --git a/fdbserver/workloads/BackupCorrectness.actor.cpp b/fdbserver/workloads/BackupCorrectness.actor.cpp index 595eca5470..0be64bd065 100644 --- a/fdbserver/workloads/BackupCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupCorrectness.actor.cpp @@ -285,7 +285,8 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { // Stop the differential backup, if enabled if (stopDifferentialDelay) { - TEST(!stopDifferentialFuture.isReady()); // Restore starts at specified time - stopDifferential not ready + CODE_PROBE(!stopDifferentialFuture.isReady(), + "Restore starts at specified time - stopDifferential not ready"); wait(stopDifferentialFuture); TraceEvent("BARW_DoBackupWaitToDiscontinue", randomID) .detail("Tag", printable(tag)) @@ -540,7 +541,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { } } - TEST(!startRestore.isReady()); // Restore starts at specified time + CODE_PROBE(!startRestore.isReady(), "Restore starts at specified time"); wait(startRestore); if (lastBackupContainer && self->performRestore) { diff --git a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp index 415744a677..cdd0d8ae70 100644 --- a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp @@ -327,7 +327,8 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { // Stop the differential backup, if enabled if (stopDifferentialDelay) { - TEST(!stopDifferentialFuture.isReady()); // Restore starts at specified time - stopDifferential not ready + CODE_PROBE(!stopDifferentialFuture.isReady(), + "Restore starts at specified time - stopDifferential not ready"); wait(stopDifferentialFuture); TraceEvent("BARW_DoBackupWaitToDiscontinue", randomID) .detail("Tag", printable(tag)) @@ -632,7 +633,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { } } - TEST(!startRestore.isReady()); // Restore starts at specified time + CODE_PROBE(!startRestore.isReady(), "Restore starts at specified time"); wait(startRestore); if (self->performRestore) { diff --git a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp index b73b135e83..e1d2aa3dce 100644 --- a/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp +++ b/fdbserver/workloads/BlobGranuleCorrectnessWorkload.actor.cpp @@ -452,10 +452,10 @@ struct BlobGranuleCorrectnessWorkload : TestWorkload { } } } - TEST(beginCollapsed > 0); // BGCorrectness got collapsed request with beginVersion > 0 - TEST(beginNotCollapsed > 0); // BGCorrectness got un-collapsed request with beginVersion > 0 - TEST(beginCollapsed > 0 && - beginNotCollapsed > 0); // BGCorrectness got both collapsed and uncollapsed in the same request! + CODE_PROBE(beginCollapsed > 0, "BGCorrectness got collapsed request with beginVersion > 0"); + CODE_PROBE(beginNotCollapsed > 0, "BGCorrectness got un-collapsed request with beginVersion > 0"); + CODE_PROBE(beginCollapsed > 0 && beginNotCollapsed > 0, + "BGCorrectness got both collapsed and uncollapsed in the same request!"); while (checkIt != threadData->keyData.end() && checkIt->first < endKeyExclusive) { uint32_t key = checkIt->first; diff --git a/fdbserver/workloads/CommitBugCheck.actor.cpp b/fdbserver/workloads/CommitBugCheck.actor.cpp index 4da04cd7bf..d5a5f22c11 100644 --- a/fdbserver/workloads/CommitBugCheck.actor.cpp +++ b/fdbserver/workloads/CommitBugCheck.actor.cpp @@ -49,7 +49,7 @@ struct CommitBugWorkload : TestWorkload { break; } catch (Error& e) { TraceEvent("CommitBugSetVal1Error").error(e); - TEST(e.code() == error_code_commit_unknown_result); // Commit unknown result + CODE_PROBE(e.code() == error_code_commit_unknown_result, "Commit unknown result"); wait(tr.onError(e)); } } @@ -140,7 +140,7 @@ struct CommitBugWorkload : TestWorkload { break; } else { - TEST(true); // Commit conflict + CODE_PROBE(true, "Commit conflict"); TraceEvent("CommitBug2Error").error(e).detail("AttemptedNum", i + 1); wait(tr.onError(e)); diff --git a/fdbserver/workloads/ConfigureDatabase.actor.cpp b/fdbserver/workloads/ConfigureDatabase.actor.cpp index 2b60f6d2f2..6ead37daaf 100644 --- a/fdbserver/workloads/ConfigureDatabase.actor.cpp +++ b/fdbserver/workloads/ConfigureDatabase.actor.cpp @@ -124,17 +124,17 @@ std::string generateRegions() { int satellite_replication_type = deterministicRandom()->randomInt(0, 3); switch (satellite_replication_type) { case 0: { - TEST(true); // Simulated cluster using no satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using no satellite redundancy mode"); break; } case 1: { - TEST(true); // Simulated cluster using two satellite fast redundancy mode + CODE_PROBE(true, "Simulated cluster using two satellite fast redundancy mode"); primaryObj["satellite_redundancy_mode"] = "two_satellite_fast"; remoteObj["satellite_redundancy_mode"] = "two_satellite_fast"; break; } case 2: { - TEST(true); // Simulated cluster using two satellite safe redundancy mode + CODE_PROBE(true, "Simulated cluster using two satellite safe redundancy mode"); primaryObj["satellite_redundancy_mode"] = "two_satellite_safe"; remoteObj["satellite_redundancy_mode"] = "two_satellite_safe"; break; @@ -147,21 +147,21 @@ std::string generateRegions() { switch (satellite_replication_type) { case 0: { // FIXME: implement - TEST(true); // Simulated cluster using custom satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using custom satellite redundancy mode"); break; } case 1: { - TEST(true); // Simulated cluster using no satellite redundancy mode (<5 datacenters) + CODE_PROBE(true, "Simulated cluster using no satellite redundancy mode (<5 datacenters)"); break; } case 2: { - TEST(true); // Simulated cluster using single satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using single satellite redundancy mode"); primaryObj["satellite_redundancy_mode"] = "one_satellite_single"; remoteObj["satellite_redundancy_mode"] = "one_satellite_single"; break; } case 3: { - TEST(true); // Simulated cluster using double satellite redundancy mode + CODE_PROBE(true, "Simulated cluster using double satellite redundancy mode"); primaryObj["satellite_redundancy_mode"] = "one_satellite_double"; remoteObj["satellite_redundancy_mode"] = "one_satellite_double"; break; @@ -180,20 +180,20 @@ std::string generateRegions() { switch (remote_replication_type) { case 0: { // FIXME: implement - TEST(true); // Simulated cluster using custom remote redundancy mode + CODE_PROBE(true, "Simulated cluster using custom remote redundancy mode"); break; } case 1: { - TEST(true); // Simulated cluster using default remote redundancy mode + CODE_PROBE(true, "Simulated cluster using default remote redundancy mode"); break; } case 2: { - TEST(true); // Simulated cluster using single remote redundancy mode + CODE_PROBE(true, "Simulated cluster using single remote redundancy mode"); result += " remote_single"; break; } case 3: { - TEST(true); // Simulated cluster using double remote redundancy mode + CODE_PROBE(true, "Simulated cluster using double remote redundancy mode"); result += " remote_double"; break; } @@ -437,7 +437,7 @@ struct ConfigureDatabaseWorkload : TestWorkload { false))); } else if (randomChoice == 8) { if (self->allowTestStorageMigration) { - TEST(true); // storage migration type change + CODE_PROBE(true, "storage migration type change"); // randomly configuring perpetual_storage_wiggle_locality state std::string randomPerpetualWiggleLocality; diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 83479c2beb..4ba610a2ce 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -206,8 +206,8 @@ struct ConsistencyCheckWorkload : TestWorkload { } ACTOR Future runCheck(Database cx, ConsistencyCheckWorkload* self) { - TEST(self->performQuiescentChecks); // Quiescent consistency check - TEST(!self->performQuiescentChecks); // Non-quiescent consistency check + CODE_PROBE(self->performQuiescentChecks, "Quiescent consistency check"); + CODE_PROBE(!self->performQuiescentChecks, "Non-quiescent consistency check"); if (self->firstClient || self->distributed) { try { @@ -1275,7 +1275,7 @@ struct ConsistencyCheckWorkload : TestWorkload { for (int i = 0; i < initialSize; i++) { auto tssPair = tssMapping.find(storageServers[i]); if (tssPair != tssMapping.end()) { - TEST(true); // TSS checked in consistency check + CODE_PROBE(true, "TSS checked in consistency check"); storageServers.push_back(tssPair->second.id()); storageServerInterfaces.push_back(tssPair->second); } diff --git a/fdbserver/workloads/Cycle.actor.cpp b/fdbserver/workloads/Cycle.actor.cpp index 6d2fb054f3..5407411cd1 100644 --- a/fdbserver/workloads/Cycle.actor.cpp +++ b/fdbserver/workloads/Cycle.actor.cpp @@ -261,7 +261,7 @@ struct CycleWorkload : TestWorkload { retryCount++; TraceEvent(retryCount > 20 ? SevWarnAlways : SevWarn, "CycleCheckError").error(e); if (g_network->isSimulated() && retryCount > 50) { - TEST(true); // Cycle check enable speedUpSimulation because too many transaction_too_old() + CODE_PROBE(true, "Cycle check enable speedUpSimulation because too many transaction_too_old()"); // try to make the read window back to normal size (5 * version_per_sec) g_simulator.speedUpSimulation = true; } diff --git a/fdbserver/workloads/DataDistributionMetrics.actor.cpp b/fdbserver/workloads/DataDistributionMetrics.actor.cpp index 4cb59c5768..63b1cde4ff 100644 --- a/fdbserver/workloads/DataDistributionMetrics.actor.cpp +++ b/fdbserver/workloads/DataDistributionMetrics.actor.cpp @@ -151,7 +151,7 @@ struct DataDistributionMetricsWorkload : KVWorkload { ASSERT(result[i].key.startsWith(ddStatsRange.begin)); std::string errorStr; auto valueObj = readJSONStrictly(result[i].value.toString()).get_obj(); - TEST(true); // data_distribution_stats schema validation + CODE_PROBE(true, "data_distribution_stats schema validation"); if (!schemaMatch(schema, valueObj, errorStr, SevError, true)) { TraceEvent(SevError, "DataDistributionStatsSchemaValidationFailed") .detail("ErrorStr", errorStr.c_str()) diff --git a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp index 9598607830..35db37c671 100644 --- a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp +++ b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp @@ -216,7 +216,7 @@ struct DifferentClustersSameRVWorkload : TestWorkload { state Future> val2 = tr2.get(self->keyToRead); wait(success(val1) && success(val2)); // We're reading from different db's with the same read version. We can get a different value. - TEST(val1.get() != val2.get()); // reading from different dbs with the same version + CODE_PROBE(val1.get() != val2.get(), "reading from different dbs with the same version"); } catch (Error& e) { wait(tr1.onError(e) && tr2.onError(e)); } diff --git a/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp b/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp index 3cd575db11..c55529a74c 100644 --- a/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp +++ b/fdbserver/workloads/EncryptKeyProxyTest.actor.cpp @@ -329,7 +329,7 @@ struct EncryptKeyProxyTestWorkload : TestWorkload { } Future start(Database const& cx) override { - TEST(true); // Testing + CODE_PROBE(true, "Testing"); if (!enableTest) { return Void(); } diff --git a/fdbserver/workloads/Increment.actor.cpp b/fdbserver/workloads/Increment.actor.cpp index ee65c19a62..a6a4c2d668 100644 --- a/fdbserver/workloads/Increment.actor.cpp +++ b/fdbserver/workloads/Increment.actor.cpp @@ -110,7 +110,7 @@ struct Increment : TestWorkload { } } bool incrementCheckData(const VectorRef& data, Version v, Increment* self) { - TEST(self->transactions.getValue()); // incrementCheckData transaction has value + CODE_PROBE(self->transactions.getValue(), "incrementCheckData transaction has value"); if (self->transactions.getValue() && data.size() == 0) { TraceEvent(SevError, "TestFailure") .detail("Reason", "No successful increments") diff --git a/fdbserver/workloads/LockDatabase.actor.cpp b/fdbserver/workloads/LockDatabase.actor.cpp index 85cfdc6837..535f0c17cb 100644 --- a/fdbserver/workloads/LockDatabase.actor.cpp +++ b/fdbserver/workloads/LockDatabase.actor.cpp @@ -113,7 +113,7 @@ struct LockDatabaseWorkload : TestWorkload { self->ok = false; return Void(); } catch (Error& e) { - TEST(e.code() == error_code_database_locked); // Database confirmed locked + CODE_PROBE(e.code() == error_code_database_locked, "Database confirmed locked"); wait(tr.onError(e)); } } diff --git a/fdbserver/workloads/MachineAttrition.actor.cpp b/fdbserver/workloads/MachineAttrition.actor.cpp index e2a1f5492c..fb272fcf07 100644 --- a/fdbserver/workloads/MachineAttrition.actor.cpp +++ b/fdbserver/workloads/MachineAttrition.actor.cpp @@ -322,7 +322,7 @@ struct MachineAttritionWorkload : TestWorkload { .detail("MachinesToKill", self->machinesToKill) .detail("MachinesToLeave", self->machinesToLeave) .detail("Machines", self->machines.size()); - TEST(true); // Killing a machine + CODE_PROBE(true, "Killing a machine"); delayBeforeKill = deterministicRandom()->random01() * meanDelay; wait(delay(delayBeforeKill)); @@ -345,11 +345,11 @@ struct MachineAttritionWorkload : TestWorkload { // decide on a machine to kill state LocalityData targetMachine = self->machines.back(); if (BUGGIFY_WITH_PROB(0.01)) { - TEST(true); // Marked a zone for maintenance before killing it + CODE_PROBE(true, "Marked a zone for maintenance before killing it"); wait(success( setHealthyZone(cx, targetMachine.zoneId().get(), deterministicRandom()->random01() * 20))); } else if (BUGGIFY_WITH_PROB(0.005)) { - TEST(true); // Disable DD for all storage server failures + CODE_PROBE(true, "Disable DD for all storage server failures"); self->ignoreSSFailures = uncancellable(ignoreSSFailuresForDuration(cx, deterministicRandom()->random01() * 5)); } diff --git a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp index f2bbbde625..4127490cd2 100644 --- a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp +++ b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp @@ -119,7 +119,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { return; } f = success(ryw.get(LiteralStringRef("\xff\xff/status/json"))); - TEST(!f.isReady()); // status json not ready + CODE_PROBE(!f.isReady(), "status json not ready"); } ASSERT(f.isError()); ASSERT(f.getError().code() == error_code_transaction_cancelled); @@ -233,7 +233,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { .detail("TestValue", printable(res2[i].value)); return false; } - TEST(true); // Special key space keys equal + CODE_PROBE(true, "Special key space keys equal"); } return true; } @@ -335,7 +335,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { wait(success(tx->getRange( KeyRangeRef(LiteralStringRef("\xff\xff/transaction/"), LiteralStringRef("\xff\xff/transaction0")), CLIENT_KNOBS->TOO_MANY))); - TEST(true); // read transaction special keyrange + CODE_PROBE(true, "read transaction special keyrange"); tx->reset(); } catch (Error& e) { throw; @@ -361,7 +361,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { KeySelector begin = KeySelectorRef(readConflictRangeKeysRange.begin, false, 1); KeySelector end = KeySelectorRef(LiteralStringRef("\xff\xff/transaction0"), false, 0); wait(success(tx->getRange(begin, end, GetRangeLimits(CLIENT_KNOBS->TOO_MANY)))); - TEST(true); // end key selector inside module range + CODE_PROBE(true, "end key selector inside module range"); tx->reset(); } catch (Error& e) { throw; @@ -489,8 +489,8 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { ACTOR static Future testConflictRanges(Database cx_, bool read, SpecialKeySpaceCorrectnessWorkload* self) { state StringRef prefix = read ? readConflictRangeKeysRange.begin : writeConflictRangeKeysRange.begin; - TEST(read); // test read conflict range special key implementation - TEST(!read); // test write conflict range special key implementation + CODE_PROBE(read, "test read conflict range special key implementation"); + CODE_PROBE(!read, "test write conflict range special key implementation"); // Get a default special key range instance Database cx = cx_->clone(); state Reference tx = makeReference(cx); @@ -545,7 +545,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { throw; return Void(); } - TEST(true); // Read write conflict range of committed transaction + CODE_PROBE(true, "Read write conflict range of committed transaction"); } try { wait(success(tx->get(LiteralStringRef("\xff\xff/1314109/i_hope_this_isn't_registered")))); diff --git a/fdbserver/workloads/Throttling.actor.cpp b/fdbserver/workloads/Throttling.actor.cpp index fc906c8e55..2a28e141c2 100644 --- a/fdbserver/workloads/Throttling.actor.cpp +++ b/fdbserver/workloads/Throttling.actor.cpp @@ -138,7 +138,7 @@ struct ThrottlingWorkload : KVWorkload { ASSERT(k.startsWith(LiteralStringRef("\xff\xff/metrics/health/"))); auto valueObj = readJSONStrictly(v.toString()).get_obj(); if (k.removePrefix(LiteralStringRef("\xff\xff/metrics/health/")) == LiteralStringRef("aggregate")) { - TEST(true); // Test aggregate health metrics schema + CODE_PROBE(true, "Test aggregate health metrics schema"); std::string errorStr; if (!schemaMatch(aggregateSchema, valueObj, errorStr, SevError, true)) { TraceEvent(SevError, "AggregateHealthSchemaValidationFailed") @@ -150,7 +150,7 @@ struct ThrottlingWorkload : KVWorkload { self->tokenBucket.transactionRate = tpsLimit * self->throttlingMultiplier / self->clientCount; } else if (k.removePrefix(LiteralStringRef("\xff\xff/metrics/health/")) .startsWith(LiteralStringRef("storage/"))) { - TEST(true); // Test storage health metrics schema + CODE_PROBE(true, "Test storage health metrics schema"); UID::fromString(k.removePrefix(LiteralStringRef("\xff\xff/metrics/health/storage/")) .toString()); // Will throw if it's not a valid uid std::string errorStr; @@ -162,7 +162,7 @@ struct ThrottlingWorkload : KVWorkload { } } else if (k.removePrefix(LiteralStringRef("\xff\xff/metrics/health/")) .startsWith(LiteralStringRef("log/"))) { - TEST(true); // Test log health metrics schema + CODE_PROBE(true, "Test log health metrics schema"); UID::fromString(k.removePrefix(LiteralStringRef("\xff\xff/metrics/health/log/")) .toString()); // Will throw if it's not a valid uid std::string errorStr; diff --git a/fdbserver/workloads/VersionStamp.actor.cpp b/fdbserver/workloads/VersionStamp.actor.cpp index 89ee83c254..deb900d788 100644 --- a/fdbserver/workloads/VersionStamp.actor.cpp +++ b/fdbserver/workloads/VersionStamp.actor.cpp @@ -205,7 +205,7 @@ struct VersionStampWorkload : TestWorkload { if (self->failIfDataLost) { ASSERT(result.size() == self->key_commit.size()); } else { - TEST(result.size() > 0); // Not all data should always be lost. + CODE_PROBE(result.size() > 0, "Not all data should always be lost."); } //TraceEvent("VST_Check0").detail("Size", result.size()).detail("NodeCount", self->nodeCount).detail("KeyCommit", self->key_commit.size()).detail("ReadVersion", readVersion); @@ -260,7 +260,7 @@ struct VersionStampWorkload : TestWorkload { if (self->failIfDataLost) { ASSERT(result.size() == self->versionStampKey_commit.size()); } else { - TEST(result.size() > 0); // Not all data should always be lost (2) + CODE_PROBE(result.size() > 0, "Not all data should always be lost (2)"); } //TraceEvent("VST_Check1").detail("Size", result.size()).detail("VsKeyCommitSize", self->versionStampKey_commit.size()); diff --git a/fdbserver/workloads/WriteDuringRead.actor.cpp b/fdbserver/workloads/WriteDuringRead.actor.cpp index c7253748da..78df96173f 100644 --- a/fdbserver/workloads/WriteDuringRead.actor.cpp +++ b/fdbserver/workloads/WriteDuringRead.actor.cpp @@ -85,8 +85,8 @@ struct WriteDuringReadWorkload : TestWorkload { minNode = std::max(minNode, nodes - newNodes); nodes = newNodes; - TEST(adjacentKeys && - (nodes + minNode) > CLIENT_KNOBS->KEY_SIZE_LIMIT); // WriteDuringReadWorkload testing large keys + CODE_PROBE(adjacentKeys && (nodes + minNode) > CLIENT_KNOBS->KEY_SIZE_LIMIT, + "WriteDuringReadWorkload testing large keys"); useExtraDB = g_simulator.extraDB != nullptr; if (useExtraDB) { diff --git a/flow/BlobCipher.cpp b/flow/BlobCipher.cpp index 9c5cdcc37e..559b68343d 100644 --- a/flow/BlobCipher.cpp +++ b/flow/BlobCipher.cpp @@ -409,7 +409,7 @@ Reference EncryptBlobCipherAes265Ctr::encrypt(const uint8_t* plainte const int plaintextLen, BlobCipherEncryptHeader* header, Arena& arena) { - TEST(true); // Encrypting data with BlobCipher + CODE_PROBE(true, "Encrypting data with BlobCipher"); memset(reinterpret_cast(header), 0, sizeof(BlobCipherEncryptHeader)); @@ -673,7 +673,7 @@ Reference DecryptBlobCipherAes256Ctr::decrypt(const uint8_t* ciphert const int ciphertextLen, const BlobCipherEncryptHeader& header, Arena& arena) { - TEST(true); // Decrypting data with BlobCipher + CODE_PROBE(true, "Decrypting data with BlobCipher"); verifyEncryptHeaderMetadata(header); @@ -741,7 +741,7 @@ HmacSha256DigestGen::~HmacSha256DigestGen() { } StringRef HmacSha256DigestGen::digest(const unsigned char* data, size_t len, Arena& arena) { - TEST(true); // Digest generation + CODE_PROBE(true, "Digest generation"); unsigned int digestLen = HMAC_size(ctx); auto digest = new (arena) unsigned char[digestLen]; if (HMAC_Update(ctx, data, len) != 1) { diff --git a/flow/CodeProbe.cpp b/flow/CodeProbe.cpp new file mode 100644 index 0000000000..b5c1c9d2ea --- /dev/null +++ b/flow/CodeProbe.cpp @@ -0,0 +1,293 @@ +/* + * CodeProbe.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "flow/CodeProbe.h" +#include "flow/CodeProbeUtils.h" +#include "flow/Arena.h" +#include "flow/network.h" + +#include +#include +#include +#include +#include + +namespace probe { + +namespace { + +std::vector fromStrings(std::vector const& ctxs) { + std::vector res; + for (auto const& ctx : ctxs) { + std::string c; + c.reserve(ctx.size()); + std::transform(ctx.begin(), ctx.end(), std::back_inserter(c), [](char c) { return std::tolower(c); }); + if (c == "all") { + res.push_back(ExecutionContext::Net2); + res.push_back(ExecutionContext::Simulation); + } else if (c == "simulation") { + res.push_back(ExecutionContext::Simulation); + } else if (c == "net2") { + res.push_back(ExecutionContext::Net2); + } else { + throw invalid_option_value(); + } + } + std::sort(res.begin(), res.end()); + res.erase(std::unique(res.begin(), res.end()), res.end()); + return res; +} + +std::string_view normalizePath(const char* path) { + std::string_view srcBase(FDB_SOURCE_DIR); + std::string_view binBase(FDB_SOURCE_DIR); + std::string_view filename(path); + if (srcBase.size() < filename.size() && filename.substr(0, srcBase.size()) == srcBase) { + filename.remove_prefix(srcBase.size()); + } else if (binBase.size() < filename.size() && filename.substr(0, binBase.size()) == binBase) { + filename.remove_prefix(binBase.size()); + } + if (filename[0] == '/') { + filename.remove_prefix(1); + } + return filename; +} + +struct CodeProbes { + struct Location { + std::string_view file; + unsigned line; + Location(std::string_view file, unsigned line) : file(file), line(line) {} + bool operator==(Location const& rhs) const { return line == rhs.line && file == rhs.file; } + bool operator!=(Location const& rhs) const { return line != rhs.line && file != rhs.file; } + bool operator<(Location const& rhs) const { + if (file < rhs.file) { + return true; + } else if (file == rhs.file) { + return line < rhs.line; + } else { + return false; + } + } + bool operator<=(Location const& rhs) const { + if (file < rhs.file) { + return true; + } else if (file == rhs.file) { + return line <= rhs.line; + } else { + return false; + } + } + bool operator>(Location const& rhs) const { return rhs < *this; } + bool operator>=(Location const& rhs) const { return rhs <= *this; } + }; + + std::multimap codeProbes; + + void traceMissedProbes(Optional context) const; + + void add(ICodeProbe const* probe) { + Location loc(probe->filename(), probe->line()); + codeProbes.emplace(loc, probe); + } + + static CodeProbes& instance() { + static CodeProbes probes; + return probes; + } + + void verify() const { + std::map, ICodeProbe const*> comments; + for (auto probe : codeProbes) { + auto file = probe.first.file; + auto comment = probe.second->comment(); + auto commentEntry = std::make_pair(file, std::string_view(comment)); + ASSERT(file == probe.second->filename()); + auto iter = comments.find(commentEntry); + if (iter != comments.end() && probe.second->line() != iter->second->line()) { + fmt::print("ERROR ({}:{}): {} isn't unique in file {}. Previously seen here: {}:{}\n", + probe.first.file, + probe.first.line, + iter->first.second, + probe.second->filename(), + iter->second->filename(), + iter->second->line()); + // ICodeProbe const& fst = *iter->second; + // ICodeProbe const& snd = *probe.second; + // fmt::print("\t1st Type: {}\n", boost::core::demangle(typeid(fst).name())); + // fmt::print("\t2nd Type: {}\n", boost::core::demangle(typeid(snd).name())); + // fmt::print("\n"); + // fmt::print("\t1st Comment: {}\n", fst.comment()); + // fmt::print("\t2nd Comment: {}\n", snd.comment()); + // fmt::print("\n"); + // fmt::print("\t1st CompUnit: {}\n", fst.compilationUnit()); + // fmt::print("\t2nd CompUnit: {}\n", snd.compilationUnit()); + // fmt::print("\n"); + } else { + comments.emplace(commentEntry, probe.second); + } + } + } + + void printXML() const { + verify(); + fmt::print("\n"); + fmt::print("\n"); + if (codeProbes.empty()) { + fmt::print("\t\n"); + fmt::print("\t\n"); + } else { + std::vector files; + fmt::print("\t\n"); + for (auto probe : codeProbes) { + files.push_back(probe.first.file); + fmt::print("\t\t\n", + probe.first.file, + probe.first.line, + probe.second->comment(), + probe.second->condition()); + } + fmt::print("\t\n"); + fmt::print("\t\n"); + for (auto const& f : files) { + fmt::print("\t\t{}\n", f); + } + fmt::print("\t\n"); + } + fmt::print("\n"); + } + + void printJSON(std::vector const& context = std::vector()) const { + verify(); + do { + struct foo {}; + foo f; + fmt::print("{}\n", boost::core::demangle(typeid(f).name())); + } while (false); + do { + struct foo {}; + foo f; + fmt::print("{}\n", boost::core::demangle(typeid(f).name())); + } while (false); + auto contexts = fromStrings(context); + const ICodeProbe* prev = nullptr; + for (auto probe : codeProbes) { + auto p = probe.second; + if (!contexts.empty()) { + bool print = false; + for (auto c : contexts) { + print = print || p->expectInContext(c); + } + if (!print) { + continue; + } + } + if (prev == nullptr || *prev != *probe.second) { + fmt::print( + "{{ \"File\": \"{}\", \"Line\": {}, \"Comment\": \"{}\", \"Condition\": \"{}\", \"Function\": " + "\"{}\" }}\n", + probe.first.file, + p->line(), + p->comment(), + p->condition(), + p->function()); + } + prev = probe.second; + } + } +}; + +size_t hash_value(CodeProbes::Location const& location) { + size_t seed = 0; + boost::hash_combine(seed, location.file); + boost::hash_combine(seed, location.line); + return seed; +} + +void CodeProbes::traceMissedProbes(Optional context) const { + boost::unordered_map locations; + for (auto probe : codeProbes) { + decltype(locations.begin()) iter; + std::tie(iter, std::ignore) = locations.emplace(probe.first, false); + iter->second = iter->second || probe.second->wasHit(); + } + for (auto probe : codeProbes) { + auto iter = locations.find(probe.first); + ASSERT(iter != locations.end()); + if (!iter->second) { + iter->second = true; + probe.second->trace(false); + } + } +} + +} // namespace + +std::string functionNameFromInnerType(const char* name) { + auto res = boost::core::demangle(name); + auto pos = res.find_last_of(':'); + ASSERT(pos != res.npos); + return res.substr(0, pos - 1); +} + +void registerProbe(const ICodeProbe& probe) { + CodeProbes::instance().add(&probe); +} + +void traceMissedProbes(Optional context) { + CodeProbes::instance().traceMissedProbes(context); +} + +ICodeProbe::~ICodeProbe() {} + +bool ICodeProbe::operator==(const ICodeProbe& other) const { + return filename() == other.filename() && line() == other.line(); +} + +bool ICodeProbe::operator!=(const ICodeProbe& other) const { + return !(*this == other); +} + +std::string_view ICodeProbe::filename() const { + return normalizePath(filePath()); +} + +void ICodeProbe::printProbesXML() { + CodeProbes::instance().printXML(); +} + +void ICodeProbe::printProbesJSON(std::vector const& ctxs) { + CodeProbes::instance().printJSON(ctxs); +} + +// annotations +namespace assert { + +bool NoSim::operator()(ICodeProbe* self) const { + return !g_network->isSimulated(); +} + +bool SimOnly::operator()(ICodeProbe* self) const { + return g_network->isSimulated(); +} + +} // namespace assert + +} // namespace probe diff --git a/flow/StreamCipher.cpp b/flow/StreamCipher.cpp index cf5f8d7b2f..6663e90e59 100644 --- a/flow/StreamCipher.cpp +++ b/flow/StreamCipher.cpp @@ -131,7 +131,7 @@ EncryptionStreamCipher::EncryptionStreamCipher(const StreamCipherKey* key, const } StringRef EncryptionStreamCipher::encrypt(unsigned char const* plaintext, int len, Arena& arena) { - TEST(true); // Encrypting data with StreamCipher + CODE_PROBE(true, "Encrypting data with StreamCipher"); auto ciphertext = new (arena) unsigned char[len + AES_BLOCK_SIZE]; int bytes{ 0 }; EVP_EncryptUpdate(cipher.getCtx(), ciphertext, &bytes, plaintext, len); @@ -153,7 +153,7 @@ DecryptionStreamCipher::DecryptionStreamCipher(const StreamCipherKey* key, const } StringRef DecryptionStreamCipher::decrypt(unsigned char const* ciphertext, int len, Arena& arena) { - TEST(true); // Decrypting data with StreamCipher + CODE_PROBE(true, "Decrypting data with StreamCipher"); auto plaintext = new (arena) unsigned char[len]; int bytesDecrypted{ 0 }; EVP_DecryptUpdate(cipher.getCtx(), plaintext, &bytesDecrypted, ciphertext, len); @@ -174,7 +174,7 @@ HmacSha256StreamCipher::HmacSha256StreamCipher() : cipher(EVP_MAX_KEY_LENGTH) { } StringRef HmacSha256StreamCipher::digest(unsigned char const* data, int len, Arena& arena) { - TEST(true); // Digest using StreamCipher + CODE_PROBE(true, "Digest using StreamCipher"); unsigned int digestLen = HMAC_size(cipher.getHmacCtx()); auto digest = new (arena) unsigned char[digestLen]; HMAC_Update(cipher.getHmacCtx(), data, len); diff --git a/flow/TDMetric.cpp b/flow/TDMetric.cpp index 0d66e286f0..a84c57072c 100644 --- a/flow/TDMetric.cpp +++ b/flow/TDMetric.cpp @@ -134,7 +134,7 @@ bool TDMetricCollection::canLog(int level) const { void TDMetricCollection::checkRoll(uint64_t t, int64_t usedBytes) { currentTimeBytes += usedBytes; if (currentTimeBytes > 1e6) { - TEST(true); // metrics were rolled + CODE_PROBE(true, "metrics were rolled"); currentTimeBytes = 0; rollTimes.push_back(t); for (auto& it : metricMap) diff --git a/flow/config.h.cmake b/flow/config.h.cmake index 864d7c68fb..cad9c61186 100644 --- a/flow/config.h.cmake +++ b/flow/config.h.cmake @@ -5,6 +5,8 @@ # define FDB_CLEAN_BUILD #endif // FDB_RELEASE #cmakedefine OPEN_FOR_IDE +#define FDB_SOURCE_DIR "${CMAKE_SOURCE_DIR}" +#define FDB_BINARY_DIR "${CMAKE_BINARY_DIR}" #ifdef WIN32 # define _WIN32_WINNT ${WINDOWS_TARGET} # define WINVER ${WINDOWS_TARGET} diff --git a/flow/flow.cpp b/flow/flow.cpp index e758ffbfd3..388ee5a4be 100644 --- a/flow/flow.cpp +++ b/flow/flow.cpp @@ -261,7 +261,7 @@ int vsformat(std::string& outputString, const char* form, va_list args) { return -1; } - TEST(true); // large format result + CODE_PROBE(true, "large format result"); outputString.resize(size + 1); size = vsnprintf(&outputString[0], outputString.size(), form, args); diff --git a/flow/include/flow/Arena.h b/flow/include/flow/Arena.h index bfef0b4a6b..d777383f1d 100644 --- a/flow/include/flow/Arena.h +++ b/flow/include/flow/Arena.h @@ -41,6 +41,8 @@ #include #include #include +#include +#include // TrackIt is a zero-size class for tracking constructions, destructions, and assignments of instances // of a class. Just inherit TrackIt from T to enable tracking of construction and destruction of @@ -725,6 +727,15 @@ StringRef LiteralStringRefHelper(const char* str) { } // namespace literal_string_ref #define LiteralStringRef(str) literal_string_ref::LiteralStringRefHelper(str) +template <> +struct fmt::formatter : formatter { + template + auto format(const StringRef& str, FormatContext& ctx) -> decltype(ctx.out()) { + std::string_view view(reinterpret_cast(str.begin()), str.size()); + return formatter::format(view, ctx); + } +}; + inline StringRef operator"" _sr(const char* str, size_t size) { return StringRef(reinterpret_cast(str), size); } diff --git a/flow/include/flow/CodeProbe.h b/flow/include/flow/CodeProbe.h new file mode 100644 index 0000000000..b82fa5c7a8 --- /dev/null +++ b/flow/include/flow/CodeProbe.h @@ -0,0 +1,305 @@ +/* + * CodeProbe.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef FLOW_TESTPROBE_H_ +#define FLOW_TESTPROBE_H_ + +#include "flow/Knobs.h" +#include "flow/Trace.h" + +namespace probe { + +struct ICodeProbe; + +enum class AnnotationType { Decoration, Assertion, Context, Functional }; +enum class ExecutionContext { Simulation, Net2 }; + +namespace context { +struct Net2 { + constexpr static AnnotationType type = AnnotationType::Context; + constexpr static ExecutionContext value = ExecutionContext::Net2; +}; +struct Sim2 { + constexpr static AnnotationType type = AnnotationType::Context; + constexpr static ExecutionContext value = ExecutionContext::Simulation; +}; + +constexpr Net2 net2; +constexpr Sim2 sim2; + +template +struct OrContext { + typename std::remove_const::type left; + typename std::remove_const::type right; + constexpr OrContext(Left left, Right right) : left(left), right(right) {} + constexpr bool operator()(ExecutionContext context) const { return left(context) || right(context); } +}; + +template +constexpr std::enable_if_t> +operator|(Left const& lhs, Right const& rhs) { + return OrContext(lhs, rhs); +} + +} // namespace context + +namespace assert { +struct NoSim { + constexpr static AnnotationType type = AnnotationType::Assertion; + bool operator()(ICodeProbe* self) const; +}; +struct SimOnly { + constexpr static AnnotationType type = AnnotationType::Assertion; + bool operator()(ICodeProbe* self) const; +}; + +template +struct AssertOr { + typename std::remove_const::type left; + typename std::remove_const::type right; + constexpr AssertOr() {} + constexpr bool operator()(ICodeProbe* self) const { return left(self) || right(self); } +}; +template +struct AssertAnd { + typename std::remove_const::type left; + typename std::remove_const::type right; + constexpr AssertAnd() {} + constexpr bool operator()(ICodeProbe* self) const { return left(self) && right(self); } +}; +template +struct AssertNot { + typename std::remove_const::type other; + constexpr bool operator()(ICodeProbe* self) const { return !other(self); } +}; + +template +constexpr std::enable_if_t> +operator||(Left const& lhs, Right const& rhs) { + return AssertOr(); +} +template +constexpr std::enable_if_t> +operator&&(Left const& lhs, Right const& rhs) { + return AssertAnd(); +} + +template +constexpr std::enable_if_t> operator!(T const&) { + return AssertNot(); +} + +constexpr SimOnly simOnly; +constexpr auto noSim = !simOnly; + +} // namespace assert + +namespace func { + +struct Deduplicate { + constexpr static AnnotationType type = AnnotationType::Functional; +}; + +constexpr Deduplicate deduplicate; + +} // namespace func + +template +struct CodeProbeAnnotations; + +template <> +struct CodeProbeAnnotations<> { + static constexpr bool providesContext = false; + void hit(ICodeProbe* self) {} + void trace(const ICodeProbe*, BaseTraceEvent&, bool) const {} + constexpr bool expectContext(ExecutionContext context, bool prevHadSomeContext = false) const { + return !prevHadSomeContext; + } + constexpr bool deduplicate() const { return false; } +}; + +template +struct CodeProbeAnnotations { + using HeadType = typename std::remove_const::type; + using ChildType = CodeProbeAnnotations; + + static constexpr bool providesContext = HeadType::type == AnnotationType::Context || ChildType::providesContext; + static_assert(HeadType::type != AnnotationType::Context || !ChildType::providesContext, + "Only one context annotation can be used"); + + HeadType head; + ChildType tail; + + void hit(ICodeProbe* self) { + if constexpr (Head::type == AnnotationType::Assertion) { + ASSERT(head(self)); + } + tail.hit(self); + } + void trace(const ICodeProbe* self, BaseTraceEvent& evt, bool condition) const { + if constexpr (Head::type == AnnotationType::Decoration) { + head.trace(self, evt, condition); + } + tail.trace(self, evt, condition); + } + // This should behave like the following: + // 1. If no context is passed in the code probe, we expect to see this in every context + // 2. Otherwise we will return true iff the execution context we're looking for has been passed to the probe + constexpr bool expectContext(ExecutionContext context, bool prevHadSomeContext = false) const { + if constexpr (HeadType::type == AnnotationType::Context) { + if (HeadType::value == context) { + return true; + } else { + return tail.expectContext(context, true); + } + } else { + return tail.expectContext(context, prevHadSomeContext); + } + } + + constexpr bool deduplicate() const { + if constexpr (std::is_same_v) { + return true; + } else { + return tail.deduplicate(); + } + } +}; + +struct ICodeProbe { + virtual ~ICodeProbe(); + + bool operator==(ICodeProbe const& other) const; + bool operator!=(ICodeProbe const& other) const; + + std::string_view filename() const; + virtual const char* filePath() const = 0; + virtual unsigned line() const = 0; + virtual const char* comment() const = 0; + virtual const char* condition() const = 0; + virtual const char* compilationUnit() const = 0; + virtual void trace(bool) const = 0; + virtual bool wasHit() const = 0; + virtual unsigned hitCount() const = 0; + virtual bool expectInContext(ExecutionContext context) const = 0; + virtual std::string function() const = 0; + virtual bool deduplicate() const = 0; + + static void printProbesXML(); + static void printProbesJSON(std::vector const& ctxs = std::vector()); +}; + +void registerProbe(ICodeProbe const& probe); +std::string functionNameFromInnerType(const char* name); + +template +struct CodeProbeImpl : ICodeProbe { + static CodeProbeImpl* instancePtr() { return &_instance; } + static CodeProbeImpl& instance() { return _instance; } + void hit() { + if (_hitCount++ == 0) { + trace(true); + } + annotations.hit(this); + } + + void trace(bool condition) const override { + TraceEvent evt(intToSeverity(FLOW_KNOBS->CODE_COV_TRACE_EVENT_SEVERITY), "CodeCoverage"); + evt.detail("File", filename()) + .detail("Line", Line) + .detail("Condition", Condition::value()) + .detail("ProbeHit", condition) + .detail("Comment", Comment::value()); + annotations.trace(this, evt, condition); + } + bool wasHit() const override { return _hitCount > 0; } + unsigned hitCount() const override { return _hitCount; } + + const char* filePath() const override { return FileName::value(); } + unsigned line() const override { return Line; } + const char* comment() const override { return Comment::value(); } + const char* condition() const override { return Condition::value(); } + const char* compilationUnit() const override { return CompUnit::value(); } + bool expectInContext(ExecutionContext context) const override { return annotations.expectContext(context); } + std::string function() const override { return functionNameFromInnerType(typeid(FileName).name()); } + bool deduplicate() const override { return annotations.deduplicate(); } + +private: + CodeProbeImpl() { registerProbe(*this); } + inline static CodeProbeImpl _instance; + unsigned _hitCount = 0; + Annotations annotations; +}; + +template +CodeProbeImpl>& probeInstance( + Annotations&... annotations) { + return CodeProbeImpl>:: + instance(); +} + +} // namespace probe + +#ifdef COMPILATION_UNIT +#define CODE_PROBE_QUOTE(x) #x +#define CODE_PROBE_EXPAND_AND_QUOTE(x) CODE_PROBE_QUOTE(x) +#define CODE_PROBE_COMPILATION_UNIT CODE_PROBE_EXPAND_AND_QUOTE(COMPILATION_UNIT) +#else +#define CODE_PROBE_COMPILATION_UNIT "COMPILATION_UNIT not set" +#endif + +#define _CODE_PROBE_IMPL(file, line, condition, comment, compUnit, fileType, condType, commentType, compUnitType, ...) \ + struct fileType { \ + constexpr static const char* value() { return file; } \ + }; \ + struct condType { \ + constexpr static const char* value() { return #condition; } \ + }; \ + struct commentType { \ + constexpr static const char* value() { return comment; } \ + }; \ + struct compUnitType { \ + constexpr static const char* value() { return compUnit; } \ + }; \ + if (condition) { \ + probe::probeInstance(__VA_ARGS__).hit(); \ + } + +#define _CODE_PROBE_T2(type, counter) type##counter +#define _CODE_PROBE_T(type, counter) _CODE_PROBE_T2(type, counter) + +#define CODE_PROBE(condition, comment, ...) \ + do { \ + _CODE_PROBE_IMPL(__FILE__, \ + __LINE__, \ + condition, \ + comment, \ + CODE_PROBE_COMPILATION_UNIT, \ + _CODE_PROBE_T(FileType, __LINE__), \ + _CODE_PROBE_T(CondType, __LINE__), \ + _CODE_PROBE_T(CommentType, __LINE__), \ + _CODE_PROBE_T(CompilationUnitType, __LINE__), \ + __VA_ARGS__) \ + } while (false) + +#endif // FLOW_TESTPROBE_H_ diff --git a/flow/include/flow/CodeProbeUtils.h b/flow/include/flow/CodeProbeUtils.h new file mode 100644 index 0000000000..07b04dcd98 --- /dev/null +++ b/flow/include/flow/CodeProbeUtils.h @@ -0,0 +1,32 @@ +/* + * CodeProbeUtils.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef FLOW_CODE_PROBE_UTILS_H +#define FLOW_CODE_PROBE_UTILS_H +#include "flow/CodeProbe.h" +#include "flow/Arena.h" + +namespace probe { + +void traceMissedProbes(Optional context); + +} + +#endif // FLOW_CODE_PROBE_UTILS_H diff --git a/flow/include/flow/flow.h b/flow/include/flow/flow.h index a3c156c9d1..8ae2e72ebd 100644 --- a/flow/include/flow/flow.h +++ b/flow/include/flow/flow.h @@ -46,6 +46,7 @@ #include #include +#include "flow/CodeProbe.h" #include "flow/Platform.h" #include "flow/FastAlloc.h" #include "flow/IRandom.h" @@ -59,15 +60,9 @@ #include #define TEST(condition) \ - if (!(condition)) { \ - } else { \ - static BaseTraceEvent* __test = \ - &(TraceEvent(intToSeverity(FLOW_KNOBS->CODE_COV_TRACE_EVENT_SEVERITY), "CodeCoverage") \ - .detail("File", __FILE__) \ - .detail("Line", __LINE__) \ - .detail("Condition", #condition)); \ - (void)__test; \ - } + do { \ + static_assert(false, "TEST macros are deprecated, please use CODE_PROBE instead"); \ + } while (false) /* usage: diff --git a/flow/include/flow/genericactors.actor.h b/flow/include/flow/genericactors.actor.h index c548ee02c3..92afe90643 100644 --- a/flow/include/flow/genericactors.actor.h +++ b/flow/include/flow/genericactors.actor.h @@ -1420,7 +1420,8 @@ private: } return Void(); } catch (...) { - TEST(true); // If we get cancelled here, we are holding the lock but the caller doesn't know, so release it + CODE_PROBE(true, + "If we get cancelled here, we are holding the lock but the caller doesn't know, so release it"); lock->release(amount); throw; }