foundationdb/fdbserver/ConfigNode.actor.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

842 lines
36 KiB
C++
Raw Normal View History

2021-04-15 13:06:37 +08:00
/*
* ConfigNode.actor.cpp
2021-04-15 13:06:37 +08:00
*
* This source file is part of the FoundationDB open source project
*
2022-03-22 04:36:23 +08:00
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
2021-04-15 13:06:37 +08:00
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <map>
#include "fdbclient/SystemData.h"
#include "fdbserver/ConfigNode.h"
2021-04-15 13:06:37 +08:00
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/OnDemandStore.h"
#include "flow/Arena.h"
2021-04-15 13:06:37 +08:00
#include "flow/genericactors.actor.h"
#include "flow/UnitTest.h"
2021-04-15 13:06:37 +08:00
#include "flow/actorcompiler.h" // This must be the last #include.
namespace {
const KeyRef coordinatorsHashKey = "id"_sr;
2021-05-19 09:32:56 +08:00
const KeyRef lastCompactedVersionKey = "lastCompactedVersion"_sr;
const KeyRef currentGenerationKey = "currentGeneration"_sr;
2022-02-02 14:27:12 +08:00
const KeyRef registeredKey = "registered"_sr;
const KeyRef lockedKey = "locked"_sr;
2021-05-19 09:32:56 +08:00
const KeyRangeRef kvKeys = KeyRangeRef("kv/"_sr, "kv0"_sr);
const KeyRangeRef mutationKeys = KeyRangeRef("mutation/"_sr, "mutation0"_sr);
const KeyRangeRef annotationKeys = KeyRangeRef("annotation/"_sr, "annotation0"_sr);
Key versionedAnnotationKey(Version version) {
ASSERT_GE(version, 0);
return BinaryWriter::toValue(bigEndian64(version), IncludeVersion()).withPrefix(annotationKeys.begin);
}
Version getVersionFromVersionedAnnotationKey(KeyRef versionedAnnotationKey) {
return fromBigEndian64(BinaryReader::fromStringRef<uint64_t>(
versionedAnnotationKey.removePrefix(annotationKeys.begin), IncludeVersion()));
}
Key versionedMutationKey(Version version, uint32_t index) {
ASSERT_GE(version, 0);
BinaryWriter bw(IncludeVersion());
bw << bigEndian64(version);
bw << bigEndian32(index);
return bw.toValue().withPrefix(mutationKeys.begin);
}
Version getVersionFromVersionedMutationKey(KeyRef versionedMutationKey) {
uint64_t bigEndianResult;
ASSERT(versionedMutationKey.startsWith(mutationKeys.begin));
BinaryReader br(versionedMutationKey.removePrefix(mutationKeys.begin), IncludeVersion());
br >> bigEndianResult;
return fromBigEndian64(bigEndianResult);
}
2021-10-26 01:17:22 +08:00
template <typename T>
void assertCommitted(RangeResult const& range, VectorRef<T> versionedConfigs, std::function<Version(KeyRef)> fn) {
if (range.size() == 0) {
return;
}
// Verify every versioned value read from disk (minus the last one which
// may not be committed on a quorum) exists in the rollforward changes.
for (auto it = range.begin(); it != std::prev(range.end()); ++it) {
Version version = fn(it->key);
auto resultIt = std::find_if(
versionedConfigs.begin(), versionedConfigs.end(), [version](T const& o) { return o.version == version; });
ASSERT(resultIt != versionedConfigs.end());
}
}
} // namespace
2021-04-15 13:06:37 +08:00
TEST_CASE("/fdbserver/ConfigDB/ConfigNode/Internal/versionedMutationKeys") {
std::vector<Key> keys;
for (Version version = 0; version < 1000; ++version) {
for (int index = 0; index < 5; ++index) {
keys.push_back(versionedMutationKey(version, index));
}
}
for (int i = 0; i < 5000; ++i) {
ASSERT(getVersionFromVersionedMutationKey(keys[i]) == i / 5);
}
return Void();
}
TEST_CASE("/fdbserver/ConfigDB/ConfigNode/Internal/versionedMutationKeyOrdering") {
Standalone<VectorRef<KeyRef>> keys;
for (Version version = 0; version < 1000; ++version) {
for (auto index = 0; index < 5; ++index) {
keys.push_back_deep(keys.arena(), versionedMutationKey(version, index));
}
}
for (auto index = 0; index < 1000; ++index) {
keys.push_back_deep(keys.arena(), versionedMutationKey(1000, index));
}
ASSERT(std::is_sorted(keys.begin(), keys.end()));
return Void();
}
class ConfigNodeImpl {
UID id;
OnDemandStore kvStore;
2021-05-14 07:39:58 +08:00
CounterCollection cc;
// Follower counters
Counter compactRequests;
2021-08-25 08:31:28 +08:00
Counter rollbackRequests;
Counter rollforwardRequests;
2021-05-14 07:39:58 +08:00
Counter successfulChangeRequests;
Counter failedChangeRequests;
Counter snapshotRequests;
Counter getCommittedVersionRequests;
Counter lockRequests;
2021-05-14 07:39:58 +08:00
// Transaction counters
Counter successfulCommits;
Counter failedCommits;
Counter setMutations;
Counter clearMutations;
Counter getValueRequests;
2021-08-07 06:20:12 +08:00
Counter getGenerationRequests;
2021-05-14 07:39:58 +08:00
Future<Void> logger;
ACTOR static Future<Optional<size_t>> getCoordinatorsHash(ConfigNodeImpl* self) {
Optional<Value> value = wait(self->kvStore->readValue(coordinatorsHashKey));
if (!value.present()) {
return Optional<size_t>();
}
return BinaryReader::fromStringRef<size_t>(value.get(), IncludeVersion());
}
ACTOR static Future<Optional<size_t>> getLocked(ConfigNodeImpl* self) {
Optional<Value> value = wait(self->kvStore->readValue(lockedKey));
if (!value.present()) {
return false;
}
return BinaryReader::fromStringRef<Optional<size_t>>(value.get(), IncludeVersion());
}
ACTOR static Future<ConfigGeneration> getGeneration(ConfigNodeImpl* self) {
state ConfigGeneration generation;
Optional<Value> value = wait(self->kvStore->readValue(currentGenerationKey));
if (value.present()) {
generation = BinaryReader::fromStringRef<ConfigGeneration>(value.get(), IncludeVersion());
} else {
self->kvStore->set(KeyValueRef(currentGenerationKey, BinaryWriter::toValue(generation, IncludeVersion())));
wait(self->kvStore->commit());
}
return generation;
}
ACTOR static Future<Version> getLastCompactedVersion(ConfigNodeImpl* self) {
Optional<Value> value = wait(self->kvStore->readValue(lastCompactedVersionKey));
state Version lastCompactedVersion = 0;
if (value.present()) {
lastCompactedVersion = BinaryReader::fromStringRef<Version>(value.get(), IncludeVersion());
} else {
self->kvStore->set(
KeyValueRef(lastCompactedVersionKey, BinaryWriter::toValue(lastCompactedVersion, IncludeVersion())));
wait(self->kvStore->commit());
}
return lastCompactedVersion;
}
// Returns all commit annotations between for commits with version in [startVersion, endVersion]
ACTOR static Future<Standalone<VectorRef<VersionedConfigCommitAnnotationRef>>> getAnnotations(ConfigNodeImpl* self,
Version startVersion,
Version endVersion) {
Key startKey = versionedAnnotationKey(startVersion);
Key endKey = versionedAnnotationKey(endVersion + 1);
state KeyRangeRef keys(startKey, endKey);
RangeResult range = wait(self->kvStore->readRange(keys));
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> result;
for (const auto& kv : range) {
auto version = getVersionFromVersionedAnnotationKey(kv.key);
ASSERT_LE(version, endVersion);
auto annotation = BinaryReader::fromStringRef<ConfigCommitAnnotation>(kv.value, IncludeVersion());
result.emplace_back_deep(result.arena(), version, annotation);
}
return result;
}
// Returns all mutations with version in [startVersion, endVersion]
ACTOR static Future<Standalone<VectorRef<VersionedConfigMutationRef>>> getMutations(ConfigNodeImpl* self,
Version startVersion,
Version endVersion) {
Key startKey = versionedMutationKey(startVersion, 0);
Key endKey = versionedMutationKey(endVersion + 1, 0);
state KeyRangeRef keys(startKey, endKey);
RangeResult range = wait(self->kvStore->readRange(keys));
Standalone<VectorRef<VersionedConfigMutationRef>> result;
for (const auto& kv : range) {
auto version = getVersionFromVersionedMutationKey(kv.key);
ASSERT_LE(version, endVersion);
auto mutation = ObjectReader::fromStringRef<ConfigMutation>(kv.value, IncludeVersion());
result.emplace_back_deep(result.arena(), version, mutation);
}
return result;
}
ACTOR static Future<Void> getChanges(ConfigNodeImpl* self, ConfigFollowerGetChangesRequest req) {
Version lastCompactedVersion = wait(getLastCompactedVersion(self));
if (req.lastSeenVersion < lastCompactedVersion) {
2021-05-14 07:39:58 +08:00
++self->failedChangeRequests;
2021-04-24 08:49:57 +08:00
req.reply.sendError(version_already_compacted());
return Void();
2021-04-24 08:49:57 +08:00
}
state Version committedVersion =
wait(map(getGeneration(self), [](auto const& gen) { return gen.committedVersion; }));
2022-02-02 14:27:12 +08:00
if (committedVersion < req.mostRecentVersion) {
// Handle a very rare case where a ConfigNode loses data between
// responding with a committed version and responding to the
// subsequent get changes request.
CODE_PROBE(true, "ConfigNode data loss occurred on a minority of coordinators");
2022-02-02 14:27:12 +08:00
req.reply.sendError(process_behind()); // Reuse the process_behind error
return Void();
}
state Standalone<VectorRef<VersionedConfigMutationRef>> versionedMutations =
wait(getMutations(self, req.lastSeenVersion + 1, committedVersion));
state Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> versionedAnnotations =
wait(getAnnotations(self, req.lastSeenVersion + 1, committedVersion));
2021-08-03 06:50:10 +08:00
TraceEvent(SevDebug, "ConfigNodeSendingChanges", self->id)
.detail("ReqLastSeenVersion", req.lastSeenVersion)
.detail("ReqMostRecentVersion", req.mostRecentVersion)
.detail("CommittedVersion", committedVersion)
.detail("NumMutations", versionedMutations.size())
.detail("NumCommits", versionedAnnotations.size());
2021-05-14 07:39:58 +08:00
++self->successfulChangeRequests;
req.reply.send(ConfigFollowerGetChangesReply{ committedVersion, versionedMutations, versionedAnnotations });
return Void();
}
// New transactions increment the database's current live version. This effectively serves as a lock, providing
// serializability
ACTOR static Future<Void> getNewGeneration(ConfigNodeImpl* self, ConfigTransactionGetGenerationRequest req) {
state Optional<size_t> coordinatorsHash = wait(getCoordinatorsHash(self));
ASSERT(coordinatorsHash.present());
if (req.coordinatorsHash != coordinatorsHash.get()) {
req.reply.sendError(coordinators_changed());
return Void();
}
state ConfigGeneration generation = wait(getGeneration(self));
++generation.liveVersion;
if (req.lastSeenLiveVersion.present()) {
CODE_PROBE(req.lastSeenLiveVersion.get() >= generation.liveVersion,
"Node is lagging behind some other node");
generation.liveVersion = std::max(generation.liveVersion, req.lastSeenLiveVersion.get() + 1);
}
self->kvStore->set(KeyValueRef(currentGenerationKey, BinaryWriter::toValue(generation, IncludeVersion())));
2021-04-15 13:06:37 +08:00
wait(self->kvStore->commit());
req.reply.send(ConfigTransactionGetGenerationReply{ generation });
2021-04-15 13:06:37 +08:00
return Void();
}
ACTOR static Future<Void> get(ConfigNodeImpl* self, ConfigTransactionGetRequest req) {
state Optional<size_t> locked = wait(getLocked(self));
if (locked.present()) {
CODE_PROBE(true, "attempting to read from a locked ConfigNode");
req.reply.sendError(coordinators_changed());
return Void();
}
state Optional<size_t> coordinatorsHash = wait(getCoordinatorsHash(self));
ASSERT(coordinatorsHash.present());
if (req.coordinatorsHash != coordinatorsHash.get()) {
req.reply.sendError(coordinators_changed());
return Void();
}
ConfigGeneration currentGeneration = wait(getGeneration(self));
if (req.generation != currentGeneration) {
// TODO: Also send information about highest seen version
2021-04-15 13:06:37 +08:00
req.reply.sendError(transaction_too_old());
return Void();
}
state Optional<Value> serializedValue =
2021-05-12 03:00:44 +08:00
wait(self->kvStore->readValue(BinaryWriter::toValue(req.key, IncludeVersion()).withPrefix(kvKeys.begin)));
state Optional<KnobValue> value;
if (serializedValue.present()) {
value = ObjectReader::fromStringRef<KnobValue>(serializedValue.get(), IncludeVersion());
}
Standalone<VectorRef<VersionedConfigMutationRef>> versionedMutations =
wait(getMutations(self, 0, req.generation.committedVersion));
for (const auto& versionedMutation : versionedMutations) {
const auto& mutation = versionedMutation.mutation;
if (mutation.getKey() == req.key) {
if (mutation.isSet()) {
value = mutation.getValue();
} else {
value = {};
}
}
2021-04-15 13:06:37 +08:00
}
req.reply.send(ConfigTransactionGetReply{ value });
2021-04-15 13:06:37 +08:00
return Void();
}
// Retrieve all configuration classes that contain explicitly defined knobs
// TODO: Currently it is possible that extra configuration classes may be returned, we
// may want to fix this to clean up the contract
ACTOR static Future<Void> getConfigClasses(ConfigNodeImpl* self, ConfigTransactionGetConfigClassesRequest req) {
state Optional<size_t> locked = wait(getLocked(self));
if (locked.present()) {
CODE_PROBE(true, "attempting to read config classes from locked ConfigNode");
req.reply.sendError(coordinators_changed());
return Void();
}
ConfigGeneration currentGeneration = wait(getGeneration(self));
if (req.generation != currentGeneration) {
req.reply.sendError(transaction_too_old());
return Void();
}
state RangeResult snapshot = wait(self->kvStore->readRange(kvKeys));
state std::set<Key> configClassesSet;
for (const auto& kv : snapshot) {
auto configKey =
BinaryReader::fromStringRef<ConfigKey>(kv.key.removePrefix(kvKeys.begin), IncludeVersion());
if (configKey.configClass.present()) {
configClassesSet.insert(configKey.configClass.get());
}
}
state Version lastCompactedVersion = wait(getLastCompactedVersion(self));
state Standalone<VectorRef<VersionedConfigMutationRef>> mutations =
wait(getMutations(self, lastCompactedVersion + 1, req.generation.committedVersion));
for (const auto& versionedMutation : mutations) {
auto configClass = versionedMutation.mutation.getConfigClass();
if (configClass.present()) {
configClassesSet.insert(configClass.get());
}
}
Standalone<VectorRef<KeyRef>> configClasses;
for (const auto& configClass : configClassesSet) {
configClasses.push_back_deep(configClasses.arena(), configClass);
}
req.reply.send(ConfigTransactionGetConfigClassesReply{ configClasses });
return Void();
}
// Retrieve all knobs explicitly defined for the specified configuration class
ACTOR static Future<Void> getKnobs(ConfigNodeImpl* self, ConfigTransactionGetKnobsRequest req) {
state Optional<size_t> locked = wait(getLocked(self));
if (locked.present()) {
CODE_PROBE(true, "attempting to read knobs from locked ConfigNode");
req.reply.sendError(coordinators_changed());
return Void();
}
ConfigGeneration currentGeneration = wait(getGeneration(self));
if (req.generation != currentGeneration) {
req.reply.sendError(transaction_too_old());
return Void();
}
// FIXME: Filtering after reading from disk is very inefficient
state RangeResult snapshot = wait(self->kvStore->readRange(kvKeys));
state std::set<Key> knobSet;
for (const auto& kv : snapshot) {
auto configKey =
BinaryReader::fromStringRef<ConfigKey>(kv.key.removePrefix(kvKeys.begin), IncludeVersion());
2021-06-22 11:19:36 +08:00
if (configKey.configClass.template castTo<Key>() == req.configClass) {
knobSet.insert(configKey.knobName);
}
}
state Version lastCompactedVersion = wait(getLastCompactedVersion(self));
state Standalone<VectorRef<VersionedConfigMutationRef>> mutations =
wait(getMutations(self, lastCompactedVersion + 1, req.generation.committedVersion));
for (const auto& versionedMutation : mutations) {
2021-06-22 11:19:36 +08:00
if (versionedMutation.mutation.getConfigClass().template castTo<Key>() == req.configClass) {
if (versionedMutation.mutation.isSet()) {
knobSet.insert(versionedMutation.mutation.getKnobName());
} else {
knobSet.erase(versionedMutation.mutation.getKnobName());
}
}
}
Standalone<VectorRef<KeyRef>> knobNames;
for (const auto& knobName : knobSet) {
knobNames.push_back_deep(knobNames.arena(), knobName);
}
req.reply.send(ConfigTransactionGetKnobsReply{ knobNames });
return Void();
}
ACTOR static Future<Void> commitMutations(ConfigNodeImpl* self,
Standalone<VectorRef<VersionedConfigMutationRef>> mutations,
2021-08-28 08:52:33 +08:00
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> annotations,
Version commitVersion) {
Version latestVersion = 0;
int index = 0;
for (const auto& mutation : mutations) {
if (mutation.version > commitVersion) {
continue;
}
// Mutations should be in ascending version order.
ASSERT_GE(mutation.version, latestVersion);
if (mutation.version > latestVersion) {
latestVersion = mutation.version;
index = 0;
}
Key key = versionedMutationKey(mutation.version, index++);
Value value = ObjectWriter::toValue(mutation.mutation, IncludeVersion());
if (mutation.mutation.isSet()) {
TraceEvent("ConfigNodeSetting")
.detail("ConfigClass", mutation.mutation.getConfigClass())
.detail("KnobName", mutation.mutation.getKnobName())
.detail("Value", mutation.mutation.getValue().toString())
.detail("Version", mutation.version);
++self->setMutations;
} else {
++self->clearMutations;
}
self->kvStore->set(KeyValueRef(key, value));
}
2021-08-28 08:52:33 +08:00
for (const auto& annotation : annotations) {
self->kvStore->set(KeyValueRef(versionedAnnotationKey(annotation.version),
BinaryWriter::toValue(annotation.annotation, IncludeVersion())));
}
ConfigGeneration newGeneration = { commitVersion, commitVersion };
self->kvStore->set(KeyValueRef(currentGenerationKey, BinaryWriter::toValue(newGeneration, IncludeVersion())));
wait(self->kvStore->commit());
++self->successfulCommits;
return Void();
}
ACTOR static Future<Void> commit(ConfigNodeImpl* self, ConfigTransactionCommitRequest req) {
state Optional<size_t> locked = wait(getLocked(self));
if (locked.present()) {
CODE_PROBE(true, "attempting to write to locked ConfigNode");
req.reply.sendError(coordinators_changed());
return Void();
}
state Optional<size_t> coordinatorsHash = wait(getCoordinatorsHash(self));
ASSERT(coordinatorsHash.present());
if (req.coordinatorsHash != coordinatorsHash.get()) {
req.reply.sendError(coordinators_changed());
return Void();
}
ConfigGeneration currentGeneration = wait(getGeneration(self));
if (req.generation.committedVersion != currentGeneration.committedVersion) {
2021-05-14 07:39:58 +08:00
++self->failedCommits;
req.reply.sendError(commit_unknown_result());
return Void();
} else if (req.generation.liveVersion != currentGeneration.liveVersion) {
++self->failedCommits;
req.reply.sendError(not_committed());
2021-04-15 13:06:37 +08:00
return Void();
}
Standalone<VectorRef<VersionedConfigMutationRef>> mutations;
for (const auto& mutation : req.mutations) {
mutations.emplace_back_deep(mutations.arena(), req.generation.liveVersion, mutation);
2021-04-15 13:06:37 +08:00
}
2021-08-28 08:52:33 +08:00
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> annotations;
annotations.emplace_back_deep(annotations.arena(), req.generation.liveVersion, req.annotation);
wait(commitMutations(self, mutations, annotations, req.generation.liveVersion));
2021-04-15 13:06:37 +08:00
req.reply.send(Void());
return Void();
}
ACTOR static Future<Void> serve(ConfigNodeImpl* self, ConfigTransactionInterface const* cti) {
2021-04-15 13:06:37 +08:00
loop {
choose {
when(ConfigTransactionGetGenerationRequest req = waitNext(cti->getGeneration.getFuture())) {
2021-08-07 06:20:12 +08:00
++self->getGenerationRequests;
wait(getNewGeneration(self, req));
2021-04-15 13:06:37 +08:00
}
when(ConfigTransactionGetRequest req = waitNext(cti->get.getFuture())) {
2021-05-14 07:39:58 +08:00
++self->getValueRequests;
wait(get(self, req));
2021-04-15 13:06:37 +08:00
}
when(ConfigTransactionCommitRequest req = waitNext(cti->commit.getFuture())) {
wait(commit(self, req));
2021-04-15 13:06:37 +08:00
}
when(ConfigTransactionGetConfigClassesRequest req = waitNext(cti->getClasses.getFuture())) {
wait(getConfigClasses(self, req));
}
when(ConfigTransactionGetKnobsRequest req = waitNext(cti->getKnobs.getFuture())) {
wait(getKnobs(self, req));
}
when(wait(self->kvStore->getError())) { ASSERT(false); }
2021-04-15 13:06:37 +08:00
}
}
}
ACTOR static Future<Void> getSnapshotAndChanges(ConfigNodeImpl* self,
2021-05-19 06:28:44 +08:00
ConfigFollowerGetSnapshotAndChangesRequest req) {
state ConfigFollowerGetSnapshotAndChangesReply reply;
RangeResult data = wait(self->kvStore->readRange(kvKeys));
for (const auto& kv : data) {
reply
2021-05-13 02:56:45 +08:00
.snapshot[BinaryReader::fromStringRef<ConfigKey>(kv.key.removePrefix(kvKeys.begin), IncludeVersion())] =
ObjectReader::fromStringRef<KnobValue>(kv.value, IncludeVersion());
}
2021-05-19 06:28:44 +08:00
wait(store(reply.snapshotVersion, getLastCompactedVersion(self)));
wait(store(reply.changes, getMutations(self, reply.snapshotVersion + 1, req.mostRecentVersion)));
wait(store(reply.annotations, getAnnotations(self, reply.snapshotVersion + 1, req.mostRecentVersion)));
2021-08-03 06:50:10 +08:00
TraceEvent(SevDebug, "ConfigNodeGettingSnapshot", self->id)
2021-05-19 06:28:44 +08:00
.detail("SnapshotVersion", reply.snapshotVersion)
.detail("SnapshotSize", reply.snapshot.size())
2022-02-02 14:27:12 +08:00
.detail("ChangesSize", reply.changes.size());
req.reply.send(reply);
return Void();
}
// Apply mutations from the WAL in mutationKeys into the kvKeys key space.
// Periodic compaction prevents the database from growing too large, and improve read performance.
// However, commit annotations for compacted mutations are lost
ACTOR static Future<Void> compact(ConfigNodeImpl* self, ConfigFollowerCompactRequest req) {
state Version lastCompactedVersion = wait(getLastCompactedVersion(self));
TraceEvent(SevInfo, "ConfigNodeCompacting", self->id)
.detail("Version", req.version)
.detail("LastCompacted", lastCompactedVersion);
if (req.version <= lastCompactedVersion) {
req.reply.send(Void());
return Void();
}
Standalone<VectorRef<VersionedConfigMutationRef>> versionedMutations =
wait(getMutations(self, lastCompactedVersion + 1, req.version));
self->kvStore->clear(
KeyRangeRef(versionedMutationKey(lastCompactedVersion + 1, 0), versionedMutationKey(req.version + 1, 0)));
self->kvStore->clear(
KeyRangeRef(versionedAnnotationKey(lastCompactedVersion + 1), versionedAnnotationKey(req.version + 1)));
for (const auto& versionedMutation : versionedMutations) {
const auto& version = versionedMutation.version;
const auto& mutation = versionedMutation.mutation;
if (version > req.version) {
break;
} else {
2021-08-03 06:50:10 +08:00
TraceEvent(SevDebug, "ConfigNodeCompactionApplyingMutation", self->id)
.detail("IsSet", mutation.isSet())
.detail("MutationVersion", version)
.detail("LastCompactedVersion", lastCompactedVersion)
.detail("ReqVersion", req.version);
auto serializedKey = BinaryWriter::toValue(mutation.getKey(), IncludeVersion());
if (mutation.isSet()) {
self->kvStore->set(KeyValueRef(serializedKey.withPrefix(kvKeys.begin),
ObjectWriter::toValue(mutation.getValue(), IncludeVersion())));
} else {
self->kvStore->clear(singleKeyRange(serializedKey.withPrefix(kvKeys.begin)));
}
lastCompactedVersion = version;
}
}
self->kvStore->set(
KeyValueRef(lastCompactedVersionKey, BinaryWriter::toValue(lastCompactedVersion, IncludeVersion())));
wait(self->kvStore->commit());
req.reply.send(Void());
return Void();
}
ACTOR static Future<Void> rollforward(ConfigNodeImpl* self, ConfigFollowerRollforwardRequest req) {
state Version lastCompactedVersion = wait(getLastCompactedVersion(self));
if (req.lastKnownCommitted < lastCompactedVersion) {
req.reply.sendError(version_already_compacted());
return Void();
}
state ConfigGeneration currentGeneration = wait(getGeneration(self));
2021-08-28 08:52:33 +08:00
if (req.lastKnownCommitted != currentGeneration.committedVersion) {
req.reply.sendError(transaction_too_old());
return Void();
}
TraceEvent("ConfigNodeRollforward", self->id)
2022-02-02 14:27:12 +08:00
.detail("RollbackTo", req.rollback)
.detail("Target", req.target)
.detail("LastKnownCommitted", req.lastKnownCommitted)
.detail("Committed", currentGeneration.committedVersion)
.detail("CurrentGeneration", currentGeneration.toString())
.detail("LastCompactedVersion", lastCompactedVersion);
// Rollback to prior known committed version to erase any commits not
// made on a quorum.
if (req.rollback.present() && req.rollback.get() < currentGeneration.committedVersion) {
if (g_network->isSimulated()) {
2021-10-26 01:17:22 +08:00
RangeResult mutationRange = wait(self->kvStore->readRange(
KeyRangeRef(versionedMutationKey(req.rollback.get() + 1, 0),
versionedMutationKey(currentGeneration.committedVersion + 1, 0))));
// assertCommitted(mutationRange, req.mutations, getVersionFromVersionedMutationKey);
RangeResult annotationRange = wait(self->kvStore->readRange(
KeyRangeRef(versionedAnnotationKey(req.rollback.get() + 1),
versionedAnnotationKey(currentGeneration.committedVersion + 1))));
// assertCommitted(annotationRange, req.annotations, getVersionFromVersionedAnnotationKey);
}
2021-10-26 01:17:22 +08:00
self->kvStore->clear(KeyRangeRef(versionedMutationKey(req.rollback.get() + 1, 0),
versionedMutationKey(currentGeneration.committedVersion + 1, 0)));
self->kvStore->clear(KeyRangeRef(versionedAnnotationKey(req.rollback.get() + 1),
versionedAnnotationKey(currentGeneration.committedVersion + 1)));
currentGeneration.committedVersion = req.rollback.get();
if (req.rollback.get() < lastCompactedVersion) {
self->kvStore->set(
KeyValueRef(lastCompactedVersionKey, BinaryWriter::toValue(req.rollback.get(), IncludeVersion())));
}
// The mutation commit loop below should persist the new generation
// to disk, so we don't need to do it here.
}
// Now rollforward by applying all mutations between last known
// committed version and rollforward version.
if (req.mutations.size() > 0) {
ASSERT_GT(req.mutations.size(), 0);
ASSERT_GT(req.mutations[0].version, currentGeneration.committedVersion);
wait(commitMutations(self, req.mutations, req.annotations, req.target));
}
req.reply.send(Void());
return Void();
}
ACTOR static Future<Void> getCommittedVersion(ConfigNodeImpl* self, ConfigFollowerGetCommittedVersionRequest req) {
state Version lastCompacted = wait(getLastCompactedVersion(self));
state ConfigGeneration generation = wait(getGeneration(self));
bool isRegistered = wait(registered(self));
req.reply.send(ConfigFollowerGetCommittedVersionReply{
isRegistered, lastCompacted, generation.liveVersion, generation.committedVersion });
return Void();
}
// Requires ConfigNodes to register with the ConfigBroadcaster before being
// allowed to respond to most requests. The ConfigBroadcaster will first
// ask the ConfigNode whether it is registered (kickstarted by the worker
// registering with the cluster controller). Then, the ConfigBroadcaster
// will send the ConfigNode a ready message, containing a snapshot if the
// ConfigNode is a new coordinator and needs updated state, or empty
// otherwise.
ACTOR static Future<Void> serve(ConfigNodeImpl* self, ConfigBroadcastInterface const* cbi, bool infinite) {
loop {
// Normally, the ConfigBroadcaster will first send a
// ConfigBroadcastRegisteredRequest, followed by a
// ConfigBroadcastReadyRequest. However, if the cluster controller
// the broadcaster is running on fails in between sending these two
// messages, the new broadcaster may need to resend its registered
// request. So we need to listen for either message in a loop to
// guarantee no pattern of failure will result in a stuck
// ConfigNode.
choose {
when(state ConfigBroadcastRegisteredRequest req = waitNext(cbi->registered.getFuture())) {
state bool isRegistered = wait(registered(self));
ConfigGeneration generation = wait(getGeneration(self));
TraceEvent("ConfigNodeSendingRegisteredReply", self->id)
.detail("Generation", generation.toString());
req.reply.send(ConfigBroadcastRegisteredReply{ isRegistered, generation.committedVersion });
}
when(state ConfigBroadcastReadyRequest readyReq = waitNext(cbi->ready.getFuture())) {
state Optional<size_t> locked = wait(getLocked(self));
// New ConfigNodes with no previous state should always
// apply snapshots from the ConfigBroadcaster. Otherwise,
// the ConfigNode must be part of a new generation to
// accept a snapshot. An existing ConfigNode that restarts
// shouldn't apply a snapshot and overwrite its state if
// the set of coordinators hasn't changed.
if ((!infinite && !locked.present()) ||
(locked.present() && locked.get() != readyReq.coordinatorsHash)) {
// Apply snapshot if necessary.
if (readyReq.snapshot.size() > 0) {
for (const auto& [configKey, knobValue] : readyReq.snapshot) {
TraceEvent("ConfigNodeSettingFromSnapshot", self->id)
.detail("ConfigClass", configKey.configClass)
.detail("KnobName", configKey.knobName)
.detail("Value", knobValue.toString())
.detail("Version", readyReq.snapshotVersion);
self->kvStore->set(KeyValueRef(
BinaryWriter::toValue(configKey, IncludeVersion()).withPrefix(kvKeys.begin),
ObjectWriter::toValue(knobValue, IncludeVersion())));
}
ConfigGeneration newGeneration = { readyReq.snapshotVersion, readyReq.liveVersion };
self->kvStore->set(KeyValueRef(currentGenerationKey,
BinaryWriter::toValue(newGeneration, IncludeVersion())));
// Clear out any mutations to the keys. If these
// aren't cleared, they will overwrite the
// snapshotted values when the knobs are read.
self->kvStore->clear(KeyRangeRef(versionedMutationKey(0, 0),
versionedMutationKey(readyReq.snapshotVersion + 1, 0)));
self->kvStore->clear(KeyRangeRef(versionedAnnotationKey(0),
versionedAnnotationKey(readyReq.snapshotVersion + 1)));
self->kvStore->set(
KeyValueRef(lastCompactedVersionKey,
BinaryWriter::toValue(readyReq.snapshotVersion, IncludeVersion())));
}
// Make sure freshly up to date ConfigNode isn't
// locked! This is possible if it was a coordinator in
// a previous generation.
self->kvStore->set(
KeyValueRef(lockedKey, BinaryWriter::toValue(Optional<size_t>(), IncludeVersion())));
}
self->kvStore->set(KeyValueRef(coordinatorsHashKey,
BinaryWriter::toValue(readyReq.coordinatorsHash, IncludeVersion())));
wait(self->kvStore->commit());
TraceEvent("ConfigNodeReady", self->id).detail("WasLocked", locked.present());
readyReq.reply.send(ConfigBroadcastReadyReply{});
if (!infinite) {
return Void();
}
}
}
}
}
ACTOR static Future<Void> serveRegistered(ConfigNodeImpl* self, ConfigFollowerInterface const* cfi) {
loop {
choose {
when(ConfigFollowerCompactRequest req = waitNext(cfi->compact.getFuture())) {
++self->compactRequests;
wait(compact(self, req));
}
}
}
}
// Many of the ConfigNode interfaces need to be served before the
// ConfigNode is officially registered with the ConfigBroadcaster. This is
// necessary due to edge cases around coordinator changes. For example, a
// ConfigNode that loses its coordinator status but then restarts before
// serving its snapshot to the new coordinators needs to be able to
// continue serving its snapshot interface when it restarts, even though it
// is no longer a coordinator.
ACTOR static Future<Void> serveUnregistered(ConfigNodeImpl* self, ConfigFollowerInterface const* cfi) {
loop {
choose {
2021-05-19 06:28:44 +08:00
when(ConfigFollowerGetSnapshotAndChangesRequest req =
waitNext(cfi->getSnapshotAndChanges.getFuture())) {
2021-05-14 07:39:58 +08:00
++self->snapshotRequests;
2021-05-19 06:28:44 +08:00
wait(getSnapshotAndChanges(self, req));
}
when(ConfigFollowerGetChangesRequest req = waitNext(cfi->getChanges.getFuture())) {
wait(getChanges(self, req));
}
when(ConfigFollowerRollforwardRequest req = waitNext(cfi->rollforward.getFuture())) {
++self->rollforwardRequests;
wait(rollforward(self, req));
}
when(ConfigFollowerGetCommittedVersionRequest req = waitNext(cfi->getCommittedVersion.getFuture())) {
++self->getCommittedVersionRequests;
wait(getCommittedVersion(self, req));
}
when(state ConfigFollowerLockRequest req = waitNext(cfi->lock.getFuture())) {
++self->lockRequests;
Optional<size_t> coordinatorsHash = wait(getCoordinatorsHash(self));
if (!coordinatorsHash.present() || coordinatorsHash.get() == req.coordinatorsHash) {
TraceEvent("ConfigNodeLocking", self->id).log();
self->kvStore->set(KeyValueRef(registeredKey, BinaryWriter::toValue(false, IncludeVersion())));
self->kvStore->set(KeyValueRef(
lockedKey,
BinaryWriter::toValue(Optional<size_t>(req.coordinatorsHash), IncludeVersion())));
wait(self->kvStore->commit());
2022-02-02 14:27:12 +08:00
}
req.reply.send(Void());
2022-02-02 14:27:12 +08:00
}
when(wait(self->kvStore->getError())) { ASSERT(false); }
2022-02-02 14:27:12 +08:00
}
}
}
ACTOR static Future<Void> serve(ConfigNodeImpl* self,
ConfigBroadcastInterface const* cbi,
ConfigTransactionInterface const* cti,
ConfigFollowerInterface const* cfi) {
state Future<Void> serveUnregisteredFuture = serveUnregistered(self, cfi);
2022-02-02 14:27:12 +08:00
wait(serve(self, cbi, false));
self->kvStore->set(KeyValueRef(registeredKey, BinaryWriter::toValue(true, IncludeVersion())));
wait(self->kvStore->commit());
// Shouldn't return (coordinationServer will throw an error if it does).
wait(serve(self, cbi, true) || serve(self, cti) || serveRegistered(self, cfi) || serveUnregisteredFuture);
2022-02-02 14:27:12 +08:00
return Void();
}
ACTOR static Future<bool> registered(ConfigNodeImpl* self) {
Optional<Value> value = wait(self->kvStore->readValue(registeredKey));
return value.present();
}
2021-04-15 13:06:37 +08:00
public:
ConfigNodeImpl(std::string const& folder)
: id(deterministicRandom()->randomUniqueID()), kvStore(folder, id, "globalconf-"), cc("ConfigNode"),
compactRequests("CompactRequests", cc), rollbackRequests("RollbackRequests", cc),
2021-08-25 08:31:28 +08:00
rollforwardRequests("RollforwardRequests", cc), successfulChangeRequests("SuccessfulChangeRequests", cc),
failedChangeRequests("FailedChangeRequests", cc), snapshotRequests("SnapshotRequests", cc),
getCommittedVersionRequests("GetCommittedVersionRequests", cc), lockRequests("LockRequests", cc),
successfulCommits("SuccessfulCommits", cc), failedCommits("FailedCommits", cc),
setMutations("SetMutations", cc), clearMutations("ClearMutations", cc),
2021-08-07 06:20:12 +08:00
getValueRequests("GetValueRequests", cc), getGenerationRequests("GetGenerationRequests", cc) {
2021-08-03 06:50:10 +08:00
logger = traceCounters("ConfigNodeMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ConfigNode");
TraceEvent(SevInfo, "StartingConfigNode", id).detail("KVStoreAlreadyExists", kvStore.exists());
}
2022-02-02 14:27:12 +08:00
Future<Void> serve(ConfigBroadcastInterface const& cbi,
ConfigTransactionInterface const& cti,
ConfigFollowerInterface const& cfi) {
return serve(this, &cbi, &cti, &cfi);
}
Future<Void> serve(ConfigTransactionInterface const& cti) { return serve(this, &cti); }
Future<Void> serve(ConfigFollowerInterface const& cfi) {
return serveUnregistered(this, &cfi) && serveRegistered(this, &cfi);
}
2021-09-10 04:50:07 +08:00
void close() { kvStore.close(); }
Future<Void> onClosed() { return kvStore.onClosed(); }
2021-04-15 13:06:37 +08:00
};
ConfigNode::ConfigNode(std::string const& folder) : impl(PImpl<ConfigNodeImpl>::create(folder)) {}
2021-04-15 13:06:37 +08:00
ConfigNode::~ConfigNode() = default;
2021-04-15 13:06:37 +08:00
Future<Void> ConfigNode::serve(ConfigBroadcastInterface const& cbi,
ConfigTransactionInterface const& cti,
ConfigFollowerInterface const& cfi) {
return impl->serve(cbi, cti, cfi);
}
Future<Void> ConfigNode::serve(ConfigTransactionInterface const& cti) {
return impl->serve(cti);
}
Future<Void> ConfigNode::serve(ConfigFollowerInterface const& cfi) {
return impl->serve(cfi);
2021-04-15 13:06:37 +08:00
}
2021-09-10 04:50:07 +08:00
void ConfigNode::close() {
impl->close();
}
Future<Void> ConfigNode::onClosed() {
return impl->onClosed();
}