2019-02-08 09:02:29 +08:00
|
|
|
/*
|
|
|
|
* TLogServer.actor.cpp
|
|
|
|
*
|
|
|
|
* This source file is part of the FoundationDB open source project
|
|
|
|
*
|
|
|
|
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
|
|
|
|
*
|
|
|
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
|
|
|
* you may not use this file except in compliance with the License.
|
|
|
|
* You may obtain a copy of the License at
|
|
|
|
*
|
|
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
*
|
|
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
|
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
* See the License for the specific language governing permissions and
|
|
|
|
* limitations under the License.
|
|
|
|
*/
|
|
|
|
|
|
|
|
#include "flow/Hash3.h"
|
|
|
|
#include "flow/UnitTest.h"
|
2019-02-20 14:02:07 +08:00
|
|
|
#include "fdbclient/NativeAPI.actor.h"
|
2019-02-08 09:02:29 +08:00
|
|
|
#include "fdbclient/Notified.h"
|
|
|
|
#include "fdbclient/KeyRangeMap.h"
|
2019-04-24 07:17:54 +08:00
|
|
|
#include "fdbclient/RunTransaction.actor.h"
|
2019-04-25 03:45:28 +08:00
|
|
|
#include "fdbclient/SystemData.h"
|
2019-02-20 14:02:07 +08:00
|
|
|
#include "fdbserver/WorkerInterface.actor.h"
|
2019-02-08 09:02:29 +08:00
|
|
|
#include "fdbserver/TLogInterface.h"
|
|
|
|
#include "fdbserver/Knobs.h"
|
|
|
|
#include "fdbserver/IKeyValueStore.h"
|
|
|
|
#include "flow/ActorCollection.h"
|
|
|
|
#include "fdbrpc/FailureMonitor.h"
|
|
|
|
#include "fdbserver/IDiskQueue.h"
|
|
|
|
#include "fdbrpc/sim_validation.h"
|
2019-03-12 03:11:17 +08:00
|
|
|
#include "fdbrpc/simulator.h"
|
2020-07-10 07:39:15 +08:00
|
|
|
#include "fdbrpc/Stats.h"
|
2019-02-08 09:02:29 +08:00
|
|
|
#include "fdbserver/ServerDBInfo.h"
|
|
|
|
#include "fdbserver/LogSystem.h"
|
|
|
|
#include "fdbserver/WaitFailure.h"
|
|
|
|
#include "fdbserver/RecoveryState.h"
|
2019-04-21 03:58:24 +08:00
|
|
|
#include "fdbserver/FDBExecHelper.actor.h"
|
2021-03-11 02:06:03 +08:00
|
|
|
#include "flow/actorcompiler.h" // This must be the last #include.
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
using std::make_pair;
|
|
|
|
using std::max;
|
2021-03-11 02:06:03 +08:00
|
|
|
using std::min;
|
|
|
|
using std::pair;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
namespace oldTLog_6_0 {
|
|
|
|
|
|
|
|
struct TLogQueueEntryRef {
|
|
|
|
UID id;
|
|
|
|
Version version;
|
|
|
|
Version knownCommittedVersion;
|
|
|
|
StringRef messages;
|
|
|
|
|
|
|
|
TLogQueueEntryRef() : version(0), knownCommittedVersion(0) {}
|
2021-03-11 02:06:03 +08:00
|
|
|
TLogQueueEntryRef(Arena& a, TLogQueueEntryRef const& from)
|
|
|
|
: version(from.version), knownCommittedVersion(from.knownCommittedVersion), id(from.id),
|
|
|
|
messages(a, from.messages) {}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
template <class Ar>
|
|
|
|
void serialize(Ar& ar) {
|
|
|
|
serializer(ar, version, messages, knownCommittedVersion, id);
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
size_t expectedSize() const { return messages.expectedSize(); }
|
2019-02-08 09:02:29 +08:00
|
|
|
};
|
|
|
|
|
|
|
|
struct AlternativeTLogQueueEntryRef {
|
|
|
|
UID id;
|
|
|
|
Version version;
|
|
|
|
Version knownCommittedVersion;
|
|
|
|
std::vector<TagsAndMessage>* alternativeMessages;
|
|
|
|
|
2020-08-19 05:18:50 +08:00
|
|
|
AlternativeTLogQueueEntryRef() : version(0), knownCommittedVersion(0), alternativeMessages(nullptr) {}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
template <class Ar>
|
|
|
|
void serialize(Ar& ar) {
|
|
|
|
ASSERT(!ar.isDeserializing && alternativeMessages);
|
|
|
|
uint32_t msgSize = expectedSize();
|
|
|
|
serializer(ar, version, msgSize);
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& msg : *alternativeMessages) {
|
|
|
|
ar.serializeBytes(msg.message);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
serializer(ar, knownCommittedVersion, id);
|
|
|
|
}
|
|
|
|
|
|
|
|
uint32_t expectedSize() const {
|
|
|
|
uint32_t msgSize = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& msg : *alternativeMessages) {
|
2019-02-08 09:02:29 +08:00
|
|
|
msgSize += msg.message.size();
|
|
|
|
}
|
|
|
|
return msgSize;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
typedef Standalone<TLogQueueEntryRef> TLogQueueEntry;
|
|
|
|
struct LogData;
|
|
|
|
struct TLogData;
|
|
|
|
|
|
|
|
struct TLogQueue : public IClosable {
|
|
|
|
public:
|
2021-03-11 02:06:03 +08:00
|
|
|
TLogQueue(IDiskQueue* queue, UID dbgid) : queue(queue), dbgid(dbgid) {}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
// Each packet in the queue is
|
|
|
|
// uint32_t payloadSize
|
|
|
|
// uint8_t payload[payloadSize] (begins with uint64_t protocolVersion via IncludeVersion)
|
|
|
|
// uint8_t validFlag
|
|
|
|
|
|
|
|
// TLogQueue is a durable queue of TLogQueueEntry objects with an interface similar to IDiskQueue
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// TLogQueue pushes (but not commits) are atomic - after commit fails to return, a prefix of entire calls to push
|
|
|
|
// are durable. This is
|
|
|
|
// implemented on top of the weaker guarantee of IDiskQueue::commit (that a prefix of bytes is durable) using
|
|
|
|
// validFlag and by padding any incomplete packet with zeros after recovery.
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
// Before calling push, pop, or commit, the user must call readNext() until it throws
|
|
|
|
// end_of_stream(). It may not be called again thereafter.
|
2021-03-11 02:06:03 +08:00
|
|
|
Future<TLogQueueEntry> readNext(TLogData* tLog) { return readNext(this, tLog); }
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
template <class T>
|
2021-03-11 02:06:03 +08:00
|
|
|
void push(T const& qe, Reference<LogData> logData);
|
|
|
|
void pop(Version upTo, Reference<LogData> logData);
|
2019-02-08 09:02:29 +08:00
|
|
|
Future<Void> commit() { return queue->commit(); }
|
|
|
|
|
|
|
|
// Implements IClosable
|
2021-01-26 09:55:43 +08:00
|
|
|
Future<Void> getError() override { return queue->getError(); }
|
|
|
|
Future<Void> onClosed() override { return queue->onClosed(); }
|
|
|
|
void dispose() override {
|
|
|
|
queue->dispose();
|
|
|
|
delete this;
|
|
|
|
}
|
|
|
|
void close() override {
|
|
|
|
queue->close();
|
|
|
|
delete this;
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
private:
|
|
|
|
IDiskQueue* queue;
|
|
|
|
UID dbgid;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void updateVersionSizes(const TLogQueueEntry& result, TLogData* tLog);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR static Future<TLogQueueEntry> readNext(TLogQueue* self, TLogData* tLog) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state TLogQueueEntry result;
|
|
|
|
state int zeroFillSize = 0;
|
|
|
|
|
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
Standalone<StringRef> h = wait(self->queue->readNext(sizeof(uint32_t)));
|
2019-02-08 09:02:29 +08:00
|
|
|
if (h.size() != sizeof(uint32_t)) {
|
|
|
|
if (h.size()) {
|
2021-03-11 02:06:03 +08:00
|
|
|
TEST(true); // Zero fill within size field
|
2019-02-08 09:02:29 +08:00
|
|
|
int payloadSize = 0;
|
|
|
|
memcpy(&payloadSize, h.begin(), h.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
zeroFillSize = sizeof(uint32_t) - h.size(); // zero fill the size itself
|
|
|
|
zeroFillSize += payloadSize + 1; // and then the contents and valid flag
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
state uint32_t payloadSize = *(uint32_t*)h.begin();
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(payloadSize < (100 << 20));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
Standalone<StringRef> e = wait(self->queue->readNext(payloadSize + 1));
|
|
|
|
if (e.size() != payloadSize + 1) {
|
|
|
|
TEST(true); // Zero fill within payload
|
|
|
|
zeroFillSize = payloadSize + 1 - e.size();
|
2019-02-08 09:02:29 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (e[payloadSize]) {
|
|
|
|
Arena a = e.arena();
|
2021-03-11 02:06:03 +08:00
|
|
|
ArenaReader ar(a, e.substr(0, payloadSize), IncludeVersion());
|
2019-02-08 09:02:29 +08:00
|
|
|
ar >> result;
|
|
|
|
self->updateVersionSizes(result, tLog);
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (zeroFillSize) {
|
2021-03-11 02:06:03 +08:00
|
|
|
TEST(true); // Fixing a partial commit at the end of the tlog queue
|
|
|
|
for (int i = 0; i < zeroFillSize; i++)
|
|
|
|
self->queue->push(StringRef((const uint8_t*)"", 1));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
throw end_of_stream();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
////// Persistence format (for self->persistentData)
|
|
|
|
|
|
|
|
// Immutable keys
|
2021-03-11 02:06:03 +08:00
|
|
|
static const KeyValueRef persistFormat(LiteralStringRef("Format"), LiteralStringRef("FoundationDB/LogServer/2/4"));
|
|
|
|
static const KeyRangeRef persistFormatReadableRange(LiteralStringRef("FoundationDB/LogServer/2/3"),
|
|
|
|
LiteralStringRef("FoundationDB/LogServer/2/5"));
|
|
|
|
static const KeyRangeRef persistRecoveryCountKeys =
|
|
|
|
KeyRangeRef(LiteralStringRef("DbRecoveryCount/"), LiteralStringRef("DbRecoveryCount0"));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
// Updated on updatePersistentData()
|
2021-03-11 02:06:03 +08:00
|
|
|
static const KeyRangeRef persistCurrentVersionKeys =
|
|
|
|
KeyRangeRef(LiteralStringRef("version/"), LiteralStringRef("version0"));
|
|
|
|
static const KeyRangeRef persistKnownCommittedVersionKeys =
|
|
|
|
KeyRangeRef(LiteralStringRef("knownCommitted/"), LiteralStringRef("knownCommitted0"));
|
|
|
|
static const KeyRangeRef persistLocalityKeys =
|
|
|
|
KeyRangeRef(LiteralStringRef("Locality/"), LiteralStringRef("Locality0"));
|
|
|
|
static const KeyRangeRef persistLogRouterTagsKeys =
|
|
|
|
KeyRangeRef(LiteralStringRef("LogRouterTags/"), LiteralStringRef("LogRouterTags0"));
|
|
|
|
static const KeyRangeRef persistTxsTagsKeys = KeyRangeRef(LiteralStringRef("TxsTags/"), LiteralStringRef("TxsTags0"));
|
2019-02-08 09:02:29 +08:00
|
|
|
static const KeyRange persistTagMessagesKeys = prefixRange(LiteralStringRef("TagMsg/"));
|
|
|
|
static const KeyRange persistTagPoppedKeys = prefixRange(LiteralStringRef("TagPop/"));
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static Key persistTagMessagesKey(UID id, Tag tag, Version version) {
|
|
|
|
BinaryWriter wr(Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
wr.serializeBytes(persistTagMessagesKeys.begin);
|
|
|
|
wr << id;
|
|
|
|
wr << tag;
|
2021-03-11 02:06:03 +08:00
|
|
|
wr << bigEndian64(version);
|
2019-03-29 02:52:50 +08:00
|
|
|
return wr.toValue();
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static Key persistTagPoppedKey(UID id, Tag tag) {
|
2019-02-08 09:02:29 +08:00
|
|
|
BinaryWriter wr(Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
wr.serializeBytes(persistTagPoppedKeys.begin);
|
2019-02-08 09:02:29 +08:00
|
|
|
wr << id;
|
|
|
|
wr << tag;
|
2019-03-29 02:52:50 +08:00
|
|
|
return wr.toValue();
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static Value persistTagPoppedValue(Version popped) {
|
|
|
|
return BinaryWriter::toValue(popped, Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static Tag decodeTagPoppedKey(KeyRef id, KeyRef key) {
|
2019-02-08 09:02:29 +08:00
|
|
|
Tag s;
|
2021-03-11 02:06:03 +08:00
|
|
|
BinaryReader rd(key.removePrefix(persistTagPoppedKeys.begin).removePrefix(id), Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
rd >> s;
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static Version decodeTagPoppedValue(ValueRef value) {
|
|
|
|
return BinaryReader::fromStringRef<Version>(value, Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static StringRef stripTagMessagesKey(StringRef key) {
|
|
|
|
return key.substr(sizeof(UID) + sizeof(Tag) + persistTagMessagesKeys.begin.size());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
static Version decodeTagMessagesKey(StringRef key) {
|
|
|
|
return bigEndian64(BinaryReader::fromStringRef<Version>(stripTagMessagesKey(key), Unversioned()));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
struct TLogData : NonCopyable {
|
|
|
|
AsyncTrigger newLogData;
|
|
|
|
Deque<UID> queueOrder;
|
|
|
|
std::map<UID, Reference<struct LogData>> id_data;
|
|
|
|
|
|
|
|
UID dbgid;
|
2020-02-13 07:11:38 +08:00
|
|
|
UID workerID;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
IKeyValueStore* persistentData;
|
|
|
|
IDiskQueue* rawPersistentQueue;
|
2021-03-11 02:06:03 +08:00
|
|
|
TLogQueue* persistentQueue;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
int64_t diskQueueCommitBytes;
|
2021-03-11 02:06:03 +08:00
|
|
|
AsyncVar<bool>
|
|
|
|
largeDiskQueueCommitBytes; // becomes true when diskQueueCommitBytes is greater than MAX_QUEUE_COMMIT_BYTES
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
Reference<AsyncVar<ServerDBInfo>> dbInfo;
|
2019-04-24 07:17:54 +08:00
|
|
|
Database cx;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
NotifiedVersion queueCommitEnd;
|
|
|
|
Version queueCommitBegin;
|
|
|
|
|
|
|
|
int64_t instanceID;
|
|
|
|
int64_t bytesInput;
|
|
|
|
int64_t bytesDurable;
|
2021-03-11 02:06:03 +08:00
|
|
|
int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling.
|
2019-02-08 09:02:29 +08:00
|
|
|
int64_t overheadBytesInput;
|
|
|
|
int64_t overheadBytesDurable;
|
|
|
|
|
|
|
|
WorkerCache<TLogInterface> tlogCache;
|
|
|
|
|
|
|
|
PromiseStream<Future<Void>> sharedActors;
|
|
|
|
Promise<Void> terminated;
|
|
|
|
FlowLock concurrentLogRouterReads;
|
|
|
|
FlowLock persistentDataCommitLock;
|
|
|
|
|
2019-02-28 07:40:33 +08:00
|
|
|
bool ignorePopRequest; // ignore pop request from storage servers
|
|
|
|
double ignorePopDeadline; // time until which the ignorePopRequest will be
|
|
|
|
// honored
|
|
|
|
std::string ignorePopUid; // callers that set ignorePopRequest will set this
|
|
|
|
// extra state, used to validate the ownership of
|
|
|
|
// the set and for callers that unset will
|
|
|
|
// be able to match it up
|
|
|
|
std::string dataFolder; // folder where data is stored
|
2019-04-03 20:47:19 +08:00
|
|
|
std::map<Tag, Version> toBePopped; // map of Tag->Version for all the pops
|
2021-03-11 02:06:03 +08:00
|
|
|
// that came when ignorePopRequest was set
|
2019-03-20 06:12:47 +08:00
|
|
|
Reference<AsyncVar<bool>> degraded;
|
2019-11-06 10:07:30 +08:00
|
|
|
std::vector<TagsAndMessage> tempTagMessages;
|
2019-03-09 00:46:34 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TLogData(UID dbgid,
|
|
|
|
UID workerID,
|
|
|
|
IKeyValueStore* persistentData,
|
|
|
|
IDiskQueue* persistentQueue,
|
|
|
|
Reference<AsyncVar<ServerDBInfo>> dbInfo,
|
|
|
|
Reference<AsyncVar<bool>> degraded,
|
|
|
|
std::string folder)
|
|
|
|
: dbgid(dbgid), workerID(workerID), instanceID(deterministicRandom()->randomUniqueID().first()),
|
|
|
|
persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
|
|
|
persistentQueue(new TLogQueue(persistentQueue, dbgid)), dbInfo(dbInfo), degraded(degraded), queueCommitBegin(0),
|
|
|
|
queueCommitEnd(0), diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0),
|
|
|
|
targetVolatileBytes(SERVER_KNOBS->TLOG_SPILL_THRESHOLD), overheadBytesInput(0), overheadBytesDurable(0),
|
|
|
|
concurrentLogRouterReads(SERVER_KNOBS->CONCURRENT_LOG_ROUTER_READS), ignorePopRequest(false),
|
|
|
|
ignorePopDeadline(), ignorePopUid(), dataFolder(folder), toBePopped() {
|
2021-07-03 06:04:42 +08:00
|
|
|
cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
};
|
|
|
|
|
|
|
|
struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|
|
|
struct TagData : NonCopyable, public ReferenceCounted<TagData> {
|
|
|
|
std::deque<std::pair<Version, LengthPrefixedStringRef>> versionMessages;
|
2021-03-11 02:06:03 +08:00
|
|
|
bool
|
|
|
|
nothingPersistent; // true means tag is *known* to have no messages in persistentData. false means nothing.
|
|
|
|
bool poppedRecently; // `popped` has changed since last updatePersistentData
|
|
|
|
Version popped; // see popped version tracking contract below
|
2019-02-08 09:02:29 +08:00
|
|
|
bool unpoppedRecovered;
|
|
|
|
Tag tag;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TagData(Tag tag, Version popped, bool nothingPersistent, bool poppedRecently, bool unpoppedRecovered)
|
|
|
|
: tag(tag), nothingPersistent(nothingPersistent), popped(popped), poppedRecently(poppedRecently),
|
|
|
|
unpoppedRecovered(unpoppedRecovered) {}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2020-06-10 08:33:41 +08:00
|
|
|
TagData(TagData&& r) noexcept
|
|
|
|
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
|
|
|
|
poppedRecently(r.poppedRecently), popped(r.popped), tag(r.tag), unpoppedRecovered(r.unpoppedRecovered) {}
|
|
|
|
void operator=(TagData&& r) noexcept {
|
2019-02-08 09:02:29 +08:00
|
|
|
versionMessages = std::move(r.versionMessages);
|
|
|
|
nothingPersistent = r.nothingPersistent;
|
|
|
|
poppedRecently = r.poppedRecently;
|
|
|
|
popped = r.popped;
|
|
|
|
tag = r.tag;
|
|
|
|
unpoppedRecovered = r.unpoppedRecovered;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Erase messages not needed to update *from* versions >= before (thus, messages with toversion <= before)
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> eraseMessagesBefore(TagData* self,
|
|
|
|
Version before,
|
|
|
|
TLogData* tlogData,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
TaskPriority taskID) {
|
|
|
|
while (!self->versionMessages.empty() && self->versionMessages.front().first < before) {
|
2019-02-08 09:02:29 +08:00
|
|
|
Version version = self->versionMessages.front().first;
|
2021-03-11 02:06:03 +08:00
|
|
|
std::pair<int, int>& sizes = logData->version_sizes[version];
|
2019-02-08 09:02:29 +08:00
|
|
|
int64_t messagesErased = 0;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
while (!self->versionMessages.empty() && self->versionMessages.front().first == version) {
|
2019-02-08 09:02:29 +08:00
|
|
|
auto const& m = self->versionMessages.front();
|
|
|
|
++messagesErased;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->tag.locality != tagLocalityTxs && self->tag != txsTag) {
|
2019-02-08 09:02:29 +08:00
|
|
|
sizes.first -= m.second.expectedSize();
|
|
|
|
} else {
|
|
|
|
sizes.second -= m.second.expectedSize();
|
|
|
|
}
|
|
|
|
|
|
|
|
self->versionMessages.pop_front();
|
|
|
|
}
|
|
|
|
|
|
|
|
int64_t bytesErased = messagesErased * SERVER_KNOBS->VERSION_MESSAGES_ENTRY_BYTES_WITH_OVERHEAD;
|
|
|
|
logData->bytesDurable += bytesErased;
|
|
|
|
tlogData->bytesDurable += bytesErased;
|
|
|
|
tlogData->overheadBytesDurable += bytesErased;
|
|
|
|
wait(yield(taskID));
|
|
|
|
}
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
Future<Void> eraseMessagesBefore(Version before,
|
|
|
|
TLogData* tlogData,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
TaskPriority taskID) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return eraseMessagesBefore(this, before, tlogData, logData, taskID);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
Map<Version, IDiskQueue::location>
|
|
|
|
versionLocation; // For the version of each entry that was push()ed, the end location of the serialized bytes
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
/*
|
|
|
|
Popped version tracking contract needed by log system to implement ILogCursor::popped():
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
- Log server tracks for each (possible) tag a popped_version
|
|
|
|
Impl: TagData::popped (in memory) and persistTagPoppedKeys (in persistentData)
|
|
|
|
- popped_version(tag) is <= the maximum version for which log server (or a predecessor) is ever asked to pop the
|
|
|
|
tag Impl: Only increased by tLogPop() in response to either a pop request or recovery from a predecessor
|
|
|
|
- popped_version(tag) is > the maximum version for which log server is unable to peek messages due to previous
|
|
|
|
pops (on this server or a predecessor) Impl: Increased by tLogPop() atomically with erasing messages from memory;
|
|
|
|
persisted by updatePersistentData() atomically with erasing messages from store; messages are not erased from queue
|
|
|
|
where popped_version is not persisted
|
|
|
|
- LockTLogReply returns all tags which either have messages, or which have nonzero popped_versions
|
|
|
|
Impl: tag_data is present for all such tags
|
|
|
|
- peek(tag, v) returns the popped_version for tag if that is greater than v
|
|
|
|
Impl: Check tag_data->popped (after all waits)
|
2019-02-08 09:02:29 +08:00
|
|
|
*/
|
|
|
|
|
|
|
|
AsyncTrigger stopCommit;
|
|
|
|
bool stopped, initialized;
|
|
|
|
DBRecoveryCount recoveryCount;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
VersionMetricHandle persistentDataVersion,
|
|
|
|
persistentDataDurableVersion; // The last version number in the portion of the log (written|durable) to
|
|
|
|
// persistentData
|
2019-02-08 09:02:29 +08:00
|
|
|
NotifiedVersion version, queueCommittedVersion;
|
|
|
|
Version queueCommittingVersion;
|
|
|
|
Version knownCommittedVersion, durableKnownCommittedVersion, minKnownCommittedVersion;
|
|
|
|
|
2020-03-11 06:05:13 +08:00
|
|
|
// Track lastUpdate time for parallel peek and detect stall on tLogs
|
2019-07-10 09:20:20 +08:00
|
|
|
struct PeekTrackerData {
|
2019-07-16 08:05:39 +08:00
|
|
|
std::map<int, Promise<std::pair<Version, bool>>> sequence_version;
|
2019-07-10 09:20:20 +08:00
|
|
|
double lastUpdate;
|
2020-04-23 14:35:48 +08:00
|
|
|
|
|
|
|
Tag tag;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-04-23 14:35:48 +08:00
|
|
|
double lastLogged;
|
|
|
|
int64_t totalPeeks;
|
|
|
|
int64_t replyBytes;
|
|
|
|
int64_t duplicatePeeks;
|
|
|
|
double queueTime;
|
|
|
|
double queueMax;
|
|
|
|
double blockTime;
|
|
|
|
double blockMax;
|
|
|
|
double workTime;
|
|
|
|
double workMax;
|
|
|
|
|
|
|
|
int64_t unblockedPeeks;
|
|
|
|
double idleTime;
|
|
|
|
double idleMax;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
PeekTrackerData() : lastUpdate(0) { resetMetrics(); }
|
2020-04-23 14:35:48 +08:00
|
|
|
|
|
|
|
void resetMetrics() {
|
|
|
|
lastLogged = now();
|
|
|
|
totalPeeks = 0;
|
|
|
|
replyBytes = 0;
|
|
|
|
duplicatePeeks = 0;
|
|
|
|
queueTime = 0;
|
|
|
|
queueMax = 0;
|
|
|
|
blockTime = 0;
|
|
|
|
blockMax = 0;
|
|
|
|
workTime = 0;
|
|
|
|
workMax = 0;
|
|
|
|
unblockedPeeks = 0;
|
|
|
|
idleTime = 0;
|
|
|
|
idleMax = 0;
|
|
|
|
}
|
2019-07-10 09:20:20 +08:00
|
|
|
};
|
|
|
|
|
|
|
|
std::map<UID, PeekTrackerData> peekTracker;
|
|
|
|
|
2019-02-08 09:02:29 +08:00
|
|
|
Deque<std::pair<Version, Standalone<VectorRef<uint8_t>>>> messageBlocks;
|
2021-03-11 02:06:03 +08:00
|
|
|
std::vector<std::vector<Reference<TagData>>> tag_data; // tag.locality | tag.id
|
2019-02-08 09:02:29 +08:00
|
|
|
int unpoppedRecoveredTags;
|
|
|
|
|
|
|
|
Reference<TagData> getTagData(Tag tag) {
|
2019-04-02 04:56:45 +08:00
|
|
|
int idx = tag.toTagDataIndex();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (idx >= tag_data.size()) {
|
|
|
|
tag_data.resize(idx + 1);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tag.id >= tag_data[idx].size()) {
|
|
|
|
tag_data[idx].resize(tag.id + 1);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
return tag_data[idx][tag.id];
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// only callable after getTagData returns a null reference
|
|
|
|
Reference<TagData> createTagData(Tag tag,
|
|
|
|
Version popped,
|
|
|
|
bool nothingPersistent,
|
|
|
|
bool poppedRecently,
|
|
|
|
bool unpoppedRecovered) {
|
|
|
|
if (tag.locality != tagLocalityLogRouter && tag.locality != tagLocalityTxs && tag != txsTag && allTags.size() &&
|
|
|
|
!allTags.count(tag) && popped <= recoveredAt) {
|
2019-02-08 09:02:29 +08:00
|
|
|
popped = recoveredAt + 1;
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
Reference<TagData> newTagData =
|
|
|
|
Reference<TagData>(new TagData(tag, popped, nothingPersistent, poppedRecently, unpoppedRecovered));
|
2019-04-02 04:56:45 +08:00
|
|
|
tag_data[tag.toTagDataIndex()][tag.id] = newTagData;
|
2019-02-08 09:02:29 +08:00
|
|
|
return newTagData;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
Map<Version, std::pair<int, int>> version_sizes;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
CounterCollection cc;
|
|
|
|
Counter bytesInput;
|
|
|
|
Counter bytesDurable;
|
|
|
|
|
|
|
|
UID logId;
|
|
|
|
Version newPersistentDataVersion;
|
|
|
|
Future<Void> removed;
|
|
|
|
PromiseStream<Future<Void>> addActor;
|
|
|
|
TLogData* tLogData;
|
|
|
|
Promise<Void> recoveryComplete, committingQueue;
|
|
|
|
Version unrecoveredBefore, recoveredAt;
|
|
|
|
|
|
|
|
Reference<AsyncVar<Reference<ILogSystem>>> logSystem;
|
|
|
|
Tag remoteTag;
|
|
|
|
bool isPrimary;
|
|
|
|
int logRouterTags;
|
|
|
|
Version logRouterPoppedVersion, logRouterPopToVersion;
|
|
|
|
int8_t locality;
|
|
|
|
UID recruitmentID;
|
|
|
|
std::set<Tag> allTags;
|
|
|
|
Future<Void> terminated;
|
2019-05-10 05:56:54 +08:00
|
|
|
FlowLock execOpLock;
|
2019-04-08 20:23:48 +08:00
|
|
|
bool execOpCommitInProgress;
|
2019-06-20 09:15:09 +08:00
|
|
|
int txsTags;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
explicit LogData(TLogData* tLogData,
|
|
|
|
TLogInterface interf,
|
|
|
|
Tag remoteTag,
|
|
|
|
bool isPrimary,
|
|
|
|
int logRouterTags,
|
|
|
|
int txsTags,
|
|
|
|
UID recruitmentID,
|
|
|
|
std::vector<Tag> tags,
|
|
|
|
std::string context)
|
|
|
|
: tLogData(tLogData), knownCommittedVersion(0), logId(interf.id()), cc("TLog", interf.id().toString()),
|
|
|
|
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), remoteTag(remoteTag), isPrimary(isPrimary),
|
|
|
|
logRouterTags(logRouterTags), txsTags(txsTags), recruitmentID(recruitmentID),
|
|
|
|
logSystem(new AsyncVar<Reference<ILogSystem>>()), logRouterPoppedVersion(0), durableKnownCommittedVersion(0),
|
|
|
|
minKnownCommittedVersion(0), allTags(tags.begin(), tags.end()), terminated(tLogData->terminated.getFuture()),
|
|
|
|
// These are initialized differently on init() or recovery
|
|
|
|
recoveryCount(), stopped(false), initialized(false), queueCommittingVersion(0),
|
|
|
|
newPersistentDataVersion(invalidVersion), unrecoveredBefore(1), recoveredAt(1), unpoppedRecoveredTags(0),
|
|
|
|
logRouterPopToVersion(0), locality(tagLocalityInvalid), execOpCommitInProgress(false) {
|
|
|
|
startRole(Role::TRANSACTION_LOG,
|
|
|
|
interf.id(),
|
|
|
|
tLogData->workerID,
|
|
|
|
{ { "SharedTLog", tLogData->dbgid.shortString() } },
|
|
|
|
context);
|
2020-05-09 07:27:57 +08:00
|
|
|
addActor.send(traceRole(Role::TRANSACTION_LOG, interf.id()));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
persistentDataVersion.init(LiteralStringRef("TLog.PersistentDataVersion"), cc.id);
|
|
|
|
persistentDataDurableVersion.init(LiteralStringRef("TLog.PersistentDataDurableVersion"), cc.id);
|
|
|
|
version.initMetric(LiteralStringRef("TLog.Version"), cc.id);
|
|
|
|
queueCommittedVersion.initMetric(LiteralStringRef("TLog.QueueCommittedVersion"), cc.id);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
specialCounter(cc, "Version", [this]() { return this->version.get(); });
|
|
|
|
specialCounter(cc, "QueueCommittedVersion", [this]() { return this->queueCommittedVersion.get(); });
|
|
|
|
specialCounter(cc, "PersistentDataVersion", [this]() { return this->persistentDataVersion; });
|
|
|
|
specialCounter(cc, "PersistentDataDurableVersion", [this]() { return this->persistentDataDurableVersion; });
|
|
|
|
specialCounter(cc, "KnownCommittedVersion", [this]() { return this->knownCommittedVersion; });
|
|
|
|
specialCounter(cc, "QueuePoppedVersion", [this]() { return this->persistentDataDurableVersion; });
|
|
|
|
specialCounter(cc, "SharedBytesInput", [tLogData]() { return tLogData->bytesInput; });
|
|
|
|
specialCounter(cc, "SharedBytesDurable", [tLogData]() { return tLogData->bytesDurable; });
|
|
|
|
specialCounter(cc, "SharedOverheadBytesInput", [tLogData]() { return tLogData->overheadBytesInput; });
|
|
|
|
specialCounter(cc, "SharedOverheadBytesDurable", [tLogData]() { return tLogData->overheadBytesDurable; });
|
|
|
|
specialCounter(
|
|
|
|
cc, "KvstoreBytesUsed", [tLogData]() { return tLogData->persistentData->getStorageBytes().used; });
|
|
|
|
specialCounter(
|
|
|
|
cc, "KvstoreBytesFree", [tLogData]() { return tLogData->persistentData->getStorageBytes().free; });
|
|
|
|
specialCounter(cc, "KvstoreBytesAvailable", [tLogData]() {
|
|
|
|
return tLogData->persistentData->getStorageBytes().available;
|
|
|
|
});
|
|
|
|
specialCounter(
|
|
|
|
cc, "KvstoreBytesTotal", [tLogData]() { return tLogData->persistentData->getStorageBytes().total; });
|
|
|
|
specialCounter(
|
|
|
|
cc, "QueueDiskBytesUsed", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().used; });
|
|
|
|
specialCounter(
|
|
|
|
cc, "QueueDiskBytesFree", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().free; });
|
|
|
|
specialCounter(cc, "QueueDiskBytesAvailable", [tLogData]() {
|
|
|
|
return tLogData->rawPersistentQueue->getStorageBytes().available;
|
|
|
|
});
|
|
|
|
specialCounter(
|
|
|
|
cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; });
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
~LogData() {
|
2021-06-23 05:50:01 +08:00
|
|
|
endRole(Role::TRANSACTION_LOG, logId, "Error", true);
|
|
|
|
|
|
|
|
if (!terminated.isReady()) {
|
|
|
|
tLogData->bytesDurable += bytesInput.getValue() - bytesDurable.getValue();
|
|
|
|
TraceEvent("TLogBytesWhenRemoved", logId)
|
|
|
|
.detail("SharedBytesInput", tLogData->bytesInput)
|
|
|
|
.detail("SharedBytesDurable", tLogData->bytesDurable)
|
|
|
|
.detail("LocalBytesInput", bytesInput.getValue())
|
|
|
|
.detail("LocalBytesDurable", bytesDurable.getValue());
|
|
|
|
|
|
|
|
ASSERT_ABORT(tLogData->bytesDurable <= tLogData->bytesInput);
|
|
|
|
|
|
|
|
Key logIdKey = BinaryWriter::toValue(logId, Unversioned());
|
|
|
|
tLogData->persistentData->clear(singleKeyRange(logIdKey.withPrefix(persistCurrentVersionKeys.begin)));
|
|
|
|
tLogData->persistentData->clear(
|
|
|
|
singleKeyRange(logIdKey.withPrefix(persistKnownCommittedVersionKeys.begin)));
|
|
|
|
tLogData->persistentData->clear(singleKeyRange(logIdKey.withPrefix(persistLocalityKeys.begin)));
|
|
|
|
tLogData->persistentData->clear(singleKeyRange(logIdKey.withPrefix(persistLogRouterTagsKeys.begin)));
|
|
|
|
tLogData->persistentData->clear(singleKeyRange(logIdKey.withPrefix(persistTxsTagsKeys.begin)));
|
|
|
|
tLogData->persistentData->clear(singleKeyRange(logIdKey.withPrefix(persistRecoveryCountKeys.begin)));
|
|
|
|
Key msgKey = logIdKey.withPrefix(persistTagMessagesKeys.begin);
|
|
|
|
tLogData->persistentData->clear(KeyRangeRef(msgKey, strinc(msgKey)));
|
|
|
|
Key poppedKey = logIdKey.withPrefix(persistTagPoppedKeys.begin);
|
|
|
|
tLogData->persistentData->clear(KeyRangeRef(poppedKey, strinc(poppedKey)));
|
|
|
|
}
|
2019-07-10 09:20:20 +08:00
|
|
|
|
2021-06-23 05:50:01 +08:00
|
|
|
for (auto it = peekTracker.begin(); it != peekTracker.end(); ++it) {
|
|
|
|
for (auto seq : it->second.sequence_version) {
|
|
|
|
if (!seq.second.isSet()) {
|
|
|
|
seq.second.sendError(operation_obsolete());
|
2019-07-10 09:20:20 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
LogEpoch epoch() const { return recoveryCount; }
|
|
|
|
};
|
|
|
|
|
|
|
|
template <class T>
|
2021-03-11 02:06:03 +08:00
|
|
|
void TLogQueue::push(T const& qe, Reference<LogData> logData) {
|
|
|
|
BinaryWriter wr(Unversioned()); // outer framing is not versioned
|
2019-02-08 09:02:29 +08:00
|
|
|
wr << uint32_t(0);
|
2021-03-11 02:06:03 +08:00
|
|
|
IncludeVersion(ProtocolVersion::withTLogQueueEntryRef()).write(wr); // payload is versioned
|
2019-02-08 09:02:29 +08:00
|
|
|
wr << qe;
|
|
|
|
wr << uint8_t(1);
|
|
|
|
*(uint32_t*)wr.getData() = wr.getLength() - sizeof(uint32_t) - sizeof(uint8_t);
|
2021-03-11 02:06:03 +08:00
|
|
|
auto loc = queue->push(wr.toValue());
|
2019-02-08 09:02:29 +08:00
|
|
|
//TraceEvent("TLogQueueVersionWritten", dbgid).detail("Size", wr.getLength() - sizeof(uint32_t) - sizeof(uint8_t)).detail("Loc", loc);
|
|
|
|
logData->versionLocation[qe.version] = loc;
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
void TLogQueue::pop(Version upTo, Reference<LogData> logData) {
|
2019-02-08 09:02:29 +08:00
|
|
|
// Keep only the given and all subsequent version numbers
|
|
|
|
// Find the first version >= upTo
|
|
|
|
auto v = logData->versionLocation.lower_bound(upTo);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (v == logData->versionLocation.begin())
|
|
|
|
return;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (v == logData->versionLocation.end()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
v = logData->versionLocation.lastItem();
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2019-02-08 09:02:29 +08:00
|
|
|
v.decrementNonEnd();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
queue->pop(v->value);
|
|
|
|
logData->versionLocation.erase(logData->versionLocation.begin(),
|
|
|
|
v); // ... and then we erase that previous version and all prior versions
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
void TLogQueue::updateVersionSizes(const TLogQueueEntry& result, TLogData* tLog) {
|
2019-02-08 09:02:29 +08:00
|
|
|
auto it = tLog->id_data.find(result.id);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (it != tLog->id_data.end()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
it->second->versionLocation[result.version] = queue->getNextReadLocation();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogLock(TLogData* self, ReplyPromise<TLogLockResult> reply, Reference<LogData> logData) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Version stopVersion = logData->version.get();
|
|
|
|
|
|
|
|
TEST(true); // TLog stopped by recovering master
|
2021-03-11 02:06:03 +08:00
|
|
|
TEST(logData->stopped); // logData already stopped
|
|
|
|
TEST(!logData->stopped); // logData not yet stopped
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("TLogStop", logData->logId)
|
|
|
|
.detail("Ver", stopVersion)
|
|
|
|
.detail("IsStopped", logData->stopped)
|
|
|
|
.detail("QueueCommitted", logData->queueCommittedVersion.get());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
logData->stopped = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!logData->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->recoveryComplete.sendError(end_of_stream());
|
|
|
|
}
|
|
|
|
|
|
|
|
// Lock once the current version has been committed
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(logData->queueCommittedVersion.whenAtLeast(stopVersion));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
ASSERT(stopVersion == logData->version.get());
|
|
|
|
|
|
|
|
TLogLockResult result;
|
|
|
|
result.end = stopVersion;
|
|
|
|
result.knownCommittedVersion = logData->knownCommittedVersion;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("TLogStop2", self->dbgid)
|
|
|
|
.detail("LogId", logData->logId)
|
|
|
|
.detail("Ver", stopVersion)
|
|
|
|
.detail("IsStopped", logData->stopped)
|
|
|
|
.detail("QueueCommitted", logData->queueCommittedVersion.get())
|
|
|
|
.detail("KnownCommitted", result.knownCommittedVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
reply.send(result);
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void updatePersistentPopped(TLogData* self, Reference<LogData> logData, Reference<LogData::TagData> data) {
|
|
|
|
if (!data->poppedRecently)
|
|
|
|
return;
|
|
|
|
self->persistentData->set(
|
|
|
|
KeyValueRef(persistTagPoppedKey(logData->logId, data->tag), persistTagPoppedValue(data->popped)));
|
2019-02-08 09:02:29 +08:00
|
|
|
data->poppedRecently = false;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (data->nothingPersistent)
|
|
|
|
return;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentData->clear(KeyRangeRef(persistTagMessagesKey(logData->logId, data->tag, Version(0)),
|
|
|
|
persistTagMessagesKey(logData->logId, data->tag, data->popped)));
|
2019-02-08 09:02:29 +08:00
|
|
|
if (data->popped > logData->persistentDataVersion)
|
|
|
|
data->nothingPersistent = true;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> updatePersistentData(TLogData* self, Reference<LogData> logData, Version newPersistentDataVersion) {
|
2019-02-08 09:02:29 +08:00
|
|
|
// PERSIST: Changes self->persistentDataVersion and writes and commits the relevant changes
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(newPersistentDataVersion <= logData->version.get());
|
|
|
|
ASSERT(newPersistentDataVersion <= logData->queueCommittedVersion.get());
|
|
|
|
ASSERT(newPersistentDataVersion > logData->persistentDataVersion);
|
|
|
|
ASSERT(logData->persistentDataVersion == logData->persistentDataDurableVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
//TraceEvent("UpdatePersistentData", self->dbgid).detail("Seq", newPersistentDataSeq);
|
|
|
|
|
|
|
|
state bool anyData = false;
|
|
|
|
|
|
|
|
// For all existing tags
|
|
|
|
state int tagLocality = 0;
|
|
|
|
state int tagId = 0;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (tagLocality = 0; tagLocality < logData->tag_data.size(); tagLocality++) {
|
|
|
|
for (tagId = 0; tagId < logData->tag_data[tagLocality].size(); tagId++) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Reference<LogData::TagData> tagData = logData->tag_data[tagLocality][tagId];
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tagData) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Version currentVersion = 0;
|
|
|
|
// Clear recently popped versions from persistentData if necessary
|
2021-03-11 02:06:03 +08:00
|
|
|
updatePersistentPopped(self, logData, tagData);
|
2019-02-08 09:02:29 +08:00
|
|
|
// Transfer unpopped messages with version numbers less than newPersistentDataVersion to persistentData
|
2021-03-11 02:06:03 +08:00
|
|
|
state std::deque<std::pair<Version, LengthPrefixedStringRef>>::iterator msg =
|
|
|
|
tagData->versionMessages.begin();
|
|
|
|
while (msg != tagData->versionMessages.end() && msg->first <= newPersistentDataVersion) {
|
2019-02-08 09:02:29 +08:00
|
|
|
currentVersion = msg->first;
|
|
|
|
anyData = true;
|
|
|
|
tagData->nothingPersistent = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
BinaryWriter wr(Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (; msg != tagData->versionMessages.end() && msg->first == currentVersion; ++msg)
|
2019-02-08 09:02:29 +08:00
|
|
|
wr << msg->second.toStringRef();
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentData->set(
|
|
|
|
KeyValueRef(persistTagMessagesKey(logData->logId, tagData->tag, currentVersion), wr.toValue()));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2019-06-25 17:47:35 +08:00
|
|
|
Future<Void> f = yield(TaskPriority::UpdateStorage);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!f.isReady()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
wait(f);
|
2021-03-11 02:06:03 +08:00
|
|
|
msg = std::upper_bound(tagData->versionMessages.begin(),
|
|
|
|
tagData->versionMessages.end(),
|
|
|
|
std::make_pair(currentVersion, LengthPrefixedStringRef()),
|
|
|
|
CompareFirst<std::pair<Version, LengthPrefixedStringRef>>());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-06-25 17:47:35 +08:00
|
|
|
wait(yield(TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentData->set(
|
|
|
|
KeyValueRef(BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistCurrentVersionKeys.begin),
|
|
|
|
BinaryWriter::toValue(newPersistentDataVersion, Unversioned())));
|
|
|
|
self->persistentData->set(KeyValueRef(
|
|
|
|
BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistKnownCommittedVersionKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->knownCommittedVersion, Unversioned())));
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->persistentDataVersion = newPersistentDataVersion;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(self->persistentData->commit()); // SOMEDAY: This seems to be running pretty often, should we slow it down???
|
|
|
|
wait(delay(0, TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Now that the changes we made to persistentData are durable, erase the data we moved from memory and the queue,
|
|
|
|
// increase bytesDurable accordingly, and update persistentDataDurableVersion.
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TEST(anyData); // TLog moved data to persistentData
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->persistentDataDurableVersion = newPersistentDataVersion;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (tagLocality = 0; tagLocality < logData->tag_data.size(); tagLocality++) {
|
|
|
|
for (tagId = 0; tagId < logData->tag_data[tagLocality].size(); tagId++) {
|
|
|
|
if (logData->tag_data[tagLocality][tagId]) {
|
|
|
|
wait(logData->tag_data[tagLocality][tagId]->eraseMessagesBefore(
|
|
|
|
newPersistentDataVersion + 1, self, logData, TaskPriority::UpdateStorage));
|
2019-06-25 17:47:35 +08:00
|
|
|
wait(yield(TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->version_sizes.erase(logData->version_sizes.begin(),
|
|
|
|
logData->version_sizes.lower_bound(logData->persistentDataDurableVersion));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2019-06-25 17:47:35 +08:00
|
|
|
wait(yield(TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
while (!logData->messageBlocks.empty() && logData->messageBlocks.front().first <= newPersistentDataVersion) {
|
|
|
|
int64_t bytesErased =
|
|
|
|
int64_t(logData->messageBlocks.front().second.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->bytesDurable += bytesErased;
|
|
|
|
self->bytesDurable += bytesErased;
|
|
|
|
logData->messageBlocks.pop_front();
|
2019-06-25 17:47:35 +08:00
|
|
|
wait(yield(TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->bytesDurable.getValue() > logData->bytesInput.getValue() || self->bytesDurable > self->bytesInput) {
|
|
|
|
TraceEvent(SevError, "BytesDurableTooLarge", logData->logId)
|
|
|
|
.detail("SharedBytesInput", self->bytesInput)
|
|
|
|
.detail("SharedBytesDurable", self->bytesDurable)
|
|
|
|
.detail("LocalBytesInput", logData->bytesInput.getValue())
|
|
|
|
.detail("LocalBytesDurable", logData->bytesDurable.getValue());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
ASSERT(logData->bytesDurable.getValue() <= logData->bytesInput.getValue());
|
|
|
|
ASSERT(self->bytesDurable <= self->bytesInput);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->queueCommitEnd.get() > 0)
|
|
|
|
self->persistentQueue->pop(
|
|
|
|
newPersistentDataVersion + 1,
|
|
|
|
logData); // SOMEDAY: this can cause a slow task (~0.5ms), presumably from erasing too many versions. Should
|
|
|
|
// we limit the number of versions cleared at a time?
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogPopCore(TLogData* self, Tag inputTag, Version to, Reference<LogData> logData) {
|
2019-11-13 05:01:29 +08:00
|
|
|
if (self->ignorePopRequest) {
|
|
|
|
TraceEvent(SevDebug, "IgnoringPopRequest").detail("IgnorePopDeadline", self->ignorePopDeadline);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->toBePopped.find(inputTag) == self->toBePopped.end() || to > self->toBePopped[inputTag]) {
|
2019-11-13 05:01:29 +08:00
|
|
|
self->toBePopped[inputTag] = to;
|
|
|
|
}
|
|
|
|
// add the pop to the toBePopped map
|
|
|
|
TraceEvent(SevDebug, "IgnoringPopRequest")
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("IgnorePopDeadline", self->ignorePopDeadline)
|
|
|
|
.detail("Tag", inputTag.toString())
|
|
|
|
.detail("Version", to);
|
2019-11-13 05:01:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
state Version upTo = to;
|
|
|
|
int8_t tagLocality = inputTag.locality;
|
2019-07-24 02:45:04 +08:00
|
|
|
if (isPseudoLocality(tagLocality)) {
|
|
|
|
if (logData->logSystem->get().isValid()) {
|
2019-08-11 01:31:25 +08:00
|
|
|
upTo = logData->logSystem->get()->popPseudoLocalityTag(inputTag, to);
|
2019-07-24 02:45:04 +08:00
|
|
|
tagLocality = tagLocalityLogRouter;
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent(SevWarn, "TLogPopNoLogSystem", self->dbgid)
|
|
|
|
.detail("Locality", tagLocality)
|
|
|
|
.detail("Version", upTo);
|
2019-08-17 12:40:57 +08:00
|
|
|
return Void();
|
2019-07-24 02:45:04 +08:00
|
|
|
}
|
2019-11-13 05:01:29 +08:00
|
|
|
}
|
|
|
|
state Tag tag(tagLocality, inputTag.id);
|
|
|
|
auto tagData = logData->getTagData(tag);
|
|
|
|
if (!tagData) {
|
|
|
|
tagData = logData->createTagData(tag, upTo, true, true, false);
|
|
|
|
} else if (upTo > tagData->popped) {
|
|
|
|
tagData->popped = upTo;
|
|
|
|
tagData->poppedRecently = true;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tagData->unpoppedRecovered && upTo > logData->recoveredAt) {
|
2019-11-13 05:01:29 +08:00
|
|
|
tagData->unpoppedRecovered = false;
|
|
|
|
logData->unpoppedRecoveredTags--;
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("TLogPoppedTag", logData->logId)
|
|
|
|
.detail("Tags", logData->unpoppedRecoveredTags)
|
|
|
|
.detail("Tag", tag.toString())
|
|
|
|
.detail("DurableKCVer", logData->durableKnownCommittedVersion)
|
|
|
|
.detail("RecoveredAt", logData->recoveredAt);
|
|
|
|
if (logData->unpoppedRecoveredTags == 0 && logData->durableKnownCommittedVersion >= logData->recoveredAt &&
|
|
|
|
logData->recoveryComplete.canBeSet()) {
|
2019-11-13 05:01:29 +08:00
|
|
|
logData->recoveryComplete.send(Void());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (upTo > logData->persistentDataDurableVersion)
|
|
|
|
wait(tagData->eraseMessagesBefore(upTo, self, logData, TaskPriority::TLogPop));
|
|
|
|
//TraceEvent("TLogPop", self->dbgid).detail("Tag", tag.toString()).detail("To", upTo);
|
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogPop(TLogData* self, TLogPopRequest req, Reference<LogData> logData) {
|
2019-11-13 05:01:29 +08:00
|
|
|
// timeout check for ignorePopRequest
|
|
|
|
if (self->ignorePopRequest && (g_network->now() > self->ignorePopDeadline)) {
|
|
|
|
|
|
|
|
TraceEvent("EnableTLogPlayAllIgnoredPops");
|
|
|
|
// use toBePopped and issue all the pops
|
|
|
|
state std::map<Tag, Version>::iterator it;
|
|
|
|
state vector<Future<Void>> ignoredPops;
|
|
|
|
self->ignorePopRequest = false;
|
|
|
|
self->ignorePopUid = "";
|
|
|
|
self->ignorePopDeadline = 0.0;
|
|
|
|
for (it = self->toBePopped.begin(); it != self->toBePopped.end(); it++) {
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("PlayIgnoredPop").detail("Tag", it->first.toString()).detail("Version", it->second);
|
2019-11-13 05:01:29 +08:00
|
|
|
ignoredPops.push_back(tLogPopCore(self, it->first, it->second, logData));
|
|
|
|
}
|
|
|
|
self->toBePopped.clear();
|
|
|
|
wait(waitForAll(ignoredPops));
|
|
|
|
TraceEvent("ResetIgnorePopRequest")
|
|
|
|
.detail("Now", g_network->now())
|
|
|
|
.detail("IgnorePopRequest", self->ignorePopRequest)
|
|
|
|
.detail("IgnorePopDeadline", self->ignorePopDeadline);
|
|
|
|
}
|
|
|
|
wait(tLogPopCore(self, req.tag, req.to, logData));
|
|
|
|
req.reply.send(Void());
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// This function (and updatePersistentData, which is called by this function) run at a low priority and can soak up all
|
|
|
|
// CPU resources. For this reason, they employ aggressive use of yields to avoid causing slow tasks that could introduce
|
|
|
|
// latencies for more important work (e.g. commits).
|
|
|
|
ACTOR Future<Void> updateStorage(TLogData* self) {
|
|
|
|
while (self->queueOrder.size() && !self->id_data.count(self->queueOrder.front())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
self->queueOrder.pop_front();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!self->queueOrder.size()) {
|
|
|
|
wait(delay(BUGGIFY ? SERVER_KNOBS->BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL
|
|
|
|
: SERVER_KNOBS->TLOG_STORAGE_MIN_UPDATE_INTERVAL,
|
|
|
|
TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
state Reference<LogData> logData = self->id_data[self->queueOrder.front()];
|
|
|
|
state Version nextVersion = 0;
|
|
|
|
state int totalSize = 0;
|
|
|
|
|
|
|
|
state FlowLock::Releaser commitLockReleaser;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// FIXME: This policy for calculating the cache pop version could end up popping recent data in the remote DC after
|
|
|
|
// two consecutive recoveries.
|
|
|
|
// It also does not protect against spilling the cache tag directly, so it is theoretically possible to spill this
|
|
|
|
// tag; which is not intended to ever happen.
|
2019-11-13 05:01:29 +08:00
|
|
|
Optional<Version> cachePopVersion;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : self->id_data) {
|
|
|
|
if (!it.second->stopped) {
|
|
|
|
if (it.second->version.get() - it.second->unrecoveredBefore >
|
|
|
|
SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT + SERVER_KNOBS->MAX_CACHE_VERSIONS) {
|
2019-11-13 05:01:29 +08:00
|
|
|
cachePopVersion = it.second->version.get() - SERVER_KNOBS->MAX_CACHE_VERSIONS;
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (cachePopVersion.present()) {
|
2019-11-13 05:01:29 +08:00
|
|
|
state std::vector<Future<Void>> cachePopFutures;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : self->id_data) {
|
|
|
|
cachePopFutures.push_back(tLogPop(self, TLogPopRequest(cachePopVersion.get(), 0, cacheTag), it.second));
|
2019-11-13 05:01:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(waitForAll(cachePopFutures));
|
2019-11-13 05:01:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->stopped) {
|
2019-07-30 14:40:28 +08:00
|
|
|
if (self->bytesInput - self->bytesDurable >= self->targetVolatileBytes) {
|
2021-03-11 02:06:03 +08:00
|
|
|
while (logData->persistentDataDurableVersion != logData->version.get()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
totalSize = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
Map<Version, std::pair<int, int>>::iterator sizeItr = logData->version_sizes.begin();
|
2019-02-08 09:02:29 +08:00
|
|
|
nextVersion = logData->version.get();
|
2021-03-11 02:06:03 +08:00
|
|
|
while (totalSize < SERVER_KNOBS->UPDATE_STORAGE_BYTE_LIMIT && sizeItr != logData->version_sizes.end()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
totalSize += sizeItr->value.first + sizeItr->value.second;
|
|
|
|
++sizeItr;
|
|
|
|
nextVersion = sizeItr == logData->version_sizes.end() ? logData->version.get() : sizeItr->key;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(logData->queueCommittedVersion.whenAtLeast(nextVersion));
|
|
|
|
wait(delay(0, TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
//TraceEvent("TlogUpdatePersist", self->dbgid).detail("LogId", logData->logId).detail("NextVersion", nextVersion).detail("Version", logData->version.get()).detail("PersistentDataDurableVer", logData->persistentDataDurableVersion).detail("QueueCommitVer", logData->queueCommittedVersion.get()).detail("PersistDataVer", logData->persistentDataVersion);
|
|
|
|
if (nextVersion > logData->persistentDataVersion) {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(self->persistentDataCommitLock.take());
|
2019-02-08 09:02:29 +08:00
|
|
|
commitLockReleaser = FlowLock::Releaser(self->persistentDataCommitLock);
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(updatePersistentData(self, logData, nextVersion));
|
2019-02-08 09:02:29 +08:00
|
|
|
commitLockReleaser.release();
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(BUGGIFY ? SERVER_KNOBS->BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL
|
|
|
|
: SERVER_KNOBS->TLOG_STORAGE_MIN_UPDATE_INTERVAL,
|
|
|
|
TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->removed.isReady()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->persistentDataDurableVersion == logData->version.get()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
self->queueOrder.pop_front();
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(0.0, TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(BUGGIFY ? SERVER_KNOBS->BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL
|
|
|
|
: SERVER_KNOBS->TLOG_STORAGE_MIN_UPDATE_INTERVAL,
|
|
|
|
TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (logData->initialized) {
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(self->queueOrder.size() == 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->version_sizes.empty()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
nextVersion = logData->version.get();
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
Map<Version, std::pair<int, int>>::iterator sizeItr = logData->version_sizes.begin();
|
|
|
|
while (totalSize < SERVER_KNOBS->UPDATE_STORAGE_BYTE_LIMIT && sizeItr != logData->version_sizes.end() &&
|
|
|
|
(logData->bytesInput.getValue() - logData->bytesDurable.getValue() - totalSize >=
|
|
|
|
self->targetVolatileBytes ||
|
|
|
|
sizeItr->value.first == 0)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
totalSize += sizeItr->value.first + sizeItr->value.second;
|
|
|
|
++sizeItr;
|
|
|
|
nextVersion = sizeItr == logData->version_sizes.end() ? logData->version.get() : sizeItr->key;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
//TraceEvent("UpdateStorageVer", logData->logId).detail("NextVersion", nextVersion).detail("PersistentDataVersion", logData->persistentDataVersion).detail("TotalSize", totalSize);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(logData->queueCommittedVersion.whenAtLeast(nextVersion));
|
|
|
|
wait(delay(0, TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
if (nextVersion > logData->persistentDataVersion) {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(self->persistentDataCommitLock.take());
|
2019-02-08 09:02:29 +08:00
|
|
|
commitLockReleaser = FlowLock::Releaser(self->persistentDataCommitLock);
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(updatePersistentData(self, logData, nextVersion));
|
2019-02-08 09:02:29 +08:00
|
|
|
commitLockReleaser.release();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (totalSize < SERVER_KNOBS->UPDATE_STORAGE_BYTE_LIMIT) {
|
|
|
|
wait(delay(BUGGIFY ? SERVER_KNOBS->BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL
|
|
|
|
: SERVER_KNOBS->TLOG_STORAGE_MIN_UPDATE_INTERVAL,
|
|
|
|
TaskPriority::UpdateStorage));
|
|
|
|
} else {
|
|
|
|
// recovery wants to commit to persistant data when updatePersistentData is not active, this delay ensures
|
|
|
|
// that immediately after updatePersist returns another one has not been started yet.
|
|
|
|
wait(delay(0.0, TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(BUGGIFY ? SERVER_KNOBS->BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL
|
|
|
|
: SERVER_KNOBS->TLOG_STORAGE_MIN_UPDATE_INTERVAL,
|
|
|
|
TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> updateStorageLoop(TLogData* self) {
|
2019-06-25 17:47:35 +08:00
|
|
|
wait(delay(0, TaskPriority::UpdateStorage));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
loop { wait(updateStorage(self)); }
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void commitMessages(TLogData* self,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
Version version,
|
|
|
|
const std::vector<TagsAndMessage>& taggedMessages) {
|
|
|
|
// SOMEDAY: This method of copying messages is reasonably memory efficient, but it's still a lot of bytes copied.
|
|
|
|
// Find a way to do the memory allocation right as we receive the messages in the network layer.
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
int64_t addedBytes = 0;
|
|
|
|
int64_t overheadBytes = 0;
|
|
|
|
int expectedBytes = 0;
|
|
|
|
int txsBytes = 0;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!taggedMessages.size()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
int msgSize = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& i : taggedMessages) {
|
2019-02-08 09:02:29 +08:00
|
|
|
msgSize += i.message.size();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Grab the last block in the blocks list so we can share its arena
|
|
|
|
// We pop all of the elements of it to create a "fresh" vector that starts at the end of the previous vector
|
|
|
|
Standalone<VectorRef<uint8_t>> block;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->messageBlocks.empty()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
block = Standalone<VectorRef<uint8_t>>();
|
|
|
|
block.reserve(block.arena(), std::max<int64_t>(SERVER_KNOBS->TLOG_MESSAGE_BLOCK_BYTES, msgSize));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2019-02-08 09:02:29 +08:00
|
|
|
block = logData->messageBlocks.back().second;
|
|
|
|
}
|
|
|
|
|
|
|
|
block.pop_front(block.size());
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& msg : taggedMessages) {
|
|
|
|
if (msg.message.size() > block.capacity() - block.size()) {
|
2019-04-25 01:59:19 +08:00
|
|
|
logData->messageBlocks.emplace_back(version, block);
|
2019-02-08 09:02:29 +08:00
|
|
|
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
|
|
|
block = Standalone<VectorRef<uint8_t>>();
|
|
|
|
block.reserve(block.arena(), std::max<int64_t>(SERVER_KNOBS->TLOG_MESSAGE_BLOCK_BYTES, msgSize));
|
|
|
|
}
|
|
|
|
|
|
|
|
block.append(block.arena(), msg.message.begin(), msg.message.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto tag : msg.tags) {
|
|
|
|
if (logData->locality == tagLocalitySatellite) {
|
|
|
|
if (!(tag.locality == tagLocalityTxs || tag.locality == tagLocalityLogRouter || tag == txsTag)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
continue;
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (!(logData->locality == tagLocalitySpecial || logData->locality == tag.locality ||
|
|
|
|
tag.locality < 0)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tag.locality == tagLocalityLogRouter) {
|
|
|
|
if (!logData->logRouterTags) {
|
2019-02-08 09:02:29 +08:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
tag.id = tag.id % logData->logRouterTags;
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tag.locality == tagLocalityTxs) {
|
2019-07-17 16:25:09 +08:00
|
|
|
if (logData->txsTags > 0) {
|
|
|
|
tag.id = tag.id % logData->txsTags;
|
|
|
|
} else {
|
|
|
|
tag = txsTag;
|
|
|
|
}
|
2019-06-20 09:15:09 +08:00
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
Reference<LogData::TagData> tagData = logData->getTagData(tag);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!tagData) {
|
2019-02-08 09:02:29 +08:00
|
|
|
tagData = logData->createTagData(tag, 0, true, true, false);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (version >= tagData->popped) {
|
2021-03-11 02:06:03 +08:00
|
|
|
tagData->versionMessages.emplace_back(
|
|
|
|
version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size())));
|
|
|
|
if (tagData->versionMessages.back().second.expectedSize() > SERVER_KNOBS->MAX_MESSAGE_SIZE) {
|
|
|
|
TraceEvent(SevWarnAlways, "LargeMessage")
|
|
|
|
.detail("Size", tagData->versionMessages.back().second.expectedSize());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2019-06-20 09:15:09 +08:00
|
|
|
if (tag.locality != tagLocalityTxs && tag != txsTag) {
|
2019-02-08 09:02:29 +08:00
|
|
|
expectedBytes += tagData->versionMessages.back().second.expectedSize();
|
|
|
|
} else {
|
|
|
|
txsBytes += tagData->versionMessages.back().second.expectedSize();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// The factor of VERSION_MESSAGES_OVERHEAD is intended to be an overestimate of the actual memory used
|
|
|
|
// to store this data in a std::deque. In practice, this number is probably something like 528/512
|
|
|
|
// ~= 1.03, but this could vary based on the implementation. There will also be a fixed overhead per
|
|
|
|
// std::deque, but its size should be trivial relative to the size of the TLog queue and can be thought
|
|
|
|
// of as increasing the capacity of the queue slightly.
|
2019-02-08 09:02:29 +08:00
|
|
|
overheadBytes += SERVER_KNOBS->VERSION_MESSAGES_ENTRY_BYTES_WITH_OVERHEAD;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
msgSize -= msg.message.size();
|
|
|
|
}
|
2019-04-25 01:59:19 +08:00
|
|
|
logData->messageBlocks.emplace_back(version, block);
|
2019-02-08 09:02:29 +08:00
|
|
|
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
|
|
|
addedBytes += overheadBytes;
|
|
|
|
|
|
|
|
logData->version_sizes[version] = std::make_pair(expectedBytes, txsBytes);
|
|
|
|
logData->bytesInput += addedBytes;
|
|
|
|
self->bytesInput += addedBytes;
|
|
|
|
self->overheadBytesInput += overheadBytes;
|
|
|
|
|
|
|
|
//TraceEvent("TLogPushed", self->dbgid).detail("Bytes", addedBytes).detail("MessageBytes", messages.size()).detail("Tags", tags.size()).detail("ExpectedBytes", expectedBytes).detail("MCount", mCount).detail("TCount", tCount);
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void commitMessages(TLogData* self, Reference<LogData> logData, Version version, Arena arena, StringRef messages) {
|
|
|
|
ArenaReader rd(arena, messages, Unversioned());
|
2019-11-06 10:07:30 +08:00
|
|
|
self->tempTagMessages.clear();
|
2021-03-11 02:06:03 +08:00
|
|
|
while (!rd.empty()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TagsAndMessage tagsAndMsg;
|
2019-09-05 05:52:09 +08:00
|
|
|
tagsAndMsg.loadFromArena(&rd, nullptr);
|
2019-11-06 10:07:30 +08:00
|
|
|
self->tempTagMessages.push_back(std::move(tagsAndMsg));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2019-11-06 10:07:30 +08:00
|
|
|
commitMessages(self, logData, version, self->tempTagMessages);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
Version poppedVersion(Reference<LogData> self, Tag tag) {
|
2019-02-08 09:02:29 +08:00
|
|
|
auto tagData = self->getTagData(tag);
|
|
|
|
if (!tagData) {
|
2019-08-01 08:45:21 +08:00
|
|
|
if (tag == txsTag || tag.locality == tagLocalityTxs) {
|
2021-03-11 02:06:03 +08:00
|
|
|
return 0;
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
return self->recoveredAt;
|
|
|
|
}
|
|
|
|
return tagData->popped;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
std::deque<std::pair<Version, LengthPrefixedStringRef>>& getVersionMessages(Reference<LogData> self, Tag tag) {
|
2019-02-08 09:02:29 +08:00
|
|
|
auto tagData = self->getTagData(tag);
|
|
|
|
if (!tagData) {
|
|
|
|
static std::deque<std::pair<Version, LengthPrefixedStringRef>> empty;
|
|
|
|
return empty;
|
|
|
|
}
|
|
|
|
return tagData->versionMessages;
|
|
|
|
};
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void peekMessagesFromMemory(Reference<LogData> self,
|
|
|
|
TLogPeekRequest const& req,
|
|
|
|
BinaryWriter& messages,
|
|
|
|
Version& endVersion) {
|
|
|
|
ASSERT(!messages.getLength());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
auto& deque = getVersionMessages(self, req.tag);
|
2019-03-19 06:03:43 +08:00
|
|
|
//TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1);
|
|
|
|
auto it = std::lower_bound(deque.begin(),
|
|
|
|
deque.end(),
|
|
|
|
std::make_pair(begin, LengthPrefixedStringRef()),
|
|
|
|
CompareFirst<std::pair<Version, LengthPrefixedStringRef>>());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
Version currentVersion = -1;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (; it != deque.end(); ++it) {
|
|
|
|
if (it->first != currentVersion) {
|
2019-02-08 09:02:29 +08:00
|
|
|
if (messages.getLength() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) {
|
|
|
|
endVersion = currentVersion + 1;
|
|
|
|
//TraceEvent("TLogPeekMessagesReached2", self->dbgid);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
currentVersion = it->first;
|
2019-09-06 02:30:02 +08:00
|
|
|
messages << VERSION_HEADER << currentVersion;
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
messages << it->second.toStringRef();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference<LogData> logData) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state BinaryWriter messages(Unversioned());
|
|
|
|
state BinaryWriter messages2(Unversioned());
|
|
|
|
state int sequence = -1;
|
|
|
|
state UID peekId;
|
2020-04-23 14:35:48 +08:00
|
|
|
state double queueStart = now();
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) {
|
2020-04-30 04:50:13 +08:00
|
|
|
req.tag.id = req.tag.id % logData->txsTags;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.sequence.present()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
try {
|
|
|
|
peekId = req.sequence.get().first;
|
|
|
|
sequence = req.sequence.get().second;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS &&
|
|
|
|
logData->peekTracker.find(peekId) == logData->peekTracker.end()) {
|
2019-10-23 08:04:57 +08:00
|
|
|
throw operation_obsolete();
|
2019-10-23 08:04:57 +08:00
|
|
|
}
|
2019-07-16 07:43:43 +08:00
|
|
|
auto& trackerData = logData->peekTracker[peekId];
|
2019-07-16 09:51:24 +08:00
|
|
|
if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) {
|
2020-04-23 14:35:48 +08:00
|
|
|
trackerData.tag = req.tag;
|
2019-07-16 09:51:24 +08:00
|
|
|
trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled));
|
2019-07-16 07:43:43 +08:00
|
|
|
}
|
|
|
|
auto seqBegin = trackerData.sequence_version.begin();
|
2021-03-11 02:06:03 +08:00
|
|
|
while (trackerData.sequence_version.size() &&
|
|
|
|
seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) {
|
|
|
|
if (seqBegin->second.canBeSet()) {
|
2019-10-15 09:05:06 +08:00
|
|
|
seqBegin->second.sendError(operation_obsolete());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2019-07-16 07:43:43 +08:00
|
|
|
trackerData.sequence_version.erase(seqBegin);
|
|
|
|
seqBegin = trackerData.sequence_version.begin();
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (trackerData.sequence_version.size() && sequence < seqBegin->first) {
|
2019-10-15 09:05:06 +08:00
|
|
|
throw operation_obsolete();
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2019-07-16 07:43:43 +08:00
|
|
|
|
2020-04-23 14:35:48 +08:00
|
|
|
Future<std::pair<Version, bool>> fPrevPeekData = trackerData.sequence_version[sequence].getFuture();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (fPrevPeekData.isReady()) {
|
2020-04-23 14:35:48 +08:00
|
|
|
trackerData.unblockedPeeks++;
|
|
|
|
double t = now() - trackerData.lastUpdate;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (t > trackerData.idleMax)
|
|
|
|
trackerData.idleMax = t;
|
2020-04-23 14:35:48 +08:00
|
|
|
trackerData.idleTime += t;
|
|
|
|
}
|
2019-07-16 07:43:43 +08:00
|
|
|
trackerData.lastUpdate = now();
|
2020-04-23 14:35:48 +08:00
|
|
|
std::pair<Version, bool> prevPeekData = wait(fPrevPeekData);
|
|
|
|
|
2020-01-22 11:09:07 +08:00
|
|
|
req.begin = std::max(prevPeekData.first, req.begin);
|
2019-07-16 08:05:39 +08:00
|
|
|
req.onlySpilled = prevPeekData.second;
|
2019-07-16 07:43:43 +08:00
|
|
|
wait(yield());
|
2021-03-11 02:06:03 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
|
2019-10-15 09:05:06 +08:00
|
|
|
req.reply.sendError(e);
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
} else {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-23 14:35:48 +08:00
|
|
|
state double blockStart = now();
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.returnIfBlocked && logData->version.get() < req.begin) {
|
2019-02-08 09:02:29 +08:00
|
|
|
req.reply.sendError(end_of_stream());
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.sequence.present()) {
|
2019-07-16 07:43:43 +08:00
|
|
|
auto& trackerData = logData->peekTracker[peekId];
|
2021-03-11 02:06:03 +08:00
|
|
|
auto& sequenceData = trackerData.sequence_version[sequence + 1];
|
2019-07-16 07:43:43 +08:00
|
|
|
if (!sequenceData.isSet()) {
|
2019-07-16 08:05:39 +08:00
|
|
|
sequenceData.send(std::make_pair(req.begin, req.onlySpilled));
|
2019-07-16 07:43:43 +08:00
|
|
|
}
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2019-03-19 06:03:43 +08:00
|
|
|
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
|
2019-02-08 09:02:29 +08:00
|
|
|
// Wait until we have something to return that the caller doesn't already have
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->version.get() < req.begin) {
|
|
|
|
wait(logData->version.whenAtLeast(req.begin));
|
|
|
|
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) {
|
|
|
|
wait(self->concurrentLogRouterReads.take());
|
2019-02-08 09:02:29 +08:00
|
|
|
state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads);
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(0.0, TaskPriority::Low));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) {
|
2019-05-04 03:55:41 +08:00
|
|
|
// Reading spilled data will almost always imply that the storage server is >5s behind the rest
|
|
|
|
// of the cluster. We shouldn't prioritize spending CPU on helping this server catch up
|
|
|
|
// slightly faster over keeping the rest of the cluster operating normally.
|
|
|
|
// txsTag is only ever peeked on recovery, and we would still wish to prioritize requests
|
|
|
|
// that impact recovery duration.
|
2019-06-25 17:47:35 +08:00
|
|
|
wait(delay(0, TaskPriority::TLogSpilledPeekReply));
|
2019-05-04 03:55:41 +08:00
|
|
|
}
|
|
|
|
|
2020-04-23 14:35:48 +08:00
|
|
|
state double workStart = now();
|
|
|
|
|
2019-02-08 09:02:29 +08:00
|
|
|
Version poppedVer = poppedVersion(logData, req.tag);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (poppedVer > req.begin) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TLogPeekReply rep;
|
|
|
|
rep.maxKnownVersion = logData->version.get();
|
|
|
|
rep.minKnownCommittedVersion = logData->minKnownCommittedVersion;
|
|
|
|
rep.popped = poppedVer;
|
|
|
|
rep.end = poppedVer;
|
2019-05-15 08:07:49 +08:00
|
|
|
rep.onlySpilled = false;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.sequence.present()) {
|
2019-07-10 09:20:20 +08:00
|
|
|
auto& trackerData = logData->peekTracker[peekId];
|
2021-03-11 02:06:03 +08:00
|
|
|
auto& sequenceData = trackerData.sequence_version[sequence + 1];
|
2019-02-08 09:02:29 +08:00
|
|
|
trackerData.lastUpdate = now();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
|
2019-10-15 09:05:06 +08:00
|
|
|
req.reply.sendError(operation_obsolete());
|
2019-07-16 07:43:43 +08:00
|
|
|
if (!sequenceData.isSet())
|
2019-10-15 09:05:06 +08:00
|
|
|
sequenceData.sendError(operation_obsolete());
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (sequenceData.isSet()) {
|
|
|
|
if (sequenceData.getFuture().get().first != rep.end) {
|
|
|
|
TEST(true); // tlog peek second attempt ended at a different version
|
2019-10-15 09:05:06 +08:00
|
|
|
req.reply.sendError(operation_obsolete());
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
} else {
|
2019-07-09 13:13:09 +08:00
|
|
|
sequenceData.send(std::make_pair(rep.end, rep.onlySpilled));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
rep.begin = req.begin;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.send(rep);
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
state Version endVersion = logData->version.get() + 1;
|
2019-05-15 08:07:49 +08:00
|
|
|
state bool onlySpilled = false;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// grab messages from disk
|
2019-03-19 06:03:43 +08:00
|
|
|
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.begin <= logData->persistentDataDurableVersion) {
|
|
|
|
// Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We
|
|
|
|
// may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if
|
|
|
|
// an initial attempt to read from disk results in insufficient data and the required data is no longer in
|
|
|
|
// memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the
|
|
|
|
// result?
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2019-05-15 08:07:49 +08:00
|
|
|
if (req.onlySpilled) {
|
|
|
|
endVersion = logData->persistentDataDurableVersion + 1;
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
peekMessagesFromMemory(logData, req, messages2, endVersion);
|
2019-05-15 08:07:49 +08:00
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult kvs = wait(self->persistentData->readRange(
|
2021-03-11 02:06:03 +08:00
|
|
|
KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin),
|
|
|
|
persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)),
|
|
|
|
SERVER_KNOBS->DESIRED_TOTAL_BYTES,
|
|
|
|
SERVER_KNOBS->DESIRED_TOTAL_BYTES));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2019-03-19 06:03:43 +08:00
|
|
|
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& kv : kvs) {
|
2019-02-08 09:02:29 +08:00
|
|
|
auto ver = decodeTagMessagesKey(kv.key);
|
2019-09-06 02:30:02 +08:00
|
|
|
messages << VERSION_HEADER << ver;
|
2019-02-08 09:02:29 +08:00
|
|
|
messages.serializeBytes(kv.value);
|
|
|
|
}
|
|
|
|
|
2019-05-15 08:07:49 +08:00
|
|
|
if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) {
|
2019-02-08 09:02:29 +08:00
|
|
|
endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1;
|
2019-05-15 08:07:49 +08:00
|
|
|
onlySpilled = true;
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
messages.serializeBytes(messages2.toValue());
|
2019-05-15 08:07:49 +08:00
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
peekMessagesFromMemory(logData, req, messages, endVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
|
|
|
|
}
|
|
|
|
|
|
|
|
TLogPeekReply reply;
|
|
|
|
reply.maxKnownVersion = logData->version.get();
|
|
|
|
reply.minKnownCommittedVersion = logData->minKnownCommittedVersion;
|
2019-03-29 02:52:50 +08:00
|
|
|
reply.messages = messages.toValue();
|
2019-02-08 09:02:29 +08:00
|
|
|
reply.end = endVersion;
|
2019-05-15 08:07:49 +08:00
|
|
|
reply.onlySpilled = onlySpilled;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().address);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.sequence.present()) {
|
2019-07-10 09:20:20 +08:00
|
|
|
auto& trackerData = logData->peekTracker[peekId];
|
2019-02-08 09:02:29 +08:00
|
|
|
trackerData.lastUpdate = now();
|
2020-04-23 14:35:48 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
double queueT = blockStart - queueStart;
|
|
|
|
double blockT = workStart - blockStart;
|
|
|
|
double workT = now() - workStart;
|
2020-04-23 14:35:48 +08:00
|
|
|
|
|
|
|
trackerData.totalPeeks++;
|
|
|
|
trackerData.replyBytes += reply.messages.size();
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (queueT > trackerData.queueMax)
|
|
|
|
trackerData.queueMax = queueT;
|
|
|
|
if (blockT > trackerData.blockMax)
|
|
|
|
trackerData.blockMax = blockT;
|
|
|
|
if (workT > trackerData.workMax)
|
|
|
|
trackerData.workMax = workT;
|
2020-04-23 14:35:48 +08:00
|
|
|
|
|
|
|
trackerData.queueTime += queueT;
|
|
|
|
trackerData.blockTime += blockT;
|
|
|
|
trackerData.workTime += workT;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
auto& sequenceData = trackerData.sequence_version[sequence + 1];
|
|
|
|
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
|
2019-10-15 09:05:06 +08:00
|
|
|
req.reply.sendError(operation_obsolete());
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!sequenceData.isSet())
|
2019-10-15 09:05:06 +08:00
|
|
|
sequenceData.sendError(operation_obsolete());
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (sequenceData.isSet()) {
|
2020-04-23 14:35:48 +08:00
|
|
|
trackerData.duplicatePeeks++;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (sequenceData.getFuture().get().first != reply.end) {
|
|
|
|
TEST(true); // tlog peek second attempt ended at a different version (2)
|
2019-10-15 09:05:06 +08:00
|
|
|
req.reply.sendError(operation_obsolete());
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
} else {
|
2019-07-09 13:13:09 +08:00
|
|
|
sequenceData.send(std::make_pair(reply.end, reply.onlySpilled));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
reply.begin = req.begin;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.send(reply);
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> doQueueCommit(TLogData* self,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
std::vector<Reference<LogData>> missingFinalCommit) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Version ver = logData->version.get();
|
2021-03-11 02:06:03 +08:00
|
|
|
state Version commitNumber = self->queueCommitBegin + 1;
|
2019-02-08 09:02:29 +08:00
|
|
|
state Version knownCommittedVersion = logData->knownCommittedVersion;
|
|
|
|
self->queueCommitBegin = commitNumber;
|
|
|
|
logData->queueCommittingVersion = ver;
|
|
|
|
|
2019-11-06 10:07:30 +08:00
|
|
|
g_network->setCurrentTask(TaskPriority::TLogCommitReply);
|
2019-02-08 09:02:29 +08:00
|
|
|
Future<Void> c = self->persistentQueue->commit();
|
|
|
|
self->diskQueueCommitBytes = 0;
|
|
|
|
self->largeDiskQueueCommitBytes.set(false);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(ioDegradedOrTimeoutError(
|
|
|
|
c, SERVER_KNOBS->MAX_STORAGE_COMMIT_TIME, self->degraded, SERVER_KNOBS->TLOG_DEGRADED_DURATION));
|
|
|
|
if (g_network->isSimulated() && !g_simulator.speedUpSimulation && BUGGIFY_WITH_PROB(0.0001)) {
|
2019-03-12 03:11:17 +08:00
|
|
|
wait(delay(6.0));
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(self->queueCommitEnd.whenAtLeast(commitNumber - 1));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Calling check_yield instead of yield to avoid a destruction ordering problem in simulation
|
|
|
|
if (g_network->check_yield(g_network->getCurrentTask())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
wait(delay(0, g_network->getCurrentTask()));
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(ver > logData->queueCommittedVersion.get());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
logData->durableKnownCommittedVersion = knownCommittedVersion;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->unpoppedRecoveredTags == 0 && knownCommittedVersion >= logData->recoveredAt &&
|
|
|
|
logData->recoveryComplete.canBeSet()) {
|
|
|
|
TraceEvent("TLogRecoveryComplete", logData->logId)
|
|
|
|
.detail("Tags", logData->unpoppedRecoveredTags)
|
|
|
|
.detail("DurableKCVer", logData->durableKnownCommittedVersion)
|
|
|
|
.detail("RecoveredAt", logData->recoveredAt);
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->recoveryComplete.send(Void());
|
|
|
|
}
|
|
|
|
|
|
|
|
//TraceEvent("TLogCommitDurable", self->dbgid).detail("Version", ver);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->logSystem->get() &&
|
|
|
|
(!logData->isPrimary || logData->logRouterPoppedVersion < logData->logRouterPopToVersion)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->logRouterPoppedVersion = ver;
|
|
|
|
logData->logSystem->get()->pop(ver, logData->remoteTag, knownCommittedVersion, logData->locality);
|
|
|
|
}
|
|
|
|
|
|
|
|
logData->queueCommittedVersion.set(ver);
|
|
|
|
self->queueCommitEnd.set(commitNumber);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : missingFinalCommit) {
|
|
|
|
TraceEvent("TLogCommitMissingFinalCommit", self->dbgid)
|
|
|
|
.detail("LogId", logData->logId)
|
|
|
|
.detail("Version", it->version.get())
|
|
|
|
.detail("QueueVer", it->queueCommittedVersion.get());
|
|
|
|
TEST(true); // A TLog was replaced before having a chance to commit its queue
|
|
|
|
it->queueCommittedVersion.set(it->version.get());
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> commitQueue(TLogData* self) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Reference<LogData> logData;
|
2021-05-20 04:26:01 +08:00
|
|
|
state std::vector<Reference<LogData>> missingFinalCommit;
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
loop {
|
|
|
|
int foundCount = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : self->id_data) {
|
|
|
|
if (!it.second->stopped) {
|
|
|
|
logData = it.second;
|
|
|
|
foundCount++;
|
|
|
|
} else if (it.second->version.get() >
|
|
|
|
std::max(it.second->queueCommittingVersion, it.second->queueCommittedVersion.get())) {
|
|
|
|
missingFinalCommit.push_back(it.second);
|
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
ASSERT(foundCount < 2);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!foundCount) {
|
|
|
|
wait(self->newLogData.onTrigger());
|
2019-02-08 09:02:29 +08:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("CommitQueueNewLog", self->dbgid)
|
|
|
|
.detail("LogId", logData->logId)
|
|
|
|
.detail("Version", logData->version.get())
|
|
|
|
.detail("Committing", logData->queueCommittingVersion)
|
|
|
|
.detail("Commmitted", logData->queueCommittedVersion.get());
|
|
|
|
if (logData->committingQueue.canBeSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->committingQueue.send(Void());
|
|
|
|
}
|
|
|
|
|
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->stopped && logData->version.get() == std::max(logData->queueCommittingVersion,
|
|
|
|
logData->queueCommittedVersion.get())) {
|
|
|
|
wait(logData->queueCommittedVersion.whenAtLeast(logData->version.get()));
|
2019-02-08 09:02:29 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(logData->version.whenAtLeast(
|
|
|
|
std::max(logData->queueCommittingVersion, logData->queueCommittedVersion.get()) + 1))) {
|
|
|
|
while (self->queueCommitBegin != self->queueCommitEnd.get() &&
|
|
|
|
!self->largeDiskQueueCommitBytes.get()) {
|
|
|
|
wait(self->queueCommitEnd.whenAtLeast(self->queueCommitBegin) ||
|
|
|
|
self->largeDiskQueueCommitBytes.onChange());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2019-04-03 06:27:37 +08:00
|
|
|
self->sharedActors.send(doQueueCommit(self, logData, missingFinalCommit));
|
2021-03-11 02:06:03 +08:00
|
|
|
missingFinalCommit.clear();
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
when(wait(self->newLogData.onTrigger())) {}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogCommit(TLogData* self,
|
|
|
|
TLogCommitRequest req,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
PromiseStream<Void> warningCollectorInput) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Optional<UID> tlogDebugID;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.debugID.present()) {
|
2019-05-11 05:01:52 +08:00
|
|
|
tlogDebugID = nondeterministicRandom()->randomUniqueID();
|
2019-02-08 09:02:29 +08:00
|
|
|
g_traceBatch.addAttach("CommitAttachID", req.debugID.get().first(), tlogDebugID.get().first());
|
|
|
|
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.BeforeWaitForVersion");
|
|
|
|
}
|
|
|
|
|
|
|
|
logData->minKnownCommittedVersion = std::max(logData->minKnownCommittedVersion, req.minKnownCommittedVersion);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(logData->version.whenAtLeast(req.prevVersion));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Calling check_yield instead of yield to avoid a destruction ordering problem in simulation
|
|
|
|
if (g_network->check_yield(g_network->getCurrentTask())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
wait(delay(0, g_network->getCurrentTask()));
|
|
|
|
}
|
|
|
|
|
|
|
|
state double waitStartT = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
while (self->bytesInput - self->bytesDurable >= SERVER_KNOBS->TLOG_HARD_LIMIT_BYTES && !logData->stopped) {
|
2019-02-08 09:02:29 +08:00
|
|
|
if (now() - waitStartT >= 1) {
|
|
|
|
TraceEvent(SevWarn, "TLogUpdateLag", logData->logId)
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("Version", logData->version.get())
|
|
|
|
.detail("PersistentDataVersion", logData->persistentDataVersion)
|
|
|
|
.detail("PersistentDataDurableVersion", logData->persistentDataDurableVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
waitStartT = now();
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delayJittered(.005, TaskPriority::TLogCommit));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->stopped) {
|
|
|
|
req.reply.sendError(tlog_stopped());
|
2019-05-10 09:15:17 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->version.get() ==
|
|
|
|
req.prevVersion) { // Not a duplicate (check relies on critical section between here self->version.set() below!)
|
|
|
|
if (req.debugID.present())
|
2019-02-08 09:02:29 +08:00
|
|
|
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.Before");
|
|
|
|
|
2019-02-28 07:40:33 +08:00
|
|
|
//TraceEvent("TLogCommit", logData->logId).detail("Version", req.version);
|
|
|
|
commitMessages(self, logData, req.version, req.arena, req.messages);
|
|
|
|
|
|
|
|
logData->knownCommittedVersion = std::max(logData->knownCommittedVersion, req.knownCommittedVersion);
|
|
|
|
|
2019-07-23 06:44:49 +08:00
|
|
|
TLogQueueEntryRef qe;
|
2019-05-26 05:12:18 +08:00
|
|
|
// Log the changes to the persistent queue, to be committed by commitQueue()
|
|
|
|
qe.version = req.version;
|
|
|
|
qe.knownCommittedVersion = logData->knownCommittedVersion;
|
|
|
|
qe.messages = req.messages;
|
|
|
|
qe.id = logData->logId;
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentQueue->push(qe, logData);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
self->diskQueueCommitBytes += qe.expectedSize();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->diskQueueCommitBytes > SERVER_KNOBS->MAX_QUEUE_COMMIT_BYTES) {
|
2019-02-08 09:02:29 +08:00
|
|
|
self->largeDiskQueueCommitBytes.set(true);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Notifies the commitQueue actor to commit persistentQueue, and also unblocks tLogPeekMessages actors
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->version.set(req.version);
|
2019-04-24 07:17:54 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.debugID.present())
|
2019-02-08 09:02:29 +08:00
|
|
|
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.AfterTLogCommit");
|
|
|
|
}
|
|
|
|
// Send replies only once all prior messages have been received and committed.
|
|
|
|
state Future<Void> stopped = logData->stopCommit.onTrigger();
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(
|
|
|
|
timeoutWarning(logData->queueCommittedVersion.whenAtLeast(req.version) || stopped, 0.1, warningCollectorInput));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (stopped.isReady()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(logData->stopped);
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.sendError(tlog_stopped());
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (req.debugID.present())
|
2019-02-08 09:02:29 +08:00
|
|
|
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.After");
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.send(logData->durableKnownCommittedVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> initPersistentState(TLogData* self, Reference<LogData> logData) {
|
|
|
|
wait(self->persistentDataCommitLock.take());
|
2019-02-08 09:02:29 +08:00
|
|
|
state FlowLock::Releaser commitLockReleaser(self->persistentDataCommitLock);
|
|
|
|
|
|
|
|
// PERSIST: Initial setup of persistentData for a brand new tLog for a new database
|
2021-03-11 02:06:03 +08:00
|
|
|
state IKeyValueStore* storage = self->persistentData;
|
|
|
|
wait(ioTimeoutError(storage->init(), SERVER_KNOBS->TLOG_MAX_CREATE_DURATION));
|
|
|
|
storage->set(persistFormat);
|
|
|
|
storage->set(
|
|
|
|
KeyValueRef(BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistCurrentVersionKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->version.get(), Unversioned())));
|
|
|
|
storage->set(KeyValueRef(
|
|
|
|
BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistKnownCommittedVersionKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->knownCommittedVersion, Unversioned())));
|
|
|
|
storage->set(KeyValueRef(BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistLocalityKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->locality, Unversioned())));
|
|
|
|
storage->set(
|
|
|
|
KeyValueRef(BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistLogRouterTagsKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->logRouterTags, Unversioned())));
|
|
|
|
storage->set(KeyValueRef(BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistTxsTagsKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->txsTags, Unversioned())));
|
|
|
|
storage->set(
|
|
|
|
KeyValueRef(BinaryWriter::toValue(logData->logId, Unversioned()).withPrefix(persistRecoveryCountKeys.begin),
|
|
|
|
BinaryWriter::toValue(logData->recoveryCount, Unversioned())));
|
|
|
|
|
|
|
|
for (auto tag : logData->allTags) {
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(!logData->getTagData(tag));
|
|
|
|
logData->createTagData(tag, 0, true, true, true);
|
2021-03-11 02:06:03 +08:00
|
|
|
updatePersistentPopped(self, logData, logData->getTagData(tag));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
TraceEvent("TLogInitCommit", logData->logId);
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(ioTimeoutError(self->persistentData->commit(), SERVER_KNOBS->TLOG_MAX_CREATE_DURATION));
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> rejoinMasters(TLogData* self,
|
|
|
|
TLogInterface tli,
|
|
|
|
DBRecoveryCount recoveryCount,
|
|
|
|
Future<Void> registerWithMaster,
|
|
|
|
bool isPrimary) {
|
|
|
|
state UID lastMasterID(0, 0);
|
2019-02-08 09:02:29 +08:00
|
|
|
loop {
|
|
|
|
auto const& inf = self->dbInfo->get();
|
2021-03-11 02:06:03 +08:00
|
|
|
bool isDisplaced =
|
|
|
|
!std::count(inf.priorCommittedLogServers.begin(), inf.priorCommittedLogServers.end(), tli.id());
|
|
|
|
if (isPrimary) {
|
|
|
|
isDisplaced =
|
|
|
|
isDisplaced && inf.recoveryCount >= recoveryCount && inf.recoveryState != RecoveryState::UNINITIALIZED;
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
isDisplaced = isDisplaced &&
|
|
|
|
((inf.recoveryCount > recoveryCount && inf.recoveryState != RecoveryState::UNINITIALIZED) ||
|
|
|
|
(inf.recoveryCount == recoveryCount && inf.recoveryState == RecoveryState::FULLY_RECOVERED));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2019-05-30 05:45:34 +08:00
|
|
|
isDisplaced = isDisplaced && !inf.logSystemConfig.hasTLog(tli.id());
|
|
|
|
if (isDisplaced) {
|
|
|
|
TraceEvent("TLogDisplaced", tli.id())
|
|
|
|
.detail("Reason", "DBInfoDoesNotContain")
|
|
|
|
.detail("RecoveryCount", recoveryCount)
|
|
|
|
.detail("InfRecoveryCount", inf.recoveryCount)
|
|
|
|
.detail("RecoveryState", (int)inf.recoveryState)
|
|
|
|
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs))
|
|
|
|
.detail("PriorLogs", describe(inf.priorCommittedLogServers))
|
|
|
|
.detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
if (BUGGIFY)
|
|
|
|
wait(delay(SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01()));
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_removed();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (registerWithMaster.isReady()) {
|
|
|
|
if (self->dbInfo->get().master.id() != lastMasterID) {
|
|
|
|
// The TLogRejoinRequest is needed to establish communications with a new master, which doesn't have our
|
|
|
|
// TLogInterface
|
2019-02-08 09:02:29 +08:00
|
|
|
TLogRejoinRequest req(tli);
|
2020-02-15 04:33:43 +08:00
|
|
|
TraceEvent("TLogRejoining", tli.id()).detail("Master", self->dbInfo->get().master.id());
|
2019-02-08 09:02:29 +08:00
|
|
|
choose {
|
2019-08-29 05:40:50 +08:00
|
|
|
when(TLogRejoinReply rep =
|
|
|
|
wait(brokenPromiseToNever(self->dbInfo->get().master.tlogRejoin.getReply(req)))) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (rep.masterIsRecovered)
|
|
|
|
lastMasterID = self->dbInfo->get().master.id();
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(self->dbInfo->onChange())) {}
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(self->dbInfo->onChange());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(registerWithMaster || self->dbInfo->onChange());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> respondToRecovered(TLogInterface tli, Promise<Void> recoveryComplete) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state bool finishedRecovery = true;
|
|
|
|
try {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(recoveryComplete.getFuture());
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() != error_code_end_of_stream) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
finishedRecovery = false;
|
|
|
|
}
|
|
|
|
TraceEvent("TLogRespondToRecovered", tli.id()).detail("Finished", finishedRecovery);
|
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
TLogRecoveryFinishedRequest req = waitNext(tli.recoveryFinished.getFuture());
|
|
|
|
if (finishedRecovery) {
|
2019-02-08 09:02:29 +08:00
|
|
|
req.reply.send(Void());
|
|
|
|
} else {
|
|
|
|
req.reply.send(Never());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> cleanupPeekTrackers(LogData* logData) {
|
2019-02-08 09:02:29 +08:00
|
|
|
loop {
|
|
|
|
double minTimeUntilExpiration = SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME;
|
2019-07-10 09:20:20 +08:00
|
|
|
auto it = logData->peekTracker.begin();
|
2021-03-11 02:06:03 +08:00
|
|
|
while (it != logData->peekTracker.end()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
double timeUntilExpiration = it->second.lastUpdate + SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME - now();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (timeUntilExpiration < 1.0e-6) {
|
|
|
|
for (auto seq : it->second.sequence_version) {
|
|
|
|
if (!seq.second.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
seq.second.sendError(timed_out());
|
|
|
|
}
|
|
|
|
}
|
2019-07-10 09:20:20 +08:00
|
|
|
it = logData->peekTracker.erase(it);
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
|
|
|
minTimeUntilExpiration = std::min(minTimeUntilExpiration, timeUntilExpiration);
|
|
|
|
++it;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(minTimeUntilExpiration));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> logPeekTrackers(LogData* logData) {
|
2020-04-23 14:35:48 +08:00
|
|
|
loop {
|
|
|
|
int64_t logThreshold = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->peekTracker.size() > SERVER_KNOBS->PEEK_LOGGING_AMOUNT) {
|
2020-04-23 14:35:48 +08:00
|
|
|
std::vector<int64_t> peekCounts;
|
|
|
|
peekCounts.reserve(logData->peekTracker.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : logData->peekTracker) {
|
2020-04-23 14:35:48 +08:00
|
|
|
peekCounts.push_back(it.second.totalPeeks);
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
size_t pivot = peekCounts.size() - SERVER_KNOBS->PEEK_LOGGING_AMOUNT;
|
|
|
|
std::nth_element(peekCounts.begin(), peekCounts.begin() + pivot, peekCounts.end());
|
|
|
|
logThreshold = std::max<int64_t>(1, peekCounts[pivot]);
|
2020-04-23 14:35:48 +08:00
|
|
|
}
|
|
|
|
int logCount = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : logData->peekTracker) {
|
|
|
|
if (it.second.totalPeeks >= logThreshold) {
|
2020-04-23 14:35:48 +08:00
|
|
|
logCount++;
|
|
|
|
TraceEvent("PeekMetrics", logData->logId)
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("Tag", it.second.tag.toString())
|
|
|
|
.detail("Elapsed", now() - it.second.lastLogged)
|
|
|
|
.detail("MeanReplyBytes", it.second.replyBytes / it.second.totalPeeks)
|
|
|
|
.detail("TotalPeeks", it.second.totalPeeks)
|
|
|
|
.detail("UnblockedPeeks", it.second.unblockedPeeks)
|
|
|
|
.detail("DuplicatePeeks", it.second.duplicatePeeks)
|
|
|
|
.detail("Sequence",
|
|
|
|
it.second.sequence_version.size() ? it.second.sequence_version.begin()->first : -1)
|
|
|
|
.detail("IdleSeconds", it.second.idleTime)
|
|
|
|
.detail("IdleMax", it.second.idleMax)
|
|
|
|
.detail("QueueSeconds", it.second.queueTime)
|
|
|
|
.detail("QueueMax", it.second.queueMax)
|
|
|
|
.detail("BlockSeconds", it.second.blockTime)
|
|
|
|
.detail("BlockMax", it.second.blockMax)
|
|
|
|
.detail("WorkSeconds", it.second.workTime)
|
|
|
|
.detail("WorkMax", it.second.workMax);
|
2020-04-23 14:35:48 +08:00
|
|
|
it.second.resetMetrics();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(SERVER_KNOBS->PEEK_LOGGING_DELAY * std::max(1, logCount)));
|
2020-04-23 14:35:48 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void getQueuingMetrics(TLogData* self, Reference<LogData> logData, TLogQueuingMetricsRequest const& req) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TLogQueuingMetricsReply reply;
|
|
|
|
reply.localTime = now();
|
|
|
|
reply.instanceID = self->instanceID;
|
|
|
|
reply.bytesInput = self->bytesInput;
|
|
|
|
reply.bytesDurable = self->bytesDurable;
|
|
|
|
reply.storageBytes = self->persistentData->getStorageBytes();
|
2021-03-11 02:06:03 +08:00
|
|
|
// FIXME: Add the knownCommittedVersion to this message and change ratekeeper to use that version.
|
2019-02-08 09:02:29 +08:00
|
|
|
reply.v = logData->durableKnownCommittedVersion;
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.send(reply);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogSnapCreate(TLogSnapRequest snapReq, TLogData* self, Reference<LogData> logData) {
|
2019-06-20 02:16:30 +08:00
|
|
|
if (self->ignorePopUid != snapReq.snapUID.toString()) {
|
|
|
|
snapReq.reply.sendError(operation_failed());
|
|
|
|
return Void();
|
|
|
|
}
|
2019-07-21 16:00:29 +08:00
|
|
|
ExecCmdValueString snapArg(snapReq.snapPayload);
|
2019-06-20 02:16:30 +08:00
|
|
|
try {
|
2019-08-29 01:52:56 +08:00
|
|
|
int err = wait(execHelper(&snapArg, snapReq.snapUID, self->dataFolder, snapReq.role.toString()));
|
2019-06-20 02:16:30 +08:00
|
|
|
|
|
|
|
std::string uidStr = snapReq.snapUID.toString();
|
|
|
|
TraceEvent("ExecTraceTLog")
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("Uid", uidStr)
|
|
|
|
.detail("Status", err)
|
|
|
|
.detail("Role", snapReq.role)
|
|
|
|
.detail("Value", self->dataFolder)
|
|
|
|
.detail("ExecPayload", snapReq.snapPayload)
|
|
|
|
.detail("PersistentDataVersion", logData->persistentDataVersion)
|
|
|
|
.detail("PersistentDatadurableVersion", logData->persistentDataDurableVersion)
|
|
|
|
.detail("QueueCommittedVersion", logData->queueCommittedVersion.get())
|
|
|
|
.detail("Version", logData->version.get());
|
2019-06-20 02:16:30 +08:00
|
|
|
|
2019-07-13 07:26:28 +08:00
|
|
|
if (err != 0) {
|
|
|
|
throw operation_failed();
|
|
|
|
}
|
2019-06-20 02:16:30 +08:00
|
|
|
snapReq.reply.send(Void());
|
|
|
|
} catch (Error& e) {
|
2019-07-23 06:44:49 +08:00
|
|
|
TraceEvent("TLogSnapCreateError").error(e, true /*includeCancelled */);
|
2019-07-21 16:00:29 +08:00
|
|
|
if (e.code() != error_code_operation_cancelled) {
|
2019-07-05 01:20:29 +08:00
|
|
|
snapReq.reply.sendError(e);
|
2019-07-21 16:00:29 +08:00
|
|
|
} else {
|
|
|
|
throw e;
|
2019-07-05 01:20:29 +08:00
|
|
|
}
|
2019-06-20 02:16:30 +08:00
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogEnablePopReq(TLogEnablePopRequest enablePopReq, TLogData* self, Reference<LogData> logData) {
|
2019-07-05 01:37:33 +08:00
|
|
|
if (self->ignorePopUid != enablePopReq.snapUID.toString()) {
|
|
|
|
TraceEvent(SevWarn, "TLogPopDisableEnableUidMismatch")
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("IgnorePopUid", self->ignorePopUid)
|
|
|
|
.detail("UidStr", enablePopReq.snapUID.toString());
|
2019-07-05 01:37:33 +08:00
|
|
|
enablePopReq.reply.sendError(operation_failed());
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
TraceEvent("EnableTLogPlayAllIgnoredPops2");
|
|
|
|
// use toBePopped and issue all the pops
|
|
|
|
std::map<Tag, Version>::iterator it;
|
2019-07-13 01:56:27 +08:00
|
|
|
vector<Future<Void>> ignoredPops;
|
2019-07-05 01:37:33 +08:00
|
|
|
self->ignorePopRequest = false;
|
|
|
|
self->ignorePopDeadline = 0.0;
|
|
|
|
self->ignorePopUid = "";
|
|
|
|
for (it = self->toBePopped.begin(); it != self->toBePopped.end(); it++) {
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("PlayIgnoredPop").detail("Tag", it->first.toString()).detail("Version", it->second);
|
2019-07-05 01:37:33 +08:00
|
|
|
ignoredPops.push_back(tLogPopCore(self, it->first, it->second, logData));
|
|
|
|
}
|
|
|
|
TraceEvent("TLogExecCmdPopEnable")
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("UidStr", enablePopReq.snapUID.toString())
|
|
|
|
.detail("IgnorePopUid", self->ignorePopUid)
|
|
|
|
.detail("IgnporePopRequest", self->ignorePopRequest)
|
|
|
|
.detail("IgnporePopDeadline", self->ignorePopDeadline)
|
|
|
|
.detail("PersistentDataVersion", logData->persistentDataVersion)
|
|
|
|
.detail("PersistentDatadurableVersion", logData->persistentDataDurableVersion)
|
|
|
|
.detail("QueueCommittedVersion", logData->queueCommittedVersion.get())
|
|
|
|
.detail("Version", logData->version.get());
|
2019-07-05 01:37:33 +08:00
|
|
|
wait(waitForAll(ignoredPops));
|
|
|
|
self->toBePopped.clear();
|
|
|
|
enablePopReq.reply.send(Void());
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> serveTLogInterface(TLogData* self,
|
|
|
|
TLogInterface tli,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
PromiseStream<Void> warningCollectorInput) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Future<Void> dbInfoChange = Void();
|
|
|
|
|
|
|
|
loop choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(dbInfoChange)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
dbInfoChange = self->dbInfo->onChange();
|
|
|
|
bool found = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS) {
|
|
|
|
for (auto& logs : self->dbInfo->get().logSystemConfig.tLogs) {
|
|
|
|
if (std::count(logs.tLogs.begin(), logs.tLogs.end(), logData->logId)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
found = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (found && self->dbInfo->get().logSystemConfig.recruitmentID == logData->recruitmentID) {
|
|
|
|
logData->logSystem->set(ILogSystem::fromServerDBInfo(self->dbgid, self->dbInfo->get()));
|
|
|
|
if (!logData->isPrimary) {
|
|
|
|
logData->logSystem->get()->pop(logData->logRouterPoppedVersion,
|
|
|
|
logData->remoteTag,
|
|
|
|
logData->durableKnownCommittedVersion,
|
|
|
|
logData->locality);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!logData->isPrimary && logData->stopped) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("TLogAlreadyStopped", self->dbgid);
|
|
|
|
logData->removed = logData->removed && logData->logSystem->get()->endEpoch();
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
logData->logSystem->set(Reference<ILogSystem>());
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) {
|
|
|
|
logData->addActor.send(tLogPeekMessages(self, req, logData));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) {
|
2019-03-11 22:31:44 +08:00
|
|
|
logData->addActor.send(tLogPop(self, req, logData));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogCommitRequest req = waitNext(tli.commit.getFuture())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
//TraceEvent("TLogCommitReq", logData->logId).detail("Ver", req.version).detail("PrevVer", req.prevVersion).detail("LogVer", logData->version.get());
|
|
|
|
ASSERT(logData->isPrimary);
|
|
|
|
TEST(logData->stopped); // TLogCommitRequest while stopped
|
|
|
|
if (!logData->stopped)
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->addActor.send(tLogCommit(self, req, logData, warningCollectorInput));
|
2019-02-08 09:02:29 +08:00
|
|
|
else
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.sendError(tlog_stopped());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(ReplyPromise<TLogLockResult> reply = waitNext(tli.lock.getFuture())) {
|
|
|
|
logData->addActor.send(tLogLock(self, reply, logData));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogQueuingMetricsRequest req = waitNext(tli.getQueuingMetrics.getFuture())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
getQueuingMetrics(self, logData, req);
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogConfirmRunningRequest req = waitNext(tli.confirmRunning.getFuture())) {
|
|
|
|
if (req.debugID.present()) {
|
2019-05-11 05:01:52 +08:00
|
|
|
UID tlogDebugID = nondeterministicRandom()->randomUniqueID();
|
2019-02-08 09:02:29 +08:00
|
|
|
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), tlogDebugID.first());
|
|
|
|
g_traceBatch.addEvent("TransactionDebug", tlogDebugID.first(), "TLogServer.TLogConfirmRunningRequest");
|
|
|
|
}
|
|
|
|
if (!logData->stopped)
|
|
|
|
req.reply.send(Void());
|
|
|
|
else
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.sendError(tlog_stopped());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogDisablePopRequest req = waitNext(tli.disablePopRequest.getFuture())) {
|
2019-06-20 02:16:30 +08:00
|
|
|
if (self->ignorePopUid != "") {
|
|
|
|
TraceEvent(SevWarn, "TLogPopDisableonDisable")
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("IgnorePopUid", self->ignorePopUid)
|
|
|
|
.detail("UidStr", req.snapUID.toString())
|
|
|
|
.detail("PersistentDataVersion", logData->persistentDataVersion)
|
|
|
|
.detail("PersistentDatadurableVersion", logData->persistentDataDurableVersion)
|
|
|
|
.detail("QueueCommittedVersion", logData->queueCommittedVersion.get())
|
|
|
|
.detail("Version", logData->version.get());
|
2019-07-05 01:37:33 +08:00
|
|
|
req.reply.sendError(operation_failed());
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
// FIXME: As part of reverting snapshot V1, make ignorePopUid a UID instead of string
|
2019-07-21 16:00:29 +08:00
|
|
|
self->ignorePopRequest = true;
|
2019-07-05 01:37:33 +08:00
|
|
|
self->ignorePopUid = req.snapUID.toString();
|
|
|
|
self->ignorePopDeadline = g_network->now() + SERVER_KNOBS->TLOG_IGNORE_POP_AUTO_ENABLE_DELAY;
|
|
|
|
req.reply.send(Void());
|
2019-06-20 02:16:30 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogEnablePopRequest enablePopReq = waitNext(tli.enablePopRequest.getFuture())) {
|
|
|
|
logData->addActor.send(tLogEnablePopReq(enablePopReq, self, logData));
|
2019-06-20 02:16:30 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogSnapRequest snapReq = waitNext(tli.snapRequest.getFuture())) {
|
|
|
|
logData->addActor.send(tLogSnapCreate(snapReq, self, logData));
|
2019-06-20 02:16:30 +08:00
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
void removeLog(TLogData* self, Reference<LogData> logData) {
|
|
|
|
TraceEvent("TLogRemoved", logData->logId)
|
|
|
|
.detail("Input", logData->bytesInput.getValue())
|
|
|
|
.detail("Durable", logData->bytesDurable.getValue());
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->stopped = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!logData->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->recoveryComplete.sendError(end_of_stream());
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->addActor = PromiseStream<Future<Void>>(); // there could be items still in the promise stream if one of the
|
|
|
|
// actors threw an error immediately
|
2019-02-08 09:02:29 +08:00
|
|
|
self->id_data.erase(logData->logId);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->id_data.size()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return;
|
|
|
|
} else {
|
|
|
|
throw worker_removed();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> pullAsyncData(TLogData* self,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
std::vector<Tag> tags,
|
|
|
|
Version beginVersion,
|
|
|
|
Optional<Version> endVersion,
|
|
|
|
bool poppedIsKnownCommitted) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state Future<Void> dbInfoChange = Void();
|
|
|
|
state Reference<ILogSystem::IPeekCursor> r;
|
|
|
|
state Version tagAt = beginVersion;
|
|
|
|
state Version lastVer = 0;
|
|
|
|
|
|
|
|
while (!endVersion.present() || logData->version.get() < endVersion.get()) {
|
|
|
|
loop {
|
|
|
|
choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(r ? r->getMore(TaskPriority::TLogCommit) : Never())) { break; }
|
|
|
|
when(wait(dbInfoChange)) {
|
|
|
|
if (logData->logSystem->get()) {
|
|
|
|
r = logData->logSystem->get()->peek(logData->logId, tagAt, endVersion, tags, true);
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
|
|
|
r = Reference<ILogSystem::IPeekCursor>();
|
|
|
|
}
|
|
|
|
dbInfoChange = logData->logSystem->onChange();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
state double waitStartT = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
while (self->bytesInput - self->bytesDurable >= SERVER_KNOBS->TLOG_HARD_LIMIT_BYTES && !logData->stopped) {
|
2019-02-08 09:02:29 +08:00
|
|
|
if (now() - waitStartT >= 1) {
|
|
|
|
TraceEvent(SevWarn, "TLogUpdateLag", logData->logId)
|
2021-03-11 02:06:03 +08:00
|
|
|
.detail("Version", logData->version.get())
|
|
|
|
.detail("PersistentDataVersion", logData->persistentDataVersion)
|
|
|
|
.detail("PersistentDataDurableVersion", logData->persistentDataDurableVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
waitStartT = now();
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delayJittered(.005, TaskPriority::TLogCommit));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
state Version ver = 0;
|
|
|
|
state std::vector<TagsAndMessage> messages;
|
|
|
|
loop {
|
|
|
|
state bool foundMessage = r->hasMessage();
|
|
|
|
if (!foundMessage || r->version().version != ver) {
|
|
|
|
ASSERT(r->version().version > lastVer);
|
|
|
|
if (ver) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->stopped || (endVersion.present() && ver > endVersion.get())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (poppedIsKnownCommitted) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->knownCommittedVersion = std::max(logData->knownCommittedVersion, r->popped());
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->minKnownCommittedVersion =
|
|
|
|
std::max(logData->minKnownCommittedVersion, r->getMinKnownCommittedVersion());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
commitMessages(self, logData, ver, messages);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->terminated.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Log the changes to the persistent queue, to be committed by commitQueue()
|
|
|
|
AlternativeTLogQueueEntryRef qe;
|
|
|
|
qe.version = ver;
|
|
|
|
qe.knownCommittedVersion = logData->knownCommittedVersion;
|
|
|
|
qe.alternativeMessages = &messages;
|
|
|
|
qe.id = logData->logId;
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentQueue->push(qe, logData);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
self->diskQueueCommitBytes += qe.expectedSize();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->diskQueueCommitBytes > SERVER_KNOBS->MAX_QUEUE_COMMIT_BYTES) {
|
2019-02-08 09:02:29 +08:00
|
|
|
self->largeDiskQueueCommitBytes.set(true);
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Notifies the commitQueue actor to commit persistentQueue, and also unblocks tLogPeekMessages
|
|
|
|
// actors
|
|
|
|
logData->version.set(ver);
|
|
|
|
wait(yield(TaskPriority::TLogCommit));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
lastVer = ver;
|
|
|
|
ver = r->version().version;
|
|
|
|
messages.clear();
|
|
|
|
|
|
|
|
if (!foundMessage) {
|
|
|
|
ver--;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (ver > logData->version.get()) {
|
|
|
|
if (logData->stopped || (endVersion.present() && ver > endVersion.get())) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (poppedIsKnownCommitted) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->knownCommittedVersion = std::max(logData->knownCommittedVersion, r->popped());
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->minKnownCommittedVersion =
|
|
|
|
std::max(logData->minKnownCommittedVersion, r->getMinKnownCommittedVersion());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->terminated.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Log the changes to the persistent queue, to be committed by commitQueue()
|
|
|
|
TLogQueueEntryRef qe;
|
|
|
|
qe.version = ver;
|
|
|
|
qe.knownCommittedVersion = logData->knownCommittedVersion;
|
|
|
|
qe.messages = StringRef();
|
|
|
|
qe.id = logData->logId;
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentQueue->push(qe, logData);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
self->diskQueueCommitBytes += qe.expectedSize();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->diskQueueCommitBytes > SERVER_KNOBS->MAX_QUEUE_COMMIT_BYTES) {
|
2019-02-08 09:02:29 +08:00
|
|
|
self->largeDiskQueueCommitBytes.set(true);
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Notifies the commitQueue actor to commit persistentQueue, and also unblocks tLogPeekMessages
|
|
|
|
// actors
|
|
|
|
logData->version.set(ver);
|
|
|
|
wait(yield(TaskPriority::TLogCommit));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-25 01:59:19 +08:00
|
|
|
messages.emplace_back(r->getMessageWithTags(), r->getTags());
|
2019-02-08 09:02:29 +08:00
|
|
|
r->nextMessage();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
tagAt = std::max(r->version().version, logData->version.get() + 1);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogCore(TLogData* self,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
TLogInterface tli,
|
|
|
|
bool pulledRecoveryVersions) {
|
|
|
|
if (logData->removed.isReady()) {
|
|
|
|
wait(delay(0)); // to avoid iterator invalidation in restorePersistentState when removed is already ready
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(logData->removed.isError());
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->removed.getError().code() != error_code_worker_removed) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw logData->removed.getError();
|
|
|
|
}
|
|
|
|
|
|
|
|
removeLog(self, logData);
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
state PromiseStream<Void> warningCollectorInput;
|
2021-03-11 02:06:03 +08:00
|
|
|
state Future<Void> warningCollector =
|
|
|
|
timeoutWarningCollector(warningCollectorInput.getFuture(), 1.0, "TLogQueueCommitSlow", self->dbgid);
|
|
|
|
state Future<Void> error = actorCollection(logData->addActor.getFuture());
|
|
|
|
|
|
|
|
logData->addActor.send(waitFailureServer(tli.waitFailure.getFuture()));
|
|
|
|
logData->addActor.send(logData->removed);
|
|
|
|
// FIXME: update tlogMetrics to include new information, or possibly only have one copy for the shared instance
|
|
|
|
logData->addActor.send(traceCounters("TLogMetrics",
|
|
|
|
logData->logId,
|
|
|
|
SERVER_KNOBS->STORAGE_LOGGING_DELAY,
|
|
|
|
&logData->cc,
|
|
|
|
logData->logId.toString() + "/TLogMetrics"));
|
|
|
|
logData->addActor.send(serveTLogInterface(self, tli, logData, warningCollectorInput));
|
|
|
|
logData->addActor.send(cleanupPeekTrackers(logData.getPtr()));
|
|
|
|
logData->addActor.send(logPeekTrackers(logData.getPtr()));
|
|
|
|
|
|
|
|
if (!logData->isPrimary) {
|
2019-02-08 09:02:29 +08:00
|
|
|
std::vector<Tag> tags;
|
|
|
|
tags.push_back(logData->remoteTag);
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->addActor.send(
|
|
|
|
pullAsyncData(self,
|
|
|
|
logData,
|
|
|
|
tags,
|
|
|
|
pulledRecoveryVersions ? logData->recoveredAt + 1 : logData->unrecoveredBefore,
|
|
|
|
Optional<Version>(),
|
|
|
|
true));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
try {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(error);
|
2019-02-08 09:02:29 +08:00
|
|
|
throw internal_error();
|
2021-03-11 02:06:03 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() != error_code_worker_removed)
|
2019-02-08 09:02:29 +08:00
|
|
|
throw;
|
|
|
|
|
|
|
|
removeLog(self, logData);
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> checkEmptyQueue(TLogData* self) {
|
|
|
|
TraceEvent("TLogCheckEmptyQueueBegin", self->dbgid);
|
|
|
|
try {
|
2021-03-11 02:06:03 +08:00
|
|
|
TLogQueueEntry r = wait(self->persistentQueue->readNext(self));
|
2019-02-08 09:02:29 +08:00
|
|
|
throw internal_error();
|
|
|
|
} catch (Error& e) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (e.code() != error_code_end_of_stream)
|
|
|
|
throw;
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("TLogCheckEmptyQueueEnd", self->dbgid);
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> checkRecovered(TLogData* self) {
|
|
|
|
TraceEvent("TLogCheckRecoveredBegin", self->dbgid);
|
2021-03-11 02:06:03 +08:00
|
|
|
Optional<Value> v = wait(self->persistentData->readValue(StringRef()));
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("TLogCheckRecoveredEnd", self->dbgid);
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> restorePersistentState(TLogData* self,
|
|
|
|
LocalityData locality,
|
|
|
|
Promise<Void> oldLog,
|
|
|
|
Promise<Void> recovered,
|
|
|
|
PromiseStream<InitializeTLogRequest> tlogRequests) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state double startt = now();
|
|
|
|
state Reference<LogData> logData;
|
|
|
|
state KeyRange tagKeys;
|
|
|
|
// PERSIST: Read basic state from persistentData; replay persistentQueue but don't erase it
|
|
|
|
|
|
|
|
TraceEvent("TLogRestorePersistentState", self->dbgid);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
state IKeyValueStore* storage = self->persistentData;
|
2019-02-08 09:02:29 +08:00
|
|
|
wait(storage->init());
|
|
|
|
state Future<Optional<Value>> fFormat = storage->readValue(persistFormat.key);
|
2021-05-04 04:14:16 +08:00
|
|
|
state Future<RangeResult> fVers = storage->readRange(persistCurrentVersionKeys);
|
|
|
|
state Future<RangeResult> fKnownCommitted = storage->readRange(persistKnownCommittedVersionKeys);
|
|
|
|
state Future<RangeResult> fLocality = storage->readRange(persistLocalityKeys);
|
|
|
|
state Future<RangeResult> fLogRouterTags = storage->readRange(persistLogRouterTagsKeys);
|
|
|
|
state Future<RangeResult> fTxsTags = storage->readRange(persistTxsTagsKeys);
|
|
|
|
state Future<RangeResult> fRecoverCounts = storage->readRange(persistRecoveryCountKeys);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
// FIXME: metadata in queue?
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(waitForAll(std::vector{ fFormat }));
|
|
|
|
wait(waitForAll(std::vector{ fVers, fKnownCommitted, fLocality, fLogRouterTags, fTxsTags, fRecoverCounts }));
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (fFormat.get().present() && !persistFormatReadableRange.contains(fFormat.get().get())) {
|
|
|
|
// FIXME: remove when we no longer need to test upgrades from 4.X releases
|
|
|
|
if (g_network->isSimulated()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("ElapsedTime").detail("SimTime", now()).detail("RealTime", 0).detail("RandomUnseed", 0);
|
|
|
|
flushAndExit(0);
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent(SevError, "UnsupportedDBFormat", self->dbgid)
|
|
|
|
.detail("Format", fFormat.get().get())
|
|
|
|
.detail("Expected", persistFormat.value.toString());
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_recovery_failed();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!fFormat.get().present()) {
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult v = wait(self->persistentData->readRange(KeyRangeRef(StringRef(), LiteralStringRef("\xff")), 1));
|
2019-02-08 09:02:29 +08:00
|
|
|
if (!v.size()) {
|
|
|
|
TEST(true); // The DB is completely empty, so it was never initialized. Delete it.
|
|
|
|
throw worker_removed();
|
|
|
|
} else {
|
|
|
|
// This should never happen
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent(SevError, "NoDBFormatKey", self->dbgid).detail("FirstKey", v[0].key);
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(false);
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_recovery_failed();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
state std::vector<Future<ErrorOr<Void>>> removed;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (fFormat.get().get() == LiteralStringRef("FoundationDB/LogServer/2/3")) {
|
|
|
|
// FIXME: need for upgrades from 5.X to 6.0, remove once this upgrade path is no longer needed
|
|
|
|
if (recovered.canBeSet())
|
|
|
|
recovered.send(Void());
|
2019-02-08 09:02:29 +08:00
|
|
|
oldLog.send(Void());
|
2021-03-11 02:06:03 +08:00
|
|
|
while (!tlogRequests.isEmpty()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
tlogRequests.getFuture().pop().reply.sendError(recruitment_failed());
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(oldTLog_4_6::tLog(
|
|
|
|
self->persistentData, self->rawPersistentQueue, self->dbInfo, locality, self->dbgid, self->workerID));
|
2019-02-08 09:02:29 +08:00
|
|
|
throw internal_error();
|
|
|
|
}
|
|
|
|
|
|
|
|
ASSERT(fVers.get().size() == fRecoverCounts.get().size());
|
|
|
|
|
|
|
|
state std::map<UID, int8_t> id_locality;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : fLocality.get()) {
|
|
|
|
id_locality[BinaryReader::fromStringRef<UID>(it.key.removePrefix(persistLocalityKeys.begin), Unversioned())] =
|
|
|
|
BinaryReader::fromStringRef<int8_t>(it.value, Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
state std::map<UID, int> id_logRouterTags;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : fLogRouterTags.get()) {
|
|
|
|
id_logRouterTags[BinaryReader::fromStringRef<UID>(it.key.removePrefix(persistLogRouterTagsKeys.begin),
|
|
|
|
Unversioned())] =
|
|
|
|
BinaryReader::fromStringRef<int>(it.value, Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2019-06-20 09:15:09 +08:00
|
|
|
state std::map<UID, int> id_txsTags;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : fTxsTags.get()) {
|
|
|
|
id_txsTags[BinaryReader::fromStringRef<UID>(it.key.removePrefix(persistTxsTagsKeys.begin), Unversioned())] =
|
|
|
|
BinaryReader::fromStringRef<int>(it.value, Unversioned());
|
2019-06-20 09:15:09 +08:00
|
|
|
}
|
|
|
|
|
2019-02-08 09:02:29 +08:00
|
|
|
state std::map<UID, Version> id_knownCommitted;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : fKnownCommitted.get()) {
|
|
|
|
id_knownCommitted[BinaryReader::fromStringRef<UID>(it.key.removePrefix(persistKnownCommittedVersionKeys.begin),
|
|
|
|
Unversioned())] =
|
|
|
|
BinaryReader::fromStringRef<Version>(it.value, Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
state int idx = 0;
|
|
|
|
state Promise<Void> registerWithMaster;
|
|
|
|
state std::map<UID, TLogInterface> id_interf;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (idx = 0; idx < fVers.get().size(); idx++) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state KeyRef rawId = fVers.get()[idx].key.removePrefix(persistCurrentVersionKeys.begin);
|
2021-03-11 02:06:03 +08:00
|
|
|
UID id1 = BinaryReader::fromStringRef<UID>(rawId, Unversioned());
|
|
|
|
UID id2 = BinaryReader::fromStringRef<UID>(
|
|
|
|
fRecoverCounts.get()[idx].key.removePrefix(persistRecoveryCountKeys.begin), Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(id1 == id2);
|
|
|
|
|
|
|
|
TLogInterface recruited(id1, self->dbgid, locality);
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
DUMPTOKEN(recruited.peekMessages);
|
|
|
|
DUMPTOKEN(recruited.popMessages);
|
|
|
|
DUMPTOKEN(recruited.commit);
|
|
|
|
DUMPTOKEN(recruited.lock);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.confirmRunning);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.recoveryFinished);
|
|
|
|
DUMPTOKEN(recruited.disablePopRequest);
|
|
|
|
DUMPTOKEN(recruited.enablePopRequest);
|
|
|
|
DUMPTOKEN(recruited.snapRequest);
|
|
|
|
|
|
|
|
// We do not need the remoteTag, because we will not be loading any additional data
|
|
|
|
logData = Reference<LogData>(new LogData(self,
|
|
|
|
recruited,
|
|
|
|
Tag(),
|
|
|
|
true,
|
|
|
|
id_logRouterTags[id1],
|
|
|
|
id_txsTags[id1],
|
|
|
|
UID(),
|
|
|
|
std::vector<Tag>(),
|
|
|
|
"Restored"));
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->locality = id_locality[id1];
|
|
|
|
logData->stopped = true;
|
|
|
|
self->id_data[id1] = logData;
|
|
|
|
id_interf[id1] = recruited;
|
|
|
|
|
|
|
|
logData->knownCommittedVersion = id_knownCommitted[id1];
|
2021-03-11 02:06:03 +08:00
|
|
|
Version ver = BinaryReader::fromStringRef<Version>(fVers.get()[idx].value, Unversioned());
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->persistentDataVersion = ver;
|
|
|
|
logData->persistentDataDurableVersion = ver;
|
|
|
|
logData->version.set(ver);
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->recoveryCount =
|
|
|
|
BinaryReader::fromStringRef<DBRecoveryCount>(fRecoverCounts.get()[idx].value, Unversioned());
|
|
|
|
logData->removed =
|
|
|
|
rejoinMasters(self, recruited, logData->recoveryCount, registerWithMaster.getFuture(), false);
|
2019-02-08 09:02:29 +08:00
|
|
|
removed.push_back(errorOr(logData->removed));
|
|
|
|
|
|
|
|
TraceEvent("TLogRestorePersistentStateVer", id1).detail("Ver", ver);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion
|
|
|
|
// might be lost, but that is fine because we will get the corresponding data back, too.
|
|
|
|
tagKeys = prefixRange(rawId.withPrefix(persistTagPoppedKeys.begin));
|
2019-02-08 09:02:29 +08:00
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->removed.isReady())
|
|
|
|
break;
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult data = wait(self->persistentData->readRange(tagKeys, BUGGIFY ? 3 : 1 << 30, 1 << 20));
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!data.size())
|
|
|
|
break;
|
|
|
|
((KeyRangeRef&)tagKeys) = KeyRangeRef(keyAfter(data.back().key, tagKeys.arena()), tagKeys.end);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& kv : data) {
|
2019-02-08 09:02:29 +08:00
|
|
|
Tag tag = decodeTagPoppedKey(rawId, kv.key);
|
|
|
|
Version popped = decodeTagPoppedValue(kv.value);
|
|
|
|
TraceEvent("TLogRestorePopped", logData->logId).detail("Tag", tag.toString()).detail("To", popped);
|
|
|
|
auto tagData = logData->getTagData(tag);
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(!tagData);
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->createTagData(tag, popped, false, false, false);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
state Future<Void> allRemoved = waitForAll(removed);
|
2021-03-11 02:06:03 +08:00
|
|
|
state UID lastId = UID(1, 1); // initialized so it will not compare equal to a default UID
|
2019-02-08 09:02:29 +08:00
|
|
|
state double recoverMemoryLimit = SERVER_KNOBS->TLOG_RECOVER_MEMORY_LIMIT;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (BUGGIFY)
|
|
|
|
recoverMemoryLimit =
|
|
|
|
std::max<double>(SERVER_KNOBS->BUGGIFY_RECOVER_MEMORY_LIMIT, SERVER_KNOBS->TLOG_SPILL_THRESHOLD);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
try {
|
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (allRemoved.isReady()) {
|
|
|
|
TEST(true); // all tlogs removed during queue recovery
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_removed();
|
|
|
|
}
|
|
|
|
choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(TLogQueueEntry qe = wait(self->persistentQueue->readNext(self))) {
|
|
|
|
if (!self->queueOrder.size() || self->queueOrder.back() != qe.id)
|
|
|
|
self->queueOrder.push_back(qe.id);
|
|
|
|
if (qe.id != lastId) {
|
2019-02-08 09:02:29 +08:00
|
|
|
lastId = qe.id;
|
|
|
|
auto it = self->id_data.find(qe.id);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (it != self->id_data.end()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData = it->second;
|
|
|
|
} else {
|
|
|
|
logData = Reference<LogData>();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
//TraceEvent("TLogRecoveredQE", self->dbgid).detail("LogId", qe.id).detail("Ver", qe.version).detail("MessageBytes", qe.messages.size()).detail("Tags", qe.tags.size())
|
2021-03-11 02:06:03 +08:00
|
|
|
// .detail("Tag0", qe.tags.size() ? qe.tags[0].tag : invalidTag).detail("Version",
|
|
|
|
// logData->version.get());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData) {
|
|
|
|
logData->knownCommittedVersion =
|
|
|
|
std::max(logData->knownCommittedVersion, qe.knownCommittedVersion);
|
|
|
|
if (qe.version > logData->version.get()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
commitMessages(self, logData, qe.version, qe.arena(), qe.messages);
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->version.set(qe.version);
|
|
|
|
logData->queueCommittedVersion.set(qe.version);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
while (self->bytesInput - self->bytesDurable >= recoverMemoryLimit) {
|
2021-03-11 02:06:03 +08:00
|
|
|
TEST(true); // Flush excess data during TLog queue recovery
|
|
|
|
TraceEvent("FlushLargeQueueDuringRecovery", self->dbgid)
|
|
|
|
.detail("BytesInput", self->bytesInput)
|
|
|
|
.detail("BytesDurable", self->bytesDurable)
|
|
|
|
.detail("Version", logData->version.get())
|
|
|
|
.detail("PVer", logData->persistentDataVersion);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(updateStorage(self))) {}
|
|
|
|
when(wait(allRemoved)) { throw worker_removed(); }
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(allRemoved)) { throw worker_removed(); }
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (e.code() != error_code_end_of_stream)
|
|
|
|
throw;
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("TLogRestorePersistentStateDone", self->dbgid).detail("Took", now() - startt);
|
|
|
|
TEST(now() - startt >= 1.0); // TLog recovery took more than 1 second
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : self->id_data) {
|
|
|
|
if (it.second->queueCommittedVersion.get() == 0) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("TLogZeroVersion", self->dbgid).detail("LogId", it.first);
|
|
|
|
it.second->queueCommittedVersion.set(it.second->version.get());
|
|
|
|
}
|
|
|
|
it.second->recoveryComplete.sendError(end_of_stream());
|
2021-03-11 02:06:03 +08:00
|
|
|
self->sharedActors.send(tLogCore(self, it.second, id_interf[it.first], false));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (registerWithMaster.canBeSet())
|
|
|
|
registerWithMaster.send(Void());
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
bool tlogTerminated(TLogData* self, IKeyValueStore* persistentData, TLogQueue* persistentQueue, Error const& e) {
|
|
|
|
// Dispose the IKVS (destroying its data permanently) only if this shutdown is definitely permanent. Otherwise just
|
|
|
|
// close it.
|
2019-02-08 09:02:29 +08:00
|
|
|
if (e.code() == error_code_worker_removed || e.code() == error_code_recruitment_failed) {
|
|
|
|
persistentData->dispose();
|
|
|
|
persistentQueue->dispose();
|
|
|
|
} else {
|
|
|
|
persistentData->close();
|
|
|
|
persistentQueue->close();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (e.code() == error_code_worker_removed || e.code() == error_code_recruitment_failed ||
|
|
|
|
e.code() == error_code_file_not_found) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("TLogTerminated", self->dbgid).error(e, true);
|
|
|
|
return true;
|
|
|
|
} else
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> updateLogSystem(TLogData* self,
|
|
|
|
Reference<LogData> logData,
|
|
|
|
LogSystemConfig recoverFrom,
|
|
|
|
Reference<AsyncVar<Reference<ILogSystem>>> logSystem) {
|
2019-02-08 09:02:29 +08:00
|
|
|
loop {
|
|
|
|
bool found = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->dbInfo->get().logSystemConfig.recruitmentID == logData->recruitmentID) {
|
|
|
|
if (self->dbInfo->get().logSystemConfig.isNextGenerationOf(recoverFrom)) {
|
|
|
|
logSystem->set(ILogSystem::fromOldLogSystemConfig(
|
|
|
|
logData->logId, self->dbInfo->get().myLocality, self->dbInfo->get().logSystemConfig));
|
2019-02-08 09:02:29 +08:00
|
|
|
found = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (self->dbInfo->get().logSystemConfig.isEqualIds(recoverFrom)) {
|
|
|
|
logSystem->set(ILogSystem::fromLogSystemConfig(
|
|
|
|
logData->logId, self->dbInfo->get().myLocality, self->dbInfo->get().logSystemConfig, false, true));
|
2019-02-08 09:02:29 +08:00
|
|
|
found = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (self->dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS) {
|
|
|
|
logSystem->set(ILogSystem::fromLogSystemConfig(
|
|
|
|
logData->logId, self->dbInfo->get().myLocality, self->dbInfo->get().logSystemConfig, true));
|
2019-02-08 09:02:29 +08:00
|
|
|
found = true;
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!found) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logSystem->set(Reference<ILogSystem>());
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->logSystem->get()->pop(logData->logRouterPoppedVersion,
|
|
|
|
logData->remoteTag,
|
|
|
|
logData->durableKnownCommittedVersion,
|
|
|
|
logData->locality);
|
|
|
|
}
|
|
|
|
TraceEvent("TLogUpdate", self->dbgid)
|
|
|
|
.detail("LogId", logData->logId)
|
|
|
|
.detail("RecruitmentID", logData->recruitmentID)
|
|
|
|
.detail("DbRecruitmentID", self->dbInfo->get().logSystemConfig.recruitmentID)
|
|
|
|
.detail("RecoverFrom", recoverFrom.toString())
|
|
|
|
.detail("DbInfo", self->dbInfo->get().logSystemConfig.toString())
|
|
|
|
.detail("Found", found)
|
|
|
|
.detail("LogSystem", (bool)logSystem->get())
|
|
|
|
.detail("RecoveryState", (int)self->dbInfo->get().recoveryState);
|
|
|
|
for (auto it : self->dbInfo->get().logSystemConfig.oldTLogs) {
|
2019-02-08 09:02:29 +08:00
|
|
|
TraceEvent("TLogUpdateOld", self->dbgid).detail("LogId", logData->logId).detail("DbInfo", it.toString());
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(self->dbInfo->onChange());
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLogStart(TLogData* self, InitializeTLogRequest req, LocalityData locality) {
|
2019-02-08 09:02:29 +08:00
|
|
|
state TLogInterface recruited(self->dbgid, locality);
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
DUMPTOKEN(recruited.peekMessages);
|
|
|
|
DUMPTOKEN(recruited.popMessages);
|
|
|
|
DUMPTOKEN(recruited.commit);
|
|
|
|
DUMPTOKEN(recruited.lock);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.confirmRunning);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.recoveryFinished);
|
|
|
|
DUMPTOKEN(recruited.disablePopRequest);
|
|
|
|
DUMPTOKEN(recruited.enablePopRequest);
|
|
|
|
DUMPTOKEN(recruited.snapRequest);
|
|
|
|
|
|
|
|
for (auto it : self->id_data) {
|
|
|
|
if (!it.second->stopped) {
|
|
|
|
TraceEvent("TLogStoppedByNewRecruitment", self->dbgid)
|
|
|
|
.detail("StoppedId", it.first.toString())
|
|
|
|
.detail("RecruitedId", recruited.id())
|
|
|
|
.detail("EndEpoch", it.second->logSystem->get().getPtr() != 0);
|
|
|
|
if (!it.second->isPrimary && it.second->logSystem->get()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
it.second->removed = it.second->removed && it.second->logSystem->get()->endEpoch();
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (it.second->committingQueue.canBeSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
it.second->committingQueue.sendError(worker_removed());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
it.second->stopped = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!it.second->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
it.second->recoveryComplete.sendError(end_of_stream());
|
|
|
|
}
|
|
|
|
it.second->stopCommit.trigger();
|
|
|
|
}
|
|
|
|
|
2020-02-13 07:11:38 +08:00
|
|
|
bool recovering = (req.recoverFrom.logSystemType == LogSystemType::tagPartitioned);
|
2021-03-11 02:06:03 +08:00
|
|
|
state Reference<LogData> logData = Reference<LogData>(new LogData(self,
|
|
|
|
recruited,
|
|
|
|
req.remoteTag,
|
|
|
|
req.isPrimary,
|
|
|
|
req.logRouterTags,
|
|
|
|
req.txsTags,
|
|
|
|
req.recruitmentID,
|
|
|
|
req.allTags,
|
|
|
|
recovering ? "Recovered" : "Recruited"));
|
2019-02-08 09:02:29 +08:00
|
|
|
self->id_data[recruited.id()] = logData;
|
|
|
|
logData->locality = req.locality;
|
|
|
|
logData->recoveryCount = req.epoch;
|
|
|
|
logData->removed = rejoinMasters(self, recruited, req.epoch, Future<Void>(Void()), req.isPrimary);
|
|
|
|
self->queueOrder.push_back(recruited.id());
|
|
|
|
|
|
|
|
TraceEvent("TLogStart", logData->logId);
|
|
|
|
state Future<Void> updater;
|
|
|
|
state bool pulledRecoveryVersions = false;
|
|
|
|
try {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->removed.isReady()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw logData->removed.getError();
|
|
|
|
}
|
|
|
|
|
2020-02-13 07:11:38 +08:00
|
|
|
if (recovering) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->unrecoveredBefore = req.startVersion;
|
|
|
|
logData->recoveredAt = req.recoverAt;
|
|
|
|
logData->knownCommittedVersion = req.startVersion - 1;
|
|
|
|
logData->persistentDataVersion = logData->unrecoveredBefore - 1;
|
|
|
|
logData->persistentDataDurableVersion = logData->unrecoveredBefore - 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->queueCommittedVersion.set(logData->unrecoveredBefore - 1);
|
|
|
|
logData->version.set(logData->unrecoveredBefore - 1);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
logData->unpoppedRecoveredTags = req.allTags.size();
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(initPersistentState(self, logData) || logData->removed);
|
|
|
|
|
|
|
|
TraceEvent("TLogRecover", self->dbgid)
|
|
|
|
.detail("LogId", logData->logId)
|
|
|
|
.detail("At", req.recoverAt)
|
|
|
|
.detail("Known", req.knownCommittedVersion)
|
|
|
|
.detail("Unrecovered", logData->unrecoveredBefore)
|
|
|
|
.detail("Tags", describe(req.recoverTags))
|
|
|
|
.detail("Locality", req.locality)
|
|
|
|
.detail("LogRouterTags", logData->logRouterTags);
|
|
|
|
|
|
|
|
if (logData->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_removed();
|
|
|
|
}
|
|
|
|
|
|
|
|
updater = updateLogSystem(self, logData, req.recoverFrom, logData->logSystem);
|
|
|
|
|
|
|
|
logData->initialized = true;
|
|
|
|
self->newLogData.trigger();
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if ((req.isPrimary || req.recoverFrom.logRouterTags == 0) && !logData->stopped &&
|
|
|
|
logData->unrecoveredBefore <= req.recoverAt) {
|
|
|
|
if (req.recoverFrom.logRouterTags > 0 && req.locality != tagLocalitySatellite) {
|
2019-02-08 09:02:29 +08:00
|
|
|
logData->logRouterPopToVersion = req.recoverAt;
|
|
|
|
std::vector<Tag> tags;
|
|
|
|
tags.push_back(logData->remoteTag);
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(pullAsyncData(self, logData, tags, logData->unrecoveredBefore, req.recoverAt, true) ||
|
|
|
|
logData->removed);
|
|
|
|
} else if (!req.recoverTags.empty()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(logData->unrecoveredBefore > req.knownCommittedVersion);
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(pullAsyncData(
|
|
|
|
self, logData, req.recoverTags, req.knownCommittedVersion + 1, req.recoverAt, false) ||
|
|
|
|
logData->removed);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
pulledRecoveryVersions = true;
|
|
|
|
logData->knownCommittedVersion = req.recoverAt;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if ((req.isPrimary || req.recoverFrom.logRouterTags == 0) && logData->version.get() < req.recoverAt &&
|
|
|
|
!logData->stopped) {
|
2019-02-08 09:02:29 +08:00
|
|
|
// Log the changes to the persistent queue, to be committed by commitQueue()
|
|
|
|
TLogQueueEntryRef qe;
|
|
|
|
qe.version = req.recoverAt;
|
|
|
|
qe.knownCommittedVersion = logData->knownCommittedVersion;
|
|
|
|
qe.messages = StringRef();
|
|
|
|
qe.id = logData->logId;
|
2021-03-11 02:06:03 +08:00
|
|
|
self->persistentQueue->push(qe, logData);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
self->diskQueueCommitBytes += qe.expectedSize();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->diskQueueCommitBytes > SERVER_KNOBS->MAX_QUEUE_COMMIT_BYTES) {
|
2019-02-08 09:02:29 +08:00
|
|
|
self->largeDiskQueueCommitBytes.set(true);
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->version.set(req.recoverAt);
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_removed();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
logData->addActor.send(respondToRecovered(recruited, logData->recoveryComplete));
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
|
|
|
// Brand new tlog, initialization has already been done by caller
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(initPersistentState(self, logData) || logData->removed);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (logData->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw worker_removed();
|
|
|
|
}
|
|
|
|
|
|
|
|
logData->initialized = true;
|
|
|
|
self->newLogData.trigger();
|
|
|
|
|
|
|
|
logData->recoveryComplete.send(Void());
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(logData->committingQueue.getFuture() || logData->removed);
|
|
|
|
} catch (Error& e) {
|
2020-01-17 09:37:17 +08:00
|
|
|
req.reply.sendError(recruitment_failed());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (e.code() != error_code_worker_removed) {
|
2019-02-08 09:02:29 +08:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(0.0)); // if multiple recruitment requests were already in the promise stream make sure they are all
|
|
|
|
// started before any are removed
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
removeLog(self, logData);
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
req.reply.send(recruited);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent("TLogReady", logData->logId)
|
|
|
|
.detail("AllTags", describe(req.allTags))
|
|
|
|
.detail("Locality", logData->locality);
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
updater = Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(tLogCore(self, logData, recruited, pulledRecoveryVersions));
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2019-07-30 14:40:28 +08:00
|
|
|
ACTOR Future<Void> startSpillingInTenSeconds(TLogData* self, UID tlogId, Reference<AsyncVar<UID>> activeSharedTLog) {
|
|
|
|
wait(delay(10));
|
|
|
|
if (activeSharedTLog->get() != tlogId) {
|
|
|
|
// TODO: This should fully spill, but currently doing so will cause us to no longer update poppedVersion
|
|
|
|
// and QuietDatabase will hang thinking our TLog is behind.
|
|
|
|
self->targetVolatileBytes = SERVER_KNOBS->REFERENCE_SPILL_UPDATE_STORAGE_BYTE_LIMIT * 2;
|
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2019-02-08 09:02:29 +08:00
|
|
|
// New tLog (if !recoverFrom.size()) or restore from network
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> tLog(IKeyValueStore* persistentData,
|
|
|
|
IDiskQueue* persistentQueue,
|
|
|
|
Reference<AsyncVar<ServerDBInfo>> db,
|
|
|
|
LocalityData locality,
|
|
|
|
PromiseStream<InitializeTLogRequest> tlogRequests,
|
|
|
|
UID tlogId,
|
|
|
|
UID workerID,
|
|
|
|
bool restoreFromDisk,
|
|
|
|
Promise<Void> oldLog,
|
|
|
|
Promise<Void> recovered,
|
|
|
|
std::string folder,
|
|
|
|
Reference<AsyncVar<bool>> degraded,
|
|
|
|
Reference<AsyncVar<UID>> activeSharedTLog) {
|
|
|
|
state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder);
|
|
|
|
state Future<Void> error = actorCollection(self.sharedActors.getFuture());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
TraceEvent("SharedTlog", tlogId);
|
|
|
|
try {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (restoreFromDisk) {
|
|
|
|
wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests));
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(ioTimeoutError(checkEmptyQueue(&self) && checkRecovered(&self),
|
|
|
|
SERVER_KNOBS->TLOG_MAX_CREATE_DURATION));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// Disk errors need a chance to kill this actor.
|
2019-02-08 09:02:29 +08:00
|
|
|
wait(delay(0.000001));
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (recovered.canBeSet())
|
|
|
|
recovered.send(Void());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
self.sharedActors.send(commitQueue(&self));
|
|
|
|
self.sharedActors.send(updateStorageLoop(&self));
|
2019-10-24 02:15:54 +08:00
|
|
|
state Future<Void> activeSharedChange = Void();
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
loop {
|
|
|
|
choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(InitializeTLogRequest req = waitNext(tlogRequests.getFuture())) {
|
|
|
|
if (!self.tlogCache.exists(req.recruitmentID)) {
|
|
|
|
self.tlogCache.set(req.recruitmentID, req.reply.getFuture());
|
|
|
|
self.sharedActors.send(
|
|
|
|
self.tlogCache.removeOnReady(req.recruitmentID, tLogStart(&self, req, locality)));
|
2019-02-08 09:02:29 +08:00
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
forwardPromise(req.reply, self.tlogCache.get(req.recruitmentID));
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(error)) { throw internal_error(); }
|
|
|
|
when(wait(activeSharedChange)) {
|
2019-07-30 14:40:28 +08:00
|
|
|
if (activeSharedTLog->get() == tlogId) {
|
|
|
|
self.targetVolatileBytes = SERVER_KNOBS->TLOG_SPILL_THRESHOLD;
|
|
|
|
} else {
|
2021-03-11 02:06:03 +08:00
|
|
|
self.sharedActors.send(startSpillingInTenSeconds(&self, tlogId, activeSharedTLog));
|
2019-07-30 14:40:28 +08:00
|
|
|
}
|
2019-10-24 02:15:54 +08:00
|
|
|
activeSharedChange = activeSharedTLog->onChange();
|
2019-07-30 14:40:28 +08:00
|
|
|
}
|
2019-02-08 09:02:29 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
self.terminated.send(Void());
|
|
|
|
TraceEvent("TLogError", tlogId).error(e, true);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (recovered.canBeSet())
|
|
|
|
recovered.send(Void());
|
2019-02-08 09:02:29 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
while (!tlogRequests.isEmpty()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
tlogRequests.getFuture().pop().reply.sendError(recruitment_failed());
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : self.id_data) {
|
|
|
|
if (!it.second->recoveryComplete.isSet()) {
|
2019-02-08 09:02:29 +08:00
|
|
|
it.second->recoveryComplete.sendError(end_of_stream());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tlogTerminated(&self, persistentData, self.persistentQueue, e)) {
|
2019-02-08 09:02:29 +08:00
|
|
|
return Void();
|
|
|
|
} else {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// UNIT TESTS
|
|
|
|
struct DequeAllocatorStats {
|
|
|
|
static int64_t allocatedBytes;
|
|
|
|
};
|
|
|
|
|
|
|
|
int64_t DequeAllocatorStats::allocatedBytes = 0;
|
|
|
|
|
|
|
|
template <class T>
|
|
|
|
struct DequeAllocator : std::allocator<T> {
|
2021-03-11 02:06:03 +08:00
|
|
|
template <typename U>
|
2019-02-08 09:02:29 +08:00
|
|
|
struct rebind {
|
|
|
|
typedef DequeAllocator<U> other;
|
|
|
|
};
|
|
|
|
|
|
|
|
DequeAllocator() {}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
template <typename U>
|
2019-02-08 09:02:29 +08:00
|
|
|
DequeAllocator(DequeAllocator<U> const& u) : std::allocator<T>(u) {}
|
|
|
|
|
|
|
|
T* allocate(std::size_t n, std::allocator<void>::const_pointer hint = 0) {
|
|
|
|
DequeAllocatorStats::allocatedBytes += n * sizeof(T);
|
2021-03-11 02:06:03 +08:00
|
|
|
// fprintf(stderr, "Allocating %lld objects for %lld bytes (total allocated: %lld)\n", n, n * sizeof(T),
|
|
|
|
// DequeAllocatorStats::allocatedBytes);
|
2019-02-08 09:02:29 +08:00
|
|
|
return std::allocator<T>::allocate(n, hint);
|
|
|
|
}
|
|
|
|
void deallocate(T* p, std::size_t n) {
|
|
|
|
DequeAllocatorStats::allocatedBytes -= n * sizeof(T);
|
2021-03-11 02:06:03 +08:00
|
|
|
// fprintf(stderr, "Deallocating %lld objects for %lld bytes (total allocated: %lld)\n", n, n * sizeof(T),
|
|
|
|
// DequeAllocatorStats::allocatedBytes);
|
2019-02-08 09:02:29 +08:00
|
|
|
return std::allocator<T>::deallocate(p, n);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
TEST_CASE("/fdbserver/tlogserver/VersionMessagesOverheadFactor") {
|
2019-02-08 09:02:29 +08:00
|
|
|
|
|
|
|
typedef std::pair<Version, LengthPrefixedStringRef> TestType; // type used by versionMessages
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int i = 1; i < 9; ++i) {
|
|
|
|
for (int j = 0; j < 20; ++j) {
|
2019-02-08 09:02:29 +08:00
|
|
|
DequeAllocatorStats::allocatedBytes = 0;
|
|
|
|
DequeAllocator<TestType> allocator;
|
|
|
|
std::deque<TestType, DequeAllocator<TestType>> d(allocator);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
int numElements = deterministicRandom()->randomInt(pow(10, i - 1), pow(10, i));
|
|
|
|
for (int k = 0; k < numElements; ++k) {
|
2019-02-08 09:02:29 +08:00
|
|
|
d.push_back(TestType());
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
int removedElements = 0; // deterministicRandom()->randomInt(0, numElements); // FIXME: the overhead factor
|
|
|
|
// does not accurately account for removal!
|
|
|
|
for (int k = 0; k < removedElements; ++k) {
|
2019-02-08 09:02:29 +08:00
|
|
|
d.pop_front();
|
|
|
|
}
|
|
|
|
|
|
|
|
int64_t dequeBytes = DequeAllocatorStats::allocatedBytes + sizeof(std::deque<TestType>);
|
2021-03-11 02:06:03 +08:00
|
|
|
int64_t insertedBytes = (numElements - removedElements) * sizeof(TestType);
|
|
|
|
double overheadFactor =
|
|
|
|
std::max<double>(insertedBytes, dequeBytes - 10000) /
|
|
|
|
insertedBytes; // We subtract 10K here as an estimated upper bound for the fixed cost of an std::deque
|
|
|
|
// fprintf(stderr, "%d elements (%d inserted, %d removed):\n", numElements-removedElements, numElements,
|
|
|
|
// removedElements); fprintf(stderr, "Allocated %lld bytes to store %lld bytes (%lf overhead factor)\n",
|
|
|
|
// dequeBytes, insertedBytes, overheadFactor);
|
2019-02-08 09:02:29 +08:00
|
|
|
ASSERT(overheadFactor * 1024 <= SERVER_KNOBS->VERSION_MESSAGES_OVERHEAD_FACTOR_1024THS);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
} // namespace oldTLog_6_0
|