Merge remote-tracking branch 'origin/master' into paxos-config-db
This commit is contained in:
commit
3418c20867
|
@ -157,6 +157,7 @@ if(CMAKE_SYSTEM_NAME STREQUAL "FreeBSD")
|
|||
endif()
|
||||
|
||||
include(CompileBoost)
|
||||
include(GetMsgpack)
|
||||
add_subdirectory(flow)
|
||||
add_subdirectory(fdbrpc)
|
||||
add_subdirectory(fdbclient)
|
||||
|
|
|
@ -189,10 +189,19 @@ int cleanup(FDBTransaction* transaction, mako_args_t* args) {
|
|||
free(prefixstr);
|
||||
len += 1;
|
||||
|
||||
retryTxn:
|
||||
clock_gettime(CLOCK_MONOTONIC_COARSE, &timer_start);
|
||||
|
||||
fdb_transaction_clear_range(transaction, (uint8_t*)beginstr, len + 1, (uint8_t*)endstr, len + 1);
|
||||
if (commit_transaction(transaction) != FDB_SUCCESS)
|
||||
goto failExit;
|
||||
switch (commit_transaction(transaction)) {
|
||||
case (FDB_SUCCESS):
|
||||
break;
|
||||
case (FDB_ERROR_RETRY):
|
||||
fdb_transaction_reset(transaction);
|
||||
goto retryTxn;
|
||||
default:
|
||||
goto failExit;
|
||||
}
|
||||
|
||||
fdb_transaction_reset(transaction);
|
||||
clock_gettime(CLOCK_MONOTONIC_COARSE, &timer_end);
|
||||
|
@ -308,11 +317,19 @@ int populate(FDBTransaction* transaction,
|
|||
|
||||
/* commit every 100 inserts (default) */
|
||||
if (i % args->txnspec.ops[OP_INSERT][OP_COUNT] == 0) {
|
||||
retryTxn:
|
||||
if (stats->xacts % args->sampling == 0) {
|
||||
clock_gettime(CLOCK_MONOTONIC, &timer_start_commit);
|
||||
}
|
||||
if (commit_transaction(transaction) != FDB_SUCCESS)
|
||||
goto failExit;
|
||||
|
||||
switch (commit_transaction(transaction)) {
|
||||
case (FDB_SUCCESS):
|
||||
break;
|
||||
case (FDB_ERROR_RETRY):
|
||||
goto retryTxn;
|
||||
default:
|
||||
goto failExit;
|
||||
}
|
||||
|
||||
/* xact latency stats */
|
||||
if (stats->xacts % args->sampling == 0) {
|
||||
|
@ -337,20 +354,36 @@ int populate(FDBTransaction* transaction,
|
|||
xacts++; /* for throttling */
|
||||
}
|
||||
}
|
||||
|
||||
if (stats->xacts % args->sampling == 0) {
|
||||
clock_gettime(CLOCK_MONOTONIC, &timer_start_commit);
|
||||
}
|
||||
if (commit_transaction(transaction) != FDB_SUCCESS)
|
||||
goto failExit;
|
||||
|
||||
/* xact latency stats */
|
||||
if (stats->xacts % args->sampling == 0) {
|
||||
clock_gettime(CLOCK_MONOTONIC, &timer_per_xact_end);
|
||||
update_op_lat_stats(
|
||||
&timer_start_commit, &timer_per_xact_end, OP_COMMIT, stats, block, elem_size, is_memory_allocated);
|
||||
update_op_lat_stats(
|
||||
&timer_per_xact_start, &timer_per_xact_end, OP_TRANSACTION, stats, block, elem_size, is_memory_allocated);
|
||||
time_t start_time_sec, current_time_sec;
|
||||
time(&start_time_sec);
|
||||
int is_committed = false;
|
||||
// will hit FDB_ERROR_RETRY if running mako with multi-version client
|
||||
while (!is_committed) {
|
||||
if (stats->xacts % args->sampling == 0) {
|
||||
clock_gettime(CLOCK_MONOTONIC, &timer_start_commit);
|
||||
}
|
||||
int rc;
|
||||
if ((rc = commit_transaction(transaction) != FDB_SUCCESS)) {
|
||||
if (rc == FDB_ERROR_RETRY) {
|
||||
time(¤t_time_sec);
|
||||
if (difftime(current_time_sec, start_time_sec) > 5) {
|
||||
goto failExit;
|
||||
} else {
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
goto failExit;
|
||||
}
|
||||
}
|
||||
is_committed = true;
|
||||
/* xact latency stats */
|
||||
if (stats->xacts % args->sampling == 0) {
|
||||
clock_gettime(CLOCK_MONOTONIC, &timer_per_xact_end);
|
||||
update_op_lat_stats(
|
||||
&timer_start_commit, &timer_per_xact_end, OP_COMMIT, stats, block, elem_size, is_memory_allocated);
|
||||
update_op_lat_stats(
|
||||
&timer_per_xact_start, &timer_per_xact_end, OP_TRANSACTION, stats, block, elem_size, is_memory_allocated);
|
||||
}
|
||||
}
|
||||
|
||||
clock_gettime(CLOCK_MONOTONIC, &timer_end);
|
||||
|
|
|
@ -332,15 +332,16 @@ def transaction(logger):
|
|||
output7 = run_fdbcli_command('get', 'key')
|
||||
assert output7 == "`key': not found"
|
||||
|
||||
def get_fdb_process_addresses():
|
||||
def get_fdb_process_addresses(logger):
|
||||
# get all processes' network addresses
|
||||
output = run_fdbcli_command('kill')
|
||||
logger.debug(output)
|
||||
# except the first line, each line is one process
|
||||
addresses = output.split('\n')[1:]
|
||||
assert len(addresses) == process_number
|
||||
return addresses
|
||||
|
||||
@enable_logging()
|
||||
@enable_logging(logging.DEBUG)
|
||||
def coordinators(logger):
|
||||
# we should only have one coordinator for now
|
||||
output1 = run_fdbcli_command('coordinators')
|
||||
|
@ -354,7 +355,7 @@ def coordinators(logger):
|
|||
assert coordinator_list[0]['address'] == coordinators
|
||||
# verify the cluster description
|
||||
assert get_value_from_status_json(True, 'cluster', 'connection_string').startswith('{}:'.format(cluster_description))
|
||||
addresses = get_fdb_process_addresses()
|
||||
addresses = get_fdb_process_addresses(logger)
|
||||
# set all 5 processes as coordinators and update the cluster description
|
||||
new_cluster_description = 'a_simple_description'
|
||||
run_fdbcli_command('coordinators', *addresses, 'description={}'.format(new_cluster_description))
|
||||
|
@ -365,11 +366,12 @@ def coordinators(logger):
|
|||
# auto change should go back to 1 coordinator
|
||||
run_fdbcli_command('coordinators', 'auto')
|
||||
assert len(get_value_from_status_json(True, 'client', 'coordinators', 'coordinators')) == 1
|
||||
wait_for_database_available(logger)
|
||||
|
||||
@enable_logging()
|
||||
@enable_logging(logging.DEBUG)
|
||||
def exclude(logger):
|
||||
# get all processes' network addresses
|
||||
addresses = get_fdb_process_addresses()
|
||||
addresses = get_fdb_process_addresses(logger)
|
||||
logger.debug("Cluster processes: {}".format(' '.join(addresses)))
|
||||
# There should be no excluded process for now
|
||||
no_excluded_process_output = 'There are currently no servers or localities excluded from the database.'
|
||||
|
@ -377,16 +379,28 @@ def exclude(logger):
|
|||
assert no_excluded_process_output in output1
|
||||
# randomly pick one and exclude the process
|
||||
excluded_address = random.choice(addresses)
|
||||
# If we see "not enough space" error, use FORCE option to proceed
|
||||
# this should be a safe operation as we do not need any storage space for the test
|
||||
force = False
|
||||
# sometimes we need to retry the exclude
|
||||
while True:
|
||||
logger.debug("Excluding process: {}".format(excluded_address))
|
||||
error_message = run_fdbcli_command_and_get_error('exclude', excluded_address)
|
||||
if force:
|
||||
error_message = run_fdbcli_command_and_get_error('exclude', 'FORCE', excluded_address)
|
||||
else:
|
||||
error_message = run_fdbcli_command_and_get_error('exclude', excluded_address)
|
||||
if error_message == 'WARNING: {} is a coordinator!'.format(excluded_address):
|
||||
# exclude coordinator will print the warning, verify the randomly selected process is the coordinator
|
||||
coordinator_list = get_value_from_status_json(True, 'client', 'coordinators', 'coordinators')
|
||||
assert len(coordinator_list) == 1
|
||||
assert coordinator_list[0]['address'] == excluded_address
|
||||
break
|
||||
elif 'ERROR: This exclude may cause the total free space in the cluster to drop below 10%.' in error_message:
|
||||
# exclude the process may cause the free space not enough
|
||||
# use FORCE option to ignore it and proceed
|
||||
assert not force
|
||||
force = True
|
||||
logger.debug("Use FORCE option to exclude the process")
|
||||
elif not error_message:
|
||||
break
|
||||
else:
|
||||
|
@ -400,6 +414,7 @@ def exclude(logger):
|
|||
# check the include is successful
|
||||
output4 = run_fdbcli_command('exclude')
|
||||
assert no_excluded_process_output in output4
|
||||
wait_for_database_available(logger)
|
||||
|
||||
# read the system key 'k', need to enable the option first
|
||||
def read_system_key(k):
|
||||
|
@ -427,6 +442,13 @@ def throttle(logger):
|
|||
assert enable_flag == "`0'"
|
||||
# TODO : test manual throttling, not easy to do now
|
||||
|
||||
def wait_for_database_available(logger):
|
||||
# sometimes the change takes some time to have effect and the database can be unavailable at that time
|
||||
# this is to wait until the database is available again
|
||||
while not get_value_from_status_json(True, 'client', 'database_status', 'available'):
|
||||
logger.debug("Database unavailable for now, wait for one second")
|
||||
time.sleep(1)
|
||||
|
||||
if __name__ == '__main__':
|
||||
# fdbcli_tests.py <path_to_fdbcli_binary> <path_to_fdb_cluster_file> <process_number>
|
||||
assert len(sys.argv) == 4, "Please pass arguments: <path_to_fdbcli_binary> <path_to_fdb_cluster_file> <process_number>"
|
||||
|
@ -450,10 +472,7 @@ if __name__ == '__main__':
|
|||
throttle()
|
||||
else:
|
||||
assert process_number > 1, "Process number should be positive"
|
||||
# the kill command which used to list processes seems to not work as expected sometime
|
||||
# which makes the test flaky.
|
||||
# We need to figure out the reason and then re-enable these tests
|
||||
#coordinators()
|
||||
#exclude()
|
||||
coordinators()
|
||||
exclude()
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,20 @@
|
|||
find_package(msgpack 3.3.0 EXACT QUIET CONFIG)
|
||||
|
||||
add_library(msgpack INTERFACE)
|
||||
|
||||
if(msgpack_FOUND)
|
||||
target_link_libraries(msgpack INTERFACE msgpackc-cxx)
|
||||
else()
|
||||
include(ExternalProject)
|
||||
ExternalProject_add(msgpackProject
|
||||
URL "https://github.com/msgpack/msgpack-c/releases/download/cpp-3.3.0/msgpack-3.3.0.tar.gz"
|
||||
URL_HASH SHA256=6e114d12a5ddb8cb11f669f83f32246e484a8addd0ce93f274996f1941c1f07b
|
||||
CONFIGURE_COMMAND ""
|
||||
BUILD_COMMAND ""
|
||||
INSTALL_COMMAND ""
|
||||
)
|
||||
|
||||
ExternalProject_Get_property(msgpackProject SOURCE_DIR)
|
||||
target_include_directories(msgpack SYSTEM INTERFACE "${SOURCE_DIR}/include")
|
||||
add_dependencies(msgpack msgpackProject)
|
||||
endif()
|
|
@ -506,7 +506,7 @@ ACTOR Future<Void> decode_logs(DecodeParams params) {
|
|||
wait(process_file(container, logs[idx], uid, params));
|
||||
idx++;
|
||||
}
|
||||
TraceEvent("DecodeDone", uid);
|
||||
TraceEvent("DecodeDone", uid).log();
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,394 @@
|
|||
/*
|
||||
* ActorLineageProfiler.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "flow/flow.h"
|
||||
#include "flow/singleton.h"
|
||||
#include "fdbrpc/IAsyncFile.h"
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
#include "fdbclient/NameLineage.h"
|
||||
#include <msgpack.hpp>
|
||||
#include <memory>
|
||||
#include <typeindex>
|
||||
#include <boost/endian/conversion.hpp>
|
||||
#include <boost/asio.hpp>
|
||||
|
||||
using namespace std::literals;
|
||||
|
||||
class Packer : public msgpack::packer<msgpack::sbuffer> {
|
||||
struct visitor_t {
|
||||
using VisitorMap = std::unordered_map<std::type_index, std::function<void(std::any const&, Packer& packer)>>;
|
||||
VisitorMap visitorMap;
|
||||
|
||||
template <class T>
|
||||
static void any_visitor(std::any const& val, Packer& packer) {
|
||||
const T& v = std::any_cast<const T&>(val);
|
||||
packer.pack(v);
|
||||
}
|
||||
|
||||
template <class... Args>
|
||||
struct populate_visitor_map;
|
||||
template <class Head, class... Tail>
|
||||
struct populate_visitor_map<Head, Tail...> {
|
||||
static void populate(VisitorMap& map) {
|
||||
map.emplace(std::type_index(typeid(Head)), any_visitor<Head>);
|
||||
populate_visitor_map<Tail...>::populate(map);
|
||||
}
|
||||
};
|
||||
template <class Head>
|
||||
struct populate_visitor_map<Head> {
|
||||
static void populate(VisitorMap&) {}
|
||||
};
|
||||
|
||||
visitor_t() {
|
||||
populate_visitor_map<int64_t,
|
||||
uint64_t,
|
||||
bool,
|
||||
float,
|
||||
double,
|
||||
std::string,
|
||||
std::string_view,
|
||||
std::vector<std::any>,
|
||||
std::vector<std::string>,
|
||||
std::vector<std::string_view>,
|
||||
std::map<std::string, std::any>,
|
||||
std::map<std::string_view, std::any>,
|
||||
std::vector<std::map<std::string_view, std::any>>>::populate(visitorMap);
|
||||
}
|
||||
|
||||
void visit(const std::any& val, Packer& packer) {
|
||||
auto iter = visitorMap.find(val.type());
|
||||
if (iter == visitorMap.end()) {
|
||||
TraceEvent(SevError, "PackerTypeNotFound").detail("Type", val.type().name());
|
||||
} else {
|
||||
iter->second(val, packer);
|
||||
}
|
||||
}
|
||||
};
|
||||
msgpack::sbuffer sbuffer;
|
||||
// Initializing visitor_t involves building a type-map. As this is a relatively expensive operation, we don't want
|
||||
// to do this each time we create a Packer object. So visitor_t is a stateless class and we only use it as a
|
||||
// visitor.
|
||||
crossbow::singleton<visitor_t> visitor;
|
||||
|
||||
public:
|
||||
Packer() : msgpack::packer<msgpack::sbuffer>(sbuffer) {}
|
||||
|
||||
void pack(std::any const& val) { visitor->visit(val, *this); }
|
||||
|
||||
void pack(bool val) {
|
||||
if (val) {
|
||||
pack_true();
|
||||
} else {
|
||||
pack_false();
|
||||
}
|
||||
}
|
||||
|
||||
void pack(uint64_t val) {
|
||||
if (val <= std::numeric_limits<uint8_t>::max()) {
|
||||
pack_uint8(uint8_t(val));
|
||||
} else if (val <= std::numeric_limits<uint16_t>::max()) {
|
||||
pack_uint16(uint16_t(val));
|
||||
} else if (val <= std::numeric_limits<uint32_t>::max()) {
|
||||
pack_uint32(uint32_t(val));
|
||||
} else {
|
||||
pack_uint64(val);
|
||||
}
|
||||
}
|
||||
|
||||
void pack(int64_t val) {
|
||||
if (val >= 0) {
|
||||
this->pack(uint64_t(val));
|
||||
} else if (val >= std::numeric_limits<uint8_t>::min()) {
|
||||
pack_int8(int8_t(val));
|
||||
} else if (val >= std::numeric_limits<int16_t>::min()) {
|
||||
pack_int16(int16_t(val));
|
||||
} else if (val >= std::numeric_limits<int32_t>::min()) {
|
||||
pack_int32(int32_t(val));
|
||||
} else if (val >= std::numeric_limits<int64_t>::min()) {
|
||||
pack_int64(int64_t(val));
|
||||
}
|
||||
}
|
||||
|
||||
void pack(float val) { pack_float(val); }
|
||||
void pack(double val) { pack_double(val); }
|
||||
void pack(std::string const& str) {
|
||||
pack_str(str.size());
|
||||
pack_str_body(str.data(), str.size());
|
||||
}
|
||||
|
||||
void pack(std::string_view val) {
|
||||
pack_str(val.size());
|
||||
pack_str_body(val.data(), val.size());
|
||||
}
|
||||
|
||||
template <class K, class V>
|
||||
void pack(std::map<K, V> const& map) {
|
||||
pack_map(map.size());
|
||||
for (const auto& p : map) {
|
||||
pack(p.first);
|
||||
pack(p.second);
|
||||
}
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void pack(std::vector<T> const& val) {
|
||||
pack_array(val.size());
|
||||
for (const auto& v : val) {
|
||||
pack(v);
|
||||
}
|
||||
}
|
||||
|
||||
std::pair<char*, unsigned> getbuf() {
|
||||
unsigned size = sbuffer.size();
|
||||
return std::make_pair(sbuffer.release(), size);
|
||||
}
|
||||
};
|
||||
|
||||
IALPCollectorBase::IALPCollectorBase() {
|
||||
SampleCollector::instance().addCollector(this);
|
||||
}
|
||||
|
||||
std::map<std::string_view, std::any> SampleCollectorT::collect(ActorLineage* lineage) {
|
||||
ASSERT(lineage != nullptr);
|
||||
std::map<std::string_view, std::any> out;
|
||||
for (auto& collector : collectors) {
|
||||
auto val = collector->collect(lineage);
|
||||
if (val.has_value()) {
|
||||
out[collector->name()] = val.value();
|
||||
}
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
std::shared_ptr<Sample> SampleCollectorT::collect() {
|
||||
auto sample = std::make_shared<Sample>();
|
||||
double time = g_network->now();
|
||||
sample->time = time;
|
||||
for (auto& p : getSamples) {
|
||||
Packer packer;
|
||||
std::vector<std::map<std::string_view, std::any>> samples;
|
||||
auto sampleVec = p.second();
|
||||
for (auto& val : sampleVec) {
|
||||
auto m = collect(val.getPtr());
|
||||
if (!m.empty()) {
|
||||
samples.emplace_back(std::move(m));
|
||||
}
|
||||
}
|
||||
if (!samples.empty()) {
|
||||
packer.pack(samples);
|
||||
sample->data[p.first] = packer.getbuf();
|
||||
}
|
||||
}
|
||||
return sample;
|
||||
}
|
||||
|
||||
void SampleCollection_t::collect(const Reference<ActorLineage>& lineage) {
|
||||
ASSERT(lineage.isValid());
|
||||
_currentLineage = lineage;
|
||||
auto sample = _collector->collect();
|
||||
ASSERT(sample);
|
||||
{
|
||||
Lock _{ mutex };
|
||||
data.emplace_back(sample);
|
||||
}
|
||||
auto min = std::min(data.back()->time - windowSize, data.back()->time);
|
||||
double oldest = data.front()->time;
|
||||
// we don't need to check for data.empty() in this loop (or the inner loop) as we know that we will end
|
||||
// up with at least one entry which is the most recent sample
|
||||
while (oldest < min) {
|
||||
Lock _{ mutex };
|
||||
// we remove at most 10 elements at a time. This is so we don't block the main thread for too long.
|
||||
for (int i = 0; i < 10 && oldest < min; ++i) {
|
||||
data.pop_front();
|
||||
oldest = data.front()->time;
|
||||
}
|
||||
}
|
||||
// TODO: Should only call ingest when deleting from memory
|
||||
config->ingest(sample);
|
||||
}
|
||||
|
||||
std::vector<std::shared_ptr<Sample>> SampleCollection_t::get(double from /*= 0.0*/,
|
||||
double to /*= std::numeric_limits<double>::max()*/) const {
|
||||
Lock _{ mutex };
|
||||
std::vector<std::shared_ptr<Sample>> res;
|
||||
for (const auto& sample : data) {
|
||||
if (sample->time > to) {
|
||||
break;
|
||||
} else if (sample->time >= from) {
|
||||
res.push_back(sample);
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
void sample(LineageReference* lineagePtr) {
|
||||
if (!lineagePtr->isValid()) { return; }
|
||||
(*lineagePtr)->modify(&NameLineage::actorName) = lineagePtr->actorName();
|
||||
boost::asio::post(ActorLineageProfiler::instance().context(), [lineage = LineageReference::addRef(lineagePtr->getPtr())]() {
|
||||
SampleCollection::instance().collect(lineage);
|
||||
});
|
||||
}
|
||||
|
||||
struct ProfilerImpl {
|
||||
boost::asio::io_context context;
|
||||
boost::asio::executor_work_guard<decltype(context.get_executor())> workGuard;
|
||||
boost::asio::steady_timer timer;
|
||||
std::thread mainThread;
|
||||
unsigned frequency;
|
||||
|
||||
SampleCollection collection;
|
||||
|
||||
ProfilerImpl() : workGuard(context.get_executor()), timer(context) {
|
||||
mainThread = std::thread([this]() { context.run(); });
|
||||
}
|
||||
~ProfilerImpl() {
|
||||
setFrequency(0);
|
||||
workGuard.reset();
|
||||
mainThread.join();
|
||||
}
|
||||
|
||||
void profileHandler(boost::system::error_code const& ec) {
|
||||
if (ec) {
|
||||
return;
|
||||
}
|
||||
startSampling = true;
|
||||
timer = boost::asio::steady_timer(context, std::chrono::microseconds(1000000 / frequency));
|
||||
timer.async_wait([this](auto const& ec) { profileHandler(ec); });
|
||||
}
|
||||
|
||||
void setFrequency(unsigned frequency) {
|
||||
boost::asio::post(context, [this, frequency]() {
|
||||
this->frequency = frequency;
|
||||
timer.cancel();
|
||||
if (frequency > 0) {
|
||||
profileHandler(boost::system::error_code{});
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
ActorLineageProfilerT::ActorLineageProfilerT() : impl(new ProfilerImpl()) {
|
||||
// collection->collector()->addGetter(WaitState::Network,
|
||||
// std::bind(&ActorLineageSet::copy, std::ref(g_network->getActorLineageSet())));
|
||||
// collection->collector()->addGetter(
|
||||
// WaitState::Disk,
|
||||
// std::bind(&ActorLineageSet::copy, std::ref(IAsyncFileSystem::filesystem()->getActorLineageSet())));
|
||||
collection->collector()->addGetter(WaitState::Running, []() {
|
||||
return std::vector<Reference<ActorLineage>>({ SampleCollection::instance().getLineage() });
|
||||
});
|
||||
}
|
||||
|
||||
ActorLineageProfilerT::~ActorLineageProfilerT() {
|
||||
delete impl;
|
||||
}
|
||||
|
||||
void ActorLineageProfilerT::setFrequency(unsigned frequency) {
|
||||
impl->setFrequency(frequency);
|
||||
}
|
||||
|
||||
boost::asio::io_context& ActorLineageProfilerT::context() {
|
||||
return impl->context;
|
||||
}
|
||||
|
||||
SampleIngestor::~SampleIngestor() {}
|
||||
|
||||
void ProfilerConfigT::reset(std::map<std::string, std::string> const& config) {
|
||||
bool expectNoMore = false, useFluentD = false, useTCP = false;
|
||||
std::string endpoint;
|
||||
ConfigError err;
|
||||
for (auto& kv : config) {
|
||||
if (expectNoMore) {
|
||||
err.description = format("Unexpected option %s", kv.first.c_str());
|
||||
throw err;
|
||||
}
|
||||
if (kv.first == "ingestor") {
|
||||
std::string val = kv.second;
|
||||
std::for_each(val.begin(), val.end(), [](auto c) { return std::tolower(c); });
|
||||
if (val == "none") {
|
||||
setBackend(std::make_shared<NoneIngestor>());
|
||||
} else if (val == "fluentd") {
|
||||
useFluentD = true;
|
||||
} else {
|
||||
err.description = format("Unsupported ingestor: %s", val.c_str());
|
||||
throw err;
|
||||
}
|
||||
} else if (kv.first == "ingestor_endpoint") {
|
||||
endpoint = kv.second;
|
||||
} else if (kv.first == "ingestor_protocol") {
|
||||
auto val = kv.second;
|
||||
std::for_each(val.begin(), val.end(), [](auto c) { return std::tolower(c); });
|
||||
if (val == "tcp") {
|
||||
useTCP = true;
|
||||
} else if (val == "udp") {
|
||||
useTCP = false;
|
||||
} else {
|
||||
err.description = format("Unsupported protocol for fluentd: %s", kv.second.c_str());
|
||||
throw err;
|
||||
}
|
||||
} else {
|
||||
err.description = format("Unknown option %s", kv.first.c_str());
|
||||
throw err;
|
||||
}
|
||||
}
|
||||
if (useFluentD) {
|
||||
if (endpoint.empty()) {
|
||||
err.description = "Endpoint is required for fluentd ingestor";
|
||||
throw err;
|
||||
}
|
||||
NetworkAddress address;
|
||||
try {
|
||||
address = NetworkAddress::parse(endpoint);
|
||||
} catch (Error& e) {
|
||||
err.description = format("Can't parse address %s", endpoint.c_str());
|
||||
throw err;
|
||||
}
|
||||
setBackend(std::make_shared<FluentDIngestor>(
|
||||
useTCP ? FluentDIngestor::Protocol::TCP : FluentDIngestor::Protocol::UDP, address));
|
||||
}
|
||||
}
|
||||
|
||||
std::map<std::string, std::string> ProfilerConfigT::getConfig() const {
|
||||
std::map<std::string, std::string> res;
|
||||
if (ingestor) {
|
||||
ingestor->getConfig(res);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
// Callback used to update the sampling profilers run frequency whenever the
|
||||
// frequency changes.
|
||||
void samplingProfilerUpdateFrequency(std::optional<std::any> freq) {
|
||||
double frequency = 0;
|
||||
if (freq.has_value()) {
|
||||
frequency = std::any_cast<double>(freq.value());
|
||||
}
|
||||
TraceEvent(SevInfo, "SamplingProfilerUpdateFrequency").detail("Frequency", frequency);
|
||||
ActorLineageProfiler::instance().setFrequency(frequency);
|
||||
}
|
||||
|
||||
// Callback used to update the sample collector window size.
|
||||
void samplingProfilerUpdateWindow(std::optional<std::any> window) {
|
||||
double duration = 0;
|
||||
if (window.has_value()) {
|
||||
duration = std::any_cast<double>(window.value());
|
||||
}
|
||||
TraceEvent(SevInfo, "SamplingProfilerUpdateWindow").detail("Duration", duration);
|
||||
SampleCollection::instance().setWindowSize(duration);
|
||||
}
|
|
@ -0,0 +1,192 @@
|
|||
/*
|
||||
* ActorLineageProfiler.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "fdbclient/AnnotateActor.h"
|
||||
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <any>
|
||||
#include <vector>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include "flow/singleton.h"
|
||||
#include "flow/flow.h"
|
||||
|
||||
void samplingProfilerUpdateFrequency(std::optional<std::any> freq);
|
||||
void samplingProfilerUpdateWindow(std::optional<std::any> window);
|
||||
|
||||
struct IALPCollectorBase {
|
||||
virtual std::optional<std::any> collect(ActorLineage*) = 0;
|
||||
virtual const std::string_view& name() = 0;
|
||||
IALPCollectorBase();
|
||||
};
|
||||
|
||||
template <class T>
|
||||
struct IALPCollector : IALPCollectorBase {
|
||||
const std::string_view& name() override { return T::name; }
|
||||
};
|
||||
|
||||
struct Sample : std::enable_shared_from_this<Sample> {
|
||||
double time = 0.0;
|
||||
Sample() {}
|
||||
Sample(Sample const&) = delete;
|
||||
Sample& operator=(Sample const&) = delete;
|
||||
std::unordered_map<WaitState, std::pair<char*, unsigned>> data;
|
||||
~Sample() {
|
||||
std::for_each(data.begin(), data.end(), [](std::pair<WaitState, std::pair<char*, unsigned>> entry) {
|
||||
::free(entry.second.first);
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
class SampleIngestor : std::enable_shared_from_this<SampleIngestor> {
|
||||
public:
|
||||
virtual ~SampleIngestor();
|
||||
virtual void ingest(std::shared_ptr<Sample> const& sample) = 0;
|
||||
virtual void getConfig(std::map<std::string, std::string>&) const = 0;
|
||||
};
|
||||
|
||||
class NoneIngestor : public SampleIngestor {
|
||||
public:
|
||||
void ingest(std::shared_ptr<Sample> const& sample) override {}
|
||||
void getConfig(std::map<std::string, std::string>& res) const override { res["ingestor"] = "none"; }
|
||||
};
|
||||
|
||||
// The FluentD ingestor uses the pimpl idiom. This is to make compilation less heavy weight as this implementation has
|
||||
// dependencies to boost::asio
|
||||
struct FluentDIngestorImpl;
|
||||
|
||||
class FluentDIngestor : public SampleIngestor {
|
||||
public: // Public Types
|
||||
enum class Protocol { TCP, UDP };
|
||||
|
||||
private: // members
|
||||
FluentDIngestorImpl* impl;
|
||||
|
||||
public: // interface
|
||||
void ingest(std::shared_ptr<Sample> const& sample) override;
|
||||
FluentDIngestor(Protocol protocol, NetworkAddress& endpoint);
|
||||
void getConfig(std::map<std::string, std::string>& res) const override;
|
||||
~FluentDIngestor();
|
||||
};
|
||||
|
||||
struct ConfigError {
|
||||
std::string description;
|
||||
};
|
||||
|
||||
class ProfilerConfigT {
|
||||
private: // private types
|
||||
using Lock = std::unique_lock<std::mutex>;
|
||||
friend class crossbow::create_static<ProfilerConfigT>;
|
||||
|
||||
private: // members
|
||||
std::shared_ptr<SampleIngestor> ingestor = std::make_shared<NoneIngestor>();
|
||||
|
||||
private: // construction
|
||||
ProfilerConfigT() {}
|
||||
ProfilerConfigT(ProfilerConfigT const&) = delete;
|
||||
ProfilerConfigT& operator=(ProfilerConfigT const&) = delete;
|
||||
void setBackend(std::shared_ptr<SampleIngestor> ingestor) { this->ingestor = ingestor; }
|
||||
|
||||
public:
|
||||
void ingest(std::shared_ptr<Sample> sample) { ingestor->ingest(sample); }
|
||||
void reset(std::map<std::string, std::string> const& config);
|
||||
std::map<std::string, std::string> getConfig() const;
|
||||
};
|
||||
|
||||
using ProfilerConfig = crossbow::singleton<ProfilerConfigT>;
|
||||
|
||||
class SampleCollectorT {
|
||||
public: // Types
|
||||
friend struct crossbow::create_static<SampleCollectorT>;
|
||||
using Getter = std::function<std::vector<Reference<ActorLineage>>()>;
|
||||
|
||||
private:
|
||||
std::vector<IALPCollectorBase*> collectors;
|
||||
std::map<WaitState, Getter> getSamples;
|
||||
SampleCollectorT() {}
|
||||
std::map<std::string_view, std::any> collect(ActorLineage* lineage);
|
||||
|
||||
public:
|
||||
void addCollector(IALPCollectorBase* collector) { collectors.push_back(collector); }
|
||||
std::shared_ptr<Sample> collect();
|
||||
void addGetter(WaitState waitState, Getter const& getter) { getSamples[waitState] = getter; };
|
||||
};
|
||||
|
||||
using SampleCollector = crossbow::singleton<SampleCollectorT>;
|
||||
|
||||
class SampleCollection_t {
|
||||
friend struct crossbow::create_static<SampleCollection_t>;
|
||||
using Lock = std::unique_lock<std::mutex>;
|
||||
SampleCollection_t() {}
|
||||
|
||||
SampleCollector _collector;
|
||||
mutable std::mutex mutex;
|
||||
std::atomic<double> windowSize = 0.0;
|
||||
std::deque<std::shared_ptr<Sample>> data;
|
||||
ProfilerConfig config;
|
||||
Reference<ActorLineage> _currentLineage;
|
||||
|
||||
public:
|
||||
/**
|
||||
* Define how many samples the collection shoul keep. The window size is defined by time dimension.
|
||||
*
|
||||
* \param duration How long a sample should be kept in the collection.
|
||||
*/
|
||||
void setWindowSize(double duration) { windowSize.store(duration); }
|
||||
/**
|
||||
* By default returns reference counted pointers of all samples. A window can be defined in terms of absolute time.
|
||||
*
|
||||
* \param from The minimal age of all returned samples.
|
||||
* \param to The max age of all returned samples.
|
||||
*/
|
||||
std::vector<std::shared_ptr<Sample>> get(double from = 0.0, double to = std::numeric_limits<double>::max()) const;
|
||||
void collect(const Reference<ActorLineage>& lineage);
|
||||
const SampleCollector& collector() const { return _collector; }
|
||||
SampleCollector& collector() { return _collector; }
|
||||
Reference<ActorLineage> getLineage() { return _currentLineage; }
|
||||
};
|
||||
|
||||
using SampleCollection = crossbow::singleton<SampleCollection_t>;
|
||||
|
||||
struct ProfilerImpl;
|
||||
|
||||
namespace boost {
|
||||
namespace asio {
|
||||
// forward declare io_context because including boost asio is super expensive
|
||||
class io_context;
|
||||
} // namespace asio
|
||||
} // namespace boost
|
||||
|
||||
class ActorLineageProfilerT {
|
||||
friend struct crossbow::create_static<ActorLineageProfilerT>;
|
||||
ProfilerImpl* impl;
|
||||
SampleCollection collection;
|
||||
ActorLineageProfilerT();
|
||||
|
||||
public:
|
||||
~ActorLineageProfilerT();
|
||||
void setFrequency(unsigned frequency);
|
||||
boost::asio::io_context& context();
|
||||
};
|
||||
|
||||
using ActorLineageProfiler = crossbow::singleton<ActorLineageProfilerT>;
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* AnnotateActor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/AnnotateActor.h"
|
||||
|
||||
std::map<WaitState, std::function<std::vector<Reference<ActorLineage>>()>> samples;
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* AnnotateActor.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "flow/flow.h"
|
||||
#include "flow/network.h"
|
||||
|
||||
#include <string_view>
|
||||
|
||||
// Used to manually instrument waiting actors to collect samples for the
|
||||
// sampling profiler.
|
||||
struct AnnotateActor {
|
||||
unsigned index;
|
||||
bool set;
|
||||
|
||||
AnnotateActor() : set(false) {}
|
||||
|
||||
AnnotateActor(LineageReference* lineage) : set(false) {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
if (lineage->getPtr() != 0) {
|
||||
index = g_network->getActorLineageSet().insert(*lineage);
|
||||
set = (index != ActorLineageSet::npos);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
AnnotateActor(const AnnotateActor& other) = delete;
|
||||
AnnotateActor(AnnotateActor&& other) = delete;
|
||||
AnnotateActor& operator=(const AnnotateActor& other) = delete;
|
||||
|
||||
AnnotateActor& operator=(AnnotateActor&& other) {
|
||||
if (this == &other) {
|
||||
return *this;
|
||||
}
|
||||
|
||||
this->index = other.index;
|
||||
this->set = other.set;
|
||||
|
||||
other.set = false;
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
~AnnotateActor() {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
if (set) {
|
||||
g_network->getActorLineageSet().erase(index);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
};
|
||||
|
||||
enum class WaitState { Disk, Network, Running };
|
||||
// usually we shouldn't use `using namespace` in a header file, but literals should be safe as user defined literals
|
||||
// need to be prefixed with `_`
|
||||
using namespace std::literals;
|
||||
|
||||
constexpr std::string_view to_string(WaitState st) {
|
||||
switch (st) {
|
||||
case WaitState::Disk:
|
||||
return "Disk"sv;
|
||||
case WaitState::Network:
|
||||
return "Network"sv;
|
||||
case WaitState::Running:
|
||||
return "Running"sv;
|
||||
default:
|
||||
return ""sv;
|
||||
}
|
||||
}
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
extern std::map<WaitState, std::function<std::vector<Reference<ActorLineage>>()>> samples;
|
||||
#endif
|
|
@ -330,7 +330,7 @@ void decodeBackupLogValue(Arena& arena,
|
|||
}
|
||||
} else {
|
||||
Version ver = key_version->rangeContaining(logValue.param1).value();
|
||||
//TraceEvent("ApplyMutation").detail("LogValue", logValue.toString()).detail("Version", version).detail("Ver", ver).detail("Apply", version > ver && ver != invalidVersion);
|
||||
//TraceEvent("ApplyMutation").detail("LogValue", logValue).detail("Version", version).detail("Ver", ver).detail("Apply", version > ver && ver != invalidVersion);
|
||||
if (version > ver && ver != invalidVersion) {
|
||||
if (removePrefix.size()) {
|
||||
logValue.param1 = logValue.param1.removePrefix(removePrefix);
|
||||
|
|
|
@ -1,4 +1,7 @@
|
|||
set(FDBCLIENT_SRCS
|
||||
ActorLineageProfiler.h
|
||||
ActorLineageProfiler.cpp
|
||||
AnnotateActor.cpp
|
||||
AsyncFileS3BlobStore.actor.cpp
|
||||
AsyncFileS3BlobStore.actor.h
|
||||
AsyncTaskThread.actor.cpp
|
||||
|
@ -39,6 +42,7 @@ set(FDBCLIENT_SRCS
|
|||
FDBOptions.h
|
||||
FDBTypes.cpp
|
||||
FDBTypes.h
|
||||
FluentDSampleIngestor.cpp
|
||||
FileBackupAgent.actor.cpp
|
||||
GlobalConfig.h
|
||||
GlobalConfig.actor.h
|
||||
|
@ -66,6 +70,8 @@ set(FDBCLIENT_SRCS
|
|||
MultiVersionTransaction.actor.cpp
|
||||
MultiVersionTransaction.h
|
||||
MutationList.h
|
||||
NameLineage.h
|
||||
NameLineage.cpp
|
||||
NativeAPI.actor.cpp
|
||||
NativeAPI.actor.h
|
||||
Notified.h
|
||||
|
@ -102,6 +108,8 @@ set(FDBCLIENT_SRCS
|
|||
StorageServerInterface.h
|
||||
Subspace.cpp
|
||||
Subspace.h
|
||||
StackLineage.h
|
||||
StackLineage.cpp
|
||||
SystemData.cpp
|
||||
SystemData.h
|
||||
TagThrottle.actor.cpp
|
||||
|
@ -173,8 +181,17 @@ endif()
|
|||
|
||||
add_flow_target(STATIC_LIBRARY NAME fdbclient SRCS ${FDBCLIENT_SRCS} ADDL_SRCS ${options_srcs})
|
||||
add_dependencies(fdbclient fdboptions fdb_c_options)
|
||||
target_link_libraries(fdbclient PUBLIC fdbrpc msgpack)
|
||||
|
||||
# Create a separate fdbclient library with sampling enabled. This lets
|
||||
# fdbserver retain sampling functionality in client code while disabling
|
||||
# sampling for pure clients.
|
||||
add_flow_target(STATIC_LIBRARY NAME fdbclient_sampling SRCS ${FDBCLIENT_SRCS} ADDL_SRCS ${options_srcs})
|
||||
add_dependencies(fdbclient_sampling fdboptions fdb_c_options)
|
||||
target_link_libraries(fdbclient_sampling PUBLIC fdbrpc_sampling msgpack)
|
||||
target_compile_definitions(fdbclient_sampling PRIVATE -DENABLE_SAMPLING)
|
||||
|
||||
if(BUILD_AZURE_BACKUP)
|
||||
target_link_libraries(fdbclient PUBLIC fdbrpc PRIVATE curl uuid azure-storage-lite)
|
||||
else()
|
||||
target_link_libraries(fdbclient PUBLIC fdbrpc)
|
||||
target_link_libraries(fdbclient PRIVATE curl uuid azure-storage-lite)
|
||||
target_link_libraries(fdbclient_sampling PRIVATE curl uuid azure-storage-lite)
|
||||
endif()
|
||||
|
|
|
@ -22,6 +22,9 @@
|
|||
#ifndef FDBCLIENT_CLIENTLOGEVENTS_H
|
||||
#define FDBCLIENT_CLIENTLOGEVENTS_H
|
||||
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbclient/CommitProxyInterface.h"
|
||||
|
||||
namespace FdbClientLogEvents {
|
||||
enum class EventType {
|
||||
GET_VERSION_LATENCY = 0,
|
||||
|
@ -252,7 +255,7 @@ struct EventCommit : public Event {
|
|||
.setMaxEventLength(-1)
|
||||
.detail("TransactionID", id)
|
||||
.setMaxFieldLength(maxFieldLength)
|
||||
.detail("Mutation", mutation.toString());
|
||||
.detail("Mutation", mutation);
|
||||
}
|
||||
|
||||
TraceEvent("TransactionTrace_Commit")
|
||||
|
@ -316,7 +319,7 @@ struct EventCommit_V2 : public Event {
|
|||
.setMaxEventLength(-1)
|
||||
.detail("TransactionID", id)
|
||||
.setMaxFieldLength(maxFieldLength)
|
||||
.detail("Mutation", mutation.toString());
|
||||
.detail("Mutation", mutation);
|
||||
}
|
||||
|
||||
TraceEvent("TransactionTrace_Commit")
|
||||
|
@ -430,7 +433,7 @@ struct EventCommitError : public Event {
|
|||
.setMaxEventLength(-1)
|
||||
.detail("TransactionID", id)
|
||||
.setMaxFieldLength(maxFieldLength)
|
||||
.detail("Mutation", mutation.toString());
|
||||
.detail("Mutation", mutation);
|
||||
}
|
||||
|
||||
TraceEvent("TransactionTrace_CommitError").detail("TransactionID", id).detail("ErrCode", errCode);
|
||||
|
|
|
@ -2355,7 +2355,7 @@ std::string getDRMutationStreamId(StatusObjectReader statusObj, const char* cont
|
|||
}
|
||||
}
|
||||
}
|
||||
TraceEvent(SevWarn, "DBA_TagNotPresentInStatus").detail("Tag", tagName.toString()).detail("Context", context);
|
||||
TraceEvent(SevWarn, "DBA_TagNotPresentInStatus").detail("Tag", tagName).detail("Context", context);
|
||||
throw backup_error();
|
||||
} catch (std::runtime_error& e) {
|
||||
TraceEvent(SevWarn, "DBA_GetDRMutationStreamIdFail").detail("Error", e.what());
|
||||
|
|
|
@ -0,0 +1,265 @@
|
|||
/*
|
||||
* FluentDSampleIngestor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
#include <boost/asio.hpp>
|
||||
#include <boost/asio/co_spawn.hpp>
|
||||
#include <msgpack.hpp>
|
||||
|
||||
namespace {
|
||||
|
||||
boost::asio::ip::address ipAddress(IPAddress const& n) {
|
||||
if (n.isV6()) {
|
||||
return boost::asio::ip::address_v6(n.toV6());
|
||||
} else {
|
||||
return boost::asio::ip::address_v4(n.toV4());
|
||||
}
|
||||
}
|
||||
|
||||
template <class Protocol>
|
||||
boost::asio::ip::basic_endpoint<Protocol> toEndpoint(NetworkAddress const n) {
|
||||
return boost::asio::ip::basic_endpoint<Protocol>(ipAddress(n.ip), n.port);
|
||||
}
|
||||
|
||||
struct FluentDSocket {
|
||||
virtual ~FluentDSocket() {}
|
||||
virtual void connect(NetworkAddress const& endpoint) = 0;
|
||||
virtual void send(std::shared_ptr<Sample> const& sample) = 0;
|
||||
virtual const boost::system::error_code& failed() const = 0;
|
||||
};
|
||||
|
||||
template <class Protocol, class Callback>
|
||||
class SampleSender : public std::enable_shared_from_this<SampleSender<Protocol, Callback>> {
|
||||
using Socket = typename Protocol::socket;
|
||||
using Iter = typename decltype(Sample::data)::iterator;
|
||||
Socket& socket;
|
||||
Callback callback;
|
||||
Iter iter, end;
|
||||
std::shared_ptr<Sample> sample_; // to keep from being deallocated
|
||||
|
||||
struct Buf {
|
||||
const char* data;
|
||||
const unsigned size;
|
||||
Buf(const char* data, unsigned size) : data(data), size(size) {}
|
||||
Buf(Buf const&) = delete;
|
||||
Buf& operator=(Buf const&) = delete;
|
||||
~Buf() { delete[] data; }
|
||||
};
|
||||
|
||||
void sendCompletionHandler(boost::system::error_code const& ec) {
|
||||
if (ec) {
|
||||
callback(ec);
|
||||
} else {
|
||||
++iter;
|
||||
sendNext();
|
||||
}
|
||||
}
|
||||
|
||||
void send(boost::asio::ip::tcp::socket& socket, std::shared_ptr<Buf> const& buf) {
|
||||
boost::asio::async_write(
|
||||
socket,
|
||||
boost::asio::const_buffer(buf->data, buf->size),
|
||||
[buf, this](auto const& ec, size_t) {
|
||||
this->sendCompletionHandler(ec);
|
||||
});
|
||||
}
|
||||
void send(boost::asio::ip::udp::socket& socket, std::shared_ptr<Buf> const& buf) {
|
||||
socket.async_send(
|
||||
boost::asio::const_buffer(buf->data, buf->size),
|
||||
[buf, this](auto const& ec, size_t) { this->sendCompletionHandler(ec); });
|
||||
}
|
||||
|
||||
void sendNext() {
|
||||
if (iter == end) {
|
||||
callback(boost::system::error_code());
|
||||
return;
|
||||
}
|
||||
// 1. calculate size of buffer
|
||||
unsigned size = 1; // 1 for fixmap identifier byte
|
||||
auto waitState = to_string(iter->first);
|
||||
if (waitState.size() < 32) {
|
||||
size += waitState.size() + 1;
|
||||
} else {
|
||||
size += waitState.size() + 2;
|
||||
}
|
||||
size += iter->second.second;
|
||||
// 2. allocate the buffer
|
||||
std::unique_ptr<char[]> buf(new char[size]);
|
||||
unsigned off = 0;
|
||||
// 3. serialize fixmap
|
||||
buf[off++] = 0x81; // map of size 1
|
||||
// 3.1 serialize key
|
||||
if (waitState.size() < 32) {
|
||||
buf[off++] = 0xa0 + waitState.size(); // fixstr
|
||||
} else {
|
||||
buf[off++] = 0xd9;
|
||||
buf[off++] = char(waitState.size());
|
||||
}
|
||||
memcpy(buf.get() + off, waitState.data(), waitState.size());
|
||||
off += waitState.size();
|
||||
// 3.2 append serialized value
|
||||
memcpy(buf.get() + off, iter->second.first, iter->second.second);
|
||||
// 4. send the result to fluentd
|
||||
send(socket, std::make_shared<Buf>(buf.release(), size));
|
||||
}
|
||||
|
||||
public:
|
||||
SampleSender(Socket& socket, Callback const& callback, std::shared_ptr<Sample> const& sample)
|
||||
: socket(socket),
|
||||
callback(callback),
|
||||
iter(sample->data.begin()),
|
||||
end(sample->data.end()),
|
||||
sample_(sample) {
|
||||
sendNext();
|
||||
}
|
||||
};
|
||||
|
||||
// Sample function to make instanciation of SampleSender easier
|
||||
template <class Protocol, class Callback>
|
||||
std::shared_ptr<SampleSender<Protocol, Callback>> makeSampleSender(typename Protocol::socket& socket, Callback const& callback, std::shared_ptr<Sample> const& sample) {
|
||||
return std::make_shared<SampleSender<Protocol, Callback>>(socket, callback, sample);
|
||||
}
|
||||
|
||||
template <class Protocol>
|
||||
struct FluentDSocketImpl : FluentDSocket, std::enable_shared_from_this<FluentDSocketImpl<Protocol>> {
|
||||
static constexpr unsigned MAX_QUEUE_SIZE = 100;
|
||||
boost::asio::io_context& context;
|
||||
typename Protocol::socket socket;
|
||||
FluentDSocketImpl(boost::asio::io_context& context) : context(context), socket(context) {}
|
||||
bool ready = false;
|
||||
std::deque<std::shared_ptr<Sample>> queue;
|
||||
boost::system::error_code _failed;
|
||||
|
||||
const boost::system::error_code& failed() const override { return _failed; }
|
||||
|
||||
void sendCompletionHandler(boost::system::error_code const& ec) {
|
||||
if (ec) {
|
||||
// TODO: trace error
|
||||
_failed = ec;
|
||||
return;
|
||||
}
|
||||
if (queue.empty()) {
|
||||
ready = true;
|
||||
} else {
|
||||
auto sample = queue.front();
|
||||
queue.pop_front();
|
||||
sendImpl(sample);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void sendImpl(std::shared_ptr<Sample> const& sample) {
|
||||
makeSampleSender<Protocol>(socket, [self = this->shared_from_this()](boost::system::error_code const& ec){
|
||||
self->sendCompletionHandler(ec);
|
||||
}, sample);
|
||||
}
|
||||
|
||||
void send(std::shared_ptr<Sample> const& sample) override {
|
||||
if (_failed) {
|
||||
return;
|
||||
}
|
||||
if (ready) {
|
||||
ready = false;
|
||||
sendImpl(sample);
|
||||
} else {
|
||||
if (queue.size() < MAX_QUEUE_SIZE) {
|
||||
queue.push_back(sample);
|
||||
} // TODO: else trace a warning
|
||||
}
|
||||
}
|
||||
|
||||
void connect(NetworkAddress const& endpoint) override {
|
||||
auto to = toEndpoint<Protocol>(endpoint);
|
||||
socket.async_connect(to, [self = this->shared_from_this()](boost::system::error_code const& ec) {
|
||||
if (ec) {
|
||||
// TODO: error handling
|
||||
self->_failed = ec;
|
||||
return;
|
||||
}
|
||||
self->ready = true;
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace
|
||||
|
||||
struct FluentDIngestorImpl {
|
||||
using Protocol = FluentDIngestor::Protocol;
|
||||
Protocol protocol;
|
||||
NetworkAddress endpoint;
|
||||
boost::asio::io_context& io_context;
|
||||
std::shared_ptr<FluentDSocket> socket;
|
||||
boost::asio::steady_timer retryTimer;
|
||||
FluentDIngestorImpl(Protocol protocol, NetworkAddress const& endpoint)
|
||||
: protocol(protocol), endpoint(endpoint), io_context(ActorLineageProfiler::instance().context()),
|
||||
retryTimer(io_context) {
|
||||
connect();
|
||||
}
|
||||
|
||||
~FluentDIngestorImpl() { retryTimer.cancel(); }
|
||||
|
||||
void connect() {
|
||||
switch (protocol) {
|
||||
case Protocol::TCP:
|
||||
socket.reset(new FluentDSocketImpl<boost::asio::ip::tcp>(io_context));
|
||||
break;
|
||||
case Protocol::UDP:
|
||||
socket.reset(new FluentDSocketImpl<boost::asio::ip::udp>(io_context));
|
||||
break;
|
||||
}
|
||||
socket->connect(endpoint);
|
||||
}
|
||||
|
||||
void retry() {
|
||||
retryTimer = boost::asio::steady_timer(io_context, std::chrono::seconds(1));
|
||||
retryTimer.async_wait([this](auto const& ec) {
|
||||
if (ec) {
|
||||
return;
|
||||
}
|
||||
connect();
|
||||
});
|
||||
socket.reset();
|
||||
}
|
||||
};
|
||||
|
||||
FluentDIngestor::~FluentDIngestor() {
|
||||
delete impl;
|
||||
}
|
||||
|
||||
FluentDIngestor::FluentDIngestor(Protocol protocol, NetworkAddress& endpoint)
|
||||
: impl(new FluentDIngestorImpl(protocol, endpoint)) {}
|
||||
|
||||
void FluentDIngestor::ingest(const std::shared_ptr<Sample>& sample) {
|
||||
if (!impl->socket) {
|
||||
// the connection failed in the past and we wait for a timeout before we retry
|
||||
return;
|
||||
} else if (impl->socket->failed()) {
|
||||
impl->retry();
|
||||
return;
|
||||
} else {
|
||||
impl->socket->send(sample);
|
||||
}
|
||||
}
|
||||
|
||||
void FluentDIngestor::getConfig(std::map<std::string, std::string>& res) const {
|
||||
res["ingestor"] = "fluentd";
|
||||
res["collector_endpoint"] = impl->endpoint.toString();
|
||||
res["collector_protocol"] = impl->protocol == Protocol::TCP ? "tcp" : "udp";
|
||||
}
|
|
@ -34,6 +34,9 @@ const KeyRef fdbClientInfoTxnSizeLimit = LiteralStringRef("config/fdb_client_inf
|
|||
const KeyRef transactionTagSampleRate = LiteralStringRef("config/transaction_tag_sample_rate");
|
||||
const KeyRef transactionTagSampleCost = LiteralStringRef("config/transaction_tag_sample_cost");
|
||||
|
||||
const KeyRef samplingFrequency = LiteralStringRef("visibility/sampling/frequency");
|
||||
const KeyRef samplingWindow = LiteralStringRef("visibility/sampling/window");
|
||||
|
||||
GlobalConfig::GlobalConfig(Database& cx) : cx(cx), lastUpdate(0) {}
|
||||
|
||||
GlobalConfig& GlobalConfig::globalConfig() {
|
||||
|
|
|
@ -28,6 +28,7 @@
|
|||
|
||||
#include <any>
|
||||
#include <map>
|
||||
#include <optional>
|
||||
#include <type_traits>
|
||||
#include <unordered_map>
|
||||
|
||||
|
@ -49,6 +50,9 @@ extern const KeyRef fdbClientInfoTxnSizeLimit;
|
|||
extern const KeyRef transactionTagSampleRate;
|
||||
extern const KeyRef transactionTagSampleCost;
|
||||
|
||||
extern const KeyRef samplingFrequency;
|
||||
extern const KeyRef samplingWindow;
|
||||
|
||||
// Structure used to hold the values stored by global configuration. The arena
|
||||
// is used as memory to store both the key and the value (the value is only
|
||||
// stored in the arena if it is an object; primitives are just copied).
|
||||
|
@ -78,6 +82,7 @@ public:
|
|||
g_network->setGlobal(INetwork::enGlobalConfig, config);
|
||||
config->_updater = updater(config, dbInfo);
|
||||
// Bind changes in `db` to the `dbInfoChanged` AsyncTrigger.
|
||||
// TODO: Change AsyncTrigger to a Reference
|
||||
forward(db, std::addressof(config->dbInfoChanged));
|
||||
} else {
|
||||
GlobalConfig* config = reinterpret_cast<GlobalConfig*>(g_network->global(INetwork::enGlobalConfig));
|
||||
|
@ -137,9 +142,11 @@ public:
|
|||
Future<Void> onChange();
|
||||
|
||||
// Calls \ref fn when the value associated with \ref key is changed. \ref
|
||||
// fn is passed the updated value for the key, or an empty optional if the
|
||||
// key has been cleared. If the value is an allocated object, its memory
|
||||
// remains in the control of the global configuration.
|
||||
// key should be one of the string literals defined at the top of
|
||||
// GlobalConfig.actor.cpp, to ensure memory validity. \ref fn is passed the
|
||||
// updated value for the key, or an empty optional if the key has been
|
||||
// cleared. If the value is an allocated object, its memory remains in the
|
||||
// control of global configuration.
|
||||
void trigger(KeyRef key, std::function<void(std::optional<std::any>)> fn);
|
||||
|
||||
private:
|
||||
|
@ -171,6 +178,7 @@ private:
|
|||
AsyncTrigger configChanged;
|
||||
std::unordered_map<StringRef, Reference<ConfigValue>> data;
|
||||
Version lastUpdate;
|
||||
// The key should be a global config string literal key (see the top of this class).
|
||||
std::unordered_map<KeyRef, std::function<void(std::optional<std::any>)>> callbacks;
|
||||
};
|
||||
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* NameLineage.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/NameLineage.h"
|
||||
|
||||
namespace {
|
||||
NameLineageCollector nameLineageCollector;
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* NameLineage.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <string_view>
|
||||
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
|
||||
struct NameLineage : LineageProperties<NameLineage> {
|
||||
static constexpr std::string_view name = "Actor"sv;
|
||||
const char* actorName;
|
||||
};
|
||||
|
||||
struct NameLineageCollector : IALPCollector<NameLineage> {
|
||||
NameLineageCollector() : IALPCollector() {}
|
||||
std::optional<std::any> collect(ActorLineage* lineage) override {
|
||||
auto str = lineage->get(&NameLineage::actorName);
|
||||
if (str.has_value()) {
|
||||
return std::string_view(*str, std::strlen(*str));
|
||||
} else {
|
||||
return {};
|
||||
}
|
||||
}
|
||||
};
|
|
@ -32,6 +32,8 @@
|
|||
#include "fdbrpc/FailureMonitor.h"
|
||||
#include "fdbrpc/MultiInterface.h"
|
||||
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
#include "fdbclient/AnnotateActor.h"
|
||||
#include "fdbclient/Atomic.h"
|
||||
#include "fdbclient/ClusterInterface.h"
|
||||
#include "fdbclient/CoordinationInterface.h"
|
||||
|
@ -42,6 +44,7 @@
|
|||
#include "fdbclient/KeyBackedTypes.h"
|
||||
#include "fdbclient/KeyRangeMap.h"
|
||||
#include "fdbclient/ManagementAPI.actor.h"
|
||||
#include "fdbclient/NameLineage.h"
|
||||
#include "fdbclient/CommitProxyInterface.h"
|
||||
#include "fdbclient/MonitorLeader.h"
|
||||
#include "fdbclient/MutationList.h"
|
||||
|
@ -50,6 +53,7 @@
|
|||
#include "fdbclient/SpecialKeySpace.actor.h"
|
||||
#include "fdbclient/StorageServerInterface.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/TransactionLineage.h"
|
||||
#include "fdbclient/versions.h"
|
||||
#include "fdbrpc/LoadBalance.h"
|
||||
#include "fdbrpc/Net2FileSystem.h"
|
||||
|
@ -87,6 +91,9 @@ using std::pair;
|
|||
|
||||
namespace {
|
||||
|
||||
TransactionLineageCollector transactionLineageCollector;
|
||||
NameLineageCollector nameLineageCollector;
|
||||
|
||||
template <class Interface, class Request>
|
||||
Future<REPLY_TYPE(Request)> loadBalance(
|
||||
DatabaseContext* ctx,
|
||||
|
@ -147,16 +154,25 @@ void DatabaseContext::addTssMapping(StorageServerInterface const& ssi, StorageSe
|
|||
result->second = tssi;
|
||||
}
|
||||
|
||||
// data requests duplicated for load and data comparison
|
||||
queueModel.updateTssEndpoint(ssi.getValue.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getValue.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getKey.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getKey.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getKeyValues.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getKeyValues.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.watchValue.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.watchValue.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getKeyValuesStream.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getKeyValuesStream.getEndpoint(), metrics));
|
||||
|
||||
// non-data requests duplicated for load
|
||||
queueModel.updateTssEndpoint(ssi.watchValue.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.watchValue.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.splitMetrics.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.splitMetrics.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getReadHotRanges.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getReadHotRanges.getEndpoint(), metrics));
|
||||
queueModel.updateTssEndpoint(ssi.getRangeSplitPoints.getEndpoint().token.first(),
|
||||
TSSEndpointData(tssi.id(), tssi.getRangeSplitPoints.getEndpoint(), metrics));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -168,8 +184,12 @@ void DatabaseContext::removeTssMapping(StorageServerInterface const& ssi) {
|
|||
queueModel.removeTssEndpoint(ssi.getValue.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKey.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKeyValues.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.watchValue.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getKeyValuesStream.getEndpoint().token.first());
|
||||
|
||||
queueModel.removeTssEndpoint(ssi.watchValue.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.splitMetrics.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getReadHotRanges.getEndpoint().token.first());
|
||||
queueModel.removeTssEndpoint(ssi.getRangeSplitPoints.getEndpoint().token.first());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1260,6 +1280,14 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
std::make_unique<DataDistributionImpl>(
|
||||
KeyRangeRef(LiteralStringRef("data_distribution/"), LiteralStringRef("data_distribution0"))
|
||||
.withPrefix(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::MANAGEMENT).begin)));
|
||||
registerSpecialKeySpaceModule(
|
||||
SpecialKeySpace::MODULE::ACTORLINEAGE,
|
||||
SpecialKeySpace::IMPLTYPE::READONLY,
|
||||
std::make_unique<ActorLineageImpl>(SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::ACTORLINEAGE)));
|
||||
registerSpecialKeySpaceModule(SpecialKeySpace::MODULE::ACTOR_PROFILER_CONF,
|
||||
SpecialKeySpace::IMPLTYPE::READWRITE,
|
||||
std::make_unique<ActorProfilerConf>(SpecialKeySpace::getModuleRange(
|
||||
SpecialKeySpace::MODULE::ACTOR_PROFILER_CONF)));
|
||||
}
|
||||
if (apiVersionAtLeast(630)) {
|
||||
registerSpecialKeySpaceModule(SpecialKeySpace::MODULE::TRANSACTION,
|
||||
|
@ -1761,6 +1789,8 @@ Database Database::createDatabase(Reference<ClusterConnectionFile> connFile,
|
|||
auto database = Database(db);
|
||||
GlobalConfig::create(
|
||||
database, Reference<AsyncVar<ClientDBInfo> const>(clientInfo), std::addressof(clientInfo->get()));
|
||||
GlobalConfig::globalConfig().trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
||||
GlobalConfig::globalConfig().trigger(samplingWindow, samplingProfilerUpdateWindow);
|
||||
return database;
|
||||
}
|
||||
|
||||
|
@ -2744,8 +2774,10 @@ ACTOR Future<Version> watchValue(Future<Version> version,
|
|||
cx->invalidateCache(key);
|
||||
wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, info.taskID));
|
||||
} else if (e.code() == error_code_watch_cancelled || e.code() == error_code_process_behind) {
|
||||
TEST(e.code() == error_code_watch_cancelled); // Too many watches on storage server, poll for changes
|
||||
// clang-format off
|
||||
TEST(e.code() == error_code_watch_cancelled); // Too many watches on the storage server, poll for changes instead
|
||||
TEST(e.code() == error_code_process_behind); // The storage servers are all behind
|
||||
// clang-format on
|
||||
wait(delay(CLIENT_KNOBS->WATCH_POLLING_TIME, info.taskID));
|
||||
} else if (e.code() == error_code_timed_out) { // The storage server occasionally times out watches in case
|
||||
// it was cancelled
|
||||
|
@ -3333,6 +3365,7 @@ ACTOR Future<RangeResult> getRange(Database cx,
|
|||
throw deterministicRandom()->randomChoice(
|
||||
std::vector<Error>{ transaction_too_old(), future_version() });
|
||||
}
|
||||
// state AnnotateActor annotation(currentLineage);
|
||||
GetKeyValuesReply _rep =
|
||||
wait(loadBalance(cx.getPtr(),
|
||||
beginServer.second,
|
||||
|
@ -4093,8 +4126,7 @@ SpanID generateSpanID(int transactionTracingEnabled) {
|
|||
}
|
||||
}
|
||||
|
||||
Transaction::Transaction()
|
||||
: info(TaskPriority::DefaultEndpoint, generateSpanID(true)), span(info.spanID, "Transaction"_loc) {}
|
||||
Transaction::Transaction() : info(TaskPriority::DefaultEndpoint, generateSpanID(true)) {}
|
||||
|
||||
Transaction::Transaction(Database const& cx)
|
||||
: info(cx->taskID, generateSpanID(cx->transactionTracingEnabled)), numErrors(0), options(cx),
|
||||
|
@ -6340,7 +6372,7 @@ void enableClientInfoLogging() {
|
|||
}
|
||||
|
||||
ACTOR Future<Void> snapCreate(Database cx, Standalone<StringRef> snapCmd, UID snapUID) {
|
||||
TraceEvent("SnapCreateEnter").detail("SnapCmd", snapCmd.toString()).detail("UID", snapUID);
|
||||
TraceEvent("SnapCreateEnter").detail("SnapCmd", snapCmd).detail("UID", snapUID);
|
||||
try {
|
||||
loop {
|
||||
choose {
|
||||
|
@ -6350,7 +6382,7 @@ ACTOR Future<Void> snapCreate(Database cx, Standalone<StringRef> snapCmd, UID sn
|
|||
ProxySnapRequest(snapCmd, snapUID, snapUID),
|
||||
cx->taskID,
|
||||
AtMostOnce::True))) {
|
||||
TraceEvent("SnapCreateExit").detail("SnapCmd", snapCmd.toString()).detail("UID", snapUID);
|
||||
TraceEvent("SnapCreateExit").detail("SnapCmd", snapCmd).detail("UID", snapUID);
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -180,6 +180,9 @@ struct TransactionInfo {
|
|||
// prefix/<key2> : '0' - any keys equal or larger than this key are (definitely) not conflicting keys
|
||||
std::shared_ptr<CoalescedKeyRangeMap<Value>> conflictingKeys;
|
||||
|
||||
// Only available so that Transaction can have a default constructor, for use in state variables
|
||||
TransactionInfo() : taskID(), spanID(), useProvisionalProxies() {}
|
||||
|
||||
explicit TransactionInfo(TaskPriority taskID, SpanID spanID)
|
||||
: taskID(taskID), spanID(spanID), useProvisionalProxies(false) {}
|
||||
};
|
||||
|
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* ProcessInterface.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/AnnotateActor.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
|
||||
constexpr UID WLTOKEN_PROCESS(-1, 12);
|
||||
|
||||
struct ProcessInterface {
|
||||
constexpr static FileIdentifier file_identifier = 985636;
|
||||
RequestStream<struct GetProcessInterfaceRequest> getInterface;
|
||||
RequestStream<struct ActorLineageRequest> actorLineage;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, actorLineage);
|
||||
}
|
||||
};
|
||||
|
||||
struct GetProcessInterfaceRequest {
|
||||
constexpr static FileIdentifier file_identifier = 7632546;
|
||||
ReplyPromise<ProcessInterface> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, reply);
|
||||
}
|
||||
};
|
||||
|
||||
// This type is used to send serialized sample data over the network.
|
||||
struct SerializedSample {
|
||||
constexpr static FileIdentifier file_identifier = 15785634;
|
||||
|
||||
double time;
|
||||
std::unordered_map<WaitState, std::string> data;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, time, data);
|
||||
}
|
||||
};
|
||||
|
||||
struct ActorLineageReply {
|
||||
constexpr static FileIdentifier file_identifier = 1887656;
|
||||
std::vector<SerializedSample> samples;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, samples);
|
||||
}
|
||||
};
|
||||
|
||||
struct ActorLineageRequest {
|
||||
constexpr static FileIdentifier file_identifier = 11654765;
|
||||
WaitState waitStateStart, waitStateEnd;
|
||||
time_t timeStart, timeEnd;
|
||||
ReplyPromise<ActorLineageReply> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, waitStateStart, waitStateEnd, timeStart, timeEnd, reply);
|
||||
}
|
||||
};
|
|
@ -21,6 +21,14 @@
|
|||
#include "boost/lexical_cast.hpp"
|
||||
#include "boost/algorithm/string.hpp"
|
||||
|
||||
#include <time.h>
|
||||
#include <msgpack.hpp>
|
||||
|
||||
#include <exception>
|
||||
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
#include "fdbclient/Knobs.h"
|
||||
#include "fdbclient/ProcessInterface.h"
|
||||
#include "fdbclient/GlobalConfig.actor.h"
|
||||
#include "fdbclient/SpecialKeySpace.actor.h"
|
||||
#include "flow/Arena.h"
|
||||
|
@ -67,7 +75,12 @@ std::unordered_map<SpecialKeySpace::MODULE, KeyRange> SpecialKeySpace::moduleToB
|
|||
{ SpecialKeySpace::MODULE::GLOBALCONFIG,
|
||||
KeyRangeRef(LiteralStringRef("\xff\xff/global_config/"), LiteralStringRef("\xff\xff/global_config0")) },
|
||||
{ SpecialKeySpace::MODULE::TRACING,
|
||||
KeyRangeRef(LiteralStringRef("\xff\xff/tracing/"), LiteralStringRef("\xff\xff/tracing0")) }
|
||||
KeyRangeRef(LiteralStringRef("\xff\xff/tracing/"), LiteralStringRef("\xff\xff/tracing0")) },
|
||||
{ SpecialKeySpace::MODULE::ACTORLINEAGE,
|
||||
KeyRangeRef(LiteralStringRef("\xff\xff/actor_lineage/"), LiteralStringRef("\xff\xff/actor_lineage0")) },
|
||||
{ SpecialKeySpace::MODULE::ACTOR_PROFILER_CONF,
|
||||
KeyRangeRef(LiteralStringRef("\xff\xff/actor_profiler_conf/"),
|
||||
LiteralStringRef("\xff\xff/actor_profiler_conf0")) }
|
||||
};
|
||||
|
||||
std::unordered_map<std::string, KeyRange> SpecialKeySpace::managementApiCommandToRange = {
|
||||
|
@ -104,6 +117,15 @@ std::unordered_map<std::string, KeyRange> SpecialKeySpace::managementApiCommandT
|
|||
.withPrefix(moduleToBoundary[MODULE::MANAGEMENT].begin) }
|
||||
};
|
||||
|
||||
std::unordered_map<std::string, KeyRange> SpecialKeySpace::actorLineageApiCommandToRange = {
|
||||
{ "state",
|
||||
KeyRangeRef(LiteralStringRef("state/"), LiteralStringRef("state0"))
|
||||
.withPrefix(moduleToBoundary[MODULE::ACTORLINEAGE].begin) },
|
||||
{ "time",
|
||||
KeyRangeRef(LiteralStringRef("time/"), LiteralStringRef("time0"))
|
||||
.withPrefix(moduleToBoundary[MODULE::ACTORLINEAGE].begin) }
|
||||
};
|
||||
|
||||
std::set<std::string> SpecialKeySpace::options = { "excluded/force",
|
||||
"failed/force",
|
||||
"excluded_locality/force",
|
||||
|
@ -476,10 +498,10 @@ void SpecialKeySpace::clear(ReadYourWritesTransaction* ryw, const KeyRangeRef& r
|
|||
auto begin = writeImpls[range.begin];
|
||||
auto end = writeImpls.rangeContainingKeyBefore(range.end)->value();
|
||||
if (begin != end) {
|
||||
TraceEvent(SevDebug, "SpecialKeySpaceCrossModuleClear").detail("Range", range.toString());
|
||||
TraceEvent(SevDebug, "SpecialKeySpaceCrossModuleClear").detail("Range", range);
|
||||
throw special_keys_cross_module_clear(); // ban cross module clear
|
||||
} else if (begin == nullptr) {
|
||||
TraceEvent(SevDebug, "SpecialKeySpaceNoWriteModuleFound").detail("Range", range.toString());
|
||||
TraceEvent(SevDebug, "SpecialKeySpaceNoWriteModuleFound").detail("Range", range);
|
||||
throw special_keys_no_write_module_found();
|
||||
}
|
||||
return begin->clear(ryw, range);
|
||||
|
@ -1918,6 +1940,287 @@ void ClientProfilingImpl::clear(ReadYourWritesTransaction* ryw, const KeyRef& ke
|
|||
"Clear operation is forbidden for profile client. You can set it to default to disable profiling.");
|
||||
}
|
||||
|
||||
ActorLineageImpl::ActorLineageImpl(KeyRangeRef kr) : SpecialKeyRangeReadImpl(kr) {}
|
||||
|
||||
void parse(StringRef& val, int& i) {
|
||||
i = std::stoi(val.toString());
|
||||
}
|
||||
|
||||
void parse(StringRef& val, double& d) {
|
||||
d = std::stod(val.toString());
|
||||
}
|
||||
|
||||
void parse(StringRef& val, WaitState& w) {
|
||||
if (val == LiteralStringRef("disk") || val == LiteralStringRef("Disk")) {
|
||||
w = WaitState::Disk;
|
||||
} else if (val == LiteralStringRef("network") || val == LiteralStringRef("Network")) {
|
||||
w = WaitState::Network;
|
||||
} else if (val == LiteralStringRef("running") || val == LiteralStringRef("Running")) {
|
||||
w = WaitState::Running;
|
||||
} else {
|
||||
throw std::range_error("failed to parse run state");
|
||||
}
|
||||
}
|
||||
|
||||
void parse(StringRef& val, time_t& t) {
|
||||
struct tm tm = { 0 };
|
||||
if (strptime(val.toString().c_str(), "%FT%T%z", &tm) == nullptr) {
|
||||
throw std::invalid_argument("failed to parse ISO 8601 datetime");
|
||||
}
|
||||
|
||||
long timezone = tm.tm_gmtoff;
|
||||
t = timegm(&tm);
|
||||
if (t == -1) {
|
||||
throw std::runtime_error("failed to convert ISO 8601 datetime");
|
||||
}
|
||||
t -= timezone;
|
||||
}
|
||||
|
||||
void parse(StringRef& val, NetworkAddress& a) {
|
||||
auto address = NetworkAddress::parse(val.toString());
|
||||
if (!address.isValid()) {
|
||||
throw std::invalid_argument("invalid host");
|
||||
}
|
||||
a = address;
|
||||
}
|
||||
|
||||
// Base case function for parsing function below.
|
||||
template <typename T>
|
||||
void parse(std::vector<StringRef>::iterator it, std::vector<StringRef>::iterator end, T& t1) {
|
||||
if (it == end) {
|
||||
return;
|
||||
}
|
||||
parse(*it, t1);
|
||||
}
|
||||
|
||||
// Given an iterator into a vector of string tokens, an iterator to the end of
|
||||
// the search space in the vector (exclusive), and a list of references to
|
||||
// types, parses each token in the vector into the associated type according to
|
||||
// the order of the arguments.
|
||||
//
|
||||
// For example, given the vector ["1", "1.5", "127.0.0.1:4000"] and the
|
||||
// argument list int a, double b, NetworkAddress c, after this function returns
|
||||
// each parameter passed in will hold the parsed value from the token list.
|
||||
//
|
||||
// The appropriate parsing function must be implemented for the type you wish
|
||||
// to parse. See the existing parsing functions above, and add your own if
|
||||
// necessary.
|
||||
template <typename T, typename... Types>
|
||||
void parse(std::vector<StringRef>::iterator it, std::vector<StringRef>::iterator end, T& t1, Types&... remaining) {
|
||||
// Return as soon as all tokens have been parsed. This allows parameters
|
||||
// passed at the end to act as optional parameters -- they will only be set
|
||||
// if the value exists.
|
||||
if (it == end) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
parse(*it, t1);
|
||||
parse(++it, end, remaining...);
|
||||
} catch (Error& e) {
|
||||
throw e;
|
||||
} catch (std::exception& e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<RangeResult> actorLineageGetRangeActor(ReadYourWritesTransaction* ryw,
|
||||
KeyRef prefix,
|
||||
KeyRangeRef kr) {
|
||||
state RangeResult result;
|
||||
|
||||
// Set default values for all fields. The default will be used if the field
|
||||
// is missing in the key.
|
||||
state NetworkAddress host;
|
||||
state WaitState waitStateStart = WaitState{ 0 };
|
||||
state WaitState waitStateEnd = WaitState{ 2 };
|
||||
state time_t timeStart = 0;
|
||||
state time_t timeEnd = std::numeric_limits<time_t>::max();
|
||||
state int seqStart = 0;
|
||||
state int seqEnd = std::numeric_limits<int>::max();
|
||||
|
||||
state std::vector<StringRef> beginValues = kr.begin.removePrefix(prefix).splitAny("/"_sr);
|
||||
state std::vector<StringRef> endValues = kr.end.removePrefix(prefix).splitAny("/"_sr);
|
||||
// Require index (either "state" or "time") and address:port.
|
||||
if (beginValues.size() < 2 || endValues.size() < 2) {
|
||||
ryw->setSpecialKeySpaceErrorMsg("missing required parameters (index, host)");
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
|
||||
state NetworkAddress endRangeHost;
|
||||
try {
|
||||
if (SpecialKeySpace::getActorLineageApiCommandRange("state").contains(kr)) {
|
||||
// For the range \xff\xff/actor_lineage/state/ip:port/wait-state/time/seq
|
||||
parse(beginValues.begin() + 1, beginValues.end(), host, waitStateStart, timeStart, seqStart);
|
||||
if (kr.begin != kr.end) {
|
||||
parse(endValues.begin() + 1, endValues.end(), endRangeHost, waitStateEnd, timeEnd, seqEnd);
|
||||
}
|
||||
} else if (SpecialKeySpace::getActorLineageApiCommandRange("time").contains(kr)) {
|
||||
// For the range \xff\xff/actor_lineage/time/ip:port/time/wait-state/seq
|
||||
parse(beginValues.begin() + 1, beginValues.end(), host, timeStart, waitStateStart, seqStart);
|
||||
if (kr.begin != kr.end) {
|
||||
parse(endValues.begin() + 1, endValues.end(), endRangeHost, timeEnd, waitStateEnd, seqEnd);
|
||||
}
|
||||
} else {
|
||||
ryw->setSpecialKeySpaceErrorMsg("invalid index in actor_lineage");
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (e.code() != special_keys_api_failure().code()) {
|
||||
ryw->setSpecialKeySpaceErrorMsg("failed to parse key");
|
||||
throw special_keys_api_failure();
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
if (kr.begin != kr.end && host != endRangeHost) {
|
||||
// The client doesn't know about all the hosts, so a get range covering
|
||||
// multiple hosts has no way of knowing which IP:port combos to use.
|
||||
ryw->setSpecialKeySpaceErrorMsg("the host must remain the same on both ends of the range");
|
||||
throw special_keys_api_failure();
|
||||
}
|
||||
|
||||
// Open endpoint to target process on each call. This can be optimized at
|
||||
// some point...
|
||||
state ProcessInterface process;
|
||||
process.getInterface = RequestStream<GetProcessInterfaceRequest>(Endpoint({ host }, WLTOKEN_PROCESS));
|
||||
ProcessInterface p = wait(retryBrokenPromise(process.getInterface, GetProcessInterfaceRequest{}));
|
||||
process = p;
|
||||
|
||||
ActorLineageRequest actorLineageRequest;
|
||||
actorLineageRequest.waitStateStart = waitStateStart;
|
||||
actorLineageRequest.waitStateEnd = waitStateEnd;
|
||||
actorLineageRequest.timeStart = timeStart;
|
||||
actorLineageRequest.timeEnd = timeEnd;
|
||||
ActorLineageReply reply = wait(process.actorLineage.getReply(actorLineageRequest));
|
||||
|
||||
time_t dt = 0;
|
||||
int seq = -1;
|
||||
for (const auto& sample : reply.samples) {
|
||||
time_t datetime = (time_t)sample.time;
|
||||
char buf[50];
|
||||
struct tm* tm;
|
||||
tm = localtime(&datetime);
|
||||
size_t size = strftime(buf, 50, "%FT%T%z", tm);
|
||||
std::string date(buf, size);
|
||||
|
||||
seq = dt == datetime ? seq + 1 : 0;
|
||||
dt = datetime;
|
||||
|
||||
for (const auto& [waitState, data] : sample.data) {
|
||||
if (seq < seqStart) { continue; }
|
||||
else if (seq >= seqEnd) { break; }
|
||||
|
||||
std::ostringstream streamKey;
|
||||
if (SpecialKeySpace::getActorLineageApiCommandRange("state").contains(kr)) {
|
||||
streamKey << SpecialKeySpace::getActorLineageApiCommandPrefix("state").toString() << host.toString()
|
||||
<< "/" << to_string(waitState) << "/" << date;
|
||||
} else if (SpecialKeySpace::getActorLineageApiCommandRange("time").contains(kr)) {
|
||||
streamKey << SpecialKeySpace::getActorLineageApiCommandPrefix("time").toString() << host.toString()
|
||||
<< "/" << date << "/" << to_string(waitState);
|
||||
} else {
|
||||
ASSERT(false);
|
||||
}
|
||||
streamKey << "/" << seq;
|
||||
|
||||
msgpack::object_handle oh = msgpack::unpack(data.data(), data.size());
|
||||
msgpack::object deserialized = oh.get();
|
||||
|
||||
std::ostringstream stream;
|
||||
stream << deserialized;
|
||||
|
||||
result.push_back_deep(result.arena(), KeyValueRef(streamKey.str(), stream.str()));
|
||||
}
|
||||
|
||||
if (sample.data.size() == 0) {
|
||||
std::ostringstream streamKey;
|
||||
if (SpecialKeySpace::getActorLineageApiCommandRange("state").contains(kr)) {
|
||||
streamKey << SpecialKeySpace::getActorLineageApiCommandPrefix("state").toString() << host.toString()
|
||||
<< "/Running/" << date;
|
||||
} else if (SpecialKeySpace::getActorLineageApiCommandRange("time").contains(kr)) {
|
||||
streamKey << SpecialKeySpace::getActorLineageApiCommandPrefix("time").toString() << host.toString()
|
||||
<< "/" << date << "/Running";
|
||||
} else {
|
||||
ASSERT(false);
|
||||
}
|
||||
streamKey << "/" << seq;
|
||||
result.push_back_deep(result.arena(), KeyValueRef(streamKey.str(), "{}"_sr));
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
Future<RangeResult> ActorLineageImpl::getRange(ReadYourWritesTransaction* ryw, KeyRangeRef kr) const {
|
||||
return actorLineageGetRangeActor(ryw, getKeyRange().begin, kr);
|
||||
}
|
||||
|
||||
namespace {
|
||||
std::string_view to_string_view(StringRef sr) {
|
||||
return std::string_view(reinterpret_cast<const char*>(sr.begin()), sr.size());
|
||||
}
|
||||
} // namespace
|
||||
|
||||
ActorProfilerConf::ActorProfilerConf(KeyRangeRef kr)
|
||||
: SpecialKeyRangeRWImpl(kr), config(ProfilerConfig::instance().getConfig()) {}
|
||||
|
||||
Future<RangeResult> ActorProfilerConf::getRange(ReadYourWritesTransaction* ryw, KeyRangeRef kr) const {
|
||||
RangeResult res;
|
||||
std::string_view begin(to_string_view(kr.begin.removePrefix(range.begin))),
|
||||
end(to_string_view(kr.end.removePrefix(range.begin)));
|
||||
for (auto& p : config) {
|
||||
if (p.first > end) {
|
||||
break;
|
||||
} else if (p.first > begin) {
|
||||
KeyValueRef kv;
|
||||
kv.key = StringRef(res.arena(), p.first).withPrefix(kr.begin, res.arena());
|
||||
kv.value = StringRef(res.arena(), p.second);
|
||||
res.push_back(res.arena(), kv);
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
void ActorProfilerConf::set(ReadYourWritesTransaction* ryw, const KeyRef& key, const ValueRef& value) {
|
||||
config[key.removePrefix(range.begin).toString()] = value.toString();
|
||||
ryw->getSpecialKeySpaceWriteMap().insert(key, std::make_pair(true, Optional<Value>(value)));
|
||||
didWrite = true;
|
||||
}
|
||||
|
||||
void ActorProfilerConf::clear(ReadYourWritesTransaction* ryw, const KeyRangeRef& kr) {
|
||||
std::string begin(kr.begin.removePrefix(range.begin).toString()), end(kr.end.removePrefix(range.begin).toString());
|
||||
auto first = config.lower_bound(begin);
|
||||
if (first == config.end()) {
|
||||
// nothing to clear
|
||||
return;
|
||||
}
|
||||
didWrite = true;
|
||||
auto last = config.upper_bound(end);
|
||||
config.erase(first, last);
|
||||
}
|
||||
|
||||
void ActorProfilerConf::clear(ReadYourWritesTransaction* ryw, const KeyRef& key) {
|
||||
std::string k = key.removePrefix(range.begin).toString();
|
||||
auto iter = config.find(k);
|
||||
if (iter != config.end()) {
|
||||
config.erase(iter);
|
||||
}
|
||||
didWrite = true;
|
||||
}
|
||||
|
||||
Future<Optional<std::string>> ActorProfilerConf::commit(ReadYourWritesTransaction* ryw) {
|
||||
Optional<std::string> res{};
|
||||
try {
|
||||
if (didWrite) {
|
||||
ProfilerConfig::instance().reset(config);
|
||||
}
|
||||
return res;
|
||||
} catch (ConfigError& err) {
|
||||
return Optional<std::string>{ err.description };
|
||||
}
|
||||
}
|
||||
|
||||
MaintenanceImpl::MaintenanceImpl(KeyRangeRef kr) : SpecialKeyRangeRWImpl(kr) {}
|
||||
|
||||
// Used to read the healthZoneKey
|
||||
|
|
|
@ -140,6 +140,8 @@ public:
|
|||
class SpecialKeySpace {
|
||||
public:
|
||||
enum class MODULE {
|
||||
ACTORLINEAGE, // Sampling data
|
||||
ACTOR_PROFILER_CONF, // profiler configuration
|
||||
CLUSTERFILEPATH,
|
||||
CONFIGURATION, // Configuration of the cluster
|
||||
CONNECTIONSTRING,
|
||||
|
@ -197,6 +199,12 @@ public:
|
|||
static KeyRef getManagementApiCommandPrefix(const std::string& command) {
|
||||
return managementApiCommandToRange.at(command).begin;
|
||||
}
|
||||
static KeyRangeRef getActorLineageApiCommandRange(const std::string& command) {
|
||||
return actorLineageApiCommandToRange.at(command);
|
||||
}
|
||||
static KeyRef getActorLineageApiCommandPrefix(const std::string& command) {
|
||||
return actorLineageApiCommandToRange.at(command).begin;
|
||||
}
|
||||
static Key getManagementApiCommandOptionSpecialKey(const std::string& command, const std::string& option);
|
||||
static const std::set<std::string>& getManagementApiOptionsSet() { return options; }
|
||||
static const std::set<std::string>& getTracingOptions() { return tracingOptions; }
|
||||
|
@ -225,6 +233,7 @@ private:
|
|||
static std::unordered_map<SpecialKeySpace::MODULE, KeyRange> moduleToBoundary;
|
||||
static std::unordered_map<std::string, KeyRange>
|
||||
managementApiCommandToRange; // management command to its special keys' range
|
||||
static std::unordered_map<std::string, KeyRange> actorLineageApiCommandToRange;
|
||||
static std::set<std::string> options; // "<command>/<option>"
|
||||
static std::set<std::string> tracingOptions;
|
||||
|
||||
|
@ -408,12 +417,32 @@ public:
|
|||
void clear(ReadYourWritesTransaction* ryw, const KeyRef& key) override;
|
||||
};
|
||||
|
||||
class ActorLineageImpl : public SpecialKeyRangeReadImpl {
|
||||
public:
|
||||
explicit ActorLineageImpl(KeyRangeRef kr);
|
||||
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw, KeyRangeRef kr) const override;
|
||||
};
|
||||
|
||||
class ActorProfilerConf : public SpecialKeyRangeRWImpl {
|
||||
bool didWrite = false;
|
||||
std::map<std::string, std::string> config;
|
||||
|
||||
public:
|
||||
explicit ActorProfilerConf(KeyRangeRef kr);
|
||||
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw, KeyRangeRef kr) const override;
|
||||
void set(ReadYourWritesTransaction* ryw, const KeyRef& key, const ValueRef& value) override;
|
||||
void clear(ReadYourWritesTransaction* ryw, const KeyRangeRef& range) override;
|
||||
void clear(ReadYourWritesTransaction* ryw, const KeyRef& key) override;
|
||||
Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override;
|
||||
};
|
||||
|
||||
class MaintenanceImpl : public SpecialKeyRangeRWImpl {
|
||||
public:
|
||||
explicit MaintenanceImpl(KeyRangeRef kr);
|
||||
Future<RangeResult> getRange(ReadYourWritesTransaction* ryw, KeyRangeRef kr) const override;
|
||||
Future<Optional<std::string>> commit(ReadYourWritesTransaction* ryw) override;
|
||||
};
|
||||
|
||||
class DataDistributionImpl : public SpecialKeyRangeRWImpl {
|
||||
public:
|
||||
explicit DataDistributionImpl(KeyRangeRef kr);
|
||||
|
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* StackLineage.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/StackLineage.h"
|
||||
|
||||
std::vector<StringRef> getActorStackTrace() {
|
||||
return (*currentLineage)->stack(&StackLineage::actorName);
|
||||
}
|
||||
|
||||
namespace {
|
||||
// StackLineageCollector stackLineageCollector;
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* StackLineage.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <string_view>
|
||||
|
||||
#include "flow/flow.h"
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
|
||||
extern std::vector<StringRef> getActorStackTrace();
|
||||
|
||||
struct StackLineageCollector : IALPCollector<StackLineage> {
|
||||
StackLineageCollector() : IALPCollector() {}
|
||||
std::optional<std::any> collect(ActorLineage* lineage) override {
|
||||
auto vec = lineage->stack(&StackLineage::actorName);
|
||||
|
||||
std::vector<std::string_view> res;
|
||||
for (const auto& str : vec) {
|
||||
res.push_back(std::string_view(reinterpret_cast<const char*>(str.begin()), str.size()));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
};
|
|
@ -210,6 +210,66 @@ void TSS_traceMismatch(TraceEvent& event,
|
|||
ASSERT(false);
|
||||
}
|
||||
|
||||
template <>
|
||||
bool TSS_doCompare(const SplitMetricsReply& src, const SplitMetricsReply& tss) {
|
||||
// We duplicate split metrics just for load, no need to validate replies.
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const SplitMetricsRequest& req) {
|
||||
ASSERT(false);
|
||||
return "";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const SplitMetricsRequest& req,
|
||||
const SplitMetricsReply& src,
|
||||
const SplitMetricsReply& tss) {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
template <>
|
||||
bool TSS_doCompare(const ReadHotSubRangeReply& src, const ReadHotSubRangeReply& tss) {
|
||||
// We duplicate read hot sub range metrics just for load, no need to validate replies.
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const ReadHotSubRangeRequest& req) {
|
||||
ASSERT(false);
|
||||
return "";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const ReadHotSubRangeRequest& req,
|
||||
const ReadHotSubRangeReply& src,
|
||||
const ReadHotSubRangeReply& tss) {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
template <>
|
||||
bool TSS_doCompare(const SplitRangeReply& src, const SplitRangeReply& tss) {
|
||||
// We duplicate read hot sub range metrics just for load, no need to validate replies.
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const SplitRangeRequest& req) {
|
||||
ASSERT(false);
|
||||
return "";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const SplitRangeRequest& req,
|
||||
const SplitRangeReply& src,
|
||||
const SplitRangeReply& tss) {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
// template specializations for metrics replies that should never be called because these requests aren't duplicated
|
||||
|
||||
// storage metrics
|
||||
|
@ -233,69 +293,6 @@ void TSS_traceMismatch(TraceEvent& event,
|
|||
ASSERT(false);
|
||||
}
|
||||
|
||||
// split metrics
|
||||
template <>
|
||||
bool TSS_doCompare(const SplitMetricsReply& src, const SplitMetricsReply& tss) {
|
||||
ASSERT(false);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const SplitMetricsRequest& req) {
|
||||
ASSERT(false);
|
||||
return "";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const SplitMetricsRequest& req,
|
||||
const SplitMetricsReply& src,
|
||||
const SplitMetricsReply& tss) {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
// read hot sub range
|
||||
template <>
|
||||
bool TSS_doCompare(const ReadHotSubRangeReply& src, const ReadHotSubRangeReply& tss) {
|
||||
ASSERT(false);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const ReadHotSubRangeRequest& req) {
|
||||
ASSERT(false);
|
||||
return "";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const ReadHotSubRangeRequest& req,
|
||||
const ReadHotSubRangeReply& src,
|
||||
const ReadHotSubRangeReply& tss) {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
// split range
|
||||
template <>
|
||||
bool TSS_doCompare(const SplitRangeReply& src, const SplitRangeReply& tss) {
|
||||
ASSERT(false);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* TSS_mismatchTraceName(const SplitRangeRequest& req) {
|
||||
ASSERT(false);
|
||||
return "";
|
||||
}
|
||||
|
||||
template <>
|
||||
void TSS_traceMismatch(TraceEvent& event,
|
||||
const SplitRangeRequest& req,
|
||||
const SplitRangeReply& src,
|
||||
const SplitRangeReply& tss) {
|
||||
ASSERT(false);
|
||||
}
|
||||
|
||||
// only record metrics for data reads
|
||||
|
||||
template <>
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* TransactionLineage.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbclient/TransactionLineage.h"
|
||||
|
||||
namespace {
|
||||
TransactionLineageCollector transactionLineageCollector;
|
||||
}
|
|
@ -0,0 +1,130 @@
|
|||
/*
|
||||
* TransactionLineage.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
|
||||
struct TransactionLineage : LineageProperties<TransactionLineage> {
|
||||
enum class Operation {
|
||||
Unset,
|
||||
GetValue,
|
||||
GetKey,
|
||||
GetKeyValues,
|
||||
WatchValue,
|
||||
GetConsistentReadVersion,
|
||||
Commit,
|
||||
GetKeyServersLocations
|
||||
};
|
||||
static constexpr std::string_view name = "Transaction"sv;
|
||||
uint64_t txID;
|
||||
Operation operation = Operation::Unset;
|
||||
|
||||
bool isSet(uint64_t TransactionLineage::*member) const { return this->*member > 0; }
|
||||
bool isSet(Operation TransactionLineage::*member) const { return this->*member != Operation::Unset; }
|
||||
};
|
||||
|
||||
struct TransactionLineageCollector : IALPCollector<TransactionLineage> {
|
||||
using Operation = TransactionLineage::Operation;
|
||||
std::optional<std::any> collect(ActorLineage* lineage) {
|
||||
std::map<std::string_view, std::any> res;
|
||||
auto txID = lineage->get(&TransactionLineage::txID);
|
||||
if (txID.has_value()) {
|
||||
res["ID"sv] = txID.value();
|
||||
}
|
||||
auto operation = lineage->get(&TransactionLineage::operation);
|
||||
if (operation.has_value()) {
|
||||
switch (operation.value()) {
|
||||
case Operation::Unset:
|
||||
res["operation"sv] = "Unset"sv;
|
||||
break;
|
||||
case Operation::GetValue:
|
||||
res["operation"sv] = "GetValue"sv;
|
||||
break;
|
||||
case Operation::GetKey:
|
||||
res["operation"sv] = "GetKey"sv;
|
||||
break;
|
||||
case Operation::GetKeyValues:
|
||||
res["operation"sv] = "GetKeyValues"sv;
|
||||
break;
|
||||
case Operation::WatchValue:
|
||||
res["operation"sv] = "WatchValue"sv;
|
||||
break;
|
||||
case Operation::GetConsistentReadVersion:
|
||||
res["operation"sv] = "GetConsistentReadVersion"sv;
|
||||
break;
|
||||
case Operation::Commit:
|
||||
res["operation"sv] = "Commit"sv;
|
||||
break;
|
||||
case Operation::GetKeyServersLocations:
|
||||
res["operation"sv] = "GetKeyServersLocations"sv;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (res.empty()) {
|
||||
return std::optional<std::any>{};
|
||||
} else {
|
||||
return res;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
template <class T, class V>
|
||||
class ScopedLineage {
|
||||
V before;
|
||||
V T::*member;
|
||||
bool valid = true;
|
||||
|
||||
public:
|
||||
ScopedLineage(V T::*member, V const& value) : member(member) {
|
||||
auto& val = getCurrentLineage()->modify(member);
|
||||
before = val;
|
||||
val = value;
|
||||
}
|
||||
~ScopedLineage() {
|
||||
if (!valid) {
|
||||
return;
|
||||
}
|
||||
getCurrentLineage()->modify(member) = before;
|
||||
}
|
||||
ScopedLineage(ScopedLineage<T, V>&& o) : before(std::move(o.before)), member(o.member), valid(o.valid) {
|
||||
o.release();
|
||||
}
|
||||
ScopedLineage& operator=(ScopedLineage<T, V>&& o) {
|
||||
if (valid) {
|
||||
getCurrentLineage()->modify(member) = before;
|
||||
}
|
||||
before = std::move(o.before);
|
||||
member = o.member;
|
||||
valid = o.valid;
|
||||
o.release();
|
||||
return *this;
|
||||
}
|
||||
ScopedLineage(const ScopedLineage<T, V>&) = delete;
|
||||
ScopedLineage& operator=(const ScopedLineage<T, V>&) = delete;
|
||||
void release() { valid = false; }
|
||||
};
|
||||
|
||||
template <class T, class V>
|
||||
ScopedLineage<T, V> make_scoped_lineage(V T::*member, V const& value) {
|
||||
return ScopedLineage<T, V>(member, value);
|
||||
}
|
||||
#endif
|
|
@ -244,8 +244,8 @@ public:
|
|||
TraceEvent("AsyncFileCachedDel")
|
||||
.detail("Filename", filename)
|
||||
.detail("Refcount", debugGetReferenceCount())
|
||||
.detail("CanDie", f.isReady())
|
||||
.backtrace();
|
||||
.detail("CanDie", f.isReady());
|
||||
// .backtrace();
|
||||
if (f.isReady())
|
||||
delete this;
|
||||
else
|
||||
|
|
|
@ -242,7 +242,12 @@ public:
|
|||
// result = map(result, [=](int r) mutable { KAIOLogBlockEvent(io, OpLogEntry::READY, r); return r; });
|
||||
#endif
|
||||
|
||||
return success(result);
|
||||
// auto& actorLineageSet = IAsyncFileSystem::filesystem()->getActorLineageSet();
|
||||
// auto index = actorLineageSet.insert(*currentLineage);
|
||||
// ASSERT(index != ActorLineageSet::npos);
|
||||
Future<Void> res = success(result);
|
||||
// actorLineageSet.erase(index);
|
||||
return res;
|
||||
}
|
||||
// TODO(alexmiller): Remove when we upgrade the dev docker image to >14.10
|
||||
#ifndef FALLOC_FL_ZERO_RANGE
|
||||
|
|
|
@ -63,6 +63,14 @@ add_flow_target(STATIC_LIBRARY NAME fdbrpc
|
|||
DISABLE_ACTOR_DIAGNOSTICS ${FDBRPC_SRCS_DISABLE_ACTOR_DIAGNOSTICS})
|
||||
target_include_directories(fdbrpc PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/libeio)
|
||||
target_link_libraries(fdbrpc PUBLIC flow)
|
||||
|
||||
add_flow_target(STATIC_LIBRARY NAME fdbrpc_sampling
|
||||
SRCS ${FDBRPC_SRCS}
|
||||
DISABLE_ACTOR_DIAGNOSTICS ${FDBRPC_SRCS_DISABLE_ACTOR_DIAGNOSTICS})
|
||||
target_include_directories(fdbrpc_sampling PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/libeio)
|
||||
target_link_libraries(fdbrpc_sampling PUBLIC flow_sampling)
|
||||
target_compile_definitions(fdbrpc_sampling PRIVATE -DENABLE_SAMPLING)
|
||||
|
||||
if(COMPILE_EIO)
|
||||
add_library(eio STATIC libeio/eio.c)
|
||||
if(USE_VALGRIND)
|
||||
|
@ -71,8 +79,10 @@ if(COMPILE_EIO)
|
|||
target_compile_definitions(eio PRIVATE USE_UCONTEXT)
|
||||
target_compile_options(eio BEFORE PRIVATE -w) # disable warnings for eio
|
||||
target_link_libraries(fdbrpc PRIVATE eio)
|
||||
target_link_libraries(fdbrpc_sampling PRIVATE eio)
|
||||
endif()
|
||||
if(WIN32)
|
||||
add_library(coro STATIC libcoroutine/Common.c libcoroutine/Coro.c)
|
||||
target_link_libraries(fdbrpc PRIVATE coro)
|
||||
target_link_libraries(fdbrpc_sampling PRIVATE coro)
|
||||
endif()
|
||||
|
|
|
@ -24,6 +24,7 @@
|
|||
#include "flow/UnitTest.h"
|
||||
#include "flow/DeterministicRandom.h"
|
||||
#include "flow/IThreadPool.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
#include "fdbrpc/IAsyncFile.h"
|
||||
#include "flow/TLSConfig.actor.h"
|
||||
|
@ -285,6 +286,11 @@ struct YieldMockNetwork final : INetwork, ReferenceCounted<YieldMockNetwork> {
|
|||
static TLSConfig emptyConfig;
|
||||
return emptyConfig;
|
||||
}
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& getActorLineageSet() override {
|
||||
throw std::exception();
|
||||
}
|
||||
#endif
|
||||
ProtocolVersion protocolVersion() override { return baseNetwork->protocolVersion(); }
|
||||
};
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@
|
|||
|
||||
#include <ctime>
|
||||
#include "flow/flow.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
#include "fdbrpc/IRateControl.h"
|
||||
|
||||
// All outstanding operations must be cancelled before the destructor of IAsyncFile is called.
|
||||
|
@ -119,6 +120,11 @@ public:
|
|||
// Returns the time of the last modification of the file.
|
||||
virtual Future<std::time_t> lastWriteTime(const std::string& filename) = 0;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
// Returns the shared memory data structure used to store actor lineages.
|
||||
virtual ActorLineageSet& getActorLineageSet() = 0;
|
||||
#endif
|
||||
|
||||
static IAsyncFileSystem* filesystem() { return filesystem(g_network); }
|
||||
static runCycleFuncPtr runCycleFunc() {
|
||||
return reinterpret_cast<runCycleFuncPtr>(
|
||||
|
|
|
@ -71,6 +71,7 @@ struct ProcessClass {
|
|||
Ratekeeper,
|
||||
StorageCache,
|
||||
Backup,
|
||||
Worker, // used for actor lineage tracking
|
||||
NoRole
|
||||
};
|
||||
enum ClassSource { CommandLineSource, AutoSource, DBSource, InvalidSource = -1 };
|
||||
|
|
|
@ -98,6 +98,12 @@ Future<std::time_t> Net2FileSystem::lastWriteTime(const std::string& filename) {
|
|||
return Net2AsyncFile::lastWriteTime(filename);
|
||||
}
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& Net2FileSystem::getActorLineageSet() {
|
||||
return actorLineageSet;
|
||||
}
|
||||
#endif
|
||||
|
||||
void Net2FileSystem::newFileSystem(double ioTimeout, const std::string& fileSystemPath) {
|
||||
g_network->setGlobal(INetwork::enFileSystem, (flowGlobalType) new Net2FileSystem(ioTimeout, fileSystemPath));
|
||||
}
|
||||
|
|
|
@ -39,6 +39,10 @@ public:
|
|||
|
||||
Future<Void> renameFile(std::string const& from, std::string const& to) override;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& getActorLineageSet() override;
|
||||
#endif
|
||||
|
||||
// void init();
|
||||
static void stop();
|
||||
|
||||
|
@ -52,6 +56,9 @@ public:
|
|||
dev_t fileSystemDeviceId;
|
||||
bool checkFileSystem;
|
||||
#endif
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet actorLineageSet;
|
||||
#endif
|
||||
};
|
||||
|
||||
#endif
|
||||
|
|
|
@ -31,6 +31,7 @@
|
|||
#include "flow/IThreadPool.h"
|
||||
#include "flow/ProtocolVersion.h"
|
||||
#include "flow/Util.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
#include "fdbrpc/IAsyncFile.h"
|
||||
#include "fdbrpc/AsyncFileCached.actor.h"
|
||||
#include "fdbrpc/AsyncFileEncrypted.h"
|
||||
|
@ -984,6 +985,12 @@ public:
|
|||
|
||||
bool checkRunnable() override { return net2->checkRunnable(); }
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& getActorLineageSet() override {
|
||||
return actorLineageSet;
|
||||
}
|
||||
#endif
|
||||
|
||||
void stop() override { isStopped = true; }
|
||||
void addStopCallback(std::function<void()> fn) override { stopCallbacks.emplace_back(std::move(fn)); }
|
||||
bool isSimulated() const override { return true; }
|
||||
|
@ -2127,6 +2134,10 @@ public:
|
|||
// Whether or not yield has returned true during the current iteration of the run loop
|
||||
bool yielded;
|
||||
int yield_limit; // how many more times yield may return false before next returning true
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet actorLineageSet;
|
||||
#endif
|
||||
};
|
||||
|
||||
class UDPSimSocket : public IUDPSocket, ReferenceCounted<UDPSimSocket> {
|
||||
|
@ -2519,6 +2530,12 @@ Future<std::time_t> Sim2FileSystem::lastWriteTime(const std::string& filename) {
|
|||
return fileWrites[filename];
|
||||
}
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& Sim2FileSystem::getActorLineageSet() {
|
||||
return actorLineageSet;
|
||||
}
|
||||
#endif
|
||||
|
||||
void Sim2FileSystem::newFileSystem() {
|
||||
g_network->setGlobal(INetwork::enFileSystem, (flowGlobalType) new Sim2FileSystem());
|
||||
}
|
||||
|
|
|
@ -481,6 +481,10 @@ public:
|
|||
|
||||
Future<std::time_t> lastWriteTime(const std::string& filename) override;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& getActorLineageSet() override;
|
||||
#endif
|
||||
|
||||
Future<Void> renameFile(std::string const& from, std::string const& to) override;
|
||||
|
||||
Sim2FileSystem() {}
|
||||
|
@ -488,6 +492,10 @@ public:
|
|||
~Sim2FileSystem() override {}
|
||||
|
||||
static void newFileSystem();
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet actorLineageSet;
|
||||
#endif
|
||||
};
|
||||
|
||||
#endif
|
||||
|
|
|
@ -131,8 +131,8 @@ void applyMetadataMutations(SpanID const& spanContext,
|
|||
MutationRef privatized = m;
|
||||
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
|
||||
TraceEvent(SevDebug, "SendingPrivateMutation", dbgid)
|
||||
.detail("Original", m.toString())
|
||||
.detail("Privatized", privatized.toString())
|
||||
.detail("Original", m)
|
||||
.detail("Privatized", privatized)
|
||||
.detail("Server", serverKeysDecodeServer(m.param1))
|
||||
.detail("TagKey", serverTagKeyFor(serverKeysDecodeServer(m.param1)))
|
||||
.detail("Tag", tag.toString());
|
||||
|
@ -218,8 +218,8 @@ void applyMetadataMutations(SpanID const& spanContext,
|
|||
(!m.param1.startsWith(failedLocalityPrefix) && m.param1 != failedLocalityVersionKey)) {
|
||||
auto t = txnStateStore->readValue(m.param1).get();
|
||||
TraceEvent("MutationRequiresRestart", dbgid)
|
||||
.detail("M", m.toString())
|
||||
.detail("PrevValue", t.present() ? t.get() : LiteralStringRef("(none)"))
|
||||
.detail("M", m)
|
||||
.detail("PrevValue", t.orDefault("(none)"_sr))
|
||||
.detail("ToCommit", toCommit != nullptr);
|
||||
confChange = true;
|
||||
}
|
||||
|
@ -431,7 +431,7 @@ void applyMetadataMutations(SpanID const& spanContext,
|
|||
txnStateStore->clear(range & configKeys);
|
||||
if (!excludedServersKeys.contains(range) && !failedServersKeys.contains(range) &&
|
||||
!excludedLocalityKeys.contains(range) && !failedLocalityKeys.contains(range)) {
|
||||
TraceEvent("MutationRequiresRestart", dbgid).detail("M", m.toString());
|
||||
TraceEvent("MutationRequiresRestart", dbgid).detail("M", m);
|
||||
confChange = true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -744,7 +744,6 @@ ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int
|
|||
|
||||
DEBUG_MUTATION("addMutation", message.version.version, m)
|
||||
.detail("Version", message.version.toString())
|
||||
.detail("Mutation", m)
|
||||
.detail("KCV", self->minKnownCommittedVersion)
|
||||
.detail("SavedVersion", self->savedVersion);
|
||||
|
||||
|
|
|
@ -99,8 +99,11 @@ set(FDBSERVER_SRCS
|
|||
RestoreWorkerInterface.actor.h
|
||||
Resolver.actor.cpp
|
||||
ResolverInterface.h
|
||||
RoleLineage.actor.h
|
||||
RoleLineage.actor.cpp
|
||||
ServerDBInfo.actor.h
|
||||
ServerDBInfo.h
|
||||
SigStack.cpp
|
||||
SimpleConfigConsumer.actor.cpp
|
||||
SimpleConfigConsumer.h
|
||||
SimulatedCluster.actor.cpp
|
||||
|
@ -305,6 +308,7 @@ else()
|
|||
endif()
|
||||
|
||||
target_link_libraries(fdbserver PRIVATE toml11_target jemalloc)
|
||||
# target_compile_definitions(fdbserver PRIVATE -DENABLE_SAMPLING)
|
||||
|
||||
if (GPERFTOOLS_FOUND)
|
||||
target_link_libraries(fdbserver PRIVATE gperftools)
|
||||
|
|
|
@ -3148,7 +3148,6 @@ ACTOR Future<Void> clusterWatchDatabase(ClusterControllerData* cluster, ClusterC
|
|||
dbInfo.distributor = db->serverInfo->get().distributor;
|
||||
dbInfo.ratekeeper = db->serverInfo->get().ratekeeper;
|
||||
dbInfo.latencyBandConfig = db->serverInfo->get().latencyBandConfig;
|
||||
dbInfo.configBroadcaster = db->serverInfo->get().configBroadcaster;
|
||||
|
||||
TraceEvent("CCWDB", cluster->id)
|
||||
.detail("Lifetime", dbInfo.masterLifetime.toString())
|
||||
|
@ -3734,7 +3733,7 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co
|
|||
checkOutstandingRequests(self);
|
||||
}
|
||||
|
||||
void registerWorker(RegisterWorkerRequest req, ClusterControllerData* self) {
|
||||
void registerWorker(RegisterWorkerRequest req, ClusterControllerData* self, ConfigBroadcaster* configBroadcaster) {
|
||||
const WorkerInterface& w = req.wi;
|
||||
ProcessClass newProcessClass = req.processClass;
|
||||
auto info = self->id_worker.find(w.locality.processId());
|
||||
|
@ -3823,6 +3822,13 @@ void registerWorker(RegisterWorkerRequest req, ClusterControllerData* self) {
|
|||
w.locality.processId() == self->db.serverInfo->get().master.locality.processId()) {
|
||||
self->masterProcessId = w.locality.processId();
|
||||
}
|
||||
if (configBroadcaster != nullptr) {
|
||||
self->addActor.send(configBroadcaster->registerWorker(
|
||||
req.lastSeenKnobVersion,
|
||||
req.knobConfigClassSet,
|
||||
self->id_worker[w.locality.processId()].watcher,
|
||||
self->id_worker[w.locality.processId()].details.interf.configBroadcastInterface));
|
||||
}
|
||||
checkOutstandingRequests(self);
|
||||
} else if (info->second.details.interf.id() != w.id() || req.generation >= info->second.gen) {
|
||||
if (!info->second.reply.isSet()) {
|
||||
|
@ -3841,6 +3847,13 @@ void registerWorker(RegisterWorkerRequest req, ClusterControllerData* self) {
|
|||
info->second.details.interf = w;
|
||||
info->second.watcher = workerAvailabilityWatch(w, newProcessClass, self);
|
||||
}
|
||||
if (configBroadcaster != nullptr) {
|
||||
self->addActor.send(
|
||||
configBroadcaster->registerWorker(req.lastSeenKnobVersion,
|
||||
req.knobConfigClassSet,
|
||||
info->second.watcher,
|
||||
info->second.details.interf.configBroadcastInterface));
|
||||
}
|
||||
checkOutstandingRequests(self);
|
||||
} else {
|
||||
TEST(true); // Received an old worker registration request.
|
||||
|
@ -4782,9 +4795,6 @@ ACTOR Future<Void> clusterControllerCore(ClusterControllerFullInterface interf,
|
|||
state uint64_t step = 0;
|
||||
state Future<ErrorOr<Void>> error = errorOr(actorCollection(self.addActor.getFuture()));
|
||||
|
||||
if (configDBType != ConfigDBType::DISABLED) {
|
||||
self.addActor.send(configBroadcaster.serve(self.db.serverInfo->get().configBroadcaster));
|
||||
}
|
||||
self.addActor.send(clusterWatchDatabase(&self, &self.db)); // Start the master database
|
||||
self.addActor.send(self.updateWorkerList.init(self.db.db));
|
||||
self.addActor.send(statusServer(interf.clientInterface.databaseStatus.getFuture(),
|
||||
|
@ -4842,7 +4852,7 @@ ACTOR Future<Void> clusterControllerCore(ClusterControllerFullInterface interf,
|
|||
}
|
||||
when(RegisterWorkerRequest req = waitNext(interf.registerWorker.getFuture())) {
|
||||
++self.registerWorkerRequests;
|
||||
registerWorker(req, &self);
|
||||
registerWorker(req, &self, (configDBType == ConfigDBType::DISABLED) ? nullptr : &configBroadcaster);
|
||||
}
|
||||
when(GetWorkersRequest req = waitNext(interf.getWorkers.getFuture())) {
|
||||
++self.getWorkersRequests;
|
||||
|
|
|
@ -28,6 +28,7 @@
|
|||
#include "fdbclient/CommitProxyInterface.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/TransactionLineage.h"
|
||||
#include "fdbrpc/sim_validation.h"
|
||||
#include "fdbserver/ApplyMetadataMutation.h"
|
||||
#include "fdbserver/ConflictSet.h"
|
||||
|
@ -953,10 +954,8 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
pProxyCommitData->singleKeyMutationEvent->log();
|
||||
}
|
||||
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m)
|
||||
.detail("Dbgid", pProxyCommitData->dbgid)
|
||||
.detail("To", tags)
|
||||
.detail("Mutation", m);
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m, pProxyCommitData->dbgid)
|
||||
.detail("To", tags);
|
||||
self->toCommit.addTags(tags);
|
||||
if (pProxyCommitData->cacheInfo[m.param1]) {
|
||||
self->toCommit.addTag(cacheTag);
|
||||
|
@ -969,10 +968,8 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
++firstRange;
|
||||
if (firstRange == ranges.end()) {
|
||||
// Fast path
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m)
|
||||
.detail("Dbgid", pProxyCommitData->dbgid)
|
||||
.detail("To", ranges.begin().value().tags)
|
||||
.detail("Mutation", m);
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m, pProxyCommitData->dbgid)
|
||||
.detail("To", ranges.begin().value().tags);
|
||||
|
||||
ranges.begin().value().populateTags();
|
||||
self->toCommit.addTags(ranges.begin().value().tags);
|
||||
|
@ -1007,10 +1004,8 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
trCost->get().clearIdxCosts.pop_front();
|
||||
}
|
||||
}
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m)
|
||||
.detail("Dbgid", pProxyCommitData->dbgid)
|
||||
.detail("To", allSources)
|
||||
.detail("Mutation", m);
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m, pProxyCommitData->dbgid)
|
||||
.detail("To", allSources);
|
||||
|
||||
self->toCommit.addTags(allSources);
|
||||
}
|
||||
|
@ -1418,6 +1413,7 @@ ACTOR Future<Void> commitBatch(ProxyCommitData* self,
|
|||
// WARNING: this code is run at a high priority (until the first delay(0)), so it needs to do as little work as
|
||||
// possible
|
||||
state CommitBatch::CommitBatchContext context(self, trs, currentBatchMemBytesCount);
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::Commit;
|
||||
|
||||
// Active load balancing runs at a very high priority (to obtain accurate estimate of memory used by commit batches)
|
||||
// so we need to downgrade here
|
||||
|
@ -1468,6 +1464,8 @@ void maybeAddTssMapping(GetKeyServerLocationsReply& reply,
|
|||
|
||||
ACTOR static Future<Void> doKeyServerLocationRequest(GetKeyServerLocationsRequest req, ProxyCommitData* commitData) {
|
||||
// We can't respond to these requests until we have valid txnStateStore
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetKeyServersLocations;
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
wait(commitData->validState.getFuture());
|
||||
wait(delay(0, TaskPriority::DefaultEndpoint));
|
||||
|
||||
|
@ -1543,7 +1541,7 @@ ACTOR static Future<Void> rejoinServer(CommitProxyInterface proxy, ProxyCommitDa
|
|||
// We can't respond to these requests until we have valid txnStateStore
|
||||
wait(commitData->validState.getFuture());
|
||||
|
||||
TraceEvent("ProxyReadyForReads", proxy.id());
|
||||
TraceEvent("ProxyReadyForReads", proxy.id()).log();
|
||||
|
||||
loop {
|
||||
GetStorageServerRejoinInfoRequest req = waitNext(proxy.getStorageServerRejoinInfo.getFuture());
|
||||
|
@ -1985,7 +1983,7 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
|
|||
state KeyRange txnKeys = allKeys;
|
||||
RangeResult UIDtoTagMap = commitData.txnStateStore->readRange(serverTagKeys).get();
|
||||
state std::map<Tag, UID> tag_uid;
|
||||
for (const KeyValueRef kv : UIDtoTagMap) {
|
||||
for (const KeyValueRef& kv : UIDtoTagMap) {
|
||||
tag_uid[decodeServerTagValue(kv.value)] = decodeServerTagKey(kv.key);
|
||||
}
|
||||
loop {
|
||||
|
|
|
@ -1,139 +0,0 @@
|
|||
/*
|
||||
* ConfigBroadcastFollowerInterface.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "fdbserver/ConfigFollowerInterface.h"
|
||||
|
||||
class ConfigClassSet {
|
||||
std::set<Key> classes;
|
||||
|
||||
public:
|
||||
static constexpr FileIdentifier file_identifier = 9854021;
|
||||
|
||||
bool operator==(ConfigClassSet const& rhs) const { return classes == rhs.classes; }
|
||||
bool operator!=(ConfigClassSet const& rhs) const { return !(*this == rhs); }
|
||||
|
||||
ConfigClassSet() = default;
|
||||
ConfigClassSet(VectorRef<KeyRef> configClasses) {
|
||||
for (const auto& configClass : configClasses) {
|
||||
classes.insert(configClass);
|
||||
}
|
||||
}
|
||||
|
||||
bool contains(KeyRef configClass) const { return classes.count(configClass); }
|
||||
std::set<Key> const& getClasses() const { return classes; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, classes);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Traceable<ConfigClassSet> : std::true_type {
|
||||
static std::string toString(ConfigClassSet const& value) { return describe(value.getClasses()); }
|
||||
};
|
||||
|
||||
struct ConfigBroadcastFollowerGetSnapshotReply {
|
||||
static constexpr FileIdentifier file_identifier = 8701983;
|
||||
Version version{ 0 };
|
||||
std::map<ConfigKey, KnobValue> snapshot;
|
||||
|
||||
ConfigBroadcastFollowerGetSnapshotReply() = default;
|
||||
template <class Snapshot>
|
||||
explicit ConfigBroadcastFollowerGetSnapshotReply(Version version, Snapshot&& snapshot)
|
||||
: version(version), snapshot(std::forward<Snapshot>(snapshot)) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, version, snapshot);
|
||||
}
|
||||
};
|
||||
|
||||
struct ConfigBroadcastFollowerGetSnapshotRequest {
|
||||
static constexpr FileIdentifier file_identifier = 10911924;
|
||||
ConfigClassSet configClassSet;
|
||||
ReplyPromise<ConfigBroadcastFollowerGetSnapshotReply> reply;
|
||||
|
||||
ConfigBroadcastFollowerGetSnapshotRequest() = default;
|
||||
explicit ConfigBroadcastFollowerGetSnapshotRequest(ConfigClassSet const& configClassSet)
|
||||
: configClassSet(configClassSet) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, configClassSet, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct ConfigBroadcastFollowerGetChangesReply {
|
||||
static constexpr FileIdentifier file_identifier = 4014927;
|
||||
Version mostRecentVersion;
|
||||
Standalone<VectorRef<VersionedConfigMutationRef>> changes;
|
||||
|
||||
ConfigBroadcastFollowerGetChangesReply()=default;
|
||||
explicit ConfigBroadcastFollowerGetChangesReply(Version mostRecentVersion, Standalone<VectorRef<VersionedConfigMutationRef>> const& changes)
|
||||
: mostRecentVersion(mostRecentVersion), changes(changes) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, mostRecentVersion, changes);
|
||||
}
|
||||
};
|
||||
|
||||
struct ConfigBroadcastFollowerGetChangesRequest {
|
||||
static constexpr FileIdentifier file_identifier = 601280;
|
||||
Version lastSeenVersion;
|
||||
ConfigClassSet configClassSet;
|
||||
ReplyPromise<ConfigBroadcastFollowerGetChangesReply> reply;
|
||||
|
||||
ConfigBroadcastFollowerGetChangesRequest() = default;
|
||||
explicit ConfigBroadcastFollowerGetChangesRequest(Version lastSeenVersion, ConfigClassSet const& configClassSet)
|
||||
: lastSeenVersion(lastSeenVersion), configClassSet(configClassSet) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, lastSeenVersion, configClassSet, reply);
|
||||
}
|
||||
};
|
||||
|
||||
/*
|
||||
* The ConfigBroadcaster serves a ConfigBroadcastFollowerInterface which all
|
||||
* workers use to fetch updates.
|
||||
*/
|
||||
class ConfigBroadcastFollowerInterface {
|
||||
UID _id;
|
||||
|
||||
public:
|
||||
static constexpr FileIdentifier file_identifier = 1984391;
|
||||
RequestStream<ConfigBroadcastFollowerGetSnapshotRequest> getSnapshot;
|
||||
RequestStream<ConfigBroadcastFollowerGetChangesRequest> getChanges;
|
||||
|
||||
ConfigBroadcastFollowerInterface() : _id(deterministicRandom()->randomUniqueID()) {}
|
||||
|
||||
bool operator==(ConfigBroadcastFollowerInterface const& rhs) const { return (_id == rhs._id); }
|
||||
bool operator!=(ConfigBroadcastFollowerInterface const& rhs) const { return !(*this == rhs); }
|
||||
UID id() const { return _id; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, _id, getSnapshot, getChanges);
|
||||
}
|
||||
};
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* ConfigBroadcastInterface.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "fdbserver/ConfigFollowerInterface.h"
|
||||
|
||||
class ConfigClassSet {
|
||||
std::set<Key> classes;
|
||||
|
||||
public:
|
||||
static constexpr FileIdentifier file_identifier = 9854021;
|
||||
|
||||
bool operator==(ConfigClassSet const& rhs) const { return classes == rhs.classes; }
|
||||
bool operator!=(ConfigClassSet const& rhs) const { return !(*this == rhs); }
|
||||
|
||||
ConfigClassSet() = default;
|
||||
ConfigClassSet(VectorRef<KeyRef> configClasses) {
|
||||
for (const auto& configClass : configClasses) {
|
||||
classes.insert(configClass);
|
||||
}
|
||||
}
|
||||
ConfigClassSet(std::vector<KeyRef> configClasses) {
|
||||
for (const auto& configClass : configClasses) {
|
||||
classes.insert(configClass);
|
||||
}
|
||||
}
|
||||
|
||||
bool contains(KeyRef configClass) const { return classes.count(configClass); }
|
||||
std::set<Key> const& getClasses() const { return classes; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, classes);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Traceable<ConfigClassSet> : std::true_type {
|
||||
static std::string toString(ConfigClassSet const& value) { return describe(value.getClasses()); }
|
||||
};
|
||||
|
||||
struct ConfigBroadcastSnapshotReply {
|
||||
static constexpr FileIdentifier file_identifier = 8701984;
|
||||
|
||||
ConfigBroadcastSnapshotReply() = default;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar);
|
||||
}
|
||||
};
|
||||
|
||||
struct ConfigBroadcastSnapshotRequest {
|
||||
static constexpr FileIdentifier file_identifier = 10911925;
|
||||
Version version{ 0 };
|
||||
std::map<ConfigKey, KnobValue> snapshot;
|
||||
ReplyPromise<ConfigBroadcastSnapshotReply> reply;
|
||||
|
||||
ConfigBroadcastSnapshotRequest() = default;
|
||||
template <class Snapshot>
|
||||
explicit ConfigBroadcastSnapshotRequest(Version version, Snapshot&& snapshot)
|
||||
: version(version), snapshot(std::forward<Snapshot>(snapshot)) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, version, snapshot, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct ConfigBroadcastChangesReply {
|
||||
static constexpr FileIdentifier file_identifier = 4014928;
|
||||
|
||||
ConfigBroadcastChangesReply() = default;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar);
|
||||
}
|
||||
};
|
||||
|
||||
struct ConfigBroadcastChangesRequest {
|
||||
static constexpr FileIdentifier file_identifier = 601281;
|
||||
Version mostRecentVersion;
|
||||
Standalone<VectorRef<VersionedConfigMutationRef>> changes;
|
||||
ReplyPromise<ConfigBroadcastChangesReply> reply;
|
||||
|
||||
ConfigBroadcastChangesRequest() = default;
|
||||
explicit ConfigBroadcastChangesRequest(Version mostRecentVersion,
|
||||
Standalone<VectorRef<VersionedConfigMutationRef>> const& changes)
|
||||
: mostRecentVersion(mostRecentVersion), changes(changes) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, mostRecentVersion, changes, reply);
|
||||
}
|
||||
};
|
||||
|
||||
/*
|
||||
* The ConfigBroadcaster uses a ConfigBroadcastInterface from each worker to
|
||||
* push updates made to the configuration database to the worker.
|
||||
*/
|
||||
class ConfigBroadcastInterface {
|
||||
UID _id;
|
||||
|
||||
public:
|
||||
static constexpr FileIdentifier file_identifier = 1676543;
|
||||
RequestStream<ConfigBroadcastSnapshotRequest> snapshot;
|
||||
RequestStream<ConfigBroadcastChangesRequest> changes;
|
||||
|
||||
ConfigBroadcastInterface() : _id(deterministicRandom()->randomUniqueID()) {}
|
||||
|
||||
bool operator==(ConfigBroadcastInterface const& rhs) const { return (_id == rhs._id); }
|
||||
bool operator!=(ConfigBroadcastInterface const& rhs) const { return !(*this == rhs); }
|
||||
UID id() const { return _id; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, _id, snapshot, changes);
|
||||
}
|
||||
};
|
|
@ -49,101 +49,74 @@ void remove(Container& container, K const& k) {
|
|||
} // namespace
|
||||
|
||||
class ConfigBroadcasterImpl {
|
||||
// PendingRequestStore stores a set of pending ConfigBroadcastFollowerGetChangesRequests,
|
||||
// indexed by configuration class. When an update is received, replies are sent for all
|
||||
// pending requests with affected configuration classes
|
||||
class PendingRequestStore {
|
||||
using Req = ConfigBroadcastFollowerGetChangesRequest;
|
||||
std::map<Key, std::set<Endpoint::Token>> configClassToTokens;
|
||||
std::map<Endpoint::Token, Req> tokenToRequest;
|
||||
// Holds information about each client connected to the broadcaster.
|
||||
struct BroadcastClientDetails {
|
||||
// Triggered when the worker dies.
|
||||
Future<Void> watcher;
|
||||
ConfigClassSet configClassSet;
|
||||
Version lastSeenVersion;
|
||||
ConfigBroadcastInterface broadcastInterface;
|
||||
|
||||
public:
|
||||
void addRequest(Req const& req) {
|
||||
auto token = req.reply.getEndpoint().token;
|
||||
tokenToRequest[token] = req;
|
||||
for (const auto& configClass : req.configClassSet.getClasses()) {
|
||||
configClassToTokens[configClass].insert(token);
|
||||
}
|
||||
bool operator==(BroadcastClientDetails const& rhs) const {
|
||||
return configClassSet == rhs.configClassSet && lastSeenVersion == rhs.lastSeenVersion &&
|
||||
broadcastInterface == rhs.broadcastInterface;
|
||||
}
|
||||
bool operator!=(BroadcastClientDetails const& rhs) const { return !(*this == rhs); }
|
||||
};
|
||||
|
||||
std::vector<Req> getRequestsToNotify(Standalone<VectorRef<VersionedConfigMutationRef>> const& changes) const {
|
||||
std::set<Endpoint::Token> tokenSet;
|
||||
for (const auto& change : changes) {
|
||||
if (!change.mutation.getConfigClass().present()) {
|
||||
// Update everything
|
||||
for (const auto& [token, req] : tokenToRequest) {
|
||||
if (req.lastSeenVersion < change.version) {
|
||||
tokenSet.insert(token);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Key configClass = change.mutation.getConfigClass().get();
|
||||
if (configClassToTokens.count(configClass)) {
|
||||
auto tokens = get(configClassToTokens, Key(change.mutation.getConfigClass().get()));
|
||||
for (const auto& token : tokens) {
|
||||
auto req = get(tokenToRequest, token);
|
||||
if (req.lastSeenVersion < change.version) {
|
||||
tokenSet.insert(token);
|
||||
} else {
|
||||
TEST(true); // Worker is ahead of config broadcaster
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
std::vector<Req> result;
|
||||
for (const auto& token : tokenSet) {
|
||||
result.push_back(get(tokenToRequest, token));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
std::vector<Req> getOutdatedRequests(Version newSnapshotVersion) {
|
||||
std::vector<Req> result;
|
||||
for (const auto& [token, req] : tokenToRequest) {
|
||||
if (req.lastSeenVersion < newSnapshotVersion) {
|
||||
result.push_back(req);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
void removeRequest(Req const& req) {
|
||||
auto token = req.reply.getEndpoint().token;
|
||||
for (const auto& configClass : req.configClassSet.getClasses()) {
|
||||
remove(get(configClassToTokens, configClass), token);
|
||||
// TODO: Don't leak config classes
|
||||
}
|
||||
remove(tokenToRequest, token);
|
||||
}
|
||||
} pending;
|
||||
std::map<ConfigKey, KnobValue> snapshot;
|
||||
std::deque<VersionedConfigMutation> mutationHistory;
|
||||
std::deque<VersionedConfigCommitAnnotation> annotationHistory;
|
||||
Version lastCompactedVersion;
|
||||
Version mostRecentVersion;
|
||||
std::unique_ptr<IConfigConsumer> consumer;
|
||||
Future<Void> consumerFuture;
|
||||
ActorCollection actors{ false };
|
||||
std::vector<BroadcastClientDetails> clients;
|
||||
|
||||
UID id;
|
||||
CounterCollection cc;
|
||||
Counter compactRequest;
|
||||
mutable Counter successfulChangeRequest;
|
||||
Counter successfulChangeRequest;
|
||||
Counter failedChangeRequest;
|
||||
Counter snapshotRequest;
|
||||
Future<Void> logger;
|
||||
|
||||
Future<Void> pushSnapshot(ConfigBroadcasterImpl* self,
|
||||
Version snapshotVersion,
|
||||
BroadcastClientDetails const& client) {
|
||||
if (client.lastSeenVersion >= snapshotVersion) {
|
||||
return Void();
|
||||
}
|
||||
|
||||
++snapshotRequest;
|
||||
ConfigBroadcastSnapshotRequest request;
|
||||
for (const auto& [key, value] : self->snapshot) {
|
||||
if (matchesConfigClass(client.configClassSet, key.configClass)) {
|
||||
request.snapshot[key] = value;
|
||||
}
|
||||
}
|
||||
request.version = snapshotVersion;
|
||||
TraceEvent(SevDebug, "ConfigBroadcasterSnapshotRequest", id)
|
||||
.detail("Size", request.snapshot.size())
|
||||
.detail("Version", request.version);
|
||||
return success(client.broadcastInterface.snapshot.getReply(request));
|
||||
}
|
||||
|
||||
template <class Changes>
|
||||
void sendChangesReply(ConfigBroadcastFollowerGetChangesRequest const& req, Changes const& changes) const {
|
||||
ASSERT_LT(req.lastSeenVersion, mostRecentVersion);
|
||||
ConfigBroadcastFollowerGetChangesReply reply;
|
||||
reply.mostRecentVersion = mostRecentVersion;
|
||||
Future<Void> pushChanges(BroadcastClientDetails& client, Changes const& changes) {
|
||||
// Skip if client has already seen the latest version.
|
||||
if (client.lastSeenVersion >= mostRecentVersion) {
|
||||
return Void();
|
||||
}
|
||||
|
||||
ConfigBroadcastChangesRequest req;
|
||||
for (const auto& versionedMutation : changes) {
|
||||
if (versionedMutation.version > req.lastSeenVersion &&
|
||||
matchesConfigClass(req.configClassSet, versionedMutation.mutation.getConfigClass())) {
|
||||
if (versionedMutation.version > client.lastSeenVersion &&
|
||||
matchesConfigClass(client.configClassSet, versionedMutation.mutation.getConfigClass())) {
|
||||
TraceEvent te(SevDebug, "ConfigBroadcasterSendingChangeMutation", id);
|
||||
te.detail("Version", versionedMutation.version)
|
||||
.detail("ReqLastSeenVersion", req.lastSeenVersion)
|
||||
.detail("ReqLastSeenVersion", client.lastSeenVersion)
|
||||
.detail("ConfigClass", versionedMutation.mutation.getConfigClass())
|
||||
.detail("KnobName", versionedMutation.mutation.getKnobName());
|
||||
if (versionedMutation.mutation.isSet()) {
|
||||
|
@ -152,54 +125,18 @@ class ConfigBroadcasterImpl {
|
|||
te.detail("Op", "Clear");
|
||||
}
|
||||
|
||||
reply.changes.push_back_deep(reply.changes.arena(), versionedMutation);
|
||||
req.changes.push_back_deep(req.changes.arena(), versionedMutation);
|
||||
}
|
||||
}
|
||||
req.reply.send(reply);
|
||||
++successfulChangeRequest;
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> serve(ConfigBroadcaster* self,
|
||||
ConfigBroadcasterImpl* impl,
|
||||
ConfigBroadcastFollowerInterface cbfi) {
|
||||
impl->actors.add(impl->consumer->consume(*self));
|
||||
loop {
|
||||
choose {
|
||||
when(ConfigBroadcastFollowerGetSnapshotRequest req = waitNext(cbfi.getSnapshot.getFuture())) {
|
||||
++impl->snapshotRequest;
|
||||
ConfigBroadcastFollowerGetSnapshotReply reply;
|
||||
for (const auto& [key, value] : impl->snapshot) {
|
||||
if (matchesConfigClass(req.configClassSet, key.configClass)) {
|
||||
reply.snapshot[key] = value;
|
||||
}
|
||||
}
|
||||
reply.version = impl->mostRecentVersion;
|
||||
TraceEvent(SevDebug, "ConfigBroadcasterGotSnapshotRequest", impl->id)
|
||||
.detail("Size", reply.snapshot.size())
|
||||
.detail("Version", reply.version);
|
||||
req.reply.send(reply);
|
||||
}
|
||||
when(ConfigBroadcastFollowerGetChangesRequest req = waitNext(cbfi.getChanges.getFuture())) {
|
||||
if (req.lastSeenVersion < impl->lastCompactedVersion) {
|
||||
req.reply.sendError(version_already_compacted());
|
||||
++impl->failedChangeRequest;
|
||||
continue;
|
||||
}
|
||||
if (req.lastSeenVersion < impl->mostRecentVersion) {
|
||||
impl->sendChangesReply(req, impl->mutationHistory);
|
||||
} else {
|
||||
TEST(req.lastSeenVersion > impl->mostRecentVersion); // Worker is ahead of ConfigBroadcaster
|
||||
TraceEvent(SevDebug, "ConfigBroadcasterRegisteringChangeRequest", impl->id)
|
||||
.detail("Peer", req.reply.getEndpoint().getPrimaryAddress())
|
||||
.detail("MostRecentVersion", impl->mostRecentVersion)
|
||||
.detail("ReqLastSeenVersion", req.lastSeenVersion)
|
||||
.detail("ConfigClass", req.configClassSet);
|
||||
impl->pending.addRequest(req);
|
||||
}
|
||||
}
|
||||
when(wait(impl->actors.getResult())) { ASSERT(false); }
|
||||
}
|
||||
if (req.changes.size() == 0) {
|
||||
return Void();
|
||||
}
|
||||
|
||||
client.lastSeenVersion = mostRecentVersion;
|
||||
req.mostRecentVersion = mostRecentVersion;
|
||||
++successfulChangeRequest;
|
||||
return success(client.broadcastInterface.changes.getReply(req));
|
||||
}
|
||||
|
||||
ConfigBroadcasterImpl()
|
||||
|
@ -211,25 +148,6 @@ class ConfigBroadcasterImpl {
|
|||
"ConfigBroadcasterMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ConfigBroadcasterMetrics");
|
||||
}
|
||||
|
||||
void notifyFollowers(Standalone<VectorRef<VersionedConfigMutationRef>> const& changes) {
|
||||
auto toNotify = pending.getRequestsToNotify(changes);
|
||||
TraceEvent(SevDebug, "ConfigBroadcasterNotifyingFollowers", id)
|
||||
.detail("ChangesSize", changes.size())
|
||||
.detail("ToNotify", toNotify.size());
|
||||
for (auto& req : toNotify) {
|
||||
sendChangesReply(req, changes);
|
||||
pending.removeRequest(req);
|
||||
}
|
||||
}
|
||||
|
||||
void notifyOutdatedRequests() {
|
||||
auto outdated = pending.getOutdatedRequests(mostRecentVersion);
|
||||
for (auto& req : outdated) {
|
||||
req.reply.sendError(version_already_compacted());
|
||||
pending.removeRequest(req);
|
||||
}
|
||||
}
|
||||
|
||||
void addChanges(Standalone<VectorRef<VersionedConfigMutationRef>> const& changes,
|
||||
Version mostRecentVersion,
|
||||
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> const& annotations) {
|
||||
|
@ -244,18 +162,64 @@ class ConfigBroadcasterImpl {
|
|||
snapshot.erase(mutation.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
for (auto& client : clients) {
|
||||
actors.add(brokenPromiseToNever(pushChanges(client, changes)));
|
||||
}
|
||||
}
|
||||
|
||||
template <class Snapshot>
|
||||
Future<Void> setSnapshot(Snapshot&& snapshot, Version snapshotVersion) {
|
||||
this->snapshot = std::forward<Snapshot>(snapshot);
|
||||
Future<Void> setSnapshot(Snapshot& snapshot, Version snapshotVersion) {
|
||||
this->snapshot = snapshot;
|
||||
this->lastCompactedVersion = snapshotVersion;
|
||||
std::vector<Future<Void>> futures;
|
||||
for (const auto& client : clients) {
|
||||
futures.push_back(brokenPromiseToNever(pushSnapshot(this, snapshotVersion, client)));
|
||||
}
|
||||
return waitForAll(futures);
|
||||
}
|
||||
|
||||
ACTOR template <class Snapshot>
|
||||
static Future<Void> pushSnapshotAndChanges(ConfigBroadcasterImpl* self,
|
||||
Snapshot snapshot,
|
||||
Version snapshotVersion,
|
||||
Standalone<VectorRef<VersionedConfigMutationRef>> changes,
|
||||
Version changesVersion,
|
||||
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> annotations) {
|
||||
// Make sure all snapshot messages were received before sending changes.
|
||||
wait(self->setSnapshot(snapshot, snapshotVersion));
|
||||
self->addChanges(changes, changesVersion, annotations);
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> waitForFailure(ConfigBroadcasterImpl* self,
|
||||
Future<Void> watcher,
|
||||
BroadcastClientDetails* client) {
|
||||
wait(success(watcher));
|
||||
self->clients.erase(std::remove(self->clients.begin(), self->clients.end(), *client));
|
||||
return Void();
|
||||
}
|
||||
|
||||
public:
|
||||
Future<Void> serve(ConfigBroadcaster* self, ConfigBroadcastFollowerInterface const& cbfi) {
|
||||
return serve(self, this, cbfi);
|
||||
Future<Void> registerWorker(ConfigBroadcaster* self,
|
||||
Version lastSeenVersion,
|
||||
ConfigClassSet configClassSet,
|
||||
Future<Void> watcher,
|
||||
ConfigBroadcastInterface broadcastInterface) {
|
||||
if (!consumerFuture.isValid()) {
|
||||
consumerFuture = consumer->consume(*self);
|
||||
}
|
||||
|
||||
auto client = BroadcastClientDetails{
|
||||
watcher, std::move(configClassSet), lastSeenVersion, std::move(broadcastInterface)
|
||||
};
|
||||
|
||||
// Push all dynamic knobs to worker if it isn't up to date.
|
||||
Future<Void> result = pushSnapshot(this, mostRecentVersion, client);
|
||||
|
||||
clients.push_back(std::move(client));
|
||||
actors.add(waitForFailure(this, watcher, &clients.back()));
|
||||
return result;
|
||||
}
|
||||
|
||||
void applyChanges(Standalone<VectorRef<VersionedConfigMutationRef>> const& changes,
|
||||
|
@ -267,7 +231,6 @@ public:
|
|||
.detail("NewMostRecentVersion", mostRecentVersion)
|
||||
.detail("AnnotationsSize", annotations.size());
|
||||
addChanges(changes, mostRecentVersion, annotations);
|
||||
notifyFollowers(changes);
|
||||
}
|
||||
|
||||
template <class Snapshot>
|
||||
|
@ -283,9 +246,7 @@ public:
|
|||
.detail("ChangesSize", changes.size())
|
||||
.detail("ChangesVersion", changesVersion)
|
||||
.detail("AnnotationsSize", annotations.size());
|
||||
setSnapshot(std::forward<Snapshot>(snapshot), snapshotVersion);
|
||||
addChanges(changes, changesVersion, annotations);
|
||||
notifyOutdatedRequests();
|
||||
actors.add(pushSnapshotAndChanges(this, snapshot, snapshotVersion, changes, changesVersion, annotations));
|
||||
}
|
||||
|
||||
ConfigBroadcasterImpl(ConfigFollowerInterface const& cfi) : ConfigBroadcasterImpl() {
|
||||
|
@ -361,6 +322,8 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
Future<Void> getError() const { return consumerFuture || actors.getResult(); }
|
||||
|
||||
UID getID() const { return id; }
|
||||
|
||||
static void runPendingRequestStoreTest(bool includeGlobalMutation, int expectedMatches);
|
||||
|
@ -378,8 +341,11 @@ ConfigBroadcaster& ConfigBroadcaster::operator=(ConfigBroadcaster&&) = default;
|
|||
|
||||
ConfigBroadcaster::~ConfigBroadcaster() = default;
|
||||
|
||||
Future<Void> ConfigBroadcaster::serve(ConfigBroadcastFollowerInterface const& cbfi) {
|
||||
return impl->serve(this, cbfi);
|
||||
Future<Void> ConfigBroadcaster::registerWorker(Version lastSeenVersion,
|
||||
ConfigClassSet configClassSet,
|
||||
Future<Void> watcher,
|
||||
ConfigBroadcastInterface broadcastInterface) {
|
||||
return impl->registerWorker(this, lastSeenVersion, std::move(configClassSet), watcher, broadcastInterface);
|
||||
}
|
||||
|
||||
void ConfigBroadcaster::applyChanges(Standalone<VectorRef<VersionedConfigMutationRef>> const& changes,
|
||||
|
@ -406,6 +372,10 @@ void ConfigBroadcaster::applySnapshotAndChanges(
|
|||
impl->applySnapshotAndChanges(std::move(snapshot), snapshotVersion, changes, changesVersion, annotations);
|
||||
}
|
||||
|
||||
Future<Void> ConfigBroadcaster::getError() const {
|
||||
return impl->getError();
|
||||
}
|
||||
|
||||
UID ConfigBroadcaster::getID() const {
|
||||
return impl->getID();
|
||||
}
|
||||
|
@ -417,62 +387,3 @@ JsonBuilderObject ConfigBroadcaster::getStatus() const {
|
|||
void ConfigBroadcaster::compact(Version compactionVersion) {
|
||||
impl->compact(compactionVersion);
|
||||
}
|
||||
|
||||
namespace {
|
||||
|
||||
Standalone<VectorRef<VersionedConfigMutationRef>> getTestChanges(Version version, bool includeGlobalMutation) {
|
||||
Standalone<VectorRef<VersionedConfigMutationRef>> changes;
|
||||
if (includeGlobalMutation) {
|
||||
ConfigKey key = ConfigKeyRef({}, "test_long"_sr);
|
||||
auto value = KnobValue::create(int64_t{ 5 });
|
||||
ConfigMutation mutation = ConfigMutationRef(key, value.contents());
|
||||
changes.emplace_back_deep(changes.arena(), version, mutation);
|
||||
}
|
||||
{
|
||||
ConfigKey key = ConfigKeyRef("class-A"_sr, "test_long"_sr);
|
||||
auto value = KnobValue::create(int64_t{ 5 });
|
||||
ConfigMutation mutation = ConfigMutationRef(key, value.contents());
|
||||
changes.emplace_back_deep(changes.arena(), version, mutation);
|
||||
}
|
||||
return changes;
|
||||
}
|
||||
|
||||
ConfigBroadcastFollowerGetChangesRequest getTestRequest(Version lastSeenVersion,
|
||||
std::vector<KeyRef> const& configClasses) {
|
||||
Standalone<VectorRef<KeyRef>> configClassesVector;
|
||||
for (const auto& configClass : configClasses) {
|
||||
configClassesVector.push_back_deep(configClassesVector.arena(), configClass);
|
||||
}
|
||||
return ConfigBroadcastFollowerGetChangesRequest{ lastSeenVersion, ConfigClassSet{ configClassesVector } };
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
void ConfigBroadcasterImpl::runPendingRequestStoreTest(bool includeGlobalMutation, int expectedMatches) {
|
||||
PendingRequestStore pending;
|
||||
for (Version v = 0; v < 5; ++v) {
|
||||
pending.addRequest(getTestRequest(v, {}));
|
||||
pending.addRequest(getTestRequest(v, { "class-A"_sr }));
|
||||
pending.addRequest(getTestRequest(v, { "class-B"_sr }));
|
||||
pending.addRequest(getTestRequest(v, { "class-A"_sr, "class-B"_sr }));
|
||||
}
|
||||
auto toNotify = pending.getRequestsToNotify(getTestChanges(0, includeGlobalMutation));
|
||||
ASSERT_EQ(toNotify.size(), 0);
|
||||
for (Version v = 1; v <= 5; ++v) {
|
||||
auto toNotify = pending.getRequestsToNotify(getTestChanges(v, includeGlobalMutation));
|
||||
ASSERT_EQ(toNotify.size(), expectedMatches);
|
||||
for (const auto& req : toNotify) {
|
||||
pending.removeRequest(req);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/ConfigBroadcaster/Internal/PendingRequestStore/Simple") {
|
||||
ConfigBroadcasterImpl::runPendingRequestStoreTest(false, 2);
|
||||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/ConfigBroadcaster/Internal/PendingRequestStore/GlobalMutation") {
|
||||
ConfigBroadcasterImpl::runPendingRequestStoreTest(true, 4);
|
||||
return Void();
|
||||
}
|
||||
|
|
|
@ -24,14 +24,15 @@
|
|||
#include "fdbclient/JsonBuilder.h"
|
||||
#include "fdbclient/PImpl.h"
|
||||
#include "fdbserver/CoordinationInterface.h"
|
||||
#include "fdbserver/ConfigBroadcastFollowerInterface.h"
|
||||
#include "fdbserver/ConfigBroadcastInterface.h"
|
||||
#include "fdbserver/ConfigFollowerInterface.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "flow/flow.h"
|
||||
#include <memory>
|
||||
|
||||
/*
|
||||
* The configuration broadcaster runs on the cluster controller. The broadcaster listens uses
|
||||
* an IConfigConsumer instantitation to consume updates from the configuration database, and broadcasts
|
||||
* an IConfigConsumer instantiation to consume updates from the configuration database, and broadcasts
|
||||
* these updates to all workers' local configurations
|
||||
*/
|
||||
class ConfigBroadcaster {
|
||||
|
@ -42,7 +43,10 @@ public:
|
|||
ConfigBroadcaster(ConfigBroadcaster&&);
|
||||
ConfigBroadcaster& operator=(ConfigBroadcaster&&);
|
||||
~ConfigBroadcaster();
|
||||
Future<Void> serve(ConfigBroadcastFollowerInterface const&);
|
||||
Future<Void> registerWorker(Version lastSeenVersion,
|
||||
ConfigClassSet configClassSet,
|
||||
Future<Void> watcher,
|
||||
ConfigBroadcastInterface worker);
|
||||
void applyChanges(Standalone<VectorRef<VersionedConfigMutationRef>> const& changes,
|
||||
Version mostRecentVersion,
|
||||
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> const& annotations);
|
||||
|
@ -56,6 +60,7 @@ public:
|
|||
Standalone<VectorRef<VersionedConfigMutationRef>> const& changes,
|
||||
Version changesVersion,
|
||||
Standalone<VectorRef<VersionedConfigCommitAnnotationRef>> const& annotations);
|
||||
Future<Void> getError() const;
|
||||
UID getID() const;
|
||||
JsonBuilderObject getStatus() const;
|
||||
void compact(Version compactionVersion);
|
||||
|
|
|
@ -124,7 +124,7 @@ class ReadFromLocalConfigEnvironment {
|
|||
UID id;
|
||||
std::string dataDir;
|
||||
LocalConfiguration localConfiguration;
|
||||
Reference<IAsyncListener<ConfigBroadcastFollowerInterface> const> cbfi;
|
||||
Reference<AsyncVar<ConfigBroadcastInterface> const> cbi;
|
||||
Future<Void> consumer;
|
||||
|
||||
ACTOR static Future<Void> checkEventually(LocalConfiguration const* localConfiguration,
|
||||
|
@ -146,8 +146,16 @@ class ReadFromLocalConfigEnvironment {
|
|||
|
||||
ACTOR static Future<Void> setup(ReadFromLocalConfigEnvironment* self) {
|
||||
wait(self->localConfiguration.initialize());
|
||||
if (self->cbfi) {
|
||||
self->consumer = self->localConfiguration.consume(self->cbfi);
|
||||
if (self->cbi) {
|
||||
// LocalConfiguration runs in a loop waiting for messages from the
|
||||
// broadcaster. These unit tests use the same
|
||||
// ConfigBroadcastInterface across restarts, so when "killing" the
|
||||
// old LocalConfiguration, it's necessary to make sure it is
|
||||
// completely stopped before starting the second config. This
|
||||
// prevents two actors trying to listen for the same message on the
|
||||
// same interface, causing lots of issues!
|
||||
self->consumer.cancel();
|
||||
self->consumer = self->localConfiguration.consume(self->cbi->get());
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
@ -166,10 +174,9 @@ public:
|
|||
return setup();
|
||||
}
|
||||
|
||||
void connectToBroadcaster(Reference<IAsyncListener<ConfigBroadcastFollowerInterface> const> const& cbfi) {
|
||||
ASSERT(!this->cbfi);
|
||||
this->cbfi = cbfi;
|
||||
consumer = localConfiguration.consume(cbfi);
|
||||
void connectToBroadcaster(Reference<AsyncVar<ConfigBroadcastInterface> const> const& cbi) {
|
||||
this->cbi = cbi;
|
||||
consumer = localConfiguration.consume(cbi->get());
|
||||
}
|
||||
|
||||
void checkImmediate(Optional<int64_t> expected) const {
|
||||
|
@ -187,6 +194,10 @@ public:
|
|||
LocalConfiguration& getMutableLocalConfiguration() { return localConfiguration; }
|
||||
|
||||
Future<Void> getError() const { return consumer; }
|
||||
|
||||
Version lastSeenVersion() { return localConfiguration.lastSeenVersion(); }
|
||||
|
||||
ConfigClassSet configClassSet() { return localConfiguration.configClassSet(); }
|
||||
};
|
||||
|
||||
class LocalConfigEnvironment {
|
||||
|
@ -204,7 +215,7 @@ public:
|
|||
std::string const& configPath,
|
||||
std::map<std::string, std::string> const& manualKnobOverrides = {})
|
||||
: readFrom(dataDir, configPath, manualKnobOverrides) {}
|
||||
Future<Void> setup() { return readFrom.setup(); }
|
||||
Future<Void> setup(ConfigClassSet const& configClassSet) { return readFrom.setup(); }
|
||||
Future<Void> restartLocalConfig(std::string const& newConfigPath) {
|
||||
return readFrom.restartLocalConfig(newConfigPath);
|
||||
}
|
||||
|
@ -219,15 +230,16 @@ public:
|
|||
|
||||
class BroadcasterToLocalConfigEnvironment {
|
||||
ReadFromLocalConfigEnvironment readFrom;
|
||||
Reference<AsyncVar<ConfigBroadcastFollowerInterface>> cbfi;
|
||||
Reference<AsyncVar<ConfigBroadcastInterface>> cbi;
|
||||
ConfigBroadcaster broadcaster;
|
||||
Version lastWrittenVersion{ 0 };
|
||||
Future<Void> broadcastServer;
|
||||
|
||||
ACTOR static Future<Void> setup(BroadcasterToLocalConfigEnvironment* self) {
|
||||
ACTOR static Future<Void> setup(BroadcasterToLocalConfigEnvironment* self, ConfigClassSet configClassSet) {
|
||||
wait(self->readFrom.setup());
|
||||
self->readFrom.connectToBroadcaster(IAsyncListener<ConfigBroadcastFollowerInterface>::create(self->cbfi));
|
||||
self->broadcastServer = self->broadcaster.serve(self->cbfi->get());
|
||||
self->cbi = makeReference<AsyncVar<ConfigBroadcastInterface>>();
|
||||
self->readFrom.connectToBroadcaster(self->cbi);
|
||||
self->broadcastServer = self->broadcaster.registerWorker(0, configClassSet, Never(), self->cbi->get());
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -239,10 +251,10 @@ class BroadcasterToLocalConfigEnvironment {
|
|||
|
||||
public:
|
||||
BroadcasterToLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath)
|
||||
: readFrom(dataDir, configPath, {}), cbfi(makeReference<AsyncVar<ConfigBroadcastFollowerInterface>>()),
|
||||
: readFrom(dataDir, configPath, {}), cbi(makeReference<AsyncVar<ConfigBroadcastInterface>>()),
|
||||
broadcaster(ConfigFollowerInterface{}) {}
|
||||
|
||||
Future<Void> setup() { return setup(this); }
|
||||
Future<Void> setup(ConfigClassSet const& configClassSet) { return setup(this, configClassSet); }
|
||||
|
||||
void set(Optional<KeyRef> configClass, int64_t value) {
|
||||
auto knobValue = KnobValueRef::create(value);
|
||||
|
@ -255,8 +267,10 @@ public:
|
|||
|
||||
void changeBroadcaster() {
|
||||
broadcastServer.cancel();
|
||||
cbfi->set(ConfigBroadcastFollowerInterface{});
|
||||
broadcastServer = broadcaster.serve(cbfi->get());
|
||||
cbi->set(ConfigBroadcastInterface{});
|
||||
readFrom.connectToBroadcaster(cbi);
|
||||
broadcastServer =
|
||||
broadcaster.registerWorker(readFrom.lastSeenVersion(), readFrom.configClassSet(), Never(), cbi->get());
|
||||
}
|
||||
|
||||
Future<Void> restartLocalConfig(std::string const& newConfigPath) {
|
||||
|
@ -265,7 +279,7 @@ public:
|
|||
|
||||
void compact() { broadcaster.compact(lastWrittenVersion); }
|
||||
|
||||
Future<Void> getError() const { return readFrom.getError() || broadcastServer; }
|
||||
Future<Void> getError() const { return readFrom.getError() || broadcaster.getError(); }
|
||||
};
|
||||
|
||||
class TransactionEnvironment {
|
||||
|
@ -356,31 +370,33 @@ public:
|
|||
class TransactionToLocalConfigEnvironment {
|
||||
WriteToTransactionEnvironment writeTo;
|
||||
ReadFromLocalConfigEnvironment readFrom;
|
||||
Reference<AsyncVar<ConfigBroadcastFollowerInterface>> cbfi;
|
||||
Reference<AsyncVar<ConfigBroadcastInterface>> cbi;
|
||||
ConfigBroadcaster broadcaster;
|
||||
Future<Void> broadcastServer;
|
||||
|
||||
ACTOR static Future<Void> setup(TransactionToLocalConfigEnvironment* self) {
|
||||
ACTOR static Future<Void> setup(TransactionToLocalConfigEnvironment* self, ConfigClassSet configClassSet) {
|
||||
wait(self->readFrom.setup());
|
||||
self->readFrom.connectToBroadcaster(IAsyncListener<ConfigBroadcastFollowerInterface>::create(self->cbfi));
|
||||
self->broadcastServer = self->broadcaster.serve(self->cbfi->get());
|
||||
self->cbi = makeReference<AsyncVar<ConfigBroadcastInterface>>();
|
||||
self->readFrom.connectToBroadcaster(self->cbi);
|
||||
self->broadcastServer = self->broadcaster.registerWorker(0, configClassSet, Never(), self->cbi->get());
|
||||
return Void();
|
||||
}
|
||||
|
||||
public:
|
||||
TransactionToLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath)
|
||||
: writeTo(dataDir), readFrom(dataDir, configPath, {}),
|
||||
cbfi(makeReference<AsyncVar<ConfigBroadcastFollowerInterface>>()), broadcaster(writeTo.getFollowerInterface()) {
|
||||
}
|
||||
: writeTo(dataDir), readFrom(dataDir, configPath, {}), cbi(makeReference<AsyncVar<ConfigBroadcastInterface>>()),
|
||||
broadcaster(writeTo.getFollowerInterface()) {}
|
||||
|
||||
Future<Void> setup() { return setup(this); }
|
||||
Future<Void> setup(ConfigClassSet const& configClassSet) { return setup(this, configClassSet); }
|
||||
|
||||
void restartNode() { writeTo.restartNode(); }
|
||||
|
||||
void changeBroadcaster() {
|
||||
broadcastServer.cancel();
|
||||
cbfi->set(ConfigBroadcastFollowerInterface{});
|
||||
broadcastServer = broadcaster.serve(cbfi->get());
|
||||
cbi->set(ConfigBroadcastInterface{});
|
||||
readFrom.connectToBroadcaster(cbi);
|
||||
broadcastServer =
|
||||
broadcaster.registerWorker(readFrom.lastSeenVersion(), readFrom.configClassSet(), Never(), cbi->get());
|
||||
}
|
||||
|
||||
Future<Void> restartLocalConfig(std::string const& newConfigPath) {
|
||||
|
@ -395,7 +411,7 @@ public:
|
|||
}
|
||||
Future<Void> clear(Optional<KeyRef> configClass) { return writeTo.clear(configClass); }
|
||||
Future<Void> check(Optional<int64_t> value) const { return readFrom.checkEventually(value); }
|
||||
Future<Void> getError() const { return writeTo.getError() || readFrom.getError() || broadcastServer; }
|
||||
Future<Void> getError() const { return writeTo.getError() || readFrom.getError() || broadcaster.getError(); }
|
||||
};
|
||||
|
||||
// These functions give a common interface to all environments, to improve code reuse
|
||||
|
@ -438,7 +454,7 @@ Future<Void> compact(BroadcasterToLocalConfigEnvironment& env) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testRestartLocalConfig(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
wait(env.restartLocalConfig("class-A"));
|
||||
|
@ -451,7 +467,7 @@ Future<Void> testRestartLocalConfig(UnitTestParameters params) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testRestartLocalConfigAndChangeClass(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr, "class-B"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
wait(env.restartLocalConfig("class-B"));
|
||||
|
@ -461,10 +477,30 @@ Future<Void> testRestartLocalConfigAndChangeClass(UnitTestParameters params) {
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR template <class Env>
|
||||
Future<Void> testNewLocalConfigAfterCompaction(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
wait(compact(env));
|
||||
// Erase the data dir to simulate a new worker joining the system after
|
||||
// compaction.
|
||||
platform::eraseDirectoryRecursive(params.getDataDir());
|
||||
platform::createDirectory(params.getDataDir());
|
||||
wait(env.restartLocalConfig("class-A"));
|
||||
// Reregister worker with broadcaster.
|
||||
env.changeBroadcaster();
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
wait(set(env, "class-A"_sr, 2));
|
||||
wait(check(env, int64_t{ 2 }));
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR template <class Env>
|
||||
Future<Void> testSet(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
return Void();
|
||||
|
@ -473,7 +509,7 @@ Future<Void> testSet(UnitTestParameters params) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testClear(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(clear(env, "class-A"_sr));
|
||||
wait(check(env, Optional<int64_t>{}));
|
||||
|
@ -483,7 +519,7 @@ Future<Void> testClear(UnitTestParameters params) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testGlobalSet(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, Optional<KeyRef>{}, int64_t{ 1 }));
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 10 }));
|
||||
|
@ -494,7 +530,7 @@ Future<Void> testGlobalSet(UnitTestParameters params) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testIgnore(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr, "class-B"_sr })));
|
||||
wait(set(env, "class-B"_sr, int64_t{ 1 }));
|
||||
choose {
|
||||
when(wait(delay(5))) {}
|
||||
|
@ -506,7 +542,7 @@ Future<Void> testIgnore(UnitTestParameters params) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testCompact(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(compact(env));
|
||||
wait(check(env, 1));
|
||||
|
@ -518,7 +554,7 @@ Future<Void> testCompact(UnitTestParameters params) {
|
|||
ACTOR template <class Env>
|
||||
Future<Void> testChangeBroadcaster(UnitTestParameters params) {
|
||||
state Env env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
env.changeBroadcaster();
|
||||
|
@ -594,7 +630,7 @@ TEST_CASE("/fdbserver/ConfigDB/LocalConfiguration/GlobalSet") {
|
|||
|
||||
TEST_CASE("/fdbserver/ConfigDB/LocalConfiguration/ConflictingOverrides") {
|
||||
state LocalConfigEnvironment env(params.getDataDir(), "class-A/class-B", {});
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr, "class-B"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
wait(set(env, "class-B"_sr, int64_t{ 10 }));
|
||||
env.check(10);
|
||||
|
@ -603,7 +639,7 @@ TEST_CASE("/fdbserver/ConfigDB/LocalConfiguration/ConflictingOverrides") {
|
|||
|
||||
TEST_CASE("/fdbserver/ConfigDB/LocalConfiguration/Manual") {
|
||||
state LocalConfigEnvironment env(params.getDataDir(), "class-A", { { "test_long", "1000" } });
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
env.check(1000);
|
||||
return Void();
|
||||
|
@ -649,6 +685,11 @@ TEST_CASE("/fdbserver/ConfigDB/BroadcasterToLocalConfig/Compact") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/BroadcasterToLocalConfig/RestartLocalConfigurationAfterCompaction") {
|
||||
wait(testNewLocalConfigAfterCompaction<BroadcasterToLocalConfigEnvironment>(params));
|
||||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/TransactionToLocalConfig/Set") {
|
||||
wait(testSet<TransactionToLocalConfigEnvironment>(params));
|
||||
return Void();
|
||||
|
@ -666,7 +707,7 @@ TEST_CASE("/fdbserver/ConfigDB/TransactionToLocalConfig/GlobalSet") {
|
|||
|
||||
TEST_CASE("/fdbserver/ConfigDB/TransactionToLocalConfig/RestartNode") {
|
||||
state TransactionToLocalConfigEnvironment env(params.getDataDir(), "class-A");
|
||||
wait(env.setup());
|
||||
wait(env.setup(ConfigClassSet({ "class-A"_sr })));
|
||||
wait(set(env, "class-A"_sr, int64_t{ 1 }));
|
||||
env.restartNode();
|
||||
wait(check(env, int64_t{ 1 }));
|
||||
|
@ -688,6 +729,11 @@ TEST_CASE("/fdbserver/ConfigDB/TransactionToLocalConfig/CompactNode") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/TransactionToLocalConfig/RestartLocalConfigurationAfterCompaction") {
|
||||
wait(testNewLocalConfigAfterCompaction<TransactionToLocalConfigEnvironment>(params));
|
||||
return Void();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/Transaction/Set") {
|
||||
state TransactionEnvironment env(params.getDataDir());
|
||||
wait(env.setup());
|
||||
|
@ -763,3 +809,5 @@ TEST_CASE("/fdbserver/ConfigDB/Transaction/BadRangeRead") {
|
|||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
// TODO: Test worker failure detection on ConfigBroadcaster
|
||||
|
|
|
@ -2510,7 +2510,9 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
|||
if (tss_info_by_pair.count(newServer.id())) {
|
||||
tss_info_by_pair[newServer.id()]->onTSSPairRemoved = r->onRemoved;
|
||||
if (tss_info_by_pair[newServer.id()]->wakeUpTracker.canBeSet()) {
|
||||
tss_info_by_pair[newServer.id()]->wakeUpTracker.send(Void());
|
||||
auto p = tss_info_by_pair[newServer.id()]->wakeUpTracker;
|
||||
// This callback could delete tss_info_by_pair[newServer.id()], so use a copy
|
||||
p.send(Void());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -5129,8 +5131,10 @@ ACTOR Future<Void> initializeStorage(DDTeamCollection* self,
|
|||
if (doRecruit && newServer.isError()) {
|
||||
TraceEvent(SevWarn, "DDRecruitmentError").error(newServer.getError());
|
||||
if (!newServer.isError(error_code_recruitment_failed) &&
|
||||
!newServer.isError(error_code_request_maybe_delivered))
|
||||
!newServer.isError(error_code_request_maybe_delivered)) {
|
||||
tssState->markComplete();
|
||||
throw newServer.getError();
|
||||
}
|
||||
wait(delay(SERVER_KNOBS->STORAGE_RECRUITMENT_DELAY, TaskPriority::DataDistribution));
|
||||
}
|
||||
|
||||
|
@ -5248,8 +5252,13 @@ ACTOR Future<Void> storageRecruiter(DDTeamCollection* self,
|
|||
}
|
||||
}
|
||||
int newTssToRecruit = targetTSSInDC - self->tss_info_by_pair.size() - inProgressTSSCount;
|
||||
// FIXME: Should log this if the recruit count stays the same but the other numbers update?
|
||||
if (newTssToRecruit != tssToRecruit) {
|
||||
TraceEvent("TSS_RecruitUpdated", self->distributorId).detail("Count", newTssToRecruit);
|
||||
TraceEvent("TSS_RecruitUpdated", self->distributorId)
|
||||
.detail("Desired", targetTSSInDC)
|
||||
.detail("Existing", self->tss_info_by_pair.size())
|
||||
.detail("InProgress", inProgressTSSCount)
|
||||
.detail("NotStarted", newTssToRecruit);
|
||||
tssToRecruit = newTssToRecruit;
|
||||
|
||||
// if we need to get rid of some TSS processes, signal to either cancel recruitment or kill existing TSS
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
*/
|
||||
|
||||
#include "fdbclient/Notified.h"
|
||||
#include "fdbclient/TransactionLineage.h"
|
||||
#include "fdbserver/LogSystem.h"
|
||||
#include "fdbserver/LogSystemDiskQueueAdapter.h"
|
||||
#include "fdbclient/CommitProxyInterface.h"
|
||||
|
@ -397,8 +398,11 @@ ACTOR Future<Void> queueGetReadVersionRequests(Reference<AsyncVar<ServerDBInfo>
|
|||
GrvProxyStats* stats,
|
||||
GrvTransactionRateInfo* batchRateInfo,
|
||||
TransactionTagMap<uint64_t>* transactionTagCounter) {
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetConsistentReadVersion;
|
||||
loop choose {
|
||||
when(GetReadVersionRequest req = waitNext(readVersionRequests)) {
|
||||
// auto lineage = make_scoped_lineage(&TransactionLineage::txID, req.spanContext.first());
|
||||
// getCurrentLineage()->modify(&TransactionLineage::txID) =
|
||||
// WARNING: this code is run at a high priority, so it needs to do as little work as possible
|
||||
bool canBeQueued = true;
|
||||
if (stats->txnRequestIn.getValue() - stats->txnRequestOut.getValue() >
|
||||
|
@ -720,6 +724,7 @@ ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy,
|
|||
state Span span;
|
||||
|
||||
state int64_t midShardSize = SERVER_KNOBS->MIN_SHARD_BYTES;
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetConsistentReadVersion;
|
||||
addActor.send(monitorDDMetricsChanges(&midShardSize, db));
|
||||
|
||||
addActor.send(getRate(proxy.id(),
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
|
||||
#include "fdbclient/IKnobCollection.h"
|
||||
#include "fdbrpc/Stats.h"
|
||||
#include "fdbserver/ConfigBroadcastFollowerInterface.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/LocalConfiguration.h"
|
||||
#include "fdbserver/OnDemandStore.h"
|
||||
|
@ -166,7 +165,6 @@ class LocalConfigurationImpl {
|
|||
}
|
||||
|
||||
CounterCollection cc;
|
||||
Counter broadcasterChanges;
|
||||
Counter snapshots;
|
||||
Counter changeRequestsFetched;
|
||||
Counter mutations;
|
||||
|
@ -283,39 +281,27 @@ class LocalConfigurationImpl {
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> consumeInternal(LocalConfigurationImpl* self,
|
||||
ConfigBroadcastFollowerInterface broadcaster) {
|
||||
ACTOR static Future<Void> consumeInternal(LocalConfigurationImpl* self, ConfigBroadcastInterface broadcaster) {
|
||||
loop {
|
||||
try {
|
||||
state ConfigBroadcastFollowerGetChangesReply changesReply =
|
||||
wait(broadcaster.getChanges.getReply(ConfigBroadcastFollowerGetChangesRequest{
|
||||
self->lastSeenVersion, self->configKnobOverrides.getConfigClassSet() }));
|
||||
TraceEvent(SevDebug, "LocalConfigGotChanges", self->id)
|
||||
.detail("Size", changesReply.changes.size())
|
||||
.detail("Version", changesReply.mostRecentVersion);
|
||||
wait(self->addChanges(changesReply.changes, changesReply.mostRecentVersion));
|
||||
} catch (Error& e) {
|
||||
if (e.code() == error_code_version_already_compacted) {
|
||||
state ConfigBroadcastFollowerGetSnapshotReply snapshotReply = wait(broadcaster.getSnapshot.getReply(
|
||||
ConfigBroadcastFollowerGetSnapshotRequest{ self->configKnobOverrides.getConfigClassSet() }));
|
||||
ASSERT_GT(snapshotReply.version, self->lastSeenVersion);
|
||||
choose {
|
||||
when(ConfigBroadcastSnapshotRequest snapshotReq = waitNext(broadcaster.snapshot.getFuture())) {
|
||||
ASSERT_GT(snapshotReq.version, self->lastSeenVersion);
|
||||
++self->snapshots;
|
||||
wait(setSnapshot(self, std::move(snapshotReply.snapshot), snapshotReply.version));
|
||||
} else {
|
||||
throw e;
|
||||
wait(setSnapshot(self, std::move(snapshotReq.snapshot), snapshotReq.version));
|
||||
}
|
||||
when(state ConfigBroadcastChangesRequest req = waitNext(broadcaster.changes.getFuture())) {
|
||||
wait(self->addChanges(req.changes, req.mostRecentVersion));
|
||||
req.reply.send(ConfigBroadcastChangesReply());
|
||||
}
|
||||
}
|
||||
wait(yield()); // Necessary to not immediately trigger retry?
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> consume(LocalConfigurationImpl* self,
|
||||
Reference<IAsyncListener<ConfigBroadcastFollowerInterface> const> broadcaster) {
|
||||
ACTOR static Future<Void> consume(LocalConfigurationImpl* self, ConfigBroadcastInterface broadcaster) {
|
||||
ASSERT(self->initFuture.isValid() && self->initFuture.isReady());
|
||||
loop {
|
||||
choose {
|
||||
when(wait(brokenPromiseToNever(consumeInternal(self, broadcaster->get())))) { ASSERT(false); }
|
||||
when(wait(broadcaster->onChange())) { ++self->broadcasterChanges; }
|
||||
when(wait(consumeInternal(self, broadcaster))) { ASSERT(false); }
|
||||
when(wait(self->kvStore->getError())) { ASSERT(false); }
|
||||
}
|
||||
}
|
||||
|
@ -328,8 +314,7 @@ public:
|
|||
IsTest isTest)
|
||||
: id(deterministicRandom()->randomUniqueID()), kvStore(dataFolder, id, "localconf-"),
|
||||
configKnobOverrides(configPath), manualKnobOverrides(manualKnobOverrides), cc("LocalConfiguration"),
|
||||
broadcasterChanges("BroadcasterChanges", cc), snapshots("Snapshots", cc),
|
||||
changeRequestsFetched("ChangeRequestsFetched", cc), mutations("Mutations", cc) {
|
||||
snapshots("Snapshots", cc), changeRequestsFetched("ChangeRequestsFetched", cc), mutations("Mutations", cc) {
|
||||
if (isTest) {
|
||||
testKnobCollection =
|
||||
IKnobCollection::create(IKnobCollection::Type::TEST,
|
||||
|
@ -370,12 +355,16 @@ public:
|
|||
return getKnobs().getTestKnobs();
|
||||
}
|
||||
|
||||
Future<Void> consume(Reference<IAsyncListener<ConfigBroadcastFollowerInterface> const> const& broadcaster) {
|
||||
return consume(this, broadcaster);
|
||||
Future<Void> consume(ConfigBroadcastInterface const& broadcastInterface) {
|
||||
return consume(this, broadcastInterface);
|
||||
}
|
||||
|
||||
UID getID() const { return id; }
|
||||
|
||||
Version getLastSeenVersion() const { return lastSeenVersion; }
|
||||
|
||||
ConfigClassSet configClassSet() const { return configKnobOverrides.getConfigClassSet(); }
|
||||
|
||||
static void testManualKnobOverridesInvalidName() {
|
||||
std::map<std::string, std::string> invalidOverrides;
|
||||
invalidOverrides["knob_name_that_does_not_exist"] = "";
|
||||
|
@ -451,8 +440,7 @@ TestKnobs const& LocalConfiguration::getTestKnobs() const {
|
|||
return impl->getTestKnobs();
|
||||
}
|
||||
|
||||
Future<Void> LocalConfiguration::consume(
|
||||
Reference<IAsyncListener<ConfigBroadcastFollowerInterface> const> const& broadcaster) {
|
||||
Future<Void> LocalConfiguration::consume(ConfigBroadcastInterface const& broadcaster) {
|
||||
return impl->consume(broadcaster);
|
||||
}
|
||||
|
||||
|
@ -465,6 +453,14 @@ UID LocalConfiguration::getID() const {
|
|||
return impl->getID();
|
||||
}
|
||||
|
||||
Version LocalConfiguration::lastSeenVersion() const {
|
||||
return impl->getLastSeenVersion();
|
||||
}
|
||||
|
||||
ConfigClassSet LocalConfiguration::configClassSet() const {
|
||||
return impl->configClassSet();
|
||||
}
|
||||
|
||||
TEST_CASE("/fdbserver/ConfigDB/ManualKnobOverrides/InvalidName") {
|
||||
LocalConfigurationImpl::testManualKnobOverridesInvalidName();
|
||||
return Void();
|
||||
|
|
|
@ -25,7 +25,7 @@
|
|||
#include "fdbclient/ConfigKnobs.h"
|
||||
#include "fdbclient/IKnobCollection.h"
|
||||
#include "fdbclient/PImpl.h"
|
||||
#include "fdbserver/ConfigBroadcastFollowerInterface.h"
|
||||
#include "fdbserver/ConfigBroadcastInterface.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "flow/Arena.h"
|
||||
#include "flow/Knobs.h"
|
||||
|
@ -59,8 +59,10 @@ public:
|
|||
ClientKnobs const& getClientKnobs() const;
|
||||
ServerKnobs const& getServerKnobs() const;
|
||||
TestKnobs const& getTestKnobs() const;
|
||||
Future<Void> consume(Reference<IAsyncListener<ConfigBroadcastFollowerInterface> const> const& broadcaster);
|
||||
Future<Void> consume(ConfigBroadcastInterface const& broadcastInterface);
|
||||
UID getID() const;
|
||||
Version lastSeenVersion() const;
|
||||
ConfigClassSet configClassSet() const;
|
||||
|
||||
public: // Testing
|
||||
Future<Void> addChanges(Standalone<VectorRef<VersionedConfigMutationRef>> versionedMutations,
|
||||
|
|
|
@ -111,8 +111,7 @@ void ILogSystem::ServerPeekCursor::nextMessage() {
|
|||
}
|
||||
|
||||
messageAndTags.loadFromArena(&rd, &messageVersion.sub);
|
||||
DEBUG_TAGS_AND_MESSAGE("ServerPeekCursor", messageVersion.version, messageAndTags.getRawMessage())
|
||||
.detail("CursorID", this->randomID);
|
||||
DEBUG_TAGS_AND_MESSAGE("ServerPeekCursor", messageVersion.version, messageAndTags.getRawMessage(), this->randomID);
|
||||
// Rewind and consume the header so that reader() starts from the message.
|
||||
rd.rewind();
|
||||
rd.readBytes(messageAndTags.getHeaderSize());
|
||||
|
|
|
@ -635,9 +635,8 @@ ACTOR static Future<Void> finishMoveKeys(Database occ,
|
|||
state int retries = 0;
|
||||
state FlowLock::Releaser releaser;
|
||||
|
||||
state std::vector<std::pair<UID, UID>> tssToKill;
|
||||
state std::unordered_set<UID> tssToIgnore;
|
||||
// try waiting for tss for a 2 loops, give up if they're stuck to not affect the rest of the cluster
|
||||
// try waiting for tss for a 2 loops, give up if they're behind to not affect the rest of the cluster
|
||||
state int waitForTSSCounter = 2;
|
||||
|
||||
ASSERT(!destinationTeam.empty());
|
||||
|
@ -657,22 +656,6 @@ ACTOR static Future<Void> finishMoveKeys(Database occ,
|
|||
// printf("finishMoveKeys( '%s'-'%s' )\n", begin.toString().c_str(), keys.end.toString().c_str());
|
||||
loop {
|
||||
try {
|
||||
if (tssToKill.size()) {
|
||||
TEST(true); // killing TSS because they were unavailable for movekeys
|
||||
|
||||
// Kill tss BEFORE committing main txn so that client requests don't make it to the tss when it
|
||||
// has a different shard set than its pair use a different RYW transaction since i'm too lazy
|
||||
// (and don't want to add bugs) by changing whole method to RYW. Also, using a different
|
||||
// transaction makes it commit earlier which we may need to guarantee causality of tss getting
|
||||
// removed before client sends a request to this key range on the new SS
|
||||
wait(removeTSSPairsFromCluster(occ, tssToKill));
|
||||
|
||||
for (auto& tssPair : tssToKill) {
|
||||
TraceEvent(SevWarnAlways, "TSS_KillMoveKeys").detail("TSSID", tssPair.second);
|
||||
tssToIgnore.insert(tssPair.second);
|
||||
}
|
||||
tssToKill.clear();
|
||||
}
|
||||
|
||||
tr.info.taskID = TaskPriority::MoveKeys;
|
||||
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
|
||||
|
@ -877,8 +860,8 @@ ACTOR static Future<Void> finishMoveKeys(Database occ,
|
|||
TaskPriority::MoveKeys));
|
||||
|
||||
// Check to see if we're waiting only on tss. If so, decrement the waiting counter.
|
||||
// If the waiting counter is zero, kill the slow/non-responsive tss processes before finalizing the
|
||||
// data move.
|
||||
// If the waiting counter is zero, ignore the slow/non-responsive tss processes before finalizing
|
||||
// the data move.
|
||||
if (tssReady.size()) {
|
||||
bool allSSDone = true;
|
||||
for (auto& f : serverReady) {
|
||||
|
@ -902,12 +885,9 @@ ACTOR static Future<Void> finishMoveKeys(Database occ,
|
|||
if (anyTssNotDone && waitForTSSCounter == 0) {
|
||||
for (int i = 0; i < tssReady.size(); i++) {
|
||||
if (!tssReady[i].isReady() || tssReady[i].isError()) {
|
||||
tssToKill.push_back(
|
||||
std::pair(tssReadyInterfs[i].tssPairID.get(), tssReadyInterfs[i].id()));
|
||||
tssToIgnore.insert(tssReadyInterfs[i].id());
|
||||
}
|
||||
}
|
||||
// repeat loop and go back to start to kill tss' before continuing on
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -920,22 +900,11 @@ ACTOR static Future<Void> finishMoveKeys(Database occ,
|
|||
for (int s = 0; s < tssReady.size(); s++)
|
||||
tssCount += tssReady[s].isReady() && !tssReady[s].isError();
|
||||
|
||||
/*if (tssReady.size()) {
|
||||
printf(" fMK: [%s - %s) moved data to %d/%d servers and %d/%d tss\n",
|
||||
begin.toString().c_str(),
|
||||
keys.end.toString().c_str(),
|
||||
count,
|
||||
serverReady.size(),
|
||||
tssCount,
|
||||
tssReady.size());
|
||||
} else {
|
||||
printf(" fMK: [%s - %s) moved data to %d/%d servers\n",
|
||||
begin.toString().c_str(),
|
||||
keys.end.toString().c_str(),
|
||||
count,
|
||||
serverReady.size());
|
||||
}*/
|
||||
TraceEvent(SevDebug, waitInterval.end(), relocationIntervalId).detail("ReadyServers", count);
|
||||
TraceEvent readyServersEv(SevDebug, waitInterval.end(), relocationIntervalId);
|
||||
readyServersEv.detail("ReadyServers", count);
|
||||
if (tssReady.size()) {
|
||||
readyServersEv.detail("ReadyTSS", tssCount);
|
||||
}
|
||||
|
||||
if (count == dest.size()) {
|
||||
// update keyServers, serverKeys
|
||||
|
|
|
@ -18,7 +18,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include <algorithm>
|
||||
#include <vector>
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbserver/MutationTracking.h"
|
||||
#include "fdbserver/LogProtocolMessage.h"
|
||||
#include "fdbserver/SpanContextMessage.h"
|
||||
|
@ -27,43 +29,55 @@
|
|||
#error "You cannot use mutation tracking in a clean/release build."
|
||||
#endif
|
||||
|
||||
// Track up to 2 keys in simulation via enabling MUTATION_TRACKING_ENABLED and setting the keys here.
|
||||
StringRef debugKey = LiteralStringRef("");
|
||||
StringRef debugKey2 = LiteralStringRef("\xff\xff\xff\xff");
|
||||
// If MUTATION_TRACKING_ENABLED is set, MutationTracking events will be logged for the
|
||||
// keys in debugKeys and the ranges in debugRanges.
|
||||
// Each entry is a pair of (label, keyOrRange) and the Label will be attached to the
|
||||
// MutationTracking TraceEvent for easier searching/recognition.
|
||||
std::vector<std::pair<const char *, KeyRef>> debugKeys = {
|
||||
{"SomeKey", "foo"_sr}
|
||||
};
|
||||
std::vector<std::pair<const char *, KeyRangeRef>> debugRanges = {
|
||||
{"Everything", {""_sr, "\xff\xff\xff\xff"_sr}}
|
||||
};
|
||||
|
||||
TraceEvent debugMutationEnabled(const char* context, Version version, MutationRef const& mutation) {
|
||||
if ((mutation.type == mutation.ClearRange || mutation.type == mutation.DebugKeyRange) &&
|
||||
((mutation.param1 <= debugKey && mutation.param2 > debugKey) ||
|
||||
(mutation.param1 <= debugKey2 && mutation.param2 > debugKey2))) {
|
||||
TraceEvent event("MutationTracking");
|
||||
event.detail("At", context)
|
||||
.detail("Version", version)
|
||||
.detail("MutationType", typeString[mutation.type])
|
||||
.detail("KeyBegin", mutation.param1)
|
||||
.detail("KeyEnd", mutation.param2);
|
||||
return event;
|
||||
} else if (mutation.param1 == debugKey || mutation.param1 == debugKey2) {
|
||||
TraceEvent event("MutationTracking");
|
||||
event.detail("At", context)
|
||||
.detail("Version", version)
|
||||
.detail("MutationType", typeString[mutation.type])
|
||||
.detail("Key", mutation.param1)
|
||||
.detail("Value", mutation.param2);
|
||||
return event;
|
||||
} else {
|
||||
return TraceEvent();
|
||||
TraceEvent debugMutationEnabled(const char* context, Version version, MutationRef const& mutation, UID id) {
|
||||
const char *label = nullptr;
|
||||
|
||||
for(auto &labelKey : debugKeys) {
|
||||
if(((mutation.type == mutation.ClearRange || mutation.type == mutation.DebugKeyRange) &&
|
||||
KeyRangeRef(mutation.param1, mutation.param2).contains(labelKey.second)) ||
|
||||
mutation.param1 == labelKey.second) {
|
||||
label = labelKey.first;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for(auto &labelRange : debugRanges) {
|
||||
if(((mutation.type == mutation.ClearRange || mutation.type == mutation.DebugKeyRange) &&
|
||||
KeyRangeRef(mutation.param1, mutation.param2).intersects(labelRange.second)) ||
|
||||
labelRange.second.contains(mutation.param1)) {
|
||||
label = labelRange.first;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if(label != nullptr) {
|
||||
TraceEvent event("MutationTracking", id);
|
||||
event.detail("Label", label)
|
||||
.detail("At", context)
|
||||
.detail("Version", version)
|
||||
.detail("Mutation", mutation);
|
||||
return event;
|
||||
}
|
||||
|
||||
return TraceEvent();
|
||||
}
|
||||
|
||||
TraceEvent debugKeyRangeEnabled(const char* context, Version version, KeyRangeRef const& keys) {
|
||||
if (keys.contains(debugKey) || keys.contains(debugKey2)) {
|
||||
return debugMutation(context, version, MutationRef(MutationRef::DebugKeyRange, keys.begin, keys.end));
|
||||
} else {
|
||||
return TraceEvent();
|
||||
}
|
||||
TraceEvent debugKeyRangeEnabled(const char* context, Version version, KeyRangeRef const& keys, UID id) {
|
||||
return debugMutation(context, version, MutationRef(MutationRef::DebugKeyRange, keys.begin, keys.end), id);
|
||||
}
|
||||
|
||||
TraceEvent debugTagsAndMessageEnabled(const char* context, Version version, StringRef commitBlob) {
|
||||
TraceEvent debugTagsAndMessageEnabled(const char* context, Version version, StringRef commitBlob, UID id) {
|
||||
BinaryReader rdr(commitBlob, AssumeVersion(g_network->protocolVersion()));
|
||||
while (!rdr.empty()) {
|
||||
if (*(int32_t*)rdr.peekBytes(4) == VERSION_HEADER) {
|
||||
|
@ -93,7 +107,7 @@ TraceEvent debugTagsAndMessageEnabled(const char* context, Version version, Stri
|
|||
MutationRef m;
|
||||
BinaryReader br(mutationData, AssumeVersion(rdr.protocolVersion()));
|
||||
br >> m;
|
||||
TraceEvent event = debugMutation(context, version, m);
|
||||
TraceEvent event = debugMutation(context, version, m, id);
|
||||
if (event.isEnabled()) {
|
||||
event.detail("MessageTags", msg.tags);
|
||||
return event;
|
||||
|
@ -104,23 +118,23 @@ TraceEvent debugTagsAndMessageEnabled(const char* context, Version version, Stri
|
|||
}
|
||||
|
||||
#if MUTATION_TRACKING_ENABLED
|
||||
TraceEvent debugMutation(const char* context, Version version, MutationRef const& mutation) {
|
||||
return debugMutationEnabled(context, version, mutation);
|
||||
TraceEvent debugMutation(const char* context, Version version, MutationRef const& mutation, UID id) {
|
||||
return debugMutationEnabled(context, version, mutation, id);
|
||||
}
|
||||
TraceEvent debugKeyRange(const char* context, Version version, KeyRangeRef const& keys) {
|
||||
return debugKeyRangeEnabled(context, version, keys);
|
||||
TraceEvent debugKeyRange(const char* context, Version version, KeyRangeRef const& keys, UID id) {
|
||||
return debugKeyRangeEnabled(context, version, keys, id);
|
||||
}
|
||||
TraceEvent debugTagsAndMessage(const char* context, Version version, StringRef commitBlob) {
|
||||
return debugTagsAndMessageEnabled(context, version, commitBlob);
|
||||
TraceEvent debugTagsAndMessage(const char* context, Version version, StringRef commitBlob, UID id) {
|
||||
return debugTagsAndMessageEnabled(context, version, commitBlob, id);
|
||||
}
|
||||
#else
|
||||
TraceEvent debugMutation(const char* context, Version version, MutationRef const& mutation) {
|
||||
TraceEvent debugMutation(const char* context, Version version, MutationRef const& mutation, UID id) {
|
||||
return TraceEvent();
|
||||
}
|
||||
TraceEvent debugKeyRange(const char* context, Version version, KeyRangeRef const& keys) {
|
||||
TraceEvent debugKeyRange(const char* context, Version version, KeyRangeRef const& keys, UID id) {
|
||||
return TraceEvent();
|
||||
}
|
||||
TraceEvent debugTagsAndMessage(const char* context, Version version, StringRef commitBlob) {
|
||||
TraceEvent debugTagsAndMessage(const char* context, Version version, StringRef commitBlob, UID id) {
|
||||
return TraceEvent();
|
||||
}
|
||||
#endif
|
||||
|
|
|
@ -28,19 +28,18 @@
|
|||
#define MUTATION_TRACKING_ENABLED 0
|
||||
// The keys to track are defined in the .cpp file to limit recompilation.
|
||||
|
||||
#define DEBUG_MUTATION(context, version, mutation) MUTATION_TRACKING_ENABLED&& debugMutation(context, version, mutation)
|
||||
TraceEvent debugMutation(const char* context, Version version, MutationRef const& mutation);
|
||||
#define DEBUG_MUTATION(...) MUTATION_TRACKING_ENABLED && debugMutation(__VA_ARGS__)
|
||||
TraceEvent debugMutation(const char* context, Version version, MutationRef const& mutation, UID id = UID());
|
||||
|
||||
// debugKeyRange and debugTagsAndMessage only log the *first* occurrence of a key in their range/commit.
|
||||
// TODO: Create a TraceEventGroup that forwards all calls to each element of a vector<TraceEvent>,
|
||||
// to allow "multiple" TraceEvents to be returned.
|
||||
|
||||
#define DEBUG_KEY_RANGE(context, version, keys) MUTATION_TRACKING_ENABLED&& debugKeyRange(context, version, keys)
|
||||
TraceEvent debugKeyRange(const char* context, Version version, KeyRangeRef const& keys);
|
||||
#define DEBUG_KEY_RANGE(...) MUTATION_TRACKING_ENABLED && debugKeyRange(__VA_ARGS__)
|
||||
TraceEvent debugKeyRange(const char* context, Version version, KeyRangeRef const& keys, UID id = UID());
|
||||
|
||||
#define DEBUG_TAGS_AND_MESSAGE(context, version, commitBlob) \
|
||||
MUTATION_TRACKING_ENABLED&& debugTagsAndMessage(context, version, commitBlob)
|
||||
TraceEvent debugTagsAndMessage(const char* context, Version version, StringRef commitBlob);
|
||||
#define DEBUG_TAGS_AND_MESSAGE(...) MUTATION_TRACKING_ENABLED && debugTagsAndMessage(__VA_ARGS__)
|
||||
TraceEvent debugTagsAndMessage(const char* context, Version version, StringRef commitBlob, UID id = UID());
|
||||
|
||||
// TODO: Version Tracking. If the bug is in handling a version rather than a key, then it'd be good to be able to log
|
||||
// each time that version is handled within simulation. A similar set of functions should be implemented.
|
||||
|
|
|
@ -62,9 +62,8 @@ struct StagingKey {
|
|||
void add(const MutationRef& m, LogMessageVersion newVersion) {
|
||||
ASSERT(m.type != MutationRef::SetVersionstampedKey && m.type != MutationRef::SetVersionstampedValue);
|
||||
DEBUG_MUTATION("StagingKeyAdd", newVersion.version, m)
|
||||
.detail("Version", version.toString())
|
||||
.detail("NewVersion", newVersion.toString())
|
||||
.detail("Mutation", m);
|
||||
.detail("SubVersion", version.toString())
|
||||
.detail("NewSubVersion", newVersion.toString());
|
||||
if (version == newVersion) {
|
||||
// This could happen because the same mutation can be present in
|
||||
// overlapping mutation logs, because new TLogs can copy mutations
|
||||
|
@ -84,8 +83,8 @@ struct StagingKey {
|
|||
}
|
||||
if (version < newVersion) {
|
||||
DEBUG_MUTATION("StagingKeyAdd", newVersion.version, m)
|
||||
.detail("Version", version.toString())
|
||||
.detail("NewVersion", newVersion.toString())
|
||||
.detail("SubVersion", version.toString())
|
||||
.detail("NewSubVersion", newVersion.toString())
|
||||
.detail("MType", getTypeString(type))
|
||||
.detail("Key", key)
|
||||
.detail("Val", val)
|
||||
|
|
|
@ -892,8 +892,7 @@ ACTOR Future<Void> sendMutationsToApplier(
|
|||
MutationRef mutation = mvector[splitMutationIndex];
|
||||
UID applierID = nodeIDs[splitMutationIndex];
|
||||
DEBUG_MUTATION("RestoreLoaderSplittedMutation", commitVersion.version, mutation)
|
||||
.detail("Version", commitVersion.toString())
|
||||
.detail("Mutation", mutation);
|
||||
.detail("Version", commitVersion.toString());
|
||||
// CAREFUL: The splitted mutations' lifetime is shorter than the for-loop
|
||||
// Must use deep copy for splitted mutations
|
||||
applierVersionedMutationsBuffer[applierID].push_back_deep(
|
||||
|
@ -911,8 +910,7 @@ ACTOR Future<Void> sendMutationsToApplier(
|
|||
|
||||
DEBUG_MUTATION("RestoreLoaderSendMutation", commitVersion.version, kvm)
|
||||
.detail("Applier", applierID)
|
||||
.detail("Version", commitVersion.toString())
|
||||
.detail("Mutation", kvm);
|
||||
.detail("SubVersion", commitVersion.toString());
|
||||
// kvm data is saved in pkvOps in batchData, so shallow copy is ok here.
|
||||
applierVersionedMutationsBuffer[applierID].push_back(applierVersionedMutationsBuffer[applierID].arena(),
|
||||
VersionedMutation(kvm, commitVersion));
|
||||
|
@ -996,7 +994,7 @@ void splitMutation(const KeyRangeMap<UID>& krMap,
|
|||
VectorRef<MutationRef>& mvector,
|
||||
Arena& nodeIDs_arena,
|
||||
VectorRef<UID>& nodeIDs) {
|
||||
TraceEvent(SevDebug, "FastRestoreSplitMutation").detail("Mutation", m.toString());
|
||||
TraceEvent(SevDebug, "FastRestoreSplitMutation").detail("Mutation", m);
|
||||
ASSERT(mvector.empty());
|
||||
ASSERT(nodeIDs.empty());
|
||||
auto r = krMap.intersectingRanges(KeyRangeRef(m.param1, m.param2));
|
||||
|
@ -1527,15 +1525,15 @@ TEST_CASE("/FastRestore/RestoreLoader/splitMutation") {
|
|||
KeyRange krange(KeyRangeRef(result.param1, result.param2));
|
||||
KeyRange krange2(KeyRangeRef(result2.param1, result2.param2));
|
||||
TraceEvent("Result")
|
||||
.detail("KeyRange1", krange.toString())
|
||||
.detail("KeyRange2", krange2.toString())
|
||||
.detail("KeyRange1", krange)
|
||||
.detail("KeyRange2", krange2)
|
||||
.detail("ApplierID1", applierID)
|
||||
.detail("ApplierID2", applierID2);
|
||||
if (krange != krange2 || applierID != applierID2) {
|
||||
TraceEvent(SevError, "IncorrectResult")
|
||||
.detail("Mutation", mutation.toString())
|
||||
.detail("KeyRange1", krange.toString())
|
||||
.detail("KeyRange2", krange2.toString())
|
||||
.detail("Mutation", mutation)
|
||||
.detail("KeyRange1", krange)
|
||||
.detail("KeyRange2", krange2)
|
||||
.detail("ApplierID1", applierID)
|
||||
.detail("ApplierID2", applierID2);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* RoleLineage.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2020 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbserver/RoleLineage.actor.h"
|
||||
|
||||
using namespace std::literals;
|
||||
|
||||
std::string_view RoleLineage::name = "RoleLineage"sv;
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* RoleLineage.actor.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2020 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
#include "flow/flow.h"
|
||||
#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_ROLE_LINEAGE_ACTOR_G_H)
|
||||
#define FDBSERVER_ROLE_LINEAGE_ACTOR_G_H
|
||||
#include "fdbserver/RoleLineage.actor.g.h"
|
||||
#elif !defined(FDBSERVER_ROLE_LINEAGE_ACTOR_H)
|
||||
#define FDBSERVER_ROLE_LINEAGE_ACTOR_H
|
||||
|
||||
#include "flow/singleton.h"
|
||||
#include "fdbrpc/Locality.h"
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
|
||||
#include <string_view>
|
||||
#include <msgpack.hpp>
|
||||
#include <any>
|
||||
#include "flow/actorcompiler.h" // This must be the last include
|
||||
|
||||
struct RoleLineage : LineageProperties<RoleLineage> {
|
||||
static std::string_view name;
|
||||
ProcessClass::ClusterRole role = ProcessClass::NoRole;
|
||||
|
||||
bool isSet(ProcessClass::ClusterRole RoleLineage::*member) const { return this->*member != ProcessClass::NoRole; }
|
||||
};
|
||||
|
||||
struct RoleLineageCollector : IALPCollector<RoleLineage> {
|
||||
RoleLineageCollector() : IALPCollector() {}
|
||||
std::optional<std::any> collect(ActorLineage* lineage) override {
|
||||
auto res = lineage->get(&RoleLineage::role);
|
||||
if (res.has_value()) {
|
||||
return Role::get(res.value()).abbreviation;
|
||||
} else {
|
||||
return std::optional<std::any>();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// creates a new root and sets the role lineage
|
||||
ACTOR template <class Fun>
|
||||
Future<decltype(std::declval<Fun>()())> runInRole(Fun fun, ProcessClass::ClusterRole role) {
|
||||
getCurrentLineage()->makeRoot();
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = role;
|
||||
decltype(std::declval<Fun>()()) res = wait(fun());
|
||||
return res;
|
||||
}
|
||||
|
||||
#endif
|
|
@ -26,7 +26,6 @@
|
|||
#define FDBSERVER_SERVERDBINFO_H
|
||||
#pragma once
|
||||
|
||||
#include "fdbserver/ConfigBroadcastFollowerInterface.h"
|
||||
#include "fdbserver/DataDistributorInterface.h"
|
||||
#include "fdbserver/MasterInterface.h"
|
||||
#include "fdbserver/LogSystemConfig.h"
|
||||
|
@ -63,7 +62,6 @@ struct ServerDBInfo {
|
|||
// which need to stay alive in case this recovery fails
|
||||
Optional<LatencyBandConfig> latencyBandConfig;
|
||||
int64_t infoGeneration;
|
||||
ConfigBroadcastFollowerInterface configBroadcaster;
|
||||
|
||||
ServerDBInfo()
|
||||
: recoveryCount(0), recoveryState(RecoveryState::UNINITIALIZED), logSystemConfig(0), infoGeneration(0) {}
|
||||
|
@ -87,8 +85,7 @@ struct ServerDBInfo {
|
|||
logSystemConfig,
|
||||
priorCommittedLogServers,
|
||||
latencyBandConfig,
|
||||
infoGeneration,
|
||||
configBroadcaster);
|
||||
infoGeneration);
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -0,0 +1,24 @@
|
|||
#include "flow/flow.h"
|
||||
#include "fdbclient/StackLineage.h"
|
||||
#include <csignal>
|
||||
#include <iostream>
|
||||
#include <string_view>
|
||||
|
||||
// This is not yet correct, as this is not async safe
|
||||
// However, this should be good enough for an initial
|
||||
// proof of concept.
|
||||
extern "C" void stackSignalHandler(int sig) {
|
||||
auto stack = getActorStackTrace();
|
||||
int i = 0;
|
||||
while (!stack.empty()) {
|
||||
auto s = stack.back();
|
||||
stack.pop_back();
|
||||
std::string_view n(reinterpret_cast<const char*>(s.begin()), s.size());
|
||||
std::cout << i << ": " << n << std::endl;
|
||||
++i;
|
||||
}
|
||||
}
|
||||
|
||||
void setupStackSignal() {
|
||||
std::signal(SIGUSR1, &stackSignalHandler);
|
||||
}
|
|
@ -18,8 +18,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "fdbserver/ConfigBroadcastFollowerInterface.h"
|
||||
#include "fdbserver/ConfigBroadcastInterface.h"
|
||||
#include "fdbserver/SimpleConfigConsumer.h"
|
||||
#include "flow/actorcompiler.h" // must be last include
|
||||
|
||||
class SimpleConfigConsumerImpl {
|
||||
ConfigFollowerInterface cfi;
|
||||
|
|
|
@ -1582,7 +1582,7 @@ void CacheRangeInfo::addMutation(Version version, MutationRef const& mutation) {
|
|||
// even allow them on un-assigned range?)
|
||||
TraceEvent(SevError, "DeliveredToNotAssigned")
|
||||
.detail("Version", version)
|
||||
.detail("Mutation", mutation.toString());
|
||||
.detail("Mutation", mutation);
|
||||
ASSERT(false); // Mutation delivered to notAssigned cacheRange!
|
||||
}
|
||||
}
|
||||
|
@ -1719,7 +1719,7 @@ public:
|
|||
|
||||
DEBUG_MUTATION("SCUpdateMutation", ver, m);
|
||||
if (m.param1.startsWith(systemKeys.end)) {
|
||||
//TraceEvent("SCPrivateData", data->thisServerID).detail("Mutation", m.toString()).detail("Version", ver);
|
||||
//TraceEvent("SCPrivateData", data->thisServerID).detail("Mutation", m).detail("Version", ver);
|
||||
applyPrivateCacheData(data, m);
|
||||
} else {
|
||||
splitMutation(data, data->cachedRangeMap, m, ver);
|
||||
|
@ -2011,7 +2011,7 @@ ACTOR Future<Void> pullAsyncData(StorageCacheData* data) {
|
|||
}
|
||||
} else {
|
||||
TraceEvent(SevError, "DiscardingPeekedData", data->thisServerID)
|
||||
.detail("Mutation", msg.toString())
|
||||
.detail("Mutation", msg)
|
||||
.detail("CursorVersion", cloneCursor2->version().version)
|
||||
.detail("DataVersion", data->version.get());
|
||||
}
|
||||
|
|
|
@ -1419,9 +1419,8 @@ void commitMessages(TLogData* self,
|
|||
block.reserve(block.arena(), std::max<int64_t>(SERVER_KNOBS->TLOG_MESSAGE_BLOCK_BYTES, msgSize));
|
||||
}
|
||||
|
||||
DEBUG_TAGS_AND_MESSAGE("TLogCommitMessages", version, msg.getRawMessage())
|
||||
.detail("UID", self->dbgid)
|
||||
.detail("LogId", logData->logId);
|
||||
DEBUG_TAGS_AND_MESSAGE("TLogCommitMessages", version, msg.getRawMessage(), logData->logId)
|
||||
.detail("DebugID", self->dbgid);
|
||||
block.append(block.arena(), msg.message.begin(), msg.message.size());
|
||||
for (auto tag : msg.tags) {
|
||||
if (logData->locality == tagLocalitySatellite) {
|
||||
|
@ -1552,8 +1551,7 @@ void peekMessagesFromMemory(Reference<LogData> self,
|
|||
messages << it->second.toStringRef();
|
||||
void* data = messages.getData();
|
||||
DEBUG_TAGS_AND_MESSAGE(
|
||||
"TLogPeek", currentVersion, StringRef((uint8_t*)data + offset, messages.getLength() - offset))
|
||||
.detail("LogId", self->logId)
|
||||
"TLogPeek", currentVersion, StringRef((uint8_t*)data + offset, messages.getLength() - offset), self->logId)
|
||||
.detail("PeekTag", req.tag);
|
||||
}
|
||||
}
|
||||
|
@ -1823,9 +1821,8 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
|
|||
wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags));
|
||||
for (const StringRef& msg : rawMessages) {
|
||||
messages.serializeBytes(msg);
|
||||
DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg)
|
||||
.detail("UID", self->dbgid)
|
||||
.detail("LogId", logData->logId)
|
||||
DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg, logData->logId)
|
||||
.detail("DebugID", self->dbgid)
|
||||
.detail("PeekTag", req.tag);
|
||||
}
|
||||
|
||||
|
|
|
@ -1721,7 +1721,7 @@ struct RedwoodMetrics {
|
|||
for (auto& m : metrics) {
|
||||
char c = m.first[0];
|
||||
if (c != 0 && (!skipZeroes || m.second != 0)) {
|
||||
e->detail(format("L%d%s", i + 1, m.first + (c == '-' ? 1 : 0)), m.second);
|
||||
e->detail(format("L%d%s", i, m.first + (c == '-' ? 1 : 0)), m.second);
|
||||
}
|
||||
}
|
||||
metric.events.toTraceEvent(e, i);
|
||||
|
|
|
@ -39,6 +39,7 @@
|
|||
#include "fdbrpc/MultiInterface.h"
|
||||
#include "fdbclient/ClientWorkerInterface.h"
|
||||
#include "fdbserver/RecoveryState.h"
|
||||
#include "fdbserver/ConfigBroadcastInterface.h"
|
||||
#include "flow/actorcompiler.h"
|
||||
|
||||
struct WorkerInterface {
|
||||
|
@ -67,6 +68,7 @@ struct WorkerInterface {
|
|||
RequestStream<struct WorkerSnapRequest> workerSnapReq;
|
||||
RequestStream<struct UpdateServerDBInfoRequest> updateServerDBInfo;
|
||||
|
||||
ConfigBroadcastInterface configBroadcastInterface;
|
||||
TesterInterface testerInterface;
|
||||
|
||||
UID id() const { return tLog.getEndpoint().token; }
|
||||
|
@ -117,7 +119,8 @@ struct WorkerInterface {
|
|||
execReq,
|
||||
workerSnapReq,
|
||||
backup,
|
||||
updateServerDBInfo);
|
||||
updateServerDBInfo,
|
||||
configBroadcastInterface);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -376,6 +379,8 @@ struct RegisterWorkerRequest {
|
|||
std::vector<NetworkAddress> incompatiblePeers;
|
||||
ReplyPromise<RegisterWorkerReply> reply;
|
||||
bool degraded;
|
||||
Version lastSeenKnobVersion;
|
||||
ConfigClassSet knobConfigClassSet;
|
||||
|
||||
RegisterWorkerRequest()
|
||||
: priorityInfo(ProcessClass::UnsetFit, false, ClusterControllerPriorityInfo::FitnessUnknown), degraded(false) {}
|
||||
|
@ -386,9 +391,12 @@ struct RegisterWorkerRequest {
|
|||
Generation generation,
|
||||
Optional<DataDistributorInterface> ddInterf,
|
||||
Optional<RatekeeperInterface> rkInterf,
|
||||
bool degraded)
|
||||
bool degraded,
|
||||
Version lastSeenKnobVersion,
|
||||
ConfigClassSet knobConfigClassSet)
|
||||
: wi(wi), initialClass(initialClass), processClass(processClass), priorityInfo(priorityInfo),
|
||||
generation(generation), distributorInterf(ddInterf), ratekeeperInterf(rkInterf), degraded(degraded) {}
|
||||
generation(generation), distributorInterf(ddInterf), ratekeeperInterf(rkInterf), degraded(degraded),
|
||||
lastSeenKnobVersion(lastSeenKnobVersion), knobConfigClassSet(knobConfigClassSet) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
|
@ -403,7 +411,9 @@ struct RegisterWorkerRequest {
|
|||
issues,
|
||||
incompatiblePeers,
|
||||
reply,
|
||||
degraded);
|
||||
degraded,
|
||||
lastSeenKnobVersion,
|
||||
knobConfigClassSet);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -811,6 +821,41 @@ struct Role {
|
|||
std::string abbreviation;
|
||||
bool includeInTraceRoles;
|
||||
|
||||
static const Role& get(ProcessClass::ClusterRole role) {
|
||||
switch (role) {
|
||||
case ProcessClass::Storage:
|
||||
return STORAGE_SERVER;
|
||||
case ProcessClass::TLog:
|
||||
return TRANSACTION_LOG;
|
||||
case ProcessClass::CommitProxy:
|
||||
return COMMIT_PROXY;
|
||||
case ProcessClass::GrvProxy:
|
||||
return GRV_PROXY;
|
||||
case ProcessClass::Master:
|
||||
return MASTER;
|
||||
case ProcessClass::Resolver:
|
||||
return RESOLVER;
|
||||
case ProcessClass::LogRouter:
|
||||
return LOG_ROUTER;
|
||||
case ProcessClass::ClusterController:
|
||||
return CLUSTER_CONTROLLER;
|
||||
case ProcessClass::DataDistributor:
|
||||
return DATA_DISTRIBUTOR;
|
||||
case ProcessClass::Ratekeeper:
|
||||
return RATEKEEPER;
|
||||
case ProcessClass::StorageCache:
|
||||
return STORAGE_CACHE;
|
||||
case ProcessClass::Backup:
|
||||
return BACKUP;
|
||||
case ProcessClass::Worker:
|
||||
return WORKER;
|
||||
case ProcessClass::NoRole:
|
||||
default:
|
||||
ASSERT(false);
|
||||
throw internal_error();
|
||||
}
|
||||
}
|
||||
|
||||
bool operator==(const Role& r) const { return roleName == r.roleName; }
|
||||
bool operator!=(const Role& r) const { return !(*this == r); }
|
||||
|
||||
|
|
|
@ -35,6 +35,7 @@
|
|||
#include <boost/algorithm/string.hpp>
|
||||
#include <boost/interprocess/managed_shared_memory.hpp>
|
||||
|
||||
#include "fdbclient/ActorLineageProfiler.h"
|
||||
#include "fdbclient/IKnobCollection.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
|
@ -67,6 +68,7 @@
|
|||
#include "flow/SystemMonitor.h"
|
||||
#include "flow/TLSConfig.actor.h"
|
||||
#include "flow/Tracing.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
#include "flow/UnitTest.h"
|
||||
#include "flow/FaultInjection.h"
|
||||
|
||||
|
@ -86,6 +88,8 @@
|
|||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
using namespace std::literals;
|
||||
|
||||
// clang-format off
|
||||
enum {
|
||||
OPT_CONNFILE, OPT_SEEDCONNFILE, OPT_SEEDCONNSTRING, OPT_ROLE, OPT_LISTEN, OPT_PUBLICADDR, OPT_DATAFOLDER, OPT_LOGFOLDER, OPT_PARENTPID, OPT_TRACER, OPT_NEWCONSOLE,
|
||||
|
@ -93,7 +97,7 @@ enum {
|
|||
OPT_DCID, OPT_MACHINE_CLASS, OPT_BUGGIFY, OPT_VERSION, OPT_BUILD_FLAGS, OPT_CRASHONERROR, OPT_HELP, OPT_NETWORKIMPL, OPT_NOBUFSTDOUT, OPT_BUFSTDOUTERR,
|
||||
OPT_TRACECLOCK, OPT_NUMTESTERS, OPT_DEVHELP, OPT_ROLLSIZE, OPT_MAXLOGS, OPT_MAXLOGSSIZE, OPT_KNOB, OPT_UNITTESTPARAM, OPT_TESTSERVERS, OPT_TEST_ON_SERVERS, OPT_METRICSCONNFILE,
|
||||
OPT_METRICSPREFIX, OPT_LOGGROUP, OPT_LOCALITY, OPT_IO_TRUST_SECONDS, OPT_IO_TRUST_WARN_ONLY, OPT_FILESYSTEM, OPT_PROFILER_RSS_SIZE, OPT_KVFILE,
|
||||
OPT_TRACE_FORMAT, OPT_WHITELIST_BINPATH, OPT_BLOB_CREDENTIAL_FILE, OPT_CONFIG_PATH, OPT_USE_TEST_CONFIG_DB, OPT_FAULT_INJECTION,
|
||||
OPT_TRACE_FORMAT, OPT_WHITELIST_BINPATH, OPT_BLOB_CREDENTIAL_FILE, OPT_CONFIG_PATH, OPT_USE_TEST_CONFIG_DB, OPT_FAULT_INJECTION, OPT_PROFILER
|
||||
};
|
||||
|
||||
CSimpleOpt::SOption g_rgOptions[] = {
|
||||
|
@ -173,13 +177,14 @@ CSimpleOpt::SOption g_rgOptions[] = {
|
|||
{ OPT_METRICSPREFIX, "--metrics_prefix", SO_REQ_SEP },
|
||||
{ OPT_IO_TRUST_SECONDS, "--io_trust_seconds", SO_REQ_SEP },
|
||||
{ OPT_IO_TRUST_WARN_ONLY, "--io_trust_warn_only", SO_NONE },
|
||||
{ OPT_TRACE_FORMAT , "--trace_format", SO_REQ_SEP },
|
||||
{ OPT_TRACE_FORMAT, "--trace_format", SO_REQ_SEP },
|
||||
{ OPT_WHITELIST_BINPATH, "--whitelist_binpath", SO_REQ_SEP },
|
||||
{ OPT_BLOB_CREDENTIAL_FILE, "--blob_credential_file", SO_REQ_SEP },
|
||||
{ OPT_CONFIG_PATH, "--config_path", SO_REQ_SEP },
|
||||
{ OPT_USE_TEST_CONFIG_DB, "--use_test_config_db", SO_NONE },
|
||||
{ OPT_FAULT_INJECTION, "-fi", SO_REQ_SEP },
|
||||
{ OPT_FAULT_INJECTION, "--fault_injection", SO_REQ_SEP },
|
||||
{ OPT_PROFILER, "--profiler_", SO_REQ_SEP},
|
||||
|
||||
#ifndef TLS_DISABLED
|
||||
TLS_OPTION_FLAGS
|
||||
|
@ -622,6 +627,11 @@ static void printUsage(const char* name, bool devhelp) {
|
|||
" Machine class (valid options are storage, transaction,"
|
||||
" resolution, grv_proxy, commit_proxy, master, test, unset, stateless, log, router,"
|
||||
" and cluster_controller).");
|
||||
printOptionUsage("--profiler_",
|
||||
"Set an actor profiler option. Supported options are:\n"
|
||||
" collector -- None or FluentD (FluentD requires collector_endpoint to be set)\n"
|
||||
" collector_endpoint -- IP:PORT of the fluentd server\n"
|
||||
" collector_protocol -- UDP or TCP (default is UDP)");
|
||||
#ifndef TLS_DISABLED
|
||||
printf(TLS_HELP);
|
||||
#endif
|
||||
|
@ -990,6 +1000,8 @@ struct CLIOptions {
|
|||
Standalone<StringRef> machineId;
|
||||
UnitTestParameters testParams;
|
||||
|
||||
std::map<std::string, std::string> profilerConfig;
|
||||
|
||||
static CLIOptions parseArgs(int argc, char* argv[]) {
|
||||
CLIOptions opts;
|
||||
opts.parseArgsInternal(argc, argv);
|
||||
|
@ -1064,6 +1076,18 @@ private:
|
|||
manualKnobOverrides[syn] = args.OptionArg();
|
||||
break;
|
||||
}
|
||||
case OPT_PROFILER: {
|
||||
std::string syn = args.OptionSyntax();
|
||||
std::string_view key = syn;
|
||||
auto prefix = "--profiler_"sv;
|
||||
if (key.find(prefix) != 0) {
|
||||
fprintf(stderr, "ERROR: unable to parse profiler option '%s'\n", syn.c_str());
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
key.remove_prefix(prefix.size());
|
||||
profilerConfig.emplace(key, args.OptionArg());
|
||||
break;
|
||||
};
|
||||
case OPT_UNITTESTPARAM: {
|
||||
std::string syn = args.OptionSyntax();
|
||||
if (!StringRef(syn).startsWith(LiteralStringRef("--test_"))) {
|
||||
|
@ -1481,6 +1505,13 @@ private:
|
|||
}
|
||||
}
|
||||
|
||||
try {
|
||||
ProfilerConfig::instance().reset(profilerConfig);
|
||||
} catch (ConfigError& e) {
|
||||
printf("Error seting up profiler: %s", e.description.c_str());
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
|
||||
if (seedConnString.length() && seedConnFile.length()) {
|
||||
fprintf(
|
||||
stderr, "%s\n", "--seed_cluster_file and --seed_connection_string may not both be specified at once.");
|
||||
|
@ -1621,6 +1652,11 @@ private:
|
|||
} // namespace
|
||||
|
||||
int main(int argc, char* argv[]) {
|
||||
// TODO: Remove later, this is just to force the statics to be initialized
|
||||
// otherwise the unit test won't run
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet _;
|
||||
#endif
|
||||
try {
|
||||
platformInit();
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@
|
|||
#include "fdbclient/Notified.h"
|
||||
#include "fdbclient/StatusClient.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/TransactionLineage.h"
|
||||
#include "fdbclient/VersionedMap.h"
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
|
@ -757,6 +758,7 @@ public:
|
|||
Counter loops;
|
||||
Counter fetchWaitingMS, fetchWaitingCount, fetchExecutingMS, fetchExecutingCount;
|
||||
Counter readsRejected;
|
||||
Counter wrongShardServer;
|
||||
Counter fetchedVersions;
|
||||
Counter fetchesFromLogs;
|
||||
|
||||
|
@ -777,11 +779,11 @@ public:
|
|||
updateVersions("UpdateVersions", cc), loops("Loops", cc), fetchWaitingMS("FetchWaitingMS", cc),
|
||||
fetchWaitingCount("FetchWaitingCount", cc), fetchExecutingMS("FetchExecutingMS", cc),
|
||||
fetchExecutingCount("FetchExecutingCount", cc), readsRejected("ReadsRejected", cc),
|
||||
fetchedVersions("FetchedVersions", cc), fetchesFromLogs("FetchesFromLogs", cc),
|
||||
readLatencySample("ReadLatencyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
wrongShardServer("WrongShardServer", cc), fetchedVersions("FetchedVersions", cc),
|
||||
fetchesFromLogs("FetchesFromLogs", cc), readLatencySample("ReadLatencyMetrics",
|
||||
self->thisServerID,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
readLatencyBands("ReadLatencyBands", self->thisServerID, SERVER_KNOBS->STORAGE_LOGGING_DELAY) {
|
||||
specialCounter(cc, "LastTLogVersion", [self]() { return self->lastTLogVersion; });
|
||||
specialCounter(cc, "Version", [self]() { return self->version.get(); });
|
||||
|
@ -949,8 +951,11 @@ public:
|
|||
using isLoadBalancedReply = std::is_base_of<LoadBalancedReply, Reply>;
|
||||
|
||||
template <class Reply>
|
||||
static typename std::enable_if<isLoadBalancedReply<Reply>::value, void>::type
|
||||
typename std::enable_if<isLoadBalancedReply<Reply>::value, void>::type
|
||||
sendErrorWithPenalty(const ReplyPromise<Reply>& promise, const Error& err, double penalty) {
|
||||
if (err.code() == error_code_wrong_shard_server) {
|
||||
++counters.wrongShardServer;
|
||||
}
|
||||
Reply reply;
|
||||
reply.error = err;
|
||||
reply.penalty = penalty;
|
||||
|
@ -958,8 +963,11 @@ public:
|
|||
}
|
||||
|
||||
template <class Reply>
|
||||
static typename std::enable_if<!isLoadBalancedReply<Reply>::value, void>::type
|
||||
typename std::enable_if<!isLoadBalancedReply<Reply>::value, void>::type
|
||||
sendErrorWithPenalty(const ReplyPromise<Reply>& promise, const Error& err, double) {
|
||||
if (err.code() == error_code_wrong_shard_server) {
|
||||
++counters.wrongShardServer;
|
||||
}
|
||||
promise.sendError(err);
|
||||
}
|
||||
|
||||
|
@ -1217,6 +1225,8 @@ ACTOR Future<Void> getValueQ(StorageServer* data, GetValueRequest req) {
|
|||
state int64_t resultSize = 0;
|
||||
Span span("SS:getValue"_loc, { req.spanContext });
|
||||
span.addTag("key"_sr, req.key);
|
||||
// Temporarily disabled -- this path is hit a lot
|
||||
// getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
|
||||
try {
|
||||
++data->counters.getValueQueries;
|
||||
|
@ -1267,14 +1277,16 @@ ACTOR Future<Void> getValueQ(StorageServer* data, GetValueRequest req) {
|
|||
|
||||
DEBUG_MUTATION("ShardGetValue",
|
||||
version,
|
||||
MutationRef(MutationRef::DebugKey, req.key, v.present() ? v.get() : LiteralStringRef("<null>")));
|
||||
MutationRef(MutationRef::DebugKey, req.key, v.present() ? v.get() : LiteralStringRef("<null>")),
|
||||
data->thisServerID);
|
||||
DEBUG_MUTATION("ShardGetPath",
|
||||
version,
|
||||
MutationRef(MutationRef::DebugKey,
|
||||
req.key,
|
||||
path == 0 ? LiteralStringRef("0")
|
||||
: path == 1 ? LiteralStringRef("1")
|
||||
: LiteralStringRef("2")));
|
||||
: LiteralStringRef("2")),
|
||||
data->thisServerID);
|
||||
|
||||
/*
|
||||
StorageMetrics m;
|
||||
|
@ -1382,7 +1394,8 @@ ACTOR Future<Version> watchWaitForValueChange(StorageServer* data, SpanID parent
|
|||
latest,
|
||||
MutationRef(MutationRef::DebugKey,
|
||||
metadata->key,
|
||||
reply.value.present() ? StringRef(reply.value.get()) : LiteralStringRef("<null>")));
|
||||
reply.value.present() ? StringRef(reply.value.get()) : LiteralStringRef("<null>")),
|
||||
data->thisServerID);
|
||||
|
||||
if (metadata->debugID.present())
|
||||
g_traceBatch.addEvent(
|
||||
|
@ -1915,6 +1928,7 @@ ACTOR Future<Void> getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req)
|
|||
{
|
||||
state Span span("SS:getKeyValues"_loc, { req.spanContext });
|
||||
state int64_t resultSize = 0;
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
|
||||
++data->counters.getRangeQueries;
|
||||
++data->counters.allQueries;
|
||||
|
@ -1954,7 +1968,7 @@ ACTOR Future<Void> getKeyValuesQ(StorageServer* data, GetKeyValuesRequest req)
|
|||
throw wrong_shard_server();
|
||||
}
|
||||
|
||||
state int offset1;
|
||||
state int offset1 = 0;
|
||||
state int offset2;
|
||||
state Future<Key> fBegin = req.begin.isFirstGreaterOrEqual()
|
||||
? Future<Key>(req.begin.getKey())
|
||||
|
@ -2110,17 +2124,17 @@ ACTOR Future<Void> getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe
|
|||
"TransactionDebug", req.debugID.get().first(), "storageserver.getKeyValuesStream.AfterVersion");
|
||||
//.detail("ShardBegin", shard.begin).detail("ShardEnd", shard.end);
|
||||
//} catch (Error& e) { TraceEvent("WrongShardServer", data->thisServerID).detail("Begin",
|
||||
//req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("Shard",
|
||||
// req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("Shard",
|
||||
//"None").detail("In", "getKeyValues>getShardKeyRange"); throw e; }
|
||||
|
||||
if (!selectorInRange(req.end, shard) && !(req.end.isFirstGreaterOrEqual() && req.end.getKey() == shard.end)) {
|
||||
// TraceEvent("WrongShardServer1", data->thisServerID).detail("Begin",
|
||||
//req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("ShardBegin",
|
||||
//shard.begin).detail("ShardEnd", shard.end).detail("In", "getKeyValues>checkShardExtents");
|
||||
// req.begin.toString()).detail("End", req.end.toString()).detail("Version", version).detail("ShardBegin",
|
||||
// shard.begin).detail("ShardEnd", shard.end).detail("In", "getKeyValues>checkShardExtents");
|
||||
throw wrong_shard_server();
|
||||
}
|
||||
|
||||
state int offset1;
|
||||
state int offset1 = 0;
|
||||
state int offset2;
|
||||
state Future<Key> fBegin = req.begin.isFirstGreaterOrEqual()
|
||||
? Future<Key>(req.begin.getKey())
|
||||
|
@ -2193,10 +2207,10 @@ ACTOR Future<Void> getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe
|
|||
}
|
||||
|
||||
/*for( int i = 0; i < r.data.size(); i++ ) {
|
||||
StorageMetrics m;
|
||||
m.bytesPerKSecond = r.data[i].expectedSize();
|
||||
m.iosPerKSecond = 1; //FIXME: this should be 1/r.data.size(), but we cannot do that because it is an int
|
||||
data->metrics.notify(r.data[i].key, m);
|
||||
StorageMetrics m;
|
||||
m.bytesPerKSecond = r.data[i].expectedSize();
|
||||
m.iosPerKSecond = 1; //FIXME: this should be 1/r.data.size(), but we cannot do that because it is an
|
||||
int data->metrics.notify(r.data[i].key, m);
|
||||
}*/
|
||||
|
||||
// For performance concerns, the cost of a range read is billed to the start key and end key of the
|
||||
|
@ -2256,6 +2270,7 @@ ACTOR Future<Void> getKeyValuesStreamQ(StorageServer* data, GetKeyValuesStreamRe
|
|||
ACTOR Future<Void> getKeyQ(StorageServer* data, GetKeyRequest req) {
|
||||
state Span span("SS:getKey"_loc, { req.spanContext });
|
||||
state int64_t resultSize = 0;
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
|
||||
++data->counters.getKeyQueries;
|
||||
++data->counters.allQueries;
|
||||
|
@ -2835,7 +2850,7 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
wait(data->coreStarted.getFuture() && delay(0));
|
||||
|
||||
try {
|
||||
DEBUG_KEY_RANGE("fetchKeysBegin", data->version.get(), shard->keys);
|
||||
DEBUG_KEY_RANGE("fetchKeysBegin", data->version.get(), shard->keys, data->thisServerID);
|
||||
|
||||
TraceEvent(SevDebug, interval.begin(), data->thisServerID)
|
||||
.detail("KeyBegin", shard->keys.begin)
|
||||
|
@ -2883,7 +2898,6 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
// Get the history
|
||||
state int debug_getRangeRetries = 0;
|
||||
state int debug_nextRetryToLog = 1;
|
||||
state bool isTooOld = false;
|
||||
|
||||
// FIXME: The client cache does not notice when servers are added to a team. To read from a local storage server
|
||||
// we must refresh the cache manually.
|
||||
|
@ -2927,9 +2941,13 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
.detail("Last", this_block.size() ? this_block.end()[-1].key : std::string())
|
||||
.detail("Version", fetchVersion)
|
||||
.detail("More", this_block.more);
|
||||
DEBUG_KEY_RANGE("fetchRange", fetchVersion, keys);
|
||||
for (auto k = this_block.begin(); k != this_block.end(); ++k)
|
||||
DEBUG_MUTATION("fetch", fetchVersion, MutationRef(MutationRef::SetValue, k->key, k->value));
|
||||
|
||||
DEBUG_KEY_RANGE("fetchRange", fetchVersion, keys, data->thisServerID);
|
||||
if(MUTATION_TRACKING_ENABLED) {
|
||||
for (auto k = this_block.begin(); k != this_block.end(); ++k) {
|
||||
DEBUG_MUTATION("fetch", fetchVersion, MutationRef(MutationRef::SetValue, k->key, k->value), data->thisServerID);
|
||||
}
|
||||
}
|
||||
|
||||
metricReporter.addFetchedBytes(expectedBlockSize, this_block.size());
|
||||
|
||||
|
@ -3092,8 +3110,11 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
for (auto b = batch->changes.begin() + startSize; b != batch->changes.end(); ++b) {
|
||||
ASSERT(b->version >= checkv);
|
||||
checkv = b->version;
|
||||
for (auto& m : b->mutations)
|
||||
DEBUG_MUTATION("fetchKeysFinalCommitInject", batch->changes[0].version, m);
|
||||
if(MUTATION_TRACKING_ENABLED) {
|
||||
for (auto& m : b->mutations) {
|
||||
DEBUG_MUTATION("fetchKeysFinalCommitInject", batch->changes[0].version, m, data->thisServerID);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
shard->updates.clear();
|
||||
|
@ -3195,9 +3216,7 @@ void ShardInfo::addMutation(Version version, MutationRef const& mutation) {
|
|||
else if (readWrite)
|
||||
readWrite->addMutation(version, mutation, this->keys, readWrite->updateEagerReads);
|
||||
else if (mutation.type != MutationRef::ClearRange) {
|
||||
TraceEvent(SevError, "DeliveredToNotAssigned")
|
||||
.detail("Version", version)
|
||||
.detail("Mutation", mutation.toString());
|
||||
TraceEvent(SevError, "DeliveredToNotAssigned").detail("Version", version).detail("Mutation", mutation);
|
||||
ASSERT(false); // Mutation delivered to notAssigned shard!
|
||||
}
|
||||
}
|
||||
|
@ -3221,7 +3240,7 @@ void changeServerKeys(StorageServer* data,
|
|||
validate(data);
|
||||
|
||||
// TODO(alexmiller): Figure out how to selectively enable spammy data distribution events.
|
||||
DEBUG_KEY_RANGE(nowAssigned ? "KeysAssigned" : "KeysUnassigned", version, keys);
|
||||
DEBUG_KEY_RANGE(nowAssigned ? "KeysAssigned" : "KeysUnassigned", version, keys, data->thisServerID);
|
||||
|
||||
bool isDifferent = false;
|
||||
auto existingShards = data->shards.intersectingRanges(keys);
|
||||
|
@ -3329,7 +3348,7 @@ void changeServerKeys(StorageServer* data,
|
|||
|
||||
void rollback(StorageServer* data, Version rollbackVersion, Version nextVersion) {
|
||||
TEST(true); // call to shard rollback
|
||||
DEBUG_KEY_RANGE("Rollback", rollbackVersion, allKeys);
|
||||
DEBUG_KEY_RANGE("Rollback", rollbackVersion, allKeys, data->thisServerID);
|
||||
|
||||
// We used to do a complicated dance to roll back in MVCC history. It's much simpler, and more testable,
|
||||
// to simply restart the storage server actor and restore from the persistent disk state, and then roll
|
||||
|
@ -3354,8 +3373,7 @@ void StorageServer::addMutation(Version version,
|
|||
return;
|
||||
}
|
||||
expanded = addMutationToMutationLog(mLog, expanded);
|
||||
DEBUG_MUTATION("applyMutation", version, expanded)
|
||||
.detail("UID", thisServerID)
|
||||
DEBUG_MUTATION("applyMutation", version, expanded, thisServerID)
|
||||
.detail("ShardBegin", shard.begin)
|
||||
.detail("ShardEnd", shard.end);
|
||||
applyMutation(this, expanded, mLog.arena(), mutableData());
|
||||
|
@ -3426,7 +3444,7 @@ public:
|
|||
} else {
|
||||
// FIXME: enable when DEBUG_MUTATION is active
|
||||
// for(auto m = changes[c].mutations.begin(); m; ++m) {
|
||||
// DEBUG_MUTATION("SSUpdateMutation", changes[c].version, *m);
|
||||
// DEBUG_MUTATION("SSUpdateMutation", changes[c].version, *m, data->thisServerID);
|
||||
//}
|
||||
|
||||
splitMutation(data, data->shards, m, ver);
|
||||
|
@ -3450,7 +3468,7 @@ private:
|
|||
bool processedCacheStartKey;
|
||||
|
||||
void applyPrivateData(StorageServer* data, MutationRef const& m) {
|
||||
TraceEvent(SevDebug, "SSPrivateMutation", data->thisServerID).detail("Mutation", m.toString());
|
||||
TraceEvent(SevDebug, "SSPrivateMutation", data->thisServerID).detail("Mutation", m);
|
||||
|
||||
if (processedStartKey) {
|
||||
// Because of the implementation of the krm* functions, we expect changes in pairs, [begin,end)
|
||||
|
@ -3556,7 +3574,7 @@ private:
|
|||
}
|
||||
|
||||
void applyPrivateCacheData(StorageServer* data, MutationRef const& m) {
|
||||
//TraceEvent(SevDebug, "SSPrivateCacheMutation", data->thisServerID).detail("Mutation", m.toString());
|
||||
//TraceEvent(SevDebug, "SSPrivateCacheMutation", data->thisServerID).detail("Mutation", m);
|
||||
|
||||
if (processedCacheStartKey) {
|
||||
// Because of the implementation of the krm* functions, we expect changes in pairs, [begin,end)
|
||||
|
@ -3690,7 +3708,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
|
|||
if (LogProtocolMessage::isNextIn(cloneReader)) {
|
||||
LogProtocolMessage lpm;
|
||||
cloneReader >> lpm;
|
||||
//TraceEvent(SevDebug, "SSReadingLPM", data->thisServerID).detail("Mutation", lpm.toString());
|
||||
//TraceEvent(SevDebug, "SSReadingLPM", data->thisServerID).detail("Mutation", lpm);
|
||||
dbgLastMessageWasProtocol = true;
|
||||
cloneCursor1->setProtocolVersion(cloneReader.protocolVersion());
|
||||
} else if (cloneReader.protocolVersion().hasSpanContext() &&
|
||||
|
@ -3700,7 +3718,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
|
|||
} else {
|
||||
MutationRef msg;
|
||||
cloneReader >> msg;
|
||||
// TraceEvent(SevDebug, "SSReadingLog", data->thisServerID).detail("Mutation", msg.toString());
|
||||
// TraceEvent(SevDebug, "SSReadingLog", data->thisServerID).detail("Mutation", msg);
|
||||
|
||||
if (firstMutation && msg.param1.startsWith(systemKeys.end))
|
||||
hasPrivateData = true;
|
||||
|
@ -3825,7 +3843,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
|
|||
(msg.param1.size() < 2 || msg.param1[0] != 0xff || msg.param1[1] != 0xff) &&
|
||||
deterministicRandom()->random01() < 0.05) {
|
||||
TraceEvent(SevWarnAlways, "TSSInjectDropMutation", data->thisServerID)
|
||||
.detail("Mutation", msg.toString())
|
||||
.detail("Mutation", msg)
|
||||
.detail("Version", cloneCursor2->version().toString());
|
||||
} else if (data->isTSSInQuarantine() &&
|
||||
(msg.param1.size() < 2 || msg.param1[0] != 0xff || msg.param1[1] != 0xff)) {
|
||||
|
@ -3833,11 +3851,13 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
|
|||
.suppressFor(10.0)
|
||||
.detail("Version", cloneCursor2->version().toString());
|
||||
} else if (ver != invalidVersion) { // This change belongs to a version < minVersion
|
||||
DEBUG_MUTATION("SSPeek", ver, msg).detail("ServerID", data->thisServerID);
|
||||
DEBUG_MUTATION("SSPeek", ver, msg, data->thisServerID);
|
||||
if (ver == 1) {
|
||||
TraceEvent("SSPeekMutation", data->thisServerID).log();
|
||||
//TraceEvent("SSPeekMutation", data->thisServerID).log();
|
||||
// The following trace event may produce a value with special characters
|
||||
//TraceEvent("SSPeekMutation", data->thisServerID).detail("Mutation", msg.toString()).detail("Version", cloneCursor2->version().toString());
|
||||
TraceEvent("SSPeekMutation", data->thisServerID)
|
||||
.detail("Mutation", msg)
|
||||
.detail("Version", cloneCursor2->version().toString());
|
||||
}
|
||||
|
||||
updater.applyMutation(data, msg, ver);
|
||||
|
@ -3868,7 +3888,7 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
|
|||
}
|
||||
} else
|
||||
TraceEvent(SevError, "DiscardingPeekedData", data->thisServerID)
|
||||
.detail("Mutation", msg.toString())
|
||||
.detail("Mutation", msg)
|
||||
.detail("Version", cloneCursor2->version().toString());
|
||||
}
|
||||
}
|
||||
|
@ -4145,7 +4165,6 @@ void StorageServerDisk::writeKeyValue(KeyValueRef kv) {
|
|||
}
|
||||
|
||||
void StorageServerDisk::writeMutation(MutationRef mutation) {
|
||||
// FIXME: DEBUG_MUTATION(debugContext, debugVersion, *m);
|
||||
if (mutation.type == MutationRef::SetValue) {
|
||||
storage->set(KeyValueRef(mutation.param1, mutation.param2));
|
||||
} else if (mutation.type == MutationRef::ClearRange) {
|
||||
|
@ -4158,7 +4177,7 @@ void StorageServerDisk::writeMutations(const VectorRef<MutationRef>& mutations,
|
|||
Version debugVersion,
|
||||
const char* debugContext) {
|
||||
for (const auto& m : mutations) {
|
||||
DEBUG_MUTATION(debugContext, debugVersion, m).detail("UID", data->thisServerID);
|
||||
DEBUG_MUTATION(debugContext, debugVersion, m, data->thisServerID);
|
||||
if (m.type == MutationRef::SetValue) {
|
||||
storage->set(KeyValueRef(m.param1, m.param2));
|
||||
} else if (m.type == MutationRef::ClearRange) {
|
||||
|
@ -4770,6 +4789,7 @@ ACTOR Future<Void> checkBehind(StorageServer* self) {
|
|||
}
|
||||
|
||||
ACTOR Future<Void> serveGetValueRequests(StorageServer* self, FutureStream<GetValueRequest> getValue) {
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetValue;
|
||||
loop {
|
||||
GetValueRequest req = waitNext(getValue);
|
||||
// Warning: This code is executed at extremely high priority (TaskPriority::LoadBalancedEndpoint), so downgrade
|
||||
|
@ -4787,6 +4807,7 @@ ACTOR Future<Void> serveGetValueRequests(StorageServer* self, FutureStream<GetVa
|
|||
}
|
||||
|
||||
ACTOR Future<Void> serveGetKeyValuesRequests(StorageServer* self, FutureStream<GetKeyValuesRequest> getKeyValues) {
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetKeyValues;
|
||||
loop {
|
||||
GetKeyValuesRequest req = waitNext(getKeyValues);
|
||||
|
||||
|
@ -4808,6 +4829,7 @@ ACTOR Future<Void> serveGetKeyValuesStreamRequests(StorageServer* self,
|
|||
}
|
||||
|
||||
ACTOR Future<Void> serveGetKeyRequests(StorageServer* self, FutureStream<GetKeyRequest> getKey) {
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetKey;
|
||||
loop {
|
||||
GetKeyRequest req = waitNext(getKey);
|
||||
// Warning: This code is executed at extremely high priority (TaskPriority::LoadBalancedEndpoint), so downgrade
|
||||
|
@ -4820,6 +4842,7 @@ ACTOR Future<Void> watchValueWaitForVersion(StorageServer* self,
|
|||
WatchValueRequest req,
|
||||
PromiseStream<WatchValueRequest> stream) {
|
||||
state Span span("SS:watchValueWaitForVersion"_loc, { req.spanContext });
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
try {
|
||||
wait(success(waitForVersionNoTooOld(self, req.version)));
|
||||
stream.send(req);
|
||||
|
@ -4833,9 +4856,11 @@ ACTOR Future<Void> watchValueWaitForVersion(StorageServer* self,
|
|||
|
||||
ACTOR Future<Void> serveWatchValueRequestsImpl(StorageServer* self, FutureStream<WatchValueRequest> stream) {
|
||||
loop {
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = 0;
|
||||
state WatchValueRequest req = waitNext(stream);
|
||||
state Reference<ServerWatchMetadata> metadata = self->getWatchMetadata(req.key.contents());
|
||||
state Span span("SS:serveWatchValueRequestsImpl"_loc, { req.spanContext });
|
||||
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
||||
|
||||
if (!metadata.isValid()) { // case 1: no watch set for the current key
|
||||
metadata = makeReference<ServerWatchMetadata>(req.key, req.value, req.version, req.tags, req.debugID);
|
||||
|
@ -4909,6 +4934,7 @@ ACTOR Future<Void> serveWatchValueRequestsImpl(StorageServer* self, FutureStream
|
|||
|
||||
ACTOR Future<Void> serveWatchValueRequests(StorageServer* self, FutureStream<WatchValueRequest> watchValue) {
|
||||
state PromiseStream<WatchValueRequest> stream;
|
||||
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::WatchValue;
|
||||
self->actors.add(serveWatchValueRequestsImpl(self, stream.getFuture()));
|
||||
|
||||
loop {
|
||||
|
@ -4943,8 +4969,8 @@ ACTOR Future<Void> reportStorageServerState(StorageServer* self) {
|
|||
TraceEvent(level, "FetchKeyCurrentStatus")
|
||||
.detail("Timestamp", now())
|
||||
.detail("LongestRunningTime", longestRunningFetchKeys.first)
|
||||
.detail("StartKey", longestRunningFetchKeys.second.begin.printable())
|
||||
.detail("EndKey", longestRunningFetchKeys.second.end.printable())
|
||||
.detail("StartKey", longestRunningFetchKeys.second.begin)
|
||||
.detail("EndKey", longestRunningFetchKeys.second.end)
|
||||
.detail("NumRunning", numRunningFetchKeys);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@
|
|||
|
||||
#include "fdbrpc/Locality.h"
|
||||
#include "fdbclient/GlobalConfig.actor.h"
|
||||
#include "fdbclient/ProcessInterface.h"
|
||||
#include "fdbclient/StorageServerInterface.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "flow/ActorCollection.h"
|
||||
|
@ -33,6 +34,7 @@
|
|||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/MetricLogger.actor.h"
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/RoleLineage.actor.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/WaitFailure.h"
|
||||
|
@ -49,6 +51,7 @@
|
|||
#include "flow/Profiler.h"
|
||||
#include "flow/ThreadHelper.actor.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/network.h"
|
||||
|
||||
#ifdef __linux__
|
||||
|
@ -78,6 +81,10 @@ extern IKeyValueStore* keyValueStoreCompressTestData(IKeyValueStore* store);
|
|||
#define KV_STORE(filename, uid) keyValueStoreMemory(filename, uid)
|
||||
#endif
|
||||
|
||||
namespace {
|
||||
RoleLineageCollector roleLineageCollector;
|
||||
}
|
||||
|
||||
ACTOR Future<std::vector<Endpoint>> tryDBInfoBroadcast(RequestStream<UpdateServerDBInfoRequest> stream,
|
||||
UpdateServerDBInfoRequest req) {
|
||||
ErrorOr<std::vector<Endpoint>> rep =
|
||||
|
@ -148,6 +155,8 @@ Database openDBOnServer(Reference<AsyncVar<ServerDBInfo> const> const& db,
|
|||
taskID,
|
||||
lockAware);
|
||||
GlobalConfig::create(cx, db, std::addressof(db->get().client));
|
||||
GlobalConfig::globalConfig().trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
||||
GlobalConfig::globalConfig().trigger(samplingWindow, samplingProfilerUpdateWindow);
|
||||
return cx;
|
||||
}
|
||||
|
||||
|
@ -510,7 +519,8 @@ ACTOR Future<Void> registrationClient(Reference<AsyncVar<Optional<ClusterControl
|
|||
Reference<AsyncVar<Optional<RatekeeperInterface>> const> rkInterf,
|
||||
Reference<AsyncVar<bool> const> degraded,
|
||||
Reference<ClusterConnectionFile> connFile,
|
||||
Reference<AsyncVar<std::set<std::string>> const> issues) {
|
||||
Reference<AsyncVar<std::set<std::string>> const> issues,
|
||||
LocalConfiguration* localConfig) {
|
||||
// Keeps the cluster controller (as it may be re-elected) informed that this worker exists
|
||||
// The cluster controller uses waitFailureClient to find out if we die, and returns from registrationReply
|
||||
// (requiring us to re-register) The registration request piggybacks optional distributor interface if it exists.
|
||||
|
@ -529,7 +539,9 @@ ACTOR Future<Void> registrationClient(Reference<AsyncVar<Optional<ClusterControl
|
|||
requestGeneration++,
|
||||
ddInterf->get(),
|
||||
rkInterf->get(),
|
||||
degraded->get());
|
||||
degraded->get(),
|
||||
localConfig->lastSeenVersion(),
|
||||
localConfig->configClassSet());
|
||||
for (auto const& i : issues->get()) {
|
||||
request.issues.push_back_deep(request.issues.arena(), i);
|
||||
}
|
||||
|
@ -1212,7 +1224,9 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
int64_t memoryProfileThreshold,
|
||||
std::string _coordFolder,
|
||||
std::string whitelistBinPaths,
|
||||
Reference<AsyncVar<ServerDBInfo>> dbInfo) {
|
||||
Reference<AsyncVar<ServerDBInfo>> dbInfo,
|
||||
ConfigDBType configDBType,
|
||||
LocalConfiguration* localConfig) {
|
||||
state PromiseStream<ErrorInfo> errors;
|
||||
state Reference<AsyncVar<Optional<DataDistributorInterface>>> ddInterf(
|
||||
new AsyncVar<Optional<DataDistributorInterface>>());
|
||||
|
@ -1259,6 +1273,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
metricsLogger =
|
||||
runMetrics(openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, lockAware), KeyRef(metricsPrefix));
|
||||
}
|
||||
|
||||
GlobalConfig::globalConfig().trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
||||
}
|
||||
|
||||
errorForwarders.add(resetAfter(degraded,
|
||||
|
@ -1306,6 +1322,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
DiskStore s = stores[f];
|
||||
// FIXME: Error handling
|
||||
if (s.storedComponent == DiskStore::Storage) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage;
|
||||
IKeyValueStore* kv =
|
||||
openKVStore(s.storeType, s.filename, s.storeID, memoryLimit, false, validateDataFiles);
|
||||
Future<Void> kvClosed = kv->onClosed();
|
||||
|
@ -1366,6 +1384,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
kv);
|
||||
errorForwarders.add(forwardError(errors, ssRole, recruited.id(), f));
|
||||
} else if (s.storedComponent == DiskStore::TLogData) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::TLog;
|
||||
std::string logQueueBasename;
|
||||
const std::string filename = basename(s.filename);
|
||||
if (StringRef(filename).startsWith(fileLogDataPrefix)) {
|
||||
|
@ -1474,7 +1494,12 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
rkInterf,
|
||||
degraded,
|
||||
connFile,
|
||||
issues));
|
||||
issues,
|
||||
localConfig));
|
||||
|
||||
if (configDBType != ConfigDBType::DISABLED) {
|
||||
errorForwarders.add(localConfig->consume(interf.configBroadcastInterface));
|
||||
}
|
||||
|
||||
if (SERVER_KNOBS->ENABLE_WORKER_HEALTH_MONITOR) {
|
||||
errorForwarders.add(healthMonitor(ccInterface, interf, locality, dbInfo));
|
||||
|
@ -1565,6 +1590,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
}
|
||||
}
|
||||
when(RecruitMasterRequest req = waitNext(interf.master.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Master;
|
||||
MasterInterface recruited;
|
||||
recruited.locality = locality;
|
||||
recruited.initEndpoints();
|
||||
|
@ -1587,6 +1614,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
req.reply.send(recruited);
|
||||
}
|
||||
when(InitializeDataDistributorRequest req = waitNext(interf.dataDistributor.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::DataDistributor;
|
||||
DataDistributorInterface recruited(locality);
|
||||
recruited.initEndpoints();
|
||||
|
||||
|
@ -1609,6 +1638,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
req.reply.send(recruited);
|
||||
}
|
||||
when(InitializeRatekeeperRequest req = waitNext(interf.ratekeeper.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Ratekeeper;
|
||||
RatekeeperInterface recruited(locality, req.reqId);
|
||||
recruited.initEndpoints();
|
||||
|
||||
|
@ -1635,6 +1666,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
}
|
||||
when(InitializeBackupRequest req = waitNext(interf.backup.getFuture())) {
|
||||
if (!backupWorkerCache.exists(req.reqId)) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Backup;
|
||||
BackupInterface recruited(locality);
|
||||
recruited.initEndpoints();
|
||||
|
||||
|
@ -1664,6 +1697,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
.detail("MinRecruitable", TLogVersion::MIN_RECRUITABLE);
|
||||
req.reply.sendError(internal_error());
|
||||
}
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::TLog;
|
||||
TLogOptions tLogOptions(req.logVersion, req.spillType);
|
||||
TLogFn tLogFn = tLogFnForOptions(tLogOptions);
|
||||
auto& logData = sharedLogs[SharedLogsKey(tLogOptions, req.storeType)];
|
||||
|
@ -1727,9 +1762,9 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
runningStorages.end(),
|
||||
[&req](const auto& p) { return p.second != req.storeType; }) ||
|
||||
req.seedTag != invalidTag)) {
|
||||
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage;
|
||||
bool isTss = req.tssPairIDAndVersion.present();
|
||||
|
||||
StorageServerInterface recruited(req.interfaceId);
|
||||
recruited.locality = locality;
|
||||
recruited.tssPairID = isTss ? req.tssPairIDAndVersion.get().first : Optional<UID>();
|
||||
|
@ -1800,6 +1835,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
}
|
||||
}
|
||||
when(InitializeCommitProxyRequest req = waitNext(interf.commitProxy.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::CommitProxy;
|
||||
CommitProxyInterface recruited;
|
||||
recruited.processId = locality.processId();
|
||||
recruited.provisional = false;
|
||||
|
@ -1825,6 +1862,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
req.reply.send(recruited);
|
||||
}
|
||||
when(InitializeGrvProxyRequest req = waitNext(interf.grvProxy.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::GrvProxy;
|
||||
GrvProxyInterface recruited;
|
||||
recruited.processId = locality.processId();
|
||||
recruited.provisional = false;
|
||||
|
@ -1845,6 +1884,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
req.reply.send(recruited);
|
||||
}
|
||||
when(InitializeResolverRequest req = waitNext(interf.resolver.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Resolver;
|
||||
ResolverInterface recruited;
|
||||
recruited.locality = locality;
|
||||
recruited.initEndpoints();
|
||||
|
@ -1862,6 +1903,8 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
|
|||
req.reply.send(recruited);
|
||||
}
|
||||
when(InitializeLogRouterRequest req = waitNext(interf.logRouter.getFuture())) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::LogRouter;
|
||||
TLogInterface recruited(locality);
|
||||
recruited.initEndpoints();
|
||||
|
||||
|
@ -2271,6 +2314,8 @@ ACTOR Future<Void> monitorLeaderRemotelyWithDelayedCandidacy(
|
|||
}
|
||||
}
|
||||
|
||||
extern void setupStackSignal();
|
||||
|
||||
ACTOR Future<Void> serveProtocolInfo() {
|
||||
state RequestStream<ProtocolInfoRequest> protocolInfo(
|
||||
PeerCompatibilityPolicy{ RequirePeer::AtLeast, ProtocolVersion::withStableInterfaces() });
|
||||
|
@ -2281,6 +2326,37 @@ ACTOR Future<Void> serveProtocolInfo() {
|
|||
}
|
||||
}
|
||||
|
||||
// Handles requests from ProcessInterface, an interface meant for direct
|
||||
// communication between the client and FDB processes.
|
||||
ACTOR Future<Void> serveProcess() {
|
||||
state ProcessInterface process;
|
||||
process.getInterface.makeWellKnownEndpoint(WLTOKEN_PROCESS, TaskPriority::DefaultEndpoint);
|
||||
loop {
|
||||
choose {
|
||||
when(GetProcessInterfaceRequest req = waitNext(process.getInterface.getFuture())) {
|
||||
req.reply.send(process);
|
||||
}
|
||||
when(ActorLineageRequest req = waitNext(process.actorLineage.getFuture())) {
|
||||
state SampleCollection sampleCollector;
|
||||
auto samples = sampleCollector->get(req.timeStart, req.timeEnd);
|
||||
|
||||
std::vector<SerializedSample> serializedSamples;
|
||||
for (const auto& samplePtr : samples) {
|
||||
auto serialized = SerializedSample{ .time = samplePtr->time };
|
||||
for (const auto& [waitState, pair] : samplePtr->data) {
|
||||
if (waitState >= req.waitStateStart && waitState <= req.waitStateEnd) {
|
||||
serialized.data[waitState] = std::string(pair.first, pair.second);
|
||||
}
|
||||
}
|
||||
serializedSamples.push_back(std::move(serialized));
|
||||
}
|
||||
ActorLineageReply reply{ serializedSamples };
|
||||
req.reply.send(reply);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
|
||||
LocalityData localities,
|
||||
ProcessClass processClass,
|
||||
|
@ -2297,6 +2373,8 @@ ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
|
|||
state vector<Future<Void>> actors;
|
||||
state Promise<Void> recoveredDiskFiles;
|
||||
state LocalConfiguration localConfig(dataFolder, configPath, manualKnobOverrides);
|
||||
// setupStackSignal();
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::Worker;
|
||||
|
||||
// FIXME: Initializing here causes simulation issues, these must be fixed
|
||||
/*
|
||||
|
@ -2306,6 +2384,7 @@ ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
|
|||
*/
|
||||
|
||||
actors.push_back(serveProtocolInfo());
|
||||
actors.push_back(serveProcess());
|
||||
|
||||
try {
|
||||
ServerCoordinators coordinators(connFile);
|
||||
|
@ -2339,14 +2418,6 @@ ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
|
|||
makeReference<AsyncVar<ClusterControllerPriorityInfo>>(getCCPriorityInfo(fitnessFilePath, processClass));
|
||||
auto dbInfo = makeReference<AsyncVar<ServerDBInfo>>();
|
||||
|
||||
// FIXME: Reenable this once simulation issues are resolved
|
||||
/*
|
||||
if (ConfigDBType != ConfigDBType::DISABLED) {
|
||||
actors.push_back(reportErrors(localConfig.consume(IAsyncListener<ConfigBroadcastFollowerInterface>::create(
|
||||
dbInfo, [](auto const& info) { return info.configBroadcaster; })),
|
||||
"LocalConfiguration"));
|
||||
}
|
||||
*/
|
||||
actors.push_back(reportErrors(monitorAndWriteCCPriorityInfo(fitnessFilePath, asyncPriorityInfo),
|
||||
"MonitorAndWriteCCPriorityInfo"));
|
||||
if (processClass.machineClassFitness(ProcessClass::ClusterController) == ProcessClass::NeverAssign) {
|
||||
|
@ -2377,7 +2448,9 @@ ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
|
|||
memoryProfileThreshold,
|
||||
coordFolder,
|
||||
whitelistBinPaths,
|
||||
dbInfo),
|
||||
dbInfo,
|
||||
configDBType,
|
||||
&localConfig),
|
||||
"WorkerServer",
|
||||
UID(),
|
||||
&normalWorkerErrors()));
|
||||
|
|
|
@ -663,16 +663,16 @@ struct ConsistencyCheckWorkload : TestWorkload {
|
|||
.detail(format("StorageServer%d", j).c_str(), iter_ss[j].toString())
|
||||
.detail(format("StorageServer%d", firstValidServer).c_str(),
|
||||
iter_ss[firstValidServer].toString())
|
||||
.detail("ShardBegin", printable(req.begin.getKey()))
|
||||
.detail("ShardEnd", printable(req.end.getKey()))
|
||||
.detail("ShardBegin", req.begin.getKey())
|
||||
.detail("ShardEnd", req.end.getKey())
|
||||
.detail("VersionNumber", req.version)
|
||||
.detail(format("Server%dUniques", j).c_str(), currentUniques)
|
||||
.detail(format("Server%dUniqueKey", j).c_str(), printable(currentUniqueKey))
|
||||
.detail(format("Server%dUniqueKey", j).c_str(), currentUniqueKey)
|
||||
.detail(format("Server%dUniques", firstValidServer).c_str(), referenceUniques)
|
||||
.detail(format("Server%dUniqueKey", firstValidServer).c_str(),
|
||||
printable(referenceUniqueKey))
|
||||
referenceUniqueKey)
|
||||
.detail("ValueMismatches", valueMismatches)
|
||||
.detail("ValueMismatchKey", printable(valueMismatchKey))
|
||||
.detail("ValueMismatchKey", valueMismatchKey)
|
||||
.detail("MatchingKVPairs", matchingKVPairs);
|
||||
|
||||
self->testFailure("Data inconsistent", true);
|
||||
|
@ -718,7 +718,7 @@ struct ConsistencyCheckWorkload : TestWorkload {
|
|||
if (bytesReadInRange > 0) {
|
||||
TraceEvent("CacheConsistencyCheck_ReadRange")
|
||||
.suppressFor(1.0)
|
||||
.detail("Range", printable(iter->range()))
|
||||
.detail("Range", iter->range())
|
||||
.detail("BytesRead", bytesReadInRange);
|
||||
}
|
||||
}
|
||||
|
@ -1083,8 +1083,8 @@ struct ConsistencyCheckWorkload : TestWorkload {
|
|||
TraceEvent("ConsistencyCheck_InconsistentStorageMetrics")
|
||||
.detail("ByteEstimate1", estimatedBytes[firstValidStorageServer])
|
||||
.detail("ByteEstimate2", numBytes)
|
||||
.detail("Begin", printable(shard.begin))
|
||||
.detail("End", printable(shard.end))
|
||||
.detail("Begin", shard.begin)
|
||||
.detail("End", shard.end)
|
||||
.detail("StorageServer1", storageServers[firstValidStorageServer].id())
|
||||
.detail("StorageServer2", storageServers[i].id())
|
||||
.detail("IsTSS",
|
||||
|
@ -1451,17 +1451,17 @@ struct ConsistencyCheckWorkload : TestWorkload {
|
|||
.detail(format("StorageServer%d", j).c_str(), storageServers[j].toString())
|
||||
.detail(format("StorageServer%d", firstValidServer).c_str(),
|
||||
storageServers[firstValidServer].toString())
|
||||
.detail("ShardBegin", printable(req.begin.getKey()))
|
||||
.detail("ShardEnd", printable(req.end.getKey()))
|
||||
.detail("ShardBegin", req.begin.getKey())
|
||||
.detail("ShardEnd", req.end.getKey())
|
||||
.detail("VersionNumber", req.version)
|
||||
.detail(format("Server%dUniques", j).c_str(), currentUniques)
|
||||
.detail(format("Server%dUniqueKey", j).c_str(), printable(currentUniqueKey))
|
||||
.detail(format("Server%dUniqueKey", j).c_str(), currentUniqueKey)
|
||||
.detail(format("Server%dUniques", firstValidServer).c_str(),
|
||||
referenceUniques)
|
||||
.detail(format("Server%dUniqueKey", firstValidServer).c_str(),
|
||||
printable(referenceUniqueKey))
|
||||
referenceUniqueKey)
|
||||
.detail("ValueMismatches", valueMismatches)
|
||||
.detail("ValueMismatchKey", printable(valueMismatchKey))
|
||||
.detail("ValueMismatchKey", valueMismatchKey)
|
||||
.detail("MatchingKVPairs", matchingKVPairs)
|
||||
.detail("IsTSS",
|
||||
storageServerInterfaces[j].isTss() ||
|
||||
|
@ -1673,7 +1673,7 @@ struct ConsistencyCheckWorkload : TestWorkload {
|
|||
if (bytesReadInRange > 0) {
|
||||
TraceEvent("ConsistencyCheck_ReadRange")
|
||||
.suppressFor(1.0)
|
||||
.detail("Range", printable(range))
|
||||
.detail("Range", range)
|
||||
.detail("BytesRead", bytesReadInRange);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -258,13 +258,13 @@ struct SerializabilityWorkload : TestWorkload {
|
|||
} else if (ops[opNum].mutationOp.present()) {
|
||||
auto& op = ops[opNum].mutationOp.get();
|
||||
if (op.type == MutationRef::SetValue) {
|
||||
//TraceEvent("SRL_Set").detail("Mutation", op.toString());
|
||||
//TraceEvent("SRL_Set").detail("Mutation", op);
|
||||
tr->set(op.param1, op.param2);
|
||||
} else if (op.type == MutationRef::ClearRange) {
|
||||
//TraceEvent("SRL_Clear").detail("Mutation", op.toString());
|
||||
//TraceEvent("SRL_Clear").detail("Mutation", op);
|
||||
tr->clear(KeyRangeRef(op.param1, op.param2));
|
||||
} else {
|
||||
//TraceEvent("SRL_AtomicOp").detail("Mutation", op.toString());
|
||||
//TraceEvent("SRL_AtomicOp").detail("Mutation", op);
|
||||
tr->atomicOp(op.param1, op.param2, op.type);
|
||||
}
|
||||
} else if (ops[opNum].readConflictOp.present()) {
|
||||
|
|
|
@ -73,6 +73,7 @@ set(FLOW_SRCS
|
|||
TypeTraits.h
|
||||
UnitTest.cpp
|
||||
UnitTest.h
|
||||
WriteOnlySet.actor.cpp
|
||||
XmlTraceLogFormatter.cpp
|
||||
XmlTraceLogFormatter.h
|
||||
actorcompiler.h
|
||||
|
@ -124,40 +125,57 @@ configure_file(${CMAKE_CURRENT_SOURCE_DIR}/config.h.cmake ${CMAKE_CURRENT_BINARY
|
|||
|
||||
add_flow_target(STATIC_LIBRARY NAME flow SRCS ${FLOW_SRCS})
|
||||
target_link_libraries(flow PRIVATE stacktrace)
|
||||
|
||||
add_flow_target(STATIC_LIBRARY NAME flow_sampling SRCS ${FLOW_SRCS})
|
||||
target_link_libraries(flow_sampling PRIVATE stacktrace)
|
||||
target_compile_definitions(flow_sampling PRIVATE -DENABLE_SAMPLING)
|
||||
|
||||
if (NOT APPLE AND NOT WIN32)
|
||||
set (FLOW_LIBS ${FLOW_LIBS} rt)
|
||||
elseif(WIN32)
|
||||
target_link_libraries(flow PUBLIC winmm.lib)
|
||||
target_link_libraries(flow PUBLIC psapi.lib)
|
||||
target_link_libraries(flow_sampling PUBLIC winmm.lib)
|
||||
target_link_libraries(flow_sampling PUBLIC psapi.lib)
|
||||
endif()
|
||||
if(CMAKE_SYSTEM_NAME STREQUAL "FreeBSD")
|
||||
set (FLOW_LIBS ${FLOW_LIBS} execinfo devstat)
|
||||
find_library(EIO eio)
|
||||
if(EIO)
|
||||
target_link_libraries(flow PUBLIC ${EIO})
|
||||
target_link_libraries(flow_sampling PUBLIC ${EIO})
|
||||
endif()
|
||||
endif()
|
||||
target_link_libraries(flow PRIVATE ${FLOW_LIBS})
|
||||
target_link_libraries(flow_sampling PRIVATE ${FLOW_LIBS})
|
||||
if(USE_VALGRIND)
|
||||
target_link_libraries(flow PUBLIC Valgrind)
|
||||
target_link_libraries(flow_sampling PUBLIC Valgrind)
|
||||
endif()
|
||||
if(NOT WITH_TLS)
|
||||
target_compile_definitions(flow PUBLIC TLS_DISABLED)
|
||||
target_compile_definitions(flow_sampling PUBLIC TLS_DISABLED)
|
||||
else()
|
||||
target_link_libraries(flow PUBLIC OpenSSL::SSL)
|
||||
target_link_libraries(flow_sampling PUBLIC OpenSSL::SSL)
|
||||
endif()
|
||||
target_link_libraries(flow PUBLIC Threads::Threads ${CMAKE_DL_LIBS})
|
||||
target_link_libraries(flow_sampling PUBLIC Threads::Threads ${CMAKE_DL_LIBS})
|
||||
if(USE_SANITIZER)
|
||||
target_link_libraries(flow PUBLIC boost_asan)
|
||||
target_link_libraries(flow_sampling PUBLIC boost_asan)
|
||||
else()
|
||||
target_link_libraries(flow PUBLIC boost_target)
|
||||
target_link_libraries(flow_sampling PUBLIC boost_target)
|
||||
endif()
|
||||
if(USE_VALGRIND)
|
||||
target_link_libraries(flow PUBLIC Valgrind)
|
||||
target_link_libraries(flow_sampling PUBLIC Valgrind)
|
||||
endif()
|
||||
|
||||
if(APPLE)
|
||||
find_library(IO_KIT IOKit)
|
||||
find_library(CORE_FOUNDATION CoreFoundation)
|
||||
target_link_libraries(flow PRIVATE ${IO_KIT} ${CORE_FOUNDATION})
|
||||
target_link_libraries(flow_sampling PRIVATE ${IO_KIT} ${CORE_FOUNDATION})
|
||||
endif()
|
||||
|
|
|
@ -111,6 +111,36 @@ private:
|
|||
Promise<T> promise;
|
||||
};
|
||||
|
||||
template <class T>
|
||||
class ThreadReturnPromiseStream : NonCopyable {
|
||||
public:
|
||||
ThreadReturnPromiseStream() {}
|
||||
~ThreadReturnPromiseStream() {}
|
||||
|
||||
FutureStream<T> getFuture() { // Call only on the originating thread!
|
||||
return promiseStream.getFuture();
|
||||
}
|
||||
|
||||
void send(T const& t) { // Can be called safely from another thread.
|
||||
Promise<Void> signal;
|
||||
tagAndForward(&promiseStream, t, signal.getFuture());
|
||||
g_network->onMainThread(std::move(signal),
|
||||
g_network->isOnMainThread() ? incrementPriorityIfEven(g_network->getCurrentTask())
|
||||
: TaskPriority::DefaultOnMainThread);
|
||||
}
|
||||
|
||||
void sendError(Error const& e) { // Can be called safely from another thread.
|
||||
Promise<Void> signal;
|
||||
tagAndForwardError(&promiseStream, e, signal.getFuture());
|
||||
g_network->onMainThread(std::move(signal),
|
||||
g_network->isOnMainThread() ? incrementPriorityIfEven(g_network->getCurrentTask())
|
||||
: TaskPriority::DefaultOnMainThread);
|
||||
}
|
||||
|
||||
private:
|
||||
PromiseStream<T> promiseStream;
|
||||
};
|
||||
|
||||
Reference<IThreadPool> createGenericThreadPool(int stackSize = 0);
|
||||
|
||||
class DummyThreadPool final : public IThreadPool, ReferenceCounted<DummyThreadPool> {
|
||||
|
|
|
@ -60,6 +60,86 @@ TEST_CASE("/flow/IThreadPool/NamedThread") {
|
|||
return Void();
|
||||
}
|
||||
|
||||
struct ThreadSafePromiseStreamSender final : IThreadPoolReceiver {
|
||||
ThreadSafePromiseStreamSender(ThreadReturnPromiseStream<std::string>* notifications)
|
||||
: notifications(notifications) {}
|
||||
void init() override {}
|
||||
|
||||
struct GetNameAction final : TypedAction<ThreadSafePromiseStreamSender, GetNameAction> {
|
||||
double getTimeEstimate() const override { return 3.; }
|
||||
};
|
||||
|
||||
void action(GetNameAction& a) {
|
||||
pthread_t t = pthread_self();
|
||||
const size_t arrayLen = 16;
|
||||
char name[arrayLen];
|
||||
int err = pthread_getname_np(t, name, arrayLen);
|
||||
if (err != 0) {
|
||||
std::cout << "Get name failed with error code: " << err << std::endl;
|
||||
notifications->sendError(platform_error());
|
||||
return;
|
||||
}
|
||||
notifications->send(name);
|
||||
}
|
||||
|
||||
struct FaultyAction final : TypedAction<ThreadSafePromiseStreamSender, FaultyAction> {
|
||||
double getTimeEstimate() const override { return 3.; }
|
||||
};
|
||||
|
||||
void action(FaultyAction& a) { notifications->sendError(platform_error().asInjectedFault()); }
|
||||
|
||||
private:
|
||||
ThreadReturnPromiseStream<std::string>* notifications;
|
||||
};
|
||||
|
||||
TEST_CASE("/flow/IThreadPool/ThreadReturnPromiseStream") {
|
||||
noUnseed = true;
|
||||
|
||||
state std::unique_ptr<ThreadReturnPromiseStream<std::string>> notifications(
|
||||
new ThreadReturnPromiseStream<std::string>());
|
||||
|
||||
state Reference<IThreadPool> pool = createGenericThreadPool();
|
||||
pool->addThread(new ThreadSafePromiseStreamSender(notifications.get()), "thread-foo");
|
||||
|
||||
// Warning: this action is a little racy with the call to `pthread_setname_np`. In practice,
|
||||
// ~nothing should depend on the thread name being set instantaneously. If this test ever
|
||||
// flakes, we can make `startThread` in platform a little bit more complex to clearly order
|
||||
// the actions.
|
||||
state int num = 3;
|
||||
for (int i = 0; i < num; ++i) {
|
||||
auto* a = new ThreadSafePromiseStreamSender::GetNameAction();
|
||||
pool->post(a);
|
||||
}
|
||||
|
||||
state FutureStream<std::string> futs = notifications->getFuture();
|
||||
|
||||
state int n = 0;
|
||||
while (n < num) {
|
||||
std::string name = waitNext(futs);
|
||||
if (name != "thread-foo") {
|
||||
std::cout << "Incorrect thread name: " << name << std::endl;
|
||||
ASSERT(false);
|
||||
}
|
||||
++n;
|
||||
}
|
||||
|
||||
ASSERT(n == num);
|
||||
|
||||
auto* faultyAction = new ThreadSafePromiseStreamSender::FaultyAction();
|
||||
pool->post(faultyAction);
|
||||
|
||||
try {
|
||||
std::string name = waitNext(futs);
|
||||
ASSERT(false);
|
||||
} catch (Error& e) {
|
||||
ASSERT(e.isInjectedFault());
|
||||
}
|
||||
|
||||
wait(pool->stop());
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
#else
|
||||
void forceLinkIThreadPoolTests() {}
|
||||
#endif
|
||||
|
|
|
@ -205,6 +205,10 @@ public:
|
|||
|
||||
bool checkRunnable() override;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& getActorLineageSet() override;
|
||||
#endif
|
||||
|
||||
bool useThreadPool;
|
||||
|
||||
// private:
|
||||
|
@ -227,11 +231,17 @@ public:
|
|||
TaskPriority currentTaskID;
|
||||
uint64_t tasksIssued;
|
||||
TDMetricCollection tdmetrics;
|
||||
double currentTime;
|
||||
// we read now() from a different thread. On Intel, reading a double is atomic anyways, but on other platforms it's
|
||||
// not. For portability this should be atomic
|
||||
std::atomic<double> currentTime;
|
||||
// May be accessed off the network thread, e.g. by onMainThread
|
||||
std::atomic<bool> stopped;
|
||||
mutable std::map<IPAddress, bool> addressOnHostCache;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet actorLineageSet;
|
||||
#endif
|
||||
|
||||
std::atomic<bool> started;
|
||||
|
||||
uint64_t numYields;
|
||||
|
@ -1381,6 +1391,12 @@ bool Net2::checkRunnable() {
|
|||
return !started.exchange(true);
|
||||
}
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineageSet& Net2::getActorLineageSet() {
|
||||
return actorLineageSet;
|
||||
}
|
||||
#endif
|
||||
|
||||
void Net2::run() {
|
||||
TraceEvent::setNetworkThread();
|
||||
TraceEvent("Net2Running").log();
|
||||
|
|
|
@ -51,6 +51,10 @@
|
|||
#include "flow/UnitTest.h"
|
||||
#include "flow/FaultInjection.h"
|
||||
|
||||
#include "fdbrpc/IAsyncFile.h"
|
||||
|
||||
#include "fdbclient/AnnotateActor.h"
|
||||
|
||||
#ifdef _WIN32
|
||||
#include <windows.h>
|
||||
#include <winioctl.h>
|
||||
|
|
|
@ -797,17 +797,17 @@ inline void fdb_probe_actor_exit(const char* name, unsigned long id, int index)
|
|||
#include <inttypes.h>
|
||||
static inline uint32_t hwCrc32cU8(unsigned int crc, unsigned char v) {
|
||||
uint32_t ret;
|
||||
asm volatile("crc32cb %w[r], %w[c], %w[v]" : [r] "=r"(ret) : [c] "r"(crc), [v] "r"(v));
|
||||
asm volatile("crc32cb %w[r], %w[c], %w[v]" : [ r ] "=r"(ret) : [ c ] "r"(crc), [ v ] "r"(v));
|
||||
return ret;
|
||||
}
|
||||
static inline uint32_t hwCrc32cU32(unsigned int crc, unsigned int v) {
|
||||
uint32_t ret;
|
||||
asm volatile("crc32cw %w[r], %w[c], %w[v]" : [r] "=r"(ret) : [c] "r"(crc), [v] "r"(v));
|
||||
asm volatile("crc32cw %w[r], %w[c], %w[v]" : [ r ] "=r"(ret) : [ c ] "r"(crc), [ v ] "r"(v));
|
||||
return ret;
|
||||
}
|
||||
static inline uint64_t hwCrc32cU64(uint64_t crc, uint64_t v) {
|
||||
uint64_t ret;
|
||||
asm volatile("crc32cx %w[r], %w[c], %x[v]" : [r] "=r"(ret) : [c] "r"(crc), [v] "r"(v));
|
||||
asm volatile("crc32cx %w[r], %w[c], %x[v]" : [ r ] "=r"(ret) : [ c ] "r"(crc), [ v ] "r"(v));
|
||||
return ret;
|
||||
}
|
||||
#else
|
||||
|
|
|
@ -142,6 +142,8 @@ struct Profiler {
|
|||
}
|
||||
|
||||
void signal_handler() { // async signal safe!
|
||||
static std::atomic<bool> inSigHandler = false;
|
||||
if (inSigHandler.exchange(true)) { return; }
|
||||
if (profilingEnabled) {
|
||||
double t = timer();
|
||||
output_buffer->push(*(void**)&t);
|
||||
|
@ -150,6 +152,7 @@ struct Profiler {
|
|||
output_buffer->push(addresses[i]);
|
||||
output_buffer->push((void*)-1LL);
|
||||
}
|
||||
inSigHandler.store(false);
|
||||
}
|
||||
|
||||
static void signal_handler_for_closure(int, siginfo_t* si, void*, void* self) { // async signal safe!
|
||||
|
|
|
@ -107,7 +107,7 @@ private:
|
|||
std::vector<TraceEventFields> eventBuffer;
|
||||
int loggedLength;
|
||||
int bufferLength;
|
||||
bool opened;
|
||||
std::atomic<bool> opened;
|
||||
int64_t preopenOverflowCount;
|
||||
std::string basename;
|
||||
std::string logGroup;
|
||||
|
|
|
@ -0,0 +1,275 @@
|
|||
/*
|
||||
* WriteOnlySet.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "flow/DeterministicRandom.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/UnitTest.h"
|
||||
|
||||
#include <chrono>
|
||||
#include <random>
|
||||
#include "flow/actorcompiler.h" // has to be last include
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
auto WriteOnlySet<T, IndexType, CAPACITY>::insert(const Reference<T>& lineage) -> Index {
|
||||
Index res;
|
||||
if (!freeQueue.pop(res)) {
|
||||
TraceEvent(SevWarnAlways, "NoCapacityInWriteOnlySet");
|
||||
return npos;
|
||||
}
|
||||
ASSERT(_set[res].load() == 0);
|
||||
auto ptr = reinterpret_cast<uintptr_t>(lineage.getPtr());
|
||||
ASSERT((ptr % 2) == 0); // this needs to be at least 2-byte aligned
|
||||
ASSERT(ptr != 0);
|
||||
lineage->addref();
|
||||
_set[res].store(ptr);
|
||||
return res;
|
||||
}
|
||||
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
bool WriteOnlySet<T, IndexType, CAPACITY>::eraseImpl(Index idx) {
|
||||
while (true) {
|
||||
auto ptr = _set[idx].load();
|
||||
if (ptr & LOCK) {
|
||||
_set[idx].store(0);
|
||||
freeList.push(reinterpret_cast<T*>(ptr ^ LOCK));
|
||||
return false;
|
||||
} else {
|
||||
if (_set[idx].compare_exchange_strong(ptr, 0)) {
|
||||
reinterpret_cast<T*>(ptr)->delref();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
bool WriteOnlySet<T, IndexType, CAPACITY>::erase(Index idx) {
|
||||
ASSERT(idx >= 0 && idx < CAPACITY);
|
||||
auto res = eraseImpl(idx);
|
||||
ASSERT(freeQueue.push(idx));
|
||||
return res;
|
||||
}
|
||||
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
bool WriteOnlySet<T, IndexType, CAPACITY>::replace(Index idx, const Reference<T>& lineage) {
|
||||
auto lineagePtr = reinterpret_cast<uintptr_t>(lineage.getPtr());
|
||||
if (lineage.isValid()) {
|
||||
lineage->addref();
|
||||
}
|
||||
ASSERT((lineagePtr % 2) == 0); // this needs to be at least 2-byte aligned
|
||||
|
||||
while (true) {
|
||||
auto ptr = _set[idx].load();
|
||||
if (ptr & LOCK) {
|
||||
_set[idx].store(lineagePtr);
|
||||
ASSERT(freeList.push(reinterpret_cast<T*>(ptr ^ LOCK)));
|
||||
return false;
|
||||
} else {
|
||||
if (_set[idx].compare_exchange_strong(ptr, lineagePtr)) {
|
||||
if (ptr) {
|
||||
reinterpret_cast<T*>(ptr)->delref();
|
||||
}
|
||||
return ptr != 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
WriteOnlySet<T, IndexType, CAPACITY>::WriteOnlySet() : _set(CAPACITY) {
|
||||
// insert the free indexes in reverse order
|
||||
for (unsigned i = CAPACITY; i > 0; --i) {
|
||||
freeQueue.push(i - 1);
|
||||
std::atomic_init(&_set[i - 1], uintptr_t(0));
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
std::vector<Reference<T>> WriteOnlySet<T, IndexType, CAPACITY>::copy() {
|
||||
std::vector<Reference<T>> result;
|
||||
for (int i = 0; i < CAPACITY; ++i) {
|
||||
auto ptr = _set[i].load();
|
||||
if (ptr) {
|
||||
ASSERT((ptr & LOCK) == 0); // if we lock something we need to immediately unlock after we're done copying
|
||||
// We attempt lock so this won't get deleted. We will try this only once, if the other thread removed the
|
||||
// object from the set between the previews lines and now, we just won't make it part of the result.
|
||||
if (_set[i].compare_exchange_strong(ptr, ptr | LOCK)) {
|
||||
T* entry = reinterpret_cast<T*>(ptr);
|
||||
ptr |= LOCK;
|
||||
entry->addref();
|
||||
// we try to unlock now. If this element was removed while we incremented the refcount, the element will
|
||||
// end up in the freeList, so we will decrement later.
|
||||
_set[i].compare_exchange_strong(ptr, ptr ^ LOCK);
|
||||
result.push_back(Reference(entry));
|
||||
}
|
||||
}
|
||||
}
|
||||
// after we're done we need to clean up all objects that contented on a lock. This won't be perfect (as some thread
|
||||
// might not yet added the object to the free list), but whatever we don't get now we'll clean up in the next
|
||||
// iteration
|
||||
freeList.consume_all([](auto toClean) { toClean->delref(); });
|
||||
return result;
|
||||
}
|
||||
|
||||
template <class T, class IndexType>
|
||||
WriteOnlyVariable<T, IndexType>::WriteOnlyVariable() : WriteOnlySet<T, IndexType, 1>() {}
|
||||
|
||||
template <class T, class IndexType>
|
||||
Reference<T> WriteOnlyVariable<T, IndexType>::get() {
|
||||
auto result = WriteOnlySet<T, IndexType, 1>::copy();
|
||||
return result.size() ? result.at(0) : Reference<T>();
|
||||
}
|
||||
|
||||
template <class T, class IndexType>
|
||||
bool WriteOnlyVariable<T, IndexType>::replace(const Reference<T>& element) {
|
||||
return WriteOnlySet<T, IndexType, 1>::replace(0, element);
|
||||
}
|
||||
|
||||
// Explicit instantiation
|
||||
template class WriteOnlySet<ActorLineage, unsigned, 1024>;
|
||||
template class WriteOnlyVariable<ActorLineage, unsigned>;
|
||||
|
||||
// testing code
|
||||
namespace {
|
||||
|
||||
// Some statistics
|
||||
std::atomic<unsigned long> instanceCounter = 0;
|
||||
std::atomic<unsigned long> numInserts = 0;
|
||||
std::atomic<unsigned long> numErase = 0;
|
||||
std::atomic<unsigned long> numLockedErase = 0;
|
||||
std::atomic<unsigned long> numCopied = 0;
|
||||
|
||||
// A simple object that counts the number of its instances. This is used to detect memory leaks.
|
||||
struct TestObject {
|
||||
mutable std::atomic<unsigned> _refCount = 1;
|
||||
TestObject() { instanceCounter.fetch_add(1); }
|
||||
void delref() const {
|
||||
if (--_refCount == 0) {
|
||||
delete this;
|
||||
--instanceCounter;
|
||||
}
|
||||
}
|
||||
void addref() const { ++_refCount; }
|
||||
};
|
||||
|
||||
using TestSet = WriteOnlySet<TestObject, unsigned, 128>;
|
||||
using Clock = std::chrono::steady_clock;
|
||||
|
||||
// An actor that can join a set of threads in an async way.
|
||||
ACTOR Future<Void> threadjoiner(std::shared_ptr<std::vector<std::thread>> threads, std::shared_ptr<TestSet> set) {
|
||||
loop {
|
||||
wait(delay(0.1));
|
||||
for (unsigned i = 0;;) {
|
||||
if (threads->size() == i) {
|
||||
break;
|
||||
}
|
||||
auto& t = (*threads)[i];
|
||||
if (t.joinable()) {
|
||||
t.join();
|
||||
if (i + 1 < threads->size()) {
|
||||
std::swap(*threads->rbegin(), (*threads)[i]);
|
||||
}
|
||||
threads->pop_back();
|
||||
} else {
|
||||
++i;
|
||||
}
|
||||
}
|
||||
if (threads->empty()) {
|
||||
set->copy();
|
||||
ASSERT(instanceCounter.load() == 0);
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// occasionally copy the contents of the past set.
|
||||
void testCopier(std::shared_ptr<TestSet> set, std::chrono::seconds runFor) {
|
||||
auto start = Clock::now();
|
||||
while (true) {
|
||||
if (Clock::now() - start > runFor) {
|
||||
return;
|
||||
}
|
||||
auto copy = set->copy();
|
||||
numCopied.fetch_add(copy.size());
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||
}
|
||||
}
|
||||
|
||||
// In a loop adds and removes a set of objects to the set
|
||||
void writer(std::shared_ptr<TestSet> set, std::chrono::seconds runFor) {
|
||||
auto start = Clock::now();
|
||||
std::random_device rDev;
|
||||
DeterministicRandom rnd(rDev());
|
||||
while (true) {
|
||||
unsigned inserts = 0, erases = 0;
|
||||
if (Clock::now() - start > runFor) {
|
||||
return;
|
||||
}
|
||||
std::vector<TestSet::Index> positions;
|
||||
for (int i = 0; i < rnd.randomInt(1, 101); ++i) {
|
||||
Reference<TestObject> o(new TestObject());
|
||||
auto pos = set->insert(o);
|
||||
if (pos == TestSet::npos) {
|
||||
// could not insert -- ignore
|
||||
break;
|
||||
}
|
||||
++inserts;
|
||||
ASSERT(pos < TestSet::capacity);
|
||||
positions.push_back(pos);
|
||||
}
|
||||
rnd.randomShuffle(positions);
|
||||
for (auto p : positions) {
|
||||
if (!set->erase(p)) {
|
||||
++numLockedErase;
|
||||
}
|
||||
++erases;
|
||||
}
|
||||
numInserts.fetch_add(inserts);
|
||||
numErase.fetch_add(erases);
|
||||
ASSERT(inserts == erases);
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
||||
}
|
||||
}
|
||||
|
||||
// This unit test creates 5 writer threads and one copier thread.
|
||||
TEST_CASE("/flow/WriteOnlySet") {
|
||||
if (g_network->isSimulated()) {
|
||||
// This test is not deterministic, so we shouldn't run it in simulation
|
||||
return Void();
|
||||
}
|
||||
auto set = std::make_shared<TestSet>();
|
||||
auto threads = std::make_shared<std::vector<std::thread>>();
|
||||
std::chrono::seconds runFor(10);
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
threads->emplace_back([set, runFor]() { writer(set, runFor); });
|
||||
}
|
||||
threads->emplace_back([set, runFor]() { testCopier(set, runFor); });
|
||||
wait(threadjoiner(threads, set));
|
||||
TraceEvent("WriteOnlySetTestResult")
|
||||
.detail("Inserts", numInserts.load())
|
||||
.detail("Erases", numErase.load())
|
||||
.detail("Copies", numCopied.load())
|
||||
.detail("LockedErase", numLockedErase.load());
|
||||
return Void();
|
||||
}
|
||||
} // namespace
|
||||
#endif
|
|
@ -0,0 +1,164 @@
|
|||
/*
|
||||
* WriteOnlySet.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
#include "flow/Error.h"
|
||||
#include "flow/FastRef.h"
|
||||
#include "flow/Trace.h"
|
||||
#include <boost/lockfree/queue.hpp>
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
/**
|
||||
* This is a Write-Only set that supports copying the whole content. This data structure is lock-free and allows a user
|
||||
* to insert and remove objects up to a given capacity (passed by a template).
|
||||
*
|
||||
* Template parameters:
|
||||
* \param T The type to store.
|
||||
* \param IndexType The type used as an index
|
||||
* \param CAPACITY The maximum number of object this structure can store (if a user tries to store more, insert will
|
||||
* fail gracefully)
|
||||
* \pre T implements `void addref() const` and `void delref() const`
|
||||
* \pre IndexType must have a copy constructor
|
||||
* \pre IndexType must have a trivial assignment operator
|
||||
* \pre IndexType must have a trivial destructor
|
||||
* \pre IndexType can be used as an index into a std::vector
|
||||
*/
|
||||
template <class T, class IndexType, IndexType CAPACITY>
|
||||
class WriteOnlySet {
|
||||
public:
|
||||
// The type we use for lookup into the set. Gets assigned during insert
|
||||
using Index = IndexType;
|
||||
// For now we use a fixed size capacity
|
||||
constexpr static Index npos = std::numeric_limits<Index>::max();
|
||||
constexpr static IndexType capacity = CAPACITY;
|
||||
|
||||
explicit WriteOnlySet();
|
||||
WriteOnlySet(const WriteOnlySet&) = delete;
|
||||
WriteOnlySet(WriteOnlySet&&) = delete;
|
||||
WriteOnlySet& operator=(const WriteOnlySet&) = delete;
|
||||
WriteOnlySet& operator=(WriteOnlySet&&) = delete;
|
||||
|
||||
/**
|
||||
* Attempts to insert \p lineage into the set. This method can fail if the set is full (its size is equal to its
|
||||
* capacity). Calling insert on a full set is safe but the method will return \ref npos if the operation fails.
|
||||
*
|
||||
* \param lineage A reference to the object the user wants to insert.
|
||||
* \ret An index that can later be used to erase the value again or \ref npos if the insert failed.
|
||||
* \pre lineage.getPtr() % 2 == 0 (the memory for lineage has to be at least 2 byte aligned)
|
||||
*/
|
||||
[[nodiscard]] Index insert(const Reference<T>& lineage);
|
||||
|
||||
/**
|
||||
* Erases the object associated with \p idx from the set.
|
||||
*
|
||||
* \ret Whether the reference count was decremented. Usually the return value is only interesting for testing and
|
||||
* benchmarking purposes and will in most cases be ignored. If \ref delref wasn't called, it will be called
|
||||
* later. Note that at the time the return value is checked, \ref delref might already have been called.
|
||||
*/
|
||||
bool erase(Index idx);
|
||||
|
||||
/**
|
||||
* Replaces the object associated with \p idx with \p lineage.
|
||||
*
|
||||
* \ret Whether the reference count of the replaced object was decremented. Usually the return value is only
|
||||
* interesting for testing and benchmarking purposes and will in most cases be ignored. If \ref delref
|
||||
* wasn't called, it will be called later. Note that at the time the return value is checked, \ref delref
|
||||
* might already have been called.
|
||||
*/
|
||||
bool replace(Index idx, const Reference<T>& lineage);
|
||||
|
||||
/**
|
||||
* Copies all elements that are stored in the set into a vector. This copy operation does NOT provide a snapshot of
|
||||
* the data structure. The contract is weak:
|
||||
* - All object that were in the set before copy is called and weren't removed until after copy returned are
|
||||
* guaranteed to be in the result.
|
||||
* - Any object that was inserted while copy is running might be in the result.
|
||||
* - Any object that was erased while copy is running might be in the result.
|
||||
*/
|
||||
std::vector<Reference<T>> copy();
|
||||
|
||||
protected:
|
||||
// the implementation of erase -- the wrapper just makes the function a bit more readable.
|
||||
bool eraseImpl(Index idx);
|
||||
|
||||
// the last bit of a pointer within the set is used like a boolean and true means that the object is locked. Locking
|
||||
// an object is only relevant for memory management. A locked pointer can still be erased from the set, but the
|
||||
// erase won't call delref on the object. Instead it will push the pointer into the \ref freeList and copy will call
|
||||
// delref later.
|
||||
static constexpr uintptr_t LOCK = 0b1;
|
||||
|
||||
// The actual memory
|
||||
std::vector<std::atomic<std::uintptr_t>> _set;
|
||||
static_assert(std::atomic<Index>::is_always_lock_free, "Index type can't be used as a lock-free type");
|
||||
static_assert(std::atomic<uintptr_t>::is_always_lock_free, "uintptr_t can't be used as a lock-free type");
|
||||
|
||||
// The freeQueue. On creation all indexes (0..capacity-1) are pushed into this queue. On insert one element from
|
||||
// this queue is consumed and the resulting number is used as an index into the set. On erase the index is given
|
||||
// back to the freeQueue.
|
||||
boost::lockfree::queue<Index, boost::lockfree::fixed_sized<true>, boost::lockfree::capacity<CAPACITY>> freeQueue;
|
||||
|
||||
// The freeList is used for memory management. Generally copying a shared pointer can't be done in a lock-free way.
|
||||
// Instead, when we copy the data structure we first copy the address, then attempt to set the last bit to 1 and
|
||||
// only if that succeeds we will increment the reference count. Whenever we attempt to remove an object
|
||||
// in \ref erase we remove the object from the set (using an atomic compare and swap) and only decrement the
|
||||
// reference count if the last bit is 0. If it's not we'll push the pointer into this free list.
|
||||
// \ref copy will consume all elements from this freeList each time it runs and decrements the refcount for each
|
||||
// element.
|
||||
boost::lockfree::queue<T*, boost::lockfree::fixed_sized<true>, boost::lockfree::capacity<CAPACITY>> freeList;
|
||||
};
|
||||
|
||||
/**
|
||||
* Provides a thread safe, lock-free write only variable.
|
||||
*
|
||||
* Template parameters:
|
||||
* \param T The type to store.
|
||||
* \param IndexType The type used as an index
|
||||
* \pre T implements `void addref() const` and `void delref() const`
|
||||
* \pre IndexType must have a copy constructor
|
||||
* \pre IndexType must have a trivial assignment operator
|
||||
* \pre IndexType must have a trivial destructor
|
||||
* \pre IndexType can be used as an index into a std::vector
|
||||
*/
|
||||
template <class T, class IndexType>
|
||||
class WriteOnlyVariable : private WriteOnlySet<T, IndexType, 1> {
|
||||
public:
|
||||
explicit WriteOnlyVariable();
|
||||
|
||||
/**
|
||||
* Returns a copied reference to the stored variable.
|
||||
*/
|
||||
Reference<T> get();
|
||||
|
||||
/**
|
||||
* Replaces the variable with \p lineage. \p lineage is permitted to be an invalid pointer.
|
||||
*
|
||||
* \ret Whether the reference count of the replaced object was decremented. Note that if the reference being replaced
|
||||
* is invalid, this function will always return false. If \ref delref wasn't called and the reference was valid,
|
||||
* it will be called later. Note that at the time the return value is checked, \ref delref might already have
|
||||
* been called.
|
||||
*/
|
||||
bool replace(const Reference<T>& element);
|
||||
};
|
||||
|
||||
class ActorLineage;
|
||||
extern template class WriteOnlySet<ActorLineage, unsigned, 1024>;
|
||||
|
||||
using ActorLineageSet = WriteOnlySet<ActorLineage, unsigned, 1024>;
|
||||
#endif
|
|
@ -1292,6 +1292,11 @@ namespace actorcompiler
|
|||
constructor.WriteLine("{");
|
||||
constructor.Indent(+1);
|
||||
ProbeEnter(constructor, actor.name);
|
||||
constructor.WriteLine("#ifdef ENABLE_SAMPLING");
|
||||
constructor.WriteLine("this->lineage.setActorName(\"{0}\");", actor.name);
|
||||
constructor.WriteLine("LineageScope _(&this->lineage);");
|
||||
constructor.WriteLine("#endif");
|
||||
// constructor.WriteLine("getCurrentLineage()->modify(&StackLineage::actorName) = LiteralStringRef(\"{0}\");", actor.name);
|
||||
constructor.WriteLine("this->{0};", body.call());
|
||||
ProbeExit(constructor, actor.name);
|
||||
WriteFunction(writer, constructor, constructor.BodyText);
|
||||
|
|
|
@ -1,108 +1,8 @@
|
|||
<?xml version="1.0" encoding="utf-8"?>
|
||||
<Project ToolsVersion="4.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
|
||||
<Project Sdk="Microsoft.NET.Sdk">
|
||||
|
||||
<PropertyGroup>
|
||||
<Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
|
||||
<ProductVersion>10.0.20506</ProductVersion>
|
||||
<SchemaVersion>2.0</SchemaVersion>
|
||||
<ProjectGuid>{0ECC1314-3FC2-458D-8E41-B50B4EA24E51}</ProjectGuid>
|
||||
<OutputType>Exe</OutputType>
|
||||
<AppDesignerFolder>Properties</AppDesignerFolder>
|
||||
<RootNamespace>actorcompiler</RootNamespace>
|
||||
<AssemblyName>actorcompiler</AssemblyName>
|
||||
<TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
|
||||
<FileAlignment>512</FileAlignment>
|
||||
<OutputPath>$(SolutionDir)bin\$(Configuration)\</OutputPath>
|
||||
<PublishUrl>publish\</PublishUrl>
|
||||
<Install>true</Install>
|
||||
<InstallFrom>Disk</InstallFrom>
|
||||
<UpdateEnabled>false</UpdateEnabled>
|
||||
<UpdateMode>Foreground</UpdateMode>
|
||||
<UpdateInterval>7</UpdateInterval>
|
||||
<UpdateIntervalUnits>Days</UpdateIntervalUnits>
|
||||
<UpdatePeriodically>false</UpdatePeriodically>
|
||||
<UpdateRequired>false</UpdateRequired>
|
||||
<MapFileExtensions>true</MapFileExtensions>
|
||||
<ApplicationRevision>0</ApplicationRevision>
|
||||
<ApplicationVersion>1.0.0.%2a</ApplicationVersion>
|
||||
<IsWebBootstrapper>false</IsWebBootstrapper>
|
||||
<UseApplicationTrust>false</UseApplicationTrust>
|
||||
<BootstrapperEnabled>true</BootstrapperEnabled>
|
||||
<TargetFramework>net5.0</TargetFramework>
|
||||
</PropertyGroup>
|
||||
<PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|AnyCPU'">
|
||||
<DebugSymbols>true</DebugSymbols>
|
||||
<DefineConstants>DEBUG;TRACE</DefineConstants>
|
||||
<DebugType>full</DebugType>
|
||||
<PlatformTarget>AnyCPU</PlatformTarget>
|
||||
<LangVersion>default</LangVersion>
|
||||
<ErrorReport>prompt</ErrorReport>
|
||||
<CodeAnalysisIgnoreBuiltInRuleSets>false</CodeAnalysisIgnoreBuiltInRuleSets>
|
||||
<CodeAnalysisFailOnMissingRules>false</CodeAnalysisFailOnMissingRules>
|
||||
</PropertyGroup>
|
||||
<PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
|
||||
<DefineConstants>TRACE</DefineConstants>
|
||||
<Optimize>true</Optimize>
|
||||
<DebugType>pdbonly</DebugType>
|
||||
<PlatformTarget>AnyCPU</PlatformTarget>
|
||||
<LangVersion>default</LangVersion>
|
||||
<ErrorReport>prompt</ErrorReport>
|
||||
<CodeAnalysisIgnoreBuiltInRuleSets>false</CodeAnalysisIgnoreBuiltInRuleSets>
|
||||
<CodeAnalysisIgnoreBuiltInRules>false</CodeAnalysisIgnoreBuiltInRules>
|
||||
</PropertyGroup>
|
||||
<ItemGroup>
|
||||
<Reference Include="System" />
|
||||
<Reference Include="System.Core">
|
||||
<RequiredTargetFramework>3.5</RequiredTargetFramework>
|
||||
</Reference>
|
||||
<Reference Include="System.Xml.Linq">
|
||||
<RequiredTargetFramework>3.5</RequiredTargetFramework>
|
||||
</Reference>
|
||||
<Reference Include="System.Data.DataSetExtensions">
|
||||
<RequiredTargetFramework>3.5</RequiredTargetFramework>
|
||||
</Reference>
|
||||
<Reference Include="Microsoft.CSharp">
|
||||
<RequiredTargetFramework>4.0</RequiredTargetFramework>
|
||||
</Reference>
|
||||
<Reference Include="System.Data" />
|
||||
<Reference Include="System.Xml" />
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<Compile Include="ActorCompiler.cs" />
|
||||
<Compile Include="ActorParser.cs" />
|
||||
<Compile Include="ParseTree.cs" />
|
||||
<Compile Include="Program.cs" />
|
||||
<Compile Include="Properties\AssemblyInfo.cs" />
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<BootstrapperPackage Include=".NETFramework,Version=v4.0">
|
||||
<Visible>False</Visible>
|
||||
<ProductName>Microsoft .NET Framework 4 %28x86 and x64%29</ProductName>
|
||||
<Install>true</Install>
|
||||
</BootstrapperPackage>
|
||||
<BootstrapperPackage Include="Microsoft.Net.Client.3.5">
|
||||
<Visible>False</Visible>
|
||||
<ProductName>.NET Framework 3.5 SP1 Client Profile</ProductName>
|
||||
<Install>false</Install>
|
||||
</BootstrapperPackage>
|
||||
<BootstrapperPackage Include="Microsoft.Net.Framework.3.5.SP1">
|
||||
<Visible>False</Visible>
|
||||
<ProductName>.NET Framework 3.5 SP1</ProductName>
|
||||
<Install>false</Install>
|
||||
</BootstrapperPackage>
|
||||
<BootstrapperPackage Include="Microsoft.Windows.Installer.3.1">
|
||||
<Visible>False</Visible>
|
||||
<ProductName>Windows Installer 3.1</ProductName>
|
||||
<Install>true</Install>
|
||||
</BootstrapperPackage>
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<Content Include="Actor checklist.txt" />
|
||||
</ItemGroup>
|
||||
<Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
|
||||
<!-- To modify your build process, add your task inside one of the targets below and uncomment it.
|
||||
Other similar extension points exist, see Microsoft.Common.targets.
|
||||
<Target Name="BeforeBuild">
|
||||
</Target>
|
||||
<Target Name="AfterBuild">
|
||||
</Target>
|
||||
-->
|
||||
|
||||
</Project>
|
|
@ -26,6 +26,51 @@
|
|||
#include <stdarg.h>
|
||||
#include <cinttypes>
|
||||
|
||||
std::atomic<bool> startSampling = false;
|
||||
LineageReference rootLineage;
|
||||
thread_local LineageReference* currentLineage = &rootLineage;
|
||||
|
||||
LineagePropertiesBase::~LineagePropertiesBase() {}
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
ActorLineage::ActorLineage() : properties(), parent(*currentLineage) {}
|
||||
#else
|
||||
ActorLineage::ActorLineage() : properties() {}
|
||||
#endif
|
||||
|
||||
ActorLineage::~ActorLineage() {
|
||||
for (auto property : properties) {
|
||||
delete property.properties;
|
||||
}
|
||||
}
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference getCurrentLineage() {
|
||||
if (!currentLineage->isValid() || !currentLineage->isAllocated()) {
|
||||
currentLineage->allocate();
|
||||
}
|
||||
return *currentLineage;
|
||||
}
|
||||
#endif
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
void sample(LineageReference* lineagePtr);
|
||||
|
||||
void replaceLineage(LineageReference* lineage) {
|
||||
if (!startSampling) {
|
||||
currentLineage = lineage;
|
||||
} else {
|
||||
startSampling = false;
|
||||
sample(currentLineage);
|
||||
currentLineage = lineage;
|
||||
}
|
||||
}
|
||||
#endif
|
||||
|
||||
using namespace std::literals;
|
||||
|
||||
const std::string_view StackLineage::name = "StackLineage"sv;
|
||||
|
||||
#if (defined(__linux__) || defined(__FreeBSD__)) && defined(__AVX__) && !defined(MEMORY_SANITIZER)
|
||||
// For benchmarking; need a version of rte_memcpy that doesn't live in the same compilation unit as the test.
|
||||
void* rte_memcpy_noinline(void* __restrict __dest, const void* __restrict __src, size_t __n) {
|
||||
|
|
258
flow/flow.h
258
flow/flow.h
|
@ -20,6 +20,8 @@
|
|||
|
||||
#ifndef FLOW_FLOW_H
|
||||
#define FLOW_FLOW_H
|
||||
#include "flow/Arena.h"
|
||||
#include "flow/FastRef.h"
|
||||
#pragma once
|
||||
|
||||
#pragma warning(disable : 4244 4267) // SOMEDAY: Carefully check for integer overflow issues (e.g. size_t to int
|
||||
|
@ -29,14 +31,18 @@
|
|||
|
||||
#include <vector>
|
||||
#include <queue>
|
||||
#include <stack>
|
||||
#include <map>
|
||||
#include <unordered_map>
|
||||
#include <set>
|
||||
#include <functional>
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
|
||||
#include "flow/Platform.h"
|
||||
#include "flow/FastAlloc.h"
|
||||
|
@ -46,6 +52,7 @@
|
|||
#include "flow/ThreadPrimitives.h"
|
||||
#include "flow/network.h"
|
||||
#include "flow/FileIdentifier.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
|
||||
#include <boost/version.hpp>
|
||||
|
||||
|
@ -417,6 +424,192 @@ struct SingleCallback {
|
|||
}
|
||||
};
|
||||
|
||||
struct LineagePropertiesBase {
|
||||
virtual ~LineagePropertiesBase();
|
||||
};
|
||||
|
||||
// helper class to make implementation of LineageProperties easier
|
||||
template <class Derived>
|
||||
struct LineageProperties : LineagePropertiesBase {
|
||||
// Contract:
|
||||
//
|
||||
// StringRef name = "SomeUniqueName"_str;
|
||||
|
||||
// this has to be implemented by subclasses
|
||||
// but can't be made virtual.
|
||||
// A user should implement this for any type
|
||||
// within the properies class.
|
||||
template <class Value>
|
||||
bool isSet(Value Derived::*member) const {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
struct ActorLineage : ThreadSafeReferenceCounted<ActorLineage> {
|
||||
friend class LineageReference;
|
||||
|
||||
struct Property {
|
||||
std::string_view name;
|
||||
LineagePropertiesBase* properties;
|
||||
};
|
||||
|
||||
private:
|
||||
std::vector<Property> properties;
|
||||
Reference<ActorLineage> parent;
|
||||
mutable std::mutex mutex;
|
||||
using Lock = std::unique_lock<std::mutex>;
|
||||
using Iterator = std::vector<Property>::const_iterator;
|
||||
|
||||
ActorLineage();
|
||||
Iterator find(const std::string_view& name) const {
|
||||
for (auto it = properties.cbegin(); it != properties.cend(); ++it) {
|
||||
if (it->name == name) {
|
||||
return it;
|
||||
}
|
||||
}
|
||||
return properties.end();
|
||||
}
|
||||
Property& findOrInsert(const std::string_view& name) {
|
||||
for (auto& property : properties) {
|
||||
if (property.name == name) {
|
||||
return property;
|
||||
}
|
||||
}
|
||||
properties.emplace_back(Property{ name, nullptr });
|
||||
return properties.back();
|
||||
}
|
||||
|
||||
public:
|
||||
~ActorLineage();
|
||||
bool isRoot() const {
|
||||
Lock _{ mutex };
|
||||
return parent.getPtr() == nullptr;
|
||||
}
|
||||
void makeRoot() {
|
||||
Lock _{ mutex };
|
||||
parent.clear();
|
||||
}
|
||||
template <class T, class V>
|
||||
V& modify(V T::*member) {
|
||||
Lock _{ mutex };
|
||||
auto& res = findOrInsert(T::name).properties;
|
||||
if (!res) {
|
||||
res = new T{};
|
||||
}
|
||||
T* map = static_cast<T*>(res);
|
||||
return map->*member;
|
||||
}
|
||||
template <class T, class V>
|
||||
std::optional<V> get(V T::*member) const {
|
||||
Lock _{ mutex };
|
||||
auto current = this;
|
||||
while (current != nullptr) {
|
||||
auto iter = current->find(T::name);
|
||||
if (iter != current->properties.end()) {
|
||||
T const& map = static_cast<T const&>(*iter->properties);
|
||||
if (map.isSet(member)) {
|
||||
return map.*member;
|
||||
}
|
||||
}
|
||||
current = current->parent.getPtr();
|
||||
}
|
||||
return std::optional<V>{};
|
||||
}
|
||||
template <class T, class V>
|
||||
std::vector<V> stack(V T::*member) const {
|
||||
Lock _{ mutex };
|
||||
auto current = this;
|
||||
std::vector<V> res;
|
||||
while (current != nullptr) {
|
||||
auto iter = current->find(T::name);
|
||||
if (iter != current->properties.end()) {
|
||||
T const& map = static_cast<T const&>(*iter->properties);
|
||||
if (map.isSet(member)) {
|
||||
res.push_back(map.*member);
|
||||
}
|
||||
}
|
||||
current = current->parent.getPtr();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
Reference<ActorLineage> getParent() {
|
||||
return parent;
|
||||
}
|
||||
};
|
||||
|
||||
// A Reference subclass with knowledge on the true owner of the contained
|
||||
// ActorLineage object. This class enables lazy allocation of ActorLineages.
|
||||
// LineageReference copies are generally made by child actors, which should
|
||||
// create their own ActorLineage when attempting to add lineage properties (see
|
||||
// getCurrentLineage()).
|
||||
class LineageReference : public Reference<ActorLineage> {
|
||||
public:
|
||||
LineageReference() : Reference<ActorLineage>(nullptr), actorName_(""), allocated_(false) {}
|
||||
explicit LineageReference(ActorLineage* ptr) : Reference<ActorLineage>(ptr), actorName_(""), allocated_(false) {}
|
||||
LineageReference(const LineageReference& r) : Reference<ActorLineage>(r), actorName_(""), allocated_(false) {}
|
||||
|
||||
void setActorName(const char* name) { actorName_ = name; }
|
||||
const char* actorName() { return actorName_; }
|
||||
void allocate() {
|
||||
Reference<ActorLineage>::setPtrUnsafe(new ActorLineage());
|
||||
allocated_ = true;
|
||||
}
|
||||
bool isAllocated() { return allocated_; }
|
||||
|
||||
private:
|
||||
// The actor name has to be a property of the LineageReference because all
|
||||
// actors store their own LineageReference copy, but not all actors point
|
||||
// to their own ActorLineage.
|
||||
const char* actorName_;
|
||||
bool allocated_;
|
||||
};
|
||||
|
||||
extern std::atomic<bool> startSampling;
|
||||
extern thread_local LineageReference* currentLineage;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference getCurrentLineage();
|
||||
#else
|
||||
#define getCurrentLineage() if (false) (*currentLineage)
|
||||
#endif
|
||||
void replaceLineage(LineageReference* lineage);
|
||||
|
||||
struct StackLineage : LineageProperties<StackLineage> {
|
||||
static const std::string_view name;
|
||||
StringRef actorName;
|
||||
};
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
struct LineageScope {
|
||||
LineageReference* oldLineage;
|
||||
LineageScope(LineageReference* with) : oldLineage(currentLineage) {
|
||||
replaceLineage(with);
|
||||
}
|
||||
~LineageScope() {
|
||||
replaceLineage(oldLineage);
|
||||
}
|
||||
};
|
||||
#endif
|
||||
|
||||
// This class can be used in order to modify all lineage properties
|
||||
// of actors created within a (non-actor) scope
|
||||
struct LocalLineage {
|
||||
LineageReference lineage;
|
||||
LineageReference* oldLineage;
|
||||
LocalLineage() {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
lineage.allocate();
|
||||
oldLineage = currentLineage;
|
||||
replaceLineage(&lineage);
|
||||
#endif
|
||||
}
|
||||
~LocalLineage() {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
replaceLineage(oldLineage);
|
||||
#endif
|
||||
}
|
||||
};
|
||||
|
||||
// SAV is short for Single Assignment Variable: It can be assigned for only once!
|
||||
template <class T>
|
||||
struct SAV : private Callback<T>, FastAllocated<SAV<T>> {
|
||||
|
@ -458,8 +651,9 @@ public:
|
|||
ASSERT(canBeSet());
|
||||
new (&value_storage) T(std::forward<U>(value));
|
||||
this->error_state = Error::fromCode(SET_ERROR_CODE);
|
||||
while (Callback<T>::next != this)
|
||||
while (Callback<T>::next != this) {
|
||||
Callback<T>::next->fire(this->value());
|
||||
}
|
||||
}
|
||||
|
||||
void send(Never) {
|
||||
|
@ -470,8 +664,9 @@ public:
|
|||
void sendError(Error err) {
|
||||
ASSERT(canBeSet() && int16_t(err.code()) > 0);
|
||||
this->error_state = err;
|
||||
while (Callback<T>::next != this)
|
||||
while (Callback<T>::next != this) {
|
||||
Callback<T>::next->error(err);
|
||||
}
|
||||
}
|
||||
|
||||
template <class U>
|
||||
|
@ -1050,36 +1245,77 @@ static inline void destruct(T& t) {
|
|||
|
||||
template <class ReturnValue>
|
||||
struct Actor : SAV<ReturnValue> {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference lineage = *currentLineage;
|
||||
#endif
|
||||
int8_t actor_wait_state; // -1 means actor is cancelled; 0 means actor is not waiting; 1-N mean waiting in callback
|
||||
// group #
|
||||
|
||||
Actor() : SAV<ReturnValue>(1, 1), actor_wait_state(0) { /*++actorCount;*/
|
||||
Actor() : SAV<ReturnValue>(1, 1), actor_wait_state(0) { /*++actorCount;*/ }
|
||||
// ~Actor() { --actorCount; }
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference* lineageAddr() {
|
||||
return std::addressof(lineage);
|
||||
}
|
||||
//~Actor() { --actorCount; }
|
||||
#endif
|
||||
};
|
||||
|
||||
template <>
|
||||
struct Actor<void> {
|
||||
// This specialization is for a void actor (one not returning a future, hence also uncancellable)
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference lineage = *currentLineage;
|
||||
#endif
|
||||
int8_t actor_wait_state; // 0 means actor is not waiting; 1-N mean waiting in callback group #
|
||||
|
||||
Actor() : actor_wait_state(0) { /*++actorCount;*/
|
||||
Actor() : actor_wait_state(0) { /*++actorCount;*/ }
|
||||
// ~Actor() { --actorCount; }
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference* lineageAddr() {
|
||||
return std::addressof(lineage);
|
||||
}
|
||||
//~Actor() { --actorCount; }
|
||||
#endif
|
||||
};
|
||||
|
||||
template <class ActorType, int CallbackNumber, class ValueType>
|
||||
struct ActorCallback : Callback<ValueType> {
|
||||
void fire(ValueType const& value) override { static_cast<ActorType*>(this)->a_callback_fire(this, value); }
|
||||
void error(Error e) override { static_cast<ActorType*>(this)->a_callback_error(this, e); }
|
||||
virtual void fire(ValueType const& value) override {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageScope _(static_cast<ActorType*>(this)->lineageAddr());
|
||||
#endif
|
||||
static_cast<ActorType*>(this)->a_callback_fire(this, value);
|
||||
}
|
||||
virtual void error(Error e) override {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageScope _(static_cast<ActorType*>(this)->lineageAddr());
|
||||
#endif
|
||||
static_cast<ActorType*>(this)->a_callback_error(this, e);
|
||||
}
|
||||
};
|
||||
|
||||
template <class ActorType, int CallbackNumber, class ValueType>
|
||||
struct ActorSingleCallback : SingleCallback<ValueType> {
|
||||
void fire(ValueType const& value) override { static_cast<ActorType*>(this)->a_callback_fire(this, value); }
|
||||
void fire(ValueType&& value) override { static_cast<ActorType*>(this)->a_callback_fire(this, std::move(value)); }
|
||||
void error(Error e) override { static_cast<ActorType*>(this)->a_callback_error(this, e); }
|
||||
void fire(ValueType const& value) override {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageScope _(static_cast<ActorType*>(this)->lineageAddr());
|
||||
#endif
|
||||
static_cast<ActorType*>(this)->a_callback_fire(this, value);
|
||||
}
|
||||
void fire(ValueType&& value) override {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageScope _(static_cast<ActorType*>(this)->lineageAddr());
|
||||
#endif
|
||||
static_cast<ActorType*>(this)->a_callback_fire(this, std::move(value));
|
||||
}
|
||||
void error(Error e) override {
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageScope _(static_cast<ActorType*>(this)->lineageAddr());
|
||||
#endif
|
||||
static_cast<ActorType*>(this)->a_callback_error(this, e);
|
||||
}
|
||||
};
|
||||
inline double now() {
|
||||
return g_network->now();
|
||||
|
|
|
@ -1253,6 +1253,12 @@ void tagAndForward(Promise<T>* pOutputPromise, T value, Future<Void> signal) {
|
|||
out.send(value);
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
void tagAndForward(PromiseStream<T>* pOutput, T value, Future<Void> signal) {
|
||||
wait(signal);
|
||||
pOutput->send(value);
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
void tagAndForwardError(Promise<T>* pOutputPromise, Error value, Future<Void> signal) {
|
||||
state Promise<T> out(std::move(*pOutputPromise));
|
||||
|
@ -1260,6 +1266,12 @@ void tagAndForwardError(Promise<T>* pOutputPromise, Error value, Future<Void> si
|
|||
out.sendError(value);
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
void tagAndForwardError(PromiseStream<T>* pOutput, Error value, Future<Void> signal) {
|
||||
wait(signal);
|
||||
pOutput->sendError(value);
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<T> waitOrError(Future<T> f, Future<Void> errorSignal) {
|
||||
choose {
|
||||
|
@ -1612,6 +1624,12 @@ struct YieldedFutureActor : SAV<Void>, ActorCallback<YieldedFutureActor, 1, Void
|
|||
|
||||
void destroy() override { delete this; }
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
LineageReference* lineageAddr() {
|
||||
return currentLineage;
|
||||
}
|
||||
#endif
|
||||
|
||||
void a_callback_fire(ActorCallback<YieldedFutureActor, 1, Void>*, Void) {
|
||||
if (int16_t(in_error_state.code()) == UNSET_ERROR_CODE) {
|
||||
in_error_state = Error::fromCode(SET_ERROR_CODE);
|
||||
|
|
|
@ -35,6 +35,7 @@
|
|||
#include "flow/Arena.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/WriteOnlySet.h"
|
||||
|
||||
enum class TaskPriority {
|
||||
Max = 1000000,
|
||||
|
@ -571,6 +572,11 @@ public:
|
|||
// returns false.
|
||||
virtual bool checkRunnable() = 0;
|
||||
|
||||
#ifdef ENABLE_SAMPLING
|
||||
// Returns the shared memory data structure used to store actor lineages.
|
||||
virtual ActorLineageSet& getActorLineageSet() = 0;
|
||||
#endif
|
||||
|
||||
virtual ProtocolVersion protocolVersion() = 0;
|
||||
|
||||
// Shorthand for transport().getLocalAddress()
|
||||
|
|
|
@ -0,0 +1,237 @@
|
|||
/*
|
||||
* (C) Copyright 2015 ETH Zurich Systems Group (http://www.systems.ethz.ch/) and others.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
* Contributors:
|
||||
* Markus Pilman <mpilman@inf.ethz.ch>
|
||||
* Simon Loesing <sloesing@inf.ethz.ch>
|
||||
* Thomas Etter <etterth@gmail.com>
|
||||
* Kevin Bocksrocker <kevin.bocksrocker@gmail.com>
|
||||
* Lucas Braun <braunl@inf.ethz.ch>
|
||||
*/
|
||||
#pragma once
|
||||
|
||||
#include <mutex>
|
||||
#include <memory>
|
||||
#include <cstdlib>
|
||||
#include <cassert>
|
||||
|
||||
namespace crossbow {
|
||||
|
||||
/**
|
||||
* @brief A mock mutex for disabling locking in the singleton
|
||||
*
|
||||
* This class implements the mutex concept with empty methods.
|
||||
* This can be used to disable synchronization in the singleton
|
||||
* holder.
|
||||
*/
|
||||
struct no_locking {
|
||||
void lock() {}
|
||||
void unlock() {}
|
||||
bool try_lock() { return true; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct create_static {
|
||||
static constexpr bool supports_recreation = false;
|
||||
union max_align {
|
||||
char t_[sizeof(T)];
|
||||
short int short_int_;
|
||||
long int long_int_;
|
||||
float float_;
|
||||
double double_;
|
||||
long double longDouble_;
|
||||
struct Test;
|
||||
int Test::*pMember_;
|
||||
int (Test::*pMemberFn_)(int);
|
||||
};
|
||||
|
||||
static T* create() {
|
||||
static max_align static_memory_;
|
||||
return new (&static_memory_) T;
|
||||
}
|
||||
|
||||
static void destroy(T* ptr) { ptr->~T(); }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct create_using_new {
|
||||
static constexpr bool supports_recreation = true;
|
||||
static T* create() { return new T; };
|
||||
|
||||
static void destroy(T* ptr) { delete ptr; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct create_using_malloc {
|
||||
static constexpr bool supports_recreation = true;
|
||||
static T* create() {
|
||||
void* p = std::malloc(sizeof(T));
|
||||
if (!p)
|
||||
return nullptr;
|
||||
return new (p) T;
|
||||
}
|
||||
|
||||
static void destroy(T* ptr) {
|
||||
ptr->~T();
|
||||
free(ptr);
|
||||
}
|
||||
};
|
||||
|
||||
template <class T, class allocator>
|
||||
struct create_using {
|
||||
static constexpr bool supports_recreation = true;
|
||||
static allocator alloc_;
|
||||
|
||||
static T* create() {
|
||||
T* p = alloc_.allocate(1);
|
||||
if (!p)
|
||||
return nullptr;
|
||||
alloc_.construct(p);
|
||||
return p;
|
||||
};
|
||||
|
||||
static void destroy(T* ptr) {
|
||||
alloc_.destroy(ptr);
|
||||
alloc_.deallocate(ptr, 1);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct default_lifetime {
|
||||
static void schedule_destruction(T*, void (*func)()) { std::atexit(func); }
|
||||
|
||||
static void on_dead_ref() { throw std::logic_error("Dead reference detected"); }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct phoenix_lifetime {
|
||||
static void schedule_destruction(T*, void (*func)()) { std::atexit(func); }
|
||||
|
||||
static void on_dead_ref() {}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct infinite_lifetime {
|
||||
static void schedule_destruction(T*, void (*)()) {}
|
||||
static void on_dead_ref() {}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct lifetime_traits {
|
||||
static constexpr bool supports_recreation = true;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct lifetime_traits<infinite_lifetime<T>> {
|
||||
static constexpr bool supports_recreation = false;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct lifetime_traits<default_lifetime<T>> {
|
||||
static constexpr bool supports_recreation = false;
|
||||
};
|
||||
|
||||
template <typename Type,
|
||||
typename Create = create_static<Type>,
|
||||
typename LifetimePolicy = default_lifetime<Type>,
|
||||
typename Mutex = std::mutex>
|
||||
class singleton {
|
||||
public:
|
||||
typedef Type value_type;
|
||||
typedef Type* pointer;
|
||||
typedef const Type* const_pointer;
|
||||
typedef const Type& const_reference;
|
||||
typedef Type& reference;
|
||||
|
||||
private:
|
||||
static bool destroyed_;
|
||||
static pointer instance_;
|
||||
static Mutex mutex_;
|
||||
|
||||
static void destroy() {
|
||||
if (destroyed_)
|
||||
return;
|
||||
Create::destroy(instance_);
|
||||
instance_ = nullptr;
|
||||
destroyed_ = true;
|
||||
}
|
||||
|
||||
public:
|
||||
static reference instance() {
|
||||
static_assert(Create::supports_recreation || !lifetime_traits<LifetimePolicy>::supports_recreation,
|
||||
"The creation policy does not support instance recreation, while the lifetime does support it.");
|
||||
if (!instance_) {
|
||||
std::lock_guard<Mutex> l(mutex_);
|
||||
if (!instance_) {
|
||||
if (destroyed_) {
|
||||
destroyed_ = false;
|
||||
LifetimePolicy::on_dead_ref();
|
||||
}
|
||||
instance_ = Create::create();
|
||||
LifetimePolicy::schedule_destruction(instance_, &destroy);
|
||||
}
|
||||
}
|
||||
return *instance_;
|
||||
}
|
||||
/**
|
||||
* WARNING: DO NOT EXECUTE THIS MULTITHREADED!!!
|
||||
*/
|
||||
static void destroy_instance() {
|
||||
if (instance_) {
|
||||
std::lock_guard<Mutex> l(mutex_);
|
||||
destroy();
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
pointer operator->() {
|
||||
if (!instance_) {
|
||||
instance();
|
||||
}
|
||||
return instance_;
|
||||
}
|
||||
|
||||
reference operator*() {
|
||||
if (!instance_) {
|
||||
instance();
|
||||
}
|
||||
return *instance_;
|
||||
}
|
||||
|
||||
const_pointer operator->() const {
|
||||
if (!instance_) {
|
||||
instance();
|
||||
}
|
||||
return instance_;
|
||||
}
|
||||
|
||||
const_reference operator*() const {
|
||||
if (!instance_) {
|
||||
instance();
|
||||
}
|
||||
return *instance_;
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T, typename C, typename L, typename M>
|
||||
bool singleton<T, C, L, M>::destroyed_ = false;
|
||||
|
||||
template <typename T, typename C, typename L, typename M>
|
||||
typename singleton<T, C, L, M>::pointer singleton<T, C, L, M>::instance_ = nullptr;
|
||||
|
||||
template <typename T, typename C, typename L, typename M>
|
||||
M singleton<T, C, L, M>::mutex_;
|
||||
|
||||
} // namespace crossbow
|
|
@ -0,0 +1,7 @@
|
|||
testTitle=UnitTests
|
||||
startDelay=0
|
||||
useDB=false
|
||||
|
||||
testName=UnitTests
|
||||
maxTestCases=0
|
||||
testsMatching=/flow/IThreadPool/
|
Loading…
Reference in New Issue