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
This commit is contained in:
parent
0d20cd2068
commit
1de37afd52
|
@ -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)
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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<Version>(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()));
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -1003,8 +1003,8 @@ ACTOR Future<CoordinatorsResult> changeQuorum(Database cx, Reference<IQuorumChan
|
|||
TraceEvent("AttemptingQuorumChange")
|
||||
.detail("FromCS", oldClusterConnectionString.toString())
|
||||
.detail("ToCS", newClusterConnectionString.toString());
|
||||
TEST(oldClusterKeyName != newClusterKeyName); // Quorum change with new name
|
||||
TEST(oldClusterKeyName == newClusterKeyName); // Quorum change with unchanged name
|
||||
CODE_PROBE(oldClusterKeyName != newClusterKeyName, "Quorum change with new name");
|
||||
CODE_PROBE(oldClusterKeyName == newClusterKeyName, "Quorum change with unchanged name");
|
||||
|
||||
state std::vector<Future<Optional<LeaderInfo>>> leaderServers;
|
||||
state ClientCoordinators coord(Reference<ClusterConnectionMemoryRecord>(
|
||||
|
|
|
@ -987,8 +987,9 @@ ACTOR Future<MonitorLeaderInfo> 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());
|
||||
|
|
|
@ -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>();
|
||||
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<Void> 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<ReadYourWritesTransaction>(Database(Reference<DatabaseContext>::addRef(cx)));
|
||||
state int tries = 0;
|
||||
|
@ -1154,7 +1154,7 @@ ACTOR static Future<Void> handleTssMismatches(DatabaseContext* cx) {
|
|||
// clear out txn so that the extra DatabaseContext ref gets decref'd and we can free cx
|
||||
tr = makeReference<ReadYourWritesTransaction>();
|
||||
} 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>& 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<LocationInfo> DatabaseContext::setCachedLocation(const Optional<Tenant
|
|||
int maxEvictionAttempts = 100, attempts = 0;
|
||||
auto loc = makeReference<LocationInfo>(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<Void> DatabaseContext::onConnected() {
|
|||
|
||||
ACTOR static Future<Void> switchConnectionRecordImpl(Reference<IClusterConnectionRecord> 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<Version> watchValue(Database cx, Reference<const WatchParameters> 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<Void> 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<Void> getWatchFuture(Database cx, Reference<WatchParameters> 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<Void> getWatchFuture(Database cx, Reference<WatchParameters> 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<RangeResultFamily> getExactRange(Reference<TransactionState> 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<RangeResultFamily> getExactRange(Reference<TransactionState> 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<RangeResultFamily> getExactRange(Reference<TransactionState> 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<RangeResultFamily> getRange(Reference<TransactionState> 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<RangeResultFamily> getRange(Reference<TransactionState> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<R
|
|||
Optional<TSSEndpointData> 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<Request> tssRequestStream(tssData.get().endpoint);
|
||||
|
@ -4874,7 +4876,7 @@ ACTOR Future<Void> getRangeStreamFragment(Reference<TransactionState> 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<Void> watch(Reference<Watch> 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<RangeResultFamily> 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<Void> Transaction::getRangeStream(const PromiseStream<RangeResult>& 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<Void> tryCommit(Reference<TransactionState> 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<Strin
|
|||
if (value.get().size() != 33) {
|
||||
throw invalid_option_value();
|
||||
}
|
||||
TEST(true); // Adding link in FDBTransactionOptions::SPAN_PARENT
|
||||
CODE_PROBE(true, "Adding link in FDBTransactionOptions::SPAN_PARENT");
|
||||
span.setParent(BinaryReader::fromStringRef<SpanContext>(value.get(), IncludeVersion()));
|
||||
break;
|
||||
|
||||
|
@ -6668,10 +6670,10 @@ ACTOR Future<GetReadVersionReply> 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<Version> extractReadVersion(Reference<TransactionState> 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<Version> 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<Standalone<VectorRef<ReadHotRangeWithMetrics>>> 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<VectorRef<ReadHotRangeWithMetrics>> results;
|
||||
for (int i = 0; i < nLocs; i++) {
|
||||
results.append(results.arena(),
|
||||
|
@ -7856,7 +7858,7 @@ ACTOR Future<Standalone<VectorRef<BlobGranuleChunkRef>>> 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<Void> mergeChangeFeedStream(Reference<DatabaseContext> db,
|
|||
state std::vector<Future<Void>> onErrors(interfs.size());
|
||||
state std::vector<MutationAndVersionStream> 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<UID> debugUIDs;
|
||||
|
@ -9306,13 +9308,13 @@ ACTOR Future<Void> getChangeFeedStreamActor(Reference<DatabaseContext> 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<Void> getChangeFeedStreamActor(Reference<DatabaseContext> 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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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<RangeResult> getWorkerInterfaces(Reference<IClusterConnectionRecord
|
|||
}
|
||||
|
||||
Future<Optional<Value>> 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<RangeResult> 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<RangeResult> 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<RangeResult> 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<MappedRangeResult> 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<MappedRangeResult> 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<MappedRangeResult> 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);
|
||||
|
|
|
@ -364,12 +364,12 @@ ACTOR Future<RangeResult> 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<Key, SpecialKeyRangeReadImpl*, KeyRangeRef>::Ranges ranges =
|
||||
|
@ -453,7 +453,7 @@ Future<RangeResult> 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<RangeResult> 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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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<Future<Optional<Key>>> 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> 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<bool> requeueTimedOutTasks(Reference<ReadYourWritesTransaction> tr,
|
||||
Reference<TaskBucket> 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) {
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -120,7 +120,7 @@ inline ValueRef doAppendIfFits(const Optional<ValueRef>& 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;
|
||||
}
|
||||
|
||||
|
|
|
@ -193,14 +193,14 @@ Future<Void> 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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -615,8 +615,8 @@ ACTOR Future<Void> connectionWriter(Reference<Peer> self, Reference<IConnection>
|
|||
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<Void> connectionIncoming(TransportData* self, Reference<ICon
|
|||
}
|
||||
when(Reference<Peer> 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<StringRef> 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;
|
||||
}
|
||||
|
||||
|
|
|
@ -116,7 +116,7 @@ public:
|
|||
static Future<Void> 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();
|
||||
|
|
|
@ -360,7 +360,7 @@ public:
|
|||
//(e.g. to simulate power failure)
|
||||
Future<Void> 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())));
|
||||
}
|
||||
|
||||
|
|
|
@ -140,7 +140,7 @@ Future<Void> 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<Void> 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<Void> 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<TSSEndpointData> 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<Request, P> tssRequestStream(tssData.get().endpoint);
|
||||
|
|
|
@ -47,7 +47,7 @@ Future<REPLY_TYPE(Req)> retryBrokenPromise(RequestStream<Req, P> 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<REPLY_TYPE(Req)> retryBrokenPromise(RequestStream<Req, P> 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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<uint32_t>::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<UDPSimSocket> {
|
|||
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")
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -393,7 +393,7 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> splitRange(Reference<BlobManagerData
|
|||
if (writeHot) {
|
||||
splitThreshold /= 3;
|
||||
}
|
||||
TEST(writeHot); // Change feed write hot split
|
||||
CODE_PROBE(writeHot, "Change feed write hot split");
|
||||
if (estimated.bytes > splitThreshold) {
|
||||
// only split on bytes and write rate
|
||||
state StorageMetrics splitMetrics;
|
||||
|
@ -429,7 +429,7 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> splitRange(Reference<BlobManagerData
|
|||
ASSERT(keys.back() == range.end);
|
||||
return keys;
|
||||
} else {
|
||||
TEST(writeHot); // Not splitting write-hot because granules would be too small
|
||||
CODE_PROBE(writeHot, "Not splitting write-hot because granules would be too small");
|
||||
if (BM_DEBUG) {
|
||||
printf("Not splitting range\n");
|
||||
}
|
||||
|
@ -461,7 +461,7 @@ ACTOR Future<Standalone<VectorRef<KeyRef>>> splitRange(Reference<BlobManagerData
|
|||
ACTOR Future<UID> pickWorkerForAssign(Reference<BlobManagerData> 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<Void> doRangeAssignment(Reference<BlobManagerData> 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<BlobManagerData> 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<Void> writeInitialGranuleMapping(Reference<BlobManagerData> 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<Void> maybeSplitRange(Reference<BlobManagerData> 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<VectorRef<KeyRef>> coalescedRanges;
|
||||
coalescedRanges.arena().dependsOn(newRanges.arena());
|
||||
coalescedRanges.push_back(coalescedRanges.arena(), newRanges.front());
|
||||
|
@ -1184,7 +1184,7 @@ ACTOR Future<Void> maybeSplitRange(Reference<BlobManagerData> 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<Void> persistMergeGranulesDone(Reference<BlobManagerData> 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<Void> persistMergeGranulesDone(Reference<BlobManagerData> 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<Void> maybeMergeRange(Reference<BlobManagerData> 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<Void> maybeMergeRange(Reference<BlobManagerData> 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<Void> maybeMergeRange(Reference<BlobManagerData> 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<Void> maybeMergeRange(Reference<BlobManagerData> 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<Void> maybeMergeRange(Reference<BlobManagerData> 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<Void> maybeMergeRange(Reference<BlobManagerData> 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<std::pair<UID, Version>>());
|
||||
state std::pair<UID, Version> persistMerge = wait(persistMergeGranulesStart(
|
||||
|
@ -2173,7 +2173,7 @@ ACTOR Future<Void> monitorBlobWorkerStatus(Reference<BlobManagerData> 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<Void> monitorBlobWorkerStatus(Reference<BlobManagerData> 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<Void> monitorBlobWorkerStatus(Reference<BlobManagerData> 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<Void> monitorBlobWorkerStatus(Reference<BlobManagerData> 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<std::tuple<UID, int64_t, int64_t>>& 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<std::tuple<UID, int64_t, int64_t>>& map,
|
|||
ASSERT(oldEpoch != newEpoch || oldSeqno != newSeqno);
|
||||
}
|
||||
if (newEpoch == std::numeric_limits<int64_t>::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<std::tuple<UID, int64_t, int64_t>>& 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<Void> resumeActiveMerges(Reference<BlobManagerData> 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<UID> parentGranuleIDs;
|
||||
|
@ -2641,7 +2641,7 @@ ACTOR Future<Void> recoverBlobManager(Reference<BlobManagerData> bmData) {
|
|||
|
||||
state Future<Void> resumeMergesFuture = resumeActiveMerges(bmData);
|
||||
|
||||
TEST(true); // BM doing recovery
|
||||
CODE_PROBE(true, "BM doing recovery");
|
||||
|
||||
wait(delay(0));
|
||||
|
||||
|
@ -2722,7 +2722,7 @@ ACTOR Future<Void> recoverBlobManager(Reference<BlobManagerData> 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<Void> recoverBlobManager(Reference<BlobManagerData> 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<Void> chaosRangeMover(Reference<BlobManagerData> bmData) {
|
|||
// KeyRange isn't hashable and this is only for simulation, so just use toString of range
|
||||
state std::unordered_set<std::string> 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<Void> initializeBlobWorker(Reference<BlobManagerData> 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<Void> 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<Void> doLockChecks(Reference<BlobManagerData> 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<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(bmData->db);
|
||||
|
||||
|
|
|
@ -539,7 +539,7 @@ ACTOR Future<Void> 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<Void> 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<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> 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<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> 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<BlobFileIndex> dumpInitialSnapshotFromFDB(Reference<BlobWorkerData>
|
|||
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<BlobFileIndex> checkSplitAndReSnapshot(Reference<BlobWorkerData> 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<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> 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<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> 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<Void> granuleCheckMergeCandidate(Reference<BlobWorkerData> 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<GranuleMetadata> 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<GranuleMetadata> 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<GranuleMetadata> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> 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<Void> blobGranuleLoadHistory(Reference<BlobWorkerData> 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<std::pair<KeyRange, Future<GranuleFiles>>> 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<Void> waitForVersion(Reference<GranuleMetadata> 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<Void> waitForVersion(Reference<GranuleMetadata> 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<Void> waitForVersion(Reference<GranuleMetadata> 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<Void> waitForVersion(Reference<GranuleMetadata> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> bwData, Bl
|
|||
state std::vector<std::pair<KeyRange, GranuleFiles>> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> 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<Void> handleBlobGranuleFileRequest(Reference<BlobWorkerData> 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<GranuleStartState> openGranule(Reference<BlobWorkerData> 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<int64_t, int64_t, UID> prevOwner = decodeBlobGranuleLockValue(prevLockValue.get());
|
||||
|
||||
info.granuleID = std::get<2>(prevOwner);
|
||||
|
@ -3335,7 +3334,7 @@ ACTOR Future<GranuleStartState> openGranule(Reference<BlobWorkerData> 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<GranuleStartState> openGranule(Reference<BlobWorkerData> 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<GranuleStartState> openGranule(Reference<BlobWorkerData> 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<GranuleStartState> openGranule(Reference<BlobWorkerData> 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<GranuleStartState> openGranule(Reference<BlobWorkerData> 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<Future<GranuleFiles>> 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<Reference<BlobConnectionProvider>> loadBStoreForTenant(Reference<Bl
|
|||
wait(delay(0));
|
||||
return data->bstore;
|
||||
} 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<Void> monitorRemoval(Reference<BlobWorkerData> bwData) {
|
|||
|
||||
Optional<Value> 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();
|
||||
}
|
||||
|
||||
|
|
|
@ -316,7 +316,7 @@ ACTOR Future<Void> 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<Void> 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<Void> 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
|
||||
|
|
|
@ -58,7 +58,7 @@ ACTOR Future<Void> 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<Void> 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<Void> recruitNewMaster(ClusterControllerData* cluster,
|
|||
|
||||
ACTOR Future<Void> clusterRecruitFromConfiguration(ClusterControllerData* self, Reference<RecruitWorkersInfo> 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<RecruitRemoteFromConfigurationReply> clusterRecruitRemoteFromConfig
|
|||
ClusterControllerData* self,
|
||||
Reference<RecruitRemoteWorkersInfo> 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<Void> newSeedServers(Reference<ClusterRecoveryData> 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<Void> updateLogsValue(Reference<ClusterRecoveryData> 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<Void> updateRegistration(Reference<ClusterRecoveryData> self, Refer
|
|||
std::vector<UID>()));
|
||||
} 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<Void> recoverFrom(Reference<ClusterRecoveryData> self,
|
|||
}
|
||||
when(Standalone<CommitTransactionRef> _req = wait(provisional)) {
|
||||
state Standalone<CommitTransactionRef> 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<Void> clusterRecoveryCore(Reference<ClusterRecoveryData> 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<Void> clusterRecoveryCore(Reference<ClusterRecoveryData> 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();
|
||||
}
|
||||
|
||||
|
|
|
@ -789,7 +789,7 @@ ACTOR Future<Void> 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<Void> 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<Void> 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<Tag> allSources;
|
||||
for (auto r : ranges) {
|
||||
r.value().populateTags();
|
||||
|
@ -1347,7 +1347,7 @@ ACTOR Future<Void> 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<Void> 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<Void> 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<std::set<Tag>> writtenTags;
|
||||
|
@ -2603,7 +2604,7 @@ ACTOR Future<Void> 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();
|
||||
}
|
||||
|
|
|
@ -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())));
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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<TSSPairState>(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<TSSPairState>();
|
||||
} 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<TSSPairState>()));
|
||||
}
|
||||
|
@ -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<Void> 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<TCMachineInfo> DDTeamCollection::checkAndCreateMachine(Reference<TCSer
|
|||
Reference<TCMachineInfo> 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<TCMachineInfo>(server, localityEntry);
|
||||
|
|
|
@ -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<Reference<InitialDataDistribution>> DDTxnProcessor::getInitialDataDistrib
|
|||
|
||||
Future<Void> DDTxnProcessor::waitForDataDistributionEnabled(const DDEnabledState* ddEnabledState) const {
|
||||
return DDTxnProcessorImpl::waitForDataDistributionEnabled(cx, ddEnabledState);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -881,7 +881,7 @@ ACTOR Future<std::map<NetworkAddress, std::pair<WorkerInterface, std::string>>>
|
|||
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<Void> dataDistributor(DataDistributorInterface di, Reference<AsyncV
|
|||
when(DistributorSnapRequest snapReq = waitNext(di.distributorSnapReq.getFuture())) {
|
||||
auto& snapUID = snapReq.snapUID;
|
||||
if (ddSnapReqResultMap.count(snapUID)) {
|
||||
TEST(true); // Data distributor received a duplicate finished snap request
|
||||
CODE_PROBE(true, "Data distributor received a duplicate finished snap request");
|
||||
auto result = ddSnapReqResultMap[snapUID];
|
||||
result.isError() ? snapReq.reply.sendError(result.getError()) : snapReq.reply.send(result.get());
|
||||
TraceEvent("RetryFinishedDistributorSnapRequest")
|
||||
.detail("SnapUID", snapUID)
|
||||
.detail("Result", result.isError() ? result.getError().code() : 0);
|
||||
} else if (ddSnapReqMap.count(snapReq.snapUID)) {
|
||||
TEST(true); // Data distributor received a duplicate ongoing snap request
|
||||
CODE_PROBE(true, "Data distributor received a duplicate ongoing snap request");
|
||||
TraceEvent("RetryOngoingDistributorSnapRequest").detail("SnapUID", snapUID);
|
||||
ASSERT(snapReq.snapPayload == ddSnapReqMap[snapUID].snapPayload);
|
||||
ddSnapReqMap[snapUID] = snapReq;
|
||||
|
@ -1439,4 +1439,4 @@ TEST_CASE("/DataDistributor/StorageWiggler/Order") {
|
|||
}
|
||||
ASSERT(!wiggler.getNextServerId().present());
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1429,7 +1429,7 @@ ACTOR Future<Void> 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<Void> 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<Void> 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;
|
||||
|
|
|
@ -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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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);
|
||||
|
|
|
@ -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<Void> 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<StringRef> 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<Future<Void>> 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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -661,14 +661,14 @@ ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture,
|
|||
if (tagItr != priorityThrottledTags.end()) {
|
||||
if (tagItr->second.expiration > now()) {
|
||||
if (tagItr->second.tpsRate == std::numeric_limits<double>::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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<StringRef> 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));
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ struct PageChecksumCodec {
|
|||
crc32Sum.part1 = 0;
|
||||
crc32Sum.part2 = crc32c_append(0xfdbeefdb, static_cast<uint8_t*>(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<uint32_t>((xxHash3 >> 32) & 0x00ffffff);
|
||||
xxHash3Sum.part2 = static_cast<uint32_t>(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;
|
||||
|
|
|
@ -206,7 +206,7 @@ ACTOR Future<Void> 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;
|
||||
}
|
||||
|
|
|
@ -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<ConfigKnobOverrides>(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);
|
||||
|
|
|
@ -592,7 +592,7 @@ Future<Void> 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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -317,7 +317,7 @@ ACTOR Future<Void> 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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -217,7 +217,7 @@ ACTOR Future<MoveKeysLock> 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<Void> checkMoveKeysLock(Transaction* tr,
|
|||
Optional<Value> readVal = wait(tr->get(moveKeysLockWriteKey));
|
||||
UID lastWrite = readVal.present() ? BinaryReader::fromStringRef<UID>(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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -182,7 +182,7 @@ private:
|
|||
Standalone<StringRef> 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<StringRef> 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<LogData> {
|
|||
ACTOR Future<Void> tLogLock(TLogData* self, ReplyPromise<TLogLockResult> reply, Reference<LogData> 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<Void> updatePersistentData(TLogData* self, Reference<LogData> 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<LogData> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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) {
|
||||
|
|
|
@ -148,7 +148,7 @@ private:
|
|||
Standalone<StringRef> 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<StringRef> 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<Void> tLogLock(TLogData* self, ReplyPromise<TLogLockResult> reply, Reference<LogData> 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<Void> updatePersistentData(TLogData* self, Reference<LogData> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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) {
|
||||
|
|
|
@ -156,7 +156,7 @@ private:
|
|||
Standalone<StringRef> 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<StringRef> 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<Void> tLogLock(TLogData* self, ReplyPromise<TLogLockResult> reply, Reference<LogData> 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<Void> updatePersistentData(TLogData* self, Reference<LogData> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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) {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<VectorRef<ResolverMoveRef>>());
|
||||
}
|
||||
|
|
|
@ -350,7 +350,7 @@ ACTOR Future<Void> resolveBatch(Reference<Resolver> 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<Void> resolveBatch(Reference<Resolver> 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<Void> resolveBatch(Reference<Resolver> 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<Void> resolveBatch(Reference<Resolver> 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<Void> resolveBatch(Reference<Resolver> 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());
|
||||
}
|
||||
|
||||
|
|
|
@ -48,7 +48,7 @@ Optional<double> 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<double>();
|
||||
}
|
||||
|
@ -92,14 +92,14 @@ Optional<double> 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<double>();
|
||||
}
|
||||
|
||||
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<double> RkTagThrottleCollection::autoThrottleTag(UID id,
|
|||
return Optional<double>();
|
||||
}
|
||||
} 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<double>();
|
||||
} else {
|
||||
tpsRate = computeTargetTpsRate(fractionalBusyness,
|
||||
|
@ -121,7 +121,7 @@ Optional<double> 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<double>();
|
||||
}
|
||||
|
||||
|
@ -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<ClientTagThrottleLimits> RkTagThrottleCollection::g
|
|||
if (priorityItr != manualItr->second.end()) {
|
||||
Optional<double> 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<ClientTagThrottleLimits> 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<ClientTagThrottleLimits> 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<double>::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<ClientTagThrottleLimits> 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<ClientTagThrottleLimits> 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<ClientTagThrottleLimits> 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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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<Void> 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<Void> 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<Void> 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<Void> 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<Void> restartSimulatedSystem(std::vector<Future<Void>>* 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<Future<Void>>* 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<TenantName> defaultTenant;
|
||||
state Standalone<VectorRef<TenantNameRef>> 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;
|
||||
|
|
|
@ -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<int>::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<Void> 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<T>& 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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 {
|
||||
|
|
|
@ -157,7 +157,7 @@ private:
|
|||
Standalone<StringRef> 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<StringRef> 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<Void> tLogLock(TLogData* self, ReplyPromise<TLogLockResult> reply, Reference<LogData> 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<Void> updatePersistentData(TLogData* self, Reference<LogData> 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<Void> processPopRequests(TLogData* self, Reference<LogData> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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) {
|
||||
|
|
|
@ -2167,7 +2167,7 @@ ACTOR Future<Void> TagPartitionedLogSystem::epochEnd(Reference<AsyncVar<Referenc
|
|||
}
|
||||
}
|
||||
|
||||
TEST(true); // Master recovery from pre-existing database
|
||||
CODE_PROBE(true, "Master recovery from pre-existing database");
|
||||
|
||||
// trackRejoins listens for rejoin requests from the tLogs that we are recovering from, to learn their
|
||||
// TLogInterfaces
|
||||
|
@ -2228,7 +2228,7 @@ ACTOR Future<Void> TagPartitionedLogSystem::epochEnd(Reference<AsyncVar<Referenc
|
|||
}
|
||||
if (!lockedLocalities.count(log->locality)) {
|
||||
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<Void> TagPartitionedLogSystem::epochEnd(Reference<AsyncVar<Referenc
|
|||
changes.push_back(TagPartitionedLogSystem::getDurableVersionChanged(lockResults[log], logFailed[log]));
|
||||
}
|
||||
if (maxEnd > 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<TagPartitionedLogSystem>(dbgid, locality, prevState.recoveryCount);
|
||||
|
||||
|
|
|
@ -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<double> clientRate = throttledTags.autoThrottleTag(id, tag, busyness);
|
||||
// TODO: Increment tag throttle counts here?
|
||||
if (clientRate.present()) {
|
||||
|
|
|
@ -99,7 +99,7 @@ public:
|
|||
|
||||
void addRequest(Optional<TagSet> 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())];
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ ACTOR Future<Void> waitFailureServer(FutureStream<ReplyPromise<Void>> waitFailur
|
|||
ReplyPromise<Void> 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();
|
||||
}
|
||||
|
|
|
@ -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<std::string> knobName = extractPrefixedArgument("--knob", args.OptionSyntax());
|
||||
if (!knobName.present()) {
|
||||
|
|
|
@ -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<Void>();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -1242,7 +1242,7 @@ Future<T> ioDegradedOrTimeoutError(Future<T> 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);
|
||||
}
|
||||
|
|
|
@ -160,15 +160,16 @@ ACTOR Future<Void> getVersion(Reference<MasterData> 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<Void> getVersion(Reference<MasterData> 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<Void> 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<Void> 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<Void> 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);
|
||||
|
|
|
@ -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<Void> 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<Version> 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<Void> getValue = getValueQ(
|
||||
|
@ -1744,7 +1744,7 @@ ACTOR Future<Version> 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<Version> 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<Void> 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<std::pair<ChangeFeedStreamReply, bool>> 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<std::pair<ChangeFeedStreamReply, bool>> 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<std::pair<ChangeFeedStreamReply, bool>> 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<std::pair<ChangeFeedStreamReply, bool>> 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<Void> 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<Void> 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<Key> 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<int>::max();
|
||||
GetKeyValuesReply rep2 = wait(readRange(data,
|
||||
version,
|
||||
|
@ -3330,7 +3330,7 @@ ACTOR Future<Key> 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<Void> 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<Void> 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<Void> 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<Value>& 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<Version> 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<Version> 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<Version> 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<Version> 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<Version> 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<Version> 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<std::vector<Key>> 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<std::vector<Key>> 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<std::vector<Key>> 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<ChangeFeedInfo>(new ChangeFeedInfo());
|
||||
changeFeedInfo->range = cfEntry.range;
|
||||
|
@ -5685,14 +5686,14 @@ ACTOR Future<std::vector<Key>> 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<std::vector<Key>> 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<std::vector<Key>> 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<Void> 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<Void> 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<Void> 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<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
std::unordered_set<Key> 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<Void> 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<Void> 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<Void> 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<Void> 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<bool> restoreDurableState(StorageServer* data, IKeyValueStore* stor
|
|||
data->clusterId.send(BinaryReader::fromStringRef<UID>(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<bool> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> 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<Void> memoryStoreRecover(IKeyValueStore* store, Reference<IClusterC
|
|||
|
||||
state bool canRemove = wait(canRemoveStorageServer(tr, id));
|
||||
if (!canRemove) {
|
||||
TEST(true); // it's possible that the caller had a transaction in flight that assigned keys to the
|
||||
// server. Wait for it to reverse its mistake.
|
||||
CODE_PROBE(true,
|
||||
"it's possible that the caller had a transaction in flight that assigned keys to the "
|
||||
"server. Wait for it to reverse its mistake.");
|
||||
wait(delayJittered(SERVER_KNOBS->REMOVE_RETRY_DELAY, TaskPriority::UpdateStorage));
|
||||
tr->reset();
|
||||
TraceEvent("RemoveStorageServerRetrying")
|
||||
|
@ -9377,7 +9380,7 @@ ACTOR Future<Void> replaceTSSInterface(StorageServer* self, StorageServerInterfa
|
|||
Optional<Value> 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();
|
||||
}
|
||||
|
|
|
@ -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<Void> handleIOErrors(Future<Void> 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<Void> workerServer(Reference<IClusterConnectionRecord> 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<Void> workerServer(Reference<IClusterConnectionRecord> 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<Void> workerServer(Reference<IClusterConnectionRecord> 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<Void> workerServer(Reference<IClusterConnectionRecord> 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<Void> workerServer(Reference<IClusterConnectionRecord> 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<Void> workerServer(Reference<IClusterConnectionRecord> 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);
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -206,8 +206,8 @@ struct ConsistencyCheckWorkload : TestWorkload {
|
|||
}
|
||||
|
||||
ACTOR Future<Void> 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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -216,7 +216,7 @@ struct DifferentClustersSameRVWorkload : TestWorkload {
|
|||
state Future<Optional<Value>> 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));
|
||||
}
|
||||
|
|
|
@ -329,7 +329,7 @@ struct EncryptKeyProxyTestWorkload : TestWorkload {
|
|||
}
|
||||
|
||||
Future<Void> start(Database const& cx) override {
|
||||
TEST(true); // Testing
|
||||
CODE_PROBE(true, "Testing");
|
||||
if (!enableTest) {
|
||||
return Void();
|
||||
}
|
||||
|
|
|
@ -110,7 +110,7 @@ struct Increment : TestWorkload {
|
|||
}
|
||||
}
|
||||
bool incrementCheckData(const VectorRef<KeyValueRef>& 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")
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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<Void> 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<ReadYourWritesTransaction> tx = makeReference<ReadYourWritesTransaction>(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"))));
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -409,7 +409,7 @@ Reference<EncryptBuf> 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<uint8_t*>(header), 0, sizeof(BlobCipherEncryptHeader));
|
||||
|
||||
|
@ -673,7 +673,7 @@ Reference<EncryptBuf> 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) {
|
||||
|
|
|
@ -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 <map>
|
||||
#include <fmt/format.h>
|
||||
#include <boost/core/demangle.hpp>
|
||||
#include <boost/unordered_map.hpp>
|
||||
#include <typeinfo>
|
||||
|
||||
namespace probe {
|
||||
|
||||
namespace {
|
||||
|
||||
std::vector<ExecutionContext> fromStrings(std::vector<std::string> const& ctxs) {
|
||||
std::vector<ExecutionContext> 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<Location, ICodeProbe const*> codeProbes;
|
||||
|
||||
void traceMissedProbes(Optional<ExecutionContext> 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<std::pair<std::string_view, std::string_view>, 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("<?xml version=\"1.0\" encoding=\"utf-8\"?>\n");
|
||||
fmt::print("<CoverageTool>\n");
|
||||
if (codeProbes.empty()) {
|
||||
fmt::print("\t<CoverageCases/>\n");
|
||||
fmt::print("\t<Inputs/>\n");
|
||||
} else {
|
||||
std::vector<std::string_view> files;
|
||||
fmt::print("\t<CoverageCases>\n");
|
||||
for (auto probe : codeProbes) {
|
||||
files.push_back(probe.first.file);
|
||||
fmt::print("\t\t<Case File=\"{}\" Line=\"{}\" Comment=\"{}\" Condition=\"{}\"/>\n",
|
||||
probe.first.file,
|
||||
probe.first.line,
|
||||
probe.second->comment(),
|
||||
probe.second->condition());
|
||||
}
|
||||
fmt::print("\t</CoverageCases>\n");
|
||||
fmt::print("\t<Inputs>\n");
|
||||
for (auto const& f : files) {
|
||||
fmt::print("\t\t<Input>{}</Input>\n", f);
|
||||
}
|
||||
fmt::print("\t</Inputs>\n");
|
||||
}
|
||||
fmt::print("</CoverageTool>\n");
|
||||
}
|
||||
|
||||
void printJSON(std::vector<std::string> const& context = std::vector<std::string>()) 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<ExecutionContext> context) const {
|
||||
boost::unordered_map<Location, bool> 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<ExecutionContext> 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<std::string> 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
|
|
@ -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);
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue