Merge pull request #3772 from yliucode/rename

Rename master proxy as commit proxy
This commit is contained in:
Jingyu Zhou 2020-09-16 20:06:50 -07:00 committed by GitHub
commit ea27e4bd9c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
63 changed files with 642 additions and 600 deletions

View File

@ -24,22 +24,22 @@ def parse_args():
# (e)nd of a span with a better given name
locationToPhase = {
"NativeAPI.commit.Before": [],
"MasterProxyServer.batcher": [("b", "Commit")],
"MasterProxyServer.commitBatch.Before": [],
"MasterProxyServer.commitBatch.GettingCommitVersion": [("b", "CommitVersion")],
"MasterProxyServer.commitBatch.GotCommitVersion": [("e", "CommitVersion")],
"CommitProxyServer.batcher": [("b", "Commit")],
"CommitProxyServer.commitBatch.Before": [],
"CommitProxyServer.commitBatch.GettingCommitVersion": [("b", "CommitVersion")],
"CommitProxyServer.commitBatch.GotCommitVersion": [("e", "CommitVersion")],
"Resolver.resolveBatch.Before": [("b", "Resolver.PipelineWait")],
"Resolver.resolveBatch.AfterQueueSizeCheck": [],
"Resolver.resolveBatch.AfterOrderer": [("e", "Resolver.PipelineWait"), ("b", "Resolver.Conflicts")],
"Resolver.resolveBatch.After": [("e", "Resolver.Conflicts")],
"MasterProxyServer.commitBatch.AfterResolution": [("b", "Proxy.Processing")],
"MasterProxyServer.commitBatch.ProcessingMutations": [],
"MasterProxyServer.commitBatch.AfterStoreCommits": [("e", "Proxy.Processing")],
"CommitProxyServer.commitBatch.AfterResolution": [("b", "Proxy.Processing")],
"CommitProxyServer.commitBatch.ProcessingMutations": [],
"CommitProxyServer.commitBatch.AfterStoreCommits": [("e", "Proxy.Processing")],
"TLog.tLogCommit.BeforeWaitForVersion": [("b", "TLog.PipelineWait")],
"TLog.tLogCommit.Before": [("e", "TLog.PipelineWait")],
"TLog.tLogCommit.AfterTLogCommit": [("b", "TLog.FSync")],
"TLog.tLogCommit.After": [("e", "TLog.FSync")],
"MasterProxyServer.commitBatch.AfterLogPush": [("e", "Commit")],
"CommitProxyServer.commitBatch.AfterLogPush": [("e", "Commit")],
"NativeAPI.commit.After": [],
}

View File

@ -16,7 +16,7 @@ As an essential component of a database system, backup and restore is commonly u
## Background
FDB backup system continuously scan the databases key-value space, save key-value pairs and mutations at versions into range files and log files in blob storage. Specifically, mutation logs are generated at Proxy, and are written to transaction logs along with regular mutations. In production clusters like CK clusters, backup system is always on, which means each mutation is written twice to transaction logs, consuming about half of write bandwidth and about 40% of Proxy CPU time.
FDB backup system continuously scan the databases key-value space, save key-value pairs and mutations at versions into range files and log files in blob storage. Specifically, mutation logs are generated at CommitProxy, and are written to transaction logs along with regular mutations. In production clusters like CK clusters, backup system is always on, which means each mutation is written twice to transaction logs, consuming about half of write bandwidth and about 40% of CommitProxy CPU time.
The design of old backup system is [here](https://github.com/apple/foundationdb/blob/master/design/backup.md), and the data format of range files and mutations files is [here](https://github.com/apple/foundationdb/blob/master/design/backup-dataFormat.md). The technical overview of FDB is [here](https://github.com/apple/foundationdb/wiki/Technical-Overview-of-the-Database). The FDB recovery is described in this [doc](https://github.com/apple/foundationdb/blob/master/design/recovery-internals.md).
@ -37,7 +37,7 @@ The design of old backup system is [here](https://github.com/apple/foundationdb/
Feature priorities: Feature 1, 2, 3, 4, 5 are must-have; Feature 6 is better to have.
1. **Write bandwidth reduction by half**: removes the requirement to generate backup mutations at the Proxy, thus reduce TLog write bandwidth usage by half and significantly improve Proxy CPU usage;
1. **Write bandwidth reduction by half**: removes the requirement to generate backup mutations at the CommitProxy, thus reduce TLog write bandwidth usage by half and significantly improve CommitProxy CPU usage;
2. **Correctness**: The restored database must be consistent: each *restored* state (i.e., key-value pair) at a version `v` must match the original state at version `v`.
3. **Performance**: The backup system should be performant, mostly measured as a small CPU overhead on transaction logs and backup workers. The version lag on backup workers is an indicator of performance.
4. **Fault-tolerant**: The backup system should be fault-tolerant to node failures in the FDB cluster.
@ -153,9 +153,9 @@ The requirement of the new backup system raises several design challenges:
**Master**: The master is responsible for coordinating the transition of the FDB transaction sub-system from one generation to the next. In particular, the master recruits backup workers during the recovery.
**Transaction Logs (TLogs)**: The transaction logs make mutations durable to disk for fast commit latencies. The logs receive commits from the proxy in version order, and only respond to the proxy once the data has been written and fsync'ed to an append only mutation log on disk. Storage servers retrieve mutations from TLogs. Once the storage servers have persisted mutations, storage servers then pop the mutations from the TLogs.
**Transaction Logs (TLogs)**: The transaction logs make mutations durable to disk for fast commit latencies. The logs receive commits from the commit proxy in version order, and only respond to the commit proxy once the data has been written and fsync'ed to an append only mutation log on disk. Storage servers retrieve mutations from TLogs. Once the storage servers have persisted mutations, storage servers then pop the mutations from the TLogs.
**Proxy**: The proxies are responsible for committing transactions, and tracking the storage servers responsible for each range of keys. In the old backup system, Proxies are responsible to group mutations into backup mutations and write them to the database.
**CommitProxy**: The commit proxies are responsible for committing transactions, and tracking the storage servers responsible for each range of keys. In the old backup system, Proxies are responsible to group mutations into backup mutations and write them to the database.
**GrvProxy**: The GRV proxies are responsible for providing read versions.
## System overview

View File

@ -40,7 +40,7 @@ FoundationDB may return the following error codes from API functions. If you nee
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| external_client_already_loaded | 1040| External client has already been loaded |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| proxy_memory_limit_exceeded | 1042| Proxy commit memory limit exceeded |
| proxy_memory_limit_exceeded | 1042| CommitProxy commit memory limit exceeded |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| batch_transaction_throttled | 1051| Batch GRV request rate limit exceeded |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+

View File

@ -104,7 +104,7 @@ Field Name Description
``Name for the snapshot file`` recommended name for the disk snapshot cluster-name:ip-addr:port:UID
================================ ======================================================== ========================================================
``snapshot create binary`` will not be invoked on processes which does not have any persistent data (for example, Cluster Controller or Master or MasterProxy). Since these processes are stateless, there is no need for a snapshot. Any specialized configuration knobs used for one of these stateless processes need to be copied and restored externally.
``snapshot create binary`` will not be invoked on processes which does not have any persistent data (for example, Cluster Controller or Master or CommitProxy). Since these processes are stateless, there is no need for a snapshot. Any specialized configuration knobs used for one of these stateless processes need to be copied and restored externally.
Management of disk snapshots
----------------------------

View File

@ -27,7 +27,7 @@
"storage",
"transaction",
"resolution",
"proxy",
"commit_proxy",
"grv_proxy",
"master",
"test",
@ -61,7 +61,7 @@
"role":{
"$enum":[
"master",
"proxy",
"commit_proxy",
"grv_proxy",
"log",
"storage",
@ -447,7 +447,7 @@
],
"recovery_state":{
"required_resolvers":1,
"required_proxies":1,
"required_commit_proxies":1,
"required_grv_proxies":1,
"name":{ // "fully_recovered" is the healthy state; other states are normal to transition through but not to persist in
"$enum":[
@ -633,11 +633,11 @@
"address":"10.0.4.1"
}
],
"auto_proxies":3,
"auto_commit_proxies":3,
"auto_resolvers":1,
"auto_logs":3,
"backup_worker_enabled":1,
"proxies":5 // this field will be absent if a value has not been explicitly set
"commit_proxies":5 // this field will be absent if a value has not been explicitly set
},
"data":{
"least_operating_space_bytes_log_server":0,

View File

@ -470,8 +470,8 @@ void initHelp() {
"All keys between BEGINKEY (inclusive) and ENDKEY (exclusive) are cleared from the database. This command will succeed even if the specified range is empty, but may fail because of conflicts." ESCAPINGK);
helpMap["configure"] = CommandHelp(
"configure [new] "
"<single|double|triple|three_data_hall|three_datacenter|ssd|memory|memory-radixtree-beta|proxies=<PROXIES>|grv_"
"proxies=<GRV_PROXIES>|logs=<LOGS>|resolvers=<RESOLVERS>>*",
"<single|double|triple|three_data_hall|three_datacenter|ssd|memory|memory-radixtree-beta|commit_proxies=<"
"COMMIT_PROXIES>|grv_proxies=<GRV_PROXIES>|logs=<LOGS>|resolvers=<RESOLVERS>>*",
"change the database configuration",
"The `new' option, if present, initializes a new database with the given configuration rather than changing "
"the configuration of an existing one. When used, both a redundancy mode and a storage engine must be "
@ -479,13 +479,14 @@ void initHelp() {
"of data (survive one failure).\n triple - three copies of data (survive two failures).\n three_data_hall - "
"See the Admin Guide.\n three_datacenter - See the Admin Guide.\n\nStorage engine:\n ssd - B-Tree storage "
"engine optimized for solid state disks.\n memory - Durable in-memory storage engine for small "
"datasets.\n\nproxies=<PROXIES>: Sets the desired number of proxies in the cluster. Must be at least 1, or set "
"to -1 which restores the number of proxies to the default value.\n\ngrv_proxies=<GRV_PROXIES>: Sets the "
"desired number of GRV proxies in the cluster. Must be at least 1, or set to -1 which restores the number of "
"proxies to the default value.\n\nlogs=<LOGS>: Sets the desired number of log servers in the cluster. Must be "
"at least 1, or set to -1 which restores the number of logs to the default value.\n\nresolvers=<RESOLVERS>: "
"Sets the desired number of resolvers in the cluster. Must be at least 1, or set to -1 which restores the "
"number of resolvers to the default value.\n\nSee the FoundationDB Administration Guide for more information.");
"datasets.\n\ncommit_proxies=<COMMIT_PROXIES>: Sets the desired number of commit proxies in the cluster. Must "
"be at least 1, or set to -1 which restores the number of commit proxies to the default "
"value.\n\ngrv_proxies=<GRV_PROXIES>: Sets the desired number of GRV proxies in the cluster. Must be at least "
"1, or set to -1 which restores the number of GRV proxies to the default value.\n\nlogs=<LOGS>: Sets the "
"desired number of log servers in the cluster. Must be at least 1, or set to -1 which restores the number of "
"logs to the default value.\n\nresolvers=<RESOLVERS>: Sets the desired number of resolvers in the cluster. "
"Must be at least 1, or set to -1 which restores the number of resolvers to the default value.\n\nSee the "
"FoundationDB Administration Guide for more information.");
helpMap["fileconfigure"] = CommandHelp(
"fileconfigure [new] <FILENAME>",
"change the database configuration from a file",
@ -871,12 +872,13 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
fatalRecoveryState = true;
if (name == "recruiting_transaction_servers") {
description += format("\nNeed at least %d log servers across unique zones, %d proxies, "
"%d GRV proxies and %d resolvers.",
recoveryState["required_logs"].get_int(),
recoveryState["required_proxies"].get_int(),
recoveryState["required_grv_proxies"].get_int(),
recoveryState["required_resolvers"].get_int());
description +=
format("\nNeed at least %d log servers across unique zones, %d commit proxies, "
"%d GRV proxies and %d resolvers.",
recoveryState["required_logs"].get_int(),
recoveryState["required_commit_proxies"].get_int(),
recoveryState["required_grv_proxies"].get_int(),
recoveryState["required_resolvers"].get_int());
if (statusObjCluster.has("machines") && statusObjCluster.has("processes")) {
auto numOfNonExcludedProcessesAndZones = getNumOfNonExcludedProcessAndZones(statusObjCluster);
description += format("\nHave %d non-excluded processes on %d machines across %d zones.", numOfNonExcludedProcessesAndZones.first, getNumofNonExcludedMachines(statusObjCluster), numOfNonExcludedProcessesAndZones.second);
@ -1026,8 +1028,8 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
outputString += format("\n Exclusions - %d (type `exclude' for details)", excludedServersArr.size());
}
if (statusObjConfig.get("proxies", intVal))
outputString += format("\n Desired Proxies - %d", intVal);
if (statusObjConfig.get("commit_proxies", intVal))
outputString += format("\n Desired Commit Proxies - %d", intVal);
if (statusObjConfig.get("grv_proxies", intVal))
outputString += format("\n Desired GRV Proxies - %d", intVal);
@ -1790,14 +1792,14 @@ ACTOR Future<bool> configure( Database db, std::vector<StringRef> tokens, Refere
bool noChanges = conf.get().old_replication == conf.get().auto_replication &&
conf.get().old_logs == conf.get().auto_logs &&
conf.get().old_proxies == conf.get().auto_proxies &&
conf.get().old_commit_proxies == conf.get().auto_commit_proxies &&
conf.get().old_grv_proxies == conf.get().auto_grv_proxies &&
conf.get().old_resolvers == conf.get().auto_resolvers &&
conf.get().old_processes_with_transaction == conf.get().auto_processes_with_transaction &&
conf.get().old_machines_with_transaction == conf.get().auto_machines_with_transaction;
bool noDesiredChanges = noChanges && conf.get().old_logs == conf.get().desired_logs &&
conf.get().old_proxies == conf.get().desired_proxies &&
conf.get().old_commit_proxies == conf.get().desired_commit_proxies &&
conf.get().old_grv_proxies == conf.get().desired_grv_proxies &&
conf.get().old_resolvers == conf.get().desired_resolvers;
@ -1816,8 +1818,11 @@ ACTOR Future<bool> configure( Database db, std::vector<StringRef> tokens, Refere
outputString += format("| replication | %16s | %16s |\n", conf.get().old_replication.c_str(), conf.get().auto_replication.c_str());
outputString += format("| logs | %16d | %16d |", conf.get().old_logs, conf.get().auto_logs);
outputString += conf.get().auto_logs != conf.get().desired_logs ? format(" (manually set; would be %d)\n", conf.get().desired_logs) : "\n";
outputString += format("| proxies | %16d | %16d |", conf.get().old_proxies, conf.get().auto_proxies);
outputString += conf.get().auto_proxies != conf.get().desired_proxies ? format(" (manually set; would be %d)\n", conf.get().desired_proxies) : "\n";
outputString += format("| commit_proxies | %16d | %16d |", conf.get().old_commit_proxies,
conf.get().auto_commit_proxies);
outputString += conf.get().auto_commit_proxies != conf.get().desired_commit_proxies
? format(" (manually set; would be %d)\n", conf.get().desired_commit_proxies)
: "\n";
outputString += format("| grv_proxies | %16d | %16d |", conf.get().old_grv_proxies,
conf.get().auto_grv_proxies);
outputString += conf.get().auto_grv_proxies != conf.get().desired_grv_proxies
@ -2531,11 +2536,24 @@ void onOffGenerator(const char* text, const char *line, std::vector<std::string>
}
void configureGenerator(const char* text, const char *line, std::vector<std::string>& lc) {
const char* opts[] = {
"new", "single", "double", "triple", "three_data_hall", "three_datacenter", "ssd",
"ssd-1", "ssd-2", "memory", "memory-1", "memory-2", "memory-radixtree-beta", "proxies=",
"grv_proxies=", "logs=", "resolvers=", nullptr
};
const char* opts[] = { "new",
"single",
"double",
"triple",
"three_data_hall",
"three_datacenter",
"ssd",
"ssd-1",
"ssd-2",
"memory",
"memory-1",
"memory-2",
"memory-radixtree-beta",
"commit_proxies=",
"grv_proxies=",
"logs=",
"resolvers=",
nullptr };
arrayGenerator(text, line, opts, lc);
}

View File

@ -33,7 +33,7 @@ set(FDBCLIENT_SRCS
Knobs.h
ManagementAPI.actor.cpp
ManagementAPI.actor.h
MasterProxyInterface.h
CommitProxyInterface.h
MetricLogger.actor.cpp
MetricLogger.h
MonitorLeader.actor.cpp

View File

@ -25,7 +25,7 @@
#include "fdbclient/FDBTypes.h"
#include "fdbrpc/FailureMonitor.h"
#include "fdbclient/Status.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
// Streams from WorkerInterface that are safe and useful to call from a client.
// A ClientWorkerInterface is embedded as the first element of a WorkerInterface.

View File

@ -25,7 +25,7 @@
#include "fdbclient/FDBTypes.h"
#include "fdbrpc/FailureMonitor.h"
#include "fdbclient/Status.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/ClientWorkerInterface.h"
struct ClusterInterface {

View File

@ -1,6 +1,6 @@
/*
* MasterProxyInterface.h
* CommitProxyInterface.h
*
* This source file is part of the FoundationDB open source project
*
@ -19,8 +19,8 @@
* limitations under the License.
*/
#ifndef FDBCLIENT_MASTERPROXYINTERFACE_H
#define FDBCLIENT_MASTERPROXYINTERFACE_H
#ifndef FDBCLIENT_COMMITPROXYINTERFACE_H
#define FDBCLIENT_COMMITPROXYINTERFACE_H
#pragma once
#include <utility>
@ -36,7 +36,7 @@
#include "fdbrpc/TimedRequest.h"
#include "GrvProxyInterface.h"
struct MasterProxyInterface {
struct CommitProxyInterface {
constexpr static FileIdentifier file_identifier = 8954922;
enum { LocationAwareLoadBalance = 1 };
enum { AlwaysFresh = 1 };
@ -59,8 +59,8 @@ struct MasterProxyInterface {
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(); }
bool operator==(CommitProxyInterface const& r) const { return id() == r.id(); }
bool operator!=(CommitProxyInterface const& r) const { return id() != r.id(); }
NetworkAddress address() const { return commit.getEndpoint().getPrimaryAddress(); }
template <class Archive>
@ -100,9 +100,10 @@ struct MasterProxyInterface {
struct ClientDBInfo {
constexpr static FileIdentifier file_identifier = 5355080;
UID id; // Changes each time anything else changes
vector< GrvProxyInterface > grvProxies;
vector< MasterProxyInterface > masterProxies;
Optional<MasterProxyInterface> firstProxy; //not serialized, used for commitOnFirstProxy when the proxies vector has been shrunk
vector<GrvProxyInterface> grvProxies;
vector<CommitProxyInterface> commitProxies;
Optional<CommitProxyInterface>
firstCommitProxy; // not serialized, used for commitOnFirstProxy when the commit proxies vector has been shrunk
double clientTxnInfoSampleRate;
int64_t clientTxnInfoSizeLimit;
Optional<Value> forward;
@ -122,7 +123,7 @@ struct ClientDBInfo {
if constexpr (!is_fb_function<Archive>) {
ASSERT(ar.protocolVersion().isValid());
}
serializer(ar, grvProxies, masterProxies, id, clientTxnInfoSampleRate, clientTxnInfoSizeLimit, forward,
serializer(ar, grvProxies, commitProxies, id, clientTxnInfoSampleRate, clientTxnInfoSizeLimit, forward,
transactionTagSampleRate, transactionTagSampleCost);
}
};

View File

@ -25,7 +25,7 @@
#include "fdbclient/FDBTypes.h"
#include "fdbrpc/fdbrpc.h"
#include "fdbrpc/Locality.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/ClusterInterface.h"
const int MAX_CLUSTER_FILE_BYTES = 60000;

View File

@ -29,12 +29,12 @@ DatabaseConfiguration::DatabaseConfiguration()
void DatabaseConfiguration::resetInternal() {
// does NOT reset rawConfiguration
initialized = false;
proxyCount = grvProxyCount = resolverCount = desiredTLogCount = tLogWriteAntiQuorum = tLogReplicationFactor =
commitProxyCount = grvProxyCount = resolverCount = desiredTLogCount = tLogWriteAntiQuorum = tLogReplicationFactor =
storageTeamSize = desiredLogRouterCount = -1;
tLogVersion = TLogVersion::DEFAULT;
tLogDataStoreType = storageServerStoreType = KeyValueStoreType::END;
tLogSpillType = TLogSpillType::DEFAULT;
autoProxyCount = CLIENT_KNOBS->DEFAULT_AUTO_PROXIES;
autoCommitProxyCount = CLIENT_KNOBS->DEFAULT_AUTO_COMMIT_PROXIES;
autoGrvProxyCount = CLIENT_KNOBS->DEFAULT_AUTO_GRV_PROXIES;
autoResolverCount = CLIENT_KNOBS->DEFAULT_AUTO_RESOLVERS;
autoDesiredTLogCount = CLIENT_KNOBS->DEFAULT_AUTO_LOGS;
@ -165,40 +165,39 @@ void DatabaseConfiguration::setDefaultReplicationPolicy() {
bool DatabaseConfiguration::isValid() const {
if( !(initialized &&
tLogWriteAntiQuorum >= 0 &&
tLogWriteAntiQuorum <= tLogReplicationFactor/2 &&
tLogReplicationFactor >= 1 &&
storageTeamSize >= 1 &&
getDesiredProxies() >= 1 &&
getDesiredGrvProxies() >= 1 &&
getDesiredLogs() >= 1 &&
getDesiredResolvers() >= 1 &&
tLogVersion != TLogVersion::UNSET &&
tLogVersion >= TLogVersion::MIN_RECRUITABLE &&
tLogVersion <= TLogVersion::MAX_SUPPORTED &&
tLogDataStoreType != KeyValueStoreType::END &&
tLogSpillType != TLogSpillType::UNSET &&
!(tLogSpillType == TLogSpillType::REFERENCE && tLogVersion < TLogVersion::V3) &&
storageServerStoreType != KeyValueStoreType::END &&
autoProxyCount >= 1 &&
autoGrvProxyCount >= 1 &&
autoResolverCount >= 1 &&
autoDesiredTLogCount >= 1 &&
storagePolicy &&
tLogPolicy &&
getDesiredRemoteLogs() >= 1 &&
remoteTLogReplicationFactor >= 0 &&
repopulateRegionAntiQuorum >= 0 &&
repopulateRegionAntiQuorum <= 1 &&
usableRegions >= 1 &&
usableRegions <= 2 &&
regions.size() <= 2 &&
( usableRegions == 1 || regions.size() == 2 ) &&
( regions.size() == 0 || regions[0].priority >= 0 ) &&
( regions.size() == 0 || tLogPolicy->info() != "dcid^2 x zoneid^2 x 1") ) ) { //We cannot specify regions with three_datacenter replication
tLogWriteAntiQuorum >= 0 &&
tLogWriteAntiQuorum <= tLogReplicationFactor/2 &&
tLogReplicationFactor >= 1 &&
storageTeamSize >= 1 &&
getDesiredCommitProxies() >= 1 &&
getDesiredGrvProxies() >= 1 &&
getDesiredLogs() >= 1 &&
getDesiredResolvers() >= 1 &&
tLogVersion != TLogVersion::UNSET &&
tLogVersion >= TLogVersion::MIN_RECRUITABLE &&
tLogVersion <= TLogVersion::MAX_SUPPORTED &&
tLogDataStoreType != KeyValueStoreType::END &&
tLogSpillType != TLogSpillType::UNSET &&
!(tLogSpillType == TLogSpillType::REFERENCE && tLogVersion < TLogVersion::V3) &&
storageServerStoreType != KeyValueStoreType::END &&
autoCommitProxyCount >= 1 &&
autoGrvProxyCount >= 1 &&
autoResolverCount >= 1 &&
autoDesiredTLogCount >= 1 &&
storagePolicy &&
tLogPolicy &&
getDesiredRemoteLogs() >= 1 &&
remoteTLogReplicationFactor >= 0 &&
repopulateRegionAntiQuorum >= 0 &&
repopulateRegionAntiQuorum <= 1 &&
usableRegions >= 1 &&
usableRegions <= 2 &&
regions.size() <= 2 &&
( usableRegions == 1 || regions.size() == 2 ) &&
( regions.size() == 0 || regions[0].priority >= 0 ) &&
( regions.size() == 0 || tLogPolicy->info() != "dcid^2 x zoneid^2 x 1") ) ) { //We cannot specify regions with three_datacenter replication
return false;
}
std::set<Key> dcIds;
dcIds.insert(Key());
for(auto& r : regions) {
@ -318,11 +317,11 @@ StatusObject DatabaseConfiguration::toJSON(bool noPolicies) const {
if (desiredTLogCount != -1 || isOverridden("logs")) {
result["logs"] = desiredTLogCount;
}
if (proxyCount != -1 || isOverridden("proxies")) {
result["proxies"] = proxyCount;
if (commitProxyCount != -1 || isOverridden("commit_proxies")) {
result["commit_proxies"] = commitProxyCount;
}
if (grvProxyCount != -1 || isOverridden("grv_proxies")) {
result["grv_proxies"] = proxyCount;
result["grv_proxies"] = commitProxyCount;
}
if (resolverCount != -1 || isOverridden("resolvers")) {
result["resolvers"] = resolverCount;
@ -336,8 +335,8 @@ StatusObject DatabaseConfiguration::toJSON(bool noPolicies) const {
if (repopulateRegionAntiQuorum != 0 || isOverridden("repopulate_anti_quorum")) {
result["repopulate_anti_quorum"] = repopulateRegionAntiQuorum;
}
if (autoProxyCount != CLIENT_KNOBS->DEFAULT_AUTO_PROXIES || isOverridden("auto_proxies")) {
result["auto_proxies"] = autoProxyCount;
if (autoCommitProxyCount != CLIENT_KNOBS->DEFAULT_AUTO_COMMIT_PROXIES || isOverridden("auto_commit_proxies")) {
result["auto_commit_proxies"] = autoCommitProxyCount;
}
if (autoGrvProxyCount != CLIENT_KNOBS->DEFAULT_AUTO_GRV_PROXIES || isOverridden("auto_grv_proxies")) {
result["auto_grv_proxies"] = autoGrvProxyCount;
@ -419,8 +418,8 @@ bool DatabaseConfiguration::setInternal(KeyRef key, ValueRef value) {
if (ck == LiteralStringRef("initialized")) {
initialized = true;
} else if (ck == LiteralStringRef("proxies")) {
parse(&proxyCount, value);
} else if (ck == LiteralStringRef("commit_proxies")) {
parse(&commitProxyCount, value);
} else if (ck == LiteralStringRef("grv_proxies")) {
parse(&grvProxyCount, value);
} else if (ck == LiteralStringRef("resolvers")) {
@ -459,8 +458,8 @@ bool DatabaseConfiguration::setInternal(KeyRef key, ValueRef value) {
} else if (ck == LiteralStringRef("storage_engine")) {
parse((&type), value);
storageServerStoreType = (KeyValueStoreType::StoreType)type;
} else if (ck == LiteralStringRef("auto_proxies")) {
parse(&autoProxyCount, value);
} else if (ck == LiteralStringRef("auto_commit_proxies")) {
parse(&autoCommitProxyCount, value);
} else if (ck == LiteralStringRef("auto_grv_proxies")) {
parse(&autoGrvProxyCount, value);
} else if (ck == LiteralStringRef("auto_resolvers")) {

View File

@ -149,9 +149,9 @@ struct DatabaseConfiguration {
return std::min(tLogReplicationFactor - 1 - tLogWriteAntiQuorum, storageTeamSize - 1);
}
// Proxy Servers
int32_t proxyCount;
int32_t autoProxyCount;
// CommitProxy Servers
int32_t commitProxyCount;
int32_t autoCommitProxyCount;
int32_t grvProxyCount;
int32_t autoGrvProxyCount;
@ -192,7 +192,10 @@ struct DatabaseConfiguration {
bool isExcludedServer( NetworkAddressList ) const;
std::set<AddressExclusion> getExcludedServers() const;
int32_t getDesiredProxies() const { if(proxyCount == -1) return autoProxyCount; return proxyCount; }
int32_t getDesiredCommitProxies() const {
if (commitProxyCount == -1) return autoCommitProxyCount;
return commitProxyCount;
}
int32_t getDesiredGrvProxies() const {
if (grvProxyCount == -1) return autoGrvProxyCount;
return grvProxyCount;

View File

@ -29,7 +29,7 @@
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/KeyRangeMap.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/SpecialKeySpace.actor.h"
#include "fdbrpc/QueueModel.h"
#include "fdbrpc/MultiInterface.h"
@ -68,7 +68,7 @@ struct LocationInfo : MultiInterface<ReferencedInterface<StorageServerInterface>
}
};
using ProxyInfo = ModelInterface<MasterProxyInterface>;
using CommitProxyInfo = ModelInterface<CommitProxyInterface>;
using GrvProxyInfo = ModelInterface<GrvProxyInterface>;
class ClientTagThrottleData : NonCopyable {
@ -165,8 +165,8 @@ public:
bool sampleOnCost(uint64_t cost) const;
void updateProxies();
Reference<ProxyInfo> getMasterProxies(bool useProvisionalProxies);
Future<Reference<ProxyInfo>> getMasterProxiesFuture(bool useProvisionalProxies);
Reference<CommitProxyInfo> getCommitProxies(bool useProvisionalProxies);
Future<Reference<CommitProxyInfo>> getCommitProxiesFuture(bool useProvisionalProxies);
Reference<GrvProxyInfo> getGrvProxies(bool useProvisionalProxies);
Future<Void> onProxiesChanged();
Future<HealthMetrics> getHealthMetrics(bool detailed);
@ -219,9 +219,9 @@ public:
Reference<AsyncVar<Reference<ClusterConnectionFile>>> connectionFile;
AsyncTrigger proxiesChangeTrigger;
Future<Void> monitorProxiesInfoChange;
Reference<ProxyInfo> masterProxies;
Reference<CommitProxyInfo> commitProxies;
Reference<GrvProxyInfo> grvProxies;
bool proxyProvisional;
bool proxyProvisional; // Provisional commit proxy and grv proxy are used at the same time.
UID proxiesLastChange;
LocalityData clientLocality;
QueueModel queueModel;

View File

@ -27,6 +27,8 @@
// with RateKeeper to gather health information of the cluster.
struct GrvProxyInterface {
constexpr static FileIdentifier file_identifier = 8743216;
enum { LocationAwareLoadBalance = 1 };
enum { AlwaysFresh = 1 };
Optional<Key> processId;
bool provisional;

View File

@ -52,7 +52,7 @@ void ClientKnobs::initialize(bool randomize) {
init( COORDINATOR_RECONNECTION_DELAY, 1.0 );
init( CLIENT_EXAMPLE_AMOUNT, 20 );
init( MAX_CLIENT_STATUS_AGE, 1.0 );
init( MAX_MASTER_PROXY_CONNECTIONS, 5 ); if( randomize && BUGGIFY ) MAX_MASTER_PROXY_CONNECTIONS = 1;
init( MAX_COMMIT_PROXY_CONNECTIONS, 5 ); if( randomize && BUGGIFY ) MAX_COMMIT_PROXY_CONNECTIONS = 1;
init( MAX_GRV_PROXY_CONNECTIONS, 3 ); if( randomize && BUGGIFY ) MAX_GRV_PROXY_CONNECTIONS = 1;
init( STATUS_IDLE_TIMEOUT, 120.0 );
@ -171,7 +171,7 @@ void ClientKnobs::initialize(bool randomize) {
init( MIN_CLEANUP_SECONDS, 3600.0 );
// Configuration
init( DEFAULT_AUTO_PROXIES, 3 );
init( DEFAULT_AUTO_COMMIT_PROXIES, 3 );
init( DEFAULT_AUTO_GRV_PROXIES, 1 );
init( DEFAULT_AUTO_RESOLVERS, 1 );
init( DEFAULT_AUTO_LOGS, 3 );

View File

@ -46,7 +46,7 @@ public:
double COORDINATOR_RECONNECTION_DELAY;
int CLIENT_EXAMPLE_AMOUNT;
double MAX_CLIENT_STATUS_AGE;
int MAX_MASTER_PROXY_CONNECTIONS;
int MAX_COMMIT_PROXY_CONNECTIONS;
int MAX_GRV_PROXY_CONNECTIONS;
double STATUS_IDLE_TIMEOUT;
@ -167,7 +167,7 @@ public:
double MIN_CLEANUP_SECONDS;
// Configuration
int32_t DEFAULT_AUTO_PROXIES;
int32_t DEFAULT_AUTO_COMMIT_PROXIES;
int32_t DEFAULT_AUTO_GRV_PROXIES;
int32_t DEFAULT_AUTO_RESOLVERS;
int32_t DEFAULT_AUTO_LOGS;

View File

@ -78,8 +78,9 @@ std::map<std::string, std::string> configForToken( std::string const& mode ) {
std::string key = mode.substr(0, pos);
std::string value = mode.substr(pos+1);
if ((key == "logs" || key == "proxies" || key == "grv_proxies" || key == "resolvers" || key == "remote_logs" ||
key == "log_routers" || key == "usable_regions" || key == "repopulate_anti_quorum") &&
if ((key == "logs" || key == "commit_proxies" || key == "grv_proxies" || key == "resolvers" ||
key == "remote_logs" || key == "log_routers" || key == "usable_regions" ||
key == "repopulate_anti_quorum") &&
isInteger(value)) {
out[p+key] = value;
}
@ -656,7 +657,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
}
if (processClass.classType() == ProcessClass::TransactionClass ||
processClass.classType() == ProcessClass::ProxyClass ||
processClass.classType() == ProcessClass::CommitProxyClass ||
processClass.classType() == ProcessClass::GrvProxyClass ||
processClass.classType() == ProcessClass::ResolutionClass ||
processClass.classType() == ProcessClass::StatelessClass ||
@ -701,7 +702,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
if (proc.second == ProcessClass::StatelessClass) {
existingStatelessCount++;
}
if(proc.second == ProcessClass::ProxyClass) {
if (proc.second == ProcessClass::CommitProxyClass) {
existingProxyCount++;
}
if (proc.second == ProcessClass::GrvProxyClass) {
@ -734,19 +735,18 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
resolverCount = result.old_resolvers;
}
result.desired_proxies = std::max(std::min(12, processCount / 15), 1);
result.desired_commit_proxies = std::max(std::min(12, processCount / 15), 1);
int proxyCount;
if (!statusObjConfig.get("proxies", result.old_proxies)) {
result.old_proxies = CLIENT_KNOBS->DEFAULT_AUTO_PROXIES;
statusObjConfig.get("auto_proxies", result.old_proxies);
result.auto_proxies = result.desired_proxies;
proxyCount = result.auto_proxies;
if (!statusObjConfig.get("commit_proxies", result.old_commit_proxies)) {
result.old_commit_proxies = CLIENT_KNOBS->DEFAULT_AUTO_COMMIT_PROXIES;
statusObjConfig.get("auto_commit_proxies", result.old_commit_proxies);
result.auto_commit_proxies = result.desired_commit_proxies;
proxyCount = result.auto_commit_proxies;
} else {
result.auto_proxies = result.old_proxies;
proxyCount = result.old_proxies;
result.auto_commit_proxies = result.old_commit_proxies;
proxyCount = result.old_commit_proxies;
}
// Need to configure a good number.
result.desired_grv_proxies = std::max(std::min(4, processCount / 20), 1);
int grvProxyCount;
if (!statusObjConfig.get("grv_proxies", result.old_grv_proxies)) {
@ -857,8 +857,8 @@ ACTOR Future<ConfigurationResult::Type> autoConfig( Database cx, ConfigureAutoRe
if (conf.auto_logs != conf.old_logs)
tr.set(configKeysPrefix.toString() + "auto_logs", format("%d", conf.auto_logs));
if(conf.auto_proxies != conf.old_proxies)
tr.set(configKeysPrefix.toString() + "auto_proxies", format("%d", conf.auto_proxies));
if (conf.auto_commit_proxies != conf.old_commit_proxies)
tr.set(configKeysPrefix.toString() + "auto_commit_proxies", format("%d", conf.auto_commit_proxies));
if (conf.auto_grv_proxies != conf.old_grv_proxies)
tr.set(configKeysPrefix.toString() + "auto_grv_proxies", format("%d", conf.auto_grv_proxies));

View File

@ -86,7 +86,7 @@ struct ConfigureAutoResult {
int32_t machines;
std::string old_replication;
int32_t old_proxies;
int32_t old_commit_proxies;
int32_t old_grv_proxies;
int32_t old_resolvers;
int32_t old_logs;
@ -94,23 +94,24 @@ struct ConfigureAutoResult {
int32_t old_machines_with_transaction;
std::string auto_replication;
int32_t auto_proxies;
int32_t auto_commit_proxies;
int32_t auto_grv_proxies;
int32_t auto_resolvers;
int32_t auto_logs;
int32_t auto_processes_with_transaction;
int32_t auto_machines_with_transaction;
int32_t desired_proxies;
int32_t desired_commit_proxies;
int32_t desired_grv_proxies;
int32_t desired_resolvers;
int32_t desired_logs;
ConfigureAutoResult()
: processes(-1), machines(-1), old_proxies(-1), old_grv_proxies(-1), old_resolvers(-1), old_logs(-1),
old_processes_with_transaction(-1), old_machines_with_transaction(-1), auto_proxies(-1), auto_grv_proxies(-1),
auto_resolvers(-1), auto_logs(-1), auto_processes_with_transaction(-1), auto_machines_with_transaction(-1),
desired_proxies(-1), desired_grv_proxies(-1), desired_resolvers(-1), desired_logs(-1) {}
: processes(-1), machines(-1), old_commit_proxies(-1), old_grv_proxies(-1), old_resolvers(-1), old_logs(-1),
old_processes_with_transaction(-1), old_machines_with_transaction(-1), auto_commit_proxies(-1),
auto_grv_proxies(-1), auto_resolvers(-1), auto_logs(-1), auto_processes_with_transaction(-1),
auto_machines_with_transaction(-1), desired_commit_proxies(-1), desired_grv_proxies(-1), desired_resolvers(-1),
desired_logs(-1) {}
bool isValid() const { return processes != -1; }
};

View File

@ -624,8 +624,8 @@ ACTOR Future<Void> getClientInfoFromLeader( Reference<AsyncVar<Optional<ClusterC
choose {
when( ClientDBInfo ni = wait( brokenPromiseToNever( knownLeader->get().get().clientInterface.openDatabase.getReply( req ) ) ) ) {
TraceEvent("MonitorLeaderForProxiesGotClientInfo", knownLeader->get().get().clientInterface.id())
.detail("MasterProxy0", ni.masterProxies.size() ? ni.masterProxies[0].id() : UID())
.detail("GrvProxy0", ni.grvProxies.size() ? ni.grvProxies[0].id() : UID())
.detail("CommitProxy0", ni.commitProxies.size() ? ni.commitProxies[0].id() : UID())
.detail("GrvProxy0", ni.grvProxies.size() ? ni.grvProxies[0].id() : UID())
.detail("ClientID", ni.id);
clientData->clientInfo->set(CachedSerialization<ClientDBInfo>(ni));
}
@ -681,24 +681,25 @@ ACTOR Future<Void> monitorLeaderForProxies( Key clusterKey, vector<NetworkAddres
}
}
void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastMasterProxyUIDs, std::vector<MasterProxyInterface>& lastMasterProxies,
std::vector<UID>& lastGrvProxyUIDs, std::vector<GrvProxyInterface>& lastGrvProxies) {
if(ni.masterProxies.size() > CLIENT_KNOBS->MAX_MASTER_PROXY_CONNECTIONS) {
std::vector<UID> masterProxyUIDs;
for(auto& masterProxy : ni.masterProxies) {
masterProxyUIDs.push_back(masterProxy.id());
void shrinkProxyList(ClientDBInfo& ni, std::vector<UID>& lastCommitProxyUIDs,
std::vector<CommitProxyInterface>& lastCommitProxies, std::vector<UID>& lastGrvProxyUIDs,
std::vector<GrvProxyInterface>& lastGrvProxies) {
if (ni.commitProxies.size() > CLIENT_KNOBS->MAX_COMMIT_PROXY_CONNECTIONS) {
std::vector<UID> commitProxyUIDs;
for (auto& commitProxy : ni.commitProxies) {
commitProxyUIDs.push_back(commitProxy.id());
}
if(masterProxyUIDs != lastMasterProxyUIDs) {
lastMasterProxyUIDs.swap(masterProxyUIDs);
lastMasterProxies = ni.masterProxies;
deterministicRandom()->randomShuffle(lastMasterProxies);
lastMasterProxies.resize(CLIENT_KNOBS->MAX_MASTER_PROXY_CONNECTIONS);
for(int i = 0; i < lastMasterProxies.size(); i++) {
TraceEvent("ConnectedMasterProxy").detail("MasterProxy", lastMasterProxies[i].id());
if (commitProxyUIDs != lastCommitProxyUIDs) {
lastCommitProxyUIDs.swap(commitProxyUIDs);
lastCommitProxies = ni.commitProxies;
deterministicRandom()->randomShuffle(lastCommitProxies);
lastCommitProxies.resize(CLIENT_KNOBS->MAX_COMMIT_PROXY_CONNECTIONS);
for (int i = 0; i < lastCommitProxies.size(); i++) {
TraceEvent("ConnectedCommitProxy").detail("CommitProxy", lastCommitProxies[i].id());
}
}
ni.firstProxy = ni.masterProxies[0];
ni.masterProxies = lastMasterProxies;
ni.firstCommitProxy = ni.commitProxies[0];
ni.commitProxies = lastCommitProxies;
}
if(ni.grvProxies.size() > CLIENT_KNOBS->MAX_GRV_PROXY_CONNECTIONS) {
std::vector<UID> grvProxyUIDs;
@ -719,14 +720,16 @@ void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastMasterProxyUIDs, s
}
// Leader is the process that will be elected by coordinators as the cluster controller
ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration( Reference<ClusterConnectionFile> connFile, Reference<AsyncVar<ClientDBInfo>> clientInfo, MonitorLeaderInfo info, Reference<ReferencedObject<Standalone<VectorRef<ClientVersionRef>>>> supportedVersions, Key traceLogGroup) {
ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration(
Reference<ClusterConnectionFile> connFile, Reference<AsyncVar<ClientDBInfo>> clientInfo, MonitorLeaderInfo info,
Reference<ReferencedObject<Standalone<VectorRef<ClientVersionRef>>>> supportedVersions, Key traceLogGroup) {
state ClusterConnectionString cs = info.intermediateConnFile->getConnectionString();
state vector<NetworkAddress> addrs = cs.coordinators();
state int idx = 0;
state int successIdx = 0;
state Optional<double> incorrectTime;
state std::vector<UID> lastProxyUIDs;
state std::vector<MasterProxyInterface> lastProxies;
state std::vector<UID> lastCommitProxyUIDs;
state std::vector<CommitProxyInterface> lastCommitProxies;
state std::vector<UID> lastGrvProxyUIDs;
state std::vector<GrvProxyInterface> lastGrvProxies;
@ -780,7 +783,7 @@ ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration( Reference<ClusterCo
connFile->notifyConnected();
auto& ni = rep.get().mutate();
shrinkProxyList(ni, lastProxyUIDs, lastProxies, lastGrvProxyUIDs, lastGrvProxies);
shrinkProxyList(ni, lastCommitProxyUIDs, lastCommitProxies, lastGrvProxyUIDs, lastGrvProxies);
clientInfo->set( ni );
successIdx = idx;
} else {

View File

@ -25,7 +25,7 @@
#include "fdbclient/FDBTypes.h"
#include "fdbclient/CoordinationInterface.h"
#include "fdbclient/ClusterInterface.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#define CLUSTER_FILE_ENV_VAR_NAME "FDB_CLUSTER_FILE"
@ -67,8 +67,9 @@ Future<Void> monitorLeaderForProxies( Value const& key, vector<NetworkAddress> c
Future<Void> monitorProxies( Reference<AsyncVar<Reference<ClusterConnectionFile>>> const& connFile, Reference<AsyncVar<ClientDBInfo>> const& clientInfo, Reference<ReferencedObject<Standalone<VectorRef<ClientVersionRef>>>> const& supportedVersions, Key const& traceLogGroup );
void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastMasterProxyUIDs, std::vector<MasterProxyInterface>& lastMasterProxies,
std::vector<UID>& lastGrvProxyUIDs, std::vector<GrvProxyInterface>& lastGrvProxies);
void shrinkProxyList(ClientDBInfo& ni, std::vector<UID>& lastCommitProxyUIDs,
std::vector<CommitProxyInterface>& lastCommitProxies, std::vector<UID>& lastGrvProxyUIDs,
std::vector<GrvProxyInterface>& lastGrvProxies);
#ifndef __INTEL_COMPILER
#pragma region Implementation

View File

@ -151,7 +151,8 @@ public:
}
}
//FIXME: this is re-implemented on the master proxy to include a yield, any changes to this function should also done there
// FIXME: this is re-implemented on the commit proxy to include a yield, any changes to this function should also
// done there
template <class Ar>
void serialize_save( Ar& ar ) const {
serializer(ar, totalBytes);

View File

@ -40,7 +40,7 @@
#include "fdbclient/KeyRangeMap.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/MonitorLeader.h"
#include "fdbclient/MutationList.h"
#include "fdbclient/ReadYourWrites.h"
@ -484,15 +484,15 @@ ACTOR static Future<Void> clientStatusUpdateActor(DatabaseContext *cx) {
}
ACTOR static Future<Void> monitorProxiesChange(Reference<AsyncVar<ClientDBInfo>> clientDBInfo, AsyncTrigger *triggerVar) {
state vector< MasterProxyInterface > curProxies;
state vector<CommitProxyInterface> curCommitProxies;
state vector< GrvProxyInterface > curGrvProxies;
curProxies = clientDBInfo->get().masterProxies;
curCommitProxies = clientDBInfo->get().commitProxies;
curGrvProxies = clientDBInfo->get().grvProxies;
loop{
wait(clientDBInfo->onChange());
if (clientDBInfo->get().masterProxies != curProxies || clientDBInfo->get().grvProxies != curGrvProxies) {
curProxies = clientDBInfo->get().masterProxies;
if (clientDBInfo->get().commitProxies != curCommitProxies || clientDBInfo->get().grvProxies != curGrvProxies) {
curCommitProxies = clientDBInfo->get().commitProxies;
curGrvProxies = clientDBInfo->get().grvProxies;
triggerVar->trigger();
}
@ -881,7 +881,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
specialKeySpace(std::make_unique<SpecialKeySpace>(specialKeys.begin, specialKeys.end, /* test */ false)) {
dbId = deterministicRandom()->randomUniqueID();
connected = (clientInfo->get().masterProxies.size() && clientInfo->get().grvProxies.size())
connected = (clientInfo->get().commitProxies.size() && clientInfo->get().grvProxies.size())
? Void()
: clientInfo->onChange();
@ -1164,9 +1164,9 @@ void DatabaseContext::setOption( FDBDatabaseOptions::Option option, Optional<Str
break;
case FDBDatabaseOptions::MACHINE_ID:
clientLocality = LocalityData( clientLocality.processId(), value.present() ? Standalone<StringRef>(value.get()) : Optional<Standalone<StringRef>>(), clientLocality.machineId(), clientLocality.dcId() );
if( clientInfo->get().masterProxies.size() )
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().masterProxies) );
if( clientInfo->get().grvProxies.size() )
if (clientInfo->get().commitProxies.size())
commitProxies = Reference<CommitProxyInfo>(new CommitProxyInfo(clientInfo->get().commitProxies));
if( clientInfo->get().grvProxies.size() )
grvProxies = Reference<GrvProxyInfo>( new GrvProxyInfo( clientInfo->get().grvProxies ) );
server_interf.clear();
locationCache.insert( allKeys, Reference<LocationInfo>() );
@ -1176,9 +1176,9 @@ void DatabaseContext::setOption( FDBDatabaseOptions::Option option, Optional<Str
break;
case FDBDatabaseOptions::DATACENTER_ID:
clientLocality = LocalityData(clientLocality.processId(), clientLocality.zoneId(), clientLocality.machineId(), value.present() ? Standalone<StringRef>(value.get()) : Optional<Standalone<StringRef>>());
if( clientInfo->get().masterProxies.size() )
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().masterProxies));
if( clientInfo->get().grvProxies.size() )
if (clientInfo->get().commitProxies.size())
commitProxies = Reference<CommitProxyInfo>(new CommitProxyInfo(clientInfo->get().commitProxies));
if( clientInfo->get().grvProxies.size() )
grvProxies = Reference<GrvProxyInfo>( new GrvProxyInfo( clientInfo->get().grvProxies ));
server_interf.clear();
locationCache.insert( allKeys, Reference<LocationInfo>() );
@ -1220,13 +1220,13 @@ ACTOR static Future<Void> switchConnectionFileImpl(Reference<ClusterConnectionFi
.detail("ConnectionString", connFile->getConnectionString().toString());
// Reset state from former cluster.
self->masterProxies.clear();
self->commitProxies.clear();
self->grvProxies.clear();
self->minAcceptableReadVersion = std::numeric_limits<Version>::max();
self->invalidateCache(allKeys);
auto clearedClientInfo = self->clientInfo->get();
clearedClientInfo.masterProxies.clear();
clearedClientInfo.commitProxies.clear();
clearedClientInfo.grvProxies.clear();
clearedClientInfo.id = deterministicRandom()->randomUniqueID();
self->clientInfo->set(clearedClientInfo);
@ -1561,29 +1561,29 @@ void stopNetwork() {
void DatabaseContext::updateProxies() {
if (proxiesLastChange == clientInfo->get().id) return;
proxiesLastChange = clientInfo->get().id;
masterProxies.clear();
commitProxies.clear();
grvProxies.clear();
bool masterProxyProvisional = false, grvProxyProvisional = false;
if (clientInfo->get().masterProxies.size()) {
masterProxies = Reference<ProxyInfo>(new ProxyInfo(clientInfo->get().masterProxies));
masterProxyProvisional = clientInfo->get().masterProxies[0].provisional;
bool commitProxyProvisional = false, grvProxyProvisional = false;
if (clientInfo->get().commitProxies.size()) {
commitProxies = Reference<CommitProxyInfo>(new CommitProxyInfo(clientInfo->get().commitProxies));
commitProxyProvisional = clientInfo->get().commitProxies[0].provisional;
}
if (clientInfo->get().grvProxies.size()) {
grvProxies = Reference<GrvProxyInfo>(new GrvProxyInfo(clientInfo->get().grvProxies));
grvProxyProvisional = clientInfo->get().grvProxies[0].provisional;
}
if (clientInfo->get().masterProxies.size() && clientInfo->get().grvProxies.size()) {
ASSERT(masterProxyProvisional == grvProxyProvisional);
proxyProvisional = masterProxyProvisional;
if (clientInfo->get().commitProxies.size() && clientInfo->get().grvProxies.size()) {
ASSERT(commitProxyProvisional == grvProxyProvisional);
proxyProvisional = commitProxyProvisional;
}
}
Reference<ProxyInfo> DatabaseContext::getMasterProxies(bool useProvisionalProxies) {
Reference<CommitProxyInfo> DatabaseContext::getCommitProxies(bool useProvisionalProxies) {
updateProxies();
if (proxyProvisional && !useProvisionalProxies) {
return Reference<ProxyInfo>();
return Reference<CommitProxyInfo>();
}
return masterProxies;
return commitProxies;
}
Reference<GrvProxyInfo> DatabaseContext::getGrvProxies(bool useProvisionalProxies) {
@ -1594,19 +1594,19 @@ Reference<GrvProxyInfo> DatabaseContext::getGrvProxies(bool useProvisionalProxie
return grvProxies;
}
//Actor which will wait until the MultiInterface<MasterProxyInterface> returned by the DatabaseContext cx is not NULL
ACTOR Future<Reference<ProxyInfo>> getMasterProxiesFuture(DatabaseContext *cx, bool useProvisionalProxies) {
// Actor which will wait until the MultiInterface<CommitProxyInterface> returned by the DatabaseContext cx is not NULL
ACTOR Future<Reference<CommitProxyInfo>> getCommitProxiesFuture(DatabaseContext* cx, bool useProvisionalProxies) {
loop{
Reference<ProxyInfo> proxies = cx->getMasterProxies(useProvisionalProxies);
if (proxies)
return proxies;
Reference<CommitProxyInfo> commitProxies = cx->getCommitProxies(useProvisionalProxies);
if (commitProxies)
return commitProxies;
wait( cx->onProxiesChanged() );
}
}
//Returns a future which will not be set until the ProxyInfo of this DatabaseContext is not NULL
Future<Reference<ProxyInfo>> DatabaseContext::getMasterProxiesFuture(bool useProvisionalProxies) {
return ::getMasterProxiesFuture(this, useProvisionalProxies);
// Returns a future which will not be set until the CommitProxyInfo of this DatabaseContext is not NULL
Future<Reference<CommitProxyInfo>> DatabaseContext::getCommitProxiesFuture(bool useProvisionalProxies) {
return ::getCommitProxiesFuture(this, useProvisionalProxies);
}
void GetRangeLimits::decrement( VectorRef<KeyValueRef> const& data ) {
@ -1733,8 +1733,8 @@ ACTOR Future<pair<KeyRange, Reference<LocationInfo>>> getKeyLocation_internal(Da
++cx->transactionKeyServerLocationRequests;
choose {
when (wait(cx->onProxiesChanged())) {}
when (GetKeyServerLocationsReply rep = wait(basicLoadBalance(
cx->getMasterProxies(info.useProvisionalProxies), &MasterProxyInterface::getKeyServersLocations,
when(GetKeyServerLocationsReply rep = wait(basicLoadBalance(
cx->getCommitProxies(info.useProvisionalProxies), &CommitProxyInterface::getKeyServersLocations,
GetKeyServerLocationsRequest(span.context, key, Optional<KeyRef>(), 100, isBackward, key.arena()),
TaskPriority::DefaultPromiseEndpoint))) {
++cx->transactionKeyServerLocationRequestsCompleted;
@ -1782,8 +1782,8 @@ ACTOR Future<vector<pair<KeyRange, Reference<LocationInfo>>>> getKeyRangeLocatio
++cx->transactionKeyServerLocationRequests;
choose {
when ( wait( cx->onProxiesChanged() ) ) {}
when ( GetKeyServerLocationsReply _rep = wait(basicLoadBalance(
cx->getMasterProxies(info.useProvisionalProxies), &MasterProxyInterface::getKeyServersLocations,
when(GetKeyServerLocationsReply _rep = wait(basicLoadBalance(
cx->getCommitProxies(info.useProvisionalProxies), &CommitProxyInterface::getKeyServersLocations,
GetKeyServerLocationsRequest(span.context, keys.begin, keys.end, limit, reverse, keys.arena()),
TaskPriority::DefaultPromiseEndpoint))) {
++cx->transactionKeyServerLocationRequestsCompleted;
@ -3450,14 +3450,16 @@ ACTOR static Future<Void> tryCommit( Database cx, Reference<TransactionLogInfo>
req.debugID = commitID;
state Future<CommitID> reply;
if (options.commitOnFirstProxy) {
if(cx->clientInfo->get().firstProxy.present()) {
reply = throwErrorOr ( brokenPromiseToMaybeDelivered ( cx->clientInfo->get().firstProxy.get().commit.tryGetReply(req) ) );
if (cx->clientInfo->get().firstCommitProxy.present()) {
reply = throwErrorOr(brokenPromiseToMaybeDelivered(
cx->clientInfo->get().firstCommitProxy.get().commit.tryGetReply(req)));
} else {
const std::vector<MasterProxyInterface>& proxies = cx->clientInfo->get().masterProxies;
const std::vector<CommitProxyInterface>& proxies = cx->clientInfo->get().commitProxies;
reply = proxies.size() ? throwErrorOr ( brokenPromiseToMaybeDelivered ( proxies[0].commit.tryGetReply(req) ) ) : Never();
}
} else {
reply = basicLoadBalance( cx->getMasterProxies(info.useProvisionalProxies), &MasterProxyInterface::commit, req, TaskPriority::DefaultPromiseEndpoint, true );
reply = basicLoadBalance(cx->getCommitProxies(info.useProvisionalProxies), &CommitProxyInterface::commit,
req, TaskPriority::DefaultPromiseEndpoint, true);
}
choose {
@ -3531,8 +3533,9 @@ ACTOR static Future<Void> tryCommit( Database cx, Reference<TransactionLogInfo>
// We don't know if the commit happened, and it might even still be in flight.
if (!options.causalWriteRisky) {
// Make sure it's not still in flight, either by ensuring the master we submitted to is dead, or the version we submitted with is dead, or by committing a conflicting transaction successfully
//if ( cx->getMasterProxies()->masterGeneration <= originalMasterGeneration )
// Make sure it's not still in flight, either by ensuring the master we submitted to is dead, or the
// version we submitted with is dead, or by committing a conflicting transaction successfully
// if ( cx->getCommitProxies()->masterGeneration <= originalMasterGeneration )
// To ensure the original request is not in flight, we need a key range which intersects its read conflict ranges
// We pick a key range which also intersects its write conflict ranges, since that avoids potentially creating conflicts where there otherwise would be none
@ -4433,8 +4436,8 @@ ACTOR Future<Standalone<VectorRef<DDMetricsRef>>> waitDataDistributionMetricsLis
choose {
when(wait(cx->onProxiesChanged())) {}
when(ErrorOr<GetDDMetricsReply> rep =
wait(errorOr(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::getDDMetrics,
GetDDMetricsRequest(keys, shardLimit))))) {
wait(errorOr(basicLoadBalance(cx->getCommitProxies(false), &CommitProxyInterface::getDDMetrics,
GetDDMetricsRequest(keys, shardLimit))))) {
if (rep.isError()) {
throw rep.getError();
}
@ -4539,7 +4542,9 @@ ACTOR Future<Void> snapCreate(Database cx, Standalone<StringRef> snapCmd, UID sn
loop {
choose {
when(wait(cx->onProxiesChanged())) {}
when(wait(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::proxySnapReq, ProxySnapRequest(snapCmd, snapUID, snapUID), cx->taskID, true /*atmostOnce*/ ))) {
when(wait(basicLoadBalance(cx->getCommitProxies(false), &CommitProxyInterface::proxySnapReq,
ProxySnapRequest(snapCmd, snapUID, snapUID), cx->taskID,
true /*atmostOnce*/))) {
TraceEvent("SnapCreateExit")
.detail("SnapCmd", snapCmd.toString())
.detail("UID", snapUID);
@ -4567,8 +4572,8 @@ ACTOR Future<bool> checkSafeExclusions(Database cx, vector<AddressExclusion> exc
choose {
when(wait(cx->onProxiesChanged())) {}
when(ExclusionSafetyCheckReply _ddCheck =
wait(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::exclusionSafetyCheckReq,
req, cx->taskID))) {
wait(basicLoadBalance(cx->getCommitProxies(false),
&CommitProxyInterface::exclusionSafetyCheckReq, req, cx->taskID))) {
ddCheck = _ddCheck.safe;
break;
}

View File

@ -30,7 +30,7 @@
#include "flow/flow.h"
#include "flow/TDMetric.actor.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/CoordinationInterface.h"
#include "fdbclient/ClusterInterface.h"

View File

@ -47,7 +47,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
"storage",
"transaction",
"resolution",
"proxy",
"commit_proxy",
"grv_proxy",
"master",
"test",
@ -84,7 +84,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
"role":{
"$enum":[
"master",
"proxy",
"commit_proxy",
"grv_proxy",
"log",
"storage",
@ -486,7 +486,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
R"statusSchema(
"recovery_state":{
"required_resolvers":1,
"required_proxies":1,
"required_commit_proxies":1,
"required_grv_proxies":1,
"name":{
"$enum":[
@ -675,11 +675,11 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
"address":"10.0.4.1"
}
],
"auto_proxies":3,
"auto_commit_proxies":3,
"auto_grv_proxies":1,
"auto_resolvers":1,
"auto_logs":3,
"proxies":5,
"commit_proxies":5,
"grv_proxies":1,
"backup_worker_enabled":1
},
@ -879,11 +879,11 @@ const KeyRef JSONSchemas::clusterConfigurationSchema = LiteralStringRef(R"config
"ssd-2",
"memory"
]},
"auto_proxies":3,
"auto_commit_proxies":3,
"auto_grv_proxies":1,
"auto_resolvers":1,
"auto_logs":3,
"proxies":5
"commit_proxies":5
"grv_proxies":1
})configSchema");

View File

@ -19,7 +19,7 @@
*/
#include "fdbclient/TagThrottle.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/DatabaseContext.h"
#include "flow/actorcompiler.h" // has to be last include

View File

@ -195,7 +195,7 @@ description is not currently required but encouraged.
<Option name="next_write_no_write_conflict_range" code="30"
description="The next write performed on this transaction will not generate a write conflict range. As a result, other transactions which read the key(s) being modified by the next write will not conflict with this transaction. Care needs to be taken when using this option on a transaction that is shared between multiple threads. When setting this option, write conflict ranges will be disabled on the next write operation, regardless of what thread it is on." />
<Option name="commit_on_first_proxy" code="40"
description="Committing this transaction will bypass the normal load balancing across proxies and go directly to the specifically nominated 'first proxy'."
description="Committing this transaction will bypass the normal load balancing across commit proxies and go directly to the specifically nominated 'first commit proxy'."
hidden="true" />
<Option name="check_writes_enable" code="50"
hidden="true" />

View File

@ -63,9 +63,9 @@ ProcessClass::Fitness ProcessClass::machineClassFitness( ClusterRole role ) cons
default:
return ProcessClass::NeverAssign;
}
case ProcessClass::Proxy:
case ProcessClass::CommitProxy:
switch( _class ) {
case ProcessClass::ProxyClass:
case ProcessClass::CommitProxyClass:
return ProcessClass::BestFit;
case ProcessClass::StatelessClass:
return ProcessClass::GoodFit;
@ -92,7 +92,7 @@ ProcessClass::Fitness ProcessClass::machineClassFitness( ClusterRole role ) cons
return ProcessClass::GoodFit;
case ProcessClass::UnsetClass:
return ProcessClass::UnsetFit;
case ProcessClass::ProxyClass:
case ProcessClass::CommitProxyClass:
return ProcessClass::OkayFit;
case ProcessClass::ResolutionClass:
return ProcessClass::OkayFit;
@ -192,7 +192,7 @@ ProcessClass::Fitness ProcessClass::machineClassFitness( ClusterRole role ) cons
return ProcessClass::OkayFit;
case ProcessClass::TransactionClass:
return ProcessClass::OkayFit;
case ProcessClass::ProxyClass:
case ProcessClass::CommitProxyClass:
return ProcessClass::OkayFit;
case ProcessClass::GrvProxyClass:
return ProcessClass::OkayFit;

View File

@ -33,7 +33,7 @@ struct ProcessClass {
TransactionClass,
ResolutionClass,
TesterClass,
ProxyClass, // Process class of CommitProxy
CommitProxyClass,
GrvProxyClass,
MasterClass,
StatelessClass,
@ -53,7 +53,7 @@ struct ProcessClass {
enum ClusterRole {
Storage,
TLog,
Proxy,
CommitProxy,
GrvProxy,
Master,
Resolver,
@ -77,7 +77,7 @@ public:
if (s=="storage") _class = StorageClass;
else if (s=="transaction") _class = TransactionClass;
else if (s=="resolution") _class = ResolutionClass;
else if (s=="proxy") _class = ProxyClass;
else if (s=="commit_proxy") _class = CommitProxyClass;
else if (s=="grv_proxy") _class = GrvProxyClass;
else if (s=="master") _class = MasterClass;
else if (s=="test") _class = TesterClass;
@ -99,7 +99,7 @@ public:
if (classStr=="storage") _class = StorageClass;
else if (classStr=="transaction") _class = TransactionClass;
else if (classStr=="resolution") _class = ResolutionClass;
else if (classStr=="proxy") _class = ProxyClass;
else if (classStr=="commit_proxy") _class = CommitProxyClass;
else if (classStr=="grv_proxy") _class = GrvProxyClass;
else if (classStr=="master") _class = MasterClass;
else if (classStr=="test") _class = TesterClass;
@ -137,7 +137,7 @@ public:
case StorageClass: return "storage";
case TransactionClass: return "transaction";
case ResolutionClass: return "resolution";
case ProxyClass: return "proxy";
case CommitProxyClass: return "commit_proxy";
case GrvProxyClass: return "grv_proxy";
case MasterClass: return "master";
case TesterClass: return "test";

View File

@ -97,7 +97,8 @@ public:
case ProcessClass::StorageClass: return true;
case ProcessClass::TransactionClass: return true;
case ProcessClass::ResolutionClass: return false;
case ProcessClass::ProxyClass: return false;
case ProcessClass::CommitProxyClass:
return false;
case ProcessClass::GrvProxyClass:
return false;
case ProcessClass::MasterClass:

View File

@ -43,8 +43,8 @@ Reference<StorageInfo> getStorageInfo(UID id, std::map<UID, Reference<StorageInf
}
// It is incredibly important that any modifications to txnStateStore are done in such a way that
// the same operations will be done on all proxies at the same time. Otherwise, the data stored in
// txnStateStore will become corrupted.
// the same operations will be done on all commit proxies at the same time. Otherwise, the data
// stored in txnStateStore will become corrupted.
void applyMetadataMutations(UID const& dbgid, Arena& arena, VectorRef<MutationRef> const& mutations,
IKeyValueStore* txnStateStore, LogPushData* toCommit, bool& confChange,
Reference<ILogSystem> logSystem, Version popVersion,

View File

@ -21,7 +21,7 @@
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BackupContainer.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/SystemData.h"
#include "fdbserver/BackupInterface.h"
#include "fdbserver/BackupProgress.actor.h"

View File

@ -46,7 +46,7 @@ set(FDBSERVER_SRCS
LogSystemDiskQueueAdapter.h
LogSystemPeekCursor.actor.cpp
MasterInterface.h
MasterProxyServer.actor.cpp
CommitProxyServer.actor.cpp
masterserver.actor.cpp
MutationTracking.h
MutationTracking.cpp

View File

@ -753,20 +753,21 @@ public:
}
}
auto first_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit,
req.configuration, id_used);
auto first_commit_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::CommitProxy,
ProcessClass::ExcludeFit, req.configuration, id_used);
auto first_grv_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::GrvProxy, ProcessClass::ExcludeFit,
req.configuration, id_used);
auto first_resolver = getWorkerForRoleInDatacenter(dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit,
req.configuration, id_used);
auto proxies = getWorkersForRoleInDatacenter(dcId, ProcessClass::Proxy, req.configuration.getDesiredProxies(),
req.configuration, id_used, first_proxy);
auto commit_proxies =
getWorkersForRoleInDatacenter(dcId, ProcessClass::CommitProxy, req.configuration.getDesiredCommitProxies(),
req.configuration, id_used, first_commit_proxy);
auto grv_proxies =
getWorkersForRoleInDatacenter(dcId, ProcessClass::GrvProxy, req.configuration.getDesiredGrvProxies(),
req.configuration, id_used, first_grv_proxy);
auto resolvers = getWorkersForRoleInDatacenter( dcId, ProcessClass::Resolver, req.configuration.getDesiredResolvers(), req.configuration, id_used, first_resolver );
for (int i = 0; i < proxies.size(); i++) result.masterProxies.push_back(proxies[i].interf);
for (int i = 0; i < commit_proxies.size(); i++) result.commitProxies.push_back(commit_proxies[i].interf);
for (int i = 0; i < grv_proxies.size(); i++) result.grvProxies.push_back(grv_proxies[i].interf);
for(int i = 0; i < resolvers.size(); i++)
result.resolvers.push_back(resolvers[i].interf);
@ -800,9 +801,9 @@ public:
RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredSatelliteLogs(dcId),
ProcessClass::TLog)
.betterCount(RoleFitness(satelliteLogs, ProcessClass::TLog))) ||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(),
ProcessClass::Proxy)
.betterCount(RoleFitness(proxies, ProcessClass::Proxy)) ||
RoleFitness(SERVER_KNOBS->EXPECTED_COMMIT_PROXY_FITNESS, req.configuration.getDesiredCommitProxies(),
ProcessClass::CommitProxy)
.betterCount(RoleFitness(commit_proxies, ProcessClass::CommitProxy)) ||
RoleFitness(SERVER_KNOBS->EXPECTED_GRV_PROXY_FITNESS, req.configuration.getDesiredGrvProxies(),
ProcessClass::GrvProxy)
.betterCount(RoleFitness(grv_proxies, ProcessClass::GrvProxy)) ||
@ -911,22 +912,22 @@ public:
try {
//SOMEDAY: recruitment in other DCs besides the clusterControllerDcID will not account for the processes used by the master and cluster controller properly.
auto used = id_used;
auto first_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit,
req.configuration, used);
auto first_commit_proxy = getWorkerForRoleInDatacenter(
dcId, ProcessClass::CommitProxy, ProcessClass::ExcludeFit, req.configuration, used);
auto first_grv_proxy = getWorkerForRoleInDatacenter(
dcId, ProcessClass::GrvProxy, ProcessClass::ExcludeFit, req.configuration, used);
auto first_resolver = getWorkerForRoleInDatacenter(
dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, req.configuration, used);
auto proxies =
getWorkersForRoleInDatacenter(dcId, ProcessClass::Proxy, req.configuration.getDesiredProxies(),
req.configuration, used, first_proxy);
auto commit_proxies = getWorkersForRoleInDatacenter(dcId, ProcessClass::CommitProxy,
req.configuration.getDesiredCommitProxies(),
req.configuration, used, first_commit_proxy);
auto grv_proxies = getWorkersForRoleInDatacenter(dcId, ProcessClass::GrvProxy,
req.configuration.getDesiredGrvProxies(),
req.configuration, used, first_grv_proxy);
auto resolvers = getWorkersForRoleInDatacenter( dcId, ProcessClass::Resolver, req.configuration.getDesiredResolvers(), req.configuration, used, first_resolver );
RoleFitnessPair fitness(RoleFitness(proxies, ProcessClass::Proxy),
RoleFitnessPair fitness(RoleFitness(commit_proxies, ProcessClass::CommitProxy),
RoleFitness(grv_proxies, ProcessClass::GrvProxy),
RoleFitness(resolvers, ProcessClass::Resolver));
@ -936,8 +937,8 @@ public:
for (int i = 0; i < resolvers.size(); i++) {
result.resolvers.push_back(resolvers[i].interf);
}
for (int i = 0; i < proxies.size(); i++) {
result.masterProxies.push_back(proxies[i].interf);
for (int i = 0; i < commit_proxies.size(); i++) {
result.commitProxies.push_back(commit_proxies[i].interf);
}
for (int i = 0; i < grv_proxies.size(); i++) {
result.grvProxies.push_back(grv_proxies[i].interf);
@ -982,8 +983,8 @@ public:
.detail("Replication", req.configuration.tLogReplicationFactor)
.detail("DesiredLogs", req.configuration.getDesiredLogs())
.detail("ActualLogs", result.tLogs.size())
.detail("DesiredProxies", req.configuration.getDesiredProxies())
.detail("ActualProxies", result.masterProxies.size())
.detail("DesiredCommitProxies", req.configuration.getDesiredCommitProxies())
.detail("ActualCommitProxies", result.commitProxies.size())
.detail("DesiredGrvProxies", req.configuration.getDesiredGrvProxies())
.detail("ActualGrvProxies", result.grvProxies.size())
.detail("DesiredResolvers", req.configuration.getDesiredResolvers())
@ -993,8 +994,8 @@ public:
(RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(),
ProcessClass::TLog)
.betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(),
ProcessClass::Proxy)
RoleFitness(SERVER_KNOBS->EXPECTED_COMMIT_PROXY_FITNESS, req.configuration.getDesiredCommitProxies(),
ProcessClass::CommitProxy)
.betterCount(bestFitness.proxy) ||
RoleFitness(SERVER_KNOBS->EXPECTED_GRV_PROXY_FITNESS, req.configuration.getDesiredGrvProxies(),
ProcessClass::GrvProxy)
@ -1028,7 +1029,8 @@ public:
}
getWorkerForRoleInDatacenter( regions[0].dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, db.config, id_used, true );
getWorkerForRoleInDatacenter( regions[0].dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit, db.config, id_used, true );
getWorkerForRoleInDatacenter(regions[0].dcId, ProcessClass::CommitProxy, ProcessClass::ExcludeFit,
db.config, id_used, true);
getWorkerForRoleInDatacenter(regions[0].dcId, ProcessClass::GrvProxy, ProcessClass::ExcludeFit, db.config,
id_used, true);
@ -1129,15 +1131,13 @@ public:
}
}
// Get proxy classes
std::vector<WorkerDetails> proxyClasses;
for(auto& it : dbi.client.masterProxies) {
auto masterProxyWorker = id_worker.find(it.processId);
if ( masterProxyWorker == id_worker.end() )
return false;
if ( masterProxyWorker->second.priorityInfo.isExcluded )
return true;
proxyClasses.push_back(masterProxyWorker->second.details);
// Get commit proxy classes
std::vector<WorkerDetails> commitProxyClasses;
for (auto& it : dbi.client.commitProxies) {
auto commitProxyWorker = id_worker.find(it.processId);
if (commitProxyWorker == id_worker.end()) return false;
if (commitProxyWorker->second.priorityInfo.isExcluded) return true;
commitProxyClasses.push_back(commitProxyWorker->second.details);
}
// Get grv proxy classes
@ -1285,25 +1285,25 @@ public:
if(oldLogRoutersFit < newLogRoutersFit) return false;
// Check proxy/grvProxy/resolver fitness
RoleFitnessPair oldInFit(RoleFitness(proxyClasses, ProcessClass::Proxy),
RoleFitnessPair oldInFit(RoleFitness(commitProxyClasses, ProcessClass::CommitProxy),
RoleFitness(grvProxyClasses, ProcessClass::GrvProxy),
RoleFitness(resolverClasses, ProcessClass::Resolver));
auto first_proxy = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::Proxy,
ProcessClass::ExcludeFit, db.config, id_used, true);
auto first_commit_proxy = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::CommitProxy,
ProcessClass::ExcludeFit, db.config, id_used, true);
auto first_grv_proxy = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::GrvProxy,
ProcessClass::ExcludeFit, db.config, id_used, true);
auto first_resolver = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::Resolver,
ProcessClass::ExcludeFit, db.config, id_used, true);
auto proxies =
getWorkersForRoleInDatacenter(clusterControllerDcId, ProcessClass::Proxy, db.config.getDesiredProxies(),
db.config, id_used, first_proxy, true);
auto commit_proxies = getWorkersForRoleInDatacenter(clusterControllerDcId, ProcessClass::CommitProxy,
db.config.getDesiredCommitProxies(), db.config, id_used,
first_commit_proxy, true);
auto grv_proxies =
getWorkersForRoleInDatacenter(clusterControllerDcId, ProcessClass::GrvProxy,
db.config.getDesiredGrvProxies(), db.config, id_used, first_grv_proxy, true);
auto resolvers = getWorkersForRoleInDatacenter( clusterControllerDcId, ProcessClass::Resolver, db.config.getDesiredResolvers(), db.config, id_used, first_resolver, true );
RoleFitnessPair newInFit(RoleFitness(proxies, ProcessClass::Proxy),
RoleFitnessPair newInFit(RoleFitness(commit_proxies, ProcessClass::CommitProxy),
RoleFitness(grv_proxies, ProcessClass::GrvProxy),
RoleFitness(resolvers, ProcessClass::Resolver));
if (oldInFit.proxy.betterFitness(newInFit.proxy) || oldInFit.grvProxy.betterFitness(newInFit.grvProxy) ||
@ -1358,7 +1358,7 @@ public:
if (tlog.present() && tlog.interf().filteredLocality.processId() == processId) return true;
}
}
for (const MasterProxyInterface& interf : dbInfo.client.masterProxies) {
for (const CommitProxyInterface& interf : dbInfo.client.commitProxies) {
if (interf.processId == processId) return true;
}
for (const GrvProxyInterface& interf : dbInfo.client.grvProxies) {
@ -1393,7 +1393,7 @@ public:
}
}
}
for (const MasterProxyInterface& interf : dbInfo.client.masterProxies) {
for (const CommitProxyInterface& interf : dbInfo.client.commitProxies) {
ASSERT(interf.processId.present());
idUsed[interf.processId]++;
}
@ -1967,7 +1967,7 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
.detail("Resolvers", req.resolvers.size())
.detail("RecoveryState", (int)req.recoveryState)
.detail("RegistrationCount", req.registrationCount)
.detail("MasterProxies", req.masterProxies.size())
.detail("CommitProxies", req.commitProxies.size())
.detail("GrvProxies", req.grvProxies.size())
.detail("RecoveryCount", req.recoveryCount)
.detail("Stalled", req.recoveryStalled)
@ -2022,11 +2022,12 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
}
// Construct the client information
if (db->clientInfo->get().masterProxies != req.masterProxies || db->clientInfo->get().grvProxies != req.grvProxies) {
if (db->clientInfo->get().commitProxies != req.commitProxies ||
db->clientInfo->get().grvProxies != req.grvProxies) {
isChanged = true;
ClientDBInfo clientInfo;
clientInfo.id = deterministicRandom()->randomUniqueID();
clientInfo.masterProxies = req.masterProxies;
clientInfo.commitProxies = req.commitProxies;
clientInfo.grvProxies = req.grvProxies;
clientInfo.clientTxnInfoSampleRate = db->clientInfo->get().clientTxnInfoSampleRate;
clientInfo.clientTxnInfoSizeLimit = db->clientInfo->get().clientTxnInfoSizeLimit;

View File

@ -1,5 +1,5 @@
/*
* MasterProxyServer.actor.cpp
* CommitProxyServer.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
@ -25,7 +25,7 @@
#include "fdbclient/Atomic.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/SystemData.h"
#include "fdbrpc/sim_validation.h"
@ -42,7 +42,6 @@
#include "fdbserver/ProxyCommitData.actor.h"
#include "fdbserver/RatekeeperInterface.h"
#include "fdbserver/RecoveryState.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbserver/WaitFailure.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "flow/ActorCollection.h"
@ -229,7 +228,7 @@ ACTOR Future<Void> commitBatcher(ProxyCommitData *commitData, PromiseStream<std:
++commitData->stats.txnCommitIn;
if(req.debugID.present()) {
g_traceBatch.addEvent("CommitDebug", req.debugID.get().first(), "MasterProxyServer.batcher");
g_traceBatch.addEvent("CommitDebug", req.debugID.get().first(), "CommitProxyServer.batcher");
}
if(!batch.size()) {
@ -512,11 +511,7 @@ void CommitBatchContext::setupTraceBatch() {
}
if (debugID.present()) {
g_traceBatch.addEvent(
"CommitDebug",
debugID.get().first(),
"MasterProxyServer.commitBatch.Before"
);
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.Before");
}
}
@ -546,10 +541,8 @@ ACTOR Future<Void> preresolutionProcessing(CommitBatchContext* self) {
);
if (debugID.present()) {
g_traceBatch.addEvent(
"CommitDebug", debugID.get().first(),
"MasterProxyServer.commitBatch.GettingCommitVersion"
);
g_traceBatch.addEvent("CommitDebug", debugID.get().first(),
"CommitProxyServer.commitBatch.GettingCommitVersion");
}
GetCommitVersionRequest req(self->span.context, pProxyCommitData->commitVersionRequestNumber++,
@ -577,10 +570,7 @@ ACTOR Future<Void> preresolutionProcessing(CommitBatchContext* self) {
//TraceEvent("ProxyGotVer", pProxyContext->dbgid).detail("Commit", commitVersion).detail("Prev", prevVersion);
if (debugID.present()) {
g_traceBatch.addEvent(
"CommitDebug", debugID.get().first(),
"MasterProxyServer.commitBatch.GotCommitVersion"
);
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.GotCommitVersion");
}
return Void();
@ -639,10 +629,8 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
self->resolution.swap(*const_cast<std::vector<ResolveTransactionBatchReply>*>(&resolutionResp));
if (self->debugID.present()) {
g_traceBatch.addEvent(
"CommitDebug", self->debugID.get().first(),
"MasterProxyServer.commitBatch.AfterResolution"
);
g_traceBatch.addEvent("CommitDebug", self->debugID.get().first(),
"CommitProxyServer.commitBatch.AfterResolution");
}
return Void();
@ -972,10 +960,8 @@ ACTOR Future<Void> postResolution(CommitBatchContext* self) {
pProxyCommitData->stats.txnCommitResolved += trs.size();
if (debugID.present()) {
g_traceBatch.addEvent(
"CommitDebug", debugID.get().first(),
"MasterProxyServer.commitBatch.ProcessingMutations"
);
g_traceBatch.addEvent("CommitDebug", debugID.get().first(),
"CommitProxyServer.commitBatch.ProcessingMutations");
}
self->isMyFirstBatch = !pProxyCommitData->version;
@ -1041,7 +1027,8 @@ ACTOR Future<Void> postResolution(CommitBatchContext* self) {
self->msg = self->storeCommits.back().first.get();
if (self->debugID.present())
g_traceBatch.addEvent("CommitDebug", self->debugID.get().first(), "MasterProxyServer.commitBatch.AfterStoreCommits");
g_traceBatch.addEvent("CommitDebug", self->debugID.get().first(),
"CommitProxyServer.commitBatch.AfterStoreCommits");
// txnState (transaction subsystem state) tag: message extracted from log adapter
bool firstMessage = true;
@ -1129,7 +1116,7 @@ ACTOR Future<Void> reply(CommitBatchContext* self) {
//TraceEvent("ProxyPushed", pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion);
if (debugID.present())
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.AfterLogPush");
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.AfterLogPush");
for (auto &p : self->storeCommits) {
ASSERT(!p.second.isReady());
@ -1328,7 +1315,8 @@ ACTOR static Future<Void> doKeyServerLocationRequest( GetKeyServerLocationsReque
return Void();
}
ACTOR static Future<Void> readRequestServer( MasterProxyInterface proxy, PromiseStream<Future<Void>> addActor, ProxyCommitData* commitData ) {
ACTOR static Future<Void> readRequestServer(CommitProxyInterface proxy, PromiseStream<Future<Void>> addActor,
ProxyCommitData* commitData) {
loop {
GetKeyServerLocationsRequest req = waitNext(proxy.getKeyServersLocations.getFuture());
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
@ -1344,7 +1332,7 @@ ACTOR static Future<Void> readRequestServer( MasterProxyInterface proxy, Promise
}
}
ACTOR static Future<Void> rejoinServer( MasterProxyInterface proxy, ProxyCommitData* commitData ) {
ACTOR static Future<Void> rejoinServer(CommitProxyInterface proxy, ProxyCommitData* commitData) {
// We can't respond to these requests until we have valid txnStateStore
wait(commitData->validState.getFuture());
@ -1413,8 +1401,7 @@ ACTOR static Future<Void> rejoinServer( MasterProxyInterface proxy, ProxyCommitD
}
}
ACTOR Future<Void> ddMetricsRequestServer(MasterProxyInterface proxy, Reference<AsyncVar<ServerDBInfo>> db)
{
ACTOR Future<Void> ddMetricsRequestServer(CommitProxyInterface proxy, Reference<AsyncVar<ServerDBInfo>> db) {
loop {
choose {
when(state GetDDMetricsRequest req = waitNext(proxy.getDDMetrics.getFuture()))
@ -1496,17 +1483,17 @@ ACTOR Future<Void> monitorRemoteCommitted(ProxyCommitData* self) {
}
ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* commitData) {
TraceEvent("SnapMasterProxy_SnapReqEnter")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
TraceEvent("SnapCommitProxy_SnapReqEnter")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
try {
// whitelist check
ExecCmdValueString execArg(snapReq.snapPayload);
StringRef binPath = execArg.getBinaryPath();
if (!isWhitelisted(commitData->whitelistedBinPathVec, binPath)) {
TraceEvent("SnapMasterProxy_WhiteListCheckFailed")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
TraceEvent("SnapCommitProxy_WhiteListCheckFailed")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
throw snap_path_not_whitelisted();
}
// db fully recovered check
@ -1516,9 +1503,9 @@ ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* co
// Currently, snapshot of old tlog generation is not
// supported and hence failing the snapshot request until
// cluster is fully_recovered.
TraceEvent("SnapMasterProxy_ClusterNotFullyRecovered")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
TraceEvent("SnapCommitProxy_ClusterNotFullyRecovered")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
throw snap_not_fully_recovered_unsupported();
}
@ -1531,9 +1518,9 @@ ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* co
// FIXME: logAntiQuorum not supported, remove it later,
// In version2, we probably don't need this limtiation, but this needs to be tested.
if (logAntiQuorum > 0) {
TraceEvent("SnapMasterProxy_LogAnitQuorumNotSupported")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
TraceEvent("SnapCommitProxy_LogAnitQuorumNotSupported")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
throw snap_log_anti_quorum_unsupported();
}
@ -1547,32 +1534,32 @@ ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* co
try {
wait(throwErrorOr(ddSnapReq));
} catch (Error& e) {
TraceEvent("SnapMasterProxy_DDSnapResponseError")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID)
.error(e, true /*includeCancelled*/ );
TraceEvent("SnapCommitProxy_DDSnapResponseError")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID)
.error(e, true /*includeCancelled*/);
throw e;
}
snapReq.reply.send(Void());
} catch (Error& e) {
TraceEvent("SnapMasterProxy_SnapReqError")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID)
.error(e, true /*includeCancelled*/);
TraceEvent("SnapCommitProxy_SnapReqError")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID)
.error(e, true /*includeCancelled*/);
if (e.code() != error_code_operation_cancelled) {
snapReq.reply.sendError(e);
} else {
throw e;
}
}
TraceEvent("SnapMasterProxy_SnapReqExit")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
TraceEvent("SnapCommitProxy_SnapReqExit")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
return Void();
}
ACTOR Future<Void> proxyCheckSafeExclusion(Reference<AsyncVar<ServerDBInfo>> db, ExclusionSafetyCheckRequest req) {
TraceEvent("SafetyCheckMasterProxyBegin");
TraceEvent("SafetyCheckCommitProxyBegin");
state ExclusionSafetyCheckReply reply(false);
if (!db->get().distributor.present()) {
TraceEvent(SevWarnAlways, "DataDistributorNotPresent").detail("Operation", "ExclusionSafetyCheck");
@ -1586,7 +1573,7 @@ ACTOR Future<Void> proxyCheckSafeExclusion(Reference<AsyncVar<ServerDBInfo>> db,
DistributorExclusionSafetyCheckReply _reply = wait(throwErrorOr(safeFuture));
reply.safe = _reply.safe;
} catch (Error& e) {
TraceEvent("SafetyCheckMasterProxyResponseError").error(e);
TraceEvent("SafetyCheckCommitProxyResponseError").error(e);
if (e.code() != error_code_operation_cancelled) {
req.reply.sendError(e);
return Void();
@ -1594,7 +1581,7 @@ ACTOR Future<Void> proxyCheckSafeExclusion(Reference<AsyncVar<ServerDBInfo>> db,
throw e;
}
}
TraceEvent("SafetyCheckMasterProxyFinish");
TraceEvent("SafetyCheckCommitProxyFinish");
req.reply.send(reply);
return Void();
}
@ -1631,15 +1618,10 @@ ACTOR Future<Void> reportTxnTagCommitCost(UID myID, Reference<AsyncVar<ServerDBI
}
}
ACTOR Future<Void> masterProxyServerCore(
MasterProxyInterface proxy,
MasterInterface master,
Reference<AsyncVar<ServerDBInfo>> db,
LogEpoch epoch,
Version recoveryTransactionVersion,
bool firstProxy,
std::string whitelistBinPaths)
{
ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy, MasterInterface master,
Reference<AsyncVar<ServerDBInfo>> db, LogEpoch epoch,
Version recoveryTransactionVersion, bool firstProxy,
std::string whitelistBinPaths) {
state ProxyCommitData commitData(proxy.id(), master, proxy.getConsistentReadVersion, recoveryTransactionVersion, proxy.commit, db, firstProxy);
state Future<Sequence> sequenceFuture = (Sequence)0;
@ -1657,9 +1639,9 @@ ACTOR Future<Void> masterProxyServerCore(
state GetHealthMetricsReply detailedHealthMetricsReply;
addActor.send( waitFailureServer(proxy.waitFailure.getFuture()) );
addActor.send( traceRole(Role::MASTER_PROXY, proxy.id()) );
addActor.send(traceRole(Role::COMMIT_PROXY, proxy.id()));
//TraceEvent("ProxyInit1", proxy.id());
//TraceEvent("CommitProxyInit1", proxy.id());
// Wait until we can load the "real" logsystem, since we don't support switching them currently
while (!(commitData.db->get().master.id() == master.id() && commitData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION)) {
@ -1701,7 +1683,7 @@ ACTOR Future<Void> masterProxyServerCore(
(int)std::min<double>(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_MAX,
std::max<double>(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_MIN,
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_BASE *
pow(commitData.db->get().client.masterProxies.size(),
pow(commitData.db->get().client.commitProxies.size(),
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER)));
commitBatcherActor = commitBatcher(&commitData, batchedCommits, proxy.commit.getFuture(), commitBatchByteLimit, commitBatchesMemoryLimit);
@ -1723,7 +1705,7 @@ ACTOR Future<Void> masterProxyServerCore(
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
const vector<CommitTransactionRequest> &trs = batchedRequests.first;
int batchBytes = batchedRequests.second;
//TraceEvent("MasterProxyCTR", proxy.id()).detail("CommitTransactions", trs.size()).detail("TransactionRate", transactionRate).detail("TransactionQueue", transactionQueue.size()).detail("ReleasedTransactionCount", transactionCount);
//TraceEvent("CommitProxyCTR", proxy.id()).detail("CommitTransactions", trs.size()).detail("TransactionRate", transactionRate).detail("TransactionQueue", transactionQueue.size()).detail("ReleasedTransactionCount", transactionCount);
if (trs.size() || (commitData.db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS && now() - lastCommit >= SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL)) {
lastCommit = now();
@ -1824,27 +1806,27 @@ ACTOR Future<Void> masterProxyServerCore(
}
}
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, uint64_t recoveryCount, MasterProxyInterface myInterface) {
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, uint64_t recoveryCount,
CommitProxyInterface myInterface) {
loop{
if (db->get().recoveryCount >= recoveryCount && !std::count(db->get().client.masterProxies.begin(), db->get().client.masterProxies.end(), myInterface)) {
if (db->get().recoveryCount >= recoveryCount &&
!std::count(db->get().client.commitProxies.begin(), db->get().client.commitProxies.end(), myInterface)) {
throw worker_removed();
}
wait(db->onChange());
}
}
ACTOR Future<Void> masterProxyServer(
MasterProxyInterface proxy,
InitializeMasterProxyRequest req,
Reference<AsyncVar<ServerDBInfo>> db,
std::string whitelistBinPaths)
{
ACTOR Future<Void> commitProxyServer(CommitProxyInterface proxy, InitializeCommitProxyRequest req,
Reference<AsyncVar<ServerDBInfo>> db, std::string whitelistBinPaths) {
try {
state Future<Void> core = masterProxyServerCore(proxy, req.master, db, req.recoveryCount, req.recoveryTransactionVersion, req.firstProxy, whitelistBinPaths);
state Future<Void> core =
commitProxyServerCore(proxy, req.master, db, req.recoveryCount, req.recoveryTransactionVersion,
req.firstProxy, whitelistBinPaths);
wait(core || checkRemoved(db, req.recoveryCount, proxy));
}
catch (Error& e) {
TraceEvent("MasterProxyTerminated", proxy.id()).error(e, true);
TraceEvent("CommitProxyTerminated", proxy.id()).error(e, true);
if (e.code() != error_code_worker_removed && e.code() != error_code_tlog_stopped &&
e.code() != error_code_master_tlog_failed && e.code() != error_code_coordinators_changed &&

View File

@ -21,7 +21,7 @@
#include "fdbclient/Notified.h"
#include "fdbserver/LogSystem.h"
#include "fdbserver/LogSystemDiskQueueAdapter.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/GrvProxyInterface.h"
#include "fdbserver/WaitFailure.h"
#include "fdbserver/WorkerInterface.actor.h"
@ -443,13 +443,13 @@ ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture, std::
TEST(true); // Auto TPS rate is unlimited
}
else {
TEST(true); // Proxy returning tag throttle
TEST(true); // GRV proxy returning tag throttle
reply.tagThrottleInfo[tag.first] = tagItr->second;
}
}
else {
// This isn't required, but we might as well
TEST(true); // Proxy expiring tag throttle
TEST(true); // GRV proxy expiring tag throttle
priorityThrottledTags.erase(tagItr);
}
}

View File

@ -38,7 +38,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( MAX_VERSIONS_IN_FLIGHT_FORCED, 6e5 * VERSIONS_PER_SECOND ); //one week of versions
init( MAX_READ_TRANSACTION_LIFE_VERSIONS, 5 * VERSIONS_PER_SECOND ); if (randomize && BUGGIFY) MAX_READ_TRANSACTION_LIFE_VERSIONS = VERSIONS_PER_SECOND; else if (randomize && BUGGIFY) MAX_READ_TRANSACTION_LIFE_VERSIONS = std::max<int>(1, 0.1 * VERSIONS_PER_SECOND); else if( randomize && BUGGIFY ) MAX_READ_TRANSACTION_LIFE_VERSIONS = 10 * VERSIONS_PER_SECOND;
init( MAX_WRITE_TRANSACTION_LIFE_VERSIONS, 5 * VERSIONS_PER_SECOND ); if (randomize && BUGGIFY) MAX_WRITE_TRANSACTION_LIFE_VERSIONS=std::max<int>(1, 1 * VERSIONS_PER_SECOND);
init( MAX_COMMIT_BATCH_INTERVAL, 2.0 ); if( randomize && BUGGIFY ) MAX_COMMIT_BATCH_INTERVAL = 0.5; // Each master proxy generates a CommitTransactionBatchRequest at least this often, so that versions always advance smoothly
init( MAX_COMMIT_BATCH_INTERVAL, 2.0 ); if( randomize && BUGGIFY ) MAX_COMMIT_BATCH_INTERVAL = 0.5; // Each commit proxy generates a CommitTransactionBatchRequest at least this often, so that versions always advance smoothly
MAX_COMMIT_BATCH_INTERVAL = std::min(MAX_COMMIT_BATCH_INTERVAL, MAX_READ_TRANSACTION_LIFE_VERSIONS/double(2*VERSIONS_PER_SECOND)); // Ensure that the proxy commits 2 times every MAX_READ_TRANSACTION_LIFE_VERSIONS, otherwise the master will not give out versions fast enough
// TLogs
@ -328,7 +328,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( POLLING_FREQUENCY, 2.0 ); if( longLeaderElection ) POLLING_FREQUENCY = 8.0;
init( HEARTBEAT_FREQUENCY, 0.5 ); if( longLeaderElection ) HEARTBEAT_FREQUENCY = 1.0;
// Master Proxy and GRV Proxy
// Commit CommitProxy and GRV CommitProxy
init( START_TRANSACTION_BATCH_INTERVAL_MIN, 1e-6 );
init( START_TRANSACTION_BATCH_INTERVAL_MAX, 0.010 );
init( START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION, 0.5 );
@ -438,7 +438,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( EXPECTED_MASTER_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_TLOG_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_LOG_ROUTER_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_PROXY_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_COMMIT_PROXY_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_GRV_PROXY_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_RESOLVER_FITNESS, ProcessClass::UnsetFit );
init( RECRUITMENT_TIMEOUT, 600 ); if( randomize && BUGGIFY ) RECRUITMENT_TIMEOUT = deterministicRandom()->coinflip() ? 60.0 : 1.0;

View File

@ -37,10 +37,11 @@ public:
int64_t MAX_VERSIONS_IN_FLIGHT_FORCED;
int64_t MAX_READ_TRANSACTION_LIFE_VERSIONS;
int64_t MAX_WRITE_TRANSACTION_LIFE_VERSIONS;
double MAX_COMMIT_BATCH_INTERVAL; // Each master proxy generates a CommitTransactionBatchRequest at least this often, so that versions always advance smoothly
double MAX_COMMIT_BATCH_INTERVAL; // Each commit proxy generates a CommitTransactionBatchRequest at least this
// often, so that versions always advance smoothly
// TLogs
double TLOG_TIMEOUT; // tlog OR master proxy failure - master's reaction time
double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time
double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification
double TLOG_STORAGE_MIN_UPDATE_INTERVAL;
double BUGGIFY_TLOG_STORAGE_MIN_UPDATE_INTERVAL;
@ -262,7 +263,7 @@ public:
double POLLING_FREQUENCY;
double HEARTBEAT_FREQUENCY;
// Master Proxy
// Commit CommitProxy
double START_TRANSACTION_BATCH_INTERVAL_MIN;
double START_TRANSACTION_BATCH_INTERVAL_MAX;
double START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION;
@ -368,7 +369,7 @@ public:
int EXPECTED_MASTER_FITNESS;
int EXPECTED_TLOG_FITNESS;
int EXPECTED_LOG_ROUTER_FITNESS;
int EXPECTED_PROXY_FITNESS;
int EXPECTED_COMMIT_PROXY_FITNESS;
int EXPECTED_GRV_PROXY_FITNESS;
int EXPECTED_RESOLVER_FITNESS;
double RECRUITMENT_TIMEOUT;

View File

@ -38,7 +38,7 @@ struct MasterInterface {
RequestStream< struct ChangeCoordinatorsRequest > changeCoordinators;
RequestStream< struct GetCommitVersionRequest > getCommitVersion;
RequestStream<struct BackupWorkerDoneRequest> notifyBackupWorkerDone;
// Get the centralized live committed version reported by proxies.
// Get the centralized live committed version reported by commit proxies.
RequestStream< struct GetRawCommittedVersionRequest > getLiveCommittedVersion;
// Report a proxy's committed version.
RequestStream< struct ReportRawCommittedVersionRequest> reportLiveCommittedVersion;

View File

@ -527,7 +527,7 @@ struct RatekeeperLimits {
{}
};
struct ProxyInfo {
struct GrvProxyInfo {
int64_t totalTransactions;
int64_t batchTransactions;
uint64_t lastThrottledTagChangeId;
@ -535,7 +535,9 @@ struct ProxyInfo {
double lastUpdateTime;
double lastTagPushTime;
ProxyInfo() : totalTransactions(0), batchTransactions(0), lastUpdateTime(0), lastThrottledTagChangeId(0), lastTagPushTime(0) {}
GrvProxyInfo()
: totalTransactions(0), batchTransactions(0), lastUpdateTime(0), lastThrottledTagChangeId(0), lastTagPushTime(0) {
}
};
struct RatekeeperData {
@ -545,7 +547,7 @@ struct RatekeeperData {
Map<UID, StorageQueueInfo> storageQueueInfo;
Map<UID, TLogQueueInfo> tlogQueueInfo;
std::map<UID, ProxyInfo> proxyInfo;
std::map<UID, GrvProxyInfo> grvProxyInfo;
Smoother smoothReleasedTransactions, smoothBatchReleasedTransactions, smoothTotalDurableBytes;
HealthMetrics healthMetrics;
DatabaseConfiguration configuration;
@ -1262,31 +1264,31 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
if (deterministicRandom()->random01() < 0.1) {
std::string name = "RkUpdate" + limits->context;
TraceEvent(name.c_str(), self->id)
.detail("TPSLimit", limits->tpsLimit)
.detail("Reason", limitReason)
.detail("ReasonServerID", reasonID==UID() ? std::string() : Traceable<UID>::toString(reasonID))
.detail("ReleasedTPS", self->smoothReleasedTransactions.smoothRate())
.detail("ReleasedBatchTPS", self->smoothBatchReleasedTransactions.smoothRate())
.detail("TPSBasis", actualTps)
.detail("StorageServers", sscount)
.detail("GrvProxies", self->proxyInfo.size())
.detail("TLogs", tlcount)
.detail("WorstFreeSpaceStorageServer", worstFreeSpaceStorageServer)
.detail("WorstFreeSpaceTLog", worstFreeSpaceTLog)
.detail("WorstStorageServerQueue", worstStorageQueueStorageServer)
.detail("LimitingStorageServerQueue", limitingStorageQueueStorageServer)
.detail("WorstTLogQueue", worstStorageQueueTLog)
.detail("TotalDiskUsageBytes", totalDiskUsageBytes)
.detail("WorstStorageServerVersionLag", worstVersionLag)
.detail("LimitingStorageServerVersionLag", limitingVersionLag)
.detail("WorstStorageServerDurabilityLag", worstDurabilityLag)
.detail("LimitingStorageServerDurabilityLag", limitingDurabilityLag)
.detail("TagsAutoThrottled", self->throttledTags.autoThrottleCount())
.detail("TagsAutoThrottledBusyRead", self->throttledTags.busyReadTagCount)
.detail("TagsAutoThrottledBusyWrite", self->throttledTags.busyWriteTagCount)
.detail("TagsManuallyThrottled", self->throttledTags.manualThrottleCount())
.detail("AutoThrottlingEnabled", self->autoThrottlingEnabled)
.trackLatest(name);
.detail("TPSLimit", limits->tpsLimit)
.detail("Reason", limitReason)
.detail("ReasonServerID", reasonID == UID() ? std::string() : Traceable<UID>::toString(reasonID))
.detail("ReleasedTPS", self->smoothReleasedTransactions.smoothRate())
.detail("ReleasedBatchTPS", self->smoothBatchReleasedTransactions.smoothRate())
.detail("TPSBasis", actualTps)
.detail("StorageServers", sscount)
.detail("GrvProxies", self->grvProxyInfo.size())
.detail("TLogs", tlcount)
.detail("WorstFreeSpaceStorageServer", worstFreeSpaceStorageServer)
.detail("WorstFreeSpaceTLog", worstFreeSpaceTLog)
.detail("WorstStorageServerQueue", worstStorageQueueStorageServer)
.detail("LimitingStorageServerQueue", limitingStorageQueueStorageServer)
.detail("WorstTLogQueue", worstStorageQueueTLog)
.detail("TotalDiskUsageBytes", totalDiskUsageBytes)
.detail("WorstStorageServerVersionLag", worstVersionLag)
.detail("LimitingStorageServerVersionLag", limitingVersionLag)
.detail("WorstStorageServerDurabilityLag", worstDurabilityLag)
.detail("LimitingStorageServerDurabilityLag", limitingDurabilityLag)
.detail("TagsAutoThrottled", self->throttledTags.autoThrottleCount())
.detail("TagsAutoThrottledBusyRead", self->throttledTags.busyReadTagCount)
.detail("TagsAutoThrottledBusyWrite", self->throttledTags.busyWriteTagCount)
.detail("TagsManuallyThrottled", self->throttledTags.manualThrottleCount())
.detail("AutoThrottlingEnabled", self->autoThrottlingEnabled)
.trackLatest(name);
}
}
@ -1371,9 +1373,9 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
lastLimited = self.smoothReleasedTransactions.smoothRate() > SERVER_KNOBS->LAST_LIMITED_RATIO * self.batchLimits.tpsLimit;
double tooOld = now() - 1.0;
for(auto p=self.proxyInfo.begin(); p!=self.proxyInfo.end(); ) {
for (auto p = self.grvProxyInfo.begin(); p != self.grvProxyInfo.end();) {
if (p->second.lastUpdateTime < tooOld)
p = self.proxyInfo.erase(p);
p = self.grvProxyInfo.erase(p);
else
++p;
}
@ -1382,7 +1384,7 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
when (GetRateInfoRequest req = waitNext(rkInterf.getRateInfo.getFuture())) {
GetRateInfoReply reply;
auto& p = self.proxyInfo[ req.requesterID ];
auto& p = self.grvProxyInfo[req.requesterID];
//TraceEvent("RKMPU", req.requesterID).detail("TRT", req.totalReleasedTransactions).detail("Last", p.totalTransactions).detail("Delta", req.totalReleasedTransactions - p.totalTransactions);
if (p.totalTransactions > 0) {
self.smoothReleasedTransactions.addDelta( req.totalReleasedTransactions - p.totalTransactions );
@ -1399,8 +1401,8 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
p.batchTransactions = req.batchReleasedTransactions;
p.lastUpdateTime = now();
reply.transactionRate = self.normalLimits.tpsLimit / self.proxyInfo.size();
reply.batchTransactionRate = self.batchLimits.tpsLimit / self.proxyInfo.size();
reply.transactionRate = self.normalLimits.tpsLimit / self.grvProxyInfo.size();
reply.batchTransactionRate = self.batchLimits.tpsLimit / self.grvProxyInfo.size();
reply.leaseDuration = SERVER_KNOBS->METRIC_UPDATE_RATE;
if(p.lastThrottledTagChangeId != self.throttledTagChangeId || now() > p.lastTagPushTime + SERVER_KNOBS->TAG_THROTTLE_PUSH_INTERVAL) {

View File

@ -44,7 +44,7 @@ struct ProxyRequestsInfo {
namespace{
struct Resolver : ReferenceCounted<Resolver> {
UID dbgid;
int proxyCount, resolverCount;
int commitProxyCount, resolverCount;
NotifiedVersion version;
AsyncVar<Version> neededVersion;
@ -77,8 +77,8 @@ struct Resolver : ReferenceCounted<Resolver> {
Future<Void> logger;
Resolver( UID dbgid, int proxyCount, int resolverCount )
: dbgid(dbgid), proxyCount(proxyCount), resolverCount(resolverCount), version(-1), conflictSet( newConflictSet() ), iopsSample( SERVER_KNOBS->KEY_BYTES_PER_SAMPLE ), debugMinRecentStateVersion(0),
Resolver( UID dbgid, int commitProxyCount, int resolverCount )
: dbgid(dbgid), commitProxyCount(commitProxyCount), resolverCount(resolverCount), version(-1), conflictSet( newConflictSet() ), iopsSample( SERVER_KNOBS->KEY_BYTES_PER_SAMPLE ), debugMinRecentStateVersion(0),
cc("Resolver", dbgid.toString()),
resolveBatchIn("ResolveBatchIn", cc), resolveBatchStart("ResolveBatchStart", cc), resolvedTransactions("ResolvedTransactions", cc), resolvedBytes("ResolvedBytes", cc),
resolvedReadConflictRanges("ResolvedReadConflictRanges", cc), resolvedWriteConflictRanges("ResolvedWriteConflictRanges", cc), transactionsAccepted("TransactionsAccepted", cc),
@ -238,12 +238,12 @@ ACTOR Future<Void> resolveBatch(
//TraceEvent("ResolveBatch", self->dbgid).detail("PrevVersion", req.prevVersion).detail("Version", req.version).detail("StateTransactionVersions", self->recentStateTransactionSizes.size()).detail("StateBytes", stateBytes).detail("FirstVersion", self->recentStateTransactionSizes.empty() ? -1 : self->recentStateTransactionSizes.front().first).detail("StateMutationsIn", req.txnStateTransactions.size()).detail("StateMutationsOut", reply.stateMutations.size()).detail("From", proxyAddress);
ASSERT(!proxyInfo.outstandingBatches.empty());
ASSERT(self->proxyInfoMap.size() <= self->proxyCount+1);
ASSERT(self->proxyInfoMap.size() <= self->commitProxyCount+1);
// SOMEDAY: This is O(n) in number of proxies. O(log n) solution using appropriate data structure?
Version oldestProxyVersion = req.version;
for(auto itr = self->proxyInfoMap.begin(); itr != self->proxyInfoMap.end(); ++itr) {
//TraceEvent("ResolveBatchProxyVersion", self->dbgid).detail("Proxy", itr->first).detail("Version", itr->second.lastVersion);
//TraceEvent("ResolveBatchProxyVersion", self->dbgid).detail("CommitProxy", itr->first).detail("Version", itr->second.lastVersion);
if(itr->first.isValid()) { // Don't consider the first master request
oldestProxyVersion = std::min(itr->second.lastVersion, oldestProxyVersion);
}
@ -257,7 +257,7 @@ ACTOR Future<Void> resolveBatch(
TEST(oldestProxyVersion != req.version); // The proxy that sent this request does not have the oldest current version
bool anyPopped = false;
if(firstUnseenVersion <= oldestProxyVersion && self->proxyInfoMap.size() == self->proxyCount+1) {
if(firstUnseenVersion <= oldestProxyVersion && self->proxyInfoMap.size() == self->commitProxyCount+1) {
TEST(true); // Deleting old state transactions
self->recentStateTransactions.erase( self->recentStateTransactions.begin(), self->recentStateTransactions.upper_bound( oldestProxyVersion ) );
self->debugMinRecentStateVersion = oldestProxyVersion + 1;
@ -311,7 +311,7 @@ ACTOR Future<Void> resolverCore(
ResolverInterface resolver,
InitializeResolverRequest initReq)
{
state Reference<Resolver> self( new Resolver(resolver.id(), initReq.proxyCount, initReq.resolverCount) );
state Reference<Resolver> self(new Resolver(resolver.id(), initReq.commitProxyCount, initReq.resolverCount));
state ActorCollection actors(false);
state Future<Void> doPollMetrics = self->resolverCount > 1 ? Void() : Future<Void>(Never());
actors.add( waitFailureServer(resolver.waitFailure.getFuture()) );

View File

@ -733,7 +733,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
bool generateFearless = simple ? false : (minimumRegions > 1 || deterministicRandom()->random01() < 0.5);
datacenters = simple ? 1 : ( generateFearless ? ( minimumReplication > 0 || deterministicRandom()->random01() < 0.5 ? 4 : 6 ) : deterministicRandom()->randomInt( 1, 4 ) );
if (deterministicRandom()->random01() < 0.25) db.desiredTLogCount = deterministicRandom()->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) db.proxyCount = deterministicRandom()->randomInt(1, 7);
if (deterministicRandom()->random01() < 0.25) db.commitProxyCount = deterministicRandom()->randomInt(1, 7);
if (deterministicRandom()->random01() < 0.25) db.grvProxyCount = deterministicRandom()->randomInt(1, 4);
if (deterministicRandom()->random01() < 0.25) db.resolverCount = deterministicRandom()->randomInt(1,7);
int storage_engine_type = deterministicRandom()->randomInt(0, 4);
@ -770,7 +770,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
// set_config("memory-radixtree-beta");
if(simple) {
db.desiredTLogCount = 1;
db.proxyCount = 1;
db.commitProxyCount = 1;
db.grvProxyCount = 1;
db.resolverCount = 1;
}

View File

@ -574,7 +574,7 @@ struct RolesInfo {
*pMetricVersion = metricVersion;
return roles.insert( std::make_pair(iface.address(), obj ))->second;
}
JsonBuilderObject& addRole(std::string const& role, MasterProxyInterface& iface, EventMap const& metrics) {
JsonBuilderObject& addRole(std::string const& role, CommitProxyInterface& iface, EventMap const& metrics) {
JsonBuilderObject obj;
obj["id"] = iface.id().shortString();
obj["role"] = role;
@ -646,11 +646,10 @@ ACTOR static Future<JsonBuilderObject> processStatusFetcher(
WorkerEvents mMetrics, WorkerEvents nMetrics, WorkerEvents errors, WorkerEvents traceFileOpenErrors,
WorkerEvents programStarts, std::map<std::string, std::vector<JsonBuilderObject>> processIssues,
vector<std::pair<StorageServerInterface, EventMap>> storageServers,
vector<std::pair<TLogInterface, EventMap>> tLogs,
vector<std::pair<MasterProxyInterface, EventMap>> proxies,
vector<std::pair<GrvProxyInterface, EventMap>> grvProxies,
ServerCoordinators coordinators, Database cx, Optional<DatabaseConfiguration> configuration,
Optional<Key> healthyZone, std::set<std::string>* incomplete_reasons) {
vector<std::pair<TLogInterface, EventMap>> tLogs, vector<std::pair<CommitProxyInterface, EventMap>> commitProxies,
vector<std::pair<GrvProxyInterface, EventMap>> grvProxies, ServerCoordinators coordinators, Database cx,
Optional<DatabaseConfiguration> configuration, Optional<Key> healthyZone,
std::set<std::string>* incomplete_reasons) {
state JsonBuilderObject processMap;
@ -736,9 +735,9 @@ ACTOR static Future<JsonBuilderObject> processStatusFetcher(
roles.addCoordinatorRole(coordinator);
}
state std::vector<std::pair<MasterProxyInterface, EventMap>>::iterator proxy;
for(proxy = proxies.begin(); proxy != proxies.end(); ++proxy) {
roles.addRole( "proxy", proxy->first, proxy->second );
state std::vector<std::pair<CommitProxyInterface, EventMap>>::iterator commit_proxy;
for (commit_proxy = commitProxies.begin(); commit_proxy != commitProxies.end(); ++commit_proxy) {
roles.addRole("commit_proxy", commit_proxy->first, commit_proxy->second);
wait(yield());
}
@ -1064,14 +1063,14 @@ ACTOR static Future<JsonBuilderObject> recoveryStateStatusFetcher(WorkerDetails
// Add additional metadata for certain statuses
if (mStatusCode == RecoveryStatus::recruiting_transaction_servers) {
int requiredLogs = atoi( md.getValue("RequiredTLogs").c_str() );
int requiredProxies = atoi( md.getValue("RequiredProxies").c_str() );
int requiredCommitProxies = atoi(md.getValue("RequiredCommitProxies").c_str());
int requiredGrvProxies = atoi(md.getValue("RequiredGrvProxies").c_str());
int requiredResolvers = atoi( md.getValue("RequiredResolvers").c_str() );
//int requiredProcesses = std::max(requiredLogs, std::max(requiredResolvers, requiredProxies));
//int requiredProcesses = std::max(requiredLogs, std::max(requiredResolvers, requiredCommitProxies));
//int requiredMachines = std::max(requiredLogs, 1);
message["required_logs"] = requiredLogs;
message["required_proxies"] = requiredProxies;
message["required_commit_proxies"] = requiredCommitProxies;
message["required_grv_proxies"] = requiredGrvProxies;
message["required_resolvers"] = requiredResolvers;
} else if (mStatusCode == RecoveryStatus::locking_old_transaction_servers) {
@ -1669,9 +1668,11 @@ ACTOR static Future<vector<std::pair<TLogInterface, EventMap>>> getTLogsAndMetri
return results;
}
ACTOR static Future<vector<std::pair<MasterProxyInterface, EventMap>>> getProxiesAndMetrics(Reference<AsyncVar<ServerDBInfo>> db, std::unordered_map<NetworkAddress, WorkerInterface> address_workers) {
vector<std::pair<MasterProxyInterface, EventMap>> results = wait(getServerMetrics(
db->get().client.masterProxies, address_workers, std::vector<std::string>{ "CommitLatencyMetrics", "CommitLatencyBands" }));
ACTOR static Future<vector<std::pair<CommitProxyInterface, EventMap>>> getCommitProxiesAndMetrics(
Reference<AsyncVar<ServerDBInfo>> db, std::unordered_map<NetworkAddress, WorkerInterface> address_workers) {
vector<std::pair<CommitProxyInterface, EventMap>> results =
wait(getServerMetrics(db->get().client.commitProxies, address_workers,
std::vector<std::string>{ "CommitLatencyMetrics", "CommitLatencyBands" }));
return results;
}
@ -1755,16 +1756,18 @@ ACTOR static Future<JsonBuilderObject> workloadStatusFetcher(Reference<AsyncVar<
// Writes and conflicts
try {
state vector<Future<TraceEventFields>> proxyStatFutures;
state vector<Future<TraceEventFields>> commitProxyStatFutures;
state vector<Future<TraceEventFields>> grvProxyStatFutures;
std::map<NetworkAddress, WorkerDetails> workersMap;
for (auto const& w : workers) {
workersMap[w.interf.address()] = w;
}
for (auto &p : db->get().client.masterProxies) {
for (auto& p : db->get().client.commitProxies) {
auto worker = getWorker(workersMap, p.address());
if (worker.present())
proxyStatFutures.push_back(timeoutError(worker.get().interf.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("ProxyMetrics"))), 1.0));
commitProxyStatFutures.push_back(timeoutError(
worker.get().interf.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("ProxyMetrics"))),
1.0));
else
throw all_alternatives_failed(); // We need data from all proxies for this result to be trustworthy
}
@ -1775,7 +1778,7 @@ ACTOR static Future<JsonBuilderObject> workloadStatusFetcher(Reference<AsyncVar<
else
throw all_alternatives_failed(); // We need data from all proxies for this result to be trustworthy
}
state vector<TraceEventFields> proxyStats = wait(getAll(proxyStatFutures));
state vector<TraceEventFields> commitProxyStats = wait(getAll(commitProxyStatFutures));
state vector<TraceEventFields> grvProxyStats = wait(getAll(grvProxyStatFutures));
StatusCounter txnStartOut;
@ -1798,14 +1801,14 @@ ACTOR static Future<JsonBuilderObject> workloadStatusFetcher(Reference<AsyncVar<
txnMemoryErrors.updateValues(StatusCounter(gps.getValue("TxnRequestErrors")));
}
for (auto &ps : proxyStats) {
mutations.updateValues( StatusCounter(ps.getValue("Mutations")) );
mutationBytes.updateValues( StatusCounter(ps.getValue("MutationBytes")) );
txnConflicts.updateValues( StatusCounter(ps.getValue("TxnConflicts")) );
txnCommitOutSuccess.updateValues( StatusCounter(ps.getValue("TxnCommitOutSuccess")) );
txnKeyLocationOut.updateValues( StatusCounter(ps.getValue("KeyServerLocationOut")) );
txnMemoryErrors.updateValues( StatusCounter(ps.getValue("KeyServerLocationErrors")) );
txnMemoryErrors.updateValues( StatusCounter(ps.getValue("TxnCommitErrors")) );
for (auto& cps : commitProxyStats) {
mutations.updateValues(StatusCounter(cps.getValue("Mutations")));
mutationBytes.updateValues(StatusCounter(cps.getValue("MutationBytes")));
txnConflicts.updateValues(StatusCounter(cps.getValue("TxnConflicts")));
txnCommitOutSuccess.updateValues(StatusCounter(cps.getValue("TxnCommitOutSuccess")));
txnKeyLocationOut.updateValues(StatusCounter(cps.getValue("KeyServerLocationOut")));
txnMemoryErrors.updateValues(StatusCounter(cps.getValue("KeyServerLocationErrors")));
txnMemoryErrors.updateValues(StatusCounter(cps.getValue("TxnCommitErrors")));
}
operationsObj["writes"] = mutations.getStatus();
@ -2440,7 +2443,7 @@ ACTOR Future<StatusReply> clusterGetStatus(
getProcessIssuesAsMessages(workerIssues);
state vector<std::pair<StorageServerInterface, EventMap>> storageServers;
state vector<std::pair<TLogInterface, EventMap>> tLogs;
state vector<std::pair<MasterProxyInterface, EventMap>> proxies;
state vector<std::pair<CommitProxyInterface, EventMap>> commitProxies;
state vector<std::pair<GrvProxyInterface, EventMap>> grvProxies;
state JsonBuilderObject qos;
state JsonBuilderObject data_overlay;
@ -2504,7 +2507,8 @@ ACTOR Future<StatusReply> clusterGetStatus(
state Future<ErrorOr<vector<std::pair<StorageServerInterface, EventMap>>>> storageServerFuture = errorOr(getStorageServersAndMetrics(cx, address_workers, rkWorker));
state Future<ErrorOr<vector<std::pair<TLogInterface, EventMap>>>> tLogFuture = errorOr(getTLogsAndMetrics(db, address_workers));
state Future<ErrorOr<vector<std::pair<MasterProxyInterface, EventMap>>>> proxyFuture = errorOr(getProxiesAndMetrics(db, address_workers));
state Future<ErrorOr<vector<std::pair<CommitProxyInterface, EventMap>>>> commitProxyFuture =
errorOr(getCommitProxiesAndMetrics(db, address_workers));
state Future<ErrorOr<vector<std::pair<GrvProxyInterface, EventMap>>>> grvProxyFuture = errorOr(getGrvProxiesAndMetrics(db, address_workers));
state int minReplicasRemaining = -1;
@ -2587,13 +2591,13 @@ ACTOR Future<StatusReply> clusterGetStatus(
messages.push_back(JsonBuilder::makeMessage("log_servers_error", "Timed out trying to retrieve log servers."));
}
// ...also proxies
ErrorOr<vector<std::pair<MasterProxyInterface, EventMap>>> _proxies = wait(proxyFuture);
if (_proxies.present()) {
proxies = _proxies.get();
}
else {
messages.push_back(JsonBuilder::makeMessage("proxies_error", "Timed out trying to retrieve proxies."));
// ...also commit proxies
ErrorOr<vector<std::pair<CommitProxyInterface, EventMap>>> _commitProxies = wait(commitProxyFuture);
if (_commitProxies.present()) {
commitProxies = _commitProxies.get();
} else {
messages.push_back(
JsonBuilder::makeMessage("commit_proxies_error", "Timed out trying to retrieve commit proxies."));
}
// ...also grv proxies
@ -2614,12 +2618,10 @@ ACTOR Future<StatusReply> clusterGetStatus(
statusObj["layers"] = layers;
}
JsonBuilderObject processStatus = wait(processStatusFetcher(db, workers, pMetrics, mMetrics, networkMetrics,
latestError, traceFileOpenErrors, programStarts,
processIssues, storageServers, tLogs, proxies,
grvProxies, coordinators, cx, configuration,
loadResult.present() ? loadResult.get().healthyZone : Optional<Key>(),
&status_incomplete_reasons));
JsonBuilderObject processStatus = wait(processStatusFetcher(
db, workers, pMetrics, mMetrics, networkMetrics, latestError, traceFileOpenErrors, programStarts,
processIssues, storageServers, tLogs, commitProxies, grvProxies, coordinators, cx, configuration,
loadResult.present() ? loadResult.get().healthyZone : Optional<Key>(), &status_incomplete_reasons));
statusObj["processes"] = processStatus;
statusObj["clients"] = clientStatusFetcher(clientStatus);

View File

@ -46,7 +46,7 @@ struct WorkerInterface {
LocalityData locality;
RequestStream< struct InitializeTLogRequest > tLog;
RequestStream< struct RecruitMasterRequest > master;
RequestStream< struct InitializeMasterProxyRequest > masterProxy;
RequestStream<struct InitializeCommitProxyRequest> commitProxy;
RequestStream< struct InitializeGrvProxyRequest > grvProxy;
RequestStream< struct InitializeDataDistributorRequest > dataDistributor;
RequestStream< struct InitializeRatekeeperRequest > ratekeeper;
@ -81,7 +81,7 @@ struct WorkerInterface {
clientInterface.initEndpoints();
tLog.getEndpoint( TaskPriority::Worker );
master.getEndpoint( TaskPriority::Worker );
masterProxy.getEndpoint( TaskPriority::Worker );
commitProxy.getEndpoint(TaskPriority::Worker);
grvProxy.getEndpoint( TaskPriority::Worker );
resolver.getEndpoint( TaskPriority::Worker );
logRouter.getEndpoint( TaskPriority::Worker );
@ -93,7 +93,10 @@ struct WorkerInterface {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, clientInterface, locality, tLog, master, masterProxy, grvProxy, dataDistributor, ratekeeper, resolver, storage, logRouter, debugPing, coordinationPing, waitFailure, setMetricsRate, eventLogRequest, traceBatchDumpRequest, testerInterface, diskStoreRequest, execReq, workerSnapReq, backup, updateServerDBInfo);
serializer(ar, clientInterface, locality, tLog, master, commitProxy, grvProxy, dataDistributor, ratekeeper,
resolver, storage, logRouter, debugPing, coordinationPing, waitFailure, setMetricsRate,
eventLogRequest, traceBatchDumpRequest, testerInterface, diskStoreRequest, execReq, workerSnapReq,
backup, updateServerDBInfo);
}
};
@ -180,7 +183,7 @@ struct RegisterMasterRequest {
UID id;
LocalityData mi;
LogSystemConfig logSystemConfig;
std::vector<MasterProxyInterface> masterProxies;
std::vector<CommitProxyInterface> commitProxies;
std::vector<GrvProxyInterface> grvProxies;
std::vector<ResolverInterface> resolvers;
DBRecoveryCount recoveryCount;
@ -199,7 +202,7 @@ struct RegisterMasterRequest {
if constexpr (!is_fb_function<Ar>) {
ASSERT(ar.protocolVersion().isValid());
}
serializer(ar, id, mi, logSystemConfig, masterProxies, grvProxies, resolvers, recoveryCount, registrationCount,
serializer(ar, id, mi, logSystemConfig, commitProxies, grvProxies, resolvers, recoveryCount, registrationCount,
configuration, priorCommittedLogServers, recoveryState, recoveryStalled, reply);
}
};
@ -209,7 +212,7 @@ struct RecruitFromConfigurationReply {
std::vector<WorkerInterface> backupWorkers;
std::vector<WorkerInterface> tLogs;
std::vector<WorkerInterface> satelliteTLogs;
std::vector<WorkerInterface> masterProxies;
std::vector<WorkerInterface> commitProxies;
std::vector<WorkerInterface> grvProxies;
std::vector<WorkerInterface> resolvers;
std::vector<WorkerInterface> storageServers;
@ -221,7 +224,7 @@ struct RecruitFromConfigurationReply {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, tLogs, satelliteTLogs, masterProxies, grvProxies, resolvers, storageServers, oldLogRouters, dcId,
serializer(ar, tLogs, satelliteTLogs, commitProxies, grvProxies, resolvers, storageServers, oldLogRouters, dcId,
satelliteFallback, backupWorkers);
}
};
@ -433,13 +436,13 @@ struct RecruitMasterRequest {
}
};
struct InitializeMasterProxyRequest {
struct InitializeCommitProxyRequest {
constexpr static FileIdentifier file_identifier = 10344153;
MasterInterface master;
uint64_t recoveryCount;
Version recoveryTransactionVersion;
bool firstProxy;
ReplyPromise<MasterProxyInterface> reply;
ReplyPromise<CommitProxyInterface> reply;
template <class Ar>
void serialize(Ar& ar) {
@ -488,13 +491,13 @@ struct InitializeRatekeeperRequest {
struct InitializeResolverRequest {
constexpr static FileIdentifier file_identifier = 7413317;
uint64_t recoveryCount;
int proxyCount;
int commitProxyCount;
int resolverCount;
ReplyPromise<ResolverInterface> reply;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, recoveryCount, proxyCount, resolverCount, reply);
serializer(ar, recoveryCount, commitProxyCount, resolverCount, reply);
}
};
@ -672,7 +675,7 @@ struct Role {
static const Role STORAGE_SERVER;
static const Role TRANSACTION_LOG;
static const Role SHARED_TRANSACTION_LOG;
static const Role MASTER_PROXY;
static const Role COMMIT_PROXY;
static const Role GRV_PROXY;
static const Role MASTER;
static const Role RESOLVER;
@ -735,7 +738,7 @@ ACTOR Future<Void> storageServer(IKeyValueStore* persistentData, StorageServerIn
Reference<ClusterConnectionFile> connFile ); // changes pssi->id() to be the recovered ID); // changes pssi->id() to be the recovered ID
ACTOR Future<Void> masterServer(MasterInterface mi, Reference<AsyncVar<ServerDBInfo>> db, Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> ccInterface,
ServerCoordinators serverCoordinators, LifetimeToken lifetime, bool forceRecovery);
ACTOR Future<Void> masterProxyServer(MasterProxyInterface proxy, InitializeMasterProxyRequest req,
ACTOR Future<Void> commitProxyServer(CommitProxyInterface proxy, InitializeCommitProxyRequest req,
Reference<AsyncVar<ServerDBInfo>> db, std::string whitelistBinPaths);
ACTOR Future<Void> grvProxyServer(GrvProxyInterface proxy, InitializeGrvProxyRequest req, Reference<AsyncVar<ServerDBInfo>> db);
ACTOR Future<Void> tLog(IKeyValueStore* persistentData, IDiskQueue* persistentQueue,

View File

@ -542,9 +542,9 @@ static void printUsage( const char *name, bool devhelp ) {
" The default value is 2GiB. When specified without a unit,\n"
" MiB is assumed.\n");
printf(" -c CLASS, --class CLASS\n"
" Machine class (valid options are storage, transaction,\n"
" resolution, proxy, master, test, unset, stateless, log, router,\n"
" and cluster_controller).\n");
" Machine class (valid options are storage, transaction,\n"
" resolution, grv_proxy, proxy, master, test, unset, stateless, log, router,\n"
" and cluster_controller).\n");
#ifndef TLS_DISABLED
printf(TLS_HELP);
#endif
@ -2028,7 +2028,8 @@ int main(int argc, char* argv[]) {
}
static_assert( LBLocalityData<StorageServerInterface>::Present, "Storage server interface should be load balanced" );
static_assert( LBLocalityData<MasterProxyInterface>::Present, "Master proxy interface should be load balanced" );
static_assert(LBLocalityData<CommitProxyInterface>::Present, "Commit proxy interface should be load balanced");
static_assert(LBLocalityData<GrvProxyInterface>::Present, "GRV proxy interface should be load balanced");
static_assert( LBLocalityData<TLogInterface>::Present, "TLog interface should be load balanced" );
static_assert( !LBLocalityData<MasterInterface>::Present, "Master interface should not be load balanced" );
}

View File

@ -52,18 +52,18 @@ using std::vector;
using std::min;
using std::max;
struct ProxyVersionReplies {
struct CommitProxyVersionReplies {
std::map<uint64_t, GetCommitVersionReply> replies;
NotifiedVersion latestRequestNum;
ProxyVersionReplies(ProxyVersionReplies&& r) noexcept
CommitProxyVersionReplies(CommitProxyVersionReplies&& r) noexcept
: replies(std::move(r.replies)), latestRequestNum(std::move(r.latestRequestNum)) {}
void operator=(ProxyVersionReplies&& r) noexcept {
void operator=(CommitProxyVersionReplies&& r) noexcept {
replies = std::move(r.replies);
latestRequestNum = std::move(r.latestRequestNum);
}
ProxyVersionReplies() : latestRequestNum(0) {}
CommitProxyVersionReplies() : latestRequestNum(0) {}
};
ACTOR Future<Void> masterTerminateOnConflict( UID dbgid, Promise<Void> fullyRecovered, Future<Void> onConflict, Future<Void> switchedState ) {
@ -177,7 +177,7 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
recoveryTransactionVersion; // The first version in this epoch
double lastCommitTime;
Version liveCommittedVersion; // The largest live committed version reported by proxies.
Version liveCommittedVersion; // The largest live committed version reported by commit proxies.
bool databaseLocked;
Optional<Value> proxyMetadataVersion;
Version minKnownCommittedVersion;
@ -207,13 +207,13 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
return maxLocality + 1;
}
std::vector<MasterProxyInterface> masterProxies;
std::vector<MasterProxyInterface> provisionalMasterProxies;
std::vector<CommitProxyInterface> commitProxies;
std::vector<CommitProxyInterface> provisionalCommitProxies;
std::vector<GrvProxyInterface> grvProxies;
std::vector<GrvProxyInterface> provisionalGrvProxies;
std::vector<ResolverInterface> resolvers;
std::map<UID, ProxyVersionReplies> lastProxyVersionReplies;
std::map<UID, CommitProxyVersionReplies> lastCommitProxyVersionReplies;
Standalone<StringRef> dbId;
@ -283,21 +283,24 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
~MasterData() { if(txnStateStore) txnStateStore->close(); }
};
ACTOR Future<Void> newMasterProxies( Reference<MasterData> self, RecruitFromConfigurationReply recr ) {
vector<Future<MasterProxyInterface>> initializationReplies;
for( int i = 0; i < recr.masterProxies.size(); i++ ) {
InitializeMasterProxyRequest req;
ACTOR Future<Void> newCommitProxies(Reference<MasterData> self, RecruitFromConfigurationReply recr) {
vector<Future<CommitProxyInterface>> initializationReplies;
for (int i = 0; i < recr.commitProxies.size(); i++) {
InitializeCommitProxyRequest req;
req.master = self->myInterface;
req.recoveryCount = self->cstate.myDBState.recoveryCount + 1;
req.recoveryTransactionVersion = self->recoveryTransactionVersion;
req.firstProxy = i == 0;
TraceEvent("MasterProxyReplies",self->dbgid).detail("WorkerID", recr.masterProxies[i].id());
initializationReplies.push_back( transformErrors( throwErrorOr( recr.masterProxies[i].masterProxy.getReplyUnlessFailedFor( req, SERVER_KNOBS->TLOG_TIMEOUT, SERVER_KNOBS->MASTER_FAILURE_SLOPE_DURING_RECOVERY ) ), master_recovery_failed() ) );
TraceEvent("CommitProxyReplies", self->dbgid).detail("WorkerID", recr.commitProxies[i].id());
initializationReplies.push_back(
transformErrors(throwErrorOr(recr.commitProxies[i].commitProxy.getReplyUnlessFailedFor(
req, SERVER_KNOBS->TLOG_TIMEOUT, SERVER_KNOBS->MASTER_FAILURE_SLOPE_DURING_RECOVERY)),
master_recovery_failed()));
}
vector<MasterProxyInterface> newRecruits = wait( getAll( initializationReplies ) );
// It is required for the correctness of COMMIT_ON_FIRST_PROXY that self->proxies[0] is the firstProxy.
self->masterProxies = newRecruits;
vector<CommitProxyInterface> newRecruits = wait(getAll(initializationReplies));
// It is required for the correctness of COMMIT_ON_FIRST_PROXY that self->commitProxies[0] is the firstCommitProxy.
self->commitProxies = newRecruits;
return Void();
}
@ -322,7 +325,7 @@ ACTOR Future<Void> newResolvers( Reference<MasterData> self, RecruitFromConfigur
for( int i = 0; i < recr.resolvers.size(); i++ ) {
InitializeResolverRequest req;
req.recoveryCount = self->cstate.myDBState.recoveryCount + 1;
req.proxyCount = recr.masterProxies.size();
req.commitProxyCount = recr.commitProxies.size();
req.resolverCount = recr.resolvers.size();
TraceEvent("ResolverReplies",self->dbgid).detail("WorkerID", recr.resolvers[i].id());
initializationReplies.push_back( transformErrors( throwErrorOr( recr.resolvers[i].resolver.getReplyUnlessFailedFor( req, SERVER_KNOBS->TLOG_TIMEOUT, SERVER_KNOBS->MASTER_FAILURE_SLOPE_DURING_RECOVERY ) ), master_recovery_failed() ) );
@ -426,15 +429,15 @@ ACTOR Future<Void> newSeedServers( Reference<MasterData> self, RecruitFromConfig
return Void();
}
Future<Void> waitProxyFailure( vector<MasterProxyInterface> const& proxies ) {
Future<Void> waitCommitProxyFailure(vector<CommitProxyInterface> const& commitProxies) {
std::vector<Future<Void>> failed;
for (auto proxy : proxies) {
failed.push_back(waitFailureClient(proxy.waitFailure, SERVER_KNOBS->TLOG_TIMEOUT,
for (auto commitProxy : commitProxies) {
failed.push_back(waitFailureClient(commitProxy.waitFailure, SERVER_KNOBS->TLOG_TIMEOUT,
-SERVER_KNOBS->TLOG_TIMEOUT / SERVER_KNOBS->SECONDS_BEFORE_NO_FAILURE_DELAY,
/*trace=*/true));
}
ASSERT( failed.size() >= 1 );
return tagError<Void>(quorum( failed, 1 ), master_proxy_failed());
return tagError<Void>(quorum(failed, 1), commit_proxy_failed());
}
Future<Void> waitGrvProxyFailure( vector<GrvProxyInterface> const& grvProxies ) {
@ -499,14 +502,14 @@ ACTOR Future<Void> updateLogsValue( Reference<MasterData> self, Database cx ) {
}
Future<Void> sendMasterRegistration(MasterData* self, LogSystemConfig const& logSystemConfig,
vector<MasterProxyInterface> proxies, vector<GrvProxyInterface> grvProxies,
vector<CommitProxyInterface> commitProxies, vector<GrvProxyInterface> grvProxies,
vector<ResolverInterface> resolvers, DBRecoveryCount recoveryCount,
vector<UID> priorCommittedLogServers) {
RegisterMasterRequest masterReq;
masterReq.id = self->myInterface.id();
masterReq.mi = self->myInterface.locality;
masterReq.logSystemConfig = logSystemConfig;
masterReq.masterProxies = proxies;
masterReq.commitProxies = commitProxies;
masterReq.grvProxies = grvProxies;
masterReq.resolvers = resolvers;
masterReq.recoveryCount = recoveryCount;
@ -536,14 +539,14 @@ ACTOR Future<Void> updateRegistration( Reference<MasterData> self, Reference<ILo
.detail("Logs", describe(logSystemConfig.tLogs));
if (!self->cstateUpdated.isSet()) {
wait(sendMasterRegistration(self.getPtr(), logSystemConfig, self->provisionalMasterProxies,
wait(sendMasterRegistration(self.getPtr(), logSystemConfig, self->provisionalCommitProxies,
self->provisionalGrvProxies, self->resolvers,
self->cstate.myDBState.recoveryCount,
self->cstate.prevDBState.getPriorCommittedLogServers()));
} else {
updateLogsKey = updateLogsValue(self, cx);
wait(sendMasterRegistration(self.getPtr(), logSystemConfig, self->masterProxies, self->grvProxies, self->resolvers,
self->cstate.myDBState.recoveryCount, vector<UID>()));
wait(sendMasterRegistration(self.getPtr(), logSystemConfig, self->commitProxies, self->grvProxies,
self->resolvers, self->cstate.myDBState.recoveryCount, vector<UID>()));
}
}
}
@ -551,14 +554,15 @@ ACTOR Future<Void> updateRegistration( Reference<MasterData> self, Reference<ILo
ACTOR Future<Standalone<CommitTransactionRef>> provisionalMaster( Reference<MasterData> parent, Future<Void> activate ) {
wait(activate);
// Register a fake master proxy (to be provided right here) to make ourselves available to clients
parent->provisionalMasterProxies = vector<MasterProxyInterface>(1);
parent->provisionalMasterProxies[0].provisional = true;
parent->provisionalMasterProxies[0].initEndpoints();
// Register a fake commit proxy (to be provided right here) to make ourselves available to clients
parent->provisionalCommitProxies = vector<CommitProxyInterface>(1);
parent->provisionalCommitProxies[0].provisional = true;
parent->provisionalCommitProxies[0].initEndpoints();
parent->provisionalGrvProxies = vector<GrvProxyInterface>(1);
parent->provisionalGrvProxies[0].provisional = true;
parent->provisionalGrvProxies[0].initEndpoints();
state Future<Void> waitMasterProxyFailure = waitFailureServer(parent->provisionalMasterProxies[0].waitFailure.getFuture());
state Future<Void> waitCommitProxyFailure =
waitFailureServer(parent->provisionalCommitProxies[0].waitFailure.getFuture());
state Future<Void> waitGrvProxyFailure = waitFailureServer(parent->provisionalGrvProxies[0].waitFailure.getFuture());
parent->registrationTrigger.trigger();
@ -567,8 +571,8 @@ ACTOR Future<Standalone<CommitTransactionRef>> provisionalMaster( Reference<Mast
state Optional<Value> metadataVersion = parent->txnStateStore->readValue(metadataVersionKey).get();
// We respond to a minimal subset of the master proxy protocol. Our sole purpose is to receive a single write-only transaction
// which might repair our configuration, and return it.
// We respond to a minimal subset of the commit proxy protocol. Our sole purpose is to receive a single write-only
// transaction which might repair our configuration, and return it.
loop choose {
when ( GetReadVersionRequest req = waitNext( parent->provisionalGrvProxies[0].getConsistentReadVersion.getFuture() ) ) {
if ( req.flags & GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY && parent->lastEpochEnd ) {
@ -580,7 +584,7 @@ ACTOR Future<Standalone<CommitTransactionRef>> provisionalMaster( Reference<Mast
} else
req.reply.send(Never()); // We can't perform causally consistent reads without recovering
}
when ( CommitTransactionRequest req = waitNext( parent->provisionalMasterProxies[0].commit.getFuture() ) ) {
when(CommitTransactionRequest req = waitNext(parent->provisionalCommitProxies[0].commit.getFuture())) {
req.reply.send(Never()); // don't reply (clients always get commit_unknown_result)
auto t = &req.transaction;
if (t->read_snapshot == parent->lastEpochEnd && //< So no transactions can fall between the read snapshot and the recovery transaction this (might) be merged with
@ -600,10 +604,11 @@ ACTOR Future<Standalone<CommitTransactionRef>> provisionalMaster( Reference<Mast
}
}
}
when ( GetKeyServerLocationsRequest req = waitNext( parent->provisionalMasterProxies[0].getKeyServersLocations.getFuture() ) ) {
when(GetKeyServerLocationsRequest req =
waitNext(parent->provisionalCommitProxies[0].getKeyServersLocations.getFuture())) {
req.reply.send(Never());
}
when ( wait( waitMasterProxyFailure ) ) { throw worker_removed(); }
when(wait(waitCommitProxyFailure)) { throw worker_removed(); }
when ( wait( waitGrvProxyFailure ) ) { throw worker_removed(); }
}
}
@ -634,8 +639,8 @@ ACTOR Future<vector<Standalone<CommitTransactionRef>>> recruitEverything( Refere
.detail("Status", RecoveryStatus::names[RecoveryStatus::recruiting_transaction_servers])
.detail("RequiredTLogs", self->configuration.tLogReplicationFactor)
.detail("DesiredTLogs", self->configuration.getDesiredLogs())
.detail("RequiredProxies", 1)
.detail("DesiredProxies", self->configuration.getDesiredProxies())
.detail("RequiredCommitProxies", 1)
.detail("DesiredCommitProxies", self->configuration.getDesiredCommitProxies())
.detail("RequiredGrvProxies", 1)
.detail("DesiredGrvProxies", self->configuration.getDesiredGrvProxies())
.detail("RequiredResolvers", 1)
@ -664,20 +669,20 @@ ACTOR Future<vector<Standalone<CommitTransactionRef>>> recruitEverything( Refere
self->backupWorkers.swap(recruits.backupWorkers);
TraceEvent("MasterRecoveryState", self->dbgid)
.detail("StatusCode", RecoveryStatus::initializing_transaction_servers)
.detail("Status", RecoveryStatus::names[RecoveryStatus::initializing_transaction_servers])
.detail("MasterProxies", recruits.masterProxies.size())
.detail("GrvProxies", recruits.grvProxies.size())
.detail("TLogs", recruits.tLogs.size())
.detail("Resolvers", recruits.resolvers.size())
.detail("BackupWorkers", self->backupWorkers.size())
.trackLatest("MasterRecoveryState");
.detail("StatusCode", RecoveryStatus::initializing_transaction_servers)
.detail("Status", RecoveryStatus::names[RecoveryStatus::initializing_transaction_servers])
.detail("CommitProxies", recruits.commitProxies.size())
.detail("GrvProxies", recruits.grvProxies.size())
.detail("TLogs", recruits.tLogs.size())
.detail("Resolvers", recruits.resolvers.size())
.detail("BackupWorkers", self->backupWorkers.size())
.trackLatest("MasterRecoveryState");
// Actually, newSeedServers does both the recruiting and initialization of the seed servers; so if this is a brand new database we are sort of lying that we are
// past the recruitment phase. In a perfect world we would split that up so that the recruitment part happens above (in parallel with recruiting the transaction servers?).
wait( newSeedServers( self, recruits, seedServers ) );
state vector<Standalone<CommitTransactionRef>> confChanges;
wait(newMasterProxies(self, recruits) && newGrvProxies(self, recruits) && newResolvers(self, recruits) &&
wait(newCommitProxies(self, recruits) && newGrvProxies(self, recruits) && newResolvers(self, recruits) &&
newTLogServers(self, recruits, oldLogSystem, &confChanges));
return confChanges;
}
@ -803,7 +808,7 @@ ACTOR Future<Void> sendInitialCommitToResolvers( Reference<MasterData> self ) {
state int64_t dataOutstanding = 0;
state std::vector<Endpoint> endpoints;
for(auto& it : self->masterProxies) {
for (auto& it : self->commitProxies) {
endpoints.push_back(it.txnState.getEndpoint());
}
@ -973,9 +978,9 @@ ACTOR Future<Void> recoverFrom( Reference<MasterData> self, Reference<ILogSystem
ACTOR Future<Void> getVersion(Reference<MasterData> self, GetCommitVersionRequest req) {
state Span span("M:getVersion"_loc, { req.spanContext });
state std::map<UID, ProxyVersionReplies>::iterator proxyItr = self->lastProxyVersionReplies.find(req.requestingProxy); // lastProxyVersionReplies never changes
state std::map<UID, CommitProxyVersionReplies>::iterator proxyItr = self->lastCommitProxyVersionReplies.find(req.requestingProxy); // lastCommitProxyVersionReplies never changes
if (proxyItr == self->lastProxyVersionReplies.end()) {
if (proxyItr == self->lastCommitProxyVersionReplies.end()) {
// Request from invalid proxy (e.g. from duplicate recruitment request)
req.reply.send(Never());
return Void();
@ -1042,8 +1047,7 @@ ACTOR Future<Void> getVersion(Reference<MasterData> self, GetCommitVersionReques
ACTOR Future<Void> provideVersions(Reference<MasterData> self) {
state ActorCollection versionActors(false);
for (auto& p : self->masterProxies)
self->lastProxyVersionReplies[p.id()] = ProxyVersionReplies();
for (auto& p : self->commitProxies) self->lastCommitProxyVersionReplies[p.id()] = CommitProxyVersionReplies();
loop {
choose {
@ -1183,8 +1187,7 @@ ACTOR Future<Void> resolutionBalancing(Reference<MasterData> self) {
// TraceEvent("KeyResolver").detail("Range", it.range()).detail("Value", it.value());
self->resolverChangesVersion = self->version + 1;
for (auto& p : self->masterProxies)
self->resolverNeedingChanges.insert(p.id());
for (auto& p : self->commitProxies) self->resolverNeedingChanges.insert(p.id());
self->resolverChanges.set(movedRanges);
} catch( Error&e ) {
if(e.code() != error_code_operation_failed)
@ -1199,7 +1202,7 @@ static std::set<int> const& normalMasterErrors() {
if (s.empty()) {
s.insert( error_code_tlog_stopped );
s.insert( error_code_master_tlog_failed );
s.insert( error_code_master_proxy_failed );
s.insert(error_code_commit_proxy_failed);
s.insert( error_code_grv_proxy_failed );
s.insert( error_code_master_resolver_failed );
s.insert( error_code_master_backup_worker_failed );
@ -1544,8 +1547,8 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
recoverAndEndEpoch.cancel();
ASSERT(self->masterProxies.size() <= self->configuration.getDesiredProxies());
ASSERT(self->masterProxies.size() >= 1);
ASSERT(self->commitProxies.size() <= self->configuration.getDesiredCommitProxies());
ASSERT(self->commitProxies.size() >= 1);
ASSERT(self->grvProxies.size() <= self->configuration.getDesiredGrvProxies());
ASSERT(self->grvProxies.size() >= 1);
ASSERT( self->resolvers.size() <= self->configuration.getDesiredResolvers() );
@ -1620,10 +1623,10 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
tr.read_snapshot = self->recoveryTransactionVersion; // lastEpochEnd would make more sense, but isn't in the initial window of the resolver(s)
TraceEvent("MasterRecoveryCommit", self->dbgid);
state Future<ErrorOr<CommitID>> recoveryCommit = self->masterProxies[0].commit.tryGetReply(recoveryCommitRequest);
state Future<ErrorOr<CommitID>> recoveryCommit = self->commitProxies[0].commit.tryGetReply(recoveryCommitRequest);
self->addActor.send( self->logSystem->onError() );
self->addActor.send( waitResolverFailure( self->resolvers ) );
self->addActor.send( waitProxyFailure( self->masterProxies) );
self->addActor.send( waitCommitProxyFailure(self->commitProxies));
self->addActor.send( waitGrvProxyFailure( self->grvProxies ) );
self->addActor.send( provideVersions(self) );
self->addActor.send( serveLiveCommittedVersion(self) );
@ -1758,7 +1761,7 @@ ACTOR Future<Void> masterServer( MasterInterface mi, Reference<AsyncVar<ServerDB
}
TEST(err.code() == error_code_master_tlog_failed); // Master: terminated because of a tLog failure
TEST(err.code() == error_code_master_proxy_failed); // Master: terminated because of a proxy failure
TEST(err.code() == error_code_commit_proxy_failed); // Master: terminated because of a commit proxy failure
TEST(err.code() == error_code_grv_proxy_failed); // Master: terminated because of a GRV proxy failure
TEST(err.code() == error_code_master_resolver_failed); // Master: terminated because of a resolver failure
TEST(err.code() == error_code_master_backup_worker_failed); // Master: terminated because of a backup worker failure

View File

@ -32,7 +32,7 @@
#include "fdbclient/Atomic.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/KeyRangeMap.h"
#include "fdbclient/MasterProxyInterface.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/Notified.h"
#include "fdbclient/StatusClient.h"
@ -3974,9 +3974,13 @@ ACTOR Future<Void> replaceInterface( StorageServer* self, StorageServerInterface
loop {
state Future<Void> infoChanged = self->db->onChange();
state Reference<ProxyInfo> proxies( new ProxyInfo(self->db->get().client.masterProxies) );
state Reference<CommitProxyInfo> commitProxies(new CommitProxyInfo(self->db->get().client.commitProxies));
choose {
when( GetStorageServerRejoinInfoReply _rep = wait( proxies->size() ? basicLoadBalance( proxies, &MasterProxyInterface::getStorageServerRejoinInfo, GetStorageServerRejoinInfoRequest(ssi.id(), ssi.locality.dcId()) ) : Never() ) ) {
when(GetStorageServerRejoinInfoReply _rep =
wait(commitProxies->size()
? basicLoadBalance(commitProxies, &CommitProxyInterface::getStorageServerRejoinInfo,
GetStorageServerRejoinInfoRequest(ssi.id(), ssi.locality.dcId()))
: Never())) {
state GetStorageServerRejoinInfoReply rep = _rep;
try {
tr.reset();

View File

@ -114,13 +114,13 @@ ACTOR Future<std::vector<Endpoint>> broadcastDBInfoRequest(UpdateServerDBInfoReq
}
ACTOR static Future<Void> extractClientInfo( Reference<AsyncVar<ServerDBInfo>> db, Reference<AsyncVar<ClientDBInfo>> info ) {
state std::vector<UID> lastProxyUIDs;
state std::vector<MasterProxyInterface> lastProxies;
state std::vector<UID> lastCommitProxyUIDs;
state std::vector<CommitProxyInterface> lastCommitProxies;
state std::vector<UID> lastGrvProxyUIDs;
state std::vector<GrvProxyInterface> lastGrvProxies;
loop {
ClientDBInfo ni = db->get().client;
shrinkProxyList(ni, lastProxyUIDs, lastProxies, lastGrvProxyUIDs, lastGrvProxies);
shrinkProxyList(ni, lastCommitProxyUIDs, lastCommitProxies, lastGrvProxyUIDs, lastGrvProxies);
info->set( ni );
wait( db->onChange() );
}
@ -994,7 +994,7 @@ ACTOR Future<Void> workerServer(
DUMPTOKEN(recruited.clientInterface.profiler);
DUMPTOKEN(recruited.tLog);
DUMPTOKEN(recruited.master);
DUMPTOKEN(recruited.masterProxy);
DUMPTOKEN(recruited.commitProxy);
DUMPTOKEN(recruited.grvProxy);
DUMPTOKEN(recruited.resolver);
DUMPTOKEN(recruited.storage);
@ -1368,15 +1368,15 @@ ACTOR Future<Void> workerServer(
} else
forwardPromise( req.reply, storageCache.get( req.reqId ) );
}
when( InitializeMasterProxyRequest req = waitNext(interf.masterProxy.getFuture()) ) {
MasterProxyInterface recruited;
when(InitializeCommitProxyRequest req = waitNext(interf.commitProxy.getFuture())) {
CommitProxyInterface recruited;
recruited.processId = locality.processId();
recruited.provisional = false;
recruited.initEndpoints();
std::map<std::string, std::string> details;
details["ForMaster"] = req.master.id().shortString();
startRole( Role::MASTER_PROXY, recruited.id(), interf.id(), details );
startRole(Role::COMMIT_PROXY, recruited.id(), interf.id(), details);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.getConsistentReadVersion);
@ -1385,9 +1385,10 @@ ACTOR Future<Void> workerServer(
DUMPTOKEN(recruited.waitFailure);
DUMPTOKEN(recruited.txnState);
//printf("Recruited as masterProxyServer\n");
errorForwarders.add( zombie(recruited, forwardError( errors, Role::MASTER_PROXY, recruited.id(),
masterProxyServer( recruited, req, dbInfo, whitelistBinPaths ) ) ) );
// printf("Recruited as commitProxyServer\n");
errorForwarders.add(
zombie(recruited, forwardError(errors, Role::COMMIT_PROXY, recruited.id(),
commitProxyServer(recruited, req, dbInfo, whitelistBinPaths))));
req.reply.send(recruited);
}
when( InitializeGrvProxyRequest req = waitNext(interf.grvProxy.getFuture()) ) {
@ -1857,7 +1858,7 @@ const Role Role::WORKER("Worker", "WK", false);
const Role Role::STORAGE_SERVER("StorageServer", "SS");
const Role Role::TRANSACTION_LOG("TLog", "TL");
const Role Role::SHARED_TRANSACTION_LOG("SharedTLog", "SL", false);
const Role Role::MASTER_PROXY("MasterProxyServer", "MP");
const Role Role::COMMIT_PROXY("CommitProxyServer", "CP");
const Role Role::GRV_PROXY("GrvProxyServer", "GP");
const Role Role::MASTER("MasterServer", "MS");
const Role Role::RESOLVER("Resolver", "RV");

View File

@ -302,7 +302,7 @@ struct ConfigureDatabaseWorkload : TestWorkload {
config += generateRegions();
if (deterministicRandom()->random01() < 0.5) config += " logs=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5) config += " proxies=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5) config += " commit_proxies=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5)
config += " grv_proxies=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5) config += " resolvers=" + format("%d", randomRoleNumber());

View File

@ -365,9 +365,9 @@ struct ConsistencyCheckWorkload : TestWorkload
}
}
//Get a list of storage servers from the master and compares them with the TLogs.
//If this is a quiescent check, then each master proxy needs to respond, otherwise only one needs to respond.
//Returns false if there is a failure (in this case, keyServersPromise will never be set)
// Get a list of storage servers from the master and compares them with the TLogs.
// If this is a quiescent check, then each commit proxy needs to respond, otherwise only one needs to respond.
// Returns false if there is a failure (in this case, keyServersPromise will never be set)
ACTOR Future<bool> getKeyServers(Database cx, ConsistencyCheckWorkload *self, Promise<std::vector<std::pair<KeyRange, vector<StorageServerInterface>>>> keyServersPromise)
{
state std::vector<std::pair<KeyRange, vector<StorageServerInterface>>> keyServers;
@ -380,13 +380,14 @@ struct ConsistencyCheckWorkload : TestWorkload
state Span span(deterministicRandom()->randomUniqueID(), "WL:ConsistencyCheck"_loc);
while (begin < end) {
state Reference<ProxyInfo> proxyInfo = wait(cx->getMasterProxiesFuture(false));
state Reference<CommitProxyInfo> commitProxyInfo = wait(cx->getCommitProxiesFuture(false));
keyServerLocationFutures.clear();
for (int i = 0; i < proxyInfo->size(); i++)
for (int i = 0; i < commitProxyInfo->size(); i++)
keyServerLocationFutures.push_back(
proxyInfo->get(i, &MasterProxyInterface::getKeyServersLocations)
commitProxyInfo->get(i, &CommitProxyInterface::getKeyServersLocations)
.getReplyUnlessFailedFor(
GetKeyServerLocationsRequest(span.context, begin, end, limitKeyServers, false, Arena()), 2, 0));
GetKeyServerLocationsRequest(span.context, begin, end, limitKeyServers, false, Arena()), 2,
0));
state bool keyServersInsertedForThisIteration = false;
choose {
@ -399,8 +400,9 @@ struct ConsistencyCheckWorkload : TestWorkload
//If performing quiescent check, then all master proxies should be reachable. Otherwise, only one needs to be reachable
if (self->performQuiescentChecks && !shards.present())
{
TraceEvent("ConsistencyCheck_MasterProxyUnavailable").detail("MasterProxyID", proxyInfo->getId(i));
self->testFailure("Master proxy unavailable");
TraceEvent("ConsistencyCheck_CommitProxyUnavailable")
.detail("CommitProxyID", commitProxyInfo->getId(i));
self->testFailure("Commit proxy unavailable");
return false;
}
@ -1461,11 +1463,20 @@ struct ConsistencyCheckWorkload : TestWorkload
return false;
}
// Check proxy
ProcessClass::Fitness bestProxyFitness = getBestAvailableFitness(dcToNonExcludedClassTypes[masterDcId], ProcessClass::Proxy);
for (const auto& masterProxy : db.client.masterProxies) {
if (!nonExcludedWorkerProcessMap.count(masterProxy.address()) || nonExcludedWorkerProcessMap[masterProxy.address()].processClass.machineClassFitness(ProcessClass::Proxy) != bestProxyFitness) {
TraceEvent("ConsistencyCheck_ProxyNotBest").detail("BestProxyFitness", bestProxyFitness).detail("ExistingMasterProxyFitness", nonExcludedWorkerProcessMap.count(masterProxy.address()) ? nonExcludedWorkerProcessMap[masterProxy.address()].processClass.machineClassFitness(ProcessClass::Proxy) : -1);
// Check commit proxy
ProcessClass::Fitness bestCommitProxyFitness =
getBestAvailableFitness(dcToNonExcludedClassTypes[masterDcId], ProcessClass::CommitProxy);
for (const auto& commitProxy : db.client.commitProxies) {
if (!nonExcludedWorkerProcessMap.count(commitProxy.address()) ||
nonExcludedWorkerProcessMap[commitProxy.address()].processClass.machineClassFitness(
ProcessClass::CommitProxy) != bestCommitProxyFitness) {
TraceEvent("ConsistencyCheck_CommitProxyNotBest")
.detail("BestCommitProxyFitness", bestCommitProxyFitness)
.detail("ExistingCommitProxyFitness",
nonExcludedWorkerProcessMap.count(commitProxy.address())
? nonExcludedWorkerProcessMap[commitProxy.address()].processClass.machineClassFitness(
ProcessClass::CommitProxy)
: -1);
return false;
}
}

View File

@ -62,13 +62,13 @@ struct RollbackWorkload : TestWorkload {
ACTOR Future<Void> simulateFailure( Database cx, RollbackWorkload* self ) {
state ServerDBInfo system = self->dbInfo->get();
auto tlogs = system.logSystemConfig.allPresentLogs();
if( tlogs.empty() || system.client.masterProxies.empty() ) {
if (tlogs.empty() || system.client.commitProxies.empty()) {
TraceEvent(SevInfo, "UnableToTriggerRollback").detail("Reason", "No tlogs in System Map");
return Void();
}
state MasterProxyInterface proxy = deterministicRandom()->randomChoice( system.client.masterProxies);
state CommitProxyInterface proxy = deterministicRandom()->randomChoice(system.client.commitProxies);
int utIndex = deterministicRandom()->randomInt(0, tlogs.size());
state NetworkAddress uncloggedTLog = tlogs[utIndex].address();
@ -81,8 +81,8 @@ struct RollbackWorkload : TestWorkload {
}
TraceEvent("AttemptingToTriggerRollback")
.detail("Proxy", proxy.address())
.detail("UncloggedTLog", uncloggedTLog);
.detail("CommitProxy", proxy.address())
.detail("UncloggedTLog", uncloggedTLog);
for (int t = 0; t < tlogs.size(); t++) {
if (t != utIndex) {

View File

@ -87,19 +87,17 @@ struct TargetedKillWorkload : TestWorkload {
NetworkAddress machine;
if( self->machineToKill == "master" ) {
machine = self->dbInfo->get().master.address();
}
else if( self->machineToKill == "masterproxy" ) {
auto proxies = cx->getMasterProxies(false);
int o = deterministicRandom()->randomInt(0, proxies->size());
for( int i = 0; i < proxies->size(); i++) {
MasterProxyInterface mpi = proxies->getInterface(o);
} else if (self->machineToKill == "commitproxy") {
auto commitProxies = cx->getCommitProxies(false);
int o = deterministicRandom()->randomInt(0, commitProxies->size());
for( int i = 0; i < commitProxies->size(); i++) {
CommitProxyInterface mpi = commitProxies->getInterface(o);
machine = mpi.address();
if(machine != self->dbInfo->get().clusterInterface.getWorkers.getEndpoint().getPrimaryAddress())
break;
o = ++o%proxies->size();
o = ++o%commitProxies->size();
}
}
else if( self->machineToKill == "grvproxy" ) {
} else if (self->machineToKill == "grvproxy") {
auto grvProxies = cx->getGrvProxies(false);
int o = deterministicRandom()->randomInt(0, grvProxies->size());
for( int i = 0; i < grvProxies->size(); i++) {
@ -109,8 +107,7 @@ struct TargetedKillWorkload : TestWorkload {
break;
o = ++o%grvProxies->size();
}
}
else if( self->machineToKill == "tlog" ) {
} else if (self->machineToKill == "tlog") {
auto tlogs = self->dbInfo->get().logSystemConfig.allPresentLogs();
int o = deterministicRandom()->randomInt(0, tlogs.size());
for( int i = 0; i < tlogs.size(); i++) {
@ -120,8 +117,8 @@ struct TargetedKillWorkload : TestWorkload {
break;
o = ++o%tlogs.size();
}
}
else if( self->machineToKill == "storage" || self->machineToKill == "ss" || self->machineToKill == "storageserver" ) {
} else if (self->machineToKill == "storage" || self->machineToKill == "ss" ||
self->machineToKill == "storageserver") {
int o = deterministicRandom()->randomInt(0,storageServers.size());
for( int i = 0; i < storageServers.size(); i++) {
StorageServerInterface ssi = storageServers[o];
@ -130,8 +127,7 @@ struct TargetedKillWorkload : TestWorkload {
break;
o = ++o%storageServers.size();
}
}
else if( self->machineToKill == "clustercontroller" || self->machineToKill == "cc" ) {
} else if (self->machineToKill == "clustercontroller" || self->machineToKill == "cc") {
machine = self->dbInfo->get().clusterInterface.getWorkers.getEndpoint().getPrimaryAddress();
}

View File

@ -65,7 +65,7 @@ ERROR( database_locked, 1038, "Database is locked" )
ERROR( cluster_version_changed, 1039, "The protocol version of the cluster has changed" )
ERROR( external_client_already_loaded, 1040, "External client has already been loaded" )
ERROR( lookup_failed, 1041, "DNS lookup failed" )
ERROR( proxy_memory_limit_exceeded, 1042, "Proxy commit memory limit exceeded" )
ERROR( proxy_memory_limit_exceeded, 1042, "CommitProxy commit memory limit exceeded" )
ERROR( shutdown_in_progress, 1043, "Operation no longer supported due to shutdown" )
ERROR( serialization_failed, 1044, "Failed to deserialize an object" )
ERROR( connection_unreferenced, 1048, "No peer references for connection" )
@ -89,12 +89,12 @@ ERROR( master_tlog_failed, 1205, "Master terminating because a TLog failed" )
ERROR( worker_recovery_failed, 1206, "Recovery of a worker process failed" )
ERROR( please_reboot, 1207, "Reboot of server process requested" )
ERROR( please_reboot_delete, 1208, "Reboot of server process requested, with deletion of state" )
ERROR( master_proxy_failed, 1209, "Master terminating because a Proxy failed" )
ERROR( commit_proxy_failed, 1209, "Master terminating because a Commit CommitProxy failed" )
ERROR( master_resolver_failed, 1210, "Master terminating because a Resolver failed" )
ERROR( server_overloaded, 1211, "Server is under too much load and cannot respond" )
ERROR( master_backup_worker_failed, 1212, "Master terminating because a backup worker failed")
ERROR( tag_throttled, 1213, "Transaction tag is being throttled" )
ERROR( grv_proxy_failed, 1214, "Master terminating because a GRV Proxy failed" )
ERROR( grv_proxy_failed, 1214, "Master terminating because a GRV CommitProxy failed" )
// 15xx Platform errors
ERROR( platform_error, 1500, "Platform error" )

View File

@ -223,7 +223,7 @@
"roles" : [
{
"id" : "f29c4c66f293d1b1",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "44950eb0b3d862c0",
@ -264,7 +264,7 @@
"roles" : [
{
"id" : "175f5bed1f306159",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "e583f98ea591c52a",
@ -342,7 +342,7 @@
"roles" : [
{
"id" : "c97dc5f2e372921b",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "bbb368082d582712",

View File

@ -172,7 +172,7 @@
},
{
"id" : "066a9f0089483a5f",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "d0809246b42910f8",
@ -213,7 +213,7 @@
"roles" : [
{
"id" : "3fc3c3d9c9e3349d",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "656697882cc0e76e",
@ -254,7 +254,7 @@
"roles" : [
{
"id" : "586d54237f6bf4c7",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "09a94118dc82393a",

View File

@ -130,7 +130,7 @@
},
{
"id" : "9159f5bae811936d",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "4ef3ec0982dab9fe",
@ -171,7 +171,7 @@
"roles" : [
{
"id" : "9d158fb102da025f",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "02fe9302ba499227",

View File

@ -140,7 +140,7 @@
},
{
"id" : "00e48601e43045c9",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "0df71fd71bbc14ee",
@ -181,7 +181,7 @@
},
{
"id" : "07b3f5362cfec06b",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "bb25c74aca56ccf7",
@ -222,7 +222,7 @@
"roles" : [
{
"id" : "361d515d63a595ad",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "4022c037e26868ae",

View File

@ -118,7 +118,7 @@
},
{
"id" : "4989d9993ee37183",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "009709c84d97df4d",

View File

@ -113,7 +113,7 @@
},
{
"id" : "4989d9993ee37183",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "009709c84d97df4d",

View File

@ -154,7 +154,7 @@
},
{
"id" : "ae9fe51db979dfd1",
"role" : "proxy"
"role" : "commit_proxy"
}
],
"version" : "3.0.0-PRERELEASE"

View File

@ -121,7 +121,7 @@
},
{
"id" : "20beaadaa554dee3",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "f0a33233db8e5f67",
@ -143,7 +143,7 @@
"description" : "Recruiting new transaction servers.",
"name" : "recruiting_transaction_servers",
"required_logs" : 3,
"required_proxies" : 1,
"required_commit_proxies" : 1,
"required_grv_proxies" : 1,
"required_resolvers" : 1
},

View File

@ -2,6 +2,6 @@ Using cluster file `tcf/separatecoordinator.cluster'.
Recruiting new transaction servers.
Need at least 3 log servers, 1 proxies, 1 grv proxies and 1 resolvers.
Need at least 3 log servers, 1 commit proxies, 1 grv proxies and 1 resolvers.
Have 1 processes on 1 machines.

View File

@ -123,7 +123,7 @@
},
{
"id" : "242e27cd68b21c05",
"role" : "proxy"
"role" : "commit_proxy"
},
{
"id" : "faf07cf91f0ab29d",