foundationdb/fdbclient/MasterProxyInterface.h

471 lines
16 KiB
C
Raw Normal View History

2017-05-26 04:48:44 +08:00
/*
* MasterProxyInterface.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
*
2017-05-26 04:48:44 +08:00
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
2017-05-26 04:48:44 +08:00
* http://www.apache.org/licenses/LICENSE-2.0
*
2017-05-26 04:48:44 +08:00
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBCLIENT_MASTERPROXYINTERFACE_H
#define FDBCLIENT_MASTERPROXYINTERFACE_H
#pragma once
#include <utility>
#include <vector>
#include "fdbclient/FDBTypes.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbclient/CommitTransaction.h"
#include "fdbserver/RatekeeperInterface.h"
#include "fdbclient/TagThrottle.h"
2017-05-26 04:48:44 +08:00
#include "flow/Stats.h"
#include "fdbrpc/TimedRequest.h"
2017-05-26 04:48:44 +08:00
struct MasterProxyInterface {
constexpr static FileIdentifier file_identifier = 8954922;
2017-05-26 04:48:44 +08:00
enum { LocationAwareLoadBalance = 1 };
enum { AlwaysFresh = 1 };
2017-05-26 04:48:44 +08:00
2020-05-02 05:30:50 +08:00
Optional<Key> processId;
bool provisional;
2017-05-26 04:48:44 +08:00
RequestStream< struct CommitTransactionRequest > commit;
RequestStream< struct GetReadVersionRequest > getConsistentReadVersion; // Returns a version which (1) is committed, and (2) is >= the latest version reported committed (by a commit response) when this request was sent
// (at some point between when this request is sent and when its response is received, the latest version reported committed)
RequestStream< struct GetKeyServerLocationsRequest > getKeyServersLocations;
2017-05-26 04:48:44 +08:00
RequestStream< struct GetStorageServerRejoinInfoRequest > getStorageServerRejoinInfo;
RequestStream<ReplyPromise<Void>> waitFailure;
RequestStream< struct GetRawCommittedVersionRequest > getRawCommittedVersion;
RequestStream< struct TxnStateRequest > txnState;
RequestStream< struct GetHealthMetricsRequest > getHealthMetrics;
RequestStream< struct ProxySnapRequest > proxySnapReq;
RequestStream< struct ExclusionSafetyCheckRequest > exclusionSafetyCheckReq;
RequestStream< struct GetDDMetricsRequest > getDDMetrics;
2017-05-26 04:48:44 +08:00
UID id() const { return commit.getEndpoint().token; }
std::string toString() const { return id().shortString(); }
bool operator == (MasterProxyInterface const& r) const { return id() == r.id(); }
bool operator != (MasterProxyInterface const& r) const { return id() != r.id(); }
NetworkAddress address() const { return commit.getEndpoint().getPrimaryAddress(); }
2017-05-26 04:48:44 +08:00
template <class Archive>
void serialize(Archive& ar) {
serializer(ar, processId, provisional, commit);
if( Archive::isDeserializing ) {
getConsistentReadVersion = RequestStream< struct GetReadVersionRequest >( commit.getEndpoint().getAdjustedEndpoint(1) );
getKeyServersLocations = RequestStream< struct GetKeyServerLocationsRequest >( commit.getEndpoint().getAdjustedEndpoint(2) );
getStorageServerRejoinInfo = RequestStream< struct GetStorageServerRejoinInfoRequest >( commit.getEndpoint().getAdjustedEndpoint(3) );
waitFailure = RequestStream<ReplyPromise<Void>>( commit.getEndpoint().getAdjustedEndpoint(4) );
getRawCommittedVersion = RequestStream< struct GetRawCommittedVersionRequest >( commit.getEndpoint().getAdjustedEndpoint(5) );
txnState = RequestStream< struct TxnStateRequest >( commit.getEndpoint().getAdjustedEndpoint(6) );
getHealthMetrics = RequestStream< struct GetHealthMetricsRequest >( commit.getEndpoint().getAdjustedEndpoint(7) );
proxySnapReq = RequestStream< struct ProxySnapRequest >( commit.getEndpoint().getAdjustedEndpoint(8) );
exclusionSafetyCheckReq = RequestStream< struct ExclusionSafetyCheckRequest >( commit.getEndpoint().getAdjustedEndpoint(9) );
getDDMetrics = RequestStream< struct GetDDMetricsRequest >( commit.getEndpoint().getAdjustedEndpoint(10) );
}
2017-05-26 04:48:44 +08:00
}
void initEndpoints() {
std::vector<std::pair<FlowReceiver*, TaskPriority>> streams;
streams.push_back(commit.getReceiver(TaskPriority::ReadSocket));
streams.push_back(getConsistentReadVersion.getReceiver(TaskPriority::ReadSocket));
streams.push_back(getKeyServersLocations.getReceiver(TaskPriority::ReadSocket)); //priority lowered to TaskPriority::DefaultEndpoint on the proxy
streams.push_back(getStorageServerRejoinInfo.getReceiver(TaskPriority::ProxyStorageRejoin));
streams.push_back(waitFailure.getReceiver());
streams.push_back(getRawCommittedVersion.getReceiver(TaskPriority::ProxyGetRawCommittedVersion));
streams.push_back(txnState.getReceiver());
streams.push_back(getHealthMetrics.getReceiver());
streams.push_back(proxySnapReq.getReceiver());
streams.push_back(exclusionSafetyCheckReq.getReceiver());
2020-05-19 01:56:10 +08:00
streams.push_back(getDDMetrics.getReceiver());
FlowTransport::transport().addEndpoints(streams);
2017-05-26 04:48:44 +08:00
}
};
// ClientDBInfo is all the information needed by a database client to access the database
// It is returned (and kept up to date) by the OpenDatabaseRequest interface of ClusterInterface
struct ClientDBInfo {
constexpr static FileIdentifier file_identifier = 5355080;
UID id; // Changes each time anything else changes
vector< MasterProxyInterface > proxies;
Optional<MasterProxyInterface> firstProxy; //not serialized, used for commitOnFirstProxy when the proxies vector has been shrunk
double clientTxnInfoSampleRate;
int64_t clientTxnInfoSizeLimit;
Optional<Value> forward;
double transactionTagSampleRate;
ClientDBInfo() : clientTxnInfoSampleRate(std::numeric_limits<double>::infinity()), clientTxnInfoSizeLimit(-1), transactionTagSampleRate(CLIENT_KNOBS->READ_TAG_SAMPLE_RATE) {}
bool operator == (ClientDBInfo const& r) const { return id == r.id; }
bool operator != (ClientDBInfo const& r) const { return id != r.id; }
template <class Archive>
void serialize(Archive& ar) {
if constexpr (!is_fb_function<Archive>) {
ASSERT(ar.protocolVersion().isValid());
}
serializer(ar, proxies, id, clientTxnInfoSampleRate, clientTxnInfoSizeLimit, forward, transactionTagSampleRate);
}
};
2017-05-26 04:48:44 +08:00
struct CommitID {
constexpr static FileIdentifier file_identifier = 14254927;
2017-05-26 04:48:44 +08:00
Version version; // returns invalidVersion if transaction conflicts
uint16_t txnBatchId;
Optional<Value> metadataVersion;
Optional<Standalone<VectorRef<int>>> conflictingKRIndices;
2017-05-26 04:48:44 +08:00
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version, txnBatchId, metadataVersion, conflictingKRIndices);
2017-05-26 04:48:44 +08:00
}
CommitID() : version(invalidVersion), txnBatchId(0) {}
2020-03-27 06:52:30 +08:00
CommitID(Version version, uint16_t txnBatchId, const Optional<Value>& metadataVersion,
const Optional<Standalone<VectorRef<int>>>& conflictingKRIndices = Optional<Standalone<VectorRef<int>>>())
: version(version), txnBatchId(txnBatchId), metadataVersion(metadataVersion),
conflictingKRIndices(conflictingKRIndices) {}
2017-05-26 04:48:44 +08:00
};
struct CommitTransactionRequest : TimedRequest {
constexpr static FileIdentifier file_identifier = 93948;
enum {
FLAG_IS_LOCK_AWARE = 0x1,
FLAG_FIRST_IN_BATCH = 0x2
};
2019-03-16 01:34:57 +08:00
bool isLockAware() const { return (flags & FLAG_IS_LOCK_AWARE) != 0; }
bool firstInBatch() const { return (flags & FLAG_FIRST_IN_BATCH) != 0; }
2017-05-26 04:48:44 +08:00
Arena arena;
CommitTransactionRef transaction;
ReplyPromise<CommitID> reply;
uint32_t flags;
2017-05-26 04:48:44 +08:00
Optional<UID> debugID;
CommitTransactionRequest() : flags(0) {}
2017-05-26 04:48:44 +08:00
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, transaction, reply, arena, flags, debugID);
2017-05-26 04:48:44 +08:00
}
};
static inline int getBytes( CommitTransactionRequest const& r ) {
2017-05-26 04:48:44 +08:00
// SOMEDAY: Optimize
//return r.arena.getSize(); // NOT correct because arena can be shared!
int total = sizeof(r);
for(auto m = r.transaction.mutations.begin(); m != r.transaction.mutations.end(); ++m)
total += m->expectedSize() + CLIENT_KNOBS->PROXY_COMMIT_OVERHEAD_BYTES;
2017-05-26 04:48:44 +08:00
for(auto i = r.transaction.read_conflict_ranges.begin(); i != r.transaction.read_conflict_ranges.end(); ++i)
total += i->expectedSize();
for(auto i = r.transaction.write_conflict_ranges.begin(); i != r.transaction.write_conflict_ranges.end(); ++i)
total += i->expectedSize();
return total;
}
struct GetReadVersionReply : public BasicLoadBalancedReply {
constexpr static FileIdentifier file_identifier = 15709388;
2017-05-26 04:48:44 +08:00
Version version;
bool locked;
Optional<Value> metadataVersion;
2017-05-26 04:48:44 +08:00
2020-04-18 00:06:45 +08:00
TransactionTagMap<ClientTagThrottleLimits> tagThrottleInfo;
Initialize in default constructor for GetReadVersionReply ==10473== Uninitialised byte(s) found during client check request ==10473== at 0x1BA9ACE: sendPacket(TransportData*, ISerializeSource const&, Endpoint const&, bool, bool) (FlowTransport.actor.cpp:1252) ==10473== by 0x877C05: (anonymous namespace)::NetworkSenderActorState<GetReadVersionReply, (anonymous namespace)::NetworkSenderActor<GetReadVersionReply> >::a_body1cont2(GetReadVersionReply const&, int) [clone .isra.0] (networksender.actor .h:40) ==10473== by 0x877CC6: a_body1when1 (networksender.actor.g.h:147) ==10473== by 0x877CC6: a_callback_fire (networksender.actor.g.h:161) ==10473== by 0x877CC6: ActorCallback<(anonymous namespace)::NetworkSenderActor<GetReadVersionReply>, 0, GetReadVersionReply>::fire(GetReadVersionReply const&) (flow.h:894) ==10473== by 0xC343A7: send<GetReadVersionReply&> (flow.h:343) ==10473== by 0xC343A7: send<GetReadVersionReply&> (fdbrpc.h:124) ==10473== by 0xC343A7: (anonymous namespace)::ForwardProxyActorState<(anonymous namespace)::ForwardProxyActor>::a_body1loopBody1when2(ReplyPromise<GetReadVersionReply> const&, int) (MasterProxyServer.actor.cpp:1814) ==10473== by 0xC33C10: (anonymous namespace)::ForwardProxyActorState<(anonymous namespace)::ForwardProxyActor>::a_body1loopBody1(int) (MasterProxyServer.actor.g.cpp:8167) ==10473== by 0xC35434: a_body1loopHead1 (MasterProxyServer.actor.g.cpp:8152) ==10473== by 0xC35434: a_body1loopBody1cont2 (MasterProxyServer.actor.g.cpp:8327) ==10473== by 0xC35434: a_body1loopBody1cont1when1 (MasterProxyServer.actor.g.cpp:8333) ==10473== by 0xC35434: a_body1loopBody1cont1when1 (MasterProxyServer.actor.g.cpp:8331) ==10473== by 0xC35434: a_callback_fire (MasterProxyServer.actor.g.cpp:8347) ==10473== by 0xC35434: ActorCallback<(anonymous namespace)::ForwardProxyActor, 3, Void>::fire(Void const&) (flow.h:894) ==10473== by 0x7E7BE7: SAV<Void>::finishSendAndDelPromiseRef() (flow.h:375) ==10473== by 0x8319FD: a_body1when1 (genericactors.actor.g.h:10892) ==10473== by 0x8319FD: a_callback_fire (genericactors.actor.g.h:10920) ==10473== by 0x8319FD: ActorCallback<(anonymous namespace)::ChooseActorActor<Void>, 0, Void>::fire(Void const&) (flow.h:894) ==10473== by 0x891917: void SAV<Void>::send<Void>(Void&&) (flow.h:343) ==10473== by 0x1C47ADC: send<Void> (flow.h:674) ==10473== by 0x1C47ADC: execTask (sim2.actor.cpp:1632) ==10473== by 0x1C47ADC: Sim2::RunLoopActorState<Sim2::RunLoopActor>::a_body1loopBody1cont1(Void const&, int) (sim2.actor.cpp:975) ==10473== by 0x1C47FF2: a_body1loopBody1when1 (sim2.actor.g.cpp:5092) ==10473== by 0x1C47FF2: Sim2::RunLoopActorState<Sim2::RunLoopActor>::a_body1loopBody1(int) (sim2.actor.g.cpp:5037) ==10473== by 0x1C47A6C: a_body1loopHead1 (sim2.actor.g.cpp:5020) ==10473== by 0x1C47A6C: Sim2::RunLoopActorState<Sim2::RunLoopActor>::a_body1loopBody1cont1(Void const&, int) (sim2.actor.g.cpp:5086) ==10473== Address 0x12db1ba1 is 2,977 bytes inside a recently re-allocated block of size 4,096 alloc'd ==10473== at 0x1CC5D7F: FastAllocator<4096>::allocate() (FastAlloc.cpp:290) ==10473== by 0x1CFAA68: operator new (FastAlloc.h:193) ==10473== by 0x1CFAA68: PacketWriter::nextBuffer() (Net2Packet.cpp:59) ==10473== by 0x1CFABD6: PacketWriter::writeAhead(int, SplitBuffer*) (Net2Packet.cpp:81) ==10473== by 0x1BA97EB: sendPacket(TransportData*, ISerializeSource const&, Endpoint const&, bool, bool) (FlowTransport.actor.cpp:1199) ==10473== by 0x7DEAD1: a_body1cont2 (networksender.actor.h:40) ==10473== by 0x7DEAD1: a_body1when1 (networksender.actor.g.h:147) ==10473== by 0x7DEAD1: a_callback_fire (networksender.actor.g.h:161) ==10473== by 0x7DEAD1: ActorCallback<(anonymous namespace)::NetworkSenderActor<GetValueReply>, 0, GetValueReply>::fire(GetValueReply const&) (flow.h:894) ==10473== by 0xF22767: send<GetValueReply&> (flow.h:343) ==10473== by 0xF22767: send<GetValueReply&> (fdbrpc.h:124) ==10473== by 0xF22767: (anonymous namespace)::GetValueQActorState<(anonymous namespace)::GetValueQActor>::a_body1cont5(int) [clone .isra.0] (storageserver.actor.cpp:890) ==10473== by 0xF2305C: (anonymous namespace)::GetValueQActorState<(anonymous namespace)::GetValueQActor>::a_body1cont3(int) [clone .isra.0] (storageserver.actor.g.cpp:1592) ==10473== by 0xF23447: a_body1cont2when1 (storageserver.actor.g.cpp:1627) ==10473== by 0xF23447: (anonymous namespace)::GetValueQActorState<(anonymous namespace)::GetValueQActor>::a_body1cont2(Void const&, int) [clone .isra.0] (storageserver.actor.g.cpp:1512) ==10473== by 0xF23507: a_body1when1 (storageserver.actor.g.cpp:1523) ==10473== by 0xF23507: a_callback_fire (storageserver.actor.g.cpp:1537) ==10473== by 0xF23507: ActorCallback<(anonymous namespace)::GetValueQActor, 0, Void>::fire(Void const&) (flow.h:894) ==10473== by 0x891917: void SAV<Void>::send<Void>(Void&&) (flow.h:343) ==10473== by 0x1C47ADC: send<Void> (flow.h:674) ==10473== by 0x1C47ADC: execTask (sim2.actor.cpp:1632) ==10473== by 0x1C47ADC: Sim2::RunLoopActorState<Sim2::RunLoopActor>::a_body1loopBody1cont1(Void const&, int) (sim2.actor.cpp:975) ==10473== by 0x1C47FF2: a_body1loopBody1when1 (sim2.actor.g.cpp:5092) ==10473== by 0x1C47FF2: Sim2::RunLoopActorState<Sim2::RunLoopActor>::a_body1loopBody1(int) (sim2.actor.g.cpp:5037) ==10473== Uninitialised value was created by a stack allocation ==10473== at 0xC342D0: (anonymous namespace)::ForwardProxyActorState<(anonymous namespace)::ForwardProxyActor>::a_body1loopBody1when2(ReplyPromise<GetReadVersionReply> const&, int) (MasterProxyServer.actor.g.cpp:8213)
2019-07-20 03:53:48 +08:00
GetReadVersionReply() : version(invalidVersion), locked(false) {}
2017-05-26 04:48:44 +08:00
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, BasicLoadBalancedReply::recentRequests, version, locked, metadataVersion, tagThrottleInfo);
2017-05-26 04:48:44 +08:00
}
};
struct GetReadVersionRequest : TimedRequest {
constexpr static FileIdentifier file_identifier = 838566;
2017-05-26 04:48:44 +08:00
enum {
PRIORITY_SYSTEM_IMMEDIATE = 15 << 24, // Highest possible priority, always executed even if writes are otherwise blocked
PRIORITY_DEFAULT = 8 << 24,
PRIORITY_BATCH = 1 << 24
};
enum {
FLAG_USE_MIN_KNOWN_COMMITTED_VERSION = 4,
FLAG_USE_PROVISIONAL_PROXIES = 2,
2017-05-26 04:48:44 +08:00
FLAG_CAUSAL_READ_RISKY = 1,
FLAG_PRIORITY_MASK = PRIORITY_SYSTEM_IMMEDIATE,
};
uint32_t transactionCount;
uint32_t flags;
TransactionPriority priority;
TransactionTagMap<uint32_t> tags;
2017-05-26 04:48:44 +08:00
Optional<UID> debugID;
ReplyPromise<GetReadVersionReply> reply;
GetReadVersionRequest() : transactionCount(1), flags(0) {}
GetReadVersionRequest(uint32_t transactionCount, TransactionPriority priority, uint32_t flags = 0, TransactionTagMap<uint32_t> tags = TransactionTagMap<uint32_t>(), Optional<UID> debugID = Optional<UID>())
: transactionCount(transactionCount), priority(priority), flags(flags), tags(tags), debugID(debugID)
{
flags = flags & ~FLAG_PRIORITY_MASK;
switch(priority) {
case TransactionPriority::BATCH:
flags |= PRIORITY_BATCH;
break;
case TransactionPriority::DEFAULT:
flags |= PRIORITY_DEFAULT;
break;
case TransactionPriority::IMMEDIATE:
flags |= PRIORITY_SYSTEM_IMMEDIATE;
break;
default:
ASSERT(false);
}
}
2017-05-26 04:48:44 +08:00
bool operator < (GetReadVersionRequest const& rhs) const { return priority < rhs.priority; }
2017-05-26 04:48:44 +08:00
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, transactionCount, flags, tags, debugID, reply);
if(ar.isDeserializing) {
if((flags & PRIORITY_SYSTEM_IMMEDIATE) == PRIORITY_SYSTEM_IMMEDIATE) {
priority = TransactionPriority::IMMEDIATE;
}
else if((flags & PRIORITY_DEFAULT) == PRIORITY_DEFAULT) {
priority = TransactionPriority::DEFAULT;
}
else if((flags & PRIORITY_BATCH) == PRIORITY_BATCH) {
priority = TransactionPriority::BATCH;
}
else {
priority = TransactionPriority::DEFAULT;
}
}
2017-05-26 04:48:44 +08:00
}
};
struct GetKeyServerLocationsReply {
constexpr static FileIdentifier file_identifier = 10636023;
Arena arena;
std::vector<std::pair<KeyRangeRef, vector<StorageServerInterface>>> results;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, results, arena);
}
};
struct GetKeyServerLocationsRequest {
constexpr static FileIdentifier file_identifier = 9144680;
Arena arena;
KeyRef begin;
Optional<KeyRef> end;
int limit;
bool reverse;
ReplyPromise<GetKeyServerLocationsReply> reply;
GetKeyServerLocationsRequest() : limit(0), reverse(false) {}
GetKeyServerLocationsRequest( KeyRef const& begin, Optional<KeyRef> const& end, int limit, bool reverse, Arena const& arena ) : begin( begin ), end( end ), limit( limit ), reverse( reverse ), arena( arena ) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, begin, end, limit, reverse, reply, arena);
}
};
2017-05-26 04:48:44 +08:00
struct GetRawCommittedVersionRequest {
constexpr static FileIdentifier file_identifier = 12954034;
2017-05-26 04:48:44 +08:00
Optional<UID> debugID;
ReplyPromise<GetReadVersionReply> reply;
explicit GetRawCommittedVersionRequest(Optional<UID> const& debugID = Optional<UID>()) : debugID(debugID) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, debugID, reply);
2017-05-26 04:48:44 +08:00
}
};
struct GetStorageServerRejoinInfoReply {
constexpr static FileIdentifier file_identifier = 9469225;
2017-05-26 04:48:44 +08:00
Version version;
Tag tag;
Optional<Tag> newTag;
bool newLocality;
std::vector<std::pair<Version, Tag>> history;
2017-05-26 04:48:44 +08:00
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version, tag, newTag, newLocality, history);
2017-05-26 04:48:44 +08:00
}
};
struct GetStorageServerRejoinInfoRequest {
constexpr static FileIdentifier file_identifier = 994279;
2017-05-26 04:48:44 +08:00
UID id;
Optional<Value> dcId;
2017-05-26 04:48:44 +08:00
ReplyPromise< GetStorageServerRejoinInfoReply > reply;
GetStorageServerRejoinInfoRequest() {}
explicit GetStorageServerRejoinInfoRequest( UID const& id, Optional<Value> const& dcId ) : id(id), dcId(dcId) {}
2017-05-26 04:48:44 +08:00
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, id, dcId, reply);
2017-05-26 04:48:44 +08:00
}
};
struct TxnStateRequest {
constexpr static FileIdentifier file_identifier = 15250781;
2017-05-26 04:48:44 +08:00
Arena arena;
VectorRef<KeyValueRef> data;
Sequence sequence;
bool last;
2020-04-07 11:58:43 +08:00
std::vector<Endpoint> broadcastInfo;
2017-05-26 04:48:44 +08:00
ReplyPromise<Void> reply;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, data, sequence, last, broadcastInfo, reply, arena);
2017-05-26 04:48:44 +08:00
}
};
struct GetHealthMetricsReply
{
2019-04-10 05:29:21 +08:00
constexpr static FileIdentifier file_identifier = 11544290;
Standalone<StringRef> serialized;
HealthMetrics healthMetrics;
explicit GetHealthMetricsReply(const HealthMetrics& healthMetrics = HealthMetrics()) :
healthMetrics(healthMetrics)
{
update(healthMetrics, true, true);
}
void update(const HealthMetrics& healthMetrics, bool detailedInput, bool detailedOutput)
{
this->healthMetrics.update(healthMetrics, detailedInput, detailedOutput);
BinaryWriter bw(IncludeVersion());
bw << this->healthMetrics;
serialized = bw.toValue();
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, serialized);
if (ar.isDeserializing) {
BinaryReader br(serialized, IncludeVersion());
br >> healthMetrics;
}
}
};
2019-04-17 06:00:12 +08:00
struct GetHealthMetricsRequest
{
constexpr static FileIdentifier file_identifier = 11403900;
ReplyPromise<struct GetHealthMetricsReply> reply;
bool detailed;
explicit GetHealthMetricsRequest(bool detailed = false) : detailed(detailed) {}
template <class Ar>
void serialize(Ar& ar)
{
serializer(ar, reply, detailed);
}
};
struct GetDDMetricsReply
{
constexpr static FileIdentifier file_identifier = 7277713;
2020-05-09 08:17:27 +08:00
Standalone<VectorRef<DDMetricsRef>> storageMetricsList;
GetDDMetricsReply() {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, storageMetricsList);
}
};
struct GetDDMetricsRequest {
constexpr static FileIdentifier file_identifier = 14536812;
KeyRange keys;
int shardLimit;
ReplyPromise<struct GetDDMetricsReply> reply;
GetDDMetricsRequest() {}
explicit GetDDMetricsRequest(KeyRange const& keys, const int shardLimit) : keys(keys), shardLimit(shardLimit) {}
template<class Ar>
void serialize(Ar& ar) {
serializer(ar, keys, shardLimit, reply);
}
};
2019-06-20 02:11:50 +08:00
struct ProxySnapRequest
2019-05-22 04:55:30 +08:00
{
2019-06-20 02:11:50 +08:00
constexpr static FileIdentifier file_identifier = 22204900;
Arena arena;
2019-06-20 02:11:50 +08:00
StringRef snapPayload;
UID snapUID;
ReplyPromise<Void> reply;
Optional<UID> debugID;
2019-06-20 02:11:50 +08:00
explicit ProxySnapRequest(Optional<UID> const& debugID = Optional<UID>()) : debugID(debugID) {}
explicit ProxySnapRequest(StringRef snap, UID snapUID, Optional<UID> debugID = Optional<UID>()) : snapPayload(snap), snapUID(snapUID), debugID(debugID) {}
template <class Ar>
void serialize(Ar& ar) {
2019-06-20 02:11:50 +08:00
serializer(ar, snapPayload, snapUID, reply, arena, debugID);
}
};
struct ExclusionSafetyCheckReply
{
2019-10-17 02:30:20 +08:00
constexpr static FileIdentifier file_identifier = 11;
bool safe;
ExclusionSafetyCheckReply() : safe(false) {}
explicit ExclusionSafetyCheckReply(bool safe) : safe(safe) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, safe);
}
};
struct ExclusionSafetyCheckRequest
{
constexpr static FileIdentifier file_identifier = 13852702;
vector<AddressExclusion> exclusions;
ReplyPromise<ExclusionSafetyCheckReply> reply;
ExclusionSafetyCheckRequest() {}
explicit ExclusionSafetyCheckRequest(vector<AddressExclusion> exclusions) : exclusions(exclusions) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, exclusions, reply);
}
};
2017-05-26 04:48:44 +08:00
#endif