From 9379bab04e808ce49c039f860dd3e7dccb785f82 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 13 Jul 2021 14:28:46 -0700 Subject: [PATCH 01/26] Move trim to anonymous namespace --- fdbclient/MonitorLeader.actor.cpp | 40 +++++++++++++++++-------------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/fdbclient/MonitorLeader.actor.cpp b/fdbclient/MonitorLeader.actor.cpp index 22bbbfc4b7..920a09e247 100644 --- a/fdbclient/MonitorLeader.actor.cpp +++ b/fdbclient/MonitorLeader.actor.cpp @@ -26,6 +26,28 @@ #include "flow/Platform.h" #include "flow/actorcompiler.h" // has to be last include +namespace { + +std::string trim(std::string const& connectionString) { + // Strip out whitespace + // Strip out characters between a # and a newline + std::string trimmed; + auto end = connectionString.end(); + for (auto c = connectionString.begin(); c != end; ++c) { + if (*c == '#') { + ++c; + while (c != end && *c != '\n' && *c != '\r') + ++c; + if (c == end) + break; + } else if (*c != ' ' && *c != '\n' && *c != '\r' && *c != '\t') + trimmed += *c; + } + return trimmed; +} + +} // namespace + std::pair ClusterConnectionFile::lookupClusterFileName(std::string const& filename) { if (filename.length()) return std::make_pair(filename, false); @@ -154,24 +176,6 @@ std::string ClusterConnectionString::getErrorString(std::string const& source, E } } -std::string trim(std::string const& connectionString) { - // Strip out whitespace - // Strip out characters between a # and a newline - std::string trimmed; - auto end = connectionString.end(); - for (auto c = connectionString.begin(); c != end; ++c) { - if (*c == '#') { - ++c; - while (c != end && *c != '\n' && *c != '\r') - ++c; - if (c == end) - break; - } else if (*c != ' ' && *c != '\n' && *c != '\r' && *c != '\t') - trimmed += *c; - } - return trimmed; -} - ClusterConnectionString::ClusterConnectionString(std::string const& connectionString) { auto trimmed = trim(connectionString); From 3c1cabf04125e80d0f8ad3aef9e910b2854790e7 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Tue, 13 Jul 2021 16:43:09 -0700 Subject: [PATCH 02/26] Coordinator lets client know if it cannot communicate with cluster controller --- fdbclient/MonitorLeader.actor.cpp | 1 + fdbrpc/fdbrpc.h | 8 +++++ fdbserver/Coordination.actor.cpp | 54 +++++++++++++++++++++++++------ flow/error_definitions.h | 1 + 4 files changed, 55 insertions(+), 9 deletions(-) diff --git a/fdbclient/MonitorLeader.actor.cpp b/fdbclient/MonitorLeader.actor.cpp index 920a09e247..f8c1769384 100644 --- a/fdbclient/MonitorLeader.actor.cpp +++ b/fdbclient/MonitorLeader.actor.cpp @@ -842,6 +842,7 @@ ACTOR Future monitorProxiesOneGeneration( clientInfo->set(ni); successIdx = idx; } else { + TEST(rep.getError().code() == error_code_failed_to_progress); // Coordinator cannot talk to cluster controller idx = (idx + 1) % addrs.size(); if (idx == successIdx) { wait(delay(CLIENT_KNOBS->COORDINATOR_RECONNECTION_DELAY)); diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index df13a7fa0c..53eb6b13d5 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -123,6 +123,14 @@ public: void sendError(const E& exc) const { sav->sendError(exc); } + template + void sendErrorOr(U&& value) const { + if (value.present()) { + sav->send(std::forward(value).get()); + } else { + sav->sendError(value.getError()); + } + } Future getFuture() const { sav->addFutureRef(); diff --git a/fdbserver/Coordination.actor.cpp b/fdbserver/Coordination.actor.cpp index eeffd7b4d7..c284c2eabd 100644 --- a/fdbserver/Coordination.actor.cpp +++ b/fdbserver/Coordination.actor.cpp @@ -37,6 +37,30 @@ // This module implements coordinationServer() and the interfaces in CoordinationInterface.h +namespace { + +class LivenessChecker { + double threshold; + AsyncVar lastTime; + ACTOR static Future checkStuck(LivenessChecker const* self) { + loop { + choose { + when(wait(delayUntil(self->lastTime.get() + self->threshold))) { return Void(); } + when(wait(self->lastTime.onChange())) {} + } + } + } + +public: + explicit LivenessChecker(double threshold) : threshold(threshold), lastTime(now()) {} + + void confirmLiveness() { lastTime.set(now()); } + + Future checkStuck() const { return checkStuck(this); } +}; + +} // namespace + struct GenerationRegVal { UniqueGeneration readGen, writeGen; Optional val; @@ -179,7 +203,10 @@ TEST_CASE("/fdbserver/Coordination/localGenerationReg/simple") { ACTOR Future openDatabase(ClientData* db, int* clientCount, Reference> hasConnectedClients, - OpenDatabaseCoordRequest req) { + OpenDatabaseCoordRequest req, + Future checkStuck) { + state ErrorOr> replyContents; + ++(*clientCount); hasConnectedClients->set(true); @@ -191,18 +218,22 @@ ACTOR Future openDatabase(ClientData* db, while (db->clientInfo->get().read().id == req.knownClientInfoID && !db->clientInfo->get().read().forward.present()) { choose { + when(wait(checkStuck)) { + replyContents = failed_to_progress(); + break; + } when(wait(yieldedFuture(db->clientInfo->onChange()))) {} when(wait(delayJittered(SERVER_KNOBS->CLIENT_REGISTER_INTERVAL))) { + if (req.supportedVersions.size() > 0) { + db->clientStatusInfoMap.erase(req.reply.getEndpoint().getPrimaryAddress()); + } + replyContents = db->clientInfo->get(); break; } // The client might be long gone! } } - if (req.supportedVersions.size() > 0) { - db->clientStatusInfoMap.erase(req.reply.getEndpoint().getPrimaryAddress()); - } - - req.reply.send(db->clientInfo->get()); + req.reply.sendErrorOr(replyContents); if (--(*clientCount) == 0) { hasConnectedClients->set(false); @@ -255,6 +286,7 @@ ACTOR Future leaderRegister(LeaderElectionRegInterface interf, Key key) { state AsyncVar leaderInterface; state Reference>> currentElectedLeader = makeReference>>(); + state LivenessChecker canConnectToLeader(20.0); loop choose { when(OpenDatabaseCoordRequest req = waitNext(interf.openDatabase.getFuture())) { @@ -266,7 +298,8 @@ ACTOR Future leaderRegister(LeaderElectionRegInterface interf, Key key) { leaderMon = monitorLeaderForProxies(req.clusterKey, req.coordinators, &clientData, currentElectedLeader); } - actors.add(openDatabase(&clientData, &clientCount, hasConnectedClients, req)); + actors.add( + openDatabase(&clientData, &clientCount, hasConnectedClients, req, canConnectToLeader.checkStuck())); } } when(ElectionResultRequest req = waitNext(interf.electionResult.getFuture())) { @@ -320,8 +353,11 @@ ACTOR Future leaderRegister(LeaderElectionRegInterface interf, Key key) { // TODO: use notify to only send a heartbeat once per interval availableLeaders.erase(LeaderInfo(req.prevChangeID)); availableLeaders.insert(req.myInfo); - req.reply.send( - LeaderHeartbeatReply{ currentNominee.present() && currentNominee.get().equalInternalId(req.myInfo) }); + bool const isCurrentLeader = currentNominee.present() && currentNominee.get().equalInternalId(req.myInfo); + if (isCurrentLeader) { + canConnectToLeader.confirmLiveness(); + } + req.reply.send(LeaderHeartbeatReply{ isCurrentLeader }); } when(ForwardRequest req = waitNext(interf.forward.getFuture())) { LeaderInfo newInfo; diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 8ffb54f290..b69801cfd7 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -99,6 +99,7 @@ ERROR( master_backup_worker_failed, 1212, "Master terminating because a backup w ERROR( tag_throttled, 1213, "Transaction tag is being throttled" ) ERROR( grv_proxy_failed, 1214, "Master terminating because a GRV CommitProxy failed" ) ERROR( dd_tracker_cancelled, 1215, "The data distribution tracker has been cancelled" ) +ERROR( failed_to_progress, 1216, "Process has failed to make sufficient progress" ) // 15xx Platform errors ERROR( platform_error, 1500, "Platform error" ) From e8f0c3c98a52313114094ef27709cc3ddfe39bcc Mon Sep 17 00:00:00 2001 From: hao fu Date: Sat, 17 Jul 2021 15:56:03 -0700 Subject: [PATCH 03/26] Add RepeatableReadMultiThreadClientTest Add RepeatableReadMultiThreadClientTest to verify transactions have repeatable read. --- .../RepeatableReadMultiThreadClientTest.java | 188 ++++++++++++++++++ bindings/java/src/tests.cmake | 1 + 2 files changed, 189 insertions(+) create mode 100644 bindings/java/src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java diff --git a/bindings/java/src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java b/bindings/java/src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java new file mode 100644 index 0000000000..3358c9e760 --- /dev/null +++ b/bindings/java/src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java @@ -0,0 +1,188 @@ +/* + * RepeatableReadMultiThreadClientTest + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2021 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. + */ +package com.apple.foundationdb; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import com.apple.foundationdb.tuple.Tuple; + +import org.junit.jupiter.api.Assertions; + +/** + * This test verify transcations have repeatable read. + * 1 First set initialValue to key. + * 2 Have transactions to read the key and verify the initialValue in a loop, if it does not + * see the initialValue as the value, it set the flag to false. + * + * 3 Then have new transactions set the value and then read to verify the new value is set, + * if it does not read the new value, set the flag to false. + * + * 4 Verify that old transactions have not finished when new transactions have finished, + * then verify old transactions does not have false flag -- it means that old transactions + * are still seeting the initialValue even after new transactions set them to a new value. + */ +public class RepeatableReadMultiThreadClientTest { + public static final MultiClientHelper clientHelper = new MultiClientHelper(); + + private static final int oldValueReadCount = 30; + private static final int threadPerDB = 5; + + private static final String key = "foo"; + private static final String initialValue = "bar"; + private static final String newValue = "cool"; + private static final Map threadToOldValueReaders = new HashMap<>(); + + public static void main(String[] args) throws Exception { + FDB fdb = FDB.selectAPIVersion(710); + setupThreads(fdb); + Collection dbs = clientHelper.openDatabases(fdb); // the clientHelper will close the databases for us + System.out.println("Starting tests"); + setup(dbs); + System.out.println("Start processing and validating"); + readOldValue(dbs); + setNewValueAndRead(dbs); + System.out.println("Test finished"); + } + + private static synchronized void setupThreads(FDB fdb) { + int clientThreadsPerVersion = clientHelper.readClusterFromEnv().length; + fdb.options().setClientThreadsPerVersion(clientThreadsPerVersion); + System.out.printf("thread per version is %d\n", clientThreadsPerVersion); + fdb.options().setExternalClientDirectory("/var/dynamic-conf/lib"); + fdb.options().setTraceEnable("/tmp"); + fdb.options().setKnob("min_trace_severity=5"); + } + + private static void setup(Collection dbs) { + // 0 -> 1 -> 2 -> 3 -> 0 + for (Database db : dbs) { + db.run(tr -> { + tr.set(Tuple.from(key).pack(), Tuple.from(initialValue).pack()); + return null; + }); + } + } + + private static void readOldValue(Collection dbs) throws InterruptedException { + for (Database db : dbs) { + for (int i = 0; i < threadPerDB; i++) { + final OldValueReader oldValueReader = new OldValueReader(db); + final Thread thread = new Thread(OldValueReader.create(db)); + thread.start(); + threadToOldValueReaders.put(thread, oldValueReader); + } + } + } + + private static void setNewValueAndRead(Collection dbs) throws InterruptedException { + // threads running NewValueReader need to wait for threads to start first who run OldValueReader + Thread.sleep(1000); + final Map threads = new HashMap<>(); + for (Database db : dbs) { + for (int i = 0; i < threadPerDB; i++) { + final NewValueReader newValueReader = new NewValueReader(db); + final Thread thread = new Thread(NewValueReader.create(db)); + thread.start(); + threads.put(thread, newValueReader); + } + } + + for (Map.Entry entry : threads.entrySet()) { + entry.getKey().join(); + Assertions.assertTrue(entry.getValue().succeed, "new value reader failed to read the correct value"); + } + + for (Map.Entry entry : threadToOldValueReaders.entrySet()) { + Assertions.assertTrue(entry.getKey().isAlive(), "Old value reader finished too soon, cannot verify repeatable read, succeed is " + entry.getValue().succeed); + } + + for (Map.Entry entry : threadToOldValueReaders.entrySet()) { + entry.getKey().join(); + Assertions.assertTrue(entry.getValue().succeed, "old value reader failed to read the correct value"); + } + } + + public static class OldValueReader implements Runnable { + + private final Database db; + private boolean succeed; + + private OldValueReader(Database db) { + this.db = db; + this.succeed = true; + } + + public static OldValueReader create(Database db) { + return new OldValueReader(db); + } + + @Override + public void run() { + db.run(tr -> { + try { + for (int i = 0; i < oldValueReadCount; i++) { + byte[] result = tr.get(Tuple.from(key).pack()).join(); + String value = Tuple.fromBytes(result).getString(0); + if (!initialValue.equals(value)) { + succeed = false; + break; + } + Thread.sleep(100); + } + } + catch (Exception e) { + succeed = false; + } + return null; + }); + } + } + + public static class NewValueReader implements Runnable { + private final Database db; + private boolean succeed; + + public NewValueReader(Database db) { + this.db = db; + this.succeed = true; + } + + public static NewValueReader create(Database db) { + return new NewValueReader(db); + } + + @Override + public void run() { + db.run(tr -> { + tr.set(Tuple.from(key).pack(), Tuple.from(newValue).pack()); + return null; + }); + String value = db.run(tr -> { + byte[] result = tr.get(Tuple.from(key).pack()).join(); + return Tuple.fromBytes(result).getString(0); + }); + if (!newValue.equals(value)) { + succeed = false; + } + } + } +} diff --git a/bindings/java/src/tests.cmake b/bindings/java/src/tests.cmake index 63fb18322d..3e9dce6657 100644 --- a/bindings/java/src/tests.cmake +++ b/bindings/java/src/tests.cmake @@ -51,6 +51,7 @@ set(JAVA_INTEGRATION_TESTS src/integration/com/apple/foundationdb/BasicMultiClientIntegrationTest.java src/integration/com/apple/foundationdb/CycleMultiClientIntegrationTest.java src/integration/com/apple/foundationdb/SidebandMultiThreadClientTest.java + src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java ) # Resources that are used in integration testing, but are not explicitly test files (JUnit rules, From d46feb54682566a14eb96dafd82f02e8cee491a5 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 19 Jul 2021 20:17:46 -0700 Subject: [PATCH 04/26] Add COORDINATOR_LEADER_CONNECTION_TIMEOUT server knob --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/Coordination.actor.cpp | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 994f367292..0d3226ea35 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -652,6 +652,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // Coordination init( COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL, 1.0 ); if( randomize && BUGGIFY ) COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL = 10.0; init( ENABLE_CROSS_CLUSTER_SUPPORT, true ); if( randomize && BUGGIFY ) ENABLE_CROSS_CLUSTER_SUPPORT = false; + init( COORDINATOR_LEADER_CONNECTION_TIMEOUT, 20.0 ); // Buggification init( BUGGIFIED_EVENTUAL_CONSISTENCY, 1.0 ); diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 79600d49bb..c8a093a091 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -593,6 +593,7 @@ public: double COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL; bool ENABLE_CROSS_CLUSTER_SUPPORT; // Allow a coordinator to serve requests whose connection string does not match // the local descriptor + double COORDINATOR_LEADER_CONNECTION_TIMEOUT; // Buggification double BUGGIFIED_EVENTUAL_CONSISTENCY; diff --git a/fdbserver/Coordination.actor.cpp b/fdbserver/Coordination.actor.cpp index c284c2eabd..d9975c5ecc 100644 --- a/fdbserver/Coordination.actor.cpp +++ b/fdbserver/Coordination.actor.cpp @@ -286,7 +286,7 @@ ACTOR Future leaderRegister(LeaderElectionRegInterface interf, Key key) { state AsyncVar leaderInterface; state Reference>> currentElectedLeader = makeReference>>(); - state LivenessChecker canConnectToLeader(20.0); + state LivenessChecker canConnectToLeader(SERVER_KNOBS->COORDINATOR_LEADER_CONNECTION_TIMEOUT); loop choose { when(OpenDatabaseCoordRequest req = waitNext(interf.openDatabase.getFuture())) { From 6836e49073759f6df09b672dcfb88d13382ab5be Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 19 Jul 2021 21:00:28 -0700 Subject: [PATCH 05/26] Throw error when commitProxy gets stuck --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/CommitProxyServer.actor.cpp | 16 +++++++++++----- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 0d3226ea35..aeed11b925 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -364,6 +364,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET, 10.0 ); init( START_TRANSACTION_MAX_QUEUE_SIZE, 1e6 ); init( KEY_LOCATION_MAX_QUEUE_SIZE, 1e6 ); + init( COMMIT_PROXY_LIVENESS_TIMEOUT, 20.0 ); init( COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE, 0.0005 ); if( randomize && BUGGIFY ) COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE = 0.005; init( COMMIT_TRANSACTION_BATCH_INTERVAL_MIN, 0.001 ); if( randomize && BUGGIFY ) COMMIT_TRANSACTION_BATCH_INTERVAL_MIN = 0.1; diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index c8a093a091..18c2a93eac 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -297,6 +297,7 @@ public: double START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET; int START_TRANSACTION_MAX_QUEUE_SIZE; int KEY_LOCATION_MAX_QUEUE_SIZE; + double COMMIT_PROXY_LIVENESS_TIMEOUT; double COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE; double COMMIT_TRANSACTION_BATCH_INTERVAL_MIN; diff --git a/fdbserver/CommitProxyServer.actor.cpp b/fdbserver/CommitProxyServer.actor.cpp index 337a24e956..63fd4e9016 100644 --- a/fdbserver/CommitProxyServer.actor.cpp +++ b/fdbserver/CommitProxyServer.actor.cpp @@ -1922,10 +1922,14 @@ ACTOR Future commitProxyServerCore(CommitProxyInterface proxy, lastCommit = now(); if (trs.size() || lastCommitComplete.isReady()) { - lastCommitComplete = - commitBatch(&commitData, - const_cast*>(&batchedRequests.first), - batchBytes); + lastCommitComplete = transformError( + timeoutError( + commitBatch(&commitData, + const_cast*>(&batchedRequests.first), + batchBytes), + SERVER_KNOBS->COMMIT_PROXY_LIVENESS_TIMEOUT), + timed_out(), + failed_to_progress()); addActor.send(lastCommitComplete); } } @@ -2067,9 +2071,11 @@ ACTOR Future commitProxyServer(CommitProxyInterface proxy, if (e.code() != error_code_worker_removed && e.code() != error_code_tlog_stopped && e.code() != error_code_master_tlog_failed && e.code() != error_code_coordinators_changed && - e.code() != error_code_coordinated_state_conflict && e.code() != error_code_new_coordinators_timed_out) { + e.code() != error_code_coordinated_state_conflict && e.code() != error_code_new_coordinators_timed_out && + e.code() != error_code_failed_to_progress) { throw; } + TEST(e.code() == error_code_failed_to_progress); // Commit proxy failed to progress } return Void(); } From 1dc9839e49f0fa7c44f08ffd94b4e5c364aca0f8 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 19 Jul 2021 22:34:27 -0700 Subject: [PATCH 06/26] Increase max latency for LowLatencySingleClog test --- tests/fast/LowLatencySingleClog.toml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/fast/LowLatencySingleClog.toml b/tests/fast/LowLatencySingleClog.toml index cbfe6682e3..7a76141504 100644 --- a/tests/fast/LowLatencySingleClog.toml +++ b/tests/fast/LowLatencySingleClog.toml @@ -9,12 +9,14 @@ connectionFailuresDisableDuration = 100000 [[test.workload]] testName = 'Cycle' transactionsPerSecond = 1000.0 - testDuration = 30.0 + testDuration = 60.0 expectedRate = 0 [[test.workload]] testName = 'LowLatency' - testDuration = 30.0 + maxGRVLatency = 40.0 + maxCommitLatency = 40.0 + testDuration = 60.0 [[test.workload]] testName = 'ClogSingleConnection' From 62026ce42a525ae7e987c3ae9351e785a0c57c33 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 19 Jul 2021 22:51:47 -0700 Subject: [PATCH 07/26] Remove ReplyPromise::sendErrorOr --- fdbrpc/fdbrpc.h | 8 -------- fdbserver/Coordination.actor.cpp | 6 +++++- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 53eb6b13d5..df13a7fa0c 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -123,14 +123,6 @@ public: void sendError(const E& exc) const { sav->sendError(exc); } - template - void sendErrorOr(U&& value) const { - if (value.present()) { - sav->send(std::forward(value).get()); - } else { - sav->sendError(value.getError()); - } - } Future getFuture() const { sav->addFutureRef(); diff --git a/fdbserver/Coordination.actor.cpp b/fdbserver/Coordination.actor.cpp index d9975c5ecc..fc417a7539 100644 --- a/fdbserver/Coordination.actor.cpp +++ b/fdbserver/Coordination.actor.cpp @@ -233,7 +233,11 @@ ACTOR Future openDatabase(ClientData* db, } } - req.reply.sendErrorOr(replyContents); + if (replyContents.present()) { + req.reply.send(replyContents.get()); + } else { + req.reply.sendError(replyContents.getError()); + } if (--(*clientCount) == 0) { hasConnectedClients->set(false); From 34fecb61de698d0df8c033438fa28dab98c26513 Mon Sep 17 00:00:00 2001 From: Dan Lambright Date: Wed, 9 Jun 2021 14:24:00 -0400 Subject: [PATCH 08/26] Reject connections to clusters forwarded in the (configurable) past --- fdbclient/ServerKnobs.cpp | 1 + fdbserver/Coordination.actor.cpp | 34 +- fdbserver/Knobs.h | 644 ++++++++++++++++++++++++++++++- 3 files changed, 677 insertions(+), 2 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 6d74c2f67a..4377afaa1b 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -653,6 +653,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi // Coordination init( COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL, 1.0 ); if( randomize && BUGGIFY ) COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL = 10.0; + init( FORWARD_REQUEST_TOO_OLD, 4*24*60*60 ); if( randomize && BUGGIFY ) FORWARD_REQUEST_TOO_OLD = 60.0; init( ENABLE_CROSS_CLUSTER_SUPPORT, true ); if( randomize && BUGGIFY ) ENABLE_CROSS_CLUSTER_SUPPORT = false; // Buggification diff --git a/fdbserver/Coordination.actor.cpp b/fdbserver/Coordination.actor.cpp index eeffd7b4d7..fc603746d4 100644 --- a/fdbserver/Coordination.actor.cpp +++ b/fdbserver/Coordination.actor.cpp @@ -425,12 +425,18 @@ const KeyRangeRef fwdKeys(LiteralStringRef("\xff" LiteralStringRef("\xff" "fwe")); +// The time when forwarding was last set is stored in this range: +const KeyRangeRef fwdTimeKeys(LiteralStringRef("\xff" + "fwdTime"), + LiteralStringRef("\xff" + "fwdTimf")); struct LeaderRegisterCollection { // SOMEDAY: Factor this into a generic tool? Extend ActorCollection to support removal actions? What? ActorCollection actors; Map registerInterfaces; Map forward; OnDemandStore* pStore; + Map forwardStartTime; LeaderRegisterCollection(OnDemandStore* pStore) : actors(false), pStore(pStore) {} @@ -438,32 +444,58 @@ struct LeaderRegisterCollection { if (!self->pStore->exists()) return Void(); OnDemandStore& store = *self->pStore; - RangeResult forwardingInfo = wait(store->readRange(fwdKeys)); + state Future> forwardingInfoF = store->readRange(fwdKeys); + state Future> forwardingTimeF = store->readRange(fwdTimeKeys); + wait(success(forwardingInfoF) && success(forwardingTimeF)); + Standalone forwardingInfo = forwardingInfoF.get(); + Standalone forwardingTime = forwardingTimeF.get(); for (int i = 0; i < forwardingInfo.size(); i++) { LeaderInfo forwardInfo; forwardInfo.forward = true; forwardInfo.serializedInfo = forwardingInfo[i].value; self->forward[forwardingInfo[i].key.removePrefix(fwdKeys.begin)] = forwardInfo; } + for (int i = 0; i < forwardingTime.size(); i++) { + double time = std::stod(forwardingTime[i].value.toString().c_str()); + self->forwardStartTime[forwardingTime[i].key.removePrefix(fwdTimeKeys.begin)] = time; + } return Void(); } Future onError() { return actors.getResult(); } + // Check if the this coordinator is no longer the leader, and the new one was stored in the "forward" keyspace. + // If the "forward" keyspace was set some time ago (as configured by knob), log an error to indicate the client is + // using a very old cluster file. Optional getForward(KeyRef key) { auto i = forward.find(key); + auto t = forwardStartTime.find(key); if (i == forward.end()) return Optional(); + if (t != forwardStartTime.end()) { + double forwardTime = t->value; + if (now() - forwardTime > SERVER_KNOBS->FORWARD_REQUEST_TOO_OLD) { + TraceEvent(SevWarnAlways, "AccessOldForward") + .detail("ForwardSetSecondsAgo", now() - forwardTime) + .detail("ForwardClusterKey", key); + } + } return i->value; } + // When the lead coordinator changes, store the new connection ID in the "fwd" keyspace. + // If a request arrives using an old connection id, resend it to the new coordinator using the stored connection id. + // Store when this change took place in the fwdTime keyspace. ACTOR static Future setForward(LeaderRegisterCollection* self, KeyRef key, ClusterConnectionString conn) { + double forwardTime = now(); LeaderInfo forwardInfo; forwardInfo.forward = true; forwardInfo.serializedInfo = conn.toString(); self->forward[key] = forwardInfo; + self->forwardStartTime[key] = forwardTime; OnDemandStore& store = *self->pStore; store->set(KeyValueRef(key.withPrefix(fwdKeys.begin), conn.toString())); + store->set(KeyValueRef(key.withPrefix(fwdTimeKeys.begin), std::to_string(forwardTime))); wait(store->commit()); return Void(); } diff --git a/fdbserver/Knobs.h b/fdbserver/Knobs.h index 67f474b0eb..44b653bd68 100644 --- a/fdbserver/Knobs.h +++ b/fdbserver/Knobs.h @@ -22,4 +22,646 @@ #include "fdbclient/IKnobCollection.h" -#define SERVER_KNOBS (&IKnobCollection::getGlobalKnobCollection().getServerKnobs()) +// Disk queue +static const int _PAGE_SIZE = 4096; + +class ServerKnobs : public Knobs { +public: + // Versions + int64_t VERSIONS_PER_SECOND; + int64_t MAX_VERSIONS_IN_FLIGHT; + int64_t MAX_VERSIONS_IN_FLIGHT_FORCED; + int64_t MAX_READ_TRANSACTION_LIFE_VERSIONS; + int64_t MAX_WRITE_TRANSACTION_LIFE_VERSIONS; + double MAX_COMMIT_BATCH_INTERVAL; // Each commit proxy generates a CommitTransactionBatchRequest at least this + // often, so that versions always advance smoothly + + // TLogs + double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time + double TLOG_SLOW_REJOIN_WARN_TIMEOUT_SECS; // Warns if a tlog takes too long to rejoin + double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification + double TLOG_STORAGE_MIN_UPDATE_INTERVAL; + double BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL; + int DESIRED_TOTAL_BYTES; + int DESIRED_UPDATE_BYTES; + double UPDATE_DELAY; + int MAXIMUM_PEEK_BYTES; + int APPLY_MUTATION_BYTES; + int RECOVERY_DATA_BYTE_LIMIT; + int BUGGIFY_RECOVERY_DATA_LIMIT; + double LONG_TLOG_COMMIT_TIME; + int64_t LARGE_TLOG_COMMIT_BYTES; + double BUGGIFY_RECOVER_MEMORY_LIMIT; + double BUGGIFY_WORKER_REMOVED_MAX_LAG; + int64_t UPDATE_STORAGE_BYTE_LIMIT; + int64_t REFERENCE_SPILL_UPDATE_STORAGE_BYTE_LIMIT; + double TLOG_PEEK_DELAY; + int LEGACY_TLOG_UPGRADE_ENTRIES_PER_VERSION; + int VERSION_MESSAGES_OVERHEAD_FACTOR_1024THS; // Multiplicative factor to bound total space used to store a version + // message (measured in 1/1024ths, e.g. a value of 2048 yields a + // factor of 2). + int64_t VERSION_MESSAGES_ENTRY_BYTES_WITH_OVERHEAD; + double TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR; + int64_t TLOG_MESSAGE_BLOCK_BYTES; + int64_t MAX_MESSAGE_SIZE; + int LOG_SYSTEM_PUSHED_DATA_BLOCK_SIZE; + double PEEK_TRACKER_EXPIRATION_TIME; + int PARALLEL_GET_MORE_REQUESTS; + int MULTI_CURSOR_PRE_FETCH_LIMIT; + int64_t MAX_QUEUE_COMMIT_BYTES; + int DESIRED_OUTSTANDING_MESSAGES; + double DESIRED_GET_MORE_DELAY; + int CONCURRENT_LOG_ROUTER_READS; + int LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED; // 0==peek from primary, non-zero==peek from satellites + double DISK_QUEUE_ADAPTER_MIN_SWITCH_TIME; + double DISK_QUEUE_ADAPTER_MAX_SWITCH_TIME; + int64_t TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES; + int64_t TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; + int64_t TLOG_SPILL_REFERENCE_MAX_BYTES_PER_BATCH; + int64_t DISK_QUEUE_FILE_EXTENSION_BYTES; // When we grow the disk queue, by how many bytes should it grow? + int64_t DISK_QUEUE_FILE_SHRINK_BYTES; // When we shrink the disk queue, by how many bytes should it shrink? + int64_t DISK_QUEUE_MAX_TRUNCATE_BYTES; // A truncate larger than this will cause the file to be replaced instead. + double TLOG_DEGRADED_DURATION; + int64_t MAX_CACHE_VERSIONS; + double TXS_POPPED_MAX_DELAY; + double TLOG_MAX_CREATE_DURATION; + int PEEK_LOGGING_AMOUNT; + double PEEK_LOGGING_DELAY; + double PEEK_RESET_INTERVAL; + double PEEK_MAX_LATENCY; + bool PEEK_COUNT_SMALL_MESSAGES; + double PEEK_STATS_INTERVAL; + double PEEK_STATS_SLOW_AMOUNT; + double PEEK_STATS_SLOW_RATIO; + double PUSH_RESET_INTERVAL; + double PUSH_MAX_LATENCY; + double PUSH_STATS_INTERVAL; + double PUSH_STATS_SLOW_AMOUNT; + double PUSH_STATS_SLOW_RATIO; + int TLOG_POP_BATCH_SIZE; + + // Data distribution queue + double HEALTH_POLL_TIME; + double BEST_TEAM_STUCK_DELAY; + double BG_REBALANCE_POLLING_INTERVAL; + double BG_REBALANCE_SWITCH_CHECK_INTERVAL; + double DD_QUEUE_LOGGING_INTERVAL; + double RELOCATION_PARALLELISM_PER_SOURCE_SERVER; + int DD_QUEUE_MAX_KEY_SERVERS; + int DD_REBALANCE_PARALLELISM; + int DD_REBALANCE_RESET_AMOUNT; + double BG_DD_MAX_WAIT; + double BG_DD_MIN_WAIT; + double BG_DD_INCREASE_RATE; + double BG_DD_DECREASE_RATE; + double BG_DD_SATURATION_DELAY; + double INFLIGHT_PENALTY_HEALTHY; + double INFLIGHT_PENALTY_REDUNDANT; + double INFLIGHT_PENALTY_UNHEALTHY; + double INFLIGHT_PENALTY_ONE_LEFT; + bool USE_OLD_NEEDED_SERVERS; + + // Higher priorities are executed first + // Priority/100 is the "priority group"/"superpriority". Priority inversion + // is possible within but not between priority groups; fewer priority groups + // mean better worst case time bounds + // Maximum allowable priority is 999. + int PRIORITY_RECOVER_MOVE; + int PRIORITY_REBALANCE_UNDERUTILIZED_TEAM; + int PRIORITY_REBALANCE_OVERUTILIZED_TEAM; + int PRIORITY_TEAM_HEALTHY; + int PRIORITY_TEAM_CONTAINS_UNDESIRED_SERVER; + int PRIORITY_TEAM_REDUNDANT; + int PRIORITY_MERGE_SHARD; + int PRIORITY_POPULATE_REGION; + int PRIORITY_TEAM_UNHEALTHY; + int PRIORITY_TEAM_2_LEFT; + int PRIORITY_TEAM_1_LEFT; + int PRIORITY_TEAM_FAILED; // Priority when a server in the team is excluded as failed + int PRIORITY_TEAM_0_LEFT; + int PRIORITY_SPLIT_SHARD; + + // Data distribution + double RETRY_RELOCATESHARD_DELAY; + double DATA_DISTRIBUTION_FAILURE_REACTION_TIME; + int MIN_SHARD_BYTES, SHARD_BYTES_RATIO, SHARD_BYTES_PER_SQRT_BYTES, MAX_SHARD_BYTES, KEY_SERVER_SHARD_BYTES; + int64_t SHARD_MAX_BYTES_PER_KSEC, // Shards with more than this bandwidth will be split immediately + SHARD_MIN_BYTES_PER_KSEC, // Shards with more than this bandwidth will not be merged + SHARD_SPLIT_BYTES_PER_KSEC; // When splitting a shard, it is split into pieces with less than this bandwidth + double SHARD_MAX_READ_DENSITY_RATIO; + int64_t SHARD_READ_HOT_BANDWITH_MIN_PER_KSECONDS; + double SHARD_MAX_BYTES_READ_PER_KSEC_JITTER; + double STORAGE_METRIC_TIMEOUT; + double METRIC_DELAY; + double ALL_DATA_REMOVED_DELAY; + double INITIAL_FAILURE_REACTION_DELAY; + double CHECK_TEAM_DELAY; + double LOG_ON_COMPLETION_DELAY; + int BEST_TEAM_MAX_TEAM_TRIES; + int BEST_TEAM_OPTION_COUNT; + int BEST_OF_AMT; + double SERVER_LIST_DELAY; + double RECRUITMENT_IDLE_DELAY; + double STORAGE_RECRUITMENT_DELAY; + bool TSS_HACK_IDENTITY_MAPPING; + double TSS_RECRUITMENT_TIMEOUT; + double TSS_DD_KILL_INTERVAL; + double DATA_DISTRIBUTION_LOGGING_INTERVAL; + double DD_ENABLED_CHECK_DELAY; + double DD_STALL_CHECK_DELAY; + double DD_LOW_BANDWIDTH_DELAY; + double DD_MERGE_COALESCE_DELAY; + double STORAGE_METRICS_POLLING_DELAY; + double STORAGE_METRICS_RANDOM_DELAY; + double AVAILABLE_SPACE_RATIO_CUTOFF; + int DESIRED_TEAMS_PER_SERVER; + int MAX_TEAMS_PER_SERVER; + int64_t DD_SHARD_SIZE_GRANULARITY; + int64_t DD_SHARD_SIZE_GRANULARITY_SIM; + int DD_MOVE_KEYS_PARALLELISM; + int DD_FETCH_SOURCE_PARALLELISM; + int DD_MERGE_LIMIT; + double DD_SHARD_METRICS_TIMEOUT; + int64_t DD_LOCATION_CACHE_SIZE; + double MOVEKEYS_LOCK_POLLING_DELAY; + double DEBOUNCE_RECRUITING_DELAY; + int REBALANCE_MAX_RETRIES; + int DD_OVERLAP_PENALTY; + int DD_EXCLUDE_MIN_REPLICAS; + bool DD_VALIDATE_LOCALITY; + int DD_CHECK_INVALID_LOCALITY_DELAY; + bool DD_ENABLE_VERBOSE_TRACING; + int64_t + DD_SS_FAILURE_VERSIONLAG; // Allowed SS version lag from the current read version before marking it as failed. + int64_t DD_SS_ALLOWED_VERSIONLAG; // SS will be marked as healthy if it's version lag goes below this value. + double DD_SS_STUCK_TIME_LIMIT; // If a storage server is not getting new versions for this amount of time, then it + // becomes undesired. + int DD_TEAMS_INFO_PRINT_INTERVAL; + int DD_TEAMS_INFO_PRINT_YIELD_COUNT; + int DD_TEAM_ZERO_SERVER_LEFT_LOG_DELAY; + + // TeamRemover to remove redundant teams + bool TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER; // disable the machineTeamRemover actor + double TR_REMOVE_MACHINE_TEAM_DELAY; // wait for the specified time before try to remove next machine team + bool TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS; // guard to select which machineTeamRemover logic to use + + bool TR_FLAG_DISABLE_SERVER_TEAM_REMOVER; // disable the serverTeamRemover actor + double TR_REMOVE_SERVER_TEAM_DELAY; // wait for the specified time before try to remove next server team + double TR_REMOVE_SERVER_TEAM_EXTRA_DELAY; // serverTeamRemover waits for the delay and check DD healthyness again to + // ensure it runs after machineTeamRemover + + // Remove wrong storage engines + double DD_REMOVE_STORE_ENGINE_DELAY; // wait for the specified time before remove the next batch + + double DD_FAILURE_TIME; + double DD_ZERO_HEALTHY_TEAM_DELAY; + + // Redwood Storage Engine + int PREFIX_TREE_IMMEDIATE_KEY_SIZE_LIMIT; + int PREFIX_TREE_IMMEDIATE_KEY_SIZE_MIN; + + // KeyValueStore SQLITE + int CLEAR_BUFFER_SIZE; + double READ_VALUE_TIME_ESTIMATE; + double READ_RANGE_TIME_ESTIMATE; + double SET_TIME_ESTIMATE; + double CLEAR_TIME_ESTIMATE; + double COMMIT_TIME_ESTIMATE; + int CHECK_FREE_PAGE_AMOUNT; + double DISK_METRIC_LOGGING_INTERVAL; + int64_t SOFT_HEAP_LIMIT; + + int SQLITE_PAGE_SCAN_ERROR_LIMIT; + int SQLITE_BTREE_PAGE_USABLE; + int SQLITE_BTREE_CELL_MAX_LOCAL; + int SQLITE_BTREE_CELL_MIN_LOCAL; + int SQLITE_FRAGMENT_PRIMARY_PAGE_USABLE; + int SQLITE_FRAGMENT_OVERFLOW_PAGE_USABLE; + double SQLITE_FRAGMENT_MIN_SAVINGS; + int SQLITE_CHUNK_SIZE_PAGES; + int SQLITE_CHUNK_SIZE_PAGES_SIM; + int SQLITE_READER_THREADS; + int SQLITE_WRITE_WINDOW_LIMIT; + double SQLITE_WRITE_WINDOW_SECONDS; + + // KeyValueStoreSqlite spring cleaning + double SPRING_CLEANING_NO_ACTION_INTERVAL; + double SPRING_CLEANING_LAZY_DELETE_INTERVAL; + double SPRING_CLEANING_VACUUM_INTERVAL; + double SPRING_CLEANING_LAZY_DELETE_TIME_ESTIMATE; + double SPRING_CLEANING_VACUUM_TIME_ESTIMATE; + double SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE; + int SPRING_CLEANING_MIN_LAZY_DELETE_PAGES; + int SPRING_CLEANING_MAX_LAZY_DELETE_PAGES; + int SPRING_CLEANING_LAZY_DELETE_BATCH_SIZE; + int SPRING_CLEANING_MIN_VACUUM_PAGES; + int SPRING_CLEANING_MAX_VACUUM_PAGES; + + // KeyValueStoreMemory + int64_t REPLACE_CONTENTS_BYTES; + + // KeyValueStoreRocksDB + int ROCKSDB_BACKGROUND_PARALLELISM; + int ROCKSDB_READ_PARALLELISM; + int64_t ROCKSDB_MEMTABLE_BYTES; + bool ROCKSDB_UNSAFE_AUTO_FSYNC; + int64_t ROCKSDB_PERIODIC_COMPACTION_SECONDS; + int ROCKSDB_PREFIX_LEN; + int64_t ROCKSDB_BLOCK_CACHE_SIZE; + + // Leader election + int MAX_NOTIFICATIONS; + int MIN_NOTIFICATIONS; + double NOTIFICATION_FULL_CLEAR_TIME; + double CANDIDATE_MIN_DELAY; + double CANDIDATE_MAX_DELAY; + double CANDIDATE_GROWTH_RATE; + double POLLING_FREQUENCY; + double HEARTBEAT_FREQUENCY; + + // Commit CommitProxy + double START_TRANSACTION_BATCH_INTERVAL_MIN; + double START_TRANSACTION_BATCH_INTERVAL_MAX; + double START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; + double START_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA; + double START_TRANSACTION_BATCH_QUEUE_CHECK_INTERVAL; + double START_TRANSACTION_MAX_TRANSACTIONS_TO_START; + int START_TRANSACTION_MAX_REQUESTS_TO_START; + double START_TRANSACTION_RATE_WINDOW; + double START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET; + int START_TRANSACTION_MAX_QUEUE_SIZE; + int KEY_LOCATION_MAX_QUEUE_SIZE; + + double COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE; + double COMMIT_TRANSACTION_BATCH_INTERVAL_MIN; + double COMMIT_TRANSACTION_BATCH_INTERVAL_MAX; + double COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; + double COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA; + int COMMIT_TRANSACTION_BATCH_COUNT_MAX; + int COMMIT_TRANSACTION_BATCH_BYTES_MIN; + int COMMIT_TRANSACTION_BATCH_BYTES_MAX; + double COMMIT_TRANSACTION_BATCH_BYTES_SCALE_BASE; + double COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER; + int64_t COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT; + double COMMIT_BATCHES_MEM_FRACTION_OF_TOTAL; + double COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR; + + double RESOLVER_COALESCE_TIME; + int BUGGIFIED_ROW_LIMIT; + double PROXY_SPIN_DELAY; + double UPDATE_REMOTE_LOG_VERSION_INTERVAL; + int MAX_TXS_POP_VERSION_HISTORY; + double MIN_CONFIRM_INTERVAL; + double ENFORCED_MIN_RECOVERY_DURATION; + double REQUIRED_MIN_RECOVERY_DURATION; + bool ALWAYS_CAUSAL_READ_RISKY; + int MAX_COMMIT_UPDATES; + double MAX_PROXY_COMPUTE; + double MAX_COMPUTE_PER_OPERATION; + int PROXY_COMPUTE_BUCKETS; + double PROXY_COMPUTE_GROWTH_RATE; + int TXN_STATE_SEND_AMOUNT; + double REPORT_TRANSACTION_COST_ESTIMATION_DELAY; + bool PROXY_REJECT_BATCH_QUEUED_TOO_LONG; + + int RESET_MASTER_BATCHES; + int RESET_RESOLVER_BATCHES; + double RESET_MASTER_DELAY; + double RESET_RESOLVER_DELAY; + + // Master Server + double COMMIT_SLEEP_TIME; + double MIN_BALANCE_TIME; + int64_t MIN_BALANCE_DIFFERENCE; + double SECONDS_BEFORE_NO_FAILURE_DELAY; + int64_t MAX_TXS_SEND_MEMORY; + int64_t MAX_RECOVERY_VERSIONS; + double MAX_RECOVERY_TIME; + double PROVISIONAL_START_DELAY; + double PROVISIONAL_DELAY_GROWTH; + double PROVISIONAL_MAX_DELAY; + double SECONDS_BEFORE_RECRUIT_BACKUP_WORKER; + double CC_INTERFACE_TIMEOUT; + + // Resolver + int64_t KEY_BYTES_PER_SAMPLE; + int64_t SAMPLE_OFFSET_PER_KEY; + double SAMPLE_EXPIRATION_TIME; + double SAMPLE_POLL_TIME; + int64_t RESOLVER_STATE_MEMORY_LIMIT; + + // Backup Worker + double BACKUP_TIMEOUT; // master's reaction time for backup failure + double BACKUP_NOOP_POP_DELAY; + int BACKUP_FILE_BLOCK_BYTES; + int64_t BACKUP_LOCK_BYTES; + double BACKUP_UPLOAD_DELAY; + + // Cluster Controller + double CLUSTER_CONTROLLER_LOGGING_DELAY; + double MASTER_FAILURE_REACTION_TIME; + double MASTER_FAILURE_SLOPE_DURING_RECOVERY; + int WORKER_COORDINATION_PING_DELAY; + double SIM_SHUTDOWN_TIMEOUT; + double SHUTDOWN_TIMEOUT; + double MASTER_SPIN_DELAY; + double CC_CHANGE_DELAY; + double CC_CLASS_DELAY; + double WAIT_FOR_GOOD_RECRUITMENT_DELAY; + double WAIT_FOR_GOOD_REMOTE_RECRUITMENT_DELAY; + double ATTEMPT_RECRUITMENT_DELAY; + double WAIT_FOR_DISTRIBUTOR_JOIN_DELAY; + double WAIT_FOR_RATEKEEPER_JOIN_DELAY; + double WORKER_FAILURE_TIME; + double CHECK_OUTSTANDING_INTERVAL; + double INCOMPATIBLE_PEERS_LOGGING_INTERVAL; + double VERSION_LAG_METRIC_INTERVAL; + int64_t MAX_VERSION_DIFFERENCE; + double FORCE_RECOVERY_CHECK_DELAY; + double RATEKEEPER_FAILURE_TIME; + double REPLACE_INTERFACE_DELAY; + double REPLACE_INTERFACE_CHECK_DELAY; + double COORDINATOR_REGISTER_INTERVAL; + double CLIENT_REGISTER_INTERVAL; + + // Knobs used to select the best policy (via monte carlo) + int POLICY_RATING_TESTS; // number of tests per policy (in order to compare) + int POLICY_GENERATIONS; // number of policies to generate + + int EXPECTED_MASTER_FITNESS; + int EXPECTED_TLOG_FITNESS; + int EXPECTED_LOG_ROUTER_FITNESS; + int EXPECTED_COMMIT_PROXY_FITNESS; + int EXPECTED_GRV_PROXY_FITNESS; + int EXPECTED_RESOLVER_FITNESS; + double RECRUITMENT_TIMEOUT; + int DBINFO_SEND_AMOUNT; + double DBINFO_BATCH_DELAY; + + // Move Keys + double SHARD_READY_DELAY; + double SERVER_READY_QUORUM_INTERVAL; + double SERVER_READY_QUORUM_TIMEOUT; + double REMOVE_RETRY_DELAY; + int MOVE_KEYS_KRM_LIMIT; + int MOVE_KEYS_KRM_LIMIT_BYTES; // This must be sufficiently larger than CLIENT_KNOBS->KEY_SIZE_LIMIT + // (fdbclient/Knobs.h) to ensure that at least two entries will be returned from an + // attempt to read a key range map + int MAX_SKIP_TAGS; + double MAX_ADDED_SOURCES_MULTIPLIER; + + // FdbServer + double MIN_REBOOT_TIME; + double MAX_REBOOT_TIME; + std::string LOG_DIRECTORY; + int64_t SERVER_MEM_LIMIT; + double SYSTEM_MONITOR_FREQUENCY; + + // Ratekeeper + double SMOOTHING_AMOUNT; + double SLOW_SMOOTHING_AMOUNT; + double METRIC_UPDATE_RATE; + double DETAILED_METRIC_UPDATE_RATE; + double LAST_LIMITED_RATIO; + double RATEKEEPER_DEFAULT_LIMIT; + + int64_t TARGET_BYTES_PER_STORAGE_SERVER; + int64_t SPRING_BYTES_STORAGE_SERVER; + int64_t AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES; + int64_t TARGET_BYTES_PER_STORAGE_SERVER_BATCH; + int64_t SPRING_BYTES_STORAGE_SERVER_BATCH; + int64_t STORAGE_HARD_LIMIT_BYTES; + int64_t STORAGE_DURABILITY_LAG_HARD_MAX; + int64_t STORAGE_DURABILITY_LAG_SOFT_MAX; + + int64_t LOW_PRIORITY_STORAGE_QUEUE_BYTES; + int64_t LOW_PRIORITY_DURABILITY_LAG; + + int64_t TARGET_BYTES_PER_TLOG; + int64_t SPRING_BYTES_TLOG; + int64_t TARGET_BYTES_PER_TLOG_BATCH; + int64_t SPRING_BYTES_TLOG_BATCH; + int64_t TLOG_SPILL_THRESHOLD; + int64_t TLOG_HARD_LIMIT_BYTES; + int64_t TLOG_RECOVER_MEMORY_LIMIT; + double TLOG_IGNORE_POP_AUTO_ENABLE_DELAY; + + int64_t MAX_MANUAL_THROTTLED_TRANSACTION_TAGS; + int64_t MAX_AUTO_THROTTLED_TRANSACTION_TAGS; + double MIN_TAG_COST; + double AUTO_THROTTLE_TARGET_TAG_BUSYNESS; + double AUTO_THROTTLE_RAMP_TAG_BUSYNESS; + double AUTO_TAG_THROTTLE_RAMP_UP_TIME; + double AUTO_TAG_THROTTLE_DURATION; + double TAG_THROTTLE_PUSH_INTERVAL; + double AUTO_TAG_THROTTLE_START_AGGREGATION_TIME; + double AUTO_TAG_THROTTLE_UPDATE_FREQUENCY; + double TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL; + bool AUTO_TAG_THROTTLING_ENABLED; + + double MAX_TRANSACTIONS_PER_BYTE; + + int64_t MIN_AVAILABLE_SPACE; + double MIN_AVAILABLE_SPACE_RATIO; + double TARGET_AVAILABLE_SPACE_RATIO; + double AVAILABLE_SPACE_UPDATE_DELAY; + + double MAX_TL_SS_VERSION_DIFFERENCE; // spring starts at half this value + double MAX_TL_SS_VERSION_DIFFERENCE_BATCH; + int MAX_MACHINES_FALLING_BEHIND; + + int MAX_TPS_HISTORY_SAMPLES; + int NEEDED_TPS_HISTORY_SAMPLES; + int64_t TARGET_DURABILITY_LAG_VERSIONS; + int64_t AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS; + int64_t TARGET_DURABILITY_LAG_VERSIONS_BATCH; + int64_t DURABILITY_LAG_UNLIMITED_THRESHOLD; + double INITIAL_DURABILITY_LAG_MULTIPLIER; + double DURABILITY_LAG_REDUCTION_RATE; + double DURABILITY_LAG_INCREASE_RATE; + + double STORAGE_SERVER_LIST_FETCH_TIMEOUT; + + // disk snapshot + int64_t MAX_FORKED_PROCESS_OUTPUT; + double SNAP_CREATE_MAX_TIMEOUT; + + // Storage Metrics + double STORAGE_METRICS_AVERAGE_INTERVAL; + double STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS; + double SPLIT_JITTER_AMOUNT; + int64_t IOPS_UNITS_PER_SAMPLE; + int64_t BANDWIDTH_UNITS_PER_SAMPLE; + int64_t BYTES_READ_UNITS_PER_SAMPLE; + int64_t READ_HOT_SUB_RANGE_CHUNK_SIZE; + int64_t EMPTY_READ_PENALTY; + bool READ_SAMPLING_ENABLED; + + // Storage Server + double STORAGE_LOGGING_DELAY; + double STORAGE_SERVER_POLL_METRICS_DELAY; + double FUTURE_VERSION_DELAY; + int STORAGE_LIMIT_BYTES; + int BUGGIFY_LIMIT_BYTES; + int FETCH_BLOCK_BYTES; + int FETCH_KEYS_PARALLELISM_BYTES; + int FETCH_KEYS_LOWER_PRIORITY; + int BUGGIFY_BLOCK_BYTES; + double STORAGE_DURABILITY_LAG_REJECT_THRESHOLD; + double STORAGE_DURABILITY_LAG_MIN_RATE; + int STORAGE_COMMIT_BYTES; + double STORAGE_COMMIT_INTERVAL; + double UPDATE_SHARD_VERSION_INTERVAL; + int BYTE_SAMPLING_FACTOR; + int BYTE_SAMPLING_OVERHEAD; + int MAX_STORAGE_SERVER_WATCH_BYTES; + int MAX_BYTE_SAMPLE_CLEAR_MAP_SIZE; + double LONG_BYTE_SAMPLE_RECOVERY_DELAY; + int BYTE_SAMPLE_LOAD_PARALLELISM; + double BYTE_SAMPLE_LOAD_DELAY; + double BYTE_SAMPLE_START_DELAY; + double UPDATE_STORAGE_PROCESS_STATS_INTERVAL; + double BEHIND_CHECK_DELAY; + int BEHIND_CHECK_COUNT; + int64_t BEHIND_CHECK_VERSIONS; + double WAIT_METRICS_WRONG_SHARD_CHANCE; + int64_t MIN_TAG_READ_PAGES_RATE; + int64_t MIN_TAG_WRITE_PAGES_RATE; + double TAG_MEASUREMENT_INTERVAL; + int64_t READ_COST_BYTE_FACTOR; + bool PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS; + bool REPORT_DD_METRICS; + double DD_METRICS_REPORT_INTERVAL; + double FETCH_KEYS_TOO_LONG_TIME_CRITERIA; + double MAX_STORAGE_COMMIT_TIME; + + // Wait Failure + int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS; + double WAIT_FAILURE_DELAY_LIMIT; + + // Worker + double WORKER_LOGGING_INTERVAL; + double HEAP_PROFILER_INTERVAL; + double UNKNOWN_CC_TIMEOUT; + double DEGRADED_RESET_INTERVAL; + double DEGRADED_WARNING_LIMIT; + double DEGRADED_WARNING_RESET_DELAY; + int64_t TRACE_LOG_FLUSH_FAILURE_CHECK_INTERVAL_SECONDS; + double TRACE_LOG_PING_TIMEOUT_SECONDS; + double MIN_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS; // Listen for a leader for N seconds, and if not heard, then try to + // become the leader. + double MAX_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS; + double DBINFO_FAILED_DELAY; + + // Test harness + double WORKER_POLL_DELAY; + + // Coordination + double COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL; + double FORWARD_REQUEST_TOO_OLD; + bool ENABLE_CROSS_CLUSTER_SUPPORT; // Allow a coordinator to serve requests whose connection string does not match + // the local descriptor + + // Buggification + double BUGGIFIED_EVENTUAL_CONSISTENCY; + bool BUGGIFY_ALL_COORDINATION; + + // Status + double STATUS_MIN_TIME_BETWEEN_REQUESTS; + double MAX_STATUS_REQUESTS_PER_SECOND; + int CONFIGURATION_ROWS_TO_FETCH; + bool DISABLE_DUPLICATE_LOG_WARNING; + double HISTOGRAM_REPORT_INTERVAL; + + // IPager + int PAGER_RESERVED_PAGES; + + // IndirectShadowPager + int FREE_PAGE_VACUUM_THRESHOLD; + int VACUUM_QUEUE_SIZE; + int VACUUM_BYTES_PER_SECOND; + + // Timekeeper + int64_t TIME_KEEPER_DELAY; + int64_t TIME_KEEPER_MAX_ENTRIES; + + // Fast Restore + // TODO: After 6.3, review FR knobs, remove unneeded ones and change default value + int64_t FASTRESTORE_FAILURE_TIMEOUT; + int64_t FASTRESTORE_HEARTBEAT_INTERVAL; + double FASTRESTORE_SAMPLING_PERCENT; + int64_t FASTRESTORE_NUM_LOADERS; + int64_t FASTRESTORE_NUM_APPLIERS; + // FASTRESTORE_TXN_BATCH_MAX_BYTES is target txn size used by appliers to apply mutations + double FASTRESTORE_TXN_BATCH_MAX_BYTES; + // FASTRESTORE_VERSIONBATCH_MAX_BYTES is the maximum data size in each version batch + double FASTRESTORE_VERSIONBATCH_MAX_BYTES; + // FASTRESTORE_VB_PARALLELISM is the number of concurrently running version batches + int64_t FASTRESTORE_VB_PARALLELISM; + int64_t FASTRESTORE_VB_MONITOR_DELAY; // How quickly monitor finished version batch + double FASTRESTORE_VB_LAUNCH_DELAY; + int64_t FASTRESTORE_ROLE_LOGGING_DELAY; + int64_t FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL; // How quickly to update process metrics for restore + int64_t FASTRESTORE_ATOMICOP_WEIGHT; // workload amplication factor for atomic op + int64_t FASTRESTORE_APPLYING_PARALLELISM; // number of outstanding txns writing to dest. DB + int64_t FASTRESTORE_MONITOR_LEADER_DELAY; + int64_t FASTRESTORE_STRAGGLER_THRESHOLD_SECONDS; + bool FASTRESTORE_TRACK_REQUEST_LATENCY; // true to track reply latency of each request in a request batch + bool FASTRESTORE_TRACK_LOADER_SEND_REQUESTS; // track requests of load send mutations to appliers? + int64_t FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT; // threshold when pipelined actors should be delayed + int64_t FASTRESTORE_WAIT_FOR_MEMORY_LATENCY; + int64_t FASTRESTORE_HEARTBEAT_DELAY; // interval for master to ping loaders and appliers + int64_t + FASTRESTORE_HEARTBEAT_MAX_DELAY; // master claim a node is down if no heart beat from the node for this delay + int64_t FASTRESTORE_APPLIER_FETCH_KEYS_SIZE; // number of keys to fetch in a txn on applier + int64_t FASTRESTORE_LOADER_SEND_MUTATION_MSG_BYTES; // desired size of mutation message sent from loader to appliers + bool FASTRESTORE_GET_RANGE_VERSIONS_EXPENSIVE; // parse each range file to get (range, version) it has? + int64_t FASTRESTORE_REQBATCH_PARALLEL; // number of requests to wait on for getBatchReplies() + bool FASTRESTORE_REQBATCH_LOG; // verbose log information for getReplyBatches + int FASTRESTORE_TXN_CLEAR_MAX; // threshold to start tracking each clear op in a txn + int FASTRESTORE_TXN_RETRY_MAX; // threshold to start output error on too many retries + double FASTRESTORE_TXN_EXTRA_DELAY; // extra delay to avoid overwhelming fdb + bool FASTRESTORE_NOT_WRITE_DB; // do not write result to DB. Only for dev testing + bool FASTRESTORE_USE_RANGE_FILE; // use range file in backup + bool FASTRESTORE_USE_LOG_FILE; // use log file in backup + int64_t FASTRESTORE_SAMPLE_MSG_BYTES; // sample message desired size + double FASTRESTORE_SCHED_UPDATE_DELAY; // delay in seconds in updating process metrics + int FASTRESTORE_SCHED_TARGET_CPU_PERCENT; // release as many requests as possible when cpu usage is below the knob + int FASTRESTORE_SCHED_MAX_CPU_PERCENT; // max cpu percent when scheduler shall not release non-urgent requests + int FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS; // number of inflight requests to load backup files + int FASTRESTORE_SCHED_INFLIGHT_SEND_REQS; // number of inflight requests for loaders to send mutations to appliers + int FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE; // number of load request to release at once + int FASTRESTORE_SCHED_INFLIGHT_SENDPARAM_THRESHOLD; // we can send future VB requests if it is less than this knob + int FASTRESTORE_SCHED_SEND_FUTURE_VB_REQS_BATCH; // number of future VB sendLoadingParam requests to process at once + int FASTRESTORE_NUM_TRACE_EVENTS; + bool FASTRESTORE_EXPENSIVE_VALIDATION; // when set true, performance will be heavily affected + double FASTRESTORE_WRITE_BW_MB; // target aggregated write bandwidth from all appliers + double FASTRESTORE_RATE_UPDATE_SECONDS; // how long to update appliers target write rate + + int REDWOOD_DEFAULT_PAGE_SIZE; // Page size for new Redwood files + int REDWOOD_KVSTORE_CONCURRENT_READS; // Max number of simultaneous point or range reads in progress. + int REDWOOD_COMMIT_CONCURRENT_READS; // Max number of concurrent reads done to support commit operations + double REDWOOD_PAGE_REBUILD_MAX_SLACK; // When rebuilding pages, max slack to allow in page + int REDWOOD_LAZY_CLEAR_BATCH_SIZE_PAGES; // Number of pages to try to pop from the lazy delete queue and process at + // once + int REDWOOD_LAZY_CLEAR_MIN_PAGES; // Minimum number of pages to free before ending a lazy clear cycle, unless the + // queue is empty + int REDWOOD_LAZY_CLEAR_MAX_PAGES; // Maximum number of pages to free before ending a lazy clear cycle, unless the + // queue is empty + int64_t REDWOOD_REMAP_CLEANUP_WINDOW; // Remap remover lag interval in which to coalesce page writes + double REDWOOD_REMAP_CLEANUP_LAG; // Maximum allowed remap remover lag behind the cleanup window as a multiple of + // the window size + double REDWOOD_LOGGING_INTERVAL; + + // Server request latency measurement + int LATENCY_SAMPLE_SIZE; + double LATENCY_METRICS_LOGGING_INTERVAL; + + ServerKnobs(); + void initialize(bool randomize = false, ClientKnobs* clientKnobs = nullptr, bool isSimulated = false); +}; + +extern std::unique_ptr globalServerKnobs; +extern ServerKnobs const* SERVER_KNOBS; + +#endif From ae424f11950cb2819fcd2ebfd179536a84805e53 Mon Sep 17 00:00:00 2001 From: Dan Lambright Date: Mon, 26 Jul 2021 10:55:22 -0400 Subject: [PATCH 09/26] rebase --- fdbclient/ServerKnobs.h | 1 + fdbserver/Knobs.h | 644 +--------------------------------------- 2 files changed, 2 insertions(+), 643 deletions(-) diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 79600d49bb..734a10daab 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -593,6 +593,7 @@ public: double COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL; bool ENABLE_CROSS_CLUSTER_SUPPORT; // Allow a coordinator to serve requests whose connection string does not match // the local descriptor + double FORWARD_REQUEST_TOO_OLD; // Do not forward requests older than this setting // Buggification double BUGGIFIED_EVENTUAL_CONSISTENCY; diff --git a/fdbserver/Knobs.h b/fdbserver/Knobs.h index 44b653bd68..67f474b0eb 100644 --- a/fdbserver/Knobs.h +++ b/fdbserver/Knobs.h @@ -22,646 +22,4 @@ #include "fdbclient/IKnobCollection.h" -// Disk queue -static const int _PAGE_SIZE = 4096; - -class ServerKnobs : public Knobs { -public: - // Versions - int64_t VERSIONS_PER_SECOND; - int64_t MAX_VERSIONS_IN_FLIGHT; - int64_t MAX_VERSIONS_IN_FLIGHT_FORCED; - int64_t MAX_READ_TRANSACTION_LIFE_VERSIONS; - int64_t MAX_WRITE_TRANSACTION_LIFE_VERSIONS; - double MAX_COMMIT_BATCH_INTERVAL; // Each commit proxy generates a CommitTransactionBatchRequest at least this - // often, so that versions always advance smoothly - - // TLogs - double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time - double TLOG_SLOW_REJOIN_WARN_TIMEOUT_SECS; // Warns if a tlog takes too long to rejoin - double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification - double TLOG_STORAGE_MIN_UPDATE_INTERVAL; - double BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL; - int DESIRED_TOTAL_BYTES; - int DESIRED_UPDATE_BYTES; - double UPDATE_DELAY; - int MAXIMUM_PEEK_BYTES; - int APPLY_MUTATION_BYTES; - int RECOVERY_DATA_BYTE_LIMIT; - int BUGGIFY_RECOVERY_DATA_LIMIT; - double LONG_TLOG_COMMIT_TIME; - int64_t LARGE_TLOG_COMMIT_BYTES; - double BUGGIFY_RECOVER_MEMORY_LIMIT; - double BUGGIFY_WORKER_REMOVED_MAX_LAG; - int64_t UPDATE_STORAGE_BYTE_LIMIT; - int64_t REFERENCE_SPILL_UPDATE_STORAGE_BYTE_LIMIT; - double TLOG_PEEK_DELAY; - int LEGACY_TLOG_UPGRADE_ENTRIES_PER_VERSION; - int VERSION_MESSAGES_OVERHEAD_FACTOR_1024THS; // Multiplicative factor to bound total space used to store a version - // message (measured in 1/1024ths, e.g. a value of 2048 yields a - // factor of 2). - int64_t VERSION_MESSAGES_ENTRY_BYTES_WITH_OVERHEAD; - double TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR; - int64_t TLOG_MESSAGE_BLOCK_BYTES; - int64_t MAX_MESSAGE_SIZE; - int LOG_SYSTEM_PUSHED_DATA_BLOCK_SIZE; - double PEEK_TRACKER_EXPIRATION_TIME; - int PARALLEL_GET_MORE_REQUESTS; - int MULTI_CURSOR_PRE_FETCH_LIMIT; - int64_t MAX_QUEUE_COMMIT_BYTES; - int DESIRED_OUTSTANDING_MESSAGES; - double DESIRED_GET_MORE_DELAY; - int CONCURRENT_LOG_ROUTER_READS; - int LOG_ROUTER_PEEK_FROM_SATELLITES_PREFERRED; // 0==peek from primary, non-zero==peek from satellites - double DISK_QUEUE_ADAPTER_MIN_SWITCH_TIME; - double DISK_QUEUE_ADAPTER_MAX_SWITCH_TIME; - int64_t TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES; - int64_t TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; - int64_t TLOG_SPILL_REFERENCE_MAX_BYTES_PER_BATCH; - int64_t DISK_QUEUE_FILE_EXTENSION_BYTES; // When we grow the disk queue, by how many bytes should it grow? - int64_t DISK_QUEUE_FILE_SHRINK_BYTES; // When we shrink the disk queue, by how many bytes should it shrink? - int64_t DISK_QUEUE_MAX_TRUNCATE_BYTES; // A truncate larger than this will cause the file to be replaced instead. - double TLOG_DEGRADED_DURATION; - int64_t MAX_CACHE_VERSIONS; - double TXS_POPPED_MAX_DELAY; - double TLOG_MAX_CREATE_DURATION; - int PEEK_LOGGING_AMOUNT; - double PEEK_LOGGING_DELAY; - double PEEK_RESET_INTERVAL; - double PEEK_MAX_LATENCY; - bool PEEK_COUNT_SMALL_MESSAGES; - double PEEK_STATS_INTERVAL; - double PEEK_STATS_SLOW_AMOUNT; - double PEEK_STATS_SLOW_RATIO; - double PUSH_RESET_INTERVAL; - double PUSH_MAX_LATENCY; - double PUSH_STATS_INTERVAL; - double PUSH_STATS_SLOW_AMOUNT; - double PUSH_STATS_SLOW_RATIO; - int TLOG_POP_BATCH_SIZE; - - // Data distribution queue - double HEALTH_POLL_TIME; - double BEST_TEAM_STUCK_DELAY; - double BG_REBALANCE_POLLING_INTERVAL; - double BG_REBALANCE_SWITCH_CHECK_INTERVAL; - double DD_QUEUE_LOGGING_INTERVAL; - double RELOCATION_PARALLELISM_PER_SOURCE_SERVER; - int DD_QUEUE_MAX_KEY_SERVERS; - int DD_REBALANCE_PARALLELISM; - int DD_REBALANCE_RESET_AMOUNT; - double BG_DD_MAX_WAIT; - double BG_DD_MIN_WAIT; - double BG_DD_INCREASE_RATE; - double BG_DD_DECREASE_RATE; - double BG_DD_SATURATION_DELAY; - double INFLIGHT_PENALTY_HEALTHY; - double INFLIGHT_PENALTY_REDUNDANT; - double INFLIGHT_PENALTY_UNHEALTHY; - double INFLIGHT_PENALTY_ONE_LEFT; - bool USE_OLD_NEEDED_SERVERS; - - // Higher priorities are executed first - // Priority/100 is the "priority group"/"superpriority". Priority inversion - // is possible within but not between priority groups; fewer priority groups - // mean better worst case time bounds - // Maximum allowable priority is 999. - int PRIORITY_RECOVER_MOVE; - int PRIORITY_REBALANCE_UNDERUTILIZED_TEAM; - int PRIORITY_REBALANCE_OVERUTILIZED_TEAM; - int PRIORITY_TEAM_HEALTHY; - int PRIORITY_TEAM_CONTAINS_UNDESIRED_SERVER; - int PRIORITY_TEAM_REDUNDANT; - int PRIORITY_MERGE_SHARD; - int PRIORITY_POPULATE_REGION; - int PRIORITY_TEAM_UNHEALTHY; - int PRIORITY_TEAM_2_LEFT; - int PRIORITY_TEAM_1_LEFT; - int PRIORITY_TEAM_FAILED; // Priority when a server in the team is excluded as failed - int PRIORITY_TEAM_0_LEFT; - int PRIORITY_SPLIT_SHARD; - - // Data distribution - double RETRY_RELOCATESHARD_DELAY; - double DATA_DISTRIBUTION_FAILURE_REACTION_TIME; - int MIN_SHARD_BYTES, SHARD_BYTES_RATIO, SHARD_BYTES_PER_SQRT_BYTES, MAX_SHARD_BYTES, KEY_SERVER_SHARD_BYTES; - int64_t SHARD_MAX_BYTES_PER_KSEC, // Shards with more than this bandwidth will be split immediately - SHARD_MIN_BYTES_PER_KSEC, // Shards with more than this bandwidth will not be merged - SHARD_SPLIT_BYTES_PER_KSEC; // When splitting a shard, it is split into pieces with less than this bandwidth - double SHARD_MAX_READ_DENSITY_RATIO; - int64_t SHARD_READ_HOT_BANDWITH_MIN_PER_KSECONDS; - double SHARD_MAX_BYTES_READ_PER_KSEC_JITTER; - double STORAGE_METRIC_TIMEOUT; - double METRIC_DELAY; - double ALL_DATA_REMOVED_DELAY; - double INITIAL_FAILURE_REACTION_DELAY; - double CHECK_TEAM_DELAY; - double LOG_ON_COMPLETION_DELAY; - int BEST_TEAM_MAX_TEAM_TRIES; - int BEST_TEAM_OPTION_COUNT; - int BEST_OF_AMT; - double SERVER_LIST_DELAY; - double RECRUITMENT_IDLE_DELAY; - double STORAGE_RECRUITMENT_DELAY; - bool TSS_HACK_IDENTITY_MAPPING; - double TSS_RECRUITMENT_TIMEOUT; - double TSS_DD_KILL_INTERVAL; - double DATA_DISTRIBUTION_LOGGING_INTERVAL; - double DD_ENABLED_CHECK_DELAY; - double DD_STALL_CHECK_DELAY; - double DD_LOW_BANDWIDTH_DELAY; - double DD_MERGE_COALESCE_DELAY; - double STORAGE_METRICS_POLLING_DELAY; - double STORAGE_METRICS_RANDOM_DELAY; - double AVAILABLE_SPACE_RATIO_CUTOFF; - int DESIRED_TEAMS_PER_SERVER; - int MAX_TEAMS_PER_SERVER; - int64_t DD_SHARD_SIZE_GRANULARITY; - int64_t DD_SHARD_SIZE_GRANULARITY_SIM; - int DD_MOVE_KEYS_PARALLELISM; - int DD_FETCH_SOURCE_PARALLELISM; - int DD_MERGE_LIMIT; - double DD_SHARD_METRICS_TIMEOUT; - int64_t DD_LOCATION_CACHE_SIZE; - double MOVEKEYS_LOCK_POLLING_DELAY; - double DEBOUNCE_RECRUITING_DELAY; - int REBALANCE_MAX_RETRIES; - int DD_OVERLAP_PENALTY; - int DD_EXCLUDE_MIN_REPLICAS; - bool DD_VALIDATE_LOCALITY; - int DD_CHECK_INVALID_LOCALITY_DELAY; - bool DD_ENABLE_VERBOSE_TRACING; - int64_t - DD_SS_FAILURE_VERSIONLAG; // Allowed SS version lag from the current read version before marking it as failed. - int64_t DD_SS_ALLOWED_VERSIONLAG; // SS will be marked as healthy if it's version lag goes below this value. - double DD_SS_STUCK_TIME_LIMIT; // If a storage server is not getting new versions for this amount of time, then it - // becomes undesired. - int DD_TEAMS_INFO_PRINT_INTERVAL; - int DD_TEAMS_INFO_PRINT_YIELD_COUNT; - int DD_TEAM_ZERO_SERVER_LEFT_LOG_DELAY; - - // TeamRemover to remove redundant teams - bool TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER; // disable the machineTeamRemover actor - double TR_REMOVE_MACHINE_TEAM_DELAY; // wait for the specified time before try to remove next machine team - bool TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS; // guard to select which machineTeamRemover logic to use - - bool TR_FLAG_DISABLE_SERVER_TEAM_REMOVER; // disable the serverTeamRemover actor - double TR_REMOVE_SERVER_TEAM_DELAY; // wait for the specified time before try to remove next server team - double TR_REMOVE_SERVER_TEAM_EXTRA_DELAY; // serverTeamRemover waits for the delay and check DD healthyness again to - // ensure it runs after machineTeamRemover - - // Remove wrong storage engines - double DD_REMOVE_STORE_ENGINE_DELAY; // wait for the specified time before remove the next batch - - double DD_FAILURE_TIME; - double DD_ZERO_HEALTHY_TEAM_DELAY; - - // Redwood Storage Engine - int PREFIX_TREE_IMMEDIATE_KEY_SIZE_LIMIT; - int PREFIX_TREE_IMMEDIATE_KEY_SIZE_MIN; - - // KeyValueStore SQLITE - int CLEAR_BUFFER_SIZE; - double READ_VALUE_TIME_ESTIMATE; - double READ_RANGE_TIME_ESTIMATE; - double SET_TIME_ESTIMATE; - double CLEAR_TIME_ESTIMATE; - double COMMIT_TIME_ESTIMATE; - int CHECK_FREE_PAGE_AMOUNT; - double DISK_METRIC_LOGGING_INTERVAL; - int64_t SOFT_HEAP_LIMIT; - - int SQLITE_PAGE_SCAN_ERROR_LIMIT; - int SQLITE_BTREE_PAGE_USABLE; - int SQLITE_BTREE_CELL_MAX_LOCAL; - int SQLITE_BTREE_CELL_MIN_LOCAL; - int SQLITE_FRAGMENT_PRIMARY_PAGE_USABLE; - int SQLITE_FRAGMENT_OVERFLOW_PAGE_USABLE; - double SQLITE_FRAGMENT_MIN_SAVINGS; - int SQLITE_CHUNK_SIZE_PAGES; - int SQLITE_CHUNK_SIZE_PAGES_SIM; - int SQLITE_READER_THREADS; - int SQLITE_WRITE_WINDOW_LIMIT; - double SQLITE_WRITE_WINDOW_SECONDS; - - // KeyValueStoreSqlite spring cleaning - double SPRING_CLEANING_NO_ACTION_INTERVAL; - double SPRING_CLEANING_LAZY_DELETE_INTERVAL; - double SPRING_CLEANING_VACUUM_INTERVAL; - double SPRING_CLEANING_LAZY_DELETE_TIME_ESTIMATE; - double SPRING_CLEANING_VACUUM_TIME_ESTIMATE; - double SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE; - int SPRING_CLEANING_MIN_LAZY_DELETE_PAGES; - int SPRING_CLEANING_MAX_LAZY_DELETE_PAGES; - int SPRING_CLEANING_LAZY_DELETE_BATCH_SIZE; - int SPRING_CLEANING_MIN_VACUUM_PAGES; - int SPRING_CLEANING_MAX_VACUUM_PAGES; - - // KeyValueStoreMemory - int64_t REPLACE_CONTENTS_BYTES; - - // KeyValueStoreRocksDB - int ROCKSDB_BACKGROUND_PARALLELISM; - int ROCKSDB_READ_PARALLELISM; - int64_t ROCKSDB_MEMTABLE_BYTES; - bool ROCKSDB_UNSAFE_AUTO_FSYNC; - int64_t ROCKSDB_PERIODIC_COMPACTION_SECONDS; - int ROCKSDB_PREFIX_LEN; - int64_t ROCKSDB_BLOCK_CACHE_SIZE; - - // Leader election - int MAX_NOTIFICATIONS; - int MIN_NOTIFICATIONS; - double NOTIFICATION_FULL_CLEAR_TIME; - double CANDIDATE_MIN_DELAY; - double CANDIDATE_MAX_DELAY; - double CANDIDATE_GROWTH_RATE; - double POLLING_FREQUENCY; - double HEARTBEAT_FREQUENCY; - - // Commit CommitProxy - double START_TRANSACTION_BATCH_INTERVAL_MIN; - double START_TRANSACTION_BATCH_INTERVAL_MAX; - double START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; - double START_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA; - double START_TRANSACTION_BATCH_QUEUE_CHECK_INTERVAL; - double START_TRANSACTION_MAX_TRANSACTIONS_TO_START; - int START_TRANSACTION_MAX_REQUESTS_TO_START; - double START_TRANSACTION_RATE_WINDOW; - double START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET; - int START_TRANSACTION_MAX_QUEUE_SIZE; - int KEY_LOCATION_MAX_QUEUE_SIZE; - - double COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE; - double COMMIT_TRANSACTION_BATCH_INTERVAL_MIN; - double COMMIT_TRANSACTION_BATCH_INTERVAL_MAX; - double COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; - double COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA; - int COMMIT_TRANSACTION_BATCH_COUNT_MAX; - int COMMIT_TRANSACTION_BATCH_BYTES_MIN; - int COMMIT_TRANSACTION_BATCH_BYTES_MAX; - double COMMIT_TRANSACTION_BATCH_BYTES_SCALE_BASE; - double COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER; - int64_t COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT; - double COMMIT_BATCHES_MEM_FRACTION_OF_TOTAL; - double COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR; - - double RESOLVER_COALESCE_TIME; - int BUGGIFIED_ROW_LIMIT; - double PROXY_SPIN_DELAY; - double UPDATE_REMOTE_LOG_VERSION_INTERVAL; - int MAX_TXS_POP_VERSION_HISTORY; - double MIN_CONFIRM_INTERVAL; - double ENFORCED_MIN_RECOVERY_DURATION; - double REQUIRED_MIN_RECOVERY_DURATION; - bool ALWAYS_CAUSAL_READ_RISKY; - int MAX_COMMIT_UPDATES; - double MAX_PROXY_COMPUTE; - double MAX_COMPUTE_PER_OPERATION; - int PROXY_COMPUTE_BUCKETS; - double PROXY_COMPUTE_GROWTH_RATE; - int TXN_STATE_SEND_AMOUNT; - double REPORT_TRANSACTION_COST_ESTIMATION_DELAY; - bool PROXY_REJECT_BATCH_QUEUED_TOO_LONG; - - int RESET_MASTER_BATCHES; - int RESET_RESOLVER_BATCHES; - double RESET_MASTER_DELAY; - double RESET_RESOLVER_DELAY; - - // Master Server - double COMMIT_SLEEP_TIME; - double MIN_BALANCE_TIME; - int64_t MIN_BALANCE_DIFFERENCE; - double SECONDS_BEFORE_NO_FAILURE_DELAY; - int64_t MAX_TXS_SEND_MEMORY; - int64_t MAX_RECOVERY_VERSIONS; - double MAX_RECOVERY_TIME; - double PROVISIONAL_START_DELAY; - double PROVISIONAL_DELAY_GROWTH; - double PROVISIONAL_MAX_DELAY; - double SECONDS_BEFORE_RECRUIT_BACKUP_WORKER; - double CC_INTERFACE_TIMEOUT; - - // Resolver - int64_t KEY_BYTES_PER_SAMPLE; - int64_t SAMPLE_OFFSET_PER_KEY; - double SAMPLE_EXPIRATION_TIME; - double SAMPLE_POLL_TIME; - int64_t RESOLVER_STATE_MEMORY_LIMIT; - - // Backup Worker - double BACKUP_TIMEOUT; // master's reaction time for backup failure - double BACKUP_NOOP_POP_DELAY; - int BACKUP_FILE_BLOCK_BYTES; - int64_t BACKUP_LOCK_BYTES; - double BACKUP_UPLOAD_DELAY; - - // Cluster Controller - double CLUSTER_CONTROLLER_LOGGING_DELAY; - double MASTER_FAILURE_REACTION_TIME; - double MASTER_FAILURE_SLOPE_DURING_RECOVERY; - int WORKER_COORDINATION_PING_DELAY; - double SIM_SHUTDOWN_TIMEOUT; - double SHUTDOWN_TIMEOUT; - double MASTER_SPIN_DELAY; - double CC_CHANGE_DELAY; - double CC_CLASS_DELAY; - double WAIT_FOR_GOOD_RECRUITMENT_DELAY; - double WAIT_FOR_GOOD_REMOTE_RECRUITMENT_DELAY; - double ATTEMPT_RECRUITMENT_DELAY; - double WAIT_FOR_DISTRIBUTOR_JOIN_DELAY; - double WAIT_FOR_RATEKEEPER_JOIN_DELAY; - double WORKER_FAILURE_TIME; - double CHECK_OUTSTANDING_INTERVAL; - double INCOMPATIBLE_PEERS_LOGGING_INTERVAL; - double VERSION_LAG_METRIC_INTERVAL; - int64_t MAX_VERSION_DIFFERENCE; - double FORCE_RECOVERY_CHECK_DELAY; - double RATEKEEPER_FAILURE_TIME; - double REPLACE_INTERFACE_DELAY; - double REPLACE_INTERFACE_CHECK_DELAY; - double COORDINATOR_REGISTER_INTERVAL; - double CLIENT_REGISTER_INTERVAL; - - // Knobs used to select the best policy (via monte carlo) - int POLICY_RATING_TESTS; // number of tests per policy (in order to compare) - int POLICY_GENERATIONS; // number of policies to generate - - int EXPECTED_MASTER_FITNESS; - int EXPECTED_TLOG_FITNESS; - int EXPECTED_LOG_ROUTER_FITNESS; - int EXPECTED_COMMIT_PROXY_FITNESS; - int EXPECTED_GRV_PROXY_FITNESS; - int EXPECTED_RESOLVER_FITNESS; - double RECRUITMENT_TIMEOUT; - int DBINFO_SEND_AMOUNT; - double DBINFO_BATCH_DELAY; - - // Move Keys - double SHARD_READY_DELAY; - double SERVER_READY_QUORUM_INTERVAL; - double SERVER_READY_QUORUM_TIMEOUT; - double REMOVE_RETRY_DELAY; - int MOVE_KEYS_KRM_LIMIT; - int MOVE_KEYS_KRM_LIMIT_BYTES; // This must be sufficiently larger than CLIENT_KNOBS->KEY_SIZE_LIMIT - // (fdbclient/Knobs.h) to ensure that at least two entries will be returned from an - // attempt to read a key range map - int MAX_SKIP_TAGS; - double MAX_ADDED_SOURCES_MULTIPLIER; - - // FdbServer - double MIN_REBOOT_TIME; - double MAX_REBOOT_TIME; - std::string LOG_DIRECTORY; - int64_t SERVER_MEM_LIMIT; - double SYSTEM_MONITOR_FREQUENCY; - - // Ratekeeper - double SMOOTHING_AMOUNT; - double SLOW_SMOOTHING_AMOUNT; - double METRIC_UPDATE_RATE; - double DETAILED_METRIC_UPDATE_RATE; - double LAST_LIMITED_RATIO; - double RATEKEEPER_DEFAULT_LIMIT; - - int64_t TARGET_BYTES_PER_STORAGE_SERVER; - int64_t SPRING_BYTES_STORAGE_SERVER; - int64_t AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES; - int64_t TARGET_BYTES_PER_STORAGE_SERVER_BATCH; - int64_t SPRING_BYTES_STORAGE_SERVER_BATCH; - int64_t STORAGE_HARD_LIMIT_BYTES; - int64_t STORAGE_DURABILITY_LAG_HARD_MAX; - int64_t STORAGE_DURABILITY_LAG_SOFT_MAX; - - int64_t LOW_PRIORITY_STORAGE_QUEUE_BYTES; - int64_t LOW_PRIORITY_DURABILITY_LAG; - - int64_t TARGET_BYTES_PER_TLOG; - int64_t SPRING_BYTES_TLOG; - int64_t TARGET_BYTES_PER_TLOG_BATCH; - int64_t SPRING_BYTES_TLOG_BATCH; - int64_t TLOG_SPILL_THRESHOLD; - int64_t TLOG_HARD_LIMIT_BYTES; - int64_t TLOG_RECOVER_MEMORY_LIMIT; - double TLOG_IGNORE_POP_AUTO_ENABLE_DELAY; - - int64_t MAX_MANUAL_THROTTLED_TRANSACTION_TAGS; - int64_t MAX_AUTO_THROTTLED_TRANSACTION_TAGS; - double MIN_TAG_COST; - double AUTO_THROTTLE_TARGET_TAG_BUSYNESS; - double AUTO_THROTTLE_RAMP_TAG_BUSYNESS; - double AUTO_TAG_THROTTLE_RAMP_UP_TIME; - double AUTO_TAG_THROTTLE_DURATION; - double TAG_THROTTLE_PUSH_INTERVAL; - double AUTO_TAG_THROTTLE_START_AGGREGATION_TIME; - double AUTO_TAG_THROTTLE_UPDATE_FREQUENCY; - double TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL; - bool AUTO_TAG_THROTTLING_ENABLED; - - double MAX_TRANSACTIONS_PER_BYTE; - - int64_t MIN_AVAILABLE_SPACE; - double MIN_AVAILABLE_SPACE_RATIO; - double TARGET_AVAILABLE_SPACE_RATIO; - double AVAILABLE_SPACE_UPDATE_DELAY; - - double MAX_TL_SS_VERSION_DIFFERENCE; // spring starts at half this value - double MAX_TL_SS_VERSION_DIFFERENCE_BATCH; - int MAX_MACHINES_FALLING_BEHIND; - - int MAX_TPS_HISTORY_SAMPLES; - int NEEDED_TPS_HISTORY_SAMPLES; - int64_t TARGET_DURABILITY_LAG_VERSIONS; - int64_t AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS; - int64_t TARGET_DURABILITY_LAG_VERSIONS_BATCH; - int64_t DURABILITY_LAG_UNLIMITED_THRESHOLD; - double INITIAL_DURABILITY_LAG_MULTIPLIER; - double DURABILITY_LAG_REDUCTION_RATE; - double DURABILITY_LAG_INCREASE_RATE; - - double STORAGE_SERVER_LIST_FETCH_TIMEOUT; - - // disk snapshot - int64_t MAX_FORKED_PROCESS_OUTPUT; - double SNAP_CREATE_MAX_TIMEOUT; - - // Storage Metrics - double STORAGE_METRICS_AVERAGE_INTERVAL; - double STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS; - double SPLIT_JITTER_AMOUNT; - int64_t IOPS_UNITS_PER_SAMPLE; - int64_t BANDWIDTH_UNITS_PER_SAMPLE; - int64_t BYTES_READ_UNITS_PER_SAMPLE; - int64_t READ_HOT_SUB_RANGE_CHUNK_SIZE; - int64_t EMPTY_READ_PENALTY; - bool READ_SAMPLING_ENABLED; - - // Storage Server - double STORAGE_LOGGING_DELAY; - double STORAGE_SERVER_POLL_METRICS_DELAY; - double FUTURE_VERSION_DELAY; - int STORAGE_LIMIT_BYTES; - int BUGGIFY_LIMIT_BYTES; - int FETCH_BLOCK_BYTES; - int FETCH_KEYS_PARALLELISM_BYTES; - int FETCH_KEYS_LOWER_PRIORITY; - int BUGGIFY_BLOCK_BYTES; - double STORAGE_DURABILITY_LAG_REJECT_THRESHOLD; - double STORAGE_DURABILITY_LAG_MIN_RATE; - int STORAGE_COMMIT_BYTES; - double STORAGE_COMMIT_INTERVAL; - double UPDATE_SHARD_VERSION_INTERVAL; - int BYTE_SAMPLING_FACTOR; - int BYTE_SAMPLING_OVERHEAD; - int MAX_STORAGE_SERVER_WATCH_BYTES; - int MAX_BYTE_SAMPLE_CLEAR_MAP_SIZE; - double LONG_BYTE_SAMPLE_RECOVERY_DELAY; - int BYTE_SAMPLE_LOAD_PARALLELISM; - double BYTE_SAMPLE_LOAD_DELAY; - double BYTE_SAMPLE_START_DELAY; - double UPDATE_STORAGE_PROCESS_STATS_INTERVAL; - double BEHIND_CHECK_DELAY; - int BEHIND_CHECK_COUNT; - int64_t BEHIND_CHECK_VERSIONS; - double WAIT_METRICS_WRONG_SHARD_CHANCE; - int64_t MIN_TAG_READ_PAGES_RATE; - int64_t MIN_TAG_WRITE_PAGES_RATE; - double TAG_MEASUREMENT_INTERVAL; - int64_t READ_COST_BYTE_FACTOR; - bool PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS; - bool REPORT_DD_METRICS; - double DD_METRICS_REPORT_INTERVAL; - double FETCH_KEYS_TOO_LONG_TIME_CRITERIA; - double MAX_STORAGE_COMMIT_TIME; - - // Wait Failure - int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS; - double WAIT_FAILURE_DELAY_LIMIT; - - // Worker - double WORKER_LOGGING_INTERVAL; - double HEAP_PROFILER_INTERVAL; - double UNKNOWN_CC_TIMEOUT; - double DEGRADED_RESET_INTERVAL; - double DEGRADED_WARNING_LIMIT; - double DEGRADED_WARNING_RESET_DELAY; - int64_t TRACE_LOG_FLUSH_FAILURE_CHECK_INTERVAL_SECONDS; - double TRACE_LOG_PING_TIMEOUT_SECONDS; - double MIN_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS; // Listen for a leader for N seconds, and if not heard, then try to - // become the leader. - double MAX_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS; - double DBINFO_FAILED_DELAY; - - // Test harness - double WORKER_POLL_DELAY; - - // Coordination - double COORDINATED_STATE_ONCONFLICT_POLL_INTERVAL; - double FORWARD_REQUEST_TOO_OLD; - bool ENABLE_CROSS_CLUSTER_SUPPORT; // Allow a coordinator to serve requests whose connection string does not match - // the local descriptor - - // Buggification - double BUGGIFIED_EVENTUAL_CONSISTENCY; - bool BUGGIFY_ALL_COORDINATION; - - // Status - double STATUS_MIN_TIME_BETWEEN_REQUESTS; - double MAX_STATUS_REQUESTS_PER_SECOND; - int CONFIGURATION_ROWS_TO_FETCH; - bool DISABLE_DUPLICATE_LOG_WARNING; - double HISTOGRAM_REPORT_INTERVAL; - - // IPager - int PAGER_RESERVED_PAGES; - - // IndirectShadowPager - int FREE_PAGE_VACUUM_THRESHOLD; - int VACUUM_QUEUE_SIZE; - int VACUUM_BYTES_PER_SECOND; - - // Timekeeper - int64_t TIME_KEEPER_DELAY; - int64_t TIME_KEEPER_MAX_ENTRIES; - - // Fast Restore - // TODO: After 6.3, review FR knobs, remove unneeded ones and change default value - int64_t FASTRESTORE_FAILURE_TIMEOUT; - int64_t FASTRESTORE_HEARTBEAT_INTERVAL; - double FASTRESTORE_SAMPLING_PERCENT; - int64_t FASTRESTORE_NUM_LOADERS; - int64_t FASTRESTORE_NUM_APPLIERS; - // FASTRESTORE_TXN_BATCH_MAX_BYTES is target txn size used by appliers to apply mutations - double FASTRESTORE_TXN_BATCH_MAX_BYTES; - // FASTRESTORE_VERSIONBATCH_MAX_BYTES is the maximum data size in each version batch - double FASTRESTORE_VERSIONBATCH_MAX_BYTES; - // FASTRESTORE_VB_PARALLELISM is the number of concurrently running version batches - int64_t FASTRESTORE_VB_PARALLELISM; - int64_t FASTRESTORE_VB_MONITOR_DELAY; // How quickly monitor finished version batch - double FASTRESTORE_VB_LAUNCH_DELAY; - int64_t FASTRESTORE_ROLE_LOGGING_DELAY; - int64_t FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL; // How quickly to update process metrics for restore - int64_t FASTRESTORE_ATOMICOP_WEIGHT; // workload amplication factor for atomic op - int64_t FASTRESTORE_APPLYING_PARALLELISM; // number of outstanding txns writing to dest. DB - int64_t FASTRESTORE_MONITOR_LEADER_DELAY; - int64_t FASTRESTORE_STRAGGLER_THRESHOLD_SECONDS; - bool FASTRESTORE_TRACK_REQUEST_LATENCY; // true to track reply latency of each request in a request batch - bool FASTRESTORE_TRACK_LOADER_SEND_REQUESTS; // track requests of load send mutations to appliers? - int64_t FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT; // threshold when pipelined actors should be delayed - int64_t FASTRESTORE_WAIT_FOR_MEMORY_LATENCY; - int64_t FASTRESTORE_HEARTBEAT_DELAY; // interval for master to ping loaders and appliers - int64_t - FASTRESTORE_HEARTBEAT_MAX_DELAY; // master claim a node is down if no heart beat from the node for this delay - int64_t FASTRESTORE_APPLIER_FETCH_KEYS_SIZE; // number of keys to fetch in a txn on applier - int64_t FASTRESTORE_LOADER_SEND_MUTATION_MSG_BYTES; // desired size of mutation message sent from loader to appliers - bool FASTRESTORE_GET_RANGE_VERSIONS_EXPENSIVE; // parse each range file to get (range, version) it has? - int64_t FASTRESTORE_REQBATCH_PARALLEL; // number of requests to wait on for getBatchReplies() - bool FASTRESTORE_REQBATCH_LOG; // verbose log information for getReplyBatches - int FASTRESTORE_TXN_CLEAR_MAX; // threshold to start tracking each clear op in a txn - int FASTRESTORE_TXN_RETRY_MAX; // threshold to start output error on too many retries - double FASTRESTORE_TXN_EXTRA_DELAY; // extra delay to avoid overwhelming fdb - bool FASTRESTORE_NOT_WRITE_DB; // do not write result to DB. Only for dev testing - bool FASTRESTORE_USE_RANGE_FILE; // use range file in backup - bool FASTRESTORE_USE_LOG_FILE; // use log file in backup - int64_t FASTRESTORE_SAMPLE_MSG_BYTES; // sample message desired size - double FASTRESTORE_SCHED_UPDATE_DELAY; // delay in seconds in updating process metrics - int FASTRESTORE_SCHED_TARGET_CPU_PERCENT; // release as many requests as possible when cpu usage is below the knob - int FASTRESTORE_SCHED_MAX_CPU_PERCENT; // max cpu percent when scheduler shall not release non-urgent requests - int FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS; // number of inflight requests to load backup files - int FASTRESTORE_SCHED_INFLIGHT_SEND_REQS; // number of inflight requests for loaders to send mutations to appliers - int FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE; // number of load request to release at once - int FASTRESTORE_SCHED_INFLIGHT_SENDPARAM_THRESHOLD; // we can send future VB requests if it is less than this knob - int FASTRESTORE_SCHED_SEND_FUTURE_VB_REQS_BATCH; // number of future VB sendLoadingParam requests to process at once - int FASTRESTORE_NUM_TRACE_EVENTS; - bool FASTRESTORE_EXPENSIVE_VALIDATION; // when set true, performance will be heavily affected - double FASTRESTORE_WRITE_BW_MB; // target aggregated write bandwidth from all appliers - double FASTRESTORE_RATE_UPDATE_SECONDS; // how long to update appliers target write rate - - int REDWOOD_DEFAULT_PAGE_SIZE; // Page size for new Redwood files - int REDWOOD_KVSTORE_CONCURRENT_READS; // Max number of simultaneous point or range reads in progress. - int REDWOOD_COMMIT_CONCURRENT_READS; // Max number of concurrent reads done to support commit operations - double REDWOOD_PAGE_REBUILD_MAX_SLACK; // When rebuilding pages, max slack to allow in page - int REDWOOD_LAZY_CLEAR_BATCH_SIZE_PAGES; // Number of pages to try to pop from the lazy delete queue and process at - // once - int REDWOOD_LAZY_CLEAR_MIN_PAGES; // Minimum number of pages to free before ending a lazy clear cycle, unless the - // queue is empty - int REDWOOD_LAZY_CLEAR_MAX_PAGES; // Maximum number of pages to free before ending a lazy clear cycle, unless the - // queue is empty - int64_t REDWOOD_REMAP_CLEANUP_WINDOW; // Remap remover lag interval in which to coalesce page writes - double REDWOOD_REMAP_CLEANUP_LAG; // Maximum allowed remap remover lag behind the cleanup window as a multiple of - // the window size - double REDWOOD_LOGGING_INTERVAL; - - // Server request latency measurement - int LATENCY_SAMPLE_SIZE; - double LATENCY_METRICS_LOGGING_INTERVAL; - - ServerKnobs(); - void initialize(bool randomize = false, ClientKnobs* clientKnobs = nullptr, bool isSimulated = false); -}; - -extern std::unique_ptr globalServerKnobs; -extern ServerKnobs const* SERVER_KNOBS; - -#endif +#define SERVER_KNOBS (&IKnobCollection::getGlobalKnobCollection().getServerKnobs()) From 34f82e7a152f176a998f21bcdbfbcb7ebf677b96 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 26 Jul 2021 09:51:44 -0700 Subject: [PATCH 10/26] Do not partially reset a transaction when it is committed or fails to commit with an error. --- bindings/c/test/unit/unit_tests.cpp | 75 +++++++++++++++++++ .../source/api-version-upgrade-guide.rst | 2 + .../release-notes/release-notes-700.rst | 1 + fdbclient/NativeAPI.actor.cpp | 10 ++- 4 files changed, 86 insertions(+), 2 deletions(-) diff --git a/bindings/c/test/unit/unit_tests.cpp b/bindings/c/test/unit/unit_tests.cpp index 64ed2adddd..fe88e6b96f 100644 --- a/bindings/c/test/unit/unit_tests.cpp +++ b/bindings/c/test/unit/unit_tests.cpp @@ -2177,6 +2177,81 @@ TEST_CASE("monitor_network_busyness") { CHECK(containsGreaterZero); } +// Commit a transaction and confirm it has not been reset +TEST_CASE("commit_does_not_reset") { + fdb::Transaction tr(db); + fdb::Transaction tr2(db); + + // Commit two transactions, one that will fail with conflict and the other + // that will succeed. Ensure both transactions are not reset at the end. + while (1) { + fdb::Int64Future tr1GrvFuture = tr.get_read_version(); + fdb_error_t err = wait_future(tr1GrvFuture); + if (err) { + fdb::EmptyFuture tr1OnErrorFuture = tr.on_error(err); + fdb_check(wait_future(tr1OnErrorFuture)); + continue; + } + + int64_t tr1StartVersion; + CHECK(!tr1GrvFuture.get(&tr1StartVersion)); + + fdb::Int64Future tr2GrvFuture = tr2.get_read_version(); + err = wait_future(tr2GrvFuture); + + if (err) { + fdb::EmptyFuture tr2OnErrorFuture = tr2.on_error(err); + fdb_check(wait_future(tr2OnErrorFuture)); + continue; + } + + int64_t tr2StartVersion; + CHECK(!tr2GrvFuture.get(&tr2StartVersion)); + + tr.set(key("foo"), "bar"); + fdb::EmptyFuture tr1CommitFuture = tr.commit(); + err = wait_future(tr1CommitFuture); + if (err) { + fdb::EmptyFuture tr1OnErrorFuture = tr.on_error(err); + fdb_check(wait_future(tr1OnErrorFuture)); + continue; + } + + fdb_check(tr2.add_conflict_range(key("foo"), strinc(key("foo")), FDB_CONFLICT_RANGE_TYPE_READ)); + tr2.set(key("foo"), "bar"); + fdb::EmptyFuture tr2CommitFuture = tr2.commit(); + err = wait_future(tr2CommitFuture); + CHECK(err == 1020); // not_committed + + fdb::Int64Future tr1GrvFuture2 = tr.get_read_version(); + err = wait_future(tr1GrvFuture2); + if (err) { + fdb::EmptyFuture tr1OnErrorFuture = tr.on_error(err); + fdb_check(wait_future(tr1OnErrorFuture)); + continue; + } + + int64_t tr1EndVersion; + CHECK(!tr1GrvFuture2.get(&tr1EndVersion)); + + fdb::Int64Future tr2GrvFuture2 = tr2.get_read_version(); + err = wait_future(tr2GrvFuture2); + if (err) { + fdb::EmptyFuture tr2OnErrorFuture = tr2.on_error(err); + fdb_check(wait_future(tr2OnErrorFuture)); + continue; + } + + int64_t tr2EndVersion; + CHECK(!tr2GrvFuture2.get(&tr2EndVersion)); + + // If we reset the transaction, then the read version will change + CHECK(tr1StartVersion == tr1EndVersion); + CHECK(tr2StartVersion == tr2EndVersion); + break; + } +} + int main(int argc, char** argv) { if (argc < 3) { std::cout << "Unit tests for the FoundationDB C API.\n" diff --git a/documentation/sphinx/source/api-version-upgrade-guide.rst b/documentation/sphinx/source/api-version-upgrade-guide.rst index 707d8e3246..46e5aa6fcc 100644 --- a/documentation/sphinx/source/api-version-upgrade-guide.rst +++ b/documentation/sphinx/source/api-version-upgrade-guide.rst @@ -25,6 +25,8 @@ API version 700 General ------- +* Committing a transaction will no longer partially reset it. In particular, getting the read version from a transaction that has committed or failed to commit with an error will return the original read version. + Python bindings --------------- diff --git a/documentation/sphinx/source/release-notes/release-notes-700.rst b/documentation/sphinx/source/release-notes/release-notes-700.rst index cfc0730e90..44566955ee 100644 --- a/documentation/sphinx/source/release-notes/release-notes-700.rst +++ b/documentation/sphinx/source/release-notes/release-notes-700.rst @@ -91,6 +91,7 @@ Other Changes * The ``foundationdb`` service installed by the RPM packages will now automatically restart ``fdbmonitor`` after 60 seconds when it fails. `(PR #3841) `_ * Capture output of forked snapshot processes in trace events. `(PR #4254) `_ * Add ErrorKind field to Severity 40 trace events. `(PR #4741) `_ +* Committing a transaction will no longer partially reset it as of API version 700. `(PR #) `_ Earlier release notes --------------------- diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 20d2b9343d..0c84400ef7 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -5244,7 +5244,10 @@ ACTOR Future commitAndWatch(Transaction* self) { self->setupWatches(); } - self->reset(); + if (!self->apiVersionAtLeast(700)) { + self->reset(); + } + return Void(); } catch (Error& e) { if (e.code() != error_code_actor_cancelled) { @@ -5253,7 +5256,10 @@ ACTOR Future commitAndWatch(Transaction* self) { } self->versionstampPromise.sendError(transaction_invalid_version()); - self->reset(); + + if (!self->apiVersionAtLeast(700)) { + self->reset(); + } } throw; From e39cfd48c39ed2b6e9662f56d4aaf4a80071e85f Mon Sep 17 00:00:00 2001 From: Zhe Wu Date: Sun, 25 Jul 2021 22:37:08 -0700 Subject: [PATCH 11/26] Ignore goodRecruitmentTime and populate default PEER_LATENCY_CHECK_MIN_POPULATION --- fdbclient/ServerKnobs.cpp | 1 + fdbserver/ClusterController.actor.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 6d74c2f67a..bc568e039b 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -644,6 +644,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( DBINFO_FAILED_DELAY, 1.0 ); init( ENABLE_WORKER_HEALTH_MONITOR, false ); init( WORKER_HEALTH_MONITOR_INTERVAL, 60.0 ); + init( PEER_LATENCY_CHECK_MIN_POPULATION, 30 ); init( PEER_LATENCY_DEGRADATION_PERCENTILE, 0.90 ); init( PEER_LATENCY_DEGRADATION_THRESHOLD, 0.05 ); init( PEER_TIMEOUT_PERCENTAGE_DEGRADATION_THRESHOLD, 0.1 ); diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 97ef92b1e5..7ed3811e9a 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -4722,7 +4722,7 @@ ACTOR Future workerHealthMonitor(ClusterControllerData* self) { loop { try { while (!self->goodRecruitmentTime.isReady()) { - wait(self->goodRecruitmentTime); + wait(lowPriorityDelay(SERVER_KNOBS->CC_WORKER_HEALTH_CHECKING_INTERVAL)); } self->degradedServers = self->getServersWithDegradedLink(); From febc26a1ea3cbf58dd79241f596f8722fc9b72db Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 26 Jul 2021 11:43:07 -0700 Subject: [PATCH 12/26] Fix some cases where we were reusing a committed transaction without resetting it. --- fdbserver/ClusterController.actor.cpp | 2 ++ fdbserver/StorageCache.actor.cpp | 1 + fdbserver/workloads/CommitBugCheck.actor.cpp | 3 +++ .../workloads/DifferentClustersSameRV.actor.cpp | 1 + fdbserver/workloads/RandomSelector.actor.cpp | 13 +++++++++++++ .../workloads/SpecialKeySpaceCorrectness.actor.cpp | 1 + fdbserver/workloads/StatusWorkload.actor.cpp | 1 + 7 files changed, 22 insertions(+) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 97ef92b1e5..b02bc0b7c7 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -195,6 +195,8 @@ public: } loop { + tr.reset(); + // Wait for some changes while (!self->anyDelta.get()) wait(self->anyDelta.onChange()); diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 8f44f054d6..d3f46fa234 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -2156,6 +2156,7 @@ ACTOR Future watchInterface(StorageCacheData* self, StorageServerInterface tr.set(storageKey, storageCacheServerValue(ssi)); wait(tr.commit()); } + tr.reset(); break; } catch (Error& e) { wait(tr.onError(e)); diff --git a/fdbserver/workloads/CommitBugCheck.actor.cpp b/fdbserver/workloads/CommitBugCheck.actor.cpp index 1980d605dd..855f4de680 100644 --- a/fdbserver/workloads/CommitBugCheck.actor.cpp +++ b/fdbserver/workloads/CommitBugCheck.actor.cpp @@ -45,6 +45,7 @@ struct CommitBugWorkload : TestWorkload { try { tr.set(key, val1); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { TraceEvent("CommitBugSetVal1Error").error(e); @@ -57,6 +58,7 @@ struct CommitBugWorkload : TestWorkload { try { tr.set(key, val2); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { TraceEvent("CommitBugSetVal2Error").error(e); @@ -85,6 +87,7 @@ struct CommitBugWorkload : TestWorkload { try { tr.clear(key); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { TraceEvent("CommitBugClearValError").error(e); diff --git a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp index 1bcc0afc84..d6c21e4b2e 100644 --- a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp +++ b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp @@ -191,6 +191,7 @@ struct DifferentClustersSameRVWorkload : TestWorkload { serializer(w, x); tr.set(self->keyToRead, w.toValue()); wait(tr.commit()); + tr.reset(); } catch (Error& e) { wait(tr.onError(e)); } diff --git a/fdbserver/workloads/RandomSelector.actor.cpp b/fdbserver/workloads/RandomSelector.actor.cpp index d4c6bd00fe..500ce3b859 100644 --- a/fdbserver/workloads/RandomSelector.actor.cpp +++ b/fdbserver/workloads/RandomSelector.actor.cpp @@ -125,6 +125,7 @@ struct RandomSelectorWorkload : TestWorkload { //TraceEvent("RYOWInit").detail("Key",myKeyA).detail("Value",myValue); } wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { wait(tr.onError(e)); @@ -149,6 +150,7 @@ struct RandomSelectorWorkload : TestWorkload { try { tr.set(StringRef(clientID + "d/" + myKeyA), myValue); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { wait(tr.onError(e)); @@ -163,6 +165,7 @@ struct RandomSelectorWorkload : TestWorkload { try { tr.clear(StringRef(clientID + "d/" + myKeyA)); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { wait(tr.onError(e)); @@ -184,6 +187,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.clear(KeyRangeRef(StringRef(clientID + "d/" + myKeyA), StringRef(clientID + "d/" + myKeyB))); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { wait(tr.onError(e)); @@ -231,6 +235,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::AddValue); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -254,6 +259,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::AppendIfFits); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -277,6 +283,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::And); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -300,6 +307,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::Or); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -323,6 +331,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::Xor); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -346,6 +355,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::Max); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -369,6 +379,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::Min); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -392,6 +403,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::ByteMin); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; @@ -415,6 +427,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.set(StringRef(clientID + "z/" + myRandomIDKey), StringRef()); tr.atomicOp(StringRef(clientID + "d/" + myKeyA), myValue, MutationRef::ByteMax); wait(tr.commit()); + tr.reset(); break; } catch (Error& e) { error = e; diff --git a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp index 7c6c2006a6..03c1f3bda4 100644 --- a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp +++ b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp @@ -783,6 +783,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { Value(worker.processClass.toString())); // Set it as the same class type as before, thus only // class source will be changed wait(tx->commit()); + tx->reset(); Optional class_source = wait(tx->get( Key("process/class_source/" + address) .withPrefix( diff --git a/fdbserver/workloads/StatusWorkload.actor.cpp b/fdbserver/workloads/StatusWorkload.actor.cpp index 21c568d31e..98eecafef3 100644 --- a/fdbserver/workloads/StatusWorkload.actor.cpp +++ b/fdbserver/workloads/StatusWorkload.actor.cpp @@ -153,6 +153,7 @@ struct StatusWorkload : TestWorkload { tr.set(latencyBandConfigKey, ValueRef(config)); wait(tr.commit()); + tr.reset(); if (deterministicRandom()->random01() < 0.3) { return Void(); From 9f3e5d9bb549b8f622a9000894eb0599be6ee9fe Mon Sep 17 00:00:00 2001 From: Dan Lambright Date: Mon, 26 Jul 2021 15:58:41 -0400 Subject: [PATCH 13/26] use a BinaryWriter::toValue and BinaryReader::fromStringRef instead of encoding time using a string --- fdbserver/Coordination.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbserver/Coordination.actor.cpp b/fdbserver/Coordination.actor.cpp index fc603746d4..b7caf8e86a 100644 --- a/fdbserver/Coordination.actor.cpp +++ b/fdbserver/Coordination.actor.cpp @@ -456,7 +456,7 @@ struct LeaderRegisterCollection { self->forward[forwardingInfo[i].key.removePrefix(fwdKeys.begin)] = forwardInfo; } for (int i = 0; i < forwardingTime.size(); i++) { - double time = std::stod(forwardingTime[i].value.toString().c_str()); + double time = BinaryReader::fromStringRef(forwardingTime[i].value, Unversioned()); self->forwardStartTime[forwardingTime[i].key.removePrefix(fwdTimeKeys.begin)] = time; } return Void(); @@ -495,7 +495,7 @@ struct LeaderRegisterCollection { self->forwardStartTime[key] = forwardTime; OnDemandStore& store = *self->pStore; store->set(KeyValueRef(key.withPrefix(fwdKeys.begin), conn.toString())); - store->set(KeyValueRef(key.withPrefix(fwdTimeKeys.begin), std::to_string(forwardTime))); + store->set(KeyValueRef(key.withPrefix(fwdTimeKeys.begin), BinaryWriter::toValue(forwardTime, Unversioned()))); wait(store->commit()); return Void(); } From a38db9c6e3296ed60c7ea054f6f12d38797f39cb Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Mon, 26 Jul 2021 13:56:27 -0700 Subject: [PATCH 14/26] Add PR number --- documentation/sphinx/source/release-notes/release-notes-700.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/documentation/sphinx/source/release-notes/release-notes-700.rst b/documentation/sphinx/source/release-notes/release-notes-700.rst index 44566955ee..1246d42618 100644 --- a/documentation/sphinx/source/release-notes/release-notes-700.rst +++ b/documentation/sphinx/source/release-notes/release-notes-700.rst @@ -91,7 +91,7 @@ Other Changes * The ``foundationdb`` service installed by the RPM packages will now automatically restart ``fdbmonitor`` after 60 seconds when it fails. `(PR #3841) `_ * Capture output of forked snapshot processes in trace events. `(PR #4254) `_ * Add ErrorKind field to Severity 40 trace events. `(PR #4741) `_ -* Committing a transaction will no longer partially reset it as of API version 700. `(PR #) `_ +* Committing a transaction will no longer partially reset it as of API version 700. `(PR #5271) `_ Earlier release notes --------------------- From ff8a1e0ed2152b9382786513fd5108273a95e370 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Mon, 26 Jul 2021 16:37:55 -0700 Subject: [PATCH 15/26] Ignore warning from valgrind about F_SET_RW_HINT usage. --- contrib/TestHarness/Program.cs.cmake | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/TestHarness/Program.cs.cmake b/contrib/TestHarness/Program.cs.cmake index 28aa8687c1..e48d441c1b 100644 --- a/contrib/TestHarness/Program.cs.cmake +++ b/contrib/TestHarness/Program.cs.cmake @@ -966,6 +966,10 @@ namespace SummarizeTest // When running ASAN we expect to see this message. Boost coroutine should be using the correct asan annotations so that it shouldn't produce any false positives. continue; } + if (err.EndsWith("Warning: unimplemented fcntl command: 1036")) { + // Valgrind produces this warning when F_SET_RW_HINT is used + continue; + } if (stderrSeverity == (int)Magnesium.Severity.SevError) { error = true; From b3e22ad57387270d578505014186e90d5c549323 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Mon, 26 Jul 2021 18:10:03 -0700 Subject: [PATCH 16/26] Move stderr exception checking so that ignored output does not count against the error limit. --- contrib/TestHarness/Program.cs.cmake | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/contrib/TestHarness/Program.cs.cmake b/contrib/TestHarness/Program.cs.cmake index e48d441c1b..8b159ac2b1 100644 --- a/contrib/TestHarness/Program.cs.cmake +++ b/contrib/TestHarness/Program.cs.cmake @@ -638,6 +638,15 @@ namespace SummarizeTest { if(!String.IsNullOrEmpty(errLine.Data)) { + if (errLine.Data.EndsWith("WARNING: ASan doesn't fully support makecontext/swapcontext functions and may produce false positives in some cases!")) { + // When running ASAN we expect to see this message. Boost coroutine should be using the correct asan annotations so that it shouldn't produce any false positives. + return; + } + if (errLine.Data.EndsWith("Warning: unimplemented fcntl command: 1036")) { + // Valgrind produces this warning when F_SET_RW_HINT is used + return; + } + hasError = true; if(Errors.Count < maxErrors) { if(errLine.Data.Length > maxErrorLength) { @@ -962,14 +971,6 @@ namespace SummarizeTest int stderrBytes = 0; foreach (string err in outputErrors) { - if (err.EndsWith("WARNING: ASan doesn't fully support makecontext/swapcontext functions and may produce false positives in some cases!")) { - // When running ASAN we expect to see this message. Boost coroutine should be using the correct asan annotations so that it shouldn't produce any false positives. - continue; - } - if (err.EndsWith("Warning: unimplemented fcntl command: 1036")) { - // Valgrind produces this warning when F_SET_RW_HINT is used - continue; - } if (stderrSeverity == (int)Magnesium.Severity.SevError) { error = true; From 507c1f11e362ab967e58b856c0bc1859666509a1 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Mon, 26 Jul 2021 19:55:10 -0700 Subject: [PATCH 17/26] Add .log() to bare TraceEvent() invocations without any .detail()s to avoid clang-tidy warning about immediate destruction of object without use. --- FDBLibTLS/FDBLibTLSPolicy.cpp | 42 ++++++++-------- FDBLibTLS/FDBLibTLSSession.cpp | 16 +++--- fdbbackup/FileDecoder.actor.cpp | 2 +- .../BackupContainerLocalDirectory.actor.cpp | 2 +- fdbclient/DatabaseBackupAgent.actor.cpp | 22 ++++---- fdbclient/FileBackupAgent.actor.cpp | 10 ++-- fdbclient/MultiVersionTransaction.actor.cpp | 2 +- fdbclient/NativeAPI.actor.cpp | 14 +++--- fdbclient/ReadYourWrites.actor.cpp | 2 +- fdbclient/SystemData.cpp | 2 +- fdbrpc/FlowTransport.actor.cpp | 2 +- fdbrpc/sim2.actor.cpp | 4 +- fdbserver/ApplyMetadataMutation.cpp | 2 +- fdbserver/BackupWorker.actor.cpp | 6 +-- fdbserver/ClusterController.actor.cpp | 20 ++++---- fdbserver/CommitProxyServer.actor.cpp | 8 +-- fdbserver/CoordinatedState.actor.cpp | 2 +- fdbserver/CoroFlow.actor.cpp | 6 +-- fdbserver/CoroFlowCoro.actor.cpp | 6 +-- fdbserver/DataDistribution.actor.cpp | 50 +++++++++---------- fdbserver/GrvProxyServer.actor.cpp | 2 +- fdbserver/KeyValueStoreMemory.actor.cpp | 4 +- fdbserver/KeyValueStoreSQLite.actor.cpp | 6 +-- fdbserver/LeaderElection.actor.cpp | 8 +-- fdbserver/LogSystem.h | 2 +- fdbserver/MetricLogger.actor.cpp | 2 +- fdbserver/MoveKeys.actor.cpp | 8 +-- fdbserver/OldTLogServer_4_6.actor.cpp | 4 +- fdbserver/OldTLogServer_6_0.actor.cpp | 22 ++++---- fdbserver/OldTLogServer_6_2.actor.cpp | 20 ++++---- fdbserver/QuietDatabase.actor.cpp | 4 +- fdbserver/Ratekeeper.actor.cpp | 6 +-- fdbserver/RestoreApplier.actor.cpp | 2 +- fdbserver/RestoreApplier.actor.h | 2 +- fdbserver/RestoreController.actor.cpp | 10 ++-- fdbserver/RestoreWorker.actor.cpp | 6 +-- fdbserver/SimulatedCluster.actor.cpp | 4 +- fdbserver/StorageCache.actor.cpp | 6 +-- fdbserver/TLogServer.actor.cpp | 14 +++--- fdbserver/TagPartitionedLogSystem.actor.cpp | 12 ++--- fdbserver/masterserver.actor.cpp | 22 ++++---- fdbserver/storageserver.actor.cpp | 22 ++++---- fdbserver/tester.actor.cpp | 6 +-- fdbserver/worker.actor.cpp | 16 +++--- .../AtomicOpsApiCorrectness.actor.cpp | 18 +++---- fdbserver/workloads/AtomicRestore.actor.cpp | 10 ++-- .../workloads/AtomicSwitchover.actor.cpp | 24 ++++----- ...kupAndParallelRestoreCorrectness.actor.cpp | 2 +- .../workloads/BackupCorrectness.actor.cpp | 2 +- fdbserver/workloads/BackupToDBAbort.actor.cpp | 16 +++--- .../workloads/BackupToDBUpgrade.actor.cpp | 4 +- fdbserver/workloads/BulkSetup.actor.h | 2 +- fdbserver/workloads/ChangeConfig.actor.cpp | 8 +-- fdbserver/workloads/ConflictRange.actor.cpp | 2 +- .../workloads/ConsistencyCheck.actor.cpp | 14 +++--- fdbserver/workloads/CpuProfiler.actor.cpp | 8 +-- fdbserver/workloads/Cycle.actor.cpp | 4 +- fdbserver/workloads/DDMetrics.actor.cpp | 2 +- .../DifferentClustersSameRV.actor.cpp | 16 +++--- .../workloads/ExternalWorkload.actor.cpp | 6 +-- .../workloads/HealthMetricsApi.actor.cpp | 2 +- .../workloads/IncrementalBackup.actor.cpp | 18 +++---- fdbserver/workloads/KVStoreTest.actor.cpp | 4 +- fdbserver/workloads/KillRegion.actor.cpp | 20 ++++---- fdbserver/workloads/LogMetrics.actor.cpp | 2 +- fdbserver/workloads/LowLatency.actor.cpp | 2 +- .../workloads/MachineAttrition.actor.cpp | 10 ++-- fdbserver/workloads/ParallelRestore.actor.cpp | 2 +- fdbserver/workloads/Ping.actor.cpp | 2 +- fdbserver/workloads/PopulateTPCC.actor.cpp | 2 +- fdbserver/workloads/RandomMoveKeys.actor.cpp | 10 ++-- fdbserver/workloads/RestoreBackup.actor.cpp | 2 +- fdbserver/workloads/SimpleAtomicAdd.actor.cpp | 2 +- fdbserver/workloads/SnapTest.actor.cpp | 12 ++--- .../SpecialKeySpaceCorrectness.actor.cpp | 8 +-- fdbserver/workloads/StatusWorkload.actor.cpp | 2 +- fdbserver/workloads/Throttling.actor.cpp | 2 +- .../workloads/TimeKeeperCorrectness.actor.cpp | 6 +-- fdbserver/workloads/TriggerRecovery.actor.cpp | 2 +- fdbserver/workloads/VersionStamp.actor.cpp | 2 +- fdbserver/workloads/WriteDuringRead.actor.cpp | 2 +- .../workloads/WriteTagThrottling.actor.cpp | 2 +- flow/DeterministicRandom.cpp | 2 +- flow/Net2.actor.cpp | 10 ++-- flow/Platform.actor.cpp | 16 +++--- flow/ThreadHelper.actor.h | 4 +- flow/Trace.cpp | 2 +- flow/genericactors.actor.h | 2 +- 88 files changed, 360 insertions(+), 360 deletions(-) diff --git a/FDBLibTLS/FDBLibTLSPolicy.cpp b/FDBLibTLS/FDBLibTLSPolicy.cpp index 1f6b18e2e9..2e3142165d 100644 --- a/FDBLibTLS/FDBLibTLSPolicy.cpp +++ b/FDBLibTLS/FDBLibTLSPolicy.cpp @@ -42,7 +42,7 @@ FDBLibTLSPolicy::FDBLibTLSPolicy(Reference plugin) key_data_set(false), verify_peers_set(false) { if ((tls_cfg = tls_config_new()) == nullptr) { - TraceEvent(SevError, "FDBLibTLSConfigError"); + TraceEvent(SevError, "FDBLibTLSConfigError").log(); throw std::runtime_error("FDBLibTLSConfigError"); } @@ -67,14 +67,14 @@ ITLSSession* FDBLibTLSPolicy::create_session(bool is_client, // servername, since this will be ignored - the servername should be // matched by the verify criteria instead. if (verify_peers_set && servername != nullptr) { - TraceEvent(SevError, "FDBLibTLSVerifyPeersWithServerName"); + TraceEvent(SevError, "FDBLibTLSVerifyPeersWithServerName").log(); return nullptr; } // If verify peers has not been set, then require a server name to // avoid an accidental lack of name validation. if (!verify_peers_set && servername == nullptr) { - TraceEvent(SevError, "FDBLibTLSNoServerName"); + TraceEvent(SevError, "FDBLibTLSNoServerName").log(); return nullptr; } } @@ -123,18 +123,18 @@ struct stack_st_X509* FDBLibTLSPolicy::parse_cert_pem(const uint8_t* cert_pem, s if (cert_pem_len > INT_MAX) goto err; if ((bio = BIO_new_mem_buf((void*)cert_pem, cert_pem_len)) == nullptr) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } if ((certs = sk_X509_new_null()) == nullptr) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } ERR_clear_error(); while ((cert = PEM_read_bio_X509(bio, nullptr, password_cb, nullptr)) != nullptr) { if (!sk_X509_push(certs, cert)) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } } @@ -150,7 +150,7 @@ struct stack_st_X509* FDBLibTLSPolicy::parse_cert_pem(const uint8_t* cert_pem, s } if (sk_X509_num(certs) < 1) { - TraceEvent(SevError, "FDBLibTLSNoCerts"); + TraceEvent(SevError, "FDBLibTLSNoCerts").log(); goto err; } @@ -168,11 +168,11 @@ err: bool FDBLibTLSPolicy::set_ca_data(const uint8_t* ca_data, int ca_len) { if (ca_data_set) { - TraceEvent(SevError, "FDBLibTLSCAAlreadySet"); + TraceEvent(SevError, "FDBLibTLSCAAlreadySet").log(); return false; } if (session_created) { - TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive"); + TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive").log(); return false; } @@ -194,11 +194,11 @@ bool FDBLibTLSPolicy::set_ca_data(const uint8_t* ca_data, int ca_len) { bool FDBLibTLSPolicy::set_cert_data(const uint8_t* cert_data, int cert_len) { if (cert_data_set) { - TraceEvent(SevError, "FDBLibTLSCertAlreadySet"); + TraceEvent(SevError, "FDBLibTLSCertAlreadySet").log(); return false; } if (session_created) { - TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive"); + TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive").log(); return false; } @@ -218,11 +218,11 @@ bool FDBLibTLSPolicy::set_key_data(const uint8_t* key_data, int key_len, const c bool rc = false; if (key_data_set) { - TraceEvent(SevError, "FDBLibTLSKeyAlreadySet"); + TraceEvent(SevError, "FDBLibTLSKeyAlreadySet").log(); goto err; } if (session_created) { - TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive"); + TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive").log(); goto err; } @@ -231,7 +231,7 @@ bool FDBLibTLSPolicy::set_key_data(const uint8_t* key_data, int key_len, const c long len; if ((bio = BIO_new_mem_buf((void*)key_data, key_len)) == nullptr) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } ERR_clear_error(); @@ -241,7 +241,7 @@ bool FDBLibTLSPolicy::set_key_data(const uint8_t* key_data, int key_len, const c if ((ERR_GET_LIB(errnum) == ERR_LIB_PEM && ERR_GET_REASON(errnum) == PEM_R_BAD_DECRYPT) || (ERR_GET_LIB(errnum) == ERR_LIB_EVP && ERR_GET_REASON(errnum) == EVP_R_BAD_DECRYPT)) { - TraceEvent(SevError, "FDBLibTLSIncorrectPassword"); + TraceEvent(SevError, "FDBLibTLSIncorrectPassword").log(); } else { ERR_error_string_n(errnum, errbuf, sizeof(errbuf)); TraceEvent(SevError, "FDBLibTLSPrivateKeyError").detail("LibcryptoErrorMessage", errbuf); @@ -250,15 +250,15 @@ bool FDBLibTLSPolicy::set_key_data(const uint8_t* key_data, int key_len, const c } BIO_free(bio); if ((bio = BIO_new(BIO_s_mem())) == nullptr) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } if (!PEM_write_bio_PrivateKey(bio, key, nullptr, nullptr, 0, nullptr, nullptr)) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } if ((len = BIO_get_mem_data(bio, &data)) <= 0) { - TraceEvent(SevError, "FDBLibTLSOutOfMemory"); + TraceEvent(SevError, "FDBLibTLSOutOfMemory").log(); goto err; } if (tls_config_set_key_mem(tls_cfg, (const uint8_t*)data, len) == -1) { @@ -283,16 +283,16 @@ err: bool FDBLibTLSPolicy::set_verify_peers(int count, const uint8_t* verify_peers[], int verify_peers_len[]) { if (verify_peers_set) { - TraceEvent(SevError, "FDBLibTLSVerifyPeersAlreadySet"); + TraceEvent(SevError, "FDBLibTLSVerifyPeersAlreadySet").log(); return false; } if (session_created) { - TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive"); + TraceEvent(SevError, "FDBLibTLSPolicyAlreadyActive").log(); return false; } if (count < 1) { - TraceEvent(SevError, "FDBLibTLSNoVerifyPeers"); + TraceEvent(SevError, "FDBLibTLSNoVerifyPeers").log(); return false; } diff --git a/FDBLibTLS/FDBLibTLSSession.cpp b/FDBLibTLS/FDBLibTLSSession.cpp index 4c4c8e5bfa..75c60dd049 100644 --- a/FDBLibTLS/FDBLibTLSSession.cpp +++ b/FDBLibTLS/FDBLibTLSSession.cpp @@ -73,7 +73,7 @@ FDBLibTLSSession::FDBLibTLSSession(Reference policy, if (is_client) { if ((tls_ctx = tls_client()) == nullptr) { - TraceEvent(SevError, "FDBLibTLSClientError", uid); + TraceEvent(SevError, "FDBLibTLSClientError", uid).log(); throw std::runtime_error("FDBLibTLSClientError"); } if (tls_configure(tls_ctx, policy->tls_cfg) == -1) { @@ -88,7 +88,7 @@ FDBLibTLSSession::FDBLibTLSSession(Reference policy, } } else { if ((tls_sctx = tls_server()) == nullptr) { - TraceEvent(SevError, "FDBLibTLSServerError", uid); + TraceEvent(SevError, "FDBLibTLSServerError", uid).log(); throw std::runtime_error("FDBLibTLSServerError"); } if (tls_configure(tls_sctx, policy->tls_cfg) == -1) { @@ -250,7 +250,7 @@ std::tuple FDBLibTLSSession::check_verify(Referenceparse_cert_pem(cert_pem, cert_pem_len)) == nullptr) @@ -388,14 +388,14 @@ int FDBLibTLSSession::handshake() { int FDBLibTLSSession::read(uint8_t* data, int length) { if (!handshake_completed) { - TraceEvent(SevError, "FDBLibTLSReadHandshakeError"); + TraceEvent(SevError, "FDBLibTLSReadHandshakeError").log(); return FAILED; } ssize_t n = tls_read(tls_ctx, data, length); if (n > 0) { if (n > INT_MAX) { - TraceEvent(SevError, "FDBLibTLSReadOverflow"); + TraceEvent(SevError, "FDBLibTLSReadOverflow").log(); return FAILED; } return (int)n; @@ -415,14 +415,14 @@ int FDBLibTLSSession::read(uint8_t* data, int length) { int FDBLibTLSSession::write(const uint8_t* data, int length) { if (!handshake_completed) { - TraceEvent(SevError, "FDBLibTLSWriteHandshakeError", uid); + TraceEvent(SevError, "FDBLibTLSWriteHandshakeError", uid).log(); return FAILED; } ssize_t n = tls_write(tls_ctx, data, length); if (n > 0) { if (n > INT_MAX) { - TraceEvent(SevError, "FDBLibTLSWriteOverflow", uid); + TraceEvent(SevError, "FDBLibTLSWriteOverflow", uid).log(); return FAILED; } return (int)n; diff --git a/fdbbackup/FileDecoder.actor.cpp b/fdbbackup/FileDecoder.actor.cpp index b8e4bc138f..22e5774bcb 100644 --- a/fdbbackup/FileDecoder.actor.cpp +++ b/fdbbackup/FileDecoder.actor.cpp @@ -571,7 +571,7 @@ int main(int argc, char** argv) { } if (!param.tlsConfig.setupTLS()) { - TraceEvent(SevError, "TLSError"); + TraceEvent(SevError, "TLSError").log(); throw tls_error(); } diff --git a/fdbclient/BackupContainerLocalDirectory.actor.cpp b/fdbclient/BackupContainerLocalDirectory.actor.cpp index b89d085a64..a638ee147a 100644 --- a/fdbclient/BackupContainerLocalDirectory.actor.cpp +++ b/fdbclient/BackupContainerLocalDirectory.actor.cpp @@ -227,7 +227,7 @@ Future> BackupContainerLocalDirectory::readFile(const std: } if (g_simulator.getCurrentProcess()->uid == UID()) { - TraceEvent(SevError, "BackupContainerReadFileOnUnsetProcessID"); + TraceEvent(SevError, "BackupContainerReadFileOnUnsetProcessID").log(); } std::string uniquePath = fullPath + "." + g_simulator.getCurrentProcess()->uid.toString() + ".lnk"; unlink(uniquePath.c_str()); diff --git a/fdbclient/DatabaseBackupAgent.actor.cpp b/fdbclient/DatabaseBackupAgent.actor.cpp index a8de6819dd..deb57d6141 100644 --- a/fdbclient/DatabaseBackupAgent.actor.cpp +++ b/fdbclient/DatabaseBackupAgent.actor.cpp @@ -364,7 +364,7 @@ struct BackupRangeTaskFunc : TaskFuncBase { TEST(true); // range insert delayed because too versionMap is too large if (rangeCount > CLIENT_KNOBS->BACKUP_MAP_KEY_UPPER_LIMIT) - TraceEvent(SevWarnAlways, "DBA_KeyRangeMapTooLarge"); + TraceEvent(SevWarnAlways, "DBA_KeyRangeMapTooLarge").log(); wait(delay(1)); task->params[BackupRangeTaskFunc::keyBackupRangeBeginKey] = rangeBegin; @@ -1882,7 +1882,7 @@ struct CopyDiffLogsUpgradeTaskFunc : TaskFuncBase { state Reference onDone = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]); if (task->params[BackupAgentBase::destUid].size() == 0) { - TraceEvent("DBA_CopyDiffLogsUpgradeTaskFuncAbortInUpgrade"); + TraceEvent("DBA_CopyDiffLogsUpgradeTaskFuncAbortInUpgrade").log(); wait(success(AbortOldBackupTaskFunc::addTask(tr, taskBucket, task, TaskCompletionKey::signal(onDone)))); } else { Version beginVersion = @@ -2377,11 +2377,11 @@ void checkAtomicSwitchOverConfig(StatusObjectReader srcStatus, StatusObjectReade try { // Check if src is unlocked and dest is locked if (getLockedStatus(srcStatus) != false) { - TraceEvent(SevWarn, "DBA_AtomicSwitchOverSrcLocked"); + TraceEvent(SevWarn, "DBA_AtomicSwitchOverSrcLocked").log(); throw backup_error(); } if (getLockedStatus(destStatus) != true) { - TraceEvent(SevWarn, "DBA_AtomicSwitchOverDestUnlocked"); + TraceEvent(SevWarn, "DBA_AtomicSwitchOverDestUnlocked").log(); throw backup_error(); } // Check if mutation-stream-id matches @@ -2402,7 +2402,7 @@ void checkAtomicSwitchOverConfig(StatusObjectReader srcStatus, StatusObjectReade destDRAgents.end(), std::inserter(intersectingAgents, intersectingAgents.begin())); if (intersectingAgents.empty()) { - TraceEvent(SevWarn, "DBA_SwitchOverPossibleDRAgentsIncorrectSetup"); + TraceEvent(SevWarn, "DBA_SwitchOverPossibleDRAgentsIncorrectSetup").log(); throw backup_error(); } } catch (std::runtime_error& e) { @@ -2757,7 +2757,7 @@ public: } } - TraceEvent("DBA_SwitchoverReady"); + TraceEvent("DBA_SwitchoverReady").log(); try { wait(backupAgent->discontinueBackup(dest, tagName)); @@ -2768,7 +2768,7 @@ public: wait(success(backupAgent->waitBackup(dest, tagName, StopWhenDone::True))); - TraceEvent("DBA_SwitchoverStopped"); + TraceEvent("DBA_SwitchoverStopped").log(); state ReadYourWritesTransaction tr3(dest); loop { @@ -2789,7 +2789,7 @@ public: } } - TraceEvent("DBA_SwitchoverVersionUpgraded"); + TraceEvent("DBA_SwitchoverVersionUpgraded").log(); try { wait(drAgent.submitBackup(backupAgent->taskBucket->src, @@ -2805,15 +2805,15 @@ public: throw; } - TraceEvent("DBA_SwitchoverSubmitted"); + TraceEvent("DBA_SwitchoverSubmitted").log(); wait(success(drAgent.waitSubmitted(backupAgent->taskBucket->src, tagName))); - TraceEvent("DBA_SwitchoverStarted"); + TraceEvent("DBA_SwitchoverStarted").log(); wait(backupAgent->unlockBackup(dest, tagName)); - TraceEvent("DBA_SwitchoverUnlocked"); + TraceEvent("DBA_SwitchoverUnlocked").log(); return Void(); } diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 90cc1ab321..8c057aadd2 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -5478,7 +5478,7 @@ public: try { wait(discontinueBackup(backupAgent, ryw_tr, tagName)); wait(ryw_tr->commit()); - TraceEvent("AS_DiscontinuedBackup"); + TraceEvent("AS_DiscontinuedBackup").log(); break; } catch (Error& e) { if (e.code() == error_code_backup_unneeded || e.code() == error_code_backup_duplicate) { @@ -5489,7 +5489,7 @@ public: } wait(success(waitBackup(backupAgent, cx, tagName.toString(), StopWhenDone::True))); - TraceEvent("AS_BackupStopped"); + TraceEvent("AS_BackupStopped").log(); ryw_tr->reset(); loop { @@ -5502,7 +5502,7 @@ public: ryw_tr->clear(range); } wait(ryw_tr->commit()); - TraceEvent("AS_ClearedRange"); + TraceEvent("AS_ClearedRange").log(); break; } catch (Error& e) { wait(ryw_tr->onError(e)); @@ -5512,7 +5512,7 @@ public: Reference bc = wait(backupConfig.backupContainer().getOrThrow(cx)); if (fastRestore) { - TraceEvent("AtomicParallelRestoreStartRestore"); + TraceEvent("AtomicParallelRestoreStartRestore").log(); Version targetVersion = ::invalidVersion; wait(submitParallelRestore(cx, tagName, @@ -5533,7 +5533,7 @@ public: } return -1; } else { - TraceEvent("AS_StartRestore"); + TraceEvent("AS_StartRestore").log(); Version ver = wait(restore(backupAgent, cx, cx, diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp index eedacf80aa..663400cebf 100644 --- a/fdbclient/MultiVersionTransaction.actor.cpp +++ b/fdbclient/MultiVersionTransaction.actor.cpp @@ -1521,7 +1521,7 @@ std::vector> MultiVersionApi::copyExternalLibraryPe #else std::vector> MultiVersionApi::copyExternalLibraryPerThread(std::string path) { if (threadCount > 1) { - TraceEvent(SevError, "MultipleClientThreadsUnsupportedOnWindows"); + TraceEvent(SevError, "MultipleClientThreadsUnsupportedOnWindows").log(); throw unsupported_operation(); } std::vector> paths; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 20d2b9343d..a98cf30adc 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -488,7 +488,7 @@ ACTOR static Future transactionInfoCommitActor(Transaction* tr, std::vecto ACTOR static Future delExcessClntTxnEntriesActor(Transaction* tr, int64_t clientTxInfoSizeLimit) { state const Key clientLatencyName = CLIENT_LATENCY_INFO_PREFIX.withPrefix(fdbClientInfoPrefixRange.begin); state const Key clientLatencyAtomicCtr = CLIENT_LATENCY_INFO_CTR_PREFIX.withPrefix(fdbClientInfoPrefixRange.begin); - TraceEvent(SevInfo, "DelExcessClntTxnEntriesCalled"); + TraceEvent(SevInfo, "DelExcessClntTxnEntriesCalled").log(); loop { try { tr->reset(); @@ -496,7 +496,7 @@ ACTOR static Future delExcessClntTxnEntriesActor(Transaction* tr, int64_t tr->setOption(FDBTransactionOptions::LOCK_AWARE); Optional ctrValue = wait(tr->get(KeyRef(clientLatencyAtomicCtr), Snapshot::True)); if (!ctrValue.present()) { - TraceEvent(SevInfo, "NumClntTxnEntriesNotFound"); + TraceEvent(SevInfo, "NumClntTxnEntriesNotFound").log(); return Void(); } state int64_t txInfoSize = 0; @@ -1627,7 +1627,7 @@ ACTOR static Future switchConnectionFileImpl(Reference Transaction::commitMutations() { if (options.debugDump) { UID u = nondeterministicRandom()->randomUniqueID(); - TraceEvent("TransactionDump", u); + TraceEvent("TransactionDump", u).log(); for (auto i = tr.transaction.mutations.begin(); i != tr.transaction.mutations.end(); ++i) TraceEvent("TransactionMutation", u) .detail("T", i->type) @@ -6326,7 +6326,7 @@ void Transaction::setToken(uint64_t token) { void enableClientInfoLogging() { ASSERT(networkOptions.logClientInfo.present() == false); networkOptions.logClientInfo = true; - TraceEvent(SevInfo, "ClientInfoLoggingEnabled"); + TraceEvent(SevInfo, "ClientInfoLoggingEnabled").log(); } ACTOR Future snapCreate(Database cx, Standalone snapCmd, UID snapUID) { @@ -6380,7 +6380,7 @@ ACTOR Future checkSafeExclusions(Database cx, vector exc } throw; } - TraceEvent("ExclusionSafetyCheckCoordinators"); + TraceEvent("ExclusionSafetyCheckCoordinators").log(); state ClientCoordinators coordinatorList(cx->getConnectionFile()); state vector>> leaderServers; leaderServers.reserve(coordinatorList.clientLeaderServers.size()); @@ -6393,7 +6393,7 @@ ACTOR Future checkSafeExclusions(Database cx, vector exc choose { when(wait(smartQuorum(leaderServers, leaderServers.size() / 2 + 1, 1.0))) {} when(wait(delay(3.0))) { - TraceEvent("ExclusionSafetyCheckNoCoordinatorQuorum"); + TraceEvent("ExclusionSafetyCheckNoCoordinatorQuorum").log(); return false; } } diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index f3f9a391c7..66ca328ea8 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -1164,7 +1164,7 @@ public: if (!ryw->resetPromise.isSet()) ryw->resetPromise.sendError(transaction_timed_out()); wait(delay(deterministicRandom()->random01() * 5)); - TraceEvent("ClientBuggifyInFlightCommit"); + TraceEvent("ClientBuggifyInFlightCommit").log(); wait(ryw->tr.commit()); } diff --git a/fdbclient/SystemData.cpp b/fdbclient/SystemData.cpp index eeef762361..c3b7e85343 100644 --- a/fdbclient/SystemData.cpp +++ b/fdbclient/SystemData.cpp @@ -129,7 +129,7 @@ void decodeKeyServersValue(RangeResult result, std::sort(src.begin(), src.end()); std::sort(dest.begin(), dest.end()); if (missingIsError && (src.size() != srcTag.size() || dest.size() != destTag.size())) { - TraceEvent(SevError, "AttemptedToDecodeMissingTag"); + TraceEvent(SevError, "AttemptedToDecodeMissingTag").log(); for (const KeyValueRef& kv : result) { Tag tag = decodeServerTagValue(kv.value); UID serverID = decodeServerTagKey(kv.key); diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index d44483da12..c1bd3726a0 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -1019,7 +1019,7 @@ static void scanPackets(TransportData* transport, BUGGIFY_WITH_PROB(0.0001)) { g_simulator.lastConnectionFailure = g_network->now(); isBuggifyEnabled = true; - TraceEvent(SevInfo, "BitsFlip"); + TraceEvent(SevInfo, "BitsFlip").log(); int flipBits = 32 - (int)floor(log2(deterministicRandom()->randomUInt32())); uint32_t firstFlipByteLocation = deterministicRandom()->randomUInt32() % packetLen; diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 33da8e7ed6..2b8e4406da 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -470,12 +470,12 @@ public: state TaskPriority currentTaskID = g_network->getCurrentTask(); if (++openCount >= 3000) { - TraceEvent(SevError, "TooManyFiles"); + TraceEvent(SevError, "TooManyFiles").log(); ASSERT(false); } if (openCount == 2000) { - TraceEvent(SevWarnAlways, "DisableConnectionFailures_TooManyFiles"); + TraceEvent(SevWarnAlways, "DisableConnectionFailures_TooManyFiles").log(); g_simulator.speedUpSimulation = true; g_simulator.connectionFailuresDisableDuration = 1e6; } diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index cea35794a9..732766e9a4 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -404,7 +404,7 @@ void applyMetadataMutations(SpanID const& spanContext, confChange = true; TEST(true); // Recovering at a higher version. } else if (m.param1 == writeRecoveryKey) { - TraceEvent("WriteRecoveryKeySet", dbgid); + TraceEvent("WriteRecoveryKeySet", dbgid).log(); if (!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); TEST(true); // Snapshot created, setting writeRecoveryKey in txnStateStore diff --git a/fdbserver/BackupWorker.actor.cpp b/fdbserver/BackupWorker.actor.cpp index d6bd6a0ebb..8087259e1c 100644 --- a/fdbserver/BackupWorker.actor.cpp +++ b/fdbserver/BackupWorker.actor.cpp @@ -477,7 +477,7 @@ ACTOR Future monitorBackupStartedKeyChanges(BackupData* self, bool present if (present || !watch) return true; } else { - TraceEvent("BackupWorkerEmptyStartKey", self->myId); + TraceEvent("BackupWorkerEmptyStartKey", self->myId).log(); self->onBackupChanges(uidVersions); self->exitEarly = shouldExit; @@ -887,7 +887,7 @@ ACTOR Future pullAsyncData(BackupData* self) { state Version tagAt = std::max(self->pulledVersion.get(), std::max(self->startVersion, self->savedVersion)); state Arena prev; - TraceEvent("BackupWorkerPull", self->myId); + TraceEvent("BackupWorkerPull", self->myId).log(); loop { while (self->paused.get()) { wait(self->paused.onChange()); @@ -1017,7 +1017,7 @@ ACTOR static Future monitorWorkerPause(BackupData* self) { Optional value = wait(tr->get(backupPausedKey)); bool paused = value.present() && value.get() == LiteralStringRef("1"); if (self->paused.get() != paused) { - TraceEvent(paused ? "BackupWorkerPaused" : "BackupWorkerResumed", self->myId); + TraceEvent(paused ? "BackupWorkerPaused" : "BackupWorkerResumed", self->myId).log(); self->paused.set(paused); } diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 7ed3811e9a..987a8e03a6 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -1962,7 +1962,7 @@ public: } if (bestDC != clusterControllerDcId) { - TraceEvent("BestDCIsNotClusterDC"); + TraceEvent("BestDCIsNotClusterDC").log(); vector> dcPriority; dcPriority.push_back(bestDC); desiredDcIds.set(dcPriority); @@ -3094,7 +3094,7 @@ ACTOR Future clusterWatchDatabase(ClusterControllerData* cluster, ClusterC // When this someday is implemented, make sure forced failures still cause the master to be recruited again loop { - TraceEvent("CCWDB", cluster->id); + TraceEvent("CCWDB", cluster->id).log(); try { state double recoveryStart = now(); TraceEvent("CCWDB", cluster->id).detail("Recruiting", "Master"); @@ -3915,7 +3915,7 @@ ACTOR Future timeKeeperSetVersion(ClusterControllerData* self) { ACTOR Future timeKeeper(ClusterControllerData* self) { state KeyBackedMap versionMap(timeKeeperPrefixRange.begin); - TraceEvent("TimeKeeperStarted"); + TraceEvent("TimeKeeperStarted").log(); wait(timeKeeperSetVersion(self)); @@ -3929,7 +3929,7 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { // how long it is taking to hear responses from each other component. UID debugID = deterministicRandom()->randomUniqueID(); - TraceEvent("TimeKeeperCommit", debugID); + TraceEvent("TimeKeeperCommit", debugID).log(); tr->debugTransaction(debugID); } tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -4080,7 +4080,7 @@ ACTOR Future monitorProcessClasses(ClusterControllerData* self) { } wait(trVer.commit()); - TraceEvent("ProcessClassUpgrade"); + TraceEvent("ProcessClassUpgrade").log(); break; } catch (Error& e) { wait(trVer.onError(e)); @@ -4509,7 +4509,7 @@ ACTOR Future handleForcedRecoveries(ClusterControllerData* self, ClusterCo } wait(fCommit); } - TraceEvent("ForcedRecoveryFinish", self->id); + TraceEvent("ForcedRecoveryFinish", self->id).log(); self->db.forceRecovery = false; req.reply.send(Void()); } @@ -4518,7 +4518,7 @@ ACTOR Future handleForcedRecoveries(ClusterControllerData* self, ClusterCo ACTOR Future startDataDistributor(ClusterControllerData* self) { wait(delay(0.0)); // If master fails at the same time, give it a chance to clear master PID. - TraceEvent("CCStartDataDistributor", self->id); + TraceEvent("CCStartDataDistributor", self->id).log(); loop { try { state bool no_distributor = !self->db.serverInfo->get().distributor.present(); @@ -4585,7 +4585,7 @@ ACTOR Future monitorDataDistributor(ClusterControllerData* self) { ACTOR Future startRatekeeper(ClusterControllerData* self) { wait(delay(0.0)); // If master fails at the same time, give it a chance to clear master PID. - TraceEvent("CCStartRatekeeper", self->id); + TraceEvent("CCStartRatekeeper", self->id).log(); loop { try { state bool no_ratekeeper = !self->db.serverInfo->get().ratekeeper.present(); @@ -4702,7 +4702,7 @@ ACTOR Future dbInfoUpdater(ClusterControllerData* self) { req.serializedDbInfo = BinaryWriter::toValue(self->db.serverInfo->get(), AssumeVersion(g_network->protocolVersion())); - TraceEvent("DBInfoStartBroadcast", self->id); + TraceEvent("DBInfoStartBroadcast", self->id).log(); choose { when(std::vector notUpdated = wait(broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, Optional(), false))) { @@ -4757,7 +4757,7 @@ ACTOR Future workerHealthMonitor(ClusterControllerData* self) { } } else { self->excludedDegradedServers.clear(); - TraceEvent("DegradedServerDetectedAndSuggestRecovery"); + TraceEvent("DegradedServerDetectedAndSuggestRecovery").log(); } } } diff --git a/fdbserver/CommitProxyServer.actor.cpp b/fdbserver/CommitProxyServer.actor.cpp index 137bf3f240..e58564ecf8 100644 --- a/fdbserver/CommitProxyServer.actor.cpp +++ b/fdbserver/CommitProxyServer.actor.cpp @@ -1756,7 +1756,7 @@ ACTOR Future proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* co ACTOR Future proxyCheckSafeExclusion(Reference const> db, ExclusionSafetyCheckRequest req) { - TraceEvent("SafetyCheckCommitProxyBegin"); + TraceEvent("SafetyCheckCommitProxyBegin").log(); state ExclusionSafetyCheckReply reply(false); if (!db->get().distributor.present()) { TraceEvent(SevWarnAlways, "DataDistributorNotPresent").detail("Operation", "ExclusionSafetyCheck"); @@ -1778,7 +1778,7 @@ ACTOR Future proxyCheckSafeExclusion(Reference cons throw e; } } - TraceEvent("SafetyCheckCommitProxyFinish"); + TraceEvent("SafetyCheckCommitProxyFinish").log(); req.reply.send(reply); return Void(); } @@ -1796,7 +1796,7 @@ ACTOR Future reportTxnTagCommitCost(UID myID, TraceEvent("ProxyRatekeeperChanged", myID).detail("RKID", db->get().ratekeeper.get().id()); nextRequestTimer = Void(); } else { - TraceEvent("ProxyRatekeeperDied", myID); + TraceEvent("ProxyRatekeeperDied", myID).log(); nextRequestTimer = Never(); } } @@ -1936,7 +1936,7 @@ ACTOR Future commitProxyServerCore(CommitProxyInterface proxy, } } when(ProxySnapRequest snapReq = waitNext(proxy.proxySnapReq.getFuture())) { - TraceEvent(SevDebug, "SnapMasterEnqueue"); + TraceEvent(SevDebug, "SnapMasterEnqueue").log(); addActor.send(proxySnapCreate(snapReq, &commitData)); } when(ExclusionSafetyCheckRequest exclCheckReq = waitNext(proxy.exclusionSafetyCheckReq.getFuture())) { diff --git a/fdbserver/CoordinatedState.actor.cpp b/fdbserver/CoordinatedState.actor.cpp index 955e1c88b2..dca378af98 100644 --- a/fdbserver/CoordinatedState.actor.cpp +++ b/fdbserver/CoordinatedState.actor.cpp @@ -316,7 +316,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 - TraceEvent("QuorumChangeAbortedByConcurrency"); + TraceEvent("QuorumChangeAbortedByConcurrency").log(); throw coordinated_state_conflict(); } diff --git a/fdbserver/CoroFlow.actor.cpp b/fdbserver/CoroFlow.actor.cpp index cc719423ec..cc453ed3a2 100644 --- a/fdbserver/CoroFlow.actor.cpp +++ b/fdbserver/CoroFlow.actor.cpp @@ -173,7 +173,7 @@ class WorkPool final : public IThreadPool, public ReferenceCountedPRIORITY_TEAM_HEALTHY), wrongConfiguration(false), id(deterministicRandom()->randomUniqueID()) { if (servers.empty()) { - TraceEvent(SevInfo, "ConstructTCTeamFromEmptyServers"); + TraceEvent(SevInfo, "ConstructTCTeamFromEmptyServers").log(); } serverIDs.reserve(servers.size()); for (int i = 0; i < servers.size(); i++) { @@ -445,7 +445,7 @@ ACTOR Future> getInitialDataDistribution(Data } if (!result->mode || !ddEnabledState->isDDEnabled()) { // DD can be disabled persistently (result->mode = 0) or transiently (isDDEnabled() = 0) - TraceEvent(SevDebug, "GetInitialDataDistribution_DisabledDD"); + TraceEvent(SevDebug, "GetInitialDataDistribution_DisabledDD").log(); return result; } @@ -475,7 +475,7 @@ ACTOR Future> getInitialDataDistribution(Data wait(tr.onError(e)); ASSERT(!succeeded); // We shouldn't be retrying if we have already started modifying result in this loop - TraceEvent("GetInitialTeamsRetry", distributorId); + TraceEvent("GetInitialTeamsRetry", distributorId).log(); } } @@ -4160,14 +4160,14 @@ ACTOR Future monitorPerpetualStorageWiggle(DDTeamCollection* teamCollectio &stopWiggleSignal, finishStorageWiggleSignal.getFuture(), teamCollection)); collection.add(perpetualStorageWiggler( &stopWiggleSignal, finishStorageWiggleSignal, teamCollection, ddEnabledState)); - TraceEvent("PerpetualStorageWiggleOpen", teamCollection->distributorId); + TraceEvent("PerpetualStorageWiggleOpen", teamCollection->distributorId).log(); } else if (speed == 0) { if (!stopWiggleSignal.get()) { stopWiggleSignal.set(true); wait(collection.signalAndReset()); teamCollection->pauseWiggle->set(true); } - TraceEvent("PerpetualStorageWiggleClose", teamCollection->distributorId); + TraceEvent("PerpetualStorageWiggleClose", teamCollection->distributorId).log(); } wait(watchFuture); break; @@ -4262,7 +4262,7 @@ ACTOR Future waitHealthyZoneChange(DDTeamCollection* self) { auto p = decodeHealthyZoneValue(val.get()); if (p.first == ignoreSSFailuresZoneString) { // healthyZone is now overloaded for DD diabling purpose, which does not timeout - TraceEvent("DataDistributionDisabledForStorageServerFailuresStart", self->distributorId); + TraceEvent("DataDistributionDisabledForStorageServerFailuresStart", self->distributorId).log(); healthyZoneTimeout = Never(); } else if (p.second > tr.getReadVersion().get()) { double timeoutSeconds = @@ -4277,15 +4277,15 @@ ACTOR Future waitHealthyZoneChange(DDTeamCollection* self) { } } else if (self->healthyZone.get().present()) { // maintenance hits timeout - TraceEvent("MaintenanceZoneEndTimeout", self->distributorId); + TraceEvent("MaintenanceZoneEndTimeout", self->distributorId).log(); self->healthyZone.set(Optional()); } } else if (self->healthyZone.get().present()) { // `healthyZone` has been cleared if (self->healthyZone.get().get() == ignoreSSFailuresZoneString) { - TraceEvent("DataDistributionDisabledForStorageServerFailuresEnd", self->distributorId); + TraceEvent("DataDistributionDisabledForStorageServerFailuresEnd", self->distributorId).log(); } else { - TraceEvent("MaintenanceZoneEndManualClear", self->distributorId); + TraceEvent("MaintenanceZoneEndManualClear", self->distributorId).log(); } self->healthyZone.set(Optional()); } @@ -4432,7 +4432,7 @@ ACTOR Future storageServerFailureTracker(DDTeamCollection* self, status->isFailed = false; } else if (self->clearHealthyZoneFuture.isReady()) { self->clearHealthyZoneFuture = clearHealthyZone(self->cx); - TraceEvent("MaintenanceZoneCleared", self->distributorId); + TraceEvent("MaintenanceZoneCleared", self->distributorId).log(); self->healthyZone.set(Optional()); } } @@ -5491,7 +5491,7 @@ ACTOR Future serverGetTeamRequests(TeamCollectionInterface tci, DDTeamColl } ACTOR Future remoteRecovered(Reference const> db) { - TraceEvent("DDTrackerStarting"); + TraceEvent("DDTrackerStarting").log(); while (db->get().recoveryState < RecoveryState::ALL_LOGS_RECRUITED) { TraceEvent("DDTrackerStarting").detail("RecoveryState", (int)db->get().recoveryState); wait(db->onChange()); @@ -5625,7 +5625,7 @@ ACTOR Future waitForDataDistributionEnabled(Database cx, const DDEnabledSt try { Optional mode = wait(tr.get(dataDistributionModeKey)); if (!mode.present() && ddEnabledState->isDDEnabled()) { - TraceEvent("WaitForDDEnabledSucceeded"); + TraceEvent("WaitForDDEnabledSucceeded").log(); return Void(); } if (mode.present()) { @@ -5636,7 +5636,7 @@ ACTOR Future waitForDataDistributionEnabled(Database cx, const DDEnabledSt .detail("Mode", m) .detail("IsDDEnabled", ddEnabledState->isDDEnabled()); if (m && ddEnabledState->isDDEnabled()) { - TraceEvent("WaitForDDEnabledSucceeded"); + TraceEvent("WaitForDDEnabledSucceeded").log(); return Void(); } } @@ -5711,7 +5711,7 @@ ACTOR Future debugCheckCoalescing(Database cx) { .detail("Value", ranges[j].value); } - TraceEvent("DoneCheckingCoalescing"); + TraceEvent("DoneCheckingCoalescing").log(); return Void(); } catch (Error& e) { wait(tr.onError(e)); @@ -5807,10 +5807,10 @@ ACTOR Future dataDistribution(Reference self, state Promise removeFailedServer; try { loop { - TraceEvent("DDInitTakingMoveKeysLock", self->ddId); + TraceEvent("DDInitTakingMoveKeysLock", self->ddId).log(); MoveKeysLock lock_ = wait(takeMoveKeysLock(cx, self->ddId)); lock = lock_; - TraceEvent("DDInitTookMoveKeysLock", self->ddId); + TraceEvent("DDInitTookMoveKeysLock", self->ddId).log(); DatabaseConfiguration configuration_ = wait(getDatabaseConfiguration(cx)); configuration = configuration_; @@ -5854,7 +5854,7 @@ ACTOR Future dataDistribution(Reference self, } } - TraceEvent("DDInitUpdatedReplicaKeys", self->ddId); + TraceEvent("DDInitUpdatedReplicaKeys", self->ddId).log(); Reference initData_ = wait(getInitialDataDistribution( cx, self->ddId, @@ -5882,7 +5882,7 @@ ACTOR Future dataDistribution(Reference self, // mode may be set true by system operator using fdbcli and isDDEnabled() set to true break; } - TraceEvent("DataDistributionDisabled", self->ddId); + TraceEvent("DataDistributionDisabled", self->ddId).log(); TraceEvent("MovingData", self->ddId) .detail("InFlight", 0) @@ -5919,7 +5919,7 @@ ACTOR Future dataDistribution(Reference self, .trackLatest("TotalDataInFlightRemote"); wait(waitForDataDistributionEnabled(cx, ddEnabledState)); - TraceEvent("DataDistributionEnabled"); + TraceEvent("DataDistributionEnabled").log(); } // When/If this assertion fails, Evan owes Ben a pat on the back for his foresight @@ -6256,7 +6256,7 @@ ACTOR Future ddSnapCreateCore(DistributorSnapRequest snapReq, Reference ddSnapCreate(DistributorSnapRequest snapReq, if (!ddEnabledState->setDDEnabled(false, snapReq.snapUID)) { // disable DD before doing snapCreate, if previous snap req has already disabled DD then this operation fails // here - TraceEvent("SnapDDSetDDEnabledFailedInMemoryCheck"); + TraceEvent("SnapDDSetDDEnabledFailedInMemoryCheck").log(); snapReq.reply.sendError(operation_failed()); return Void(); } @@ -6344,18 +6344,18 @@ bool _exclusionSafetyCheck(vector& excludeServerIDs, DDTeamCollection* team ACTOR Future ddExclusionSafetyCheck(DistributorExclusionSafetyCheckRequest req, Reference self, Database cx) { - TraceEvent("DDExclusionSafetyCheckBegin", self->ddId); + TraceEvent("DDExclusionSafetyCheckBegin", self->ddId).log(); vector ssis = wait(getStorageServers(cx)); DistributorExclusionSafetyCheckReply reply(true); if (!self->teamCollection) { - TraceEvent("DDExclusionSafetyCheckTeamCollectionInvalid", self->ddId); + TraceEvent("DDExclusionSafetyCheckTeamCollectionInvalid", self->ddId).log(); reply.safe = false; req.reply.send(reply); return Void(); } // If there is only 1 team, unsafe to mark failed: team building can get stuck due to lack of servers left if (self->teamCollection->teams.size() <= 1) { - TraceEvent("DDExclusionSafetyCheckNotEnoughTeams", self->ddId); + TraceEvent("DDExclusionSafetyCheckNotEnoughTeams", self->ddId).log(); reply.safe = false; req.reply.send(reply); return Void(); @@ -6371,7 +6371,7 @@ ACTOR Future ddExclusionSafetyCheck(DistributorExclusionSafetyCheckRequest } } reply.safe = _exclusionSafetyCheck(excludeServerIDs, self->teamCollection); - TraceEvent("DDExclusionSafetyCheckFinish", self->ddId); + TraceEvent("DDExclusionSafetyCheckFinish", self->ddId).log(); req.reply.send(reply); return Void(); } diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 3ebf2931e6..28650ced5a 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -300,7 +300,7 @@ ACTOR Future getRate(UID myID, TraceEvent("ProxyRatekeeperChanged", myID).detail("RKID", db->get().ratekeeper.get().id()); nextRequestTimer = Void(); // trigger GetRate request } else { - TraceEvent("ProxyRatekeeperDied", myID); + TraceEvent("ProxyRatekeeperDied", myID).log(); nextRequestTimer = Never(); reply = Never(); } diff --git a/fdbserver/KeyValueStoreMemory.actor.cpp b/fdbserver/KeyValueStoreMemory.actor.cpp index 0008296a96..ff00e29c84 100644 --- a/fdbserver/KeyValueStoreMemory.actor.cpp +++ b/fdbserver/KeyValueStoreMemory.actor.cpp @@ -141,7 +141,7 @@ public: Future commit(bool sequential) override { if (getAvailableSize() <= 0) { - TraceEvent(SevError, "KeyValueStoreMemory_OutOfSpace", id); + TraceEvent(SevError, "KeyValueStoreMemory_OutOfSpace", id).log(); return Never(); } @@ -605,7 +605,7 @@ private: if (zeroFillSize) { if (exactRecovery) { - TraceEvent(SevError, "KVSMemExpectedExact", self->id); + TraceEvent(SevError, "KVSMemExpectedExact", self->id).log(); ASSERT(false); } diff --git a/fdbserver/KeyValueStoreSQLite.actor.cpp b/fdbserver/KeyValueStoreSQLite.actor.cpp index 6e3043f3f3..e0920fc8f5 100644 --- a/fdbserver/KeyValueStoreSQLite.actor.cpp +++ b/fdbserver/KeyValueStoreSQLite.actor.cpp @@ -727,7 +727,7 @@ struct RawCursor { try { db.checkError("BtreeCloseCursor", sqlite3BtreeCloseCursor(cursor)); } catch (...) { - TraceEvent(SevError, "RawCursorDestructionError"); + TraceEvent(SevError, "RawCursorDestructionError").log(); } delete[](char*) cursor; } @@ -1737,9 +1737,9 @@ private: freeListPages(freeListPages), cursor(nullptr), dbgid(dbgid), readThreads(*pReadThreads), checkAllChecksumsOnOpen(checkAllChecksumsOnOpen), checkIntegrityOnOpen(checkIntegrityOnOpen) {} ~Writer() override { - TraceEvent("KVWriterDestroying", dbgid); + TraceEvent("KVWriterDestroying", dbgid).log(); delete cursor; - TraceEvent("KVWriterDestroyed", dbgid); + TraceEvent("KVWriterDestroyed", dbgid).log(); } void init() override { if (checkAllChecksumsOnOpen) { diff --git a/fdbserver/LeaderElection.actor.cpp b/fdbserver/LeaderElection.actor.cpp index 2f0fdaaf3b..29bba2a955 100644 --- a/fdbserver/LeaderElection.actor.cpp +++ b/fdbserver/LeaderElection.actor.cpp @@ -156,7 +156,7 @@ ACTOR Future tryBecomeLeaderInternal(ServerCoordinators coordinators, } if (leader.present() && leader.get().second && leader.get().first.equalInternalId(myInfo)) { - TraceEvent("BecomingLeader", myInfo.changeID); + TraceEvent("BecomingLeader", myInfo.changeID).log(); ASSERT(leader.get().first.serializedInfo == proposedSerializedInterface); outSerializedLeader->set(leader.get().first.serializedInfo); iAmLeader = true; @@ -184,7 +184,7 @@ ACTOR Future tryBecomeLeaderInternal(ServerCoordinators coordinators, when(wait(nominees->onChange())) {} when(wait(badCandidateTimeout.isValid() ? badCandidateTimeout : Never())) { TEST(true); // Bad candidate timeout - TraceEvent("LeaderBadCandidateTimeout", myInfo.changeID); + TraceEvent("LeaderBadCandidateTimeout", myInfo.changeID).log(); break; } when(wait(candidacies)) { ASSERT(false); } @@ -225,7 +225,7 @@ ACTOR Future tryBecomeLeaderInternal(ServerCoordinators coordinators, //TraceEvent("StillLeader", myInfo.changeID); } // We are still leader when(wait(quorum(false_heartbeats, false_heartbeats.size() / 2 + 1))) { - TraceEvent("ReplacedAsLeader", myInfo.changeID); + TraceEvent("ReplacedAsLeader", myInfo.changeID).log(); break; } // We are definitely not leader when(wait(delay(SERVER_KNOBS->POLLING_FREQUENCY))) { @@ -243,7 +243,7 @@ ACTOR Future tryBecomeLeaderInternal(ServerCoordinators coordinators, .detail("Coordinator", coordinators.leaderElectionServers[i].candidacy.getEndpoint().getPrimaryAddress()); } - TraceEvent("ReleasingLeadership", myInfo.changeID); + TraceEvent("ReleasingLeadership", myInfo.changeID).log(); break; } // Give up on being leader, because we apparently have poor communications when(wait(asyncPriorityInfo->onChange())) {} diff --git a/fdbserver/LogSystem.h b/fdbserver/LogSystem.h index 5aa14810de..1a936edf3e 100644 --- a/fdbserver/LogSystem.h +++ b/fdbserver/LogSystem.h @@ -291,7 +291,7 @@ public: if (allLocations) { // special handling for allLocations - TraceEvent("AllLocationsSet"); + TraceEvent("AllLocationsSet").log(); for (int i = 0; i < logServers.size(); i++) { newLocations.push_back(i); } diff --git a/fdbserver/MetricLogger.actor.cpp b/fdbserver/MetricLogger.actor.cpp index aee9ea67d6..e8c8a7ab8f 100644 --- a/fdbserver/MetricLogger.actor.cpp +++ b/fdbserver/MetricLogger.actor.cpp @@ -374,7 +374,7 @@ ACTOR Future updateMetricRegistration(Database cx, MetricsConfig* config, ACTOR Future runMetrics(Future fcx, Key prefix) { // Never log to an empty prefix, it's pretty much always a bad idea. if (prefix.size() == 0) { - TraceEvent(SevWarnAlways, "TDMetricsRefusingEmptyPrefix"); + TraceEvent(SevWarnAlways, "TDMetricsRefusingEmptyPrefix").log(); return Void(); } diff --git a/fdbserver/MoveKeys.actor.cpp b/fdbserver/MoveKeys.actor.cpp index 906bca4b16..55706e458f 100644 --- a/fdbserver/MoveKeys.actor.cpp +++ b/fdbserver/MoveKeys.actor.cpp @@ -100,7 +100,7 @@ ACTOR static Future checkMoveKeysLock(Transaction* tr, const DDEnabledState* ddEnabledState, bool isWrite = true) { if (!ddEnabledState->isDDEnabled()) { - TraceEvent(SevDebug, "DDDisabledByInMemoryCheck"); + TraceEvent(SevDebug, "DDDisabledByInMemoryCheck").log(); throw movekeys_conflict(); } Optional readVal = wait(tr->get(moveKeysLockOwnerKey)); @@ -1143,7 +1143,7 @@ ACTOR Future> addStorageServer(Database cx, StorageServe if (SERVER_KNOBS->TSS_HACK_IDENTITY_MAPPING) { // THIS SHOULD NEVER BE ENABLED IN ANY NON-TESTING ENVIRONMENT - TraceEvent(SevError, "TSSIdentityMappingEnabled"); + TraceEvent(SevError, "TSSIdentityMappingEnabled").log(); tssMapDB.set(tr, server.id(), server.id()); } } @@ -1268,7 +1268,7 @@ ACTOR Future removeStorageServer(Database cx, if (SERVER_KNOBS->TSS_HACK_IDENTITY_MAPPING) { // THIS SHOULD NEVER BE ENABLED IN ANY NON-TESTING ENVIRONMENT - TraceEvent(SevError, "TSSIdentityMappingEnabled"); + TraceEvent(SevError, "TSSIdentityMappingEnabled").log(); tssMapDB.erase(tr, serverID); } else if (tssPairID.present()) { // remove the TSS from the mapping @@ -1440,7 +1440,7 @@ void seedShardServers(Arena& arena, CommitTransactionRef& tr, vectorTSS_HACK_IDENTITY_MAPPING) { // THIS SHOULD NEVER BE ENABLED IN ANY NON-TESTING ENVIRONMENT - TraceEvent(SevError, "TSSIdentityMappingEnabled"); + TraceEvent(SevError, "TSSIdentityMappingEnabled").log(); // hack key-backed map here since we can't really change CommitTransactionRef to a RYW transaction Key uidRef = Codec::pack(s.id()).pack(); tr.set(arena, uidRef.withPrefix(tssMappingKeys.begin), uidRef); diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index ce291e644c..0a561dbf7f 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1387,7 +1387,7 @@ ACTOR Future restorePersistentState(TLogData* self, LocalityData locality) state KeyRange tagKeys; // PERSIST: Read basic state from persistentData; replay persistentQueue but don't erase it - TraceEvent("TLogRestorePersistentState", self->dbgid); + TraceEvent("TLogRestorePersistentState", self->dbgid).log(); IKeyValueStore* storage = self->persistentData; state Future> fFormat = storage->readValue(persistFormat.key); @@ -1575,7 +1575,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).log(); try { wait(restorePersistentState(&self, locality)); diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 35cbc42535..998a65f463 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -876,7 +876,7 @@ ACTOR Future tLogPop(TLogData* self, TLogPopRequest req, ReferenceignorePopRequest && (g_network->now() > self->ignorePopDeadline)) { - TraceEvent("EnableTLogPlayAllIgnoredPops"); + TraceEvent("EnableTLogPlayAllIgnoredPops").log(); // use toBePopped and issue all the pops state std::map::iterator it; state vector> ignoredPops; @@ -1666,7 +1666,7 @@ ACTOR Future initPersistentState(TLogData* self, Reference logDat updatePersistentPopped(self, logData, logData->getTagData(tag)); } - TraceEvent("TLogInitCommit", logData->logId); + TraceEvent("TLogInitCommit", logData->logId).log(); wait(ioTimeoutError(self->persistentData->commit(), SERVER_KNOBS->TLOG_MAX_CREATE_DURATION)); return Void(); } @@ -1869,7 +1869,7 @@ ACTOR Future tLogEnablePopReq(TLogEnablePopRequest enablePopReq, TLogData* enablePopReq.reply.sendError(operation_failed()); return Void(); } - TraceEvent("EnableTLogPlayAllIgnoredPops2"); + TraceEvent("EnableTLogPlayAllIgnoredPops2").log(); // use toBePopped and issue all the pops std::map::iterator it; vector> ignoredPops; @@ -1923,7 +1923,7 @@ ACTOR Future serveTLogInterface(TLogData* self, } if (!logData->isPrimary && logData->stopped) { - TraceEvent("TLogAlreadyStopped", self->dbgid); + TraceEvent("TLogAlreadyStopped", self->dbgid).log(); logData->removed = logData->removed && logData->logSystem->get()->endEpoch(); } } else { @@ -2198,22 +2198,22 @@ ACTOR Future tLogCore(TLogData* self, } ACTOR Future checkEmptyQueue(TLogData* self) { - TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid); + TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid).log(); try { TLogQueueEntry r = wait(self->persistentQueue->readNext(self)); throw internal_error(); } catch (Error& e) { if (e.code() != error_code_end_of_stream) throw; - TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid); + TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid).log(); return Void(); } } ACTOR Future checkRecovered(TLogData* self) { - TraceEvent("TLogCheckRecoveredBegin", self->dbgid); + TraceEvent("TLogCheckRecoveredBegin", self->dbgid).log(); Optional v = wait(self->persistentData->readValue(StringRef())); - TraceEvent("TLogCheckRecoveredEnd", self->dbgid); + TraceEvent("TLogCheckRecoveredEnd", self->dbgid).log(); return Void(); } @@ -2227,7 +2227,7 @@ ACTOR Future restorePersistentState(TLogData* self, state KeyRange tagKeys; // PERSIST: Read basic state from persistentData; replay persistentQueue but don't erase it - TraceEvent("TLogRestorePersistentState", self->dbgid); + TraceEvent("TLogRestorePersistentState", self->dbgid).log(); state IKeyValueStore* storage = self->persistentData; wait(storage->init()); @@ -2585,7 +2585,7 @@ ACTOR Future tLogStart(TLogData* self, InitializeTLogRequest req, Locality logData->removed = rejoinMasters(self, recruited, req.epoch, Future(Void()), req.isPrimary); self->queueOrder.push_back(recruited.id()); - TraceEvent("TLogStart", logData->logId); + TraceEvent("TLogStart", logData->logId).log(); state Future updater; state bool pulledRecoveryVersions = false; try { @@ -2730,7 +2730,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).log(); try { if (restoreFromDisk) { wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index c581560faa..fd2522ec91 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1464,7 +1464,7 @@ ACTOR Future tLogPop(TLogData* self, TLogPopRequest req, ReferenceignorePopRequest && (g_network->now() > self->ignorePopDeadline)) { - TraceEvent("EnableTLogPlayAllIgnoredPops"); + TraceEvent("EnableTLogPlayAllIgnoredPops").log(); // use toBePopped and issue all the pops std::map::iterator it; vector> ignoredPops; @@ -1871,7 +1871,7 @@ ACTOR Future watchDegraded(TLogData* self) { wait(lowPriorityDelay(SERVER_KNOBS->TLOG_DEGRADED_DURATION)); - TraceEvent(SevWarnAlways, "TLogDegraded", self->dbgid); + TraceEvent(SevWarnAlways, "TLogDegraded", self->dbgid).log(); TEST(true); // TLog degraded self->degraded->set(true); return Void(); @@ -2109,7 +2109,7 @@ ACTOR Future initPersistentState(TLogData* self, Reference logDat updatePersistentPopped(self, logData, logData->getTagData(tag)); } - TraceEvent("TLogInitCommit", logData->logId); + TraceEvent("TLogInitCommit", logData->logId).log(); wait(self->persistentData->commit()); return Void(); } @@ -2312,7 +2312,7 @@ ACTOR Future tLogEnablePopReq(TLogEnablePopRequest enablePopReq, TLogData* enablePopReq.reply.sendError(operation_failed()); return Void(); } - TraceEvent("EnableTLogPlayAllIgnoredPops2"); + TraceEvent("EnableTLogPlayAllIgnoredPops2").log(); // use toBePopped and issue all the pops std::map::iterator it; state vector> ignoredPops; @@ -2657,7 +2657,7 @@ ACTOR Future tLogCore(TLogData* self, } ACTOR Future checkEmptyQueue(TLogData* self) { - TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid); + TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid).log(); try { bool recoveryFinished = wait(self->persistentQueue->initializeRecovery(0)); if (recoveryFinished) @@ -2667,15 +2667,15 @@ ACTOR Future checkEmptyQueue(TLogData* self) { } catch (Error& e) { if (e.code() != error_code_end_of_stream) throw; - TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid); + TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid).log(); return Void(); } } ACTOR Future checkRecovered(TLogData* self) { - TraceEvent("TLogCheckRecoveredBegin", self->dbgid); + TraceEvent("TLogCheckRecoveredBegin", self->dbgid).log(); Optional v = wait(self->persistentData->readValue(StringRef())); - TraceEvent("TLogCheckRecoveredEnd", self->dbgid); + TraceEvent("TLogCheckRecoveredEnd", self->dbgid).log(); return Void(); } @@ -2690,7 +2690,7 @@ ACTOR Future restorePersistentState(TLogData* self, state KeyRange tagKeys; // PERSIST: Read basic state from persistentData; replay persistentQueue but don't erase it - TraceEvent("TLogRestorePersistentState", self->dbgid); + TraceEvent("TLogRestorePersistentState", self->dbgid).log(); state IKeyValueStore* storage = self->persistentData; wait(storage->init()); @@ -3219,7 +3219,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).log(); try { if (restoreFromDisk) { wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index 359db3245c..dc4f8769a6 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -576,7 +576,7 @@ ACTOR Future repairDeadDatacenter(Database cx, // FIXME: the primary and remote can both be considered dead because excludes are not handled properly by the // datacenterDead function if (primaryDead && remoteDead) { - TraceEvent(SevWarnAlways, "CannotDisableFearlessConfiguration"); + TraceEvent(SevWarnAlways, "CannotDisableFearlessConfiguration").log(); return Void(); } if (primaryDead || remoteDead) { @@ -647,7 +647,7 @@ ACTOR Future waitForQuietDatabase(Database cx, loop { try { - TraceEvent("QuietDatabaseWaitingOnDataDistributor"); + TraceEvent("QuietDatabaseWaitingOnDataDistributor").log(); WorkerInterface distributorWorker = wait(getDataDistributorWorker(cx, dbInfo)); UID distributorUID = dbInfo->get().distributor.get().id(); TraceEvent("QuietDatabaseGotDataDistributor", distributorUID) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index a13f9583be..8a09e2f739 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -801,14 +801,14 @@ ACTOR Future monitorThrottlingChanges(RatekeeperData* self) { autoThrottlingEnabled.get().get() == LiteralStringRef("0")) { TEST(true); // Auto-throttling disabled if (self->autoThrottlingEnabled) { - TraceEvent("AutoTagThrottlingDisabled", self->id); + TraceEvent("AutoTagThrottlingDisabled", self->id).log(); } self->autoThrottlingEnabled = false; } else if (autoThrottlingEnabled.get().present() && autoThrottlingEnabled.get().get() == LiteralStringRef("1")) { TEST(true); // Auto-throttling enabled if (!self->autoThrottlingEnabled) { - TraceEvent("AutoTagThrottlingEnabled", self->id); + TraceEvent("AutoTagThrottlingEnabled", self->id).log(); } self->autoThrottlingEnabled = true; } else { @@ -870,7 +870,7 @@ ACTOR Future monitorThrottlingChanges(RatekeeperData* self) { committed = true; wait(watchFuture); - TraceEvent("RatekeeperThrottleSignaled", self->id); + TraceEvent("RatekeeperThrottleSignaled", self->id).log(); TEST(true); // Tag throttle changes detected break; } catch (Error& e) { diff --git a/fdbserver/RestoreApplier.actor.cpp b/fdbserver/RestoreApplier.actor.cpp index e1c638efd7..072aa53a5f 100644 --- a/fdbserver/RestoreApplier.actor.cpp +++ b/fdbserver/RestoreApplier.actor.cpp @@ -473,7 +473,7 @@ ACTOR static Future precomputeMutationsResult(Reference } } - TraceEvent("FastRestoreApplierGetAndComputeStagingKeysWaitOn", applierID); + TraceEvent("FastRestoreApplierGetAndComputeStagingKeysWaitOn", applierID).log(); wait(waitForAll(fGetAndComputeKeys)); // Sanity check all stagingKeys have been precomputed diff --git a/fdbserver/RestoreApplier.actor.h b/fdbserver/RestoreApplier.actor.h index 55a465fb14..bd87cf3799 100644 --- a/fdbserver/RestoreApplier.actor.h +++ b/fdbserver/RestoreApplier.actor.h @@ -317,7 +317,7 @@ struct ApplierBatchData : public ReferenceCounted { return false; } } - TraceEvent("FastRestoreApplierAllKeysPrecomputed"); + TraceEvent("FastRestoreApplierAllKeysPrecomputed").log(); return true; } diff --git a/fdbserver/RestoreController.actor.cpp b/fdbserver/RestoreController.actor.cpp index 4341fbf1cf..ad1bced5f5 100644 --- a/fdbserver/RestoreController.actor.cpp +++ b/fdbserver/RestoreController.actor.cpp @@ -714,7 +714,7 @@ ACTOR static Future> collectRestoreRequests(Database // restoreRequestTriggerKey should already been set loop { try { - TraceEvent("FastRestoreControllerPhaseCollectRestoreRequestsWait"); + TraceEvent("FastRestoreControllerPhaseCollectRestoreRequestsWait").log(); tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr.setOption(FDBTransactionOptions::LOCK_AWARE); @@ -732,7 +732,7 @@ ACTOR static Future> collectRestoreRequests(Database } break; } else { - TraceEvent(SevError, "FastRestoreControllerPhaseCollectRestoreRequestsEmptyRequests"); + TraceEvent(SevError, "FastRestoreControllerPhaseCollectRestoreRequestsEmptyRequests").log(); wait(delay(5.0)); } } catch (Error& e) { @@ -1079,7 +1079,7 @@ ACTOR static Future notifyLoadersVersionBatchFinished(std::map notifyRestoreCompleted(Reference self, bool terminate = false) { std::vector> requests; - TraceEvent("FastRestoreControllerPhaseNotifyRestoreCompletedStart"); + TraceEvent("FastRestoreControllerPhaseNotifyRestoreCompletedStart").log(); for (auto& loader : self->loadersInterf) { requests.emplace_back(loader.first, RestoreFinishRequest(terminate)); } @@ -1099,7 +1099,7 @@ ACTOR static Future notifyRestoreCompleted(Reference signalRestoreCompleted(Reference waitOnRestoreRequests(Database cx, UID nodeID = UID()) state Optional numRequests; // wait for the restoreRequestTriggerKey to be set by the client/test workload - TraceEvent("FastRestoreWaitOnRestoreRequest", nodeID); + TraceEvent("FastRestoreWaitOnRestoreRequest", nodeID).log(); loop { try { tr.reset(); @@ -288,9 +288,9 @@ ACTOR static Future waitOnRestoreRequests(Database cx, UID nodeID = UID()) if (!numRequests.present()) { state Future watchForRestoreRequest = tr.watch(restoreRequestTriggerKey); wait(tr.commit()); - TraceEvent(SevInfo, "FastRestoreWaitOnRestoreRequestTriggerKey", nodeID); + TraceEvent(SevInfo, "FastRestoreWaitOnRestoreRequestTriggerKey", nodeID).log(); wait(watchForRestoreRequest); - TraceEvent(SevInfo, "FastRestoreDetectRestoreRequestTriggerKeyChanged", nodeID); + TraceEvent(SevInfo, "FastRestoreDetectRestoreRequestTriggerKeyChanged", nodeID).log(); } else { TraceEvent(SevInfo, "FastRestoreRestoreRequestTriggerKey", nodeID) .detail("TriggerKey", numRequests.get().toString()); diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index f9916cf1bd..828bcfb93d 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -408,7 +408,7 @@ ACTOR Future runDr(Reference connFile) { wait(delay(1.0)); } - TraceEvent("StoppingDrAgents"); + TraceEvent("StoppingDrAgents").log(); for (auto it : agentFutures) { it.cancel(); @@ -2205,7 +2205,7 @@ ACTOR void setupAndRun(std::string dataFolder, TraceEvent(SevError, "SetupAndRunError").error(e); } - TraceEvent("SimulatedSystemDestruct"); + TraceEvent("SimulatedSystemDestruct").log(); g_simulator.stop(); destructed = true; wait(Never()); diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 8f44f054d6..1485e20633 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -425,7 +425,7 @@ ACTOR Future waitForVersion(StorageCacheData* data, Version version) { } if (deterministicRandom()->random01() < 0.001) - TraceEvent("WaitForVersion1000x"); + TraceEvent("WaitForVersion1000x").log(); choose { when(wait(data->version.whenAtLeast(version))) { // FIXME: A bunch of these can block with or without the following delay 0. @@ -1363,7 +1363,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang // doesn't fit on this cache. For now, we can just fail this cache role. In future, we should think // about evicting some data to make room for the remaining keys if (this_block.more) { - TraceEvent(SevDebug, "CacheWarmupMoreDataThanLimit", data->thisServerID); + TraceEvent(SevDebug, "CacheWarmupMoreDataThanLimit", data->thisServerID).log(); throw please_reboot(); } @@ -1780,7 +1780,7 @@ private: rollback(data, rollbackVersion, currentVersion); } } else { - TraceEvent(SevWarn, "SCPrivateCacheMutation: Unknown private mutation"); + TraceEvent(SevWarn, "SCPrivateCacheMutation: Unknown private mutation").log(); // ASSERT(false); // Unknown private mutation } } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index ff5b72ccdc..d8e27a78ba 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -2160,7 +2160,7 @@ ACTOR Future initPersistentState(TLogData* self, Reference logDat updatePersistentPopped(self, logData, logData->getTagData(tag)); } - TraceEvent("TLogInitCommit", logData->logId); + TraceEvent("TLogInitCommit", logData->logId).log(); wait(ioTimeoutError(self->persistentData->commit(), SERVER_KNOBS->TLOG_MAX_CREATE_DURATION)); return Void(); } @@ -2713,7 +2713,7 @@ ACTOR Future tLogCore(TLogData* self, } ACTOR Future checkEmptyQueue(TLogData* self) { - TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid); + TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid).log(); try { bool recoveryFinished = wait(self->persistentQueue->initializeRecovery(0)); if (recoveryFinished) @@ -2723,15 +2723,15 @@ ACTOR Future checkEmptyQueue(TLogData* self) { } catch (Error& e) { if (e.code() != error_code_end_of_stream) throw; - TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid); + TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid).log(); return Void(); } } ACTOR Future checkRecovered(TLogData* self) { - TraceEvent("TLogCheckRecoveredBegin", self->dbgid); + TraceEvent("TLogCheckRecoveredBegin", self->dbgid).log(); Optional v = wait(self->persistentData->readValue(StringRef())); - TraceEvent("TLogCheckRecoveredEnd", self->dbgid); + TraceEvent("TLogCheckRecoveredEnd", self->dbgid).log(); return Void(); } @@ -2746,7 +2746,7 @@ ACTOR Future restorePersistentState(TLogData* self, state KeyRange tagKeys; // PERSIST: Read basic state from persistentData; replay persistentQueue but don't erase it - TraceEvent("TLogRestorePersistentState", self->dbgid); + TraceEvent("TLogRestorePersistentState", self->dbgid).log(); state IKeyValueStore* storage = self->persistentData; wait(storage->init()); @@ -3294,7 +3294,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).log(); try { if (restoreFromDisk) { wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); diff --git a/fdbserver/TagPartitionedLogSystem.actor.cpp b/fdbserver/TagPartitionedLogSystem.actor.cpp index 2ab2b18062..d7fffa0ff6 100644 --- a/fdbserver/TagPartitionedLogSystem.actor.cpp +++ b/fdbserver/TagPartitionedLogSystem.actor.cpp @@ -415,7 +415,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted( Reference>>(), txsTag, begin, end, false, false); } @@ -1534,7 +1534,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted allTags) { - TraceEvent("RemoteLogRecruitment_WaitingForWorkers"); + TraceEvent("RemoteLogRecruitment_WaitingForWorkers").log(); state RecruitRemoteFromConfigurationReply remoteWorkers = wait(fRemoteWorkers); state Reference logSet(new LogSet()); @@ -2655,7 +2655,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCountedremoteRecoveryComplete = waitForAll(recoveryComplete); self->tLogs.push_back(logSet); - TraceEvent("RemoteLogRecruitment_CompletingRecovery"); + TraceEvent("RemoteLogRecruitment_CompletingRecovery").log(); return Void(); } @@ -3149,7 +3149,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted { reportLiveCommittedVersionRequests("ReportLiveCommittedVersionRequests", cc) { logger = traceCounters("MasterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "MasterMetrics"); if (forceRecovery && !myInterface.locality.dcId().present()) { - TraceEvent(SevError, "ForcedRecoveryRequiresDcID"); + TraceEvent(SevError, "ForcedRecoveryRequiresDcID").log(); forceRecovery = false; } } @@ -904,7 +904,7 @@ ACTOR Future readTransactionSystemState(Reference self, // make KeyValueStoreMemory guarantee immediate reads, we should be able to get rid of // the discardCommit() below and not need a writable log adapter - TraceEvent("RTSSComplete", self->dbgid); + TraceEvent("RTSSComplete", self->dbgid).log(); return Void(); } @@ -1087,7 +1087,7 @@ ACTOR Future recoverFrom(Reference self, when(Standalone _req = wait(provisional)) { state Standalone req = _req; // mutable TEST(true); // Emergency transaction processing during recovery - TraceEvent("EmergencyTransaction", self->dbgid); + TraceEvent("EmergencyTransaction", self->dbgid).log(); for (auto m = req.mutations.begin(); m != req.mutations.end(); ++m) TraceEvent("EmergencyTransactionMutation", self->dbgid) .detail("MType", m->type) @@ -1102,7 +1102,7 @@ ACTOR Future recoverFrom(Reference self, initialConfChanges->clear(); if (self->originalConfiguration.isValid() && self->configuration.usableRegions != self->originalConfiguration.usableRegions) { - TraceEvent(SevWarnAlways, "CannotChangeUsableRegions", self->dbgid); + TraceEvent(SevWarnAlways, "CannotChangeUsableRegions", self->dbgid).log(); self->configuration = self->originalConfiguration; } else { initialConfChanges->push_back(req); @@ -1500,7 +1500,7 @@ ACTOR Future trackTlogRecovery(Reference self, if (newState.oldTLogData.size() && configuration.repopulateRegionAntiQuorum > 0 && self->logSystem->remoteStorageRecovered()) { - TraceEvent(SevWarnAlways, "RecruitmentStalled_RemoteStorageRecovered", self->dbgid); + TraceEvent(SevWarnAlways, "RecruitmentStalled_RemoteStorageRecovered", self->dbgid).log(); self->recruitmentStalled->set(true); } self->registrationTrigger.trigger(); @@ -1570,7 +1570,7 @@ ACTOR static Future> getMinBackupVersion(Reference minVersion = minVersion.present() ? std::min(version, minVersion.get()) : version; } } else { - TraceEvent("EmptyBackupStartKey", self->dbgid); + TraceEvent("EmptyBackupStartKey", self->dbgid).log(); } return minVersion; @@ -1663,7 +1663,7 @@ ACTOR static Future recruitBackupWorkers(Reference self, Datab std::vector newRecruits = wait(getAll(initializationReplies)); self->logSystem->setBackupWorkers(newRecruits); - TraceEvent("BackupRecruitmentDone", self->dbgid); + TraceEvent("BackupRecruitmentDone", self->dbgid).log(); self->registrationTrigger.trigger(); return Void(); } @@ -1723,7 +1723,7 @@ ACTOR Future masterCore(Reference self) { if (g_network->isSimulated() && self->cstate.myDBState.oldTLogData.size() > CLIENT_KNOBS->MAX_GENERATIONS_SIM) { g_simulator.connectionFailuresDisableDuration = 1e6; g_simulator.speedUpSimulation = true; - TraceEvent(SevWarnAlways, "DisableConnectionFailures_TooManyGenerations"); + TraceEvent(SevWarnAlways, "DisableConnectionFailures_TooManyGenerations").log(); } } @@ -1812,7 +1812,7 @@ ACTOR Future masterCore(Reference self) { tr.set(recoveryCommitRequest.arena, snapshotEndVersionKey, (bw << self->lastEpochEnd).toValue()); // Pause the backups that got restored in this snapshot to avoid data corruption // Requires further operational work to abort the backup - TraceEvent("MasterRecoveryPauseBackupAgents"); + TraceEvent("MasterRecoveryPauseBackupAgents").log(); Key backupPauseKey = FileBackupAgent::getPauseKey(); tr.set(recoveryCommitRequest.arena, backupPauseKey, StringRef()); // Clear the key so multiple recoveries will not overwrite the first version recorded @@ -1882,7 +1882,7 @@ ACTOR Future masterCore(Reference self) { tr.read_snapshot = self->recoveryTransactionVersion; // lastEpochEnd would make more sense, but isn't in the initial // window of the resolver(s) - TraceEvent("MasterRecoveryCommit", self->dbgid); + TraceEvent("MasterRecoveryCommit", self->dbgid).log(); state Future> recoveryCommit = self->commitProxies[0].commit.tryGetReply(recoveryCommitRequest); self->addActor.send(self->logSystem->onError()); self->addActor.send(waitResolverFailure(self->resolvers)); @@ -1930,7 +1930,7 @@ ACTOR Future masterCore(Reference self) { debug_advanceMinCommittedVersion(UID(), self->recoveryTransactionVersion); if (debugResult) { - TraceEvent(self->forceRecovery ? SevWarn : SevError, "DBRecoveryDurabilityError"); + TraceEvent(self->forceRecovery ? SevWarn : SevError, "DBRecoveryDurabilityError").log(); } TraceEvent("MasterCommittedTLogs", self->dbgid) diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index ffd0ee4f40..ae9a37c540 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1189,7 +1189,7 @@ Future waitForVersion(StorageServer* data, Version version, SpanID span } if (deterministicRandom()->random01() < 0.001) { - TraceEvent("WaitForVersion1000x"); + TraceEvent("WaitForVersion1000x").log(); } return waitForVersionActor(data, version, spanContext); } @@ -3542,10 +3542,10 @@ private: ASSERT(ssId == data->thisServerID); if (m.type == MutationRef::SetValue) { TEST(true); // Putting TSS in quarantine - TraceEvent(SevWarn, "TSSQuarantineStart", data->thisServerID); + TraceEvent(SevWarn, "TSSQuarantineStart", data->thisServerID).log(); data->startTssQuarantine(); } else { - TraceEvent(SevWarn, "TSSQuarantineStop", data->thisServerID); + TraceEvent(SevWarn, "TSSQuarantineStop", data->thisServerID).log(); // dipose of this TSS throw worker_removed(); } @@ -3620,7 +3620,7 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { !g_simulator.speedUpSimulation && data->tssFaultInjectTime.present() && data->tssFaultInjectTime.get() < now()) { if (deterministicRandom()->random01() < 0.01) { - TraceEvent(SevWarnAlways, "TSSInjectDelayForever", data->thisServerID); + TraceEvent(SevWarnAlways, "TSSInjectDelayForever", data->thisServerID).log(); // small random chance to just completely get stuck here, each tss should eventually hit this in this // mode wait(tssDelayForever()); @@ -3835,7 +3835,7 @@ ACTOR Future update(StorageServer* data, bool* pReceivedUpdate) { } else if (ver != invalidVersion) { // This change belongs to a version < minVersion DEBUG_MUTATION("SSPeek", ver, msg).detail("ServerID", data->thisServerID); if (ver == 1) { - TraceEvent("SSPeekMutation", data->thisServerID); + TraceEvent("SSPeekMutation", data->thisServerID).log(); // The following trace event may produce a value with special characters //TraceEvent("SSPeekMutation", data->thisServerID).detail("Mutation", msg.toString()).detail("Version", cloneCursor2->version().toString()); } @@ -4333,15 +4333,15 @@ ACTOR Future restoreDurableState(StorageServer* data, IKeyValueStore* stor data->byteSampleRecovery = restoreByteSample(data, storage, byteSampleSampleRecovered, startByteSampleRestore.getFuture()); - TraceEvent("ReadingDurableState", data->thisServerID); + TraceEvent("ReadingDurableState", data->thisServerID).log(); wait(waitForAll(std::vector{ fFormat, fID, ftssPairID, fTssQuarantine, fVersion, fLogProtocol, fPrimaryLocality })); wait(waitForAll(std::vector{ fShardAssigned, fShardAvailable })); wait(byteSampleSampleRecovered.getFuture()); - TraceEvent("RestoringDurableState", data->thisServerID); + TraceEvent("RestoringDurableState", data->thisServerID).log(); if (!fFormat.get().present()) { // The DB was never initialized - TraceEvent("DBNeverInitialized", data->thisServerID); + TraceEvent("DBNeverInitialized", data->thisServerID).log(); storage->dispose(); data->thisServerID = UID(); data->sk = Key(); @@ -5262,7 +5262,7 @@ ACTOR Future replaceInterface(StorageServer* self, StorageServerInterface } if (self->history.size() && BUGGIFY) { - TraceEvent("SSHistoryReboot", self->thisServerID); + TraceEvent("SSHistoryReboot", self->thisServerID).log(); throw please_reboot(); } @@ -5337,7 +5337,7 @@ ACTOR Future storageServer(IKeyValueStore* persistentData, try { state double start = now(); - TraceEvent("StorageServerRebootStart", self.thisServerID); + TraceEvent("StorageServerRebootStart", self.thisServerID).log(); wait(self.storage.init()); choose { @@ -5346,7 +5346,7 @@ ACTOR Future storageServer(IKeyValueStore* persistentData, when(wait(self.storage.commit())) {} when(wait(memoryStoreRecover(persistentData, connFile, self.thisServerID))) { - TraceEvent("DisposeStorageServer", self.thisServerID); + TraceEvent("DisposeStorageServer", self.thisServerID).log(); throw worker_removed(); } } diff --git a/fdbserver/tester.actor.cpp b/fdbserver/tester.actor.cpp index 44826b4590..e819267bb1 100644 --- a/fdbserver/tester.actor.cpp +++ b/fdbserver/tester.actor.cpp @@ -817,7 +817,7 @@ ACTOR Future runWorkload(Database cx, std::vector>> checks; - TraceEvent("CheckingResults"); + TraceEvent("CheckingResults").log(); printf("checking test (%s)...\n", printable(spec.title).c_str()); @@ -1016,7 +1016,7 @@ ACTOR Future runTest(Database cx, if (spec.useDB && spec.clearAfterTest) { try { - TraceEvent("TesterClearingDatabase"); + TraceEvent("TesterClearingDatabase").log(); wait(timeoutError(clearData(cx), 1000.0)); } catch (Error& e) { TraceEvent(SevError, "ErrorClearingDatabaseAfterTest").error(e); @@ -1559,7 +1559,7 @@ ACTOR Future runTests(ReferenceonChange())) {} when(wait(testerTimeout)) { - TraceEvent(SevError, "TesterRecruitmentTimeout"); + TraceEvent(SevError, "TesterRecruitmentTimeout").log(); throw timed_out(); } } diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 28b2398bc1..70dbd39c7c 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -848,7 +848,7 @@ bool checkHighMemory(int64_t threshold, bool* error) { uint64_t page_size = sysconf(_SC_PAGESIZE); int fd = open("/proc/self/statm", O_RDONLY | O_CLOEXEC); if (fd < 0) { - TraceEvent("OpenStatmFileFailure"); + TraceEvent("OpenStatmFileFailure").log(); *error = true; return false; } @@ -857,7 +857,7 @@ bool checkHighMemory(int64_t threshold, bool* error) { char stat_buf[buf_sz]; ssize_t stat_nread = read(fd, stat_buf, buf_sz); if (stat_nread < 0) { - TraceEvent("ReadStatmFileFailure"); + TraceEvent("ReadStatmFileFailure").log(); *error = true; return false; } @@ -869,7 +869,7 @@ bool checkHighMemory(int64_t threshold, bool* error) { return true; } #else - TraceEvent("CheckHighMemoryUnsupported"); + TraceEvent("CheckHighMemoryUnsupported").log(); *error = true; #endif return false; @@ -926,7 +926,7 @@ ACTOR Future storageServerRollbackRebooter(std::set storageCacheRollbackRebooter(Future prevStorageCache, loop { ErrorOr e = wait(errorOr(prevStorageCache)); if (!e.isError()) { - TraceEvent("StorageCacheRequestedReboot1", id); + TraceEvent("StorageCacheRequestedReboot1", id).log(); return Void(); } else if (e.getError().code() != error_code_please_reboot && e.getError().code() != error_code_worker_removed) { @@ -972,7 +972,7 @@ ACTOR Future storageCacheRollbackRebooter(Future prevStorageCache, throw e.getError(); } - TraceEvent("StorageCacheRequestedReboot", id); + TraceEvent("StorageCacheRequestedReboot", id).log(); StorageServerInterface recruited; recruited.uniqueID = deterministicRandom()->randomUniqueID(); // id; @@ -1504,7 +1504,7 @@ ACTOR Future workerServer(Reference connFile, } throw please_reboot(); } else { - TraceEvent("ProcessReboot"); + TraceEvent("ProcessReboot").log(); ASSERT(!rebootReq.deleteData); flushAndExit(0); } @@ -2017,7 +2017,7 @@ ACTOR Future printOnFirstConnected(Referenceget().get().openDatabase.getEndpoint(), FailureStatus(false)) : Never())) { printf("FDBD joined cluster.\n"); - TraceEvent("FDBDConnected"); + TraceEvent("FDBDConnected").log(); return Void(); } when(wait(ci->onChange())) {} diff --git a/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp b/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp index 38c86a24a0..846c598798 100644 --- a/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp +++ b/fdbserver/workloads/AtomicOpsApiCorrectness.actor.cpp @@ -480,7 +480,7 @@ public: TraceEvent("AtomicOpCorrectnessApiWorkload").detail("OpType", "MIN"); // API Version 500 setApiVersion(&cx, 500); - TraceEvent(SevInfo, "Running Atomic Op Min Correctness Test Api Version 500"); + TraceEvent(SevInfo, "Running Atomic Op Min Correctness Test Api Version 500").log(); wait(self->testAtomicOpUnsetOnNonExistingKey(cx, self, MutationRef::Min, key)); wait(self->testAtomicOpApi( cx, self, MutationRef::Min, key, [](uint64_t val1, uint64_t val2) { return val1 < val2 ? val1 : val2; })); @@ -513,7 +513,7 @@ public: ACTOR Future testMax(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_max_"); - TraceEvent(SevInfo, "Running Atomic Op MAX Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op MAX Correctness Current Api Version").log(); wait(self->testAtomicOpSetOnNonExistingKey(cx, self, MutationRef::Max, key)); wait(self->testAtomicOpApi( cx, self, MutationRef::Max, key, [](uint64_t val1, uint64_t val2) { return val1 > val2 ? val1 : val2; })); @@ -530,7 +530,7 @@ public: TraceEvent("AtomicOpCorrectnessApiWorkload").detail("OpType", "AND"); // API Version 500 setApiVersion(&cx, 500); - TraceEvent(SevInfo, "Running Atomic Op AND Correctness Test Api Version 500"); + TraceEvent(SevInfo, "Running Atomic Op AND Correctness Test Api Version 500").log(); wait(self->testAtomicOpUnsetOnNonExistingKey(cx, self, MutationRef::And, key)); wait(self->testAtomicOpApi( cx, self, MutationRef::And, key, [](uint64_t val1, uint64_t val2) { return val1 & val2; })); @@ -563,7 +563,7 @@ public: ACTOR Future testOr(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_or_"); - TraceEvent(SevInfo, "Running Atomic Op OR Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op OR Correctness Current Api Version").log(); wait(self->testAtomicOpSetOnNonExistingKey(cx, self, MutationRef::Or, key)); wait(self->testAtomicOpApi( cx, self, MutationRef::Or, key, [](uint64_t val1, uint64_t val2) { return val1 | val2; })); @@ -576,7 +576,7 @@ public: ACTOR Future testXor(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_xor_"); - TraceEvent(SevInfo, "Running Atomic Op XOR Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op XOR Correctness Current Api Version").log(); wait(self->testAtomicOpSetOnNonExistingKey(cx, self, MutationRef::Xor, key)); wait(self->testAtomicOpApi( cx, self, MutationRef::Xor, key, [](uint64_t val1, uint64_t val2) { return val1 ^ val2; })); @@ -588,7 +588,7 @@ public: ACTOR Future testAdd(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_add_"); - TraceEvent(SevInfo, "Running Atomic Op ADD Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op ADD Correctness Current Api Version").log(); wait(self->testAtomicOpSetOnNonExistingKey(cx, self, MutationRef::AddValue, key)); wait(self->testAtomicOpApi( cx, self, MutationRef::AddValue, key, [](uint64_t val1, uint64_t val2) { return val1 + val2; })); @@ -601,7 +601,7 @@ public: ACTOR Future testCompareAndClear(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_compare_and_clear_"); - TraceEvent(SevInfo, "Running Atomic Op COMPARE_AND_CLEAR Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op COMPARE_AND_CLEAR Correctness Current Api Version").log(); wait(self->testCompareAndClearAtomicOpApi(cx, self, key, true)); wait(self->testCompareAndClearAtomicOpApi(cx, self, key, false)); return Void(); @@ -610,7 +610,7 @@ public: ACTOR Future testByteMin(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_byte_min_"); - TraceEvent(SevInfo, "Running Atomic Op BYTE_MIN Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op BYTE_MIN Correctness Current Api Version").log(); wait(self->testAtomicOpSetOnNonExistingKey(cx, self, MutationRef::ByteMin, key)); wait(self->testAtomicOpApi(cx, self, MutationRef::ByteMin, key, [](uint64_t val1, uint64_t val2) { return StringRef((const uint8_t*)&val1, sizeof(val1)) < StringRef((const uint8_t*)&val2, sizeof(val2)) @@ -626,7 +626,7 @@ public: ACTOR Future testByteMax(Database cx, AtomicOpsApiCorrectnessWorkload* self) { state Key key = self->getTestKey("test_key_byte_max_"); - TraceEvent(SevInfo, "Running Atomic Op BYTE_MAX Correctness Current Api Version"); + TraceEvent(SevInfo, "Running Atomic Op BYTE_MAX Correctness Current Api Version").log(); wait(self->testAtomicOpSetOnNonExistingKey(cx, self, MutationRef::ByteMax, key)); wait(self->testAtomicOpApi(cx, self, MutationRef::ByteMax, key, [](uint64_t val1, uint64_t val2) { return StringRef((const uint8_t*)&val1, sizeof(val1)) > StringRef((const uint8_t*)&val2, sizeof(val2)) diff --git a/fdbserver/workloads/AtomicRestore.actor.cpp b/fdbserver/workloads/AtomicRestore.actor.cpp index 66e35105ab..6035a042f8 100644 --- a/fdbserver/workloads/AtomicRestore.actor.cpp +++ b/fdbserver/workloads/AtomicRestore.actor.cpp @@ -104,14 +104,14 @@ struct AtomicRestoreWorkload : TestWorkload { throw; } - TraceEvent("AtomicRestore_Wait"); + TraceEvent("AtomicRestore_Wait").log(); wait(success(backupAgent.waitBackup(cx, BackupAgentBase::getDefaultTagName(), StopWhenDone::False))); - TraceEvent("AtomicRestore_BackupStart"); + TraceEvent("AtomicRestore_BackupStart").log(); wait(delay(self->restoreAfter * deterministicRandom()->random01())); - TraceEvent("AtomicRestore_RestoreStart"); + TraceEvent("AtomicRestore_RestoreStart").log(); if (self->fastRestore) { // New fast parallel restore - TraceEvent(SevInfo, "AtomicParallelRestore"); + TraceEvent(SevInfo, "AtomicParallelRestore").log(); wait(backupAgent.atomicParallelRestore( cx, BackupAgentBase::getDefaultTag(), self->backupRanges, self->addPrefix, self->removePrefix)); } else { // Old style restore @@ -141,7 +141,7 @@ struct AtomicRestoreWorkload : TestWorkload { g_simulator.backupAgents = ISimulator::BackupAgentType::NoBackupAgents; } - TraceEvent("AtomicRestore_Done"); + TraceEvent("AtomicRestore_Done").log(); return Void(); } }; diff --git a/fdbserver/workloads/AtomicSwitchover.actor.cpp b/fdbserver/workloads/AtomicSwitchover.actor.cpp index 3bb93fb8bf..c20570346b 100644 --- a/fdbserver/workloads/AtomicSwitchover.actor.cpp +++ b/fdbserver/workloads/AtomicSwitchover.actor.cpp @@ -53,7 +53,7 @@ struct AtomicSwitchoverWorkload : TestWorkload { ACTOR static Future _setup(Database cx, AtomicSwitchoverWorkload* self) { state DatabaseBackupAgent backupAgent(cx); try { - TraceEvent("AS_Submit1"); + TraceEvent("AS_Submit1").log(); wait(backupAgent.submitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), self->backupRanges, @@ -61,7 +61,7 @@ struct AtomicSwitchoverWorkload : TestWorkload { StringRef(), StringRef(), LockDB::True)); - TraceEvent("AS_Submit2"); + TraceEvent("AS_Submit2").log(); } catch (Error& e) { if (e.code() != error_code_backup_duplicate) throw; @@ -167,27 +167,27 @@ struct AtomicSwitchoverWorkload : TestWorkload { state DatabaseBackupAgent backupAgent(cx); state DatabaseBackupAgent restoreTool(self->extraDB); - TraceEvent("AS_Wait1"); + TraceEvent("AS_Wait1").log(); wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); - TraceEvent("AS_Ready1"); + TraceEvent("AS_Ready1").log(); wait(delay(deterministicRandom()->random01() * self->switch1delay)); - TraceEvent("AS_Switch1"); + TraceEvent("AS_Switch1").log(); wait(backupAgent.atomicSwitchover( self->extraDB, BackupAgentBase::getDefaultTag(), self->backupRanges, StringRef(), StringRef())); - TraceEvent("AS_Wait2"); + TraceEvent("AS_Wait2").log(); wait(success(restoreTool.waitBackup(cx, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); - TraceEvent("AS_Ready2"); + TraceEvent("AS_Ready2").log(); wait(delay(deterministicRandom()->random01() * self->switch2delay)); - TraceEvent("AS_Switch2"); + TraceEvent("AS_Switch2").log(); wait(restoreTool.atomicSwitchover( cx, BackupAgentBase::getDefaultTag(), self->backupRanges, StringRef(), StringRef())); - TraceEvent("AS_Wait3"); + TraceEvent("AS_Wait3").log(); wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); - TraceEvent("AS_Ready3"); + TraceEvent("AS_Ready3").log(); wait(delay(deterministicRandom()->random01() * self->stopDelay)); - TraceEvent("AS_Abort"); + TraceEvent("AS_Abort").log(); wait(backupAgent.abortBackup(self->extraDB, BackupAgentBase::getDefaultTag())); - TraceEvent("AS_Done"); + TraceEvent("AS_Done").log(); // SOMEDAY: Remove after backup agents can exist quiescently if (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) { diff --git a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp index b465e49939..a9113cbff0 100644 --- a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp @@ -384,7 +384,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { Key(), Key(), self->locked))); - TraceEvent(SevError, "BARW_RestoreAllowedOverwrittingDatabase", randomID); + TraceEvent(SevError, "BARW_RestoreAllowedOverwrittingDatabase", randomID).log(); ASSERT(false); } catch (Error& e) { if (e.code() != error_code_restore_destination_not_empty) { diff --git a/fdbserver/workloads/BackupCorrectness.actor.cpp b/fdbserver/workloads/BackupCorrectness.actor.cpp index d655342661..d28bd59ae2 100644 --- a/fdbserver/workloads/BackupCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupCorrectness.actor.cpp @@ -430,7 +430,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { Key(), Key(), self->locked))); - TraceEvent(SevError, "BARW_RestoreAllowedOverwrittingDatabase", randomID); + TraceEvent(SevError, "BARW_RestoreAllowedOverwrittingDatabase", randomID).log(); ASSERT(false); } catch (Error& e) { if (e.code() != error_code_restore_destination_not_empty) { diff --git a/fdbserver/workloads/BackupToDBAbort.actor.cpp b/fdbserver/workloads/BackupToDBAbort.actor.cpp index d2d5c3d02d..ab084ea121 100644 --- a/fdbserver/workloads/BackupToDBAbort.actor.cpp +++ b/fdbserver/workloads/BackupToDBAbort.actor.cpp @@ -52,7 +52,7 @@ struct BackupToDBAbort : TestWorkload { ACTOR static Future _setup(BackupToDBAbort* self, Database cx) { state DatabaseBackupAgent backupAgent(cx); try { - TraceEvent("BDBA_Submit1"); + TraceEvent("BDBA_Submit1").log(); wait(backupAgent.submitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), self->backupRanges, @@ -60,7 +60,7 @@ struct BackupToDBAbort : TestWorkload { StringRef(), StringRef(), LockDB::True)); - TraceEvent("BDBA_Submit2"); + TraceEvent("BDBA_Submit2").log(); } catch (Error& e) { if (e.code() != error_code_backup_duplicate) throw; @@ -79,15 +79,15 @@ struct BackupToDBAbort : TestWorkload { TraceEvent("BDBA_Start").detail("Delay", self->abortDelay); wait(delay(self->abortDelay)); - TraceEvent("BDBA_Wait"); + TraceEvent("BDBA_Wait").log(); wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); - TraceEvent("BDBA_Lock"); + TraceEvent("BDBA_Lock").log(); wait(lockDatabase(cx, self->lockid)); - TraceEvent("BDBA_Abort"); + TraceEvent("BDBA_Abort").log(); wait(backupAgent.abortBackup(self->extraDB, BackupAgentBase::getDefaultTag())); - TraceEvent("BDBA_Unlock"); + TraceEvent("BDBA_Unlock").log(); wait(backupAgent.unlockBackup(self->extraDB, BackupAgentBase::getDefaultTag())); - TraceEvent("BDBA_End"); + TraceEvent("BDBA_End").log(); // SOMEDAY: Remove after backup agents can exist quiescently if (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) { @@ -98,7 +98,7 @@ struct BackupToDBAbort : TestWorkload { } ACTOR static Future _check(BackupToDBAbort* self, Database cx) { - TraceEvent("BDBA_UnlockPrimary"); + TraceEvent("BDBA_UnlockPrimary").log(); // Too much of the tester framework expects the primary database to be unlocked, so we unlock it // once all of the workloads have finished. wait(unlockDatabase(cx, self->lockid)); diff --git a/fdbserver/workloads/BackupToDBUpgrade.actor.cpp b/fdbserver/workloads/BackupToDBUpgrade.actor.cpp index e0553058ed..5a2a34e9e7 100644 --- a/fdbserver/workloads/BackupToDBUpgrade.actor.cpp +++ b/fdbserver/workloads/BackupToDBUpgrade.actor.cpp @@ -78,7 +78,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload { auto extraFile = makeReference(*g_simulator.extraDB); extraDB = Database::createDatabase(extraFile, -1); - TraceEvent("DRU_Start"); + TraceEvent("DRU_Start").log(); } std::string description() const override { return "BackupToDBUpgrade"; } @@ -459,7 +459,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload { } } - TraceEvent("DRU_DiffRanges"); + TraceEvent("DRU_DiffRanges").log(); wait(diffRanges(prevBackupRanges, self->backupPrefix, cx, self->extraDB)); // abort backup diff --git a/fdbserver/workloads/BulkSetup.actor.h b/fdbserver/workloads/BulkSetup.actor.h index e5dc75e28f..027b8c8e57 100644 --- a/fdbserver/workloads/BulkSetup.actor.h +++ b/fdbserver/workloads/BulkSetup.actor.h @@ -284,7 +284,7 @@ Future bulkSetup(Database cx, wait(delay(1.0)); } else { wait(delay(1.0)); - TraceEvent("DynamicWarmingDone"); + TraceEvent("DynamicWarmingDone").log(); break; } } diff --git a/fdbserver/workloads/ChangeConfig.actor.cpp b/fdbserver/workloads/ChangeConfig.actor.cpp index b891367c64..44d797d00c 100644 --- a/fdbserver/workloads/ChangeConfig.actor.cpp +++ b/fdbserver/workloads/ChangeConfig.actor.cpp @@ -65,9 +65,9 @@ struct ChangeConfigWorkload : TestWorkload { // It is not safe to allow automatic failover to a region which is not fully replicated, // so wait for both regions to be fully replicated before enabling failover wait(success(changeConfig(extraDB, g_simulator.startingDisabledConfiguration, true))); - TraceEvent("WaitForReplicasExtra"); + TraceEvent("WaitForReplicasExtra").log(); wait(waitForFullReplication(extraDB)); - TraceEvent("WaitForReplicasExtraEnd"); + TraceEvent("WaitForReplicasExtraEnd").log(); } wait(success(changeConfig(extraDB, self->configMode, true))); } @@ -99,9 +99,9 @@ struct ChangeConfigWorkload : TestWorkload { // It is not safe to allow automatic failover to a region which is not fully replicated, // so wait for both regions to be fully replicated before enabling failover wait(success(changeConfig(cx, g_simulator.startingDisabledConfiguration, true))); - TraceEvent("WaitForReplicas"); + TraceEvent("WaitForReplicas").log(); wait(waitForFullReplication(cx)); - TraceEvent("WaitForReplicasEnd"); + TraceEvent("WaitForReplicasEnd").log(); } wait(success(changeConfig(cx, self->configMode, true))); } diff --git a/fdbserver/workloads/ConflictRange.actor.cpp b/fdbserver/workloads/ConflictRange.actor.cpp index 061a93289e..4ec8c12f60 100644 --- a/fdbserver/workloads/ConflictRange.actor.cpp +++ b/fdbserver/workloads/ConflictRange.actor.cpp @@ -100,7 +100,7 @@ struct ConflictRangeWorkload : TestWorkload { loop { state Transaction tr0(cx); try { - TraceEvent("ConflictRangeReset"); + TraceEvent("ConflictRangeReset").log(); insertedSet.clear(); if (self->testReadYourWrites) { diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 7ebe0b0052..0d4783671a 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -142,7 +142,7 @@ struct ConsistencyCheckWorkload : TestWorkload { } Future start(Database const& cx) override { - TraceEvent("ConsistencyCheck"); + TraceEvent("ConsistencyCheck").log(); return _start(cx, this); } @@ -186,10 +186,10 @@ struct ConsistencyCheckWorkload : TestWorkload { ACTOR Future _start(Database cx, ConsistencyCheckWorkload* self) { loop { while (self->suspendConsistencyCheck.get()) { - TraceEvent("ConsistencyCheck_Suspended"); + TraceEvent("ConsistencyCheck_Suspended").log(); wait(self->suspendConsistencyCheck.onChange()); } - TraceEvent("ConsistencyCheck_StartingOrResuming"); + TraceEvent("ConsistencyCheck_StartingOrResuming").log(); choose { when(wait(self->runCheck(cx, self))) { if (!self->indefinite) @@ -222,7 +222,7 @@ struct ConsistencyCheckWorkload : TestWorkload { } RangeResult res = wait(tr.getRange(configKeys, 1000)); if (res.size() == 1000) { - TraceEvent("ConsistencyCheck_TooManyConfigOptions"); + TraceEvent("ConsistencyCheck_TooManyConfigOptions").log(); self->testFailure("Read too many configuration options"); } for (int i = 0; i < res.size(); i++) @@ -251,7 +251,7 @@ struct ConsistencyCheckWorkload : TestWorkload { // the allowed maximum number of teams bool teamCollectionValid = wait(getTeamCollectionValid(cx, self->dbInfo)); if (!teamCollectionValid) { - TraceEvent(SevError, "ConsistencyCheck_TooManyTeams"); + TraceEvent(SevError, "ConsistencyCheck_TooManyTeams").log(); self->testFailure("The number of process or machine teams is larger than the allowed maximum " "number of teams"); } @@ -1817,7 +1817,7 @@ struct ConsistencyCheckWorkload : TestWorkload { self->testFailure("No storage server on worker"); return false; } else { - TraceEvent(SevWarn, "ConsistencyCheck_TSSMissing"); + TraceEvent(SevWarn, "ConsistencyCheck_TSSMissing").log(); } } @@ -1992,7 +1992,7 @@ struct ConsistencyCheckWorkload : TestWorkload { Optional currentKey = wait(tr.get(coordinatorsKey)); if (!currentKey.present()) { - TraceEvent("ConsistencyCheck_NoCoordinatorKey"); + TraceEvent("ConsistencyCheck_NoCoordinatorKey").log(); return false; } diff --git a/fdbserver/workloads/CpuProfiler.actor.cpp b/fdbserver/workloads/CpuProfiler.actor.cpp index cf03639ef0..25716f79e0 100644 --- a/fdbserver/workloads/CpuProfiler.actor.cpp +++ b/fdbserver/workloads/CpuProfiler.actor.cpp @@ -93,7 +93,7 @@ struct CpuProfilerWorkload : TestWorkload { if (!replies[i].get().present()) self->success = false; - TraceEvent("DoneSignalingProfiler"); + TraceEvent("DoneSignalingProfiler").log(); } return Void(); @@ -104,14 +104,14 @@ struct CpuProfilerWorkload : TestWorkload { ACTOR Future _start(Database cx, CpuProfilerWorkload* self) { wait(delay(self->initialDelay)); if (self->clientId == 0) - TraceEvent("SignalProfilerOn"); + TraceEvent("SignalProfilerOn").log(); wait(timeoutError(self->updateProfiler(true, cx, self), 60.0)); // If a duration was given, let the duration elapse and then shut the profiler off if (self->duration > 0) { wait(delay(self->duration)); if (self->clientId == 0) - TraceEvent("SignalProfilerOff"); + TraceEvent("SignalProfilerOff").log(); wait(timeoutError(self->updateProfiler(false, cx, self), 60.0)); } @@ -124,7 +124,7 @@ struct CpuProfilerWorkload : TestWorkload { // If no duration was given, then shut the profiler off now if (self->duration <= 0) { if (self->clientId == 0) - TraceEvent("SignalProfilerOff"); + TraceEvent("SignalProfilerOff").log(); wait(timeoutError(self->updateProfiler(false, cx, self), 60.0)); } diff --git a/fdbserver/workloads/Cycle.actor.cpp b/fdbserver/workloads/Cycle.actor.cpp index c80b2348f7..4e4991fb95 100644 --- a/fdbserver/workloads/Cycle.actor.cpp +++ b/fdbserver/workloads/Cycle.actor.cpp @@ -104,7 +104,7 @@ struct CycleWorkload : TestWorkload { state Transaction tr(cx); if (deterministicRandom()->random01() >= self->traceParentProbability) { state Span span("CycleClient"_loc); - TraceEvent("CycleTracingTransaction", span.context); + TraceEvent("CycleTracingTransaction", span.context).log(); tr.setOption(FDBTransactionOptions::SPAN_PARENT, BinaryWriter::toValue(span.context, Unversioned())); } @@ -154,7 +154,7 @@ struct CycleWorkload : TestWorkload { } void logTestData(const VectorRef& data) { - TraceEvent("TestFailureDetail"); + TraceEvent("TestFailureDetail").log(); int index = 0; for (auto& entry : data) { TraceEvent("CurrentDataEntry") diff --git a/fdbserver/workloads/DDMetrics.actor.cpp b/fdbserver/workloads/DDMetrics.actor.cpp index 336fc294da..bb1326a350 100644 --- a/fdbserver/workloads/DDMetrics.actor.cpp +++ b/fdbserver/workloads/DDMetrics.actor.cpp @@ -50,7 +50,7 @@ struct DDMetricsWorkload : TestWorkload { try { TraceEvent("DDMetricsWaiting").detail("StartDelay", self->startDelay); wait(delay(self->startDelay)); - TraceEvent("DDMetricsStarting"); + TraceEvent("DDMetricsStarting").log(); state double startTime = now(); loop { wait(delay(2.5)); diff --git a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp index 1bcc0afc84..1cb3a85dc0 100644 --- a/fdbserver/workloads/DifferentClustersSameRV.actor.cpp +++ b/fdbserver/workloads/DifferentClustersSameRV.actor.cpp @@ -64,7 +64,7 @@ struct DifferentClustersSameRVWorkload : TestWorkload { Future check(Database const& cx) override { if (clientId == 0 && !switchComplete) { - TraceEvent(SevError, "DifferentClustersSwitchNotComplete"); + TraceEvent(SevError, "DifferentClustersSwitchNotComplete").log(); return false; } return true; @@ -133,17 +133,17 @@ struct DifferentClustersSameRVWorkload : TestWorkload { return Void(); })); wait(lockDatabase(self->originalDB, lockUid) && lockDatabase(self->extraDB, lockUid)); - TraceEvent("DifferentClusters_LockedDatabases"); + TraceEvent("DifferentClusters_LockedDatabases").log(); std::pair> read1 = wait(doRead(self->originalDB, self)); state Version rv = read1.first; state Optional val1 = read1.second; wait(doWrite(self->extraDB, self->keyToRead, val1)); - TraceEvent("DifferentClusters_CopiedDatabase"); + TraceEvent("DifferentClusters_CopiedDatabase").log(); wait(advanceVersion(self->extraDB, rv)); - TraceEvent("DifferentClusters_AdvancedVersion"); + TraceEvent("DifferentClusters_AdvancedVersion").log(); wait(cx->switchConnectionFile( makeReference(self->extraDB->getConnectionFile()->getConnectionString()))); - TraceEvent("DifferentClusters_SwitchedConnectionFile"); + TraceEvent("DifferentClusters_SwitchedConnectionFile").log(); state Transaction tr(cx); tr.setVersion(rv); tr.setOption(FDBTransactionOptions::READ_LOCK_AWARE); @@ -160,17 +160,17 @@ struct DifferentClustersSameRVWorkload : TestWorkload { // that a storage server serves a read at |rv| even after the recovery caused by unlocking the database, and we // want to make that more likely for this test. So read at |rv| then unlock. wait(unlockDatabase(self->extraDB, lockUid)); - TraceEvent("DifferentClusters_UnlockedExtraDB"); + TraceEvent("DifferentClusters_UnlockedExtraDB").log(); ASSERT(!watchFuture.isReady() || watchFuture.isError()); wait(doWrite(self->extraDB, self->keyToWatch, Optional{ LiteralStringRef("") })); - TraceEvent("DifferentClusters_WaitingForWatch"); + TraceEvent("DifferentClusters_WaitingForWatch").log(); try { wait(timeoutError(watchFuture, (self->testDuration - self->switchAfter) / 2)); } catch (Error& e) { TraceEvent("DifferentClusters_WatchError").error(e); wait(tr.onError(e)); } - TraceEvent("DifferentClusters_Done"); + TraceEvent("DifferentClusters_Done").log(); self->switchComplete = true; wait(unlockDatabase(self->originalDB, lockUid)); // So quietDatabase can finish return Void(); diff --git a/fdbserver/workloads/ExternalWorkload.actor.cpp b/fdbserver/workloads/ExternalWorkload.actor.cpp index 64b57c85f1..448823954f 100644 --- a/fdbserver/workloads/ExternalWorkload.actor.cpp +++ b/fdbserver/workloads/ExternalWorkload.actor.cpp @@ -142,19 +142,19 @@ struct ExternalWorkload : TestWorkload, FDBWorkloadContext { .detail("WorkloadName", wName); library = loadLibrary(fullPath.c_str()); if (library == nullptr) { - TraceEvent(SevError, "ExternalWorkloadLoadError"); + TraceEvent(SevError, "ExternalWorkloadLoadError").log(); success = false; return; } workloadFactory = reinterpret_cast(loadFunction(library, "workloadFactory")); if (workloadFactory == nullptr) { - TraceEvent(SevError, "ExternalFactoryNotFound"); + TraceEvent(SevError, "ExternalFactoryNotFound").log(); success = false; return; } workloadImpl = (*workloadFactory)(FDBLoggerImpl::instance())->create(wName.toString()); if (!workloadImpl) { - TraceEvent(SevError, "WorkloadNotFound"); + TraceEvent(SevError, "WorkloadNotFound").log(); success = false; } workloadImpl->init(this); diff --git a/fdbserver/workloads/HealthMetricsApi.actor.cpp b/fdbserver/workloads/HealthMetricsApi.actor.cpp index fff9c1ae91..6a32742902 100644 --- a/fdbserver/workloads/HealthMetricsApi.actor.cpp +++ b/fdbserver/workloads/HealthMetricsApi.actor.cpp @@ -75,7 +75,7 @@ struct HealthMetricsApiWorkload : TestWorkload { Future check(Database const& cx) override { if (healthMetricsStoppedUpdating) { - TraceEvent(SevError, "HealthMetricsStoppedUpdating"); + TraceEvent(SevError, "HealthMetricsStoppedUpdating").log(); return false; } bool correctHealthMetricsState = true; diff --git a/fdbserver/workloads/IncrementalBackup.actor.cpp b/fdbserver/workloads/IncrementalBackup.actor.cpp index c273be09be..b8a6b49857 100644 --- a/fdbserver/workloads/IncrementalBackup.actor.cpp +++ b/fdbserver/workloads/IncrementalBackup.actor.cpp @@ -92,11 +92,11 @@ struct IncrementalBackupWorkload : TestWorkload { } loop { // Wait for backup container to be created and avoid race condition - TraceEvent("IBackupWaitContainer"); + TraceEvent("IBackupWaitContainer").log(); wait(success(self->backupAgent.waitBackup( cx, self->tag.toString(), StopWhenDone::False, &backupContainer, &backupUID))); if (!backupContainer.isValid()) { - TraceEvent("IBackupCheckListContainersAttempt"); + TraceEvent("IBackupCheckListContainersAttempt").log(); state std::vector containers = wait(IBackupContainer::listContainers(self->backupDir.toString())); TraceEvent("IBackupCheckListContainersSuccess") @@ -132,7 +132,7 @@ struct IncrementalBackupWorkload : TestWorkload { } if (self->stopBackup) { try { - TraceEvent("IBackupDiscontinueBackup"); + TraceEvent("IBackupDiscontinueBackup").log(); wait(self->backupAgent.discontinueBackup(cx, self->tag)); } catch (Error& e) { TraceEvent("IBackupDiscontinueBackupException").error(e); @@ -148,7 +148,7 @@ struct IncrementalBackupWorkload : TestWorkload { if (self->submitOnly) { Standalone> backupRanges; backupRanges.push_back_deep(backupRanges.arena(), normalKeys); - TraceEvent("IBackupSubmitAttempt"); + TraceEvent("IBackupSubmitAttempt").log(); try { wait(self->backupAgent.submitBackup(cx, self->backupDir, @@ -165,7 +165,7 @@ struct IncrementalBackupWorkload : TestWorkload { throw; } } - TraceEvent("IBackupSubmitSuccess"); + TraceEvent("IBackupSubmitSuccess").log(); } if (self->restoreOnly) { if (self->clearBackupAgentKeys) { @@ -189,7 +189,7 @@ struct IncrementalBackupWorkload : TestWorkload { wait(success(self->backupAgent.waitBackup( cx, self->tag.toString(), StopWhenDone::False, &backupContainer, &backupUID))); if (self->checkBeginVersion) { - TraceEvent("IBackupReadSystemKeys"); + TraceEvent("IBackupReadSystemKeys").log(); state Reference tr(new ReadYourWritesTransaction(cx)); loop { try { @@ -201,7 +201,7 @@ struct IncrementalBackupWorkload : TestWorkload { .detail("WriteRecoveryValue", writeFlag.present() ? writeFlag.get().toString() : "N/A") .detail("EndVersionValue", versionValue.present() ? versionValue.get().toString() : "N/A"); if (!versionValue.present()) { - TraceEvent("IBackupCheckSpecialKeysFailure"); + TraceEvent("IBackupCheckSpecialKeysFailure").log(); // Snapshot failed to write to special keys, possibly due to snapshot itself failing throw key_not_found(); } @@ -217,7 +217,7 @@ struct IncrementalBackupWorkload : TestWorkload { } } } - TraceEvent("IBackupStartListContainersAttempt"); + TraceEvent("IBackupStartListContainersAttempt").log(); state std::vector containers = wait(IBackupContainer::listContainers(self->backupDir.toString())); TraceEvent("IBackupStartListContainersSuccess") @@ -239,7 +239,7 @@ struct IncrementalBackupWorkload : TestWorkload { OnlyApplyMutationLogs::True, InconsistentSnapshotOnly::False, beginVersion))); - TraceEvent("IBackupRestoreSuccess"); + TraceEvent("IBackupRestoreSuccess").log(); } return Void(); } diff --git a/fdbserver/workloads/KVStoreTest.actor.cpp b/fdbserver/workloads/KVStoreTest.actor.cpp index 9c100cfa36..19fc6033c7 100644 --- a/fdbserver/workloads/KVStoreTest.actor.cpp +++ b/fdbserver/workloads/KVStoreTest.actor.cpp @@ -115,7 +115,7 @@ struct KVTest { ~KVTest() { close(); } void close() { if (store) { - TraceEvent("KVTestDestroy"); + TraceEvent("KVTestDestroy").log(); if (dispose) store->dispose(); else @@ -373,7 +373,7 @@ ACTOR Future testKVStore(KVStoreTestWorkload* workload) { state Error err; // wait( delay(1) ); - TraceEvent("GO"); + TraceEvent("GO").log(); UID id = deterministicRandom()->randomUniqueID(); std::string fn = workload->filename.size() ? workload->filename : id.toString(); diff --git a/fdbserver/workloads/KillRegion.actor.cpp b/fdbserver/workloads/KillRegion.actor.cpp index 6b86277ec2..8da635e4f2 100644 --- a/fdbserver/workloads/KillRegion.actor.cpp +++ b/fdbserver/workloads/KillRegion.actor.cpp @@ -56,11 +56,11 @@ struct KillRegionWorkload : TestWorkload { void getMetrics(vector& m) override {} ACTOR static Future _setup(KillRegionWorkload* self, Database cx) { - TraceEvent("ForceRecovery_DisablePrimaryBegin"); + TraceEvent("ForceRecovery_DisablePrimaryBegin").log(); wait(success(changeConfig(cx, g_simulator.disablePrimary, true))); - TraceEvent("ForceRecovery_WaitForRemote"); + TraceEvent("ForceRecovery_WaitForRemote").log(); wait(waitForPrimaryDC(cx, LiteralStringRef("1"))); - TraceEvent("ForceRecovery_DisablePrimaryComplete"); + TraceEvent("ForceRecovery_DisablePrimaryComplete").log(); return Void(); } @@ -74,14 +74,14 @@ struct KillRegionWorkload : TestWorkload { ACTOR static Future killRegion(KillRegionWorkload* self, Database cx) { ASSERT(g_network->isSimulated()); if (deterministicRandom()->random01() < 0.5) { - TraceEvent("ForceRecovery_DisableRemoteBegin"); + TraceEvent("ForceRecovery_DisableRemoteBegin").log(); wait(success(changeConfig(cx, g_simulator.disableRemote, true))); - TraceEvent("ForceRecovery_WaitForPrimary"); + TraceEvent("ForceRecovery_WaitForPrimary").log(); wait(waitForPrimaryDC(cx, LiteralStringRef("0"))); - TraceEvent("ForceRecovery_DisableRemoteComplete"); + TraceEvent("ForceRecovery_DisableRemoteComplete").log(); wait(success(changeConfig(cx, g_simulator.originalRegions, true))); } - TraceEvent("ForceRecovery_Wait"); + TraceEvent("ForceRecovery_Wait").log(); wait(delay(deterministicRandom()->random01() * self->testDuration)); g_simulator.killDataCenter(LiteralStringRef("0"), @@ -97,11 +97,11 @@ struct KillRegionWorkload : TestWorkload { : ISimulator::RebootAndDelete, true); - TraceEvent("ForceRecovery_Begin"); + TraceEvent("ForceRecovery_Begin").log(); wait(forceRecovery(cx->getConnectionFile(), LiteralStringRef("1"))); - TraceEvent("ForceRecovery_UsableRegions"); + TraceEvent("ForceRecovery_UsableRegions").log(); DatabaseConfiguration conf = wait(getDatabaseConfiguration(cx)); @@ -119,7 +119,7 @@ struct KillRegionWorkload : TestWorkload { wait(success(changeConfig(cx, "usable_regions=1", true))); } - TraceEvent("ForceRecovery_Complete"); + TraceEvent("ForceRecovery_Complete").log(); return Void(); } diff --git a/fdbserver/workloads/LogMetrics.actor.cpp b/fdbserver/workloads/LogMetrics.actor.cpp index 502fc5f1d4..d63581ff56 100644 --- a/fdbserver/workloads/LogMetrics.actor.cpp +++ b/fdbserver/workloads/LogMetrics.actor.cpp @@ -54,7 +54,7 @@ struct LogMetricsWorkload : TestWorkload { state BinaryWriter br(Unversioned()); vector workers = wait(getWorkers(self->dbInfo)); // vector> replies; - TraceEvent("RateChangeTrigger"); + TraceEvent("RateChangeTrigger").log(); SetMetricsLogRateRequest req(rate); for (int i = 0; i < workers.size(); i++) { workers[i].interf.setMetricsRate.send(req); diff --git a/fdbserver/workloads/LowLatency.actor.cpp b/fdbserver/workloads/LowLatency.actor.cpp index 79855fa92e..adde4c3c79 100644 --- a/fdbserver/workloads/LowLatency.actor.cpp +++ b/fdbserver/workloads/LowLatency.actor.cpp @@ -77,7 +77,7 @@ struct LowLatencyWorkload : TestWorkload { ++self->operations; loop { try { - TraceEvent("StartLowLatencyTransaction"); + TraceEvent("StartLowLatencyTransaction").log(); tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); tr.setOption(FDBTransactionOptions::LOCK_AWARE); if (doCommit) { diff --git a/fdbserver/workloads/MachineAttrition.actor.cpp b/fdbserver/workloads/MachineAttrition.actor.cpp index e46c249c6d..44d6e4f581 100644 --- a/fdbserver/workloads/MachineAttrition.actor.cpp +++ b/fdbserver/workloads/MachineAttrition.actor.cpp @@ -39,18 +39,18 @@ static std::set const& normalAttritionErrors() { ACTOR Future ignoreSSFailuresForDuration(Database cx, double duration) { // duration doesn't matter since this won't timeout - TraceEvent("IgnoreSSFailureStart"); + TraceEvent("IgnoreSSFailureStart").log(); wait(success(setHealthyZone(cx, ignoreSSFailuresZoneString, 0))); - TraceEvent("IgnoreSSFailureWait"); + TraceEvent("IgnoreSSFailureWait").log(); wait(delay(duration)); - TraceEvent("IgnoreSSFailureClear"); + TraceEvent("IgnoreSSFailureClear").log(); state Transaction tr(cx); loop { try { tr.setOption(FDBTransactionOptions::LOCK_AWARE); tr.clear(healthyZoneKey); wait(tr.commit()); - TraceEvent("IgnoreSSFailureComplete"); + TraceEvent("IgnoreSSFailureComplete").log(); return true; } catch (Error& e) { wait(tr.onError(e)); @@ -311,7 +311,7 @@ struct MachineAttritionWorkload : TestWorkload { TEST(true); // Killing a machine wait(delay(delayBeforeKill)); - TraceEvent("WorkerKillAfterDelay"); + TraceEvent("WorkerKillAfterDelay").log(); if (self->waitForVersion) { state Transaction tr(cx); diff --git a/fdbserver/workloads/ParallelRestore.actor.cpp b/fdbserver/workloads/ParallelRestore.actor.cpp index d6e39f37da..9aa112fc83 100644 --- a/fdbserver/workloads/ParallelRestore.actor.cpp +++ b/fdbserver/workloads/ParallelRestore.actor.cpp @@ -30,7 +30,7 @@ struct RunRestoreWorkerWorkload : TestWorkload { Future worker; RunRestoreWorkerWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) { - TraceEvent("RunRestoreWorkerWorkloadMX"); + TraceEvent("RunRestoreWorkerWorkloadMX").log(); } std::string description() const override { return "RunRestoreWorkerWorkload"; } diff --git a/fdbserver/workloads/Ping.actor.cpp b/fdbserver/workloads/Ping.actor.cpp index c10090a113..ae54ca07bd 100644 --- a/fdbserver/workloads/Ping.actor.cpp +++ b/fdbserver/workloads/Ping.actor.cpp @@ -259,7 +259,7 @@ struct PingWorkload : TestWorkload { // peers[i].payloadPing.getEndpoint().getPrimaryAddress(), pingId ) ); peers[i].payloadPing.send( req ); // replies.push_back( self->payloadDelayer( req, peers[i].payloadPing ) ); } - TraceEvent("PayloadPingSent", pingId); + TraceEvent("PayloadPingSent", pingId).log(); wait(waitForAll(replies)); double elapsed = now() - start; TraceEvent("PayloadPingDone", pingId).detail("Elapsed", elapsed); diff --git a/fdbserver/workloads/PopulateTPCC.actor.cpp b/fdbserver/workloads/PopulateTPCC.actor.cpp index 25a1ccc93f..63b769a61a 100644 --- a/fdbserver/workloads/PopulateTPCC.actor.cpp +++ b/fdbserver/workloads/PopulateTPCC.actor.cpp @@ -184,7 +184,7 @@ struct PopulateTPCC : TestWorkload { } } } - TraceEvent("PopulateItemsDone"); + TraceEvent("PopulateItemsDone").log(); return Void(); } diff --git a/fdbserver/workloads/RandomMoveKeys.actor.cpp b/fdbserver/workloads/RandomMoveKeys.actor.cpp index 887c6da897..967848e024 100644 --- a/fdbserver/workloads/RandomMoveKeys.actor.cpp +++ b/fdbserver/workloads/RandomMoveKeys.actor.cpp @@ -62,13 +62,13 @@ struct MoveKeysWorkload : TestWorkload { } state int oldMode = wait(setDDMode(cx, 0)); - TraceEvent("RMKStartModeSetting"); + TraceEvent("RMKStartModeSetting").log(); wait(timeout( reportErrors(self->worker(cx, self), "MoveKeysWorkloadWorkerError"), self->testDuration, Void())); // Always set the DD mode back, even if we die with an error - TraceEvent("RMKDoneMoving"); + TraceEvent("RMKDoneMoving").log(); wait(success(setDDMode(cx, oldMode))); - TraceEvent("RMKDoneModeSetting"); + TraceEvent("RMKDoneModeSetting").log(); } return Void(); } @@ -87,7 +87,7 @@ struct MoveKeysWorkload : TestWorkload { vector getRandomTeam(vector storageServers, int teamSize) { if (storageServers.size() < teamSize) { - TraceEvent(SevWarnAlways, "LessThanThreeStorageServers"); + TraceEvent(SevWarnAlways, "LessThanThreeStorageServers").log(); throw operation_failed(); } @@ -105,7 +105,7 @@ struct MoveKeysWorkload : TestWorkload { } if (t.size() < teamSize) { - TraceEvent(SevWarnAlways, "LessThanThreeUniqueMachines"); + TraceEvent(SevWarnAlways, "LessThanThreeUniqueMachines").log(); throw operation_failed(); } diff --git a/fdbserver/workloads/RestoreBackup.actor.cpp b/fdbserver/workloads/RestoreBackup.actor.cpp index 17a4355d36..360796ad89 100644 --- a/fdbserver/workloads/RestoreBackup.actor.cpp +++ b/fdbserver/workloads/RestoreBackup.actor.cpp @@ -73,7 +73,7 @@ struct RestoreBackupWorkload final : TestWorkload { .detail("TargetVersion", waitForVersion); if (desc.contiguousLogEnd.present() && desc.contiguousLogEnd.get() >= waitForVersion) { try { - TraceEvent("DiscontinuingBackup"); + TraceEvent("DiscontinuingBackup").log(); wait(self->backupAgent.discontinueBackup(cx, self->tag)); } catch (Error& e) { TraceEvent("ErrorDiscontinuingBackup").error(e); diff --git a/fdbserver/workloads/SimpleAtomicAdd.actor.cpp b/fdbserver/workloads/SimpleAtomicAdd.actor.cpp index 1ea8738696..43e836d61b 100644 --- a/fdbserver/workloads/SimpleAtomicAdd.actor.cpp +++ b/fdbserver/workloads/SimpleAtomicAdd.actor.cpp @@ -114,7 +114,7 @@ struct SimpleAtomicAddWorkload : TestWorkload { } loop { try { - TraceEvent("SAACheckKey"); + TraceEvent("SAACheckKey").log(); Optional actualValue = wait(tr.get(self->sumKey)); uint64_t actualValueInt = 0; if (actualValue.present()) { diff --git a/fdbserver/workloads/SnapTest.actor.cpp b/fdbserver/workloads/SnapTest.actor.cpp index caefb96dcd..4a1adca95c 100644 --- a/fdbserver/workloads/SnapTest.actor.cpp +++ b/fdbserver/workloads/SnapTest.actor.cpp @@ -90,7 +90,7 @@ public: // variables public: // ctor & dtor SnapTestWorkload(WorkloadContext const& wcx) : TestWorkload(wcx), numSnaps(0), maxSnapDelay(0.0), testID(0), snapUID() { - TraceEvent("SnapTestWorkloadConstructor"); + TraceEvent("SnapTestWorkloadConstructor").log(); std::string workloadName = "SnapTest"; maxRetryCntToRetrieveMessage = 10; @@ -107,11 +107,11 @@ public: // ctor & dtor public: // workload functions std::string description() const override { return "SnapTest"; } Future setup(Database const& cx) override { - TraceEvent("SnapTestWorkloadSetup"); + TraceEvent("SnapTestWorkloadSetup").log(); return Void(); } Future start(Database const& cx) override { - TraceEvent("SnapTestWorkloadStart"); + TraceEvent("SnapTestWorkloadStart").log(); if (clientId == 0) { return _start(cx, this); } @@ -120,7 +120,7 @@ public: // workload functions ACTOR Future _check(Database cx, SnapTestWorkload* self) { if (self->skipCheck) { - TraceEvent(SevWarnAlways, "SnapCheckIgnored"); + TraceEvent(SevWarnAlways, "SnapCheckIgnored").log(); return true; } state Transaction tr(cx); @@ -250,7 +250,7 @@ public: // workload functions bool backupFailed = atoi(ini.GetValue("RESTORE", "BackupFailed")); if (backupFailed) { // since backup failed, skip the restore checking - TraceEvent(SevWarnAlways, "BackupFailedSkippingRestoreCheck"); + TraceEvent(SevWarnAlways, "BackupFailedSkippingRestoreCheck").log(); return Void(); } state KeySelector begin = firstGreaterOrEqual(normalKeys.begin); @@ -265,7 +265,7 @@ public: // workload functions try { RangeResult kvRange = wait(tr.getRange(begin, end, 1000)); if (!kvRange.more && kvRange.size() == 0) { - TraceEvent("SnapTestNoMoreEntries"); + TraceEvent("SnapTestNoMoreEntries").log(); break; } diff --git a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp index 7c6c2006a6..68335b3bf8 100644 --- a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp +++ b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp @@ -721,7 +721,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { ASSERT(false); } else { // If no worker process returned, skip the test - TraceEvent(SevDebug, "EmptyWorkerListInSetClassTest"); + TraceEvent(SevDebug, "EmptyWorkerListInSetClassTest").log(); } } catch (Error& e) { if (e.code() == error_code_actor_cancelled) @@ -796,7 +796,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { tx->reset(); } else { // If no worker process returned, skip the test - TraceEvent(SevDebug, "EmptyWorkerListInSetClassTest"); + TraceEvent(SevDebug, "EmptyWorkerListInSetClassTest").log(); } } catch (Error& e) { wait(tx->onError(e)); @@ -832,7 +832,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { } } } - TraceEvent(SevDebug, "DatabaseLocked"); + TraceEvent(SevDebug, "DatabaseLocked").log(); // if database locked, fdb read should get database_locked error try { tx->reset(); @@ -851,7 +851,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { // unlock the database tx->clear(SpecialKeySpace::getManagementApiCommandPrefix("lock")); wait(tx->commit()); - TraceEvent(SevDebug, "DatabaseUnlocked"); + TraceEvent(SevDebug, "DatabaseUnlocked").log(); tx->reset(); // read should be successful RangeResult res = wait(tx->getRange(normalKeys, 1)); diff --git a/fdbserver/workloads/StatusWorkload.actor.cpp b/fdbserver/workloads/StatusWorkload.actor.cpp index 21c568d31e..708abae036 100644 --- a/fdbserver/workloads/StatusWorkload.actor.cpp +++ b/fdbserver/workloads/StatusWorkload.actor.cpp @@ -101,7 +101,7 @@ struct StatusWorkload : TestWorkload { TraceEvent(SevError, "SchemaCoverageRequirementsException").detail("What", e.what()); throw unknown_error(); } catch (...) { - TraceEvent(SevError, "SchemaCoverageRequirementsException"); + TraceEvent(SevError, "SchemaCoverageRequirementsException").log(); throw unknown_error(); } } diff --git a/fdbserver/workloads/Throttling.actor.cpp b/fdbserver/workloads/Throttling.actor.cpp index e6397da740..3665247e0c 100644 --- a/fdbserver/workloads/Throttling.actor.cpp +++ b/fdbserver/workloads/Throttling.actor.cpp @@ -112,7 +112,7 @@ struct ThrottlingWorkload : KVWorkload { } wait(tr.commit()); if (deterministicRandom()->randomInt(0, 1000) == 0) - TraceEvent("TransactionCommittedx1000"); + TraceEvent("TransactionCommittedx1000").log(); ++self->transactionsCommitted; } catch (Error& e) { if (e.code() == error_code_actor_cancelled) diff --git a/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp b/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp index 99df8fe2ff..562f45527c 100644 --- a/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp +++ b/fdbserver/workloads/TimeKeeperCorrectness.actor.cpp @@ -39,7 +39,7 @@ struct TimeKeeperCorrectnessWorkload : TestWorkload { void getMetrics(vector& m) override {} ACTOR static Future _start(Database cx, TimeKeeperCorrectnessWorkload* self) { - TraceEvent(SevInfo, "TKCorrectness_Start"); + TraceEvent(SevInfo, "TKCorrectness_Start").log(); state double start = now(); while (now() - start > self->testDuration) { @@ -60,7 +60,7 @@ struct TimeKeeperCorrectnessWorkload : TestWorkload { wait(delay(std::min(SERVER_KNOBS->TIME_KEEPER_DELAY / 10, (int64_t)1L))); } - TraceEvent(SevInfo, "TKCorrectness_Completed"); + TraceEvent(SevInfo, "TKCorrectness_Completed").log(); return Void(); } @@ -111,7 +111,7 @@ struct TimeKeeperCorrectnessWorkload : TestWorkload { } } - TraceEvent(SevInfo, "TKCorrectness_Passed"); + TraceEvent(SevInfo, "TKCorrectness_Passed").log(); return true; } catch (Error& e) { wait(tr->onError(e)); diff --git a/fdbserver/workloads/TriggerRecovery.actor.cpp b/fdbserver/workloads/TriggerRecovery.actor.cpp index ab21256c9b..753fdf4b17 100644 --- a/fdbserver/workloads/TriggerRecovery.actor.cpp +++ b/fdbserver/workloads/TriggerRecovery.actor.cpp @@ -111,7 +111,7 @@ struct TriggerRecoveryLoopWorkload : TestWorkload { else tr.set(LiteralStringRef("\xff\xff/reboot_worker"), it.second); } - TraceEvent(SevInfo, "TriggerRecoveryLoop_AttempedKillAll"); + TraceEvent(SevInfo, "TriggerRecoveryLoop_AttempedKillAll").log(); return Void(); } catch (Error& e) { wait(tr.onError(e)); diff --git a/fdbserver/workloads/VersionStamp.actor.cpp b/fdbserver/workloads/VersionStamp.actor.cpp index 6b1bd0d579..3cfa89e8f4 100644 --- a/fdbserver/workloads/VersionStamp.actor.cpp +++ b/fdbserver/workloads/VersionStamp.actor.cpp @@ -297,7 +297,7 @@ struct VersionStampWorkload : TestWorkload { wait(tr.onError(e)); } } - TraceEvent("VST_CheckEnd"); + TraceEvent("VST_CheckEnd").log(); return true; } diff --git a/fdbserver/workloads/WriteDuringRead.actor.cpp b/fdbserver/workloads/WriteDuringRead.actor.cpp index c938989d49..cbc2de23a0 100644 --- a/fdbserver/workloads/WriteDuringRead.actor.cpp +++ b/fdbserver/workloads/WriteDuringRead.actor.cpp @@ -518,7 +518,7 @@ ACTOR Future commitAndUpdateMemory(ReadYourWritesTransaction* tr, } if (failed) { - TraceEvent(SevError, "WriteConflictRangeError"); + TraceEvent(SevError, "WriteConflictRangeError").log(); for (transactionIter = transactionRanges.begin(); transactionIter != transactionRanges.end(); ++transactionIter) { TraceEvent("WCRTransaction") diff --git a/fdbserver/workloads/WriteTagThrottling.actor.cpp b/fdbserver/workloads/WriteTagThrottling.actor.cpp index 5f9e3c2a8d..9688d9bc69 100644 --- a/fdbserver/workloads/WriteTagThrottling.actor.cpp +++ b/fdbserver/workloads/WriteTagThrottling.actor.cpp @@ -135,7 +135,7 @@ struct WriteTagThrottlingWorkload : KVWorkload { return true; if (writeThrottle) { if (!badActorThrottleRetries && !goodActorThrottleRetries) { - TraceEvent(SevWarn, "NoThrottleTriggered"); + TraceEvent(SevWarn, "NoThrottleTriggered").log(); } if (badActorThrottleRetries < goodActorThrottleRetries) { TraceEvent(SevWarnAlways, "IncorrectThrottle") diff --git a/flow/DeterministicRandom.cpp b/flow/DeterministicRandom.cpp index c5043b4d62..215f66d04b 100644 --- a/flow/DeterministicRandom.cpp +++ b/flow/DeterministicRandom.cpp @@ -26,7 +26,7 @@ uint64_t DeterministicRandom::gen64() { uint64_t curr = next; next = (uint64_t(random()) << 32) ^ random(); if (TRACE_SAMPLE()) - TraceEvent(SevSample, "Random"); + TraceEvent(SevSample, "Random").log(); return curr; } diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 44572113d4..7bc4cb10ce 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -1182,7 +1182,7 @@ Net2::Net2(const TLSConfig& tlsConfig, bool useThreadPool, bool useMetrics) #endif { - TraceEvent("Net2Starting"); + TraceEvent("Net2Starting").log(); // Set the global members if (useMetrics) { @@ -1257,13 +1257,13 @@ ACTOR static Future reloadCertificatesOnChange( lifetimes.push_back(watchFileForChanges(config.getCAPathSync(), &fileChanged)); loop { wait(fileChanged.onTrigger()); - TraceEvent("TLSCertificateRefreshBegin"); + TraceEvent("TLSCertificateRefreshBegin").log(); try { LoadedTLSConfig loaded = wait(config.loadAsync()); boost::asio::ssl::context context(boost::asio::ssl::context::tls); ConfigureSSLContext(loaded, &context, onPolicyFailure); - TraceEvent(SevInfo, "TLSCertificateRefreshSucceeded"); + TraceEvent(SevInfo, "TLSCertificateRefreshSucceeded").log(); mismatches = 0; contextVar->set(ReferencedObject::from(std::move(context))); } catch (Error& e) { @@ -1385,13 +1385,13 @@ bool Net2::checkRunnable() { void Net2::run() { TraceEvent::setNetworkThread(); - TraceEvent("Net2Running"); + TraceEvent("Net2Running").log(); thread_network = this; #ifdef WIN32 if (timeBeginPeriod(1) != TIMERR_NOERROR) - TraceEvent(SevError, "TimeBeginPeriodError"); + TraceEvent(SevError, "TimeBeginPeriodError").log(); #endif timeOffsetLogger = logTimeOffset(); diff --git a/flow/Platform.actor.cpp b/flow/Platform.actor.cpp index bf19de35db..5aa40cd479 100644 --- a/flow/Platform.actor.cpp +++ b/flow/Platform.actor.cpp @@ -1227,19 +1227,19 @@ void getDiskStatistics(std::string const& directory, CFMutableDictionaryRef match = IOBSDNameMatching(kIOMasterPortDefault, kNilOptions, dev); if (!match) { - TraceEvent(SevError, "IOBSDNameMatching"); + TraceEvent(SevError, "IOBSDNameMatching").log(); throw platform_error(); } if (IOServiceGetMatchingServices(kIOMasterPortDefault, match, &disk_list) != kIOReturnSuccess) { - TraceEvent(SevError, "IOServiceGetMatchingServices"); + TraceEvent(SevError, "IOServiceGetMatchingServices").log(); throw platform_error(); } io_registry_entry_t disk = IOIteratorNext(disk_list); if (!disk) { IOObjectRelease(disk_list); - TraceEvent(SevError, "IOIteratorNext"); + TraceEvent(SevError, "IOIteratorNext").log(); throw platform_error(); } @@ -1255,7 +1255,7 @@ void getDiskStatistics(std::string const& directory, disk, (CFMutableDictionaryRef*)&disk_dict, kCFAllocatorDefault, kNilOptions) != kIOReturnSuccess) { IOObjectRelease(disk); IOObjectRelease(disk_list); - TraceEvent(SevError, "IORegistryEntryCreateCFProperties"); + TraceEvent(SevError, "IORegistryEntryCreateCFProperties").log(); throw platform_error(); } @@ -1268,7 +1268,7 @@ void getDiskStatistics(std::string const& directory, CFRelease(disk_dict); IOObjectRelease(disk); IOObjectRelease(disk_list); - TraceEvent(SevError, "CFDictionaryGetValue"); + TraceEvent(SevError, "CFDictionaryGetValue").log(); throw platform_error(); } @@ -1524,7 +1524,7 @@ SystemStatistics getSystemStatistics(std::string const& dataFolder, if ((*statState)->Query == nullptr) { initPdhStrings(*statState, dataFolder); - TraceEvent("SetupQuery"); + TraceEvent("SetupQuery").log(); handlePdhStatus(PdhOpenQuery(nullptr, NULL, &(*statState)->Query), "PdhOpenQuery"); if (!(*statState)->pdhStrings.diskDevice.empty()) { @@ -2073,7 +2073,7 @@ int getRandomSeed() { do { retryCount++; if (rand_s((unsigned int*)&randomSeed) != 0) { - TraceEvent(SevError, "WindowsRandomSeedError"); + TraceEvent(SevError, "WindowsRandomSeedError").log(); throw platform_error(); } } while (randomSeed == 0 && @@ -2093,7 +2093,7 @@ int getRandomSeed() { #endif if (randomSeed == 0) { - TraceEvent(SevError, "RandomSeedZeroError"); + TraceEvent(SevError, "RandomSeedZeroError").log(); throw platform_error(); } return randomSeed; diff --git a/flow/ThreadHelper.actor.h b/flow/ThreadHelper.actor.h index 6e2569ed0e..d6c776e992 100644 --- a/flow/ThreadHelper.actor.h +++ b/flow/ThreadHelper.actor.h @@ -242,7 +242,7 @@ public: void send(Never) { if (TRACE_SAMPLE()) - TraceEvent(SevSample, "Promise_sendNever"); + TraceEvent(SevSample, "Promise_sendNever").log(); ThreadSpinLockHolder holder(mutex); if (!canBeSetUnsafe()) ASSERT(false); // Promise fulfilled twice @@ -399,7 +399,7 @@ public: void send(const T& value) { if (TRACE_SAMPLE()) - TraceEvent(SevSample, "Promise_send"); + TraceEvent(SevSample, "Promise_send").log(); this->mutex.enter(); if (!canBeSetUnsafe()) { this->mutex.leave(); diff --git a/flow/Trace.cpp b/flow/Trace.cpp index e8655cf6cb..ceca64ca00 100644 --- a/flow/Trace.cpp +++ b/flow/Trace.cpp @@ -70,7 +70,7 @@ struct SuppressionMap { int64_t checkAndInsertSuppression(std::string type, double duration) { ASSERT(g_network); if (suppressionMap.size() >= FLOW_KNOBS->MAX_TRACE_SUPPRESSIONS) { - TraceEvent(SevWarnAlways, "ClearingTraceSuppressionMap"); + TraceEvent(SevWarnAlways, "ClearingTraceSuppressionMap").log(); suppressionMap.clear(); } diff --git a/flow/genericactors.actor.h b/flow/genericactors.actor.h index 30794d9791..fe502a3ec1 100644 --- a/flow/genericactors.actor.h +++ b/flow/genericactors.actor.h @@ -868,7 +868,7 @@ Future ioDegradedOrTimeoutError(Future what, when(T t = wait(what)) { return t; } when(wait(degradedEnd)) { TEST(true); // TLog degraded - TraceEvent(SevWarnAlways, "IoDegraded"); + TraceEvent(SevWarnAlways, "IoDegraded").log(); degraded->set(true); } } From 91e549835791d2d178ba9214ae2f20f27ef4ab08 Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Tue, 27 Jul 2021 11:17:55 -0600 Subject: [PATCH 18/26] disable simulation in ctest by default --- cmake/AddFdbTest.cmake | 6 ++++++ tests/CMakeLists.txt | 3 ++- tests/CTestCustom.ctest | 1 - tests/CTestCustom.ctest.cmake | 1 + 4 files changed, 9 insertions(+), 2 deletions(-) delete mode 100644 tests/CTestCustom.ctest create mode 100644 tests/CTestCustom.ctest.cmake diff --git a/cmake/AddFdbTest.cmake b/cmake/AddFdbTest.cmake index 752fcb8ebe..53fed78339 100644 --- a/cmake/AddFdbTest.cmake +++ b/cmake/AddFdbTest.cmake @@ -39,6 +39,9 @@ function(configure_testing) endfunction() function(verify_testing) + if(NOT ENABLE_SIMULATION_TESTS) + return() + endif() foreach(test_file IN LISTS fdb_test_files) message(SEND_ERROR "${test_file} found but it is not associated with a test") endforeach() @@ -59,6 +62,9 @@ function(add_fdb_test) set(options UNIT IGNORE) set(oneValueArgs TEST_NAME TIMEOUT) set(multiValueArgs TEST_FILES) + if (NOT ENABLE_SIMULATION_TESTS) + return() + endif() cmake_parse_arguments(ADD_FDB_TEST "${options}" "${oneValueArgs}" "${multiValueArgs}" "${ARGN}") set(this_test_timeout ${ADD_FDB_TEST_TIMEOUT}) if(NOT this_test_timeout) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 2105391eef..996d4d6322 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -2,6 +2,7 @@ include(AddFdbTest) # We need some variables to configure the test setup set(ENABLE_BUGGIFY ON CACHE BOOL "Enable buggify for tests") +set(ENABLE_SIMULATION_TESTS OFF CACHE BOOL "Enable simulation tests (useful if you can't run Joshua)") set(RUN_IGNORED_TESTS OFF CACHE BOOL "Run tests that are marked for ignore") set(TEST_KEEP_LOGS "FAILED" CACHE STRING "Which logs to keep (NONE, FAILED, ALL)") set(TEST_KEEP_SIMDIR "NONE" CACHE STRING "Which simfdb directories to keep (NONE, FAILED, ALL)") @@ -29,7 +30,7 @@ if(WITH_PYTHON) set(TestRunner "${PROJECT_SOURCE_DIR}/tests/TestRunner/TestRunner.py") - configure_file(${PROJECT_SOURCE_DIR}/tests/CTestCustom.ctest ${PROJECT_BINARY_DIR}/CTestCustom.ctest @ONLY) + configure_file(${PROJECT_SOURCE_DIR}/tests/CTestCustom.ctest.cmake ${PROJECT_BINARY_DIR}/CTestCustom.ctest @ONLY) configure_testing(TEST_DIRECTORY "${CMAKE_CURRENT_SOURCE_DIR}" ERROR_ON_ADDITIONAL_FILES diff --git a/tests/CTestCustom.ctest b/tests/CTestCustom.ctest deleted file mode 100644 index 42eb7cac5b..0000000000 --- a/tests/CTestCustom.ctest +++ /dev/null @@ -1 +0,0 @@ -set(CTEST_CUSTOM_PRE_TEST ${CTEST_CUSTOM_PRE_TEST} "@PROJECT_SOURCE_DIR@/tests/TestRunner/TestDirectory.py @PROJECT_BINARY_DIR@") diff --git a/tests/CTestCustom.ctest.cmake b/tests/CTestCustom.ctest.cmake new file mode 100644 index 0000000000..484bc07c71 --- /dev/null +++ b/tests/CTestCustom.ctest.cmake @@ -0,0 +1 @@ +set(CTEST_CUSTOM_PRE_TEST ${CTEST_CUSTOM_PRE_TEST} "@Python_EXECUTABLE@ @PROJECT_SOURCE_DIR@/tests/TestRunner/TestDirectory.py @PROJECT_BINARY_DIR@") From e1ec5f9aa4b989ef379a6dacff527d132fdf53fd Mon Sep 17 00:00:00 2001 From: Markus Pilman Date: Tue, 27 Jul 2021 11:38:25 -0600 Subject: [PATCH 19/26] Fix bug where tests wouldn't be added to correctness package --- cmake/AddFdbTest.cmake | 49 +++++++++++++++++++++--------------------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/cmake/AddFdbTest.cmake b/cmake/AddFdbTest.cmake index 53fed78339..8a4f638380 100644 --- a/cmake/AddFdbTest.cmake +++ b/cmake/AddFdbTest.cmake @@ -62,9 +62,6 @@ function(add_fdb_test) set(options UNIT IGNORE) set(oneValueArgs TEST_NAME TIMEOUT) set(multiValueArgs TEST_FILES) - if (NOT ENABLE_SIMULATION_TESTS) - return() - endif() cmake_parse_arguments(ADD_FDB_TEST "${options}" "${oneValueArgs}" "${multiValueArgs}" "${ARGN}") set(this_test_timeout ${ADD_FDB_TEST_TIMEOUT}) if(NOT this_test_timeout) @@ -125,28 +122,30 @@ function(add_fdb_test) set(VALGRIND_OPTION "--use-valgrind") endif() list(TRANSFORM ADD_FDB_TEST_TEST_FILES PREPEND "${CMAKE_CURRENT_SOURCE_DIR}/") - add_test(NAME ${test_name} - COMMAND $ ${TestRunner} - -n ${test_name} - -b ${PROJECT_BINARY_DIR} - -t ${test_type} - -O ${OLD_FDBSERVER_BINARY} - --crash - --aggregate-traces ${TEST_AGGREGATE_TRACES} - --log-format ${TEST_LOG_FORMAT} - --keep-logs ${TEST_KEEP_LOGS} - --keep-simdirs ${TEST_KEEP_SIMDIR} - --seed ${SEED} - --test-number ${assigned_id} - ${BUGGIFY_OPTION} - ${VALGRIND_OPTION} - ${ADD_FDB_TEST_TEST_FILES} - WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) - set_tests_properties("${test_name}" PROPERTIES ENVIRONMENT UBSAN_OPTIONS=print_stacktrace=1:halt_on_error=1) - get_filename_component(test_dir_full ${first_file} DIRECTORY) - if(NOT ${test_dir_full} STREQUAL "") - get_filename_component(test_dir ${test_dir_full} NAME) - set_tests_properties(${test_name} PROPERTIES TIMEOUT ${this_test_timeout} LABELS "${test_dir}") + if (ENABLE_SIMULATION_TESTS) + add_test(NAME ${test_name} + COMMAND $ ${TestRunner} + -n ${test_name} + -b ${PROJECT_BINARY_DIR} + -t ${test_type} + -O ${OLD_FDBSERVER_BINARY} + --crash + --aggregate-traces ${TEST_AGGREGATE_TRACES} + --log-format ${TEST_LOG_FORMAT} + --keep-logs ${TEST_KEEP_LOGS} + --keep-simdirs ${TEST_KEEP_SIMDIR} + --seed ${SEED} + --test-number ${assigned_id} + ${BUGGIFY_OPTION} + ${VALGRIND_OPTION} + ${ADD_FDB_TEST_TEST_FILES} + WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) + set_tests_properties("${test_name}" PROPERTIES ENVIRONMENT UBSAN_OPTIONS=print_stacktrace=1:halt_on_error=1) + get_filename_component(test_dir_full ${first_file} DIRECTORY) + if(NOT ${test_dir_full} STREQUAL "") + get_filename_component(test_dir ${test_dir_full} NAME) + set_tests_properties(${test_name} PROPERTIES TIMEOUT ${this_test_timeout} LABELS "${test_dir}") + endif() endif() # set variables used for generating test packages set(TEST_NAMES ${TEST_NAMES} ${test_name} PARENT_SCOPE) From d7fb3da607e7d93d0090209bd577cc90a7b7803f Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Tue, 27 Jul 2021 02:00:52 -0700 Subject: [PATCH 20/26] Add libatomic for building FDB using Clang --- flow/CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flow/CMakeLists.txt b/flow/CMakeLists.txt index 78d097517f..215fee3cad 100644 --- a/flow/CMakeLists.txt +++ b/flow/CMakeLists.txt @@ -137,6 +137,12 @@ if(CMAKE_SYSTEM_NAME STREQUAL "FreeBSD") target_link_libraries(flow PUBLIC ${EIO}) endif() endif() + +# For Clang in Linux environment, libatomic is required +if (UNIX AND CMAKE_CXX_COMPILER_ID MATCHES "Clang$") + set (FLOW_LIBS ${FLOW_LIBS} atomic) +endif () + target_link_libraries(flow PRIVATE ${FLOW_LIBS}) if(USE_VALGRIND) target_link_libraries(flow PUBLIC Valgrind) From 256a18e43b324a236d8c132635439ac3e77ef139 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 27 Jul 2021 12:01:32 -0700 Subject: [PATCH 21/26] Flow transport uses an ordered delay to avoid out of order reply promise stream messages --- fdbrpc/FlowTests.actor.cpp | 2 ++ fdbrpc/FlowTransport.actor.cpp | 6 +++--- fdbrpc/fdbrpc.h | 15 ++++++--------- fdbrpc/sim2.actor.cpp | 10 +++++++--- flow/Net2.actor.cpp | 6 ++++++ flow/flow.h | 3 +++ flow/network.h | 5 ++++- 7 files changed, 31 insertions(+), 16 deletions(-) diff --git a/fdbrpc/FlowTests.actor.cpp b/fdbrpc/FlowTests.actor.cpp index d3cf206c8f..04d6cae700 100644 --- a/fdbrpc/FlowTests.actor.cpp +++ b/fdbrpc/FlowTests.actor.cpp @@ -234,6 +234,8 @@ struct YieldMockNetwork final : INetwork, ReferenceCounted { Future delay(double seconds, TaskPriority taskID) override { return nextTick.getFuture(); } + Future orderedDelay(double seconds, TaskPriority taskID) override { return nextTick.getFuture(); } + Future yield(TaskPriority taskID) override { if (check_yield(taskID)) return delay(0, taskID); diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 8a6b32df56..e5c5a30632 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -922,9 +922,9 @@ ACTOR static void deliver(TransportData* self, // We want to run the task at the right priority. If the priority is higher than the current priority (which is // ReadSocket) we can just upgrade. Otherwise we'll context switch so that we don't block other tasks that might run // with a higher priority. ReplyPromiseStream needs to guarentee that messages are recieved in the order they were - // sent, so even in the case of local delivery those messages need to skip this delay. - if (priority < TaskPriority::ReadSocket || (priority != TaskPriority::NoDeliverDelay && !inReadSocket)) { - wait(delay(0, priority)); + // sent, so we are using orderedDelay. + if (priority < TaskPriority::ReadSocket || !inReadSocket) { + wait(orderedDelay(0, priority)); } else { g_network->setCurrentTask(priority); } diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index b66773307e..eeac5e9bed 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -361,7 +361,7 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, FlowTransport::transport().sendUnreliable( SerializeSource>( AcknowledgementReply(acknowledgements.bytesAcknowledged)), - acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), false); } } @@ -378,7 +378,7 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, acknowledgements.bytesAcknowledged += res.expectedSize(); FlowTransport::transport().sendUnreliable(SerializeSource>( AcknowledgementReply(acknowledgements.bytesAcknowledged)), - acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), false); } return res; @@ -389,13 +389,13 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, // Notify the server that a client is not using this ReplyPromiseStream anymore FlowTransport::transport().sendUnreliable( SerializeSource>(operation_obsolete()), - acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), false); } if (isRemoteEndpoint() && !sentError && !acknowledgements.failures.isReady()) { // The ReplyPromiseStream was cancelled before sending an error, so the storage server must have died FlowTransport::transport().sendUnreliable(SerializeSource>>(broken_promise()), - getEndpoint(TaskPriority::NoDeliverDelay), + getEndpoint(TaskPriority::ReadSocket), false); } } @@ -406,9 +406,6 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, template class ReplyPromiseStream { public: - // The endpoints of a ReplyPromiseStream must be initialized at Task::NoDeliverDelay, because a - // delay(0) in FlowTransport deliver can cause out of order delivery. - // stream.send( request ) // Unreliable at most once delivery: Delivers request unless there is a connection failure (zero or one times) @@ -416,7 +413,7 @@ public: void send(U&& value) const { if (queue->isRemoteEndpoint()) { if (!queue->acknowledgements.getRawEndpoint().isValid()) { - value.acknowledgeToken = queue->acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay).token; + value.acknowledgeToken = queue->acknowledgements.getEndpoint(TaskPriority::ReadSocket).token; } queue->acknowledgements.bytesSent += value.expectedSize(); FlowTransport::transport().sendUnreliable( @@ -477,7 +474,7 @@ public: errors->delPromiseRef(); } - const Endpoint& getEndpoint() const { return queue->getEndpoint(TaskPriority::NoDeliverDelay); } + const Endpoint& getEndpoint() const { return queue->getEndpoint(TaskPriority::ReadSocket); } bool operator==(const ReplyPromiseStream& rhs) const { return queue == rhs.queue; } bool isEmpty() const { return !queue->isReady(); } diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index fe7ded16e5..2d03d2b08e 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -858,13 +858,17 @@ public: ASSERT(taskID >= TaskPriority::Min && taskID <= TaskPriority::Max); return delay(seconds, taskID, currentProcess); } - Future delay(double seconds, TaskPriority taskID, ProcessInfo* machine) { + Future orderedDelay(double seconds, TaskPriority taskID) override { + ASSERT(taskID >= TaskPriority::Min && taskID <= TaskPriority::Max); + return delay(seconds, taskID, currentProcess, true); + } + Future delay(double seconds, TaskPriority taskID, ProcessInfo* machine, bool ordered = false) { ASSERT(seconds >= -0.0001); seconds = std::max(0.0, seconds); Future f; - if (!currentProcess->rebooting && machine == currentProcess && !currentProcess->shutdownSignal.isSet() && - FLOW_KNOBS->MAX_BUGGIFIED_DELAY > 0 && + if (!ordered && !currentProcess->rebooting && machine == currentProcess && + !currentProcess->shutdownSignal.isSet() && FLOW_KNOBS->MAX_BUGGIFIED_DELAY > 0 && deterministicRandom()->random01() < 0.25) { // FIXME: why doesnt this work when we are changing machines? seconds += FLOW_KNOBS->MAX_BUGGIFIED_DELAY * pow(deterministicRandom()->random01(), 1000.0); } diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 44572113d4..4a938e126c 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -161,6 +161,7 @@ public: double timer() override { return ::timer(); }; double timer_monotonic() override { return ::timer_monotonic(); }; Future delay(double seconds, TaskPriority taskId) override; + Future orderedDelay(double seconds, TaskPriority taskId) override; Future yield(TaskPriority taskID) override; bool check_yield(TaskPriority taskId) override; TaskPriority getCurrentTask() const override { return currentTaskID; } @@ -1750,6 +1751,11 @@ Future Net2::delay(double seconds, TaskPriority taskId) { return t->promise.getFuture(); } +Future Net2::orderedDelay(double seconds, TaskPriority taskId) { + // The regular delay already provides the required ordering property + return delay(seconds, taskId); +} + void Net2::onMainThread(Promise&& signal, TaskPriority taskID) { if (stopped) return; diff --git a/flow/flow.h b/flow/flow.h index b598f82987..4a08d37b93 100644 --- a/flow/flow.h +++ b/flow/flow.h @@ -1087,6 +1087,9 @@ inline double now() { inline Future delay(double seconds, TaskPriority taskID = TaskPriority::DefaultDelay) { return g_network->delay(seconds, taskID); } +inline Future orderedDelay(double seconds, TaskPriority taskID = TaskPriority::DefaultDelay) { + return g_network->orderedDelay(seconds, taskID); +} inline Future delayUntil(double time, TaskPriority taskID = TaskPriority::DefaultDelay) { return g_network->delay(std::max(0.0, time - g_network->now()), taskID); } diff --git a/flow/network.h b/flow/network.h index 0d0f8a2d34..77d61ca1e8 100644 --- a/flow/network.h +++ b/flow/network.h @@ -45,7 +45,6 @@ enum class TaskPriority { WriteSocket = 10000, PollEIO = 9900, DiskIOComplete = 9150, - NoDeliverDelay = 9100, LoadBalancedEndpoint = 9000, ReadSocket = 9000, AcceptSocket = 8950, @@ -507,6 +506,10 @@ public: virtual Future delay(double seconds, TaskPriority taskID) = 0; // The given future will be set after seconds have elapsed + virtual Future orderedDelay(double seconds, TaskPriority taskID) = 0; + // The given future will be set after seconds have elapsed, delays with the same time and TaskPriority will be + // executed in the order they were issues + virtual Future yield(TaskPriority taskID) = 0; // The given future will be set immediately or after higher-priority tasks have executed From 52940d38d97c059cad673880ff503fe57a6ebdbb Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 27 Jul 2021 14:12:01 -0700 Subject: [PATCH 22/26] Fix issue where GlobalConfig wasn't initialized in time when running fdbcli --exec --- fdbcli/fdbcli.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index bef9e3d43c..bcddfbc7f3 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3323,6 +3323,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { try { db = Database::createDatabase(ccf, -1, IsInternal::False); + wait(GlobalConfig::globalConfig().onInitialized()); if (!opt.exec.present()) { printf("Using cluster file `%s'.\n", ccf->getFilename().c_str()); } From d7a03cc703084a01cee5f94323358677a2622dd0 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 27 Jul 2021 14:12:17 -0700 Subject: [PATCH 23/26] Add GlobalConfig TraceEvents for easier debugging --- fdbclient/GlobalConfig.actor.cpp | 3 +++ fdbclient/GlobalConfig.actor.h | 1 + 2 files changed, 4 insertions(+) diff --git a/fdbclient/GlobalConfig.actor.cpp b/fdbclient/GlobalConfig.actor.cpp index a5b4febdea..2f2b82c332 100644 --- a/fdbclient/GlobalConfig.actor.cpp +++ b/fdbclient/GlobalConfig.actor.cpp @@ -77,6 +77,7 @@ void GlobalConfig::trigger(KeyRef key, std::functionfirst)) { @@ -174,6 +176,7 @@ ACTOR Future GlobalConfig::migrate(GlobalConfig* self) { // Updates local copy of global configuration by reading the entire key-range // from storage. ACTOR Future GlobalConfig::refresh(GlobalConfig* self) { + TraceEvent trace(SevInfo, "GlobalConfig_Refresh"); self->erase(KeyRangeRef(""_sr, "\xff"_sr)); Transaction tr(self->cx); diff --git a/fdbclient/GlobalConfig.actor.h b/fdbclient/GlobalConfig.actor.h index 444f1ab697..3c5811486b 100644 --- a/fdbclient/GlobalConfig.actor.h +++ b/fdbclient/GlobalConfig.actor.h @@ -108,6 +108,7 @@ public: // the key. template {}, bool>::type = true> const T get(KeyRef name, T defaultVal) { + TraceEvent(SevInfo, "GlobalConfig_Get").detail("Key", name); try { auto configValue = get(name); if (configValue.isValid()) { From acfb9adbd2e8763ec1b553276a230d9e2cef6360 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 27 Jul 2021 15:10:49 -0700 Subject: [PATCH 24/26] Fix ctest timeouts --- fdbcli/fdbcli.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index bcddfbc7f3..a573ffdd29 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3323,7 +3323,6 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { try { db = Database::createDatabase(ccf, -1, IsInternal::False); - wait(GlobalConfig::globalConfig().onInitialized()); if (!opt.exec.present()) { printf("Using cluster file `%s'.\n", ccf->getFilename().c_str()); } @@ -3961,6 +3960,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } + wait(GlobalConfig::globalConfig().onInitialized()); if (tokencmp(tokens[2], "get")) { if (tokens.size() != 3) { fprintf(stderr, "ERROR: Addtional arguments to `get` are not supported.\n"); From 08bc78735674f5e5e5f23e869eeeb1106168499a Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 27 Jul 2021 15:22:01 -0700 Subject: [PATCH 25/26] Add timeout --- fdbcli/fdbcli.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index a573ffdd29..7b5adedaab 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3960,7 +3960,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - wait(GlobalConfig::globalConfig().onInitialized()); + wait(timeout(GlobalConfig::globalConfig().onInitialized(), 3, Void())); if (tokencmp(tokens[2], "get")) { if (tokens.size() != 3) { fprintf(stderr, "ERROR: Addtional arguments to `get` are not supported.\n"); From b423432ee1411e3e0bc0266fb0ea91b63ee50744 Mon Sep 17 00:00:00 2001 From: Lukas Joswiak Date: Tue, 27 Jul 2021 15:45:35 -0700 Subject: [PATCH 26/26] Make wait interruptable --- fdbcli/fdbcli.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 7b5adedaab..9982a576d0 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3960,7 +3960,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { is_error = true; continue; } - wait(timeout(GlobalConfig::globalConfig().onInitialized(), 3, Void())); + wait(makeInterruptable(GlobalConfig::globalConfig().onInitialized())); if (tokencmp(tokens[2], "get")) { if (tokens.size() != 3) { fprintf(stderr, "ERROR: Addtional arguments to `get` are not supported.\n");