Merge remote-tracking branch 'origin/master' into paxos-config-db

This commit is contained in:
sfc-gh-tclinkenbeard 2021-08-19 16:18:59 -07:00
commit 658233d0e0
60 changed files with 1629 additions and 1377 deletions

View File

@ -77,19 +77,37 @@ add_dependencies(packages python_package)
if (NOT WIN32 AND NOT OPEN_FOR_IDE)
add_fdbclient_test(
NAME fdbcli_tests
NAME single_process_fdbcli_tests
COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py
${CMAKE_BINARY_DIR}/bin/fdbcli
${CMAKE_BINARY_DIR}
@CLUSTER_FILE@
1
)
add_fdbclient_test(
NAME multi_process_fdbcli_tests
PROCESS_NUMBER 5
TEST_TIMEOUT 120 # The test can take near to 1 minutes sometime, set timeout to 2 minutes to be safe
COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py
${CMAKE_BINARY_DIR}/bin/fdbcli
${CMAKE_BINARY_DIR}
@CLUSTER_FILE@
5
)
if (TARGET external_client) # external_client copies fdb_c to bindings/c/libfdb_c.so
add_fdbclient_test(
NAME single_process_external_client_fdbcli_tests
COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py
${CMAKE_BINARY_DIR}
@CLUSTER_FILE@
--external-client-library ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so
)
add_fdbclient_test(
NAME multi_process_external_client_fdbcli_tests
PROCESS_NUMBER 5
TEST_TIMEOUT 120 # The test can take near to 1 minutes sometime, set timeout to 2 minutes to be safe
COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py
${CMAKE_BINARY_DIR}
@CLUSTER_FILE@
5
--external-client-library ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so
)
endif()
endif()

View File

@ -1,14 +1,17 @@
#!/usr/bin/env python3
import sys
import os
import subprocess
import logging
import functools
import json
import time
import random
from argparse import ArgumentParser, RawDescriptionHelpFormatter
def enable_logging(level=logging.ERROR):
def enable_logging(level=logging.DEBUG):
"""Enable logging in the function with the specified logging level
Args:
@ -16,7 +19,7 @@ def enable_logging(level=logging.ERROR):
"""
def func_decorator(func):
@functools.wraps(func)
def wrapper(*args,**kwargs):
def wrapper(*args, **kwargs):
# initialize logger
logger = logging.getLogger(func.__name__)
logger.setLevel(level)
@ -32,6 +35,7 @@ def enable_logging(level=logging.ERROR):
return wrapper
return func_decorator
def run_fdbcli_command(*args):
"""run the fdbcli statement: fdbcli --exec '<arg1> <arg2> ... <argN>'.
@ -39,7 +43,8 @@ def run_fdbcli_command(*args):
string: Console output from fdbcli
"""
commands = command_template + ["{}".format(' '.join(args))]
return subprocess.run(commands, stdout=subprocess.PIPE).stdout.decode('utf-8').strip()
return subprocess.run(commands, stdout=subprocess.PIPE, env=fdbcli_env).stdout.decode('utf-8').strip()
def run_fdbcli_command_and_get_error(*args):
"""run the fdbcli statement: fdbcli --exec '<arg1> <arg2> ... <argN>'.
@ -48,7 +53,8 @@ def run_fdbcli_command_and_get_error(*args):
string: Stderr output from fdbcli
"""
commands = command_template + ["{}".format(' '.join(args))]
return subprocess.run(commands, stdout=subprocess.PIPE, stderr=subprocess.PIPE).stderr.decode('utf-8').strip()
return subprocess.run(commands, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=fdbcli_env).stderr.decode('utf-8').strip()
@enable_logging()
def advanceversion(logger):
@ -72,6 +78,7 @@ def advanceversion(logger):
logger.debug("Read version: {}".format(version4))
assert version4 >= version3
@enable_logging()
def maintenance(logger):
# expected fdbcli output when running 'maintenance' while there's no ongoing maintenance
@ -94,6 +101,7 @@ def maintenance(logger):
output3 = run_fdbcli_command('maintenance')
assert output3 == no_maintenance_output
@enable_logging()
def setclass(logger):
output1 = run_fdbcli_command('setclass')
@ -108,11 +116,11 @@ def setclass(logger):
# check class source
assert 'command_line' in class_type_line_1
# set class to a random valid type
class_types = ['storage', 'storage', 'transaction', 'resolution',
'commit_proxy', 'grv_proxy', 'master', 'stateless', 'log',
'router', 'cluster_controller', 'fast_restore', 'data_distributor',
'coordinator', 'ratekeeper', 'storage_cache', 'backup'
]
class_types = ['storage', 'storage', 'transaction', 'resolution',
'commit_proxy', 'grv_proxy', 'master', 'stateless', 'log',
'router', 'cluster_controller', 'fast_restore', 'data_distributor',
'coordinator', 'ratekeeper', 'storage_cache', 'backup'
]
random_class_type = random.choice(class_types)
logger.debug("Change to type: {}".format(random_class_type))
run_fdbcli_command('setclass', network_address, random_class_type)
@ -134,6 +142,7 @@ def setclass(logger):
logger.debug(class_type_line_3)
assert class_type_line_3 == class_type_line_1
@enable_logging()
def lockAndUnlock(logger):
# lock an unlocked database, should be successful
@ -148,7 +157,7 @@ def lockAndUnlock(logger):
output2 = run_fdbcli_command_and_get_error("lock")
assert output2 == 'ERROR: Database is locked (1038)'
# unlock the database
process = subprocess.Popen(command_template + ['unlock ' + lock_uid], stdin = subprocess.PIPE, stdout = subprocess.PIPE)
process = subprocess.Popen(command_template + ['unlock ' + lock_uid], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env)
line1 = process.stdout.readline()
# The randome passphrease we need to confirm to proceed the unlocking
line2 = process.stdout.readline()
@ -159,6 +168,7 @@ def lockAndUnlock(logger):
assert output3.decode('utf-8').strip() == 'Database unlocked.'
assert not get_value_from_status_json(True, 'cluster', 'database_lock_state', 'locked')
@enable_logging()
def kill(logger):
output1 = run_fdbcli_command('kill')
@ -168,11 +178,11 @@ def kill(logger):
address = lines[1]
logger.debug("Address: {}".format(address))
old_generation = get_value_from_status_json(False, 'cluster', 'generation')
# This is currently an issue with fdbcli,
# where you need to first run 'kill' to initialize processes' list
# This is currently an issue with fdbcli,
# where you need to first run 'kill' to initialize processes' list
# and then specify the certain process to kill
process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE)
#
process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env)
#
output2, err = process.communicate(input='kill; kill {}\n'.format(address).encode())
logger.debug(output2)
# wait for a second for the cluster recovery
@ -181,6 +191,7 @@ def kill(logger):
logger.debug("Old: {}, New: {}".format(old_generation, new_generation))
assert new_generation > old_generation
@enable_logging()
def suspend(logger):
output1 = run_fdbcli_command('suspend')
@ -200,7 +211,7 @@ def suspend(logger):
assert len(pinfo) == 1
pid = pinfo[0].split(' ')[0]
logger.debug("Pid: {}".format(pid))
process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE)
process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env)
# suspend the process for enough long time
output2, err = process.communicate(input='suspend; suspend 3600 {}\n'.format(address).encode())
# the cluster should be unavailable after the only process being suspended
@ -213,7 +224,7 @@ def suspend(logger):
kill_output = subprocess.check_output(['kill', pid]).decode().strip()
logger.debug("Kill result: {}".format(kill_output))
# The process should come back after a few time
duration = 0 # seconds we already wait
duration = 0 # seconds we already wait
while not get_value_from_status_json(False, 'client', 'database_status', 'available') and duration < 60:
logger.debug("Sleep for 1 second to wait cluster recovery")
time.sleep(1)
@ -221,6 +232,7 @@ def suspend(logger):
# at most after 60 seconds, the cluster should be available
assert get_value_from_status_json(False, 'client', 'database_status', 'available')
def get_value_from_status_json(retry, *args):
while True:
result = json.loads(run_fdbcli_command('status', 'json'))
@ -229,9 +241,10 @@ def get_value_from_status_json(retry, *args):
for arg in args:
assert arg in result
result = result[arg]
return result
@enable_logging()
def consistencycheck(logger):
consistency_check_on_output = 'ConsistencyCheck is on'
@ -245,6 +258,7 @@ def consistencycheck(logger):
output3 = run_fdbcli_command('consistencycheck')
assert output3 == consistency_check_on_output
@enable_logging()
def cache_range(logger):
# this command is currently experimental
@ -252,6 +266,7 @@ def cache_range(logger):
run_fdbcli_command('cache_range', 'set', 'a', 'b')
run_fdbcli_command('cache_range', 'clear', 'a', 'b')
@enable_logging()
def datadistribution(logger):
output1 = run_fdbcli_command('datadistribution', 'off')
@ -271,6 +286,7 @@ def datadistribution(logger):
assert output6 == 'Data distribution is enabled for rebalance.'
time.sleep(1)
@enable_logging()
def transaction(logger):
"""This test will cover the transaction related fdbcli commands.
@ -280,7 +296,7 @@ def transaction(logger):
"""
err1 = run_fdbcli_command_and_get_error('set', 'key', 'value')
assert err1 == 'ERROR: writemode must be enabled to set or clear keys in the database.'
process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE)
process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env)
transaction_flow = ['writemode on', 'begin', 'getversion', 'set key value', 'get key', 'commit']
output1, _ = process.communicate(input='\n'.join(transaction_flow).encode())
# split the output into lines
@ -299,13 +315,13 @@ def transaction(logger):
output2 = run_fdbcli_command('get', 'key')
assert output2 == "`key' is `value'"
# test rollback and read-your-write behavior
process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE)
process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env)
transaction_flow = [
'writemode on', 'begin', 'getrange a z',
'writemode on', 'begin', 'getrange a z',
'clear key', 'get key',
# 'option on READ_YOUR_WRITES_DISABLE', 'get key',
'rollback'
]
]
output3, _ = process.communicate(input='\n'.join(transaction_flow).encode())
lines = list(filter(len, output3.decode().split('\n')))[-5:]
# lines[0] == "Transaction started" and lines[1] == 'Range limited to 25 keys'
@ -316,13 +332,13 @@ def transaction(logger):
output4 = run_fdbcli_command('get', 'key')
assert output4 == "`key' is `value'"
# test read_your_write_disable option and clear the inserted key
process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE)
process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env)
transaction_flow = [
'writemode on', 'begin',
'option on READ_YOUR_WRITES_DISABLE',
'clear key', 'get key',
'commit'
]
]
output6, _ = process.communicate(input='\n'.join(transaction_flow).encode())
lines = list(filter(len, output6.decode().split('\n')))[-4:]
assert lines[1] == 'Option enabled for current transaction'
@ -332,15 +348,17 @@ def transaction(logger):
output7 = run_fdbcli_command('get', 'key')
assert output7 == "`key': not found"
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
assert len(addresses) == args.process_number
return addresses
@enable_logging(logging.DEBUG)
def coordinators(logger):
# we should only have one coordinator for now
@ -368,6 +386,7 @@ def coordinators(logger):
assert len(get_value_from_status_json(True, 'client', 'coordinators', 'coordinators')) == 1
wait_for_database_available(logger)
@enable_logging(logging.DEBUG)
def exclude(logger):
# get all processes' network addresses
@ -380,7 +399,7 @@ def exclude(logger):
# 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
# 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:
@ -417,6 +436,8 @@ def exclude(logger):
wait_for_database_available(logger)
# read the system key 'k', need to enable the option first
def read_system_key(k):
output = run_fdbcli_command('option', 'on', 'READ_SYSTEM_KEYS;', 'get', k)
if 'is' not in output:
@ -425,11 +446,14 @@ def read_system_key(k):
_, value = output.split(' is ')
return value
@enable_logging()
def throttle(logger):
# no throttled tags at the beginning
no_throttle_tags_output = 'There are no throttled tags'
assert run_fdbcli_command('throttle', 'list') == no_throttle_tags_output
output = run_fdbcli_command('throttle', 'list')
logger.debug(output)
assert output == no_throttle_tags_output
# test 'throttle enable auto'
run_fdbcli_command('throttle', 'enable', 'auto')
# verify the change is applied by reading the system key
@ -442,6 +466,7 @@ 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
@ -449,17 +474,38 @@ def wait_for_database_available(logger):
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>"
parser = ArgumentParser(formatter_class=RawDescriptionHelpFormatter,
description="""
The test calls fdbcli commands through fdbcli --exec "<command>" interactively using subprocess.
The outputs from fdbcli are returned and compared to predefined results.
Consequently, changing fdbcli outputs or breaking any commands will casue the test to fail.
Commands that are easy to test will run against a single process cluster.
For complex commands like exclude, they will run against a cluster with multiple(current set to 5) processes.
If external_client_library is given, we will disable the local client and use the external client to run fdbcli.
""")
parser.add_argument('build_dir', metavar='BUILD_DIRECTORY', help='FDB build directory')
parser.add_argument('cluster_file', metavar='CLUSTER_FILE', help='FDB cluster file')
parser.add_argument('process_number', nargs='?', metavar='PROCESS_NUMBER', help="Number of fdb processes", type=int, default=1)
parser.add_argument('--external-client-library', '-e', metavar='EXTERNAL_CLIENT_LIBRARY_PATH', help="External client library path")
args = parser.parse_args()
# keep current environment variables
fdbcli_env = os.environ.copy()
# set external client library if provided
if args.external_client_library:
# disable local client and use the external client library
fdbcli_env['FDB_NETWORK_OPTION_DISABLE_LOCAL_CLIENT'] = ''
fdbcli_env['FDB_NETWORK_OPTION_EXTERNAL_CLIENT_LIBRARY'] = args.external_client_library
# shell command template
command_template = [sys.argv[1], '-C', sys.argv[2], '--exec']
command_template = [args.build_dir + '/bin/fdbcli', '-C', args.cluster_file, '--exec']
# tests for fdbcli commands
# assertions will fail if fdbcli does not work as expected
process_number = int(sys.argv[3])
if process_number == 1:
if args.process_number == 1:
# TODO: disable for now, the change can cause the database unavailable
#advanceversion()
# advanceversion()
cache_range()
consistencycheck()
datadistribution()
@ -471,8 +517,6 @@ if __name__ == '__main__':
transaction()
throttle()
else:
assert process_number > 1, "Process number should be positive"
assert args.process_number > 1, "Process number should be positive"
coordinators()
exclude()

View File

@ -42,7 +42,7 @@ else()
set(WITH_TLS OFF)
endif()
if(WIN32)
message(STATUS "TLS is temporarilty disabled on macOS while libressl -> openssl transition happens")
message(STATUS "TLS is temporarilty disabled on Windows while libressl -> openssl transition happens")
set(WITH_TLS OFF)
endif()
endif()

View File

@ -63,7 +63,7 @@ Source IP:port 0 string The IP and port of the machine where the s
Trace ID 1 uint64 The 64-bit identifier of the trace. All spans in a trace share the same trace ID.
Span ID 2 uint64 The 64-bit identifier of the span. All spans have a unique identifier.
Start timestamp 3 double The timestamp when the operation represented by the span began.
End timestamp 4 double The timestamp when the operation represented by the span ended.
Duration 4 double The duration in seconds of the operation represented by the span.
Operation name 5 string The name of the operation the span represents.
Tags 6 map User defined tags, added manually to specify additional information.
Parent span IDs 7 vector (Optional) A list of span IDs representing parents of this span.

View File

@ -21,345 +21,16 @@
#include "fdbcli/fdbcli.actor.h"
#include "fdbclient/IClientApi.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/CommitTransaction.h"
#include "flow/Arena.h"
#include "flow/FastRef.h"
#include "flow/ThreadHelper.actor.h"
#include "flow/genericactors.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
namespace {
// Helper functions copied from TagThrottle.actor.cpp
// The only difference is transactions are changed to go through MultiversionTransaction,
// instead of the native Transaction(i.e., RYWTransaction)
ACTOR Future<bool> getValidAutoEnabled(Reference<ITransaction> tr) {
state bool result;
loop {
Optional<Value> value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey)));
if (!value.present()) {
tr->reset();
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
continue;
} else if (value.get() == LiteralStringRef("1")) {
result = true;
} else if (value.get() == LiteralStringRef("0")) {
result = false;
} else {
TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue").detail("Value", value.get());
tr->reset();
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
continue;
}
return result;
};
}
ACTOR Future<std::vector<TagThrottleInfo>> getThrottledTags(Reference<IDatabase> db,
int limit,
bool containsRecommend = false) {
state Reference<ITransaction> tr = db->createTransaction();
state bool reportAuto = containsRecommend;
loop {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
try {
if (!containsRecommend) {
wait(store(reportAuto, getValidAutoEnabled(tr)));
}
state ThreadFuture<RangeResult> f = tr->getRange(
reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit);
RangeResult throttles = wait(safeThreadFutureToFuture(f));
std::vector<TagThrottleInfo> results;
for (auto throttle : throttles) {
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key),
TagThrottleValue::fromValue(throttle.value)));
}
return results;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR Future<std::vector<TagThrottleInfo>> getRecommendedTags(Reference<IDatabase> db, int limit) {
state Reference<ITransaction> tr = db->createTransaction();
loop {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
try {
bool enableAuto = wait(getValidAutoEnabled(tr));
if (enableAuto) {
return std::vector<TagThrottleInfo>();
}
state ThreadFuture<RangeResult> f =
tr->getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit);
RangeResult throttles = wait(safeThreadFutureToFuture(f));
std::vector<TagThrottleInfo> results;
for (auto throttle : throttles) {
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key),
TagThrottleValue::fromValue(throttle.value)));
}
return results;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR Future<Void> updateThrottleCount(Reference<ITransaction> tr, int64_t delta) {
state ThreadFuture<Optional<Value>> countVal = tr->get(tagThrottleCountKey);
state ThreadFuture<Optional<Value>> limitVal = tr->get(tagThrottleLimitKey);
wait(success(safeThreadFutureToFuture(countVal)) && success(safeThreadFutureToFuture(limitVal)));
int64_t count = 0;
int64_t limit = 0;
if (countVal.get().present()) {
BinaryReader reader(countVal.get().get(), Unversioned());
reader >> count;
}
if (limitVal.get().present()) {
BinaryReader reader(limitVal.get().get(), Unversioned());
reader >> limit;
}
count += delta;
if (count > limit) {
throw too_many_tag_throttles();
}
BinaryWriter writer(Unversioned());
writer << count;
tr->set(tagThrottleCountKey, writer.toValue());
return Void();
}
void signalThrottleChange(Reference<ITransaction> tr) {
tr->atomicOp(
tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue);
}
ACTOR Future<Void> throttleTags(Reference<IDatabase> db,
TagSet tags,
double tpsRate,
double initialDuration,
TagThrottleType throttleType,
TransactionPriority priority,
Optional<double> expirationTime = Optional<double>(),
Optional<TagThrottledReason> reason = Optional<TagThrottledReason>()) {
state Reference<ITransaction> tr = db->createTransaction();
state Key key = TagThrottleKey(tags, throttleType, priority).toKey();
ASSERT(initialDuration > 0);
if (throttleType == TagThrottleType::MANUAL) {
reason = TagThrottledReason::MANUAL;
}
TagThrottleValue throttle(tpsRate,
expirationTime.present() ? expirationTime.get() : 0,
initialDuration,
reason.present() ? reason.get() : TagThrottledReason::UNSET);
BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason()));
wr << throttle;
state Value value = wr.toValue();
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
if (throttleType == TagThrottleType::MANUAL) {
Optional<Value> oldThrottle = wait(safeThreadFutureToFuture(tr->get(key)));
if (!oldThrottle.present()) {
wait(updateThrottleCount(tr, 1));
}
}
tr->set(key, value);
if (throttleType == TagThrottleType::MANUAL) {
signalThrottleChange(tr);
}
wait(safeThreadFutureToFuture(tr->commit()));
return Void();
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR Future<bool> unthrottleTags(Reference<IDatabase> db,
TagSet tags,
Optional<TagThrottleType> throttleType,
Optional<TransactionPriority> priority) {
state Reference<ITransaction> tr = db->createTransaction();
state std::vector<Key> keys;
for (auto p : allTransactionPriorities) {
if (!priority.present() || priority.get() == p) {
if (!throttleType.present() || throttleType.get() == TagThrottleType::AUTO) {
keys.push_back(TagThrottleKey(tags, TagThrottleType::AUTO, p).toKey());
}
if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) {
keys.push_back(TagThrottleKey(tags, TagThrottleType::MANUAL, p).toKey());
}
}
}
state bool removed = false;
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
state std::vector<Future<Optional<Value>>> values;
values.reserve(keys.size());
for (auto key : keys) {
values.push_back(safeThreadFutureToFuture(tr->get(key)));
}
wait(waitForAll(values));
int delta = 0;
for (int i = 0; i < values.size(); ++i) {
if (values[i].get().present()) {
if (TagThrottleKey::fromKey(keys[i]).throttleType == TagThrottleType::MANUAL) {
delta -= 1;
}
tr->clear(keys[i]);
// Report that we are removing this tag if we ever see it present.
// This protects us from getting confused if the transaction is maybe committed.
// It's ok if someone else actually ends up removing this tag at the same time
// and we aren't the ones to actually do it.
removed = true;
}
}
if (delta != 0) {
wait(updateThrottleCount(tr, delta));
}
if (removed) {
signalThrottleChange(tr);
wait(safeThreadFutureToFuture(tr->commit()));
}
return removed;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR Future<Void> enableAuto(Reference<IDatabase> db, bool enabled) {
state Reference<ITransaction> tr = db->createTransaction();
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
Optional<Value> value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey)));
if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) ||
(!enabled && value.get() != LiteralStringRef("0"))) {
tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0"));
signalThrottleChange(tr);
wait(safeThreadFutureToFuture(tr->commit()));
}
return Void();
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR Future<bool> unthrottleMatchingThrottles(Reference<IDatabase> db,
KeyRef beginKey,
KeyRef endKey,
Optional<TransactionPriority> priority,
bool onlyExpiredThrottles) {
state Reference<ITransaction> tr = db->createTransaction();
state KeySelector begin = firstGreaterOrEqual(beginKey);
state KeySelector end = firstGreaterOrEqual(endKey);
state bool removed = false;
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
// holds memory of the RangeResult
state ThreadFuture<RangeResult> f = tr->getRange(begin, end, 1000);
state RangeResult tags = wait(safeThreadFutureToFuture(f));
state uint64_t unthrottledTags = 0;
uint64_t manualUnthrottledTags = 0;
for (auto tag : tags) {
if (onlyExpiredThrottles) {
double expirationTime = TagThrottleValue::fromValue(tag.value).expirationTime;
if (expirationTime == 0 || expirationTime > now()) {
continue;
}
}
TagThrottleKey key = TagThrottleKey::fromKey(tag.key);
if (priority.present() && key.priority != priority.get()) {
continue;
}
if (key.throttleType == TagThrottleType::MANUAL) {
++manualUnthrottledTags;
}
removed = true;
tr->clear(tag.key);
unthrottledTags++;
}
if (manualUnthrottledTags > 0) {
wait(updateThrottleCount(tr, -manualUnthrottledTags));
}
if (unthrottledTags > 0) {
signalThrottleChange(tr);
}
wait(safeThreadFutureToFuture(tr->commit()));
if (!tags.more) {
return removed;
}
ASSERT(tags.size() > 0);
begin = KeySelector(firstGreaterThan(tags[tags.size() - 1].key), tags.arena());
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
Future<bool> unthrottleAll(Reference<IDatabase> db,
Optional<TagThrottleType> tagThrottleType,
Optional<TransactionPriority> priority) {
KeyRef begin = tagThrottleKeys.begin;
KeyRef end = tagThrottleKeys.end;
if (tagThrottleType.present() && tagThrottleType == TagThrottleType::AUTO) {
begin = tagThrottleAutoKeysPrefix;
} else if (tagThrottleType.present() && tagThrottleType == TagThrottleType::MANUAL) {
end = tagThrottleAutoKeysPrefix;
}
return unthrottleMatchingThrottles(db, begin, end, priority, false);
}
} // namespace
namespace fdb_cli {
ACTOR Future<bool> throttleCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens) {
@ -403,11 +74,11 @@ ACTOR Future<bool> throttleCommandActor(Reference<IDatabase> db, std::vector<Str
state std::vector<TagThrottleInfo> tags;
if (reportThrottled && reportRecommended) {
wait(store(tags, getThrottledTags(db, throttleListLimit, true)));
wait(store(tags, ThrottleApi::getThrottledTags(db, throttleListLimit, true)));
} else if (reportThrottled) {
wait(store(tags, getThrottledTags(db, throttleListLimit)));
wait(store(tags, ThrottleApi::getThrottledTags(db, throttleListLimit)));
} else if (reportRecommended) {
wait(store(tags, getRecommendedTags(db, throttleListLimit)));
wait(store(tags, ThrottleApi::getRecommendedTags(db, throttleListLimit)));
}
bool anyLogged = false;
@ -509,7 +180,7 @@ ACTOR Future<bool> throttleCommandActor(Reference<IDatabase> db, std::vector<Str
TagSet tags;
tags.addTag(tokens[3]);
wait(throttleTags(db, tags, tpsRate, duration, TagThrottleType::MANUAL, priority));
wait(ThrottleApi::throttleTags(db, tags, tpsRate, duration, TagThrottleType::MANUAL, priority));
printf("Tag `%s' has been throttled\n", tokens[3].toString().c_str());
} else if (tokencmp(tokens[1], "off")) {
int nextIndex = 2;
@ -586,7 +257,7 @@ ACTOR Future<bool> throttleCommandActor(Reference<IDatabase> db, std::vector<Str
priority.present() ? format(" at %s priority", transactionPriorityToString(priority.get(), false)) : "";
if (tags.size() > 0) {
bool success = wait(unthrottleTags(db, tags, throttleType, priority));
bool success = wait(ThrottleApi::unthrottleTags(db, tags, throttleType, priority));
if (success) {
printf("Unthrottled tag `%s'%s\n", tokens[3].toString().c_str(), priorityString.c_str());
} else {
@ -596,7 +267,7 @@ ACTOR Future<bool> throttleCommandActor(Reference<IDatabase> db, std::vector<Str
priorityString.c_str());
}
} else {
bool unthrottled = wait(unthrottleAll(db, throttleType, priority));
bool unthrottled = wait(ThrottleApi::unthrottleAll(db, throttleType, priority));
if (unthrottled) {
printf("Unthrottled all %sthrottled tags%s\n", throttleTypeString, priorityString.c_str());
} else {
@ -626,7 +297,7 @@ ACTOR Future<bool> throttleCommandActor(Reference<IDatabase> db, std::vector<Str
return false;
}
state bool autoTagThrottlingEnabled = tokencmp(tokens[1], "enable");
wait(enableAuto(db, autoTagThrottlingEnabled));
wait(ThrottleApi::enableAuto(db, autoTagThrottlingEnabled));
printf("Automatic tag throttling has been %s\n", autoTagThrottlingEnabled ? "enabled" : "disabled");
} else {
printUsage(tokens[0]);

View File

@ -36,7 +36,7 @@
#include "fdbclient/Schemas.h"
#include "fdbclient/CoordinationInterface.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbclient/Tuple.h"
#include "fdbclient/ThreadSafeTransaction.h"
@ -3195,10 +3195,10 @@ ACTOR template <class T>
Future<T> stopNetworkAfter(Future<T> what) {
try {
T t = wait(what);
g_network->stop();
API->stopNetwork();
return t;
} catch (...) {
g_network->stop();
API->stopNetwork();
throw;
}
}
@ -4685,7 +4685,7 @@ int main(int argc, char** argv) {
Future<int> cliFuture = runCli(opt);
Future<Void> timeoutFuture = opt.exit_timeout ? timeExit(opt.exit_timeout) : Never();
auto f = stopNetworkAfter(success(cliFuture) || timeoutFuture);
runNetwork();
API->runNetwork();
if (cliFuture.isReady()) {
return cliFuture.get();

View File

@ -113,7 +113,7 @@ set(FDBCLIENT_SRCS
SystemData.cpp
SystemData.h
TagThrottle.actor.cpp
TagThrottle.h
TagThrottle.actor.h
TaskBucket.actor.cpp
TaskBucket.h
TestKnobCollection.cpp

View File

@ -28,7 +28,7 @@
#include "fdbclient/FDBTypes.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbclient/CommitTransaction.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbclient/GlobalConfig.h"
#include "fdbrpc/Stats.h"

View File

@ -437,6 +437,10 @@ public:
// Requests to the storage server will no longer be duplicated to its pair TSS.
void removeTssMapping(StorageServerInterface const& ssi);
// used in template functions to create a transaction
using TransactionT = ReadYourWritesTransaction;
Reference<TransactionT> createTransaction();
private:
std::unordered_map<KeyRef, Reference<WatchMetadata>> watchMap;
};

View File

@ -41,8 +41,8 @@ typedef UID SpanID;
enum {
tagLocalitySpecial = -1, // tag with this locality means it is invalidTag (id=0), txsTag (id=1), or cacheTag (id=2)
tagLocalityLogRouter = -2,
tagLocalityRemoteLog = -3, // tag created by log router for remote tLogs
tagLocalityUpgraded = -4,
tagLocalityRemoteLog = -3, // tag created by log router for remote (aka. not in Primary DC) tLogs
tagLocalityUpgraded = -4, // tlogs with old log format
tagLocalitySatellite = -5,
tagLocalityLogRouterMapped = -6, // The pseudo tag used by log routers to pop the real LogRouter tag (i.e., -2)
tagLocalityTxs = -7,

View File

@ -88,6 +88,9 @@ public:
virtual void addref() = 0;
virtual void delref() = 0;
// used in template functions as returned Future type
template<class Type> using FutureT = ThreadFuture<Type>;
};
// An interface that represents a connection to a cluster made by a client
@ -115,6 +118,9 @@ public:
virtual ThreadFuture<Void> forceRecoveryWithDataLoss(const StringRef& dcid) = 0;
// Management API, create snapshot
virtual ThreadFuture<Void> createSnapshot(const StringRef& uid, const StringRef& snapshot_command) = 0;
// used in template functions as the Transaction type that can be created through createTransaction()
using TransactionT = ITransaction;
};
// An interface that presents the top-level FDB client API as exposed through the C bindings

View File

@ -373,7 +373,8 @@ void traceTSSErrors(const char* name, UID tssId, const std::unordered_map<int, u
ACTOR Future<Void> databaseLogger(DatabaseContext* cx) {
state double lastLogged = 0;
loop {
wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, TaskPriority::FlushTrace));
wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, TaskPriority::FlushTrace));
TraceEvent ev("TransactionMetrics", cx->dbId);
ev.detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged)
@ -384,6 +385,7 @@ ACTOR Future<Void> databaseLogger(DatabaseContext* cx) {
cx->cc.logToTraceEvent(ev);
ev.detail("LocationCacheEntryCount", cx->locationCache.size());
ev.detail("MeanLatency", cx->latencies.mean())
.detail("MedianLatency", cx->latencies.median())
.detail("Latency90", cx->latencies.percentile(0.90))
@ -6562,3 +6564,7 @@ ACTOR Future<Void> setPerpetualStorageWiggle(Database cx, bool enable, LockAware
}
return Void();
}
Reference<DatabaseContext::TransactionT> DatabaseContext::createTransaction() {
return makeReference<ReadYourWritesTransaction>(Database(Reference<DatabaseContext>::addRef(this)));
}

View File

@ -106,6 +106,7 @@ public:
inline DatabaseContext* getPtr() const { return db.getPtr(); }
inline DatabaseContext* extractPtr() { return db.extractPtr(); }
DatabaseContext* operator->() const { return db.getPtr(); }
Reference<DatabaseContext> getReference() const { return db; }
const UniqueOrderedOptionList<FDBTransactionOptions>& getTransactionDefaults() const;

View File

@ -22,8 +22,6 @@
#include "fdbclient/PaxosConfigTransaction.h"
#include "flow/actorcompiler.h" // must be last include
namespace {
// TODO: Some replicas may reply after quorum has already been achieved, and we may want to add them to the readReplicas
// list
class GetGenerationQuorum {
@ -72,8 +70,6 @@ public:
Optional<Version> getLastSeenLiveVersion() const { return lastSeenLiveVersion; }
};
} // namespace
class PaxosConfigTransactionImpl {
ConfigTransactionCommitRequest toCommit;
Future<GetGenerationQuorum::Result> getGenerationFuture;

View File

@ -175,6 +175,9 @@ public:
void setSpecialKeySpaceErrorMsg(const std::string& msg) { specialKeySpaceErrorMsg = msg; }
Transaction& getTransaction() { return tr; }
// used in template functions as returned Future type
template<typename Type> using FutureT = Future<Type>;
private:
friend class RYWImpl;

View File

@ -64,6 +64,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( TLOG_MESSAGE_BLOCK_BYTES, 10e6 );
init( TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR, double(TLOG_MESSAGE_BLOCK_BYTES) / (TLOG_MESSAGE_BLOCK_BYTES - MAX_MESSAGE_SIZE) ); //1.0121466709838096006362758832473
init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = deterministicRandom()->coinflip() ? 0.1 : 120;
init( PEEK_USING_STREAMING, true );
init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2;
init( MULTI_CURSOR_PRE_FETCH_LIMIT, 10 );
init( MAX_QUEUE_COMMIT_BYTES, 15e6 ); if( randomize && BUGGIFY ) MAX_QUEUE_COMMIT_BYTES = 5000;
@ -627,6 +628,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( FETCH_KEYS_TOO_LONG_TIME_CRITERIA, 300.0 );
init( MAX_STORAGE_COMMIT_TIME, 120.0 ); //The max fsync stall time on the storage server and tlog before marking a disk as failed
init( RANGESTREAM_LIMIT_BYTES, 2e6 ); if( randomize && BUGGIFY ) RANGESTREAM_LIMIT_BYTES = 1;
init( ENABLE_CLEAR_RANGE_EAGER_READS, true );
//Wait Failure
init( MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS, 250 ); if( randomize && BUGGIFY ) MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS = 2;

View File

@ -41,6 +41,7 @@ public:
// often, so that versions always advance smoothly
// TLogs
bool PEEK_USING_STREAMING;
double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time
double TLOG_SLOW_REJOIN_WARN_TIMEOUT_SECS; // Warns if a tlog takes too long to rejoin
double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification
@ -563,6 +564,7 @@ public:
double FETCH_KEYS_TOO_LONG_TIME_CRITERIA;
double MAX_STORAGE_COMMIT_TIME;
int64_t RANGESTREAM_LIMIT_BYTES;
bool ENABLE_CLEAR_RANGE_EAGER_READS;
// Wait Failure
int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS;

View File

@ -1964,16 +1964,29 @@ void parse(StringRef& val, WaitState& w) {
void parse(StringRef& val, time_t& t) {
struct tm tm = { 0 };
#ifdef _WIN32
std::istringstream s(val.toString());
s.imbue(std::locale(setlocale(LC_TIME, nullptr)));
s >> std::get_time(&tm, "%FT%T%z");
if (s.fail()) {
throw std::invalid_argument("failed to parse ISO 8601 datetime");
}
long timezone;
if (_get_timezone(&timezone) != 0) {
throw std::runtime_error("failed to convert ISO 8601 datetime");
}
timezone = -timezone;
#else
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;
#endif
}
void parse(StringRef& val, NetworkAddress& a) {

View File

@ -30,7 +30,7 @@
#include "fdbrpc/Stats.h"
#include "fdbrpc/TimedRequest.h"
#include "fdbrpc/TSSComparison.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "flow/UnitTest.h"
// Dead code, removed in the next protocol version

View File

@ -18,7 +18,7 @@
* limitations under the License.
*/
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/DatabaseContext.h"
@ -110,319 +110,3 @@ TagThrottleValue TagThrottleValue::fromValue(const ValueRef& value) {
reader >> throttleValue;
return throttleValue;
}
namespace ThrottleApi {
ACTOR Future<bool> getValidAutoEnabled(Transaction* tr, Database db) {
state bool result;
loop {
Optional<Value> value = wait(tr->get(tagThrottleAutoEnabledKey));
if (!value.present()) {
tr->reset();
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
continue;
} else if (value.get() == LiteralStringRef("1")) {
result = true;
} else if (value.get() == LiteralStringRef("0")) {
result = false;
} else {
TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue", db->dbId).detail("Value", value.get());
tr->reset();
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
continue;
}
return result;
};
}
void signalThrottleChange(Transaction& tr) {
tr.atomicOp(
tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue);
}
ACTOR Future<Void> updateThrottleCount(Transaction* tr, int64_t delta) {
state Future<Optional<Value>> countVal = tr->get(tagThrottleCountKey);
state Future<Optional<Value>> limitVal = tr->get(tagThrottleLimitKey);
wait(success(countVal) && success(limitVal));
int64_t count = 0;
int64_t limit = 0;
if (countVal.get().present()) {
BinaryReader reader(countVal.get().get(), Unversioned());
reader >> count;
}
if (limitVal.get().present()) {
BinaryReader reader(limitVal.get().get(), Unversioned());
reader >> limit;
}
count += delta;
if (count > limit) {
throw too_many_tag_throttles();
}
BinaryWriter writer(Unversioned());
writer << count;
tr->set(tagThrottleCountKey, writer.toValue());
return Void();
}
ACTOR Future<std::vector<TagThrottleInfo>> getThrottledTags(Database db, int limit, bool containsRecommend) {
state Transaction tr(db);
state bool reportAuto = containsRecommend;
loop {
try {
if (!containsRecommend) {
wait(store(reportAuto, getValidAutoEnabled(&tr, db)));
}
RangeResult throttles = wait(tr.getRange(
reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit));
std::vector<TagThrottleInfo> results;
for (auto throttle : throttles) {
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key),
TagThrottleValue::fromValue(throttle.value)));
}
return results;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<std::vector<TagThrottleInfo>> getRecommendedTags(Database db, int limit) {
state Transaction tr(db);
loop {
try {
bool enableAuto = wait(getValidAutoEnabled(&tr, db));
if (enableAuto) {
return std::vector<TagThrottleInfo>();
}
RangeResult throttles =
wait(tr.getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit));
std::vector<TagThrottleInfo> results;
for (auto throttle : throttles) {
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key),
TagThrottleValue::fromValue(throttle.value)));
}
return results;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<Void> throttleTags(Database db,
TagSet tags,
double tpsRate,
double initialDuration,
TagThrottleType throttleType,
TransactionPriority priority,
Optional<double> expirationTime,
Optional<TagThrottledReason> reason) {
state Transaction tr(db);
state Key key = TagThrottleKey(tags, throttleType, priority).toKey();
ASSERT(initialDuration > 0);
if (throttleType == TagThrottleType::MANUAL) {
reason = TagThrottledReason::MANUAL;
}
TagThrottleValue throttle(tpsRate,
expirationTime.present() ? expirationTime.get() : 0,
initialDuration,
reason.present() ? reason.get() : TagThrottledReason::UNSET);
BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason()));
wr << throttle;
state Value value = wr.toValue();
loop {
try {
if (throttleType == TagThrottleType::MANUAL) {
Optional<Value> oldThrottle = wait(tr.get(key));
if (!oldThrottle.present()) {
wait(updateThrottleCount(&tr, 1));
}
}
tr.set(key, value);
if (throttleType == TagThrottleType::MANUAL) {
signalThrottleChange(tr);
}
wait(tr.commit());
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<bool> unthrottleTags(Database db,
TagSet tags,
Optional<TagThrottleType> throttleType,
Optional<TransactionPriority> priority) {
state Transaction tr(db);
state std::vector<Key> keys;
for (auto p : allTransactionPriorities) {
if (!priority.present() || priority.get() == p) {
if (!throttleType.present() || throttleType.get() == TagThrottleType::AUTO) {
keys.push_back(TagThrottleKey(tags, TagThrottleType::AUTO, p).toKey());
}
if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) {
keys.push_back(TagThrottleKey(tags, TagThrottleType::MANUAL, p).toKey());
}
}
}
state bool removed = false;
loop {
try {
state std::vector<Future<Optional<Value>>> values;
values.reserve(keys.size());
for (auto key : keys) {
values.push_back(tr.get(key));
}
wait(waitForAll(values));
int delta = 0;
for (int i = 0; i < values.size(); ++i) {
if (values[i].get().present()) {
if (TagThrottleKey::fromKey(keys[i]).throttleType == TagThrottleType::MANUAL) {
delta -= 1;
}
tr.clear(keys[i]);
// Report that we are removing this tag if we ever see it present.
// This protects us from getting confused if the transaction is maybe committed.
// It's ok if someone else actually ends up removing this tag at the same time
// and we aren't the ones to actually do it.
removed = true;
}
}
if (delta != 0) {
wait(updateThrottleCount(&tr, delta));
}
if (removed) {
signalThrottleChange(tr);
wait(tr.commit());
}
return removed;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<bool> unthrottleMatchingThrottles(Database db,
KeyRef beginKey,
KeyRef endKey,
Optional<TransactionPriority> priority,
bool onlyExpiredThrottles) {
state Transaction tr(db);
state KeySelector begin = firstGreaterOrEqual(beginKey);
state KeySelector end = firstGreaterOrEqual(endKey);
state bool removed = false;
loop {
try {
state RangeResult tags = wait(tr.getRange(begin, end, 1000));
state uint64_t unthrottledTags = 0;
uint64_t manualUnthrottledTags = 0;
for (auto tag : tags) {
if (onlyExpiredThrottles) {
double expirationTime = TagThrottleValue::fromValue(tag.value).expirationTime;
if (expirationTime == 0 || expirationTime > now()) {
continue;
}
}
TagThrottleKey key = TagThrottleKey::fromKey(tag.key);
if (priority.present() && key.priority != priority.get()) {
continue;
}
if (key.throttleType == TagThrottleType::MANUAL) {
++manualUnthrottledTags;
}
removed = true;
tr.clear(tag.key);
unthrottledTags++;
}
if (manualUnthrottledTags > 0) {
wait(updateThrottleCount(&tr, -manualUnthrottledTags));
}
if (unthrottledTags > 0) {
signalThrottleChange(tr);
}
wait(tr.commit());
if (!tags.more) {
return removed;
}
ASSERT(tags.size() > 0);
begin = KeySelector(firstGreaterThan(tags[tags.size() - 1].key), tags.arena());
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
Future<bool> unthrottleAll(Database db,
Optional<TagThrottleType> tagThrottleType,
Optional<TransactionPriority> priority) {
KeyRef begin = tagThrottleKeys.begin;
KeyRef end = tagThrottleKeys.end;
if (tagThrottleType.present() && tagThrottleType == TagThrottleType::AUTO) {
begin = tagThrottleAutoKeysPrefix;
} else if (tagThrottleType.present() && tagThrottleType == TagThrottleType::MANUAL) {
end = tagThrottleAutoKeysPrefix;
}
return unthrottleMatchingThrottles(db, begin, end, priority, false);
}
Future<bool> expire(Database db) {
return unthrottleMatchingThrottles(
db, tagThrottleKeys.begin, tagThrottleKeys.end, Optional<TransactionPriority>(), true);
}
ACTOR Future<Void> enableAuto(Database db, bool enabled) {
state Transaction tr(db);
loop {
try {
Optional<Value> value = wait(tr.get(tagThrottleAutoEnabledKey));
if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) ||
(!enabled && value.get() != LiteralStringRef("0"))) {
tr.set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0"));
signalThrottleChange(tr);
wait(tr.commit());
}
return Void();
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
} // namespace ThrottleApi

View File

@ -0,0 +1,592 @@
/*
* TagThrottle.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.
*/
#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_TAG_THROTTLE_ACTOR_G_H)
#define FDBCLIENT_TAG_THROTTLE_ACTOR_G_H
#include "fdbclient/TagThrottle.actor.g.h"
#elif !defined(FDBCLIENT_TAG_THROTTLE_ACTOR_H)
#define FDBCLIENT_TAG_THROTTLE_ACTOR_H
#pragma once
#include "flow/Error.h"
#include "flow/flow.h"
#include "flow/network.h"
#include "flow/ThreadHelper.actor.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/CommitTransaction.h"
#include "flow/actorcompiler.h" // This must be the last #include.
typedef StringRef TransactionTagRef;
typedef Standalone<TransactionTagRef> TransactionTag;
class TagSet {
public:
typedef std::vector<TransactionTagRef>::const_iterator const_iterator;
TagSet() : bytes(0) {}
void addTag(TransactionTagRef tag);
size_t size() const;
const_iterator begin() const { return tags.begin(); }
const_iterator end() const { return tags.end(); }
void clear() {
tags.clear();
bytes = 0;
}
template <class Context>
void save(uint8_t* out, Context& c) const {
uint8_t* start = out;
for (const auto& tag : *this) {
*(out++) = (uint8_t)tag.size();
std::copy(tag.begin(), tag.end(), out);
out += tag.size();
}
ASSERT((size_t)(out - start) == size() + bytes);
}
template <class Context>
void load(const uint8_t* data, size_t size, Context& context) {
// const uint8_t *start = data;
const uint8_t* end = data + size;
while (data < end) {
uint8_t len = *(data++);
// Tags are already deduplicated
const auto& tag = tags.emplace_back(context.tryReadZeroCopy(data, len), len);
data += len;
bytes += tag.size();
}
ASSERT(data == end);
// Deserialized tag sets share the arena with the request that contained them
// For this reason, persisting a TagSet that shares memory with other request
// members should be done with caution.
arena = context.arena();
}
size_t getBytes() const { return bytes; }
const Arena& getArena() const { return arena; }
private:
size_t bytes;
Arena arena;
// Currently there are never >= 256 tags, so
// std::vector is faster than std::set. This may
// change if we allow more tags in the future.
std::vector<TransactionTagRef> tags;
};
template <>
struct dynamic_size_traits<TagSet> : std::true_type {
// May be called multiple times during one serialization
template <class Context>
static size_t size(const TagSet& t, Context&) {
return t.size() + t.getBytes();
}
// Guaranteed to be called only once during serialization
template <class Context>
static void save(uint8_t* out, const TagSet& t, Context& c) {
t.save(out, c);
}
// Context is an arbitrary type that is plumbed by reference throughout the
// load call tree.
template <class Context>
static void load(const uint8_t* data, size_t size, TagSet& t, Context& context) {
t.load(data, size, context);
}
};
enum class TagThrottleType : uint8_t { MANUAL, AUTO };
enum class TagThrottledReason : uint8_t { UNSET = 0, MANUAL, BUSY_READ, BUSY_WRITE };
struct TagThrottleKey {
TagSet tags;
TagThrottleType throttleType;
TransactionPriority priority;
TagThrottleKey() : throttleType(TagThrottleType::MANUAL), priority(TransactionPriority::DEFAULT) {}
TagThrottleKey(TagSet tags, TagThrottleType throttleType, TransactionPriority priority)
: tags(tags), throttleType(throttleType), priority(priority) {}
Key toKey() const;
static TagThrottleKey fromKey(const KeyRef& key);
};
struct TagThrottleValue {
double tpsRate;
double expirationTime;
double initialDuration;
TagThrottledReason reason;
TagThrottleValue() : tpsRate(0), expirationTime(0), initialDuration(0), reason(TagThrottledReason::UNSET) {}
TagThrottleValue(double tpsRate, double expirationTime, double initialDuration, TagThrottledReason reason)
: tpsRate(tpsRate), expirationTime(expirationTime), initialDuration(initialDuration), reason(reason) {}
static TagThrottleValue fromValue(const ValueRef& value);
// To change this serialization, ProtocolVersion::TagThrottleValue must be updated, and downgrades need to be
// considered
template <class Ar>
void serialize(Ar& ar) {
if (ar.protocolVersion().hasTagThrottleValueReason()) {
serializer(ar, tpsRate, expirationTime, initialDuration, reason);
} else if (ar.protocolVersion().hasTagThrottleValue()) {
serializer(ar, tpsRate, expirationTime, initialDuration);
if (ar.isDeserializing) {
reason = TagThrottledReason::UNSET;
}
}
}
};
struct TagThrottleInfo {
TransactionTag tag;
TagThrottleType throttleType;
TransactionPriority priority;
double tpsRate;
double expirationTime;
double initialDuration;
TagThrottledReason reason;
TagThrottleInfo(TransactionTag tag,
TagThrottleType throttleType,
TransactionPriority priority,
double tpsRate,
double expirationTime,
double initialDuration,
TagThrottledReason reason = TagThrottledReason::UNSET)
: tag(tag), throttleType(throttleType), priority(priority), tpsRate(tpsRate), expirationTime(expirationTime),
initialDuration(initialDuration), reason(reason) {}
TagThrottleInfo(TagThrottleKey key, TagThrottleValue value)
: throttleType(key.throttleType), priority(key.priority), tpsRate(value.tpsRate),
expirationTime(value.expirationTime), initialDuration(value.initialDuration), reason(value.reason) {
ASSERT(key.tags.size() == 1); // Multiple tags per throttle is not currently supported
tag = *key.tags.begin();
}
};
struct ClientTagThrottleLimits {
double tpsRate;
double expiration;
ClientTagThrottleLimits() : tpsRate(0), expiration(0) {}
ClientTagThrottleLimits(double tpsRate, double expiration) : tpsRate(tpsRate), expiration(expiration) {}
template <class Archive>
void serialize(Archive& ar) {
// Convert expiration time to a duration to avoid clock differences
double duration = 0;
if (!ar.isDeserializing) {
duration = expiration - now();
}
serializer(ar, tpsRate, duration);
if (ar.isDeserializing) {
expiration = now() + duration;
}
}
};
struct ClientTrCommitCostEstimation {
int opsCount = 0;
uint64_t writeCosts = 0;
std::deque<std::pair<int, uint64_t>> clearIdxCosts;
uint32_t expensiveCostEstCount = 0;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, opsCount, writeCosts, clearIdxCosts, expensiveCostEstCount);
}
};
// Keys to view and control tag throttling
extern const KeyRangeRef tagThrottleKeys;
extern const KeyRef tagThrottleKeysPrefix;
extern const KeyRef tagThrottleAutoKeysPrefix;
extern const KeyRef tagThrottleSignalKey;
extern const KeyRef tagThrottleAutoEnabledKey;
extern const KeyRef tagThrottleLimitKey;
extern const KeyRef tagThrottleCountKey;
namespace ThrottleApi {
// The template functions can be called with Native API like DatabaseContext, Transaction/ReadYourWritesTransaction
// or using IClientAPI like IDatabase, ITransaction
ACTOR template <class Tr>
Future<bool> getValidAutoEnabled(Reference<Tr> tr) {
state bool result;
loop {
Optional<Value> value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey)));
if (!value.present()) {
tr->reset();
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
continue;
} else if (value.get() == LiteralStringRef("1")) {
result = true;
} else if (value.get() == LiteralStringRef("0")) {
result = false;
} else {
TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue").detail("Value", value.get());
tr->reset();
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
continue;
}
return result;
};
}
ACTOR template <class DB>
Future<std::vector<TagThrottleInfo>> getRecommendedTags(Reference<DB> db, int limit) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
loop {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
try {
bool enableAuto = wait(getValidAutoEnabled(tr));
if (enableAuto) {
return std::vector<TagThrottleInfo>();
}
state typename DB::TransactionT::template FutureT<RangeResult> f =
tr->getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit);
RangeResult throttles = wait(safeThreadFutureToFuture(f));
std::vector<TagThrottleInfo> results;
for (auto throttle : throttles) {
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key),
TagThrottleValue::fromValue(throttle.value)));
}
return results;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR template <class DB>
Future<std::vector<TagThrottleInfo>> getThrottledTags(Reference<DB> db, int limit, bool containsRecommend = false) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
state bool reportAuto = containsRecommend;
loop {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
try {
if (!containsRecommend) {
wait(store(reportAuto, getValidAutoEnabled(tr)));
}
state typename DB::TransactionT::template FutureT<RangeResult> f = tr->getRange(
reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit);
RangeResult throttles = wait(safeThreadFutureToFuture(f));
std::vector<TagThrottleInfo> results;
for (auto throttle : throttles) {
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key),
TagThrottleValue::fromValue(throttle.value)));
}
return results;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
template <class Tr>
void signalThrottleChange(Reference<Tr> tr) {
tr->atomicOp(
tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue);
}
ACTOR template <class Tr>
Future<Void> updateThrottleCount(Reference<Tr> tr, int64_t delta) {
state typename Tr::template FutureT<Optional<Value>> countVal = tr->get(tagThrottleCountKey);
state typename Tr::template FutureT<Optional<Value>> limitVal = tr->get(tagThrottleLimitKey);
wait(success(safeThreadFutureToFuture(countVal)) && success(safeThreadFutureToFuture(limitVal)));
int64_t count = 0;
int64_t limit = 0;
if (countVal.get().present()) {
BinaryReader reader(countVal.get().get(), Unversioned());
reader >> count;
}
if (limitVal.get().present()) {
BinaryReader reader(limitVal.get().get(), Unversioned());
reader >> limit;
}
count += delta;
if (count > limit) {
throw too_many_tag_throttles();
}
BinaryWriter writer(Unversioned());
writer << count;
tr->set(tagThrottleCountKey, writer.toValue());
return Void();
}
ACTOR template <class DB>
Future<bool> unthrottleMatchingThrottles(Reference<DB> db,
KeyRef beginKey,
KeyRef endKey,
Optional<TransactionPriority> priority,
bool onlyExpiredThrottles) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
state KeySelector begin = firstGreaterOrEqual(beginKey);
state KeySelector end = firstGreaterOrEqual(endKey);
state bool removed = false;
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
// holds memory of the RangeResult
state typename DB::TransactionT::template FutureT<RangeResult> f = tr->getRange(begin, end, 1000);
state RangeResult tags = wait(safeThreadFutureToFuture(f));
state uint64_t unthrottledTags = 0;
uint64_t manualUnthrottledTags = 0;
for (auto tag : tags) {
if (onlyExpiredThrottles) {
double expirationTime = TagThrottleValue::fromValue(tag.value).expirationTime;
if (expirationTime == 0 || expirationTime > now()) {
continue;
}
}
TagThrottleKey key = TagThrottleKey::fromKey(tag.key);
if (priority.present() && key.priority != priority.get()) {
continue;
}
if (key.throttleType == TagThrottleType::MANUAL) {
++manualUnthrottledTags;
}
removed = true;
tr->clear(tag.key);
unthrottledTags++;
}
if (manualUnthrottledTags > 0) {
wait(updateThrottleCount(tr, -manualUnthrottledTags));
}
if (unthrottledTags > 0) {
signalThrottleChange(tr);
}
wait(safeThreadFutureToFuture(tr->commit()));
if (!tags.more) {
return removed;
}
ASSERT(tags.size() > 0);
begin = KeySelector(firstGreaterThan(tags[tags.size() - 1].key), tags.arena());
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
template <class DB>
Future<bool> expire(DB db) {
return unthrottleMatchingThrottles(
db, tagThrottleKeys.begin, tagThrottleKeys.end, Optional<TransactionPriority>(), true);
}
template <class DB>
Future<bool> unthrottleAll(Reference<DB> db,
Optional<TagThrottleType> tagThrottleType,
Optional<TransactionPriority> priority) {
KeyRef begin = tagThrottleKeys.begin;
KeyRef end = tagThrottleKeys.end;
if (tagThrottleType.present() && tagThrottleType == TagThrottleType::AUTO) {
begin = tagThrottleAutoKeysPrefix;
} else if (tagThrottleType.present() && tagThrottleType == TagThrottleType::MANUAL) {
end = tagThrottleAutoKeysPrefix;
}
return unthrottleMatchingThrottles(db, begin, end, priority, false);
}
ACTOR template <class DB>
Future<bool> unthrottleTags(Reference<DB> db,
TagSet tags,
Optional<TagThrottleType> throttleType,
Optional<TransactionPriority> priority) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
state std::vector<Key> keys;
for (auto p : allTransactionPriorities) {
if (!priority.present() || priority.get() == p) {
if (!throttleType.present() || throttleType.get() == TagThrottleType::AUTO) {
keys.push_back(TagThrottleKey(tags, TagThrottleType::AUTO, p).toKey());
}
if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) {
keys.push_back(TagThrottleKey(tags, TagThrottleType::MANUAL, p).toKey());
}
}
}
state bool removed = false;
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
state std::vector<Future<Optional<Value>>> values;
values.reserve(keys.size());
for (auto key : keys) {
values.push_back(safeThreadFutureToFuture(tr->get(key)));
}
wait(waitForAll(values));
int delta = 0;
for (int i = 0; i < values.size(); ++i) {
if (values[i].get().present()) {
if (TagThrottleKey::fromKey(keys[i]).throttleType == TagThrottleType::MANUAL) {
delta -= 1;
}
tr->clear(keys[i]);
// Report that we are removing this tag if we ever see it present.
// This protects us from getting confused if the transaction is maybe committed.
// It's ok if someone else actually ends up removing this tag at the same time
// and we aren't the ones to actually do it.
removed = true;
}
}
if (delta != 0) {
wait(updateThrottleCount(tr, delta));
}
if (removed) {
signalThrottleChange(tr);
wait(safeThreadFutureToFuture(tr->commit()));
}
return removed;
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR template <class DB>
Future<Void> throttleTags(Reference<DB> db,
TagSet tags,
double tpsRate,
double initialDuration,
TagThrottleType throttleType,
TransactionPriority priority,
Optional<double> expirationTime = Optional<double>(),
Optional<TagThrottledReason> reason = Optional<TagThrottledReason>()) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
state Key key = TagThrottleKey(tags, throttleType, priority).toKey();
ASSERT(initialDuration > 0);
if (throttleType == TagThrottleType::MANUAL) {
reason = TagThrottledReason::MANUAL;
}
TagThrottleValue throttle(tpsRate,
expirationTime.present() ? expirationTime.get() : 0,
initialDuration,
reason.present() ? reason.get() : TagThrottledReason::UNSET);
BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason()));
wr << throttle;
state Value value = wr.toValue();
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
if (throttleType == TagThrottleType::MANUAL) {
Optional<Value> oldThrottle = wait(safeThreadFutureToFuture(tr->get(key)));
if (!oldThrottle.present()) {
wait(updateThrottleCount(tr, 1));
}
}
tr->set(key, value);
if (throttleType == TagThrottleType::MANUAL) {
signalThrottleChange(tr);
}
wait(safeThreadFutureToFuture(tr->commit()));
return Void();
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
ACTOR template <class DB>
Future<Void> enableAuto(Reference<DB> db, bool enabled) {
state Reference<typename DB::TransactionT> tr = db->createTransaction();
loop {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
Optional<Value> value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey)));
if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) ||
(!enabled && value.get() != LiteralStringRef("0"))) {
tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0"));
signalThrottleChange<typename DB::TransactionT>(tr);
wait(safeThreadFutureToFuture(tr->commit()));
}
return Void();
} catch (Error& e) {
wait(safeThreadFutureToFuture(tr->onError(e)));
}
}
}
}; // namespace ThrottleApi
template <class Value>
using TransactionTagMap = std::unordered_map<TransactionTag, Value, std::hash<TransactionTagRef>>;
template <class Value>
using PrioritizedTransactionTagMap = std::map<TransactionPriority, TransactionTagMap<Value>>;
template <class Value>
using UIDTransactionTagMap = std::unordered_map<UID, TransactionTagMap<Value>>;
#include "flow/unactorcompiler.h"
#endif

View File

@ -1,265 +0,0 @@
/*
* TagThrottle.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.
*/
#ifndef FDBCLIENT_TAG_THROTTLE_H
#define FDBCLIENT_TAG_THROTTLE_H
#pragma once
#include "flow/Error.h"
#include "flow/flow.h"
#include "flow/network.h"
#include "fdbclient/FDBTypes.h"
#include <set>
class Database;
namespace ThrottleApi {}
typedef StringRef TransactionTagRef;
typedef Standalone<TransactionTagRef> TransactionTag;
class TagSet {
public:
typedef std::vector<TransactionTagRef>::const_iterator const_iterator;
TagSet() : bytes(0) {}
void addTag(TransactionTagRef tag);
size_t size() const;
const_iterator begin() const { return tags.begin(); }
const_iterator end() const { return tags.end(); }
void clear() {
tags.clear();
bytes = 0;
}
template <class Context>
void save(uint8_t* out, Context& c) const {
uint8_t* start = out;
for (const auto& tag : *this) {
*(out++) = (uint8_t)tag.size();
std::copy(tag.begin(), tag.end(), out);
out += tag.size();
}
ASSERT((size_t)(out - start) == size() + bytes);
}
template <class Context>
void load(const uint8_t* data, size_t size, Context& context) {
// const uint8_t *start = data;
const uint8_t* end = data + size;
while (data < end) {
uint8_t len = *(data++);
// Tags are already deduplicated
const auto& tag = tags.emplace_back(context.tryReadZeroCopy(data, len), len);
data += len;
bytes += tag.size();
}
ASSERT(data == end);
// Deserialized tag sets share the arena with the request that contained them
// For this reason, persisting a TagSet that shares memory with other request
// members should be done with caution.
arena = context.arena();
}
size_t getBytes() const { return bytes; }
const Arena& getArena() const { return arena; }
private:
size_t bytes;
Arena arena;
// Currently there are never >= 256 tags, so
// std::vector is faster than std::set. This may
// change if we allow more tags in the future.
std::vector<TransactionTagRef> tags;
};
template <>
struct dynamic_size_traits<TagSet> : std::true_type {
// May be called multiple times during one serialization
template <class Context>
static size_t size(const TagSet& t, Context&) {
return t.size() + t.getBytes();
}
// Guaranteed to be called only once during serialization
template <class Context>
static void save(uint8_t* out, const TagSet& t, Context& c) {
t.save(out, c);
}
// Context is an arbitrary type that is plumbed by reference throughout the
// load call tree.
template <class Context>
static void load(const uint8_t* data, size_t size, TagSet& t, Context& context) {
t.load(data, size, context);
}
};
enum class TagThrottleType : uint8_t { MANUAL, AUTO };
enum class TagThrottledReason : uint8_t { UNSET = 0, MANUAL, BUSY_READ, BUSY_WRITE };
struct TagThrottleKey {
TagSet tags;
TagThrottleType throttleType;
TransactionPriority priority;
TagThrottleKey() : throttleType(TagThrottleType::MANUAL), priority(TransactionPriority::DEFAULT) {}
TagThrottleKey(TagSet tags, TagThrottleType throttleType, TransactionPriority priority)
: tags(tags), throttleType(throttleType), priority(priority) {}
Key toKey() const;
static TagThrottleKey fromKey(const KeyRef& key);
};
struct TagThrottleValue {
double tpsRate;
double expirationTime;
double initialDuration;
TagThrottledReason reason;
TagThrottleValue() : tpsRate(0), expirationTime(0), initialDuration(0), reason(TagThrottledReason::UNSET) {}
TagThrottleValue(double tpsRate, double expirationTime, double initialDuration, TagThrottledReason reason)
: tpsRate(tpsRate), expirationTime(expirationTime), initialDuration(initialDuration), reason(reason) {}
static TagThrottleValue fromValue(const ValueRef& value);
// To change this serialization, ProtocolVersion::TagThrottleValue must be updated, and downgrades need to be
// considered
template <class Ar>
void serialize(Ar& ar) {
if (ar.protocolVersion().hasTagThrottleValueReason()) {
serializer(ar, tpsRate, expirationTime, initialDuration, reason);
} else if (ar.protocolVersion().hasTagThrottleValue()) {
serializer(ar, tpsRate, expirationTime, initialDuration);
if (ar.isDeserializing) {
reason = TagThrottledReason::UNSET;
}
}
}
};
struct TagThrottleInfo {
TransactionTag tag;
TagThrottleType throttleType;
TransactionPriority priority;
double tpsRate;
double expirationTime;
double initialDuration;
TagThrottledReason reason;
TagThrottleInfo(TransactionTag tag,
TagThrottleType throttleType,
TransactionPriority priority,
double tpsRate,
double expirationTime,
double initialDuration,
TagThrottledReason reason = TagThrottledReason::UNSET)
: tag(tag), throttleType(throttleType), priority(priority), tpsRate(tpsRate), expirationTime(expirationTime),
initialDuration(initialDuration), reason(reason) {}
TagThrottleInfo(TagThrottleKey key, TagThrottleValue value)
: throttleType(key.throttleType), priority(key.priority), tpsRate(value.tpsRate),
expirationTime(value.expirationTime), initialDuration(value.initialDuration), reason(value.reason) {
ASSERT(key.tags.size() == 1); // Multiple tags per throttle is not currently supported
tag = *key.tags.begin();
}
};
struct ClientTagThrottleLimits {
double tpsRate;
double expiration;
ClientTagThrottleLimits() : tpsRate(0), expiration(0) {}
ClientTagThrottleLimits(double tpsRate, double expiration) : tpsRate(tpsRate), expiration(expiration) {}
template <class Archive>
void serialize(Archive& ar) {
// Convert expiration time to a duration to avoid clock differences
double duration = 0;
if (!ar.isDeserializing) {
duration = expiration - now();
}
serializer(ar, tpsRate, duration);
if (ar.isDeserializing) {
expiration = now() + duration;
}
}
};
struct ClientTrCommitCostEstimation {
int opsCount = 0;
uint64_t writeCosts = 0;
std::deque<std::pair<int, uint64_t>> clearIdxCosts;
uint32_t expensiveCostEstCount = 0;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, opsCount, writeCosts, clearIdxCosts, expensiveCostEstCount);
}
};
namespace ThrottleApi {
Future<std::vector<TagThrottleInfo>> getThrottledTags(Database const& db,
int const& limit,
bool const& containsRecommend = false);
Future<std::vector<TagThrottleInfo>> getRecommendedTags(Database const& db, int const& limit);
Future<Void> throttleTags(Database const& db,
TagSet const& tags,
double const& tpsRate,
double const& initialDuration,
TagThrottleType const& throttleType,
TransactionPriority const& priority,
Optional<double> const& expirationTime = Optional<double>(),
Optional<TagThrottledReason> const& reason = Optional<TagThrottledReason>());
Future<bool> unthrottleTags(Database const& db,
TagSet const& tags,
Optional<TagThrottleType> const& throttleType,
Optional<TransactionPriority> const& priority);
Future<bool> unthrottleAll(Database db, Optional<TagThrottleType> throttleType, Optional<TransactionPriority> priority);
Future<bool> expire(Database db);
Future<Void> enableAuto(Database const& db, bool const& enabled);
}; // namespace ThrottleApi
template <class Value>
using TransactionTagMap = std::unordered_map<TransactionTag, Value, std::hash<TransactionTagRef>>;
template <class Value>
using PrioritizedTransactionTagMap = std::map<TransactionPriority, TransactionTagMap<Value>>;
template <class Value>
using UIDTransactionTagMap = std::unordered_map<UID, TransactionTagMap<Value>>;
#endif

View File

@ -158,7 +158,10 @@ const Endpoint& EndpointMap::insert(NetworkAddressList localAddresses,
NetworkMessageReceiver* EndpointMap::get(Endpoint::Token const& token) {
uint32_t index = token.second();
if (index < wellKnownEndpointCount && data[index].receiver == nullptr) {
TraceEvent(SevWarnAlways, "WellKnownEndpointNotAdded").detail("Token", token).detail("Index", index).backtrace();
TraceEvent(SevWarnAlways, "WellKnownEndpointNotAdded")
.detail("Token", token)
.detail("Index", index)
.backtrace();
}
if (index < data.size() && data[index].token().first() == token.first() &&
((data[index].token().second() & 0xffffffff00000000LL) | index) == token.second())
@ -923,6 +926,7 @@ ACTOR static void deliver(TransportData* self,
// ReadSocket) we can just upgrade. Otherwise we'll context switch so that we don't block other tasks that might run
// with a higher priority. ReplyPromiseStream needs to guarentee that messages are recieved in the order they were
// sent, so we are using orderedDelay.
// NOTE: don't skip delay(0) when it's local deliver since it could cause out of order object deconstruction.
if (priority < TaskPriority::ReadSocket || !inReadSocket) {
wait(orderedDelay(0, priority));
} else {

View File

@ -300,7 +300,7 @@ struct AcknowledgementReceiver final : FlowReceiver, FastAllocated<Acknowledgeme
Promise<Void> hold = ready;
hold.sendError(message.getError());
} else {
ASSERT(message.get().bytes > bytesAcknowledged);
ASSERT(message.get().bytes > bytesAcknowledged || (message.get().bytes < 0 && bytesAcknowledged > 0));
bytesAcknowledged = message.get().bytes;
if (ready.isValid() && bytesSent - bytesAcknowledged < bytesLimit) {
Promise<Void> hold = ready;
@ -393,7 +393,8 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue<T>,
false);
}
if (isRemoteEndpoint() && !sentError && !acknowledgements.failures.isReady()) {
// The ReplyPromiseStream was cancelled before sending an error, so the storage server must have died
// Notify the client ReplyPromiseStream was cancelled before sending an error, so the storage server must
// have died
FlowTransport::transport().sendUnreliable(SerializeSource<ErrorOr<EnsureTable<T>>>(broken_promise()),
getEndpoint(TaskPriority::ReadSocket),
false);
@ -413,6 +414,7 @@ public:
void send(U&& value) const {
if (queue->isRemoteEndpoint()) {
if (!queue->acknowledgements.getRawEndpoint().isValid()) {
// register acknowledge receiver on sender and tell the receiver where to send acknowledge messages
value.acknowledgeToken = queue->acknowledgements.getEndpoint(TaskPriority::ReadSocket).token;
}
queue->acknowledgements.bytesSent += value.expectedSize();
@ -474,6 +476,8 @@ public:
errors->delPromiseRef();
}
// The endpoints of a ReplyPromiseStream must be initialized at Task::ReadSocket, because with lower priorities
// a delay(0) in FlowTransport deliver can cause out of order delivery.
const Endpoint& getEndpoint() const { return queue->getEndpoint(TaskPriority::ReadSocket); }
bool operator==(const ReplyPromiseStream<T>& rhs) const { return queue == rhs.queue; }

View File

@ -197,7 +197,7 @@ struct PeerHolder {
}
};
// Implements getRepyStream, this a void actor with the same lifetime as the input ReplyPromiseStream.
// Implements getReplyStream, this a void actor with the same lifetime as the input ReplyPromiseStream.
// Because this actor holds a reference to the stream, normally it would be impossible to know when there are no other
// references. To get around this, there is a SAV inside the stream that has one less promise reference than it should
// (caused by getErrorFutureAndDelPromiseRef()). When that SAV gets a broken promise because no one besides this void

View File

@ -1900,7 +1900,7 @@ public:
KillType ktResult, ktMin = kt;
for (auto& datacenterMachine : datacenterMachines) {
if (deterministicRandom()->random01() < 0.99) {
if (deterministicRandom()->random01() < 0.99 || forceKill) {
killMachine(datacenterMachine.first, kt, true, &ktResult);
if (ktResult != kt) {
TraceEvent(SevWarn, "KillDCFail")

View File

@ -3938,13 +3938,11 @@ ACTOR Future<Void> timeKeeper(ClusterControllerData* self) {
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(self->cx);
loop {
try {
state UID debugID = deterministicRandom()->randomUniqueID();
if (!g_network->isSimulated()) {
// This is done to provide an arbitrary logged transaction every ~10s.
// FIXME: replace or augment this with logging on the proxy which tracks
// how long it is taking to hear responses from each other component.
UID debugID = deterministicRandom()->randomUniqueID();
TraceEvent("TimeKeeperCommit", debugID).log();
// how long it is taking to hear responses from each other component.
tr->debugTransaction(debugID);
}
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
@ -3959,7 +3957,9 @@ ACTOR Future<Void> timeKeeper(ClusterControllerData* self) {
Version v = tr->getReadVersion().get();
int64_t currentTime = (int64_t)now();
versionMap.set(tr, currentTime, v);
if (!g_network->isSimulated()) {
TraceEvent("TimeKeeperCommit", debugID).detail("Version", v);
}
int64_t ttl = currentTime - SERVER_KNOBS->TIME_KEEPER_DELAY * SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES;
if (ttl > 0) {
versionMap.erase(tr, 0, ttl);
@ -4858,18 +4858,18 @@ ACTOR Future<Void> clusterControllerCore(ClusterControllerFullInterface interf,
++self.getWorkersRequests;
vector<WorkerDetails> workers;
for (auto& it : self.id_worker) {
for (auto const& [id, worker] : self.id_worker) {
if ((req.flags & GetWorkersRequest::NON_EXCLUDED_PROCESSES_ONLY) &&
self.db.config.isExcludedServer(it.second.details.interf.addresses())) {
self.db.config.isExcludedServer(worker.details.interf.addresses())) {
continue;
}
if ((req.flags & GetWorkersRequest::TESTER_CLASS_ONLY) &&
it.second.details.processClass.classType() != ProcessClass::TesterClass) {
worker.details.processClass.classType() != ProcessClass::TesterClass) {
continue;
}
workers.push_back(it.second.details);
workers.push_back(worker.details);
}
req.reply.send(workers);

View File

@ -123,11 +123,11 @@ public:
class ReadFromLocalConfigEnvironment {
UID id;
std::string dataDir;
LocalConfiguration localConfiguration;
Reference<LocalConfiguration> localConfiguration;
Reference<AsyncVar<ConfigBroadcastInterface> const> cbi;
Future<Void> consumer;
ACTOR static Future<Void> checkEventually(LocalConfiguration const* localConfiguration,
ACTOR static Future<Void> checkEventually(Reference<LocalConfiguration const> localConfiguration,
Optional<int64_t> expected) {
state double lastMismatchTime = now();
loop {
@ -145,7 +145,7 @@ class ReadFromLocalConfigEnvironment {
}
ACTOR static Future<Void> setup(ReadFromLocalConfigEnvironment* self) {
wait(self->localConfiguration.initialize());
wait(self->localConfiguration->initialize());
if (self->cbi) {
// LocalConfiguration runs in a loop waiting for messages from the
// broadcaster. These unit tests use the same
@ -155,7 +155,7 @@ class ReadFromLocalConfigEnvironment {
// 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());
self->consumer = self->localConfiguration->consume(self->cbi->get());
}
return Void();
}
@ -164,40 +164,43 @@ public:
ReadFromLocalConfigEnvironment(std::string const& dataDir,
std::string const& configPath,
std::map<std::string, std::string> const& manualKnobOverrides)
: dataDir(dataDir), localConfiguration(dataDir, configPath, manualKnobOverrides, IsTest::True),
: dataDir(dataDir),
localConfiguration(makeReference<LocalConfiguration>(dataDir, configPath, manualKnobOverrides, IsTest::True)),
consumer(Never()) {}
Future<Void> setup() { return setup(this); }
Future<Void> restartLocalConfig(std::string const& newConfigPath) {
localConfiguration = LocalConfiguration(dataDir, newConfigPath, {}, IsTest::True);
std::map<std::string, std::string> manualKnobOverrides = {};
localConfiguration =
makeReference<LocalConfiguration>(dataDir, newConfigPath, manualKnobOverrides, IsTest::True);
return setup();
}
void connectToBroadcaster(Reference<AsyncVar<ConfigBroadcastInterface> const> const& cbi) {
this->cbi = cbi;
consumer = localConfiguration.consume(cbi->get());
consumer = localConfiguration->consume(cbi->get());
}
void checkImmediate(Optional<int64_t> expected) const {
if (expected.present()) {
ASSERT_EQ(localConfiguration.getTestKnobs().TEST_LONG, expected.get());
ASSERT_EQ(localConfiguration->getTestKnobs().TEST_LONG, expected.get());
} else {
ASSERT_EQ(localConfiguration.getTestKnobs().TEST_LONG, 0);
ASSERT_EQ(localConfiguration->getTestKnobs().TEST_LONG, 0);
}
}
Future<Void> checkEventually(Optional<int64_t> expected) const {
return checkEventually(&localConfiguration, expected);
return checkEventually(localConfiguration, expected);
}
LocalConfiguration& getMutableLocalConfiguration() { return localConfiguration; }
LocalConfiguration& getMutableLocalConfiguration() { return *localConfiguration; }
Future<Void> getError() const { return consumer; }
Version lastSeenVersion() { return localConfiguration.lastSeenVersion(); }
Version lastSeenVersion() { return localConfiguration->lastSeenVersion(); }
ConfigClassSet configClassSet() { return localConfiguration.configClassSet(); }
ConfigClassSet configClassSet() { return localConfiguration->configClassSet(); }
};
class LocalConfigEnvironment {

View File

@ -858,7 +858,7 @@ ACTOR Future<Void> fetchShardMetrics(DataDistributionTracker* self, GetMetricsRe
when(wait(delay(SERVER_KNOBS->DD_SHARD_METRICS_TIMEOUT, TaskPriority::DataDistribution))) {
TEST(true); // DD_SHARD_METRICS_TIMEOUT
StorageMetrics largeMetrics;
largeMetrics.bytes = SERVER_KNOBS->MAX_SHARD_BYTES;
largeMetrics.bytes = getMaxShardSize(self->dbSizeEstimate->get());
req.reply.send(largeMetrics);
}
}

View File

@ -88,6 +88,7 @@ rocksdb::Options getOptions() {
}
options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(bbOpts));
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
return options;
}

View File

@ -427,10 +427,6 @@ LocalConfiguration::LocalConfiguration(std::string const& dataFolder,
IsTest isTest)
: impl(PImpl<LocalConfigurationImpl>::create(dataFolder, configPath, manualKnobOverrides, isTest)) {}
LocalConfiguration::LocalConfiguration(LocalConfiguration&&) = default;
LocalConfiguration& LocalConfiguration::operator=(LocalConfiguration&&) = default;
LocalConfiguration::~LocalConfiguration() = default;
FlowKnobs const& LocalConfiguration::getFlowKnobs() const {

View File

@ -43,7 +43,7 @@ FDB_DECLARE_BOOLEAN_PARAM(IsTest);
* - Register with the broadcaster to receive new updates for the relevant configuration classes
* - Persist these updates when received, and restart if necessary
*/
class LocalConfiguration {
class LocalConfiguration : public ReferenceCounted<LocalConfiguration> {
PImpl<class LocalConfigurationImpl> impl;
public:
@ -51,8 +51,6 @@ public:
std::string const& configPath,
std::map<std::string, std::string> const& manualKnobOverrides,
IsTest = IsTest::False);
LocalConfiguration(LocalConfiguration&&);
LocalConfiguration& operator=(LocalConfiguration&&);
~LocalConfiguration();
FlowKnobs const& getFlowKnobs() const;
ClientKnobs const& getClientKnobs() const;

View File

@ -117,6 +117,7 @@ struct LogRouterData {
getMoreBlockedCount; // Increase by 1 if data is not available when LR tries to pull data from satellite tLog.
Future<Void> logger;
Reference<EventCacheHolder> eventCacheHolder;
int activePeekStreams = 0;
std::vector<Reference<TagData>> tag_data; // we only store data for the remote tag locality
@ -193,6 +194,7 @@ struct LogRouterData {
return int64_t(1000 * val);
});
specialCounter(cc, "Generation", [this]() { return this->generation; });
specialCounter(cc, "ActivePeekStreams", [this]() { return this->activePeekStreams; });
logger = traceCounters("LogRouterMetrics",
dbgid,
SERVER_KNOBS->WORKER_LOGGING_INTERVAL,
@ -404,18 +406,15 @@ std::deque<std::pair<Version, LengthPrefixedStringRef>>& get_version_messages(Lo
return tagData->version_messages;
};
void peekMessagesFromMemory(LogRouterData* self,
TLogPeekRequest const& req,
BinaryWriter& messages,
Version& endVersion) {
void peekMessagesFromMemory(LogRouterData* self, Tag tag, Version begin, BinaryWriter& messages, Version& endVersion) {
ASSERT(!messages.getLength());
auto& deque = get_version_messages(self, req.tag);
auto& deque = get_version_messages(self, tag);
//TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size());
auto it = std::lower_bound(deque.begin(),
deque.end(),
std::make_pair(req.begin, LengthPrefixedStringRef()),
std::make_pair(begin, LengthPrefixedStringRef()),
CompareFirst<std::pair<Version, LengthPrefixedStringRef>>());
Version currentVersion = -1;
@ -442,22 +441,30 @@ Version poppedVersion(LogRouterData* self, Tag tag) {
return tagData->popped;
}
ACTOR Future<Void> logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) {
// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request
ACTOR template <typename PromiseType>
Future<Void> logRouterPeekMessages(PromiseType replyPromise,
LogRouterData* self,
Version reqBegin,
Tag reqTag,
bool reqReturnIfBlocked = false,
bool reqOnlySpilled = false,
Optional<std::pair<UID, int>> reqSequence = Optional<std::pair<UID, int>>()) {
state BinaryWriter messages(Unversioned());
state int sequence = -1;
state UID peekId;
if (req.sequence.present()) {
if (reqSequence.present()) {
try {
peekId = req.sequence.get().first;
sequence = req.sequence.get().second;
peekId = reqSequence.get().first;
sequence = reqSequence.get().second;
if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS &&
self->peekTracker.find(peekId) == self->peekTracker.end()) {
throw operation_obsolete();
}
auto& trackerData = self->peekTracker[peekId];
if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) {
trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled));
trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled));
}
auto seqBegin = trackerData.sequence_version.begin();
// The peek cursor and this comparison need to agree about the maximum number of in-flight requests.
@ -476,12 +483,12 @@ ACTOR Future<Void> logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re
trackerData.lastUpdate = now();
std::pair<Version, bool> prevPeekData = wait(trackerData.sequence_version[sequence].getFuture());
req.begin = prevPeekData.first;
req.onlySpilled = prevPeekData.second;
reqBegin = prevPeekData.first;
reqOnlySpilled = prevPeekData.second;
wait(yield());
} catch (Error& e) {
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
replyPromise.sendError(e);
return Void();
} else {
throw;
@ -489,62 +496,62 @@ ACTOR Future<Void> logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re
}
}
//TraceEvent("LogRouterPeek1", self->dbgid).detail("From", req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin);
if (req.returnIfBlocked && self->version.get() < req.begin) {
//TraceEvent("LogRouterPeek1", self->dbgid).detail("From", replyPromise.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", reqBegin);
if (reqReturnIfBlocked && self->version.get() < reqBegin) {
//TraceEvent("LogRouterPeek2", self->dbgid);
req.reply.sendError(end_of_stream());
if (req.sequence.present()) {
replyPromise.sendError(end_of_stream());
if (reqSequence.present()) {
auto& trackerData = self->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (!sequenceData.isSet()) {
sequenceData.send(std::make_pair(req.begin, req.onlySpilled));
sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled));
}
}
return Void();
}
if (self->version.get() < req.begin) {
wait(self->version.whenAtLeast(req.begin));
if (self->version.get() < reqBegin) {
wait(self->version.whenAtLeast(reqBegin));
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
}
Version poppedVer = poppedVersion(self, req.tag);
Version poppedVer = poppedVersion(self, reqTag);
if (poppedVer > req.begin || req.begin < self->startVersion) {
if (poppedVer > reqBegin || reqBegin < self->startVersion) {
// This should only happen if a packet is sent multiple times and the reply is not needed.
// Since we are using popped differently, do not send a reply.
TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid)
.detail("Begin", req.begin)
.detail("Begin", reqBegin)
.detail("Popped", poppedVer)
.detail("Start", self->startVersion);
req.reply.send(Never());
if (req.sequence.present()) {
replyPromise.send(Never());
if (reqSequence.present()) {
auto& trackerData = self->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (!sequenceData.isSet()) {
sequenceData.send(std::make_pair(req.begin, req.onlySpilled));
sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled));
}
}
return Void();
}
Version endVersion = self->version.get() + 1;
peekMessagesFromMemory(self, req, messages, endVersion);
peekMessagesFromMemory(self, reqTag, reqBegin, messages, endVersion);
TLogPeekReply reply;
reply.maxKnownVersion = self->version.get();
reply.minKnownCommittedVersion = self->poppedVersion;
reply.messages = messages.toValue();
reply.messages = StringRef(reply.arena, messages.toValue());
reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0;
reply.end = endVersion;
reply.onlySpilled = false;
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = self->peekTracker[peekId];
trackerData.lastUpdate = now();
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -552,20 +559,60 @@ ACTOR Future<Void> logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re
if (sequenceData.isSet()) {
if (sequenceData.getFuture().get().first != reply.end) {
TEST(true); // tlog peek second attempt ended at a different version
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(reply.end, reply.onlySpilled));
}
reply.begin = req.begin;
reply.begin = reqBegin;
}
req.reply.send(reply);
replyPromise.send(reply);
//TraceEvent("LogRouterPeek4", self->dbgid);
return Void();
}
// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover
ACTOR Future<Void> logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) {
self->activePeekStreams++;
state Version begin = req.begin;
state bool onlySpilled = false;
req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes));
loop {
state TLogPeekStreamReply reply;
state Promise<TLogPeekReply> promise;
state Future<TLogPeekReply> future(promise.getFuture());
try {
wait(req.reply.onReady() && store(reply.rep, future) &&
logRouterPeekMessages(promise, self, begin, req.tag, req.returnIfBlocked, onlySpilled));
reply.rep.begin = begin;
req.reply.send(reply);
begin = reply.rep.end;
onlySpilled = reply.rep.onlySpilled;
if (reply.rep.end > self->version.get()) {
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
} else {
wait(delay(0, g_network->getCurrentTask()));
}
} catch (Error& e) {
self->activePeekStreams--;
TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid)
.detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress())
.error(e, true);
if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
return Void();
} else {
throw;
}
}
}
}
ACTOR Future<Void> cleanupPeekTrackers(LogRouterData* self) {
loop {
double minTimeUntilExpiration = SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME;
@ -643,7 +690,13 @@ ACTOR Future<Void> logRouterCore(TLogInterface interf,
logRouterData.logSystem->set(ILogSystem::fromServerDBInfo(logRouterData.dbgid, db->get(), true));
}
when(TLogPeekRequest req = waitNext(interf.peekMessages.getFuture())) {
addActor.send(logRouterPeekMessages(&logRouterData, req));
addActor.send(logRouterPeekMessages(
req.reply, &logRouterData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence));
}
when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) {
TraceEvent(SevDebug, "LogRouterPeekStream", logRouterData.dbgid)
.detail("Token", interf.peekStreamMessages.getEndpoint().token);
addActor.send(logRouterPeekStream(&logRouterData, req));
}
when(TLogPopRequest req = waitNext(interf.popMessages.getFuture())) {
// Request from remote tLog to pop data from LR

View File

@ -427,7 +427,7 @@ struct ILogSystem {
TLogPeekReply results;
ArenaReader rd;
LogMessageVersion messageVersion, end;
LogMessageVersion messageVersion, end; // the version of current message; the intended end version of current cursor
Version poppedVersion;
TagsAndMessage messageAndTags;
bool hasMsg;
@ -437,9 +437,11 @@ struct ILogSystem {
bool onlySpilled;
bool parallelGetMore;
bool usePeekStream;
int sequence;
Deque<Future<TLogPeekReply>> futureResults;
Future<Void> interfaceChanged;
Optional<ReplyPromiseStream<TLogPeekStreamReply>> peekReplyStream;
double lastReset;
Future<Void> resetCheck;

View File

@ -25,6 +25,24 @@
#include "fdbrpc/ReplicationUtils.h"
#include "flow/actorcompiler.h" // has to be last include
// create a peek stream for cursor when it's possible
ACTOR Future<Void> tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) {
if (self->peekReplyStream.present())
return Void();
else if (!self->interf || !self->interf->get().present()) {
self->peekReplyStream.reset();
return Never();
}
wait(IFailureMonitor::failureMonitor().onStateEqual(self->interf->get().interf().peekStreamMessages.getEndpoint(),
FailureStatus(false)));
self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest(
self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits<int>::max()));
TraceEvent(SevDebug, "SPC_StreamCreated", self->randomID)
.detail("PeerAddr", self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress())
.detail("PeerToken", self->interf->get().interf().peekStreamMessages.getEndpoint().token);
return Void();
}
ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference<AsyncVar<OptionalInterface<TLogInterface>>> const& interf,
Tag tag,
Version begin,
@ -33,11 +51,15 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference<AsyncVar<OptionalInterf
bool parallelGetMore)
: interf(interf), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(begin), end(end),
poppedVersion(0), hasMsg(false), randomID(deterministicRandom()->randomUniqueID()),
returnIfBlocked(returnIfBlocked), onlySpilled(false), parallelGetMore(parallelGetMore), sequence(0), lastReset(0),
resetCheck(Void()), slowReplies(0), fastReplies(0), unknownReplies(0) {
returnIfBlocked(returnIfBlocked), onlySpilled(false), parallelGetMore(parallelGetMore),
usePeekStream(SERVER_KNOBS->PEEK_USING_STREAMING), sequence(0), lastReset(0), resetCheck(Void()), slowReplies(0),
fastReplies(0), unknownReplies(0) {
this->results.maxKnownVersion = 0;
this->results.minKnownCommittedVersion = 0;
//TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace();
DisabledTraceEvent(SevDebug, "SPC_Starting", randomID)
.detail("Tag", tag.toString())
.detail("Begin", begin)
.detail("End", end);
}
ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results,
@ -50,8 +72,8 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results,
: tag(tag), results(results), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion),
end(end), poppedVersion(poppedVersion), messageAndTags(message), hasMsg(hasMsg),
randomID(deterministicRandom()->randomUniqueID()), returnIfBlocked(false), onlySpilled(false),
parallelGetMore(false), sequence(0), lastReset(0), resetCheck(Void()), slowReplies(0), fastReplies(0),
unknownReplies(0) {
parallelGetMore(false), usePeekStream(false), sequence(0), lastReset(0), resetCheck(Void()), slowReplies(0),
fastReplies(0), unknownReplies(0) {
//TraceEvent("SPC_Clone", randomID);
this->results.maxKnownVersion = 0;
this->results.minKnownCommittedVersion = 0;
@ -153,6 +175,20 @@ void ILogSystem::ServerPeekCursor::advanceTo(LogMessageVersion n) {
}
}
// This function is called after the cursor received one TLogPeekReply to update its members, which is the common logic
// in getMore helper functions.
void updateCursorWithReply(ILogSystem::ServerPeekCursor* self, const TLogPeekReply& res) {
self->results = res;
self->onlySpilled = res.onlySpilled;
if (res.popped.present())
self->poppedVersion = std::min(std::max(self->poppedVersion, res.popped.get()), self->end.version);
self->rd = ArenaReader(self->results.arena, self->results.messages, Unversioned());
LogMessageVersion skipSeq = self->messageVersion;
self->hasMsg = true;
self->nextMessage();
self->advanceTo(skipSeq);
}
ACTOR Future<Void> resetChecker(ILogSystem::ServerPeekCursor* self, NetworkAddress addr) {
self->slowReplies = 0;
self->unknownReplies = 0;
@ -208,11 +244,10 @@ ACTOR Future<TLogPeekReply> recordRequestMetrics(ILogSystem::ServerPeekCursor* s
}
ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) {
if (!self->interf || self->messageVersion >= self->end) {
if (!self->interf || self->isExhausted()) {
if (self->hasMessage())
return Void();
wait(Future<Void>(Never()));
throw internal_error();
return Never();
}
if (!self->interfaceChanged.isValid()) {
@ -253,16 +288,7 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
}
expectedBegin = res.end;
self->futureResults.pop_front();
self->results = res;
self->onlySpilled = res.onlySpilled;
if (res.popped.present())
self->poppedVersion =
std::min(std::max(self->poppedVersion, res.popped.get()), self->end.version);
self->rd = ArenaReader(self->results.arena, self->results.messages, Unversioned());
LogMessageVersion skipSeq = self->messageVersion;
self->hasMsg = true;
self->nextMessage();
self->advanceTo(skipSeq);
updateCursorWithReply(self, res);
//TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0);
return Void();
}
@ -296,10 +322,70 @@ ACTOR Future<Void> serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self,
}
}
ACTOR Future<Void> serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) {
if (!self->interf || self->isExhausted()) {
self->peekReplyStream.reset();
if (self->hasMessage())
return Void();
return Never();
}
loop {
try {
state Version expectedBegin = self->messageVersion.version;
state Future<TLogPeekReply> fPeekReply = self->peekReplyStream.present()
? map(waitAndForward(self->peekReplyStream.get().getFuture()),
[](const TLogPeekStreamReply& r) { return r.rep; })
: Never();
choose {
when(wait(self->peekReplyStream.present() ? Never() : tryEstablishPeekStream(self))) {}
when(wait(self->interf->onChange())) {
self->onlySpilled = false;
self->peekReplyStream.reset();
}
when(TLogPeekReply res = wait(
self->peekReplyStream.present()
? recordRequestMetrics(
self,
self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress(),
fPeekReply)
: Never())) {
if (res.begin.get() != expectedBegin) {
throw operation_obsolete();
}
updateCursorWithReply(self, res);
expectedBegin = res.end;
DisabledTraceEvent(SevDebug, "SPC_GetMoreB", self->randomID)
.detail("Has", self->hasMessage())
.detail("End", res.end)
.detail("Popped", res.popped.present() ? res.popped.get() : 0);
// NOTE: delay is necessary here since ReplyPromiseStream delivers reply on high priority. Here we
// change the priority to the intended one.
wait(delay(0, taskID));
return Void();
}
}
} catch (Error& e) {
DisabledTraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).error(e, true);
if (e.code() == error_code_connection_failed || e.code() == error_code_operation_obsolete) {
// NOTE: delay in order to avoid the endless retry loop block other tasks
self->peekReplyStream.reset();
wait(delay(0));
} else if (e.code() == error_code_end_of_stream) {
self->peekReplyStream.reset();
self->end.reset(self->messageVersion.version);
return Void();
} else {
throw;
}
}
}
}
ACTOR Future<Void> serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) {
if (!self->interf || self->messageVersion >= self->end) {
wait(Future<Void>(Never()));
throw internal_error();
if (!self->interf || self->isExhausted()) {
return Never();
}
try {
loop {
@ -313,16 +399,7 @@ ACTOR Future<Void> serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri
self->onlySpilled),
taskID))
: Never())) {
self->results = res;
self->onlySpilled = res.onlySpilled;
if (res.popped.present())
self->poppedVersion =
std::min(std::max(self->poppedVersion, res.popped.get()), self->end.version);
self->rd = ArenaReader(self->results.arena, self->results.messages, Unversioned());
LogMessageVersion skipSeq = self->messageVersion;
self->hasMsg = true;
self->nextMessage();
self->advanceTo(skipSeq);
updateCursorWithReply(self, res);
//TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0);
return Void();
}
@ -339,11 +416,17 @@ ACTOR Future<Void> serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri
}
Future<Void> ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) {
//TraceEvent("SPC_GetMore", randomID).detail("HasMessage", hasMessage()).detail("More", !more.isValid() || more.isReady()).detail("MessageVersion", messageVersion.toString()).detail("End", end.toString());
// TraceEvent("SPC_GetMore", randomID)
// .detail("HasMessage", hasMessage())
// .detail("More", !more.isValid() || more.isReady())
// .detail("MessageVersion", messageVersion.toString())
// .detail("End", end.toString());
if (hasMessage() && !parallelGetMore)
return Void();
if (!more.isValid() || more.isReady()) {
if (parallelGetMore || onlySpilled || futureResults.size()) {
if (usePeekStream && (tag.locality >= 0 || tag.locality == tagLocalityLogRouter || tag.locality == tagLocalityRemoteLog)) {
more = serverPeekStreamGetMore(this, taskID);
} else if (parallelGetMore || onlySpilled || futureResults.size()) {
more = serverPeekParallelGetMore(this, taskID);
} else {
more = serverPeekGetMore(this, taskID);
@ -361,6 +444,12 @@ ACTOR Future<Void> serverPeekOnFailed(ILogSystem::ServerPeekCursor* self) {
: Never())) {
return Void();
}
when(wait(self->interf->get().present()
? IFailureMonitor::failureMonitor().onStateEqual(
self->interf->get().interf().peekStreamMessages.getEndpoint(), FailureStatus())
: Never())) {
return Void();
}
when(wait(self->interf->onChange())) {}
}
}
@ -373,9 +462,14 @@ Future<Void> ILogSystem::ServerPeekCursor::onFailed() {
bool ILogSystem::ServerPeekCursor::isActive() const {
if (!interf->get().present())
return false;
if (messageVersion >= end)
if (isExhausted())
return false;
return IFailureMonitor::failureMonitor().getState(interf->get().interf().peekMessages.getEndpoint()).isAvailable();
return IFailureMonitor::failureMonitor()
.getState(interf->get().interf().peekMessages.getEndpoint())
.isAvailable() &&
IFailureMonitor::failureMonitor()
.getState(interf->get().interf().peekStreamMessages.getEndpoint())
.isAvailable();
}
bool ILogSystem::ServerPeekCursor::isExhausted() const {

View File

@ -798,8 +798,7 @@ ACTOR static Future<Void> finishMoveKeys(Database occ,
// Wait for a durable quorum of servers in destServers to have keys available (readWrite)
// They must also have at least the transaction read version so they can't "forget" the shard
// between
// now and when this transaction commits.
// between now and when this transaction commits.
state vector<Future<Void>> serverReady; // only for count below
state vector<Future<Void>> tssReady; // for waiting in parallel with tss
state vector<StorageServerInterface> tssReadyInterfs;

View File

@ -300,6 +300,7 @@ struct TLogData : NonCopyable {
int64_t instanceID;
int64_t bytesInput;
int64_t bytesDurable;
int activePeekStreams = 0;
Version prevVersion;
@ -477,6 +478,7 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
});
specialCounter(
cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; });
specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; });
}
~LogData() {
@ -931,14 +933,15 @@ ACTOR Future<Void> tLogPop(TLogData* self, TLogPopRequest req, Reference<LogData
}
void peekMessagesFromMemory(Reference<LogData> self,
TLogPeekRequest const& req,
Tag tag,
Version reqBegin,
BinaryWriter& messages,
Version& endVersion) {
OldTag oldTag = convertTag(req.tag);
OldTag oldTag = convertTag(tag);
ASSERT(!messages.getLength());
auto& deque = get_version_messages(self, oldTag);
Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1);
Version begin = std::max(reqBegin, self->persistentDataDurableVersion + 1);
auto it = std::lower_bound(deque.begin(),
deque.end(),
std::make_pair(begin, LengthPrefixedStringRef()),
@ -963,24 +966,33 @@ void peekMessagesFromMemory(Reference<LogData> self,
uint32_t subVersion;
rd >> messageLength >> subVersion;
messageLength += sizeof(uint16_t) + sizeof(Tag);
messages << messageLength << subVersion << uint16_t(1) << req.tag;
messages << messageLength << subVersion << uint16_t(1) << tag;
messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag));
messages.serializeBytes(rd.readBytes(messageLength), messageLength);
}
}
}
ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference<LogData> logData) {
// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request
ACTOR template <typename PromiseType>
Future<Void> tLogPeekMessages(PromiseType replyPromise,
TLogData* self,
Reference<LogData> logData,
Version reqBegin,
Tag reqTag,
bool reqReturnIfBlocked = false,
bool reqOnlySpilled = false,
Optional<std::pair<UID, int>> reqSequence = Optional<std::pair<UID, int>>()) {
state BinaryWriter messages(Unversioned());
state BinaryWriter messages2(Unversioned());
state int sequence = -1;
state UID peekId;
state OldTag oldTag = convertTag(req.tag);
state OldTag oldTag = convertTag(reqTag);
if (req.sequence.present()) {
if (reqSequence.present()) {
try {
peekId = req.sequence.get().first;
sequence = req.sequence.get().second;
peekId = reqSequence.get().first;
sequence = reqSequence.get().second;
if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS &&
self->peekTracker.find(peekId) == self->peekTracker.end()) {
throw operation_obsolete();
@ -989,12 +1001,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
auto& trackerData = self->peekTracker[peekId];
trackerData.lastUpdate = now();
Version ver = wait(trackerData.sequence_version[sequence].getFuture());
req.begin = std::max(ver, req.begin);
reqBegin = std::max(ver, reqBegin);
wait(yield());
}
} catch (Error& e) {
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
replyPromise.sendError(e);
return Void();
} else {
throw;
@ -1002,22 +1014,22 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
}
}
if (req.returnIfBlocked && logData->version.get() < req.begin) {
req.reply.sendError(end_of_stream());
if (reqReturnIfBlocked && logData->version.get() < reqBegin) {
replyPromise.sendError(end_of_stream());
return Void();
}
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
// Wait until we have something to return that the caller doesn't already have
if (logData->version.get() < req.begin) {
wait(logData->version.whenAtLeast(req.begin));
if (logData->version.get() < reqBegin) {
wait(logData->version.whenAtLeast(reqBegin));
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
}
state Version endVersion = logData->version.get() + 1;
Version poppedVer = poppedVersion(logData, oldTag);
if (poppedVer > req.begin) {
if (poppedVer > reqBegin) {
TLogPeekReply rep;
rep.maxKnownVersion = logData->version.get();
rep.minKnownCommittedVersion = 0;
@ -1025,12 +1037,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
rep.end = poppedVer;
rep.onlySpilled = false;
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = self->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
trackerData.lastUpdate = now();
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -1038,37 +1050,37 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
if (sequenceData.isSet()) {
if (sequenceData.getFuture().get() != rep.end) {
TEST(true); // tlog peek second attempt ended at a different version
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(rep.end);
}
rep.begin = req.begin;
rep.begin = reqBegin;
}
req.reply.send(rep);
replyPromise.send(rep);
return Void();
}
// grab messages from disk
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
if (req.begin <= logData->persistentDataDurableVersion) {
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
if (reqBegin <= logData->persistentDataDurableVersion) {
// Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We
// may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if
// an initial attempt to read from disk results in insufficient data and the required data is no longer in
// memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the
// result?
peekMessagesFromMemory(logData, req, messages2, endVersion);
peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion);
RangeResult kvs = wait(self->persistentData->readRange(
KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, req.begin),
KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, reqBegin),
persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)),
SERVER_KNOBS->DESIRED_TOTAL_BYTES,
SERVER_KNOBS->DESIRED_TOTAL_BYTES));
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
for (auto& kv : kvs) {
auto ver = decodeTagMessagesKey(kv.key);
@ -1080,7 +1092,7 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
uint32_t subVersion;
rd >> messageLength >> subVersion;
messageLength += sizeof(uint16_t) + sizeof(Tag);
messages << messageLength << subVersion << uint16_t(1) << req.tag;
messages << messageLength << subVersion << uint16_t(1) << reqTag;
messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag));
messages.serializeBytes(rd.readBytes(messageLength), messageLength);
}
@ -1091,39 +1103,79 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
else
messages.serializeBytes(messages2.toValue());
} else {
peekMessagesFromMemory(logData, req, messages, endVersion);
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion);
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
}
TLogPeekReply reply;
reply.maxKnownVersion = logData->version.get();
reply.minKnownCommittedVersion = 0;
reply.onlySpilled = false;
reply.messages = messages.toValue();
reply.messages = StringRef(reply.arena, messages.toValue());
reply.end = endVersion;
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress());
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress());
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = self->peekTracker[peekId];
trackerData.lastUpdate = now();
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (sequenceData.isSet()) {
if (sequenceData.getFuture().get() != reply.end) {
TEST(true); // tlog peek second attempt ended at a different version (2)
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(reply.end);
}
reply.begin = req.begin;
reply.begin = reqBegin;
}
req.reply.send(reply);
replyPromise.send(reply);
return Void();
}
// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover
ACTOR Future<Void> tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference<LogData> logData) {
self->activePeekStreams++;
state Version begin = req.begin;
state bool onlySpilled = false;
req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes));
loop {
state TLogPeekStreamReply reply;
state Promise<TLogPeekReply> promise;
state Future<TLogPeekReply> future(promise.getFuture());
try {
wait(req.reply.onReady() && store(reply.rep, future) &&
tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled));
reply.rep.begin = begin;
req.reply.send(reply);
begin = reply.rep.end;
onlySpilled = reply.rep.onlySpilled;
if (reply.rep.end > logData->version.get()) {
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
} else {
wait(delay(0, g_network->getCurrentTask()));
}
} catch (Error& e) {
self->activePeekStreams--;
TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId)
.detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress())
.error(e, true);
if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
return Void();
} else {
throw;
}
}
}
}
ACTOR Future<Void> doQueueCommit(TLogData* self, Reference<LogData> logData) {
state Version ver = logData->version.get();
state Version commitNumber = self->queueCommitBegin + 1;
@ -1288,7 +1340,13 @@ ACTOR Future<Void> serveTLogInterface(TLogData* self,
PromiseStream<Void> warningCollectorInput) {
loop choose {
when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) {
logData->addActor.send(tLogPeekMessages(self, req, logData));
logData->addActor.send(tLogPeekMessages(
req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence));
}
when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) {
TraceEvent(SevDebug, "TLogPeekStream", logData->logId)
.detail("Token", tli.peekStreamMessages.getEndpoint().token);
logData->addActor.send(tLogPeekStream(self, req, logData));
}
when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) {
logData->addActor.send(tLogPop(self, req, logData));
@ -1435,6 +1493,7 @@ ACTOR Future<Void> restorePersistentState(TLogData* self, LocalityData locality)
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -1574,7 +1633,7 @@ ACTOR Future<Void> tLog(IKeyValueStore* persistentData,
state TLogData self(tlogId, workerID, persistentData, persistentQueue, db);
state Future<Void> error = actorCollection(self.sharedActors.getFuture());
TraceEvent("SharedTlog", tlogId).log();
TraceEvent("SharedTlog", tlogId).detail("Version", "4.6");
try {
wait(restorePersistentState(&self, locality));

View File

@ -276,6 +276,7 @@ struct TLogData : NonCopyable {
int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling.
int64_t overheadBytesInput;
int64_t overheadBytesDurable;
int activePeekStreams = 0;
WorkerCache<TLogInterface> tlogCache;
@ -572,6 +573,7 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
});
specialCounter(
cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; });
specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; });
}
~LogData() {
@ -1172,15 +1174,16 @@ std::deque<std::pair<Version, LengthPrefixedStringRef>>& getVersionMessages(Refe
};
void peekMessagesFromMemory(Reference<LogData> self,
TLogPeekRequest const& req,
Tag tag,
Version begin,
BinaryWriter& messages,
Version& endVersion) {
ASSERT(!messages.getLength());
auto& deque = getVersionMessages(self, req.tag);
auto& deque = getVersionMessages(self, tag);
//TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size());
Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1);
begin = std::max(begin, self->persistentDataDurableVersion + 1);
auto it = std::lower_bound(deque.begin(),
deque.end(),
std::make_pair(begin, LengthPrefixedStringRef()),
@ -1203,29 +1206,38 @@ void peekMessagesFromMemory(Reference<LogData> self,
}
}
ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference<LogData> logData) {
// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request
ACTOR template <typename PromiseType>
Future<Void> tLogPeekMessages(PromiseType replyPromise,
TLogData* self,
Reference<LogData> logData,
Version reqBegin,
Tag reqTag,
bool reqReturnIfBlocked = false,
bool reqOnlySpilled = false,
Optional<std::pair<UID, int>> reqSequence = Optional<std::pair<UID, int>>()) {
state BinaryWriter messages(Unversioned());
state BinaryWriter messages2(Unversioned());
state int sequence = -1;
state UID peekId;
state double queueStart = now();
if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) {
req.tag.id = req.tag.id % logData->txsTags;
if (reqTag.locality == tagLocalityTxs && reqTag.id >= logData->txsTags && logData->txsTags > 0) {
reqTag.id = reqTag.id % logData->txsTags;
}
if (req.sequence.present()) {
if (reqSequence.present()) {
try {
peekId = req.sequence.get().first;
sequence = req.sequence.get().second;
peekId = reqSequence.get().first;
sequence = reqSequence.get().second;
if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS &&
logData->peekTracker.find(peekId) == logData->peekTracker.end()) {
throw operation_obsolete();
}
auto& trackerData = logData->peekTracker[peekId];
if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) {
trackerData.tag = req.tag;
trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled));
trackerData.tag = reqTag;
trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled));
}
auto seqBegin = trackerData.sequence_version.begin();
while (trackerData.sequence_version.size() &&
@ -1252,12 +1264,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
trackerData.lastUpdate = now();
std::pair<Version, bool> prevPeekData = wait(fPrevPeekData);
req.begin = std::max(prevPeekData.first, req.begin);
req.onlySpilled = prevPeekData.second;
reqBegin = std::max(prevPeekData.first, reqBegin);
reqOnlySpilled = prevPeekData.second;
wait(yield());
} catch (Error& e) {
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
replyPromise.sendError(e);
return Void();
} else {
throw;
@ -1267,32 +1279,32 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state double blockStart = now();
if (req.returnIfBlocked && logData->version.get() < req.begin) {
req.reply.sendError(end_of_stream());
if (req.sequence.present()) {
if (reqReturnIfBlocked && logData->version.get() < reqBegin) {
replyPromise.sendError(end_of_stream());
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (!sequenceData.isSet()) {
sequenceData.send(std::make_pair(req.begin, req.onlySpilled));
sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled));
}
}
return Void();
}
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
// Wait until we have something to return that the caller doesn't already have
if (logData->version.get() < req.begin) {
wait(logData->version.whenAtLeast(req.begin));
if (logData->version.get() < reqBegin) {
wait(logData->version.whenAtLeast(reqBegin));
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
}
if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) {
if (logData->locality != tagLocalitySatellite && reqTag.locality == tagLocalityLogRouter) {
wait(self->concurrentLogRouterReads.take());
state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads);
wait(delay(0.0, TaskPriority::Low));
}
if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) {
if (reqBegin <= logData->persistentDataDurableVersion && reqTag.locality != tagLocalityTxs && reqTag != txsTag) {
// Reading spilled data will almost always imply that the storage server is >5s behind the rest
// of the cluster. We shouldn't prioritize spending CPU on helping this server catch up
// slightly faster over keeping the rest of the cluster operating normally.
@ -1303,8 +1315,8 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state double workStart = now();
Version poppedVer = poppedVersion(logData, req.tag);
if (poppedVer > req.begin) {
Version poppedVer = poppedVersion(logData, reqTag);
if (poppedVer > reqBegin) {
TLogPeekReply rep;
rep.maxKnownVersion = logData->version.get();
rep.minKnownCommittedVersion = logData->minKnownCommittedVersion;
@ -1312,12 +1324,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
rep.end = poppedVer;
rep.onlySpilled = false;
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
trackerData.lastUpdate = now();
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -1325,16 +1337,16 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
if (sequenceData.isSet()) {
if (sequenceData.getFuture().get().first != rep.end) {
TEST(true); // tlog peek second attempt ended at a different version
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(rep.end, rep.onlySpilled));
}
rep.begin = req.begin;
rep.begin = reqBegin;
}
req.reply.send(rep);
replyPromise.send(rep);
return Void();
}
@ -1342,27 +1354,27 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state bool onlySpilled = false;
// grab messages from disk
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
if (req.begin <= logData->persistentDataDurableVersion) {
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
if (reqBegin <= logData->persistentDataDurableVersion) {
// Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We
// may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if
// an initial attempt to read from disk results in insufficient data and the required data is no longer in
// memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the
// result?
if (req.onlySpilled) {
if (reqOnlySpilled) {
endVersion = logData->persistentDataDurableVersion + 1;
} else {
peekMessagesFromMemory(logData, req, messages2, endVersion);
peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion);
}
RangeResult kvs = wait(self->persistentData->readRange(
KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin),
persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)),
KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin),
persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)),
SERVER_KNOBS->DESIRED_TOTAL_BYTES,
SERVER_KNOBS->DESIRED_TOTAL_BYTES));
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
for (auto& kv : kvs) {
auto ver = decodeTagMessagesKey(kv.key);
@ -1377,20 +1389,20 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
messages.serializeBytes(messages2.toValue());
}
} else {
peekMessagesFromMemory(logData, req, messages, endVersion);
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion);
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
}
TLogPeekReply reply;
reply.maxKnownVersion = logData->version.get();
reply.minKnownCommittedVersion = logData->minKnownCommittedVersion;
reply.messages = messages.toValue();
reply.messages = StringRef(reply.arena, messages.toValue());
reply.end = endVersion;
reply.onlySpilled = onlySpilled;
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().address);
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", replyPromise.getEndpoint().address);
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
trackerData.lastUpdate = now();
@ -1414,7 +1426,7 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -1423,19 +1435,59 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
trackerData.duplicatePeeks++;
if (sequenceData.getFuture().get().first != reply.end) {
TEST(true); // tlog peek second attempt ended at a different version (2)
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(reply.end, reply.onlySpilled));
}
reply.begin = req.begin;
reply.begin = reqBegin;
}
req.reply.send(reply);
replyPromise.send(reply);
return Void();
}
// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover
ACTOR Future<Void> tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference<LogData> logData) {
self->activePeekStreams++;
state Version begin = req.begin;
state bool onlySpilled = false;
req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes));
loop {
state TLogPeekStreamReply reply;
state Promise<TLogPeekReply> promise;
state Future<TLogPeekReply> future(promise.getFuture());
try {
wait(req.reply.onReady() && store(reply.rep, future) &&
tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled));
reply.rep.begin = begin;
req.reply.send(reply);
begin = reply.rep.end;
onlySpilled = reply.rep.onlySpilled;
if (reply.rep.end > logData->version.get()) {
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
} else {
wait(delay(0, g_network->getCurrentTask()));
}
} catch (Error& e) {
self->activePeekStreams--;
TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId)
.detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress())
.error(e, true);
if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
return Void();
} else {
throw;
}
}
}
}
ACTOR Future<Void> doQueueCommit(TLogData* self,
Reference<LogData> logData,
std::vector<Reference<LogData>> missingFinalCommit) {
@ -1930,7 +1982,13 @@ ACTOR Future<Void> serveTLogInterface(TLogData* self,
}
}
when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) {
logData->addActor.send(tLogPeekMessages(self, req, logData));
logData->addActor.send(tLogPeekMessages(
req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence));
}
when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) {
TraceEvent(SevDebug, "TLogPeekStream", logData->logId)
.detail("Token", tli.peekStreamMessages.getEndpoint().token);
logData->addActor.send(tLogPeekStream(self, req, logData));
}
when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) {
logData->addActor.send(tLogPop(self, req, logData));
@ -2327,6 +2385,7 @@ ACTOR Future<Void> restorePersistentState(TLogData* self,
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -2537,6 +2596,7 @@ ACTOR Future<Void> tLogStart(TLogData* self, InitializeTLogRequest req, Locality
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -2729,7 +2789,8 @@ ACTOR Future<Void> tLog(IKeyValueStore* persistentData,
state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder);
state Future<Void> error = actorCollection(self.sharedActors.getFuture());
TraceEvent("SharedTlog", tlogId).log();
TraceEvent("SharedTlog", tlogId).detail("Version", "6.0");
try {
if (restoreFromDisk) {
wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests));

View File

@ -339,6 +339,7 @@ struct TLogData : NonCopyable {
int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling.
int64_t overheadBytesInput;
int64_t overheadBytesDurable;
int activePeekStreams = 0;
WorkerCache<TLogInterface> tlogCache;
FlowLock peekMemoryLimiter;
@ -661,6 +662,7 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; });
specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); });
specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); });
specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; });
}
~LogData() {
@ -1440,17 +1442,19 @@ ACTOR Future<Void> tLogPopCore(TLogData* self, Tag inputTag, Version to, Referen
}
uint64_t PoppedVersionLag = logData->persistentDataDurableVersion - logData->queuePoppedVersion;
if ( SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE &&
(logData->queuePoppedVersion > 0) && //avoid generating massive events at beginning
(tagData->unpoppedRecovered || PoppedVersionLag >= SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { //when recovery or long lag
if (SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE &&
(logData->queuePoppedVersion > 0) && // avoid generating massive events at beginning
(tagData->unpoppedRecovered ||
PoppedVersionLag >=
SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { // when recovery or long lag
TraceEvent("TLogPopDetails", logData->logId)
.detail("Tag", tagData->tag.toString())
.detail("UpTo", upTo)
.detail("PoppedVersionLag", PoppedVersionLag)
.detail("MinPoppedTag", logData->minPoppedTag.toString())
.detail("QueuePoppedVersion", logData->queuePoppedVersion)
.detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False")
.detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False");
.detail("Tag", tagData->tag.toString())
.detail("UpTo", upTo)
.detail("PoppedVersionLag", PoppedVersionLag)
.detail("MinPoppedTag", logData->minPoppedTag.toString())
.detail("QueuePoppedVersion", logData->queuePoppedVersion)
.detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False")
.detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False");
}
if (upTo > logData->persistentDataDurableVersion)
wait(tagData->eraseMessagesBefore(upTo, self, logData, TaskPriority::TLogPop));
@ -1487,15 +1491,16 @@ ACTOR Future<Void> tLogPop(TLogData* self, TLogPopRequest req, Reference<LogData
}
void peekMessagesFromMemory(Reference<LogData> self,
TLogPeekRequest const& req,
Tag tag,
Version begin,
BinaryWriter& messages,
Version& endVersion) {
ASSERT(!messages.getLength());
auto& deque = getVersionMessages(self, req.tag);
auto& deque = getVersionMessages(self, tag);
//TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size());
Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1);
begin = std::max(begin, self->persistentDataDurableVersion + 1);
auto it = std::lower_bound(deque.begin(),
deque.end(),
std::make_pair(begin, LengthPrefixedStringRef()),
@ -1540,29 +1545,38 @@ ACTOR Future<std::vector<StringRef>> parseMessagesForTag(StringRef commitBlob, T
return relevantMessages;
}
ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference<LogData> logData) {
// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request
ACTOR template <typename PromiseType>
Future<Void> tLogPeekMessages(PromiseType replyPromise,
TLogData* self,
Reference<LogData> logData,
Version reqBegin,
Tag reqTag,
bool reqReturnIfBlocked = false,
bool reqOnlySpilled = false,
Optional<std::pair<UID, int>> reqSequence = Optional<std::pair<UID, int>>()) {
state BinaryWriter messages(Unversioned());
state BinaryWriter messages2(Unversioned());
state int sequence = -1;
state UID peekId;
state double queueStart = now();
if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) {
req.tag.id = req.tag.id % logData->txsTags;
if (reqTag.locality == tagLocalityTxs && reqTag.id >= logData->txsTags && logData->txsTags > 0) {
reqTag.id = reqTag.id % logData->txsTags;
}
if (req.sequence.present()) {
if (reqSequence.present()) {
try {
peekId = req.sequence.get().first;
sequence = req.sequence.get().second;
peekId = reqSequence.get().first;
sequence = reqSequence.get().second;
if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS &&
logData->peekTracker.find(peekId) == logData->peekTracker.end()) {
throw operation_obsolete();
}
auto& trackerData = logData->peekTracker[peekId];
if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) {
trackerData.tag = req.tag;
trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled));
trackerData.tag = reqTag;
trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled));
}
auto seqBegin = trackerData.sequence_version.begin();
// The peek cursor and this comparison need to agree about the maximum number of in-flight requests.
@ -1589,12 +1603,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
}
trackerData.lastUpdate = now();
std::pair<Version, bool> prevPeekData = wait(fPrevPeekData);
req.begin = std::max(prevPeekData.first, req.begin);
req.onlySpilled = prevPeekData.second;
reqBegin = std::max(prevPeekData.first, reqBegin);
reqOnlySpilled = prevPeekData.second;
wait(yield());
} catch (Error& e) {
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
replyPromise.sendError(e);
return Void();
} else {
throw;
@ -1604,32 +1618,32 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state double blockStart = now();
if (req.returnIfBlocked && logData->version.get() < req.begin) {
req.reply.sendError(end_of_stream());
if (req.sequence.present()) {
if (reqReturnIfBlocked && logData->version.get() < reqBegin) {
replyPromise.sendError(end_of_stream());
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (!sequenceData.isSet()) {
sequenceData.send(std::make_pair(req.begin, req.onlySpilled));
sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled));
}
}
return Void();
}
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
// Wait until we have something to return that the caller doesn't already have
if (logData->version.get() < req.begin) {
wait(logData->version.whenAtLeast(req.begin));
if (logData->version.get() < reqBegin) {
wait(logData->version.whenAtLeast(reqBegin));
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
}
if (req.tag.locality == tagLocalityLogRouter) {
if (reqTag.locality == tagLocalityLogRouter) {
wait(self->concurrentLogRouterReads.take());
state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads);
wait(delay(0.0, TaskPriority::Low));
}
if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) {
if (reqBegin <= logData->persistentDataDurableVersion && reqTag.locality != tagLocalityTxs && reqTag != txsTag) {
// Reading spilled data will almost always imply that the storage server is >5s behind the rest
// of the cluster. We shouldn't prioritize spending CPU on helping this server catch up
// slightly faster over keeping the rest of the cluster operating normally.
@ -1640,8 +1654,8 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state double workStart = now();
Version poppedVer = poppedVersion(logData, req.tag);
if (poppedVer > req.begin) {
Version poppedVer = poppedVersion(logData, reqTag);
if (poppedVer > reqBegin) {
TLogPeekReply rep;
rep.maxKnownVersion = logData->version.get();
rep.minKnownCommittedVersion = logData->minKnownCommittedVersion;
@ -1649,12 +1663,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
rep.end = poppedVer;
rep.onlySpilled = false;
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
trackerData.lastUpdate = now();
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -1662,16 +1676,16 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
if (sequenceData.isSet()) {
if (sequenceData.getFuture().get().first != rep.end) {
TEST(true); // tlog peek second attempt ended at a different version
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(rep.end, rep.onlySpilled));
}
rep.begin = req.begin;
rep.begin = reqBegin;
}
req.reply.send(rep);
replyPromise.send(rep);
return Void();
}
@ -1679,24 +1693,24 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state bool onlySpilled = false;
// grab messages from disk
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
if (req.begin <= logData->persistentDataDurableVersion) {
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
if (reqBegin <= logData->persistentDataDurableVersion) {
// Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We
// may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if
// an initial attempt to read from disk results in insufficient data and the required data is no longer in
// memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the
// result?
if (req.onlySpilled) {
if (reqOnlySpilled) {
endVersion = logData->persistentDataDurableVersion + 1;
} else {
peekMessagesFromMemory(logData, req, messages2, endVersion);
peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion);
}
if (req.tag.locality == tagLocalityTxs || req.tag == txsTag) {
if (reqTag.locality == tagLocalityTxs || reqTag == txsTag) {
RangeResult kvs = wait(self->persistentData->readRange(
KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin),
persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)),
KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin),
persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)),
SERVER_KNOBS->DESIRED_TOTAL_BYTES,
SERVER_KNOBS->DESIRED_TOTAL_BYTES));
@ -1716,11 +1730,11 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
// FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow.
RangeResult kvrefs = wait(self->persistentData->readRange(
KeyRangeRef(
persistTagMessageRefsKey(logData->logId, req.tag, req.begin),
persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)),
persistTagMessageRefsKey(logData->logId, reqTag, reqBegin),
persistTagMessageRefsKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)),
SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1));
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
state std::vector<std::pair<IDiskQueue::location, IDiskQueue::location>> commitLocations;
state bool earlyEnd = false;
@ -1737,7 +1751,7 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
earlyEnd = true;
break;
}
if (sd.version >= req.begin) {
if (sd.version >= reqBegin) {
firstVersion = std::min(firstVersion, sd.version);
const IDiskQueue::location end = sd.start.lo + sd.length;
commitLocations.emplace_back(sd.start, end);
@ -1779,7 +1793,7 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
messages << VERSION_HEADER << entry.version;
std::vector<StringRef> rawMessages =
wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags));
wait(parseMessagesForTag(entry.messages, reqTag, logData->logRouterTags));
for (const StringRef& msg : rawMessages) {
messages.serializeBytes(msg);
}
@ -1799,25 +1813,25 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
}
}
} else {
if (req.onlySpilled) {
if (reqOnlySpilled) {
endVersion = logData->persistentDataDurableVersion + 1;
} else {
peekMessagesFromMemory(logData, req, messages, endVersion);
peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion);
}
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
}
TLogPeekReply reply;
reply.maxKnownVersion = logData->version.get();
reply.minKnownCommittedVersion = logData->minKnownCommittedVersion;
reply.messages = messages.toValue();
reply.messages = StringRef(reply.arena, messages.toValue());
reply.end = endVersion;
reply.onlySpilled = onlySpilled;
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress());
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress());
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
trackerData.lastUpdate = now();
@ -1841,7 +1855,7 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -1850,19 +1864,59 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
trackerData.duplicatePeeks++;
if (sequenceData.getFuture().get().first != reply.end) {
TEST(true); // tlog peek second attempt ended at a different version (2)
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(reply.end, reply.onlySpilled));
}
reply.begin = req.begin;
reply.begin = reqBegin;
}
req.reply.send(reply);
replyPromise.send(reply);
return Void();
}
// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover
ACTOR Future<Void> tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference<LogData> logData) {
self->activePeekStreams++;
state Version begin = req.begin;
state bool onlySpilled = false;
req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes));
loop {
state TLogPeekStreamReply reply;
state Promise<TLogPeekReply> promise;
state Future<TLogPeekReply> future(promise.getFuture());
try {
wait(req.reply.onReady() && store(reply.rep, future) &&
tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled));
reply.rep.begin = begin;
req.reply.send(reply);
begin = reply.rep.end;
onlySpilled = reply.rep.onlySpilled;
if (reply.rep.end > logData->version.get()) {
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
} else {
wait(delay(0, g_network->getCurrentTask()));
}
} catch (Error& e) {
self->activePeekStreams--;
TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId)
.detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress())
.error(e, true);
if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
return Void();
} else {
throw;
}
}
}
}
ACTOR Future<Void> watchDegraded(TLogData* self) {
if (g_network->isSimulated() && g_simulator.speedUpSimulation) {
return Void();
@ -2373,7 +2427,13 @@ ACTOR Future<Void> serveTLogInterface(TLogData* self,
}
}
when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) {
logData->addActor.send(tLogPeekMessages(self, req, logData));
logData->addActor.send(tLogPeekMessages(
req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence));
}
when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) {
TraceEvent(SevDebug, "TLogPeekStream", logData->logId)
.detail("Token", tli.peekStreamMessages.getEndpoint().token);
logData->addActor.send(tLogPeekStream(self, req, logData));
}
when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) {
logData->addActor.send(tLogPop(self, req, logData));
@ -2788,6 +2848,7 @@ ACTOR Future<Void> restorePersistentState(TLogData* self,
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -2826,9 +2887,9 @@ ACTOR Future<Void> restorePersistentState(TLogData* self,
logsByVersion.emplace_back(ver, id1);
TraceEvent("TLogPersistentStateRestore", self->dbgid)
.detail("LogId", logData->logId)
.detail("Ver", ver)
.detail("RecoveryCount", logData->recoveryCount);
.detail("LogId", logData->logId)
.detail("Ver", ver)
.detail("RecoveryCount", logData->recoveryCount);
// Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion
// might be lost, but that is fine because we will get the corresponding data back, too.
tagKeys = prefixRange(rawId.withPrefix(persistTagPoppedKeys.begin));
@ -3019,6 +3080,7 @@ ACTOR Future<Void> tLogStart(TLogData* self, InitializeTLogRequest req, Locality
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -3218,7 +3280,8 @@ ACTOR Future<Void> tLog(IKeyValueStore* persistentData,
state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder);
state Future<Void> error = actorCollection(self.sharedActors.getFuture());
TraceEvent("SharedTlog", tlogId).log();
TraceEvent("SharedTlog", tlogId).detail("Version", "6.2");
try {
if (restoreFromDisk) {
wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests));

View File

@ -637,13 +637,12 @@ ACTOR Future<Void> waitForQuietDatabase(Database cx,
// In a simulated environment, wait 5 seconds so that workers can move to their optimal locations
if (g_network->isSimulated())
wait(delay(5.0));
// The quiet database check (which runs at the end of every test) will always time out due to active data movement.
// To get around this, quiet Database will disable the perpetual wiggle in the setup phase.
printf("Set perpetual_storage_wiggle=0 ...\n");
wait(setPerpetualStorageWiggle(cx, false, LockAware::True));
printf("Set perpetual_storage_wiggle=0 Done.\n");
// Require 3 consecutive successful quiet database checks spaced 2 second apart
state int numSuccesses = 0;

View File

@ -23,8 +23,9 @@
#include "fdbrpc/FailureMonitor.h"
#include "fdbrpc/Smoother.h"
#include "fdbrpc/simulator.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/DataDistribution.actor.h"
#include "fdbserver/RatekeeperInterface.h"
@ -527,6 +528,9 @@ struct RatekeeperLimits {
context(context) {}
};
namespace RatekeeperActorCpp {
// Differentiate from GrvProxyInfo in DatabaseContext.h
struct GrvProxyInfo {
int64_t totalTransactions;
int64_t batchTransactions;
@ -540,6 +544,8 @@ struct GrvProxyInfo {
}
};
} // namespace RatekeeperActorCpp
struct RatekeeperData {
UID id;
Database db;
@ -547,7 +553,7 @@ struct RatekeeperData {
Map<UID, StorageQueueInfo> storageQueueInfo;
Map<UID, TLogQueueInfo> tlogQueueInfo;
std::map<UID, GrvProxyInfo> grvProxyInfo;
std::map<UID, RatekeeperActorCpp::GrvProxyInfo> grvProxyInfo;
Smoother smoothReleasedTransactions, smoothBatchReleasedTransactions, smoothTotalDurableBytes;
HealthMetrics healthMetrics;
DatabaseConfiguration configuration;
@ -595,8 +601,8 @@ struct RatekeeperData {
SERVER_KNOBS->MAX_TL_SS_VERSION_DIFFERENCE_BATCH,
SERVER_KNOBS->TARGET_DURABILITY_LAG_VERSIONS_BATCH),
autoThrottlingEnabled(false) {
expiredTagThrottleCleanup =
recurring([this]() { ThrottleApi::expire(this->db); }, SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL);
expiredTagThrottleCleanup = recurring([this]() { ThrottleApi::expire(this->db.getReference()); },
SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL);
}
};
@ -942,7 +948,8 @@ void tryAutoThrottleTag(RatekeeperData* self,
TagSet tags;
tags.addTag(tag);
self->addActor.send(ThrottleApi::throttleTags(self->db,
Reference<DatabaseContext> db = Reference<DatabaseContext>::addRef(self->db.getPtr());
self->addActor.send(ThrottleApi::throttleTags(db,
tags,
clientRate.get(),
SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION,

View File

@ -19,6 +19,11 @@ extern "C" void stackSignalHandler(int sig) {
}
}
#ifdef _WIN32
#define SIGUSR1 10
#define SIGUSR2 12
#endif
void setupStackSignal() {
std::signal(SIGUSR1, &stackSignalHandler);
}

View File

@ -38,6 +38,8 @@ struct TLogInterface {
UID sharedTLogID;
RequestStream<struct TLogPeekRequest> peekMessages;
RequestStream<struct TLogPeekStreamRequest>
peekStreamMessages; // request establish a peek stream with the TLog server
RequestStream<struct TLogPopRequest> popMessages;
RequestStream<struct TLogCommitRequest> commit;
@ -68,6 +70,7 @@ struct TLogInterface {
NetworkAddressList addresses() const { return peekMessages.getEndpoint().addresses; }
void initEndpoints() {
// NOTE: the adding order should be the same as the hardcoded indices in serialize()
std::vector<std::pair<FlowReceiver*, TaskPriority>> streams;
streams.push_back(peekMessages.getReceiver(TaskPriority::TLogPeek));
streams.push_back(popMessages.getReceiver(TaskPriority::TLogPop));
@ -80,6 +83,7 @@ struct TLogInterface {
streams.push_back(disablePopRequest.getReceiver());
streams.push_back(enablePopRequest.getReceiver());
streams.push_back(snapRequest.getReceiver());
streams.push_back(peekStreamMessages.getReceiver(TaskPriority::TLogPeek));
FlowTransport::transport().addEndpoints(streams);
}
@ -106,6 +110,8 @@ struct TLogInterface {
enablePopRequest =
RequestStream<struct TLogEnablePopRequest>(peekMessages.getEndpoint().getAdjustedEndpoint(9));
snapRequest = RequestStream<struct TLogSnapRequest>(peekMessages.getEndpoint().getAdjustedEndpoint(10));
peekStreamMessages =
RequestStream<struct TLogPeekStreamRequest>(peekMessages.getEndpoint().getAdjustedEndpoint(11));
}
}
};
@ -209,6 +215,40 @@ struct TLogPeekRequest {
}
};
struct TLogPeekStreamReply : public ReplyPromiseStreamReply {
constexpr static FileIdentifier file_identifier = 10072848;
TLogPeekReply rep;
TLogPeekStreamReply() = default;
explicit TLogPeekStreamReply(const TLogPeekReply& rep) : rep(rep) {}
int expectedSize() const { return rep.messages.expectedSize() + sizeof(TLogPeekStreamReply); }
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, rep);
}
};
struct TLogPeekStreamRequest {
constexpr static FileIdentifier file_identifier = 10072821;
Arena arena;
Version begin;
Tag tag;
bool returnIfBlocked;
int limitBytes;
ReplyPromiseStream<TLogPeekStreamReply> reply;
TLogPeekStreamRequest() {}
TLogPeekStreamRequest(Version version, Tag tag, bool returnIfBlocked, int limitBytes)
: begin(version), tag(tag), returnIfBlocked(returnIfBlocked), limitBytes(limitBytes) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, arena, begin, tag, returnIfBlocked, limitBytes, reply);
}
};
struct TLogPopRequest {
constexpr static FileIdentifier file_identifier = 5556423;
Arena arena;

View File

@ -341,6 +341,7 @@ struct TLogData : NonCopyable {
int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling.
int64_t overheadBytesInput;
int64_t overheadBytesDurable;
int activePeekStreams = 0;
WorkerCache<TLogInterface> tlogCache;
FlowLock peekMemoryLimiter;
@ -667,6 +668,7 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); });
specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); });
specialCounter(cc, "Generation", [this]() { return this->recoveryCount; });
specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; });
}
~LogData() {
@ -1166,17 +1168,19 @@ ACTOR Future<Void> tLogPopCore(TLogData* self, Tag inputTag, Version to, Referen
}
uint64_t PoppedVersionLag = logData->persistentDataDurableVersion - logData->queuePoppedVersion;
if ( SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE &&
(logData->queuePoppedVersion > 0) && //avoid generating massive events at beginning
(tagData->unpoppedRecovered || PoppedVersionLag >= SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { //when recovery or long lag
if (SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE &&
(logData->queuePoppedVersion > 0) && // avoid generating massive events at beginning
(tagData->unpoppedRecovered ||
PoppedVersionLag >=
SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { // when recovery or long lag
TraceEvent("TLogPopDetails", logData->logId)
.detail("Tag", tagData->tag.toString())
.detail("UpTo", upTo)
.detail("PoppedVersionLag", PoppedVersionLag)
.detail("MinPoppedTag", logData->minPoppedTag.toString())
.detail("QueuePoppedVersion", logData->queuePoppedVersion)
.detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False")
.detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False");
.detail("Tag", tagData->tag.toString())
.detail("UpTo", upTo)
.detail("PoppedVersionLag", PoppedVersionLag)
.detail("MinPoppedTag", logData->minPoppedTag.toString())
.detail("QueuePoppedVersion", logData->queuePoppedVersion)
.detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False")
.detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False");
}
if (upTo > logData->persistentDataDurableVersion)
wait(tagData->eraseMessagesBefore(upTo, self, logData, TaskPriority::TLogPop));
@ -1518,15 +1522,16 @@ std::deque<std::pair<Version, LengthPrefixedStringRef>>& getVersionMessages(Refe
};
void peekMessagesFromMemory(Reference<LogData> self,
TLogPeekRequest const& req,
Tag tag,
Version begin,
BinaryWriter& messages,
Version& endVersion) {
ASSERT(!messages.getLength());
auto& deque = getVersionMessages(self, req.tag);
auto& deque = getVersionMessages(self, tag);
//TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size());
Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1);
begin = std::max(begin, self->persistentDataDurableVersion + 1);
auto it = std::lower_bound(deque.begin(),
deque.end(),
std::make_pair(begin, LengthPrefixedStringRef()),
@ -1552,7 +1557,7 @@ void peekMessagesFromMemory(Reference<LogData> self,
void* data = messages.getData();
DEBUG_TAGS_AND_MESSAGE(
"TLogPeek", currentVersion, StringRef((uint8_t*)data + offset, messages.getLength() - offset), self->logId)
.detail("PeekTag", req.tag);
.detail("PeekTag", tag);
}
}
@ -1578,29 +1583,38 @@ ACTOR Future<std::vector<StringRef>> parseMessagesForTag(StringRef commitBlob, T
return relevantMessages;
}
ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference<LogData> logData) {
// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request
ACTOR template <typename PromiseType>
Future<Void> tLogPeekMessages(PromiseType replyPromise,
TLogData* self,
Reference<LogData> logData,
Version reqBegin,
Tag reqTag,
bool reqReturnIfBlocked = false,
bool reqOnlySpilled = false,
Optional<std::pair<UID, int>> reqSequence = Optional<std::pair<UID, int>>()) {
state BinaryWriter messages(Unversioned());
state BinaryWriter messages2(Unversioned());
state int sequence = -1;
state UID peekId;
state double queueStart = now();
if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) {
req.tag.id = req.tag.id % logData->txsTags;
if (reqTag.locality == tagLocalityTxs && reqTag.id >= logData->txsTags && logData->txsTags > 0) {
reqTag.id = reqTag.id % logData->txsTags;
}
if (req.sequence.present()) {
if (reqSequence.present()) {
try {
peekId = req.sequence.get().first;
sequence = req.sequence.get().second;
peekId = reqSequence.get().first;
sequence = reqSequence.get().second;
if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS &&
logData->peekTracker.find(peekId) == logData->peekTracker.end()) {
throw operation_obsolete();
}
auto& trackerData = logData->peekTracker[peekId];
if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) {
trackerData.tag = req.tag;
trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled));
trackerData.tag = reqTag;
trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled));
}
auto seqBegin = trackerData.sequence_version.begin();
// The peek cursor and this comparison need to agree about the maximum number of in-flight requests.
@ -1627,12 +1641,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
}
trackerData.lastUpdate = now();
std::pair<Version, bool> prevPeekData = wait(fPrevPeekData);
req.begin = std::max(prevPeekData.first, req.begin);
req.onlySpilled = prevPeekData.second;
reqBegin = std::max(prevPeekData.first, reqBegin);
reqOnlySpilled = prevPeekData.second;
wait(yield());
} catch (Error& e) {
if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
replyPromise.sendError(e);
return Void();
} else {
throw;
@ -1642,33 +1656,33 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state double blockStart = now();
if (req.returnIfBlocked && logData->version.get() < req.begin) {
req.reply.sendError(end_of_stream());
if (req.sequence.present()) {
if (reqReturnIfBlocked && logData->version.get() < reqBegin) {
replyPromise.sendError(end_of_stream());
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
trackerData.lastUpdate = now();
if (!sequenceData.isSet()) {
sequenceData.send(std::make_pair(req.begin, req.onlySpilled));
sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled));
}
}
return Void();
}
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
//TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
// Wait until we have something to return that the caller doesn't already have
if (logData->version.get() < req.begin) {
wait(logData->version.whenAtLeast(req.begin));
if (logData->version.get() < reqBegin) {
wait(logData->version.whenAtLeast(reqBegin));
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
}
if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) {
if (logData->locality != tagLocalitySatellite && reqTag.locality == tagLocalityLogRouter) {
wait(self->concurrentLogRouterReads.take());
state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads);
wait(delay(0.0, TaskPriority::Low));
}
if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) {
if (reqBegin <= logData->persistentDataDurableVersion && reqTag.locality != tagLocalityTxs && reqTag != txsTag) {
// Reading spilled data will almost always imply that the storage server is >5s behind the rest
// of the cluster. We shouldn't prioritize spending CPU on helping this server catch up
// slightly faster over keeping the rest of the cluster operating normally.
@ -1679,8 +1693,8 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state double workStart = now();
Version poppedVer = poppedVersion(logData, req.tag);
if (poppedVer > req.begin) {
Version poppedVer = poppedVersion(logData, reqTag);
if (poppedVer > reqBegin) {
TLogPeekReply rep;
rep.maxKnownVersion = logData->version.get();
rep.minKnownCommittedVersion = logData->minKnownCommittedVersion;
@ -1688,12 +1702,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
rep.end = poppedVer;
rep.onlySpilled = false;
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
auto& sequenceData = trackerData.sequence_version[sequence + 1];
trackerData.lastUpdate = now();
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet())
sequenceData.sendError(operation_obsolete());
return Void();
@ -1701,16 +1715,16 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
if (sequenceData.isSet()) {
if (sequenceData.getFuture().get().first != rep.end) {
TEST(true); // tlog peek second attempt ended at a different version
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(rep.end, rep.onlySpilled));
}
rep.begin = req.begin;
rep.begin = reqBegin;
}
req.reply.send(rep);
replyPromise.send(rep);
return Void();
}
@ -1718,24 +1732,24 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
state bool onlySpilled = false;
// grab messages from disk
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2);
if (req.begin <= logData->persistentDataDurableVersion) {
//TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2);
if (reqBegin <= logData->persistentDataDurableVersion) {
// Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We
// may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if
// an initial attempt to read from disk results in insufficient data and the required data is no longer in
// memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the
// result?
if (req.onlySpilled) {
if (reqOnlySpilled) {
endVersion = logData->persistentDataDurableVersion + 1;
} else {
peekMessagesFromMemory(logData, req, messages2, endVersion);
peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion);
}
if (logData->shouldSpillByValue(req.tag)) {
if (logData->shouldSpillByValue(reqTag)) {
RangeResult kvs = wait(self->persistentData->readRange(
KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin),
persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)),
KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin),
persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)),
SERVER_KNOBS->DESIRED_TOTAL_BYTES,
SERVER_KNOBS->DESIRED_TOTAL_BYTES));
@ -1755,11 +1769,11 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
// FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow.
RangeResult kvrefs = wait(self->persistentData->readRange(
KeyRangeRef(
persistTagMessageRefsKey(logData->logId, req.tag, req.begin),
persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)),
persistTagMessageRefsKey(logData->logId, reqTag, reqBegin),
persistTagMessageRefsKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)),
SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1));
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence());
state std::vector<std::pair<IDiskQueue::location, IDiskQueue::location>> commitLocations;
state bool earlyEnd = false;
@ -1776,7 +1790,7 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
earlyEnd = true;
break;
}
if (sd.version >= req.begin) {
if (sd.version >= reqBegin) {
firstVersion = std::min(firstVersion, sd.version);
const IDiskQueue::location end = sd.start.lo + sd.length;
commitLocations.emplace_back(sd.start, end);
@ -1818,12 +1832,12 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
messages << VERSION_HEADER << entry.version;
std::vector<StringRef> rawMessages =
wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags));
wait(parseMessagesForTag(entry.messages, reqTag, logData->logRouterTags));
for (const StringRef& msg : rawMessages) {
messages.serializeBytes(msg);
DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg, logData->logId)
.detail("DebugID", self->dbgid)
.detail("PeekTag", req.tag);
.detail("PeekTag", reqTag);
}
lastRefMessageVersion = entry.version;
@ -1841,28 +1855,28 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
}
}
} else {
if (req.onlySpilled) {
if (reqOnlySpilled) {
endVersion = logData->persistentDataDurableVersion + 1;
} else {
peekMessagesFromMemory(logData, req, messages, endVersion);
peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion);
}
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
//TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence());
}
TLogPeekReply reply;
reply.maxKnownVersion = logData->version.get();
reply.minKnownCommittedVersion = logData->minKnownCommittedVersion;
reply.messages = messages.toValue();
reply.messages = StringRef(reply.arena, messages.toValue());
reply.end = endVersion;
reply.onlySpilled = onlySpilled;
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()).
// detail("BeginVer", req.begin).detail("EndVer", reply.end).
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", reqTag.toString()).
// detail("BeginVer", reqBegin).detail("EndVer", reply.end).
// detail("MsgBytes", reply.messages.expectedSize()).
// detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress());
// detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress());
if (req.sequence.present()) {
if (reqSequence.present()) {
auto& trackerData = logData->peekTracker[peekId];
trackerData.lastUpdate = now();
@ -1886,9 +1900,9 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
auto& sequenceData = trackerData.sequence_version[sequence + 1];
if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) {
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
if (!sequenceData.isSet()) {
// It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next
// It would technically be more correct to .send({reqBegin, reqOnlySpilled}), as the next
// request might still be in the window of active requests, but LogSystemPeekCursor will
// throw away all future responses upon getting an operation_obsolete(), so computing a
// response will probably be a waste of CPU.
@ -1900,19 +1914,59 @@ ACTOR Future<Void> tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen
trackerData.duplicatePeeks++;
if (sequenceData.getFuture().get().first != reply.end) {
TEST(true); // tlog peek second attempt ended at a different version (2)
req.reply.sendError(operation_obsolete());
replyPromise.sendError(operation_obsolete());
return Void();
}
} else {
sequenceData.send(std::make_pair(reply.end, reply.onlySpilled));
}
reply.begin = req.begin;
reply.begin = reqBegin;
}
req.reply.send(reply);
replyPromise.send(reply);
return Void();
}
// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover
ACTOR Future<Void> tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference<LogData> logData) {
self->activePeekStreams++;
state Version begin = req.begin;
state bool onlySpilled = false;
req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes));
loop {
state TLogPeekStreamReply reply;
state Promise<TLogPeekReply> promise;
state Future<TLogPeekReply> future(promise.getFuture());
try {
wait(req.reply.onReady() && store(reply.rep, future) &&
tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled));
reply.rep.begin = begin;
req.reply.send(reply);
begin = reply.rep.end;
onlySpilled = reply.rep.onlySpilled;
if (reply.rep.end > logData->version.get()) {
wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask()));
} else {
wait(delay(0, g_network->getCurrentTask()));
}
} catch (Error& e) {
self->activePeekStreams--;
TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId)
.detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress())
.error(e, true);
if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) {
req.reply.sendError(e);
return Void();
} else {
throw;
}
}
}
}
ACTOR Future<Void> doQueueCommit(TLogData* self,
Reference<LogData> logData,
std::vector<Reference<LogData>> missingFinalCommit) {
@ -2408,8 +2462,14 @@ ACTOR Future<Void> serveTLogInterface(TLogData* self,
logData->logSystem->set(Reference<ILogSystem>());
}
}
when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) {
TraceEvent(SevDebug, "TLogPeekStream", logData->logId)
.detail("Token", tli.peekStreamMessages.getEndpoint().token);
logData->addActor.send(tLogPeekStream(self, req, logData));
}
when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) {
logData->addActor.send(tLogPeekMessages(self, req, logData));
logData->addActor.send(tLogPeekMessages(
req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence));
}
when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) {
logData->addActor.send(tLogPop(self, req, logData));
@ -2664,7 +2724,7 @@ ACTOR Future<Void> tLogCore(TLogData* self,
SERVER_KNOBS->STORAGE_LOGGING_DELAY,
&logData->cc,
logData->logId.toString() + "/TLogMetrics",
[self=self](TraceEvent& te) {
[self = self](TraceEvent& te) {
StorageBytes sbTlog = self->persistentData->getStorageBytes();
te.detail("KvstoreBytesUsed", sbTlog.used);
te.detail("KvstoreBytesFree", sbTlog.free);
@ -2848,6 +2908,7 @@ ACTOR Future<Void> restorePersistentState(TLogData* self,
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -2894,9 +2955,9 @@ ACTOR Future<Void> restorePersistentState(TLogData* self,
logsByVersion.emplace_back(ver, id1);
TraceEvent("TLogPersistentStateRestore", self->dbgid)
.detail("LogId", logData->logId)
.detail("Ver", ver)
.detail("RecoveryCount", logData->recoveryCount);
.detail("LogId", logData->logId)
.detail("Ver", ver)
.detail("RecoveryCount", logData->recoveryCount);
// Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion
// might be lost, but that is fine because we will get the corresponding data back, too.
tagKeys = prefixRange(rawId.withPrefix(persistTagPoppedKeys.begin));
@ -3109,6 +3170,7 @@ ACTOR Future<Void> tLogStart(TLogData* self, InitializeTLogRequest req, Locality
recruited.initEndpoints();
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);

View File

@ -114,7 +114,7 @@ struct TesterInterface {
ACTOR Future<Void> testerServerCore(TesterInterface interf,
Reference<ClusterConnectionFile> ccf,
Reference<AsyncVar<struct ServerDBInfo>> serverDBInfo,
Reference<AsyncVar<struct ServerDBInfo> const> serverDBInfo,
LocalityData locality);
enum test_location_t { TEST_HERE, TEST_ON_SERVERS, TEST_ON_TESTERS };

View File

@ -880,8 +880,9 @@ class Database openDBOnServer(Reference<AsyncVar<ServerDBInfo> const> const& db,
TaskPriority taskID = TaskPriority::DefaultEndpoint,
LockAware = LockAware::False,
EnableLocalityLoadBalance = EnableLocalityLoadBalance::True);
ACTOR Future<Void> extractClusterInterface(Reference<AsyncVar<Optional<struct ClusterControllerFullInterface>>> a,
Reference<AsyncVar<Optional<struct ClusterInterface>>> b);
ACTOR Future<Void> extractClusterInterface(
Reference<AsyncVar<Optional<struct ClusterControllerFullInterface>> const> in,
Reference<AsyncVar<Optional<struct ClusterInterface>>> out);
ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> ccf,
LocalityData localities,
@ -925,7 +926,7 @@ ACTOR Future<Void> storageServer(
connFile); // changes pssi->id() to be the recovered ID); // changes pssi->id() to be the recovered ID
ACTOR Future<Void> masterServer(MasterInterface mi,
Reference<AsyncVar<ServerDBInfo> const> db,
Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> ccInterface,
Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
ServerCoordinators serverCoordinators,
LifetimeToken lifetime,
bool forceRecovery);

View File

@ -1981,7 +1981,7 @@ ACTOR Future<Void> masterCore(Reference<MasterData> self) {
ACTOR Future<Void> masterServer(MasterInterface mi,
Reference<AsyncVar<ServerDBInfo> const> db,
Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> ccInterface,
Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
ServerCoordinators coordinators,
LifetimeToken lifetime,
bool forceRecovery) {

View File

@ -241,10 +241,12 @@ struct UpdateEagerReadInfo {
void addMutation(MutationRef const& m) {
// SOMEDAY: Theoretically we can avoid a read if there is an earlier overlapping ClearRange
if (m.type == MutationRef::ClearRange && !m.param2.startsWith(systemKeys.end))
if (m.type == MutationRef::ClearRange && !m.param2.startsWith(systemKeys.end) &&
SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS)
keyBegin.push_back(m.param2);
else if (m.type == MutationRef::CompareAndClear) {
keyBegin.push_back(keyAfter(m.param1, arena));
if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS)
keyBegin.push_back(keyAfter(m.param1, arena));
if (keys.size() > 0 && keys.back().first == m.param1) {
// Don't issue a second read, if the last read was equal to the current key.
// CompareAndClear is likely to be used after another atomic operation on same key.
@ -260,8 +262,10 @@ struct UpdateEagerReadInfo {
}
void finishKeyBegin() {
std::sort(keyBegin.begin(), keyBegin.end());
keyBegin.resize(std::unique(keyBegin.begin(), keyBegin.end()) - keyBegin.begin());
if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) {
std::sort(keyBegin.begin(), keyBegin.end());
keyBegin.resize(std::unique(keyBegin.begin(), keyBegin.end()) - keyBegin.begin());
}
std::sort(keys.begin(), keys.end(), [](const std::pair<KeyRef, int>& lhs, const std::pair<KeyRef, int>& rhs) {
return (lhs.first < rhs.first) || (lhs.first == rhs.first && lhs.second > rhs.second);
});
@ -2384,21 +2388,22 @@ void getQueuingMetrics(StorageServer* self, StorageQueuingMetricsRequest const&
ACTOR Future<Void> doEagerReads(StorageServer* data, UpdateEagerReadInfo* eager) {
eager->finishKeyBegin();
vector<Future<Key>> keyEnd(eager->keyBegin.size());
for (int i = 0; i < keyEnd.size(); i++)
keyEnd[i] = data->storage.readNextKeyInclusive(eager->keyBegin[i]);
if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) {
vector<Future<Key>> keyEnd(eager->keyBegin.size());
for (int i = 0; i < keyEnd.size(); i++)
keyEnd[i] = data->storage.readNextKeyInclusive(eager->keyBegin[i]);
state Future<vector<Key>> futureKeyEnds = getAll(keyEnd);
state Future<vector<Key>> futureKeyEnds = getAll(keyEnd);
state vector<Key> keyEndVal = wait(futureKeyEnds);
eager->keyEnd = keyEndVal;
}
vector<Future<Optional<Value>>> value(eager->keys.size());
for (int i = 0; i < value.size(); i++)
value[i] = data->storage.readValuePrefix(eager->keys[i].first, eager->keys[i].second);
state Future<vector<Optional<Value>>> futureValues = getAll(value);
state vector<Key> keyEndVal = wait(futureKeyEnds);
vector<Optional<Value>> optionalValues = wait(futureValues);
eager->keyEnd = keyEndVal;
eager->value = optionalValues;
return Void();
@ -2507,7 +2512,7 @@ bool expandMutation(MutationRef& m,
i = d.lastLessOrEqual(m.param2);
if (i && i->isClearTo() && i->getEndKey() >= m.param2) {
m.param2 = i->getEndKey();
} else {
} else if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) {
// Expand to the next set or clear (from storage or latestVersion), and if it
// is a clear, engulf it as well
i = d.lower_bound(m.param2);

View File

@ -315,7 +315,7 @@ struct CompoundWorkload : TestWorkload {
TestWorkload* getWorkloadIface(WorkloadRequest work,
VectorRef<KeyValueRef> options,
Reference<AsyncVar<ServerDBInfo>> dbInfo) {
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
Value testName = getOption(options, LiteralStringRef("testName"), LiteralStringRef("no-test-specified"));
WorkloadContext wcx;
wcx.clientId = work.clientId;
@ -350,7 +350,7 @@ TestWorkload* getWorkloadIface(WorkloadRequest work,
return workload;
}
TestWorkload* getWorkloadIface(WorkloadRequest work, Reference<AsyncVar<ServerDBInfo>> dbInfo) {
TestWorkload* getWorkloadIface(WorkloadRequest work, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
if (work.options.size() < 1) {
TraceEvent(SevError, "TestCreationError").detail("Reason", "No options provided");
fprintf(stderr, "ERROR: No options were provided for workload.\n");
@ -602,7 +602,7 @@ ACTOR Future<Void> runWorkloadAsync(Database cx,
ACTOR Future<Void> testerServerWorkload(WorkloadRequest work,
Reference<ClusterConnectionFile> ccf,
Reference<AsyncVar<struct ServerDBInfo>> dbInfo,
Reference<AsyncVar<struct ServerDBInfo> const> dbInfo,
LocalityData locality) {
state WorkloadInterface workIface;
state bool replied = false;
@ -661,7 +661,7 @@ ACTOR Future<Void> testerServerWorkload(WorkloadRequest work,
ACTOR Future<Void> testerServerCore(TesterInterface interf,
Reference<ClusterConnectionFile> ccf,
Reference<AsyncVar<struct ServerDBInfo>> dbInfo,
Reference<AsyncVar<struct ServerDBInfo> const> dbInfo,
LocalityData locality) {
state PromiseStream<Future<Void>> addWorkload;
state Future<Void> workerFatalError = actorCollection(addWorkload.getFuture());

View File

@ -520,7 +520,7 @@ ACTOR Future<Void> registrationClient(Reference<AsyncVar<Optional<ClusterControl
Reference<AsyncVar<bool> const> degraded,
Reference<ClusterConnectionFile> connFile,
Reference<AsyncVar<std::set<std::string>> const> issues,
LocalConfiguration* localConfig) {
Reference<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.
@ -604,7 +604,7 @@ ACTOR Future<Void> registrationClient(Reference<AsyncVar<Optional<ClusterControl
}
// Returns true if `address` is used in the db (indicated by `dbInfo`) transaction system and in the db's primary DC.
bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference<AsyncVar<ServerDBInfo>> dbInfo) {
bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
const auto& dbi = dbInfo->get();
if (dbi.master.addresses().contains(address)) {
@ -625,7 +625,6 @@ bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference<AsyncVar<S
}
}
for (const auto& grvProxy : dbi.client.grvProxies) {
if (grvProxy.addresses().contains(address)) {
return true;
@ -661,7 +660,7 @@ bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference<AsyncVar<S
return false;
}
bool addressesInDbAndPrimaryDc(const NetworkAddressList& addresses, Reference<AsyncVar<ServerDBInfo>> dbInfo) {
bool addressesInDbAndPrimaryDc(const NetworkAddressList& addresses, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
return addressInDbAndPrimaryDc(addresses.address, dbInfo) ||
(addresses.secondaryAddress.present() && addressInDbAndPrimaryDc(addresses.secondaryAddress.get(), dbInfo));
}
@ -707,13 +706,15 @@ TEST_CASE("/fdbserver/worker/addressInDbAndPrimaryDc") {
// Last, tests that proxies included in the ClientDbInfo are considered as local.
NetworkAddress grvProxyAddress(IPAddress(0x26262626), 1);
GrvProxyInterface grvProxyInterf;
grvProxyInterf.getConsistentReadVersion = RequestStream<struct GetReadVersionRequest>(Endpoint({ grvProxyAddress }, UID(1, 2)));
grvProxyInterf.getConsistentReadVersion =
RequestStream<struct GetReadVersionRequest>(Endpoint({ grvProxyAddress }, UID(1, 2)));
testDbInfo.client.grvProxies.push_back(grvProxyInterf);
ASSERT(addressInDbAndPrimaryDc(grvProxyAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
NetworkAddress commitProxyAddress(IPAddress(0x37373737), 1);
CommitProxyInterface commitProxyInterf;
commitProxyInterf.commit = RequestStream<struct CommitTransactionRequest>(Endpoint({ commitProxyAddress }, UID(1, 2)));
commitProxyInterf.commit =
RequestStream<struct CommitTransactionRequest>(Endpoint({ commitProxyAddress }, UID(1, 2)));
testDbInfo.client.commitProxies.push_back(commitProxyInterf);
ASSERT(addressInDbAndPrimaryDc(commitProxyAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
@ -723,10 +724,10 @@ TEST_CASE("/fdbserver/worker/addressInDbAndPrimaryDc") {
} // namespace
// The actor that actively monitors the health of local and peer servers, and reports anomaly to the cluster controller.
ACTOR Future<Void> healthMonitor(Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> ccInterface,
ACTOR Future<Void> healthMonitor(Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
WorkerInterface interf,
LocalityData locality,
Reference<AsyncVar<ServerDBInfo>> dbInfo) {
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
loop {
Future<Void> nextHealthCheckDelay = Never();
if (dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS &&
@ -959,7 +960,7 @@ ACTOR Future<Void> storageServerRollbackRebooter(std::set<std::pair<UID, KeyValu
UID id,
LocalityData locality,
bool isTss,
Reference<AsyncVar<ServerDBInfo>> db,
Reference<AsyncVar<ServerDBInfo> const> db,
std::string folder,
ActorCollection* filesClosed,
int64_t memoryLimit,
@ -1006,7 +1007,7 @@ ACTOR Future<Void> storageServerRollbackRebooter(std::set<std::pair<UID, KeyValu
ACTOR Future<Void> storageCacheRollbackRebooter(Future<Void> prevStorageCache,
UID id,
LocalityData locality,
Reference<AsyncVar<ServerDBInfo>> db) {
Reference<AsyncVar<ServerDBInfo> const> db) {
loop {
ErrorOr<Void> e = wait(errorOr(prevStorageCache));
if (!e.isError()) {
@ -1212,7 +1213,7 @@ struct SharedLogsValue {
};
ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> ccInterface,
Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
LocalityData locality,
Reference<AsyncVar<ClusterControllerPriorityInfo>> asyncPriorityInfo,
ProcessClass initialClass,
@ -1226,7 +1227,7 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
std::string whitelistBinPaths,
Reference<AsyncVar<ServerDBInfo>> dbInfo,
ConfigDBType configDBType,
LocalConfiguration* localConfig) {
Reference<LocalConfiguration> localConfig) {
state PromiseStream<ErrorInfo> errors;
state Reference<AsyncVar<Optional<DataDistributorInterface>>> ddInterf(
new AsyncVar<Optional<DataDistributorInterface>>());
@ -1912,6 +1913,7 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
startRole(Role::LOG_ROUTER, recruited.id(), interf.id(), details);
DUMPTOKEN(recruited.peekMessages);
DUMPTOKEN(recruited.peekStreamMessages);
DUMPTOKEN(recruited.popMessages);
DUMPTOKEN(recruited.commit);
DUMPTOKEN(recruited.lock);
@ -2044,14 +2046,14 @@ ACTOR Future<Void> workerServer(Reference<ClusterConnectionFile> connFile,
}
}
ACTOR Future<Void> extractClusterInterface(Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> a,
Reference<AsyncVar<Optional<ClusterInterface>>> b) {
ACTOR Future<Void> extractClusterInterface(Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> in,
Reference<AsyncVar<Optional<ClusterInterface>>> out) {
loop {
if (a->get().present())
b->set(a->get().get().clientInterface);
if (in->get().present())
out->set(in->get().get().clientInterface);
else
b->set(Optional<ClusterInterface>());
wait(a->onChange());
out->set(Optional<ClusterInterface>());
wait(in->onChange());
}
}
@ -2086,7 +2088,7 @@ ACTOR Future<Void> printTimeout() {
return Void();
}
ACTOR Future<Void> printOnFirstConnected(Reference<AsyncVar<Optional<ClusterInterface>>> ci) {
ACTOR Future<Void> printOnFirstConnected(Reference<AsyncVar<Optional<ClusterInterface>> const> ci) {
state Future<Void> timeoutFuture = printTimeout();
loop {
choose {
@ -2372,14 +2374,15 @@ ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
ConfigDBType configDBType) {
state vector<Future<Void>> actors;
state Promise<Void> recoveredDiskFiles;
state LocalConfiguration localConfig(dataFolder, configPath, manualKnobOverrides);
state Reference<LocalConfiguration> localConfig =
makeReference<LocalConfiguration>(dataFolder, configPath, manualKnobOverrides);
// setupStackSignal();
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::Worker;
// FIXME: Initializing here causes simulation issues, these must be fixed
/*
if (configDBType != ConfigDBType::DISABLED) {
wait(localConfig.initialize());
wait(localConfig->initialize());
}
*/
@ -2450,7 +2453,7 @@ ACTOR Future<Void> fdbd(Reference<ClusterConnectionFile> connFile,
whitelistBinPaths,
dbInfo,
configDBType,
&localConfig),
localConfig),
"WorkerServer",
UID(),
&normalWorkerErrors()));

View File

@ -84,6 +84,8 @@ struct KillRegionWorkload : TestWorkload {
TraceEvent("ForceRecovery_Wait").log();
wait(delay(deterministicRandom()->random01() * self->testDuration));
// FIXME: killDataCenter breaks simulation if forceKill=false, since some processes can survive and
// partially complete a recovery
g_simulator.killDataCenter(LiteralStringRef("0"),
deterministicRandom()->random01() < 0.5 ? ISimulator::KillInstantly
: ISimulator::RebootAndDelete,

View File

@ -224,7 +224,7 @@ struct ReadWriteWorkload : KVWorkload {
Future<Void> setup(Database const& cx) override { return _setup(cx, this); }
Future<Void> start(Database const& cx) override { return _start(cx, this); }
ACTOR static Future<bool> traceDumpWorkers(Reference<AsyncVar<ServerDBInfo>> db) {
ACTOR static Future<bool> traceDumpWorkers(Reference<AsyncVar<ServerDBInfo> const> db) {
try {
loop {
choose {

View File

@ -19,7 +19,7 @@
*/
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
@ -83,7 +83,7 @@ struct TagThrottleApiWorkload : TestWorkload {
tagSet.addTag(tag);
try {
wait(ThrottleApi::throttleTags(cx,
wait(ThrottleApi::throttleTags(cx.getReference(),
tagSet,
rate,
duration,
@ -137,7 +137,7 @@ struct TagThrottleApiWorkload : TestWorkload {
}
}
bool removed = wait(ThrottleApi::unthrottleTags(cx, tagSet, throttleType, priority));
bool removed = wait(ThrottleApi::unthrottleTags(cx.getReference(), tagSet, throttleType, priority));
if (removed) {
ASSERT(erased || !throttleType.present() || throttleType.get() == TagThrottleType::AUTO);
} else {
@ -151,7 +151,9 @@ struct TagThrottleApiWorkload : TestWorkload {
TagThrottleApiWorkload* self,
Database cx,
std::map<std::pair<TransactionTag, TransactionPriority>, TagThrottleInfo> const* manuallyThrottledTags) {
std::vector<TagThrottleInfo> tags = wait(ThrottleApi::getThrottledTags(cx, CLIENT_KNOBS->TOO_MANY));
std::vector<TagThrottleInfo> tags =
wait(ThrottleApi::getThrottledTags(cx.getReference(), CLIENT_KNOBS->TOO_MANY));
int manualThrottledTags = 0;
int activeAutoThrottledTags = 0;
@ -184,7 +186,8 @@ struct TagThrottleApiWorkload : TestWorkload {
}
ACTOR Future<Void> getRecommendedTags(TagThrottleApiWorkload* self, Database cx) {
std::vector<TagThrottleInfo> tags = wait(ThrottleApi::getRecommendedTags(cx, CLIENT_KNOBS->TOO_MANY));
std::vector<TagThrottleInfo> tags =
wait(ThrottleApi::getRecommendedTags(cx.getReference(), CLIENT_KNOBS->TOO_MANY));
for (auto& tag : tags) {
ASSERT(tag.throttleType == TagThrottleType::AUTO);
@ -200,7 +203,7 @@ struct TagThrottleApiWorkload : TestWorkload {
deterministicRandom()->coinflip() ? Optional<TransactionPriority>()
: deterministicRandom()->randomChoice(allTransactionPriorities);
bool unthrottled = wait(ThrottleApi::unthrottleAll(cx, throttleType, priority));
bool unthrottled = wait(ThrottleApi::unthrottleAll(cx.getReference(), throttleType, priority));
if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) {
bool unthrottleExpected = false;
bool empty = manuallyThrottledTags->empty();
@ -227,15 +230,16 @@ struct TagThrottleApiWorkload : TestWorkload {
}
ACTOR Future<Void> enableAutoThrottling(TagThrottleApiWorkload* self, Database cx) {
state Reference<DatabaseContext> db = cx.getReference();
if (deterministicRandom()->coinflip()) {
wait(ThrottleApi::enableAuto(cx, true));
wait(ThrottleApi::enableAuto(db, true));
self->autoThrottleEnabled = true;
if (deterministicRandom()->coinflip()) {
bool unthrottled =
wait(ThrottleApi::unthrottleAll(cx, TagThrottleType::AUTO, Optional<TransactionPriority>()));
wait(ThrottleApi::unthrottleAll(db, TagThrottleType::AUTO, Optional<TransactionPriority>()));
}
} else {
wait(ThrottleApi::enableAuto(cx, false));
wait(ThrottleApi::enableAuto(db, false));
self->autoThrottleEnabled = false;
}

View File

@ -23,7 +23,7 @@
#include "fdbserver/workloads/BulkSetup.actor.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/TagThrottle.h"
#include "fdbclient/TagThrottle.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
constexpr int SAMPLE_SIZE = 10000;
@ -100,7 +100,7 @@ struct WriteTagThrottlingWorkload : KVWorkload {
wait(bulkSetup(cx, self, self->keyCount, Promise<double>()));
}
if (self->clientId == 0) {
wait(ThrottleApi::enableAuto(cx, true));
wait(ThrottleApi::enableAuto(cx.getReference(), true));
}
return Void();
}
@ -306,9 +306,10 @@ struct WriteTagThrottlingWorkload : KVWorkload {
}
ACTOR static Future<Void> throttledTagUpdater(Database cx, WriteTagThrottlingWorkload* self) {
state std::vector<TagThrottleInfo> tags;
state Reference<DatabaseContext> db = cx.getReference();
loop {
wait(delay(1.0));
wait(store(tags, ThrottleApi::getThrottledTags(cx, CLIENT_KNOBS->TOO_MANY, true)));
wait(store(tags, ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY, true)));
self->recordThrottledTags(tags);
};
}

View File

@ -49,7 +49,7 @@ struct WorkloadContext {
Standalone<VectorRef<KeyValueRef>> options;
int clientId, clientCount;
int64_t sharedRandomNumber;
Reference<AsyncVar<struct ServerDBInfo>> dbInfo;
Reference<AsyncVar<struct ServerDBInfo> const> dbInfo;
WorkloadContext();
WorkloadContext(const WorkloadContext&);

View File

@ -627,6 +627,13 @@ Future<T> safeThreadFutureToFuture(ThreadFuture<T> threadFuture) {
return threadFuture.get();
}
// do nothing, just for template functions' calls
template <class T>
Future<T> safeThreadFutureToFuture(Future<T> future) {
// do nothing
return future;
}
// Helper actor. Do not use directly!
namespace internal_thread_helper {

View File

@ -599,4 +599,5 @@ extern TraceBatch g_traceBatch;
#define DUMPTOKEN(name) \
TraceEvent("DumpToken", recruited.id()).detail("Name", #name).detail("Token", name.getEndpoint().token)
#define DisabledTraceEvent(...) false && TraceEvent()
#endif

View File

@ -75,10 +75,10 @@ Performance issues:
- When waiting for a number of things, wait a little extra time to get
the stragglers. (See the SmartQuorum() generic actor)
- If asking another asynch server to do units of work, don't queue up more
- If asking another asynchronous server to do units of work, don't queue up more
work than is necessary to keep the server busy. Likewise, if you are
busy, let your own work queue fill up to signal your requestor
busy, let your own work queue fill up to signal your requester
that you are blocked. Also do this personally with managers assigning
you stuff.
- Pass all variables as "const &" if thier size is greater than 8 bytes.
- Pass all variables as "const &" if their size is greater than 8 bytes.

View File

@ -150,13 +150,13 @@
</Component>
<Component Id='FDBCRegistryValue' Guid='{6ED940F3-75C8-4385-97D9-D7D0F211B17D}' Win64='yes'>
<RegistryKey Root='HKLM' Key='SOFTWARE\$(var.Manufacturer)\KeyValue\Client' Action='createAndRemoveOnUninstall'>
<RegistryKey Root='HKLM' Key='SOFTWARE\$(var.Manufacturer)\KeyValue\Client'>
<RegistryValue Name='Version' Type='string' Value='$(var.Version)' KeyPath='yes' />
</RegistryKey>
</Component>
<Component Id='FDBSRegistryValue' Guid='{361A9B4A-A06F-4BFB-AFEA-B5F733C8BFDF}' Win64='yes'>
<RegistryKey Root='HKLM' Key='SOFTWARE\$(var.Manufacturer)\KeyValue\Server' Action='createAndRemoveOnUninstall'>
<RegistryKey Root='HKLM' Key='SOFTWARE\$(var.Manufacturer)\KeyValue\Server'>
<RegistryValue Name='Version' Type='string' Value='$(var.Version)' KeyPath='yes' />
</RegistryKey>
</Component>

View File

@ -109,6 +109,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES pt.TXT IGNORE)
add_fdb_test(TEST_FILES randomSelector.txt IGNORE)
add_fdb_test(TEST_FILES selectorCorrectness.txt IGNORE)
add_fdb_test(TEST_FILES IThreadPool.txt IGNORE)
add_fdb_test(TEST_FILES fast/AtomicBackupCorrectness.toml)
add_fdb_test(TEST_FILES fast/AtomicBackupToDBCorrectness.toml)
add_fdb_test(TEST_FILES fast/AtomicOps.toml)