Merge branch 'master' of https://github.com/apple/foundationdb into redwood-header-changes

This commit is contained in:
Steve Atherton 2021-12-11 19:41:51 -08:00
commit 702c26d8ab
51 changed files with 914 additions and 517 deletions

View File

@ -81,6 +81,30 @@ extern "C" DLLEXPORT fdb_bool_t fdb_error_predicate(int predicate_test, fdb_erro
return false;
}
#define RETURN_FUTURE_ON_ERROR(return_type, code_to_run) \
try { \
code_to_run \
} catch (Error & e) { \
if (e.code() <= 0) \
return ((FDBFuture*)(ThreadFuture<return_type>(internal_error())).extractPtr()); \
else \
return ((FDBFuture*)(ThreadFuture<return_type>(e)).extractPtr()); \
} catch (...) { \
return ((FDBFuture*)(ThreadFuture<return_type>(unknown_error())).extractPtr()); \
}
#define RETURN_RESULT_ON_ERROR(return_type, code_to_run) \
try { \
code_to_run \
} catch (Error & e) { \
if (e.code() <= 0) \
return ((FDBResult*)(ThreadResult<return_type>(internal_error())).extractPtr()); \
else \
return ((FDBResult*)(ThreadResult<return_type>(e)).extractPtr()); \
} catch (...) { \
return ((FDBResult*)(ThreadResult<return_type>(unknown_error())).extractPtr()); \
}
#define RETURN_ON_ERROR(code_to_run) \
try { \
code_to_run \
@ -731,9 +755,10 @@ extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_estimated_range_size_bytes(F
int begin_key_name_length,
uint8_t const* end_key_name,
int end_key_name_length) {
// FIXME: this can throw inverted_range()
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
return (FDBFuture*)(TXN(tr)->getEstimatedRangeSizeBytes(range).extractPtr());
RETURN_FUTURE_ON_ERROR(
int64_t,
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
return (FDBFuture*)(TXN(tr)->getEstimatedRangeSizeBytes(range).extractPtr()););
}
extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_range_split_points(FDBTransaction* tr,
@ -742,9 +767,10 @@ extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_range_split_points(FDBTransa
uint8_t const* end_key_name,
int end_key_name_length,
int64_t chunk_size) {
// FIXME: this can throw inverted_range()
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
return (FDBFuture*)(TXN(tr)->getRangeSplitPoints(range, chunk_size).extractPtr());
RETURN_FUTURE_ON_ERROR(
Standalone<VectorRef<KeyRef>>,
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
return (FDBFuture*)(TXN(tr)->getRangeSplitPoints(range, chunk_size).extractPtr()););
}
extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_blob_granule_ranges(FDBTransaction* tr,
@ -752,9 +778,10 @@ extern "C" DLLEXPORT FDBFuture* fdb_transaction_get_blob_granule_ranges(FDBTrans
int begin_key_name_length,
uint8_t const* end_key_name,
int end_key_name_length) {
// FIXME: this can throw inverted_range()
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
return (FDBFuture*)(TXN(tr)->getBlobGranuleRanges(range).extractPtr());
RETURN_FUTURE_ON_ERROR(
Standalone<VectorRef<KeyRangeRef>>,
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
return (FDBFuture*)(TXN(tr)->getBlobGranuleRanges(range).extractPtr()););
}
extern "C" DLLEXPORT FDBResult* fdb_transaction_read_blob_granules(FDBTransaction* tr,
@ -765,23 +792,22 @@ extern "C" DLLEXPORT FDBResult* fdb_transaction_read_blob_granules(FDBTransactio
int64_t beginVersion,
int64_t readVersion,
FDBReadBlobGranuleContext granule_context) {
// FIXME: this can throw inverted_range()
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
RETURN_RESULT_ON_ERROR(
RangeResult,
KeyRangeRef range(KeyRef(begin_key_name, begin_key_name_length), KeyRef(end_key_name, end_key_name_length));
// FIXME: better way to convert?
ReadBlobGranuleContext context;
context.userContext = granule_context.userContext;
context.start_load_f = granule_context.start_load_f;
context.get_load_f = granule_context.get_load_f;
context.free_load_f = granule_context.free_load_f;
context.debugNoMaterialize = granule_context.debugNoMaterialize;
// FIXME: better way to convert?
ReadBlobGranuleContext context;
context.userContext = granule_context.userContext;
context.start_load_f = granule_context.start_load_f;
context.get_load_f = granule_context.get_load_f;
context.free_load_f = granule_context.free_load_f;
context.debugNoMaterialize = granule_context.debugNoMaterialize;
Optional<Version> rv;
if (readVersion != invalidVersion) {
rv = readVersion;
}
Optional<Version> rv;
if (readVersion != invalidVersion) { rv = readVersion; }
return (FDBResult*)(TXN(tr)->readBlobGranules(range, beginVersion, rv, context).extractPtr());
return (FDBResult*)(TXN(tr)->readBlobGranules(range, beginVersion, rv, context).extractPtr()););
}
#include "fdb_c_function_pointers.g.h"

View File

@ -70,7 +70,38 @@ FILE* debugme; /* descriptor used for debug messages */
/* unretryable error */ \
fprintf(stderr, "ERROR: fdb_transaction_on_error returned %d at %s:%d\n", err2, __FILE__, __LINE__); \
fdb_transaction_reset(_t); \
/* TODO: if we adda retry limit in the future, \
/* TODO: if we add a retry limit in the future, \
* handle the conflict stats properly. \
*/ \
return FDB_ERROR_ABORT; \
} \
if (err == 1020 /* not_committed */) { \
return FDB_ERROR_CONFLICT; \
} \
return FDB_ERROR_RETRY; \
} \
} while (0)
#define fdb_handle_result_error(_func, err, _t) \
do { \
if (err) { \
int err2; \
FDBFuture* fErr; \
if ((err != 1020 /* not_committed */) && (err != 1021 /* commit_unknown_result */) && \
(err != 1213 /* tag_throttled */)) { \
fprintf(stderr, "ERROR: Error %s (%d) occured at %s\n", #_func, err, fdb_get_error(err)); \
} else { \
fprintf(annoyme, "ERROR: Error %s (%d) occured at %s\n", #_func, err, fdb_get_error(err)); \
} \
fErr = fdb_transaction_on_error(_t, err); \
/* this will return the original error for non-retryable errors */ \
err2 = wait_future(fErr); \
fdb_future_destroy(fErr); \
if (err2) { \
/* unretryable error */ \
fprintf(stderr, "ERROR: fdb_transaction_on_error returned %d at %s:%d\n", err2, __FILE__, __LINE__); \
fdb_transaction_reset(_t); \
/* TODO: if we add a retry limit in the future, \
* handle the conflict stats properly. \
*/ \
return FDB_ERROR_ABORT; \
@ -666,10 +697,9 @@ int run_op_read_blob_granules(FDBTransaction* transaction,
if (err) {
if (err != 2037 /* blob_granule_not_materialized */) {
fprintf(stderr, "ERROR: fdb_result_get_keyvalue_array: %s\n", fdb_get_error(err));
fdb_result_destroy(r);
return FDB_ERROR_RETRY;
fdb_handle_result_error(fdb_transaction_read_blob_granules, err, transaction);
} else {
fdb_result_destroy(r);
return FDB_SUCCESS;
}
}

View File

@ -58,7 +58,7 @@ fdb_error_t wait_future(fdb::Future& f) {
void validateTimeoutDuration(double expectedSeconds, std::chrono::time_point<std::chrono::steady_clock> start) {
std::chrono::duration<double> duration = std::chrono::steady_clock::now() - start;
double actualSeconds = duration.count();
CHECK(actualSeconds >= expectedSeconds - 1e-6);
CHECK(actualSeconds >= expectedSeconds - 1e-3);
CHECK(actualSeconds < expectedSeconds * 2);
}

View File

@ -188,7 +188,7 @@ def kill(logger):
# and then specify the certain process to kill
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())
output2, err = process.communicate(input='kill; kill {}; sleep 1\n'.format(address).encode())
logger.debug(output2)
# wait for a second for the cluster recovery
time.sleep(1)
@ -218,7 +218,7 @@ def suspend(logger):
logger.debug("Pid: {}".format(pid))
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())
output2, err = process.communicate(input='suspend; suspend 3600 {}; sleep 1\n'.format(address).encode())
# the cluster should be unavailable after the only process being suspended
assert not get_value_from_status_json(False, 'client', 'database_status', 'available')
# check the process pid still exists

View File

@ -210,7 +210,7 @@ set(DEFAULT_COROUTINE_IMPL boost)
if(WIN32)
# boost coroutine not available in windows build environment for now.
set(DEFAULT_COROUTINE_IMPL libcoro)
elseif(NOT APPLE AND CMAKE_HOST_SYSTEM_PROCESSOR MATCHES "^x86")
elseif(NOT APPLE AND NOT USE_SANITIZER AND CMAKE_HOST_SYSTEM_PROCESSOR MATCHES "^x86")
# revert to libcoro for x86 linux while we investigate a performance regression
set(DEFAULT_COROUTINE_IMPL libcoro)
endif()

View File

@ -2,6 +2,12 @@
Release Notes
#############
6.3.23
======
* Fixed a bug that remoteDCIsHealthy logic is not guarded by CC_ENABLE_WORKER_HEALTH_MONITOR, which may prevent HA failback. `(PR #6106) <https://github.com/apple/foundationdb/pull/6106>`_
* Fixed a race condition with updating the coordinated state and updating the master registration. `(PR #6088) <https://github.com/apple/foundationdb/pull/6088>`_
* Changed dbinfo broadcast to be explicitly requested by the worker registration message. `(PR #6073) <https://github.com/apple/foundationdb/pull/6073>`_
6.3.22
======
* Added histograms to client GRV batcher. `(PR #5760) <https://github.com/apple/foundationdb/pull/5760>`_

View File

@ -48,14 +48,6 @@ std::string trim(std::string const& connectionString) {
return trimmed;
}
std::string trimFromHostname(std::string const& networkAddress) {
const auto& pos = networkAddress.find("(fromHostname)");
if (pos != std::string::npos) {
return networkAddress.substr(0, pos);
}
return networkAddress;
}
} // namespace
FDB_DEFINE_BOOLEAN_PARAM(ConnectionStringNeedsPersisted);
@ -280,7 +272,7 @@ std::string ClusterConnectionString::toString() const {
if (i) {
s += ',';
}
s += trimFromHostname(coord[i].toString());
s += coord[i].toString();
}
return s;
}

View File

@ -923,8 +923,7 @@ ThreadResult<RangeResult> MultiVersionTransaction::readBlobGranules(const KeyRan
if (tr.transaction) {
return tr.transaction->readBlobGranules(keyRange, beginVersion, readVersion, granuleContext);
} else {
// FIXME: handle abortable future + timeout properly
return ThreadResult<RangeResult>(transaction_cancelled());
return abortableTimeoutResult<RangeResult>(tr.onChange);
}
}
@ -1123,6 +1122,14 @@ ThreadFuture<T> MultiVersionTransaction::makeTimeout() {
});
}
template <class T>
ThreadResult<T> MultiVersionTransaction::abortableTimeoutResult(ThreadFuture<Void> abortSignal) {
ThreadFuture<T> timeoutFuture = makeTimeout<T>();
ThreadFuture<T> abortable = abortableFuture(timeoutFuture, abortSignal);
abortable.blockUntilReadyCheckOnMainThread();
return ThreadResult<T>((ThreadSingleAssignmentVar<T>*)abortable.extractPtr());
}
void MultiVersionTransaction::reset() {
persistentOptions.clear();

View File

@ -512,6 +512,9 @@ private:
template <class T>
ThreadFuture<T> makeTimeout();
template <class T>
ThreadResult<T> abortableTimeoutResult(ThreadFuture<Void> abortSignal);
TransactionInfo transaction;
TransactionInfo getTransaction();

View File

@ -6537,9 +6537,6 @@ ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> getBlobGranuleRangesActor(Trans
}
Future<Standalone<VectorRef<KeyRangeRef>>> Transaction::getBlobGranuleRanges(const KeyRange& range) {
if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) {
throw client_invalid_operation();
}
return ::getBlobGranuleRangesActor(this, range);
}
@ -6708,9 +6705,6 @@ Future<Standalone<VectorRef<BlobGranuleChunkRef>>> Transaction::readBlobGranules
Version begin,
Optional<Version> readVersion,
Version* readVersionOut) {
if (!CLIENT_KNOBS->ENABLE_BLOB_GRANULES) {
throw client_invalid_operation();
}
return readBlobGranulesActor(cx, this, range, begin, readVersion, readVersionOut);
}
@ -7376,8 +7370,8 @@ ACTOR Future<Void> getChangeFeedStreamActor(Reference<DatabaseContext> db,
if (locations.size() > 1) {
std::vector<std::pair<StorageServerInterface, KeyRange>> interfs;
for (int i = 0; i < locations.size(); i++) {
interfs.push_back(std::make_pair(locations[i].second->getInterface(chosenLocations[i]),
locations[i].first & range));
interfs.emplace_back(locations[i].second->getInterface(chosenLocations[i]),
locations[i].first & range);
}
wait(mergeChangeFeedStream(db, interfs, results, rangeID, &begin, end) || cx->connectionFileChanged());
} else {

View File

@ -38,7 +38,7 @@ struct Notified {
if (val >= limit)
return Void();
Promise<Void> p;
waiting.push(std::make_pair(limit, p));
waiting.emplace(limit, p);
return p.getFuture();
}

View File

@ -305,8 +305,6 @@ ThreadResult<RangeResult> ThreadSafeTransaction::readBlobGranules(const KeyRange
// FIXME: prevent from calling this from another main thread!
bool doMaterialize = !granule_context.debugNoMaterialize;
ISingleThreadTransaction* tr = this->tr;
KeyRange r = keyRange;
@ -328,10 +326,10 @@ ThreadResult<RangeResult> ThreadSafeTransaction::readBlobGranules(const KeyRange
Standalone<VectorRef<BlobGranuleChunkRef>> files = getFilesFuture.get();
// do this work off of fdb network threads for performance!
if (doMaterialize) {
return loadAndMaterializeBlobGranules(files, keyRange, beginVersion, readVersionOut, granule_context);
} else {
if (granule_context.debugNoMaterialize) {
return ThreadResult<RangeResult>(blob_granule_not_materialized());
} else {
return loadAndMaterializeBlobGranules(files, keyRange, beginVersion, readVersionOut, granule_context);
}
}

View File

@ -209,6 +209,7 @@ set(FDBSERVER_SRCS
workloads/MemoryKeyValueStore.h
workloads/MemoryLifetime.actor.cpp
workloads/MetricLogging.actor.cpp
workloads/MiniCycle.actor.cpp
workloads/MutationLogReaderCorrectness.actor.cpp
workloads/GetRangeAndMap.actor.cpp
workloads/ParallelRestore.actor.cpp

View File

@ -436,11 +436,11 @@ public:
// try to avoid fields in the cluster controller datacenter if everything else is equal
for (auto& it : field_count) {
fieldQueue.push(std::make_tuple(it.second.first, it.second.second, it.first));
fieldQueue.emplace(it.second.first, it.second.second, it.first);
}
for (auto& it : zone_count) {
field_zoneQueue[it.second.second].push(std::make_pair(it.second.first, it.first));
field_zoneQueue[it.second.second].emplace(it.second.first, it.first);
}
// start with the least used field, and try to find a worker with that field
@ -500,7 +500,7 @@ public:
}
for (auto& it : zone_count) {
zoneQueue.push(std::make_pair(it.second, it.first));
zoneQueue.emplace(it.second, it.first);
}
while (zoneQueue.size()) {
@ -3068,6 +3068,11 @@ public:
// Returns true if remote DC is healthy and can failover to.
bool remoteDCIsHealthy() {
// Ignore remote DC health if worker health monitor is disabled.
if (!SERVER_KNOBS->CC_ENABLE_WORKER_HEALTH_MONITOR) {
return true;
}
// When we just start, we ignore any remote DC health info since the current CC may be elected at wrong DC due
// to that all the processes are still starting.
if (machineStartTime() == 0) {
@ -3954,7 +3959,7 @@ void clusterRecruitStorage(ClusterControllerData* self, RecruitStorageRequest re
req.reply.send(rep);
} catch (Error& e) {
if (e.code() == error_code_no_more_servers) {
self->outstandingStorageRequests.push_back(std::make_pair(req, now() + SERVER_KNOBS->RECRUITMENT_TIMEOUT));
self->outstandingStorageRequests.emplace_back(req, now() + SERVER_KNOBS->RECRUITMENT_TIMEOUT);
TraceEvent(SevWarn, "RecruitStorageNotAvailable", self->id)
.detail("IsCriticalRecruitment", req.criticalRecruitment)
.error(e);
@ -3978,8 +3983,7 @@ void clusterRecruitBlobWorker(ClusterControllerData* self, RecruitBlobWorkerRequ
req.reply.send(rep);
} catch (Error& e) {
if (e.code() == error_code_no_more_servers) {
self->outstandingBlobWorkerRequests.push_back(
std::make_pair(req, now() + SERVER_KNOBS->RECRUITMENT_TIMEOUT));
self->outstandingBlobWorkerRequests.emplace_back(req, now() + SERVER_KNOBS->RECRUITMENT_TIMEOUT);
TraceEvent(SevWarn, "RecruitBlobWorkerNotAvailable", self->id).error(e);
} else {
TraceEvent(SevError, "RecruitBlobWorkerError", self->id).error(e);
@ -4486,7 +4490,7 @@ ACTOR Future<Void> statusServer(FutureStream<StatusRequest> requests,
for (auto& it : self->id_worker) {
workers.push_back(it.second.details);
if (it.second.issues.size()) {
workerIssues.push_back(ProcessIssues(it.second.details.interf.address(), it.second.issues));
workerIssues.emplace_back(it.second.details.interf.address(), it.second.issues);
}
}

View File

@ -216,24 +216,28 @@ ACTOR Future<Void> openDatabase(ClientData* db,
ClientStatusInfo(req.traceLogGroup, req.supportedVersions, req.issues);
}
while (db->clientInfo->get().read().id == req.knownClientInfoID &&
!db->clientInfo->get().read().forward.present()) {
while (!db->clientInfo->get().read().id.isValid() || (db->clientInfo->get().read().id == req.knownClientInfoID &&
!db->clientInfo->get().read().forward.present())) {
choose {
when(wait(checkStuck)) {
replyContents = failed_to_progress();
break;
}
when(wait(yieldedFuture(db->clientInfo->onChange()))) {}
when(wait(yieldedFuture(db->clientInfo->onChange()))) { replyContents = db->clientInfo->get(); }
when(wait(delayJittered(SERVER_KNOBS->CLIENT_REGISTER_INTERVAL))) {
if (req.supportedVersions.size() > 0) {
db->clientStatusInfoMap.erase(req.reply.getEndpoint().getPrimaryAddress());
if (db->clientInfo->get().read().id.isValid()) {
replyContents = db->clientInfo->get();
}
replyContents = db->clientInfo->get();
// Otherwise, we still break out of the loop and return a default_error_or.
break;
} // The client might be long gone!
}
}
if (req.supportedVersions.size() > 0) {
db->clientStatusInfoMap.erase(req.reply.getEndpoint().getPrimaryAddress());
}
if (replyContents.present()) {
req.reply.send(replyContents.get());
} else {
@ -295,7 +299,8 @@ ACTOR Future<Void> leaderRegister(LeaderElectionRegInterface interf, Key key) {
loop choose {
when(OpenDatabaseCoordRequest req = waitNext(interf.openDatabase.getFuture())) {
if (clientData.clientInfo->get().read().id != req.knownClientInfoID &&
if (clientData.clientInfo->get().read().id.isValid() &&
clientData.clientInfo->get().read().id != req.knownClientInfoID &&
!clientData.clientInfo->get().read().forward.present()) {
req.reply.send(clientData.clientInfo->get());
} else {

View File

@ -361,7 +361,7 @@ ACTOR Future<Void> readHotDetector(DataDistributionTracker* self) {
/*
ACTOR Future<Void> extrapolateShardBytes( Reference<AsyncVar<Optional<int64_t>>> inBytes,
Reference<AsyncVar<Optional<int64_t>>> outBytes ) { state std::deque< std::pair<double,int64_t> > past; loop { wait(
inBytes->onChange() ); if( inBytes->get().present() ) { past.push_back( std::make_pair(now(),inBytes->get().get()) ); if
inBytes->onChange() ); if( inBytes->get().present() ) { past.emplace_back(now(),inBytes->get().get()); if
(past.size() < 2) outBytes->set( inBytes->get() ); else { while (past.size() > 1 && past.end()[-1].first -
past.begin()[1].first > 1.0) past.pop_front(); double rate = std::max(0.0,
double(past.end()[-1].second-past.begin()[0].second)/(past.end()[-1].first - past.begin()[0].first)); outBytes->set(
@ -434,21 +434,23 @@ ACTOR Future<Void> changeSizes(DataDistributionTracker* self, KeyRange keys, int
struct HasBeenTrueFor : ReferenceCounted<HasBeenTrueFor> {
explicit HasBeenTrueFor(const Optional<ShardMetrics>& value) {
if (value.present()) {
trigger = delayJittered(std::max(0.0,
SERVER_KNOBS->DD_MERGE_COALESCE_DELAY +
value.get().lastLowBandwidthStartTime - now()),
TaskPriority::DataDistributionLow) ||
cleared.getFuture();
lowBandwidthStartTime = value.get().lastLowBandwidthStartTime;
trigger =
delayJittered(std::max(0.0, SERVER_KNOBS->DD_MERGE_COALESCE_DELAY + lowBandwidthStartTime - now()),
TaskPriority::DataDistributionLow) ||
cleared.getFuture();
}
}
Future<Void> set(double lastLowBandwidthStartTime) {
if (!trigger.isValid()) {
if (!trigger.isValid() || lowBandwidthStartTime != lastLowBandwidthStartTime) {
cleared = Promise<Void>();
trigger =
delayJittered(SERVER_KNOBS->DD_MERGE_COALESCE_DELAY + std::max(lastLowBandwidthStartTime - now(), 0.0),
TaskPriority::DataDistributionLow) ||
cleared.getFuture();
lowBandwidthStartTime = lastLowBandwidthStartTime;
}
return trigger;
}
@ -458,12 +460,14 @@ struct HasBeenTrueFor : ReferenceCounted<HasBeenTrueFor> {
}
trigger = Future<Void>();
cleared.send(Void());
lowBandwidthStartTime = 0;
}
// True if this->value is true and has been true for this->seconds
bool hasBeenTrueForLongEnough() const { return trigger.isValid() && trigger.isReady(); }
private:
double lowBandwidthStartTime = 0;
Future<Void> trigger;
Promise<Void> cleared;
};

View File

@ -267,10 +267,12 @@ struct RocksDBKeyValueStore : IKeyValueStore {
a.done.sendError(statusToError(status));
} else {
TraceEvent(SevInfo, "RocksDB").detail("Path", a.path).detail("Method", "Open");
// The current thread and main thread are same when the code runs in simulation.
// blockUntilReady() is getting the thread into deadlock state, so avoiding the
// metric logger in simulation.
if (!g_network->isSimulated()) {
if (g_network->isSimulated()) {
// The current thread and main thread are same when the code runs in simulation.
// blockUntilReady() is getting the thread into deadlock state, so directly calling
// the metricsLogger.
a.metrics = rocksDBMetricLogger(options.statistics, db) && flowLockLogger(a.readLock, a.fetchLock);
} else {
onMainThread([&] {
a.metrics =
rocksDBMetricLogger(options.statistics, db) && flowLockLogger(a.readLock, a.fetchLock);

View File

@ -4892,14 +4892,14 @@ public:
}
// Start reading the page, without caching
entries.push_back(std::make_pair(q.get(),
self->readPage(PagerEventReasons::LazyClear,
q.get().height,
snapshot,
q.get().pageID,
ioLeafPriority,
true,
false)));
entries.emplace_back(q.get(),
self->readPage(PagerEventReasons::LazyClear,
q.get().height,
snapshot,
q.get().pageID,
ioLeafPriority,
true,
false));
--toPop;
}

View File

@ -1932,6 +1932,7 @@ ACTOR Future<Void> changeFeedStreamQ(StorageServer* data, ChangeFeedStreamReques
// server, also check that the stream is closed
when(wait(req.end == std::numeric_limits<Version>::max() ? Future<Void>(Never())
: data->version.whenAtLeast(req.end))) {}
when(wait(delay(5.0))) {} // TODO REMOVE this once empty version logic is fully implemented
}
auto feed = data->uidChangeFeed.find(req.rangeID);
if (feed == data->uidChangeFeed.end() || feed->second->removing) {
@ -5232,8 +5233,8 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
if (cloneCursor2->version().version > ver && cloneCursor2->version().version > data->version.get()) {
++data->counters.updateVersions;
if (data->currentChangeFeeds.size()) {
data->changeFeedVersions.push_back(std::make_pair(
std::vector<Key>(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver));
data->changeFeedVersions.emplace_back(
std::vector<Key>(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver);
updatedChangeFeeds.insert(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end());
data->currentChangeFeeds.clear();
}
@ -5319,8 +5320,8 @@ ACTOR Future<Void> update(StorageServer* data, bool* pReceivedUpdate) {
}
data->tLogMsgsPTreeUpdatesLatencyHistogram->sampleSeconds(now() - beforeTLogMsgsUpdates);
if (data->currentChangeFeeds.size()) {
data->changeFeedVersions.push_back(std::make_pair(
std::vector<Key>(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver));
data->changeFeedVersions.emplace_back(
std::vector<Key>(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end()), ver);
updatedChangeFeeds.insert(data->currentChangeFeeds.begin(), data->currentChangeFeeds.end());
data->currentChangeFeeds.clear();
}

View File

@ -170,11 +170,13 @@ Future<Void> bulkSetup(Database cx,
Promise<std::vector<std::pair<uint64_t, double>>> ratesAtKeyCounts =
Promise<std::vector<std::pair<uint64_t, double>>>(),
int keySaveIncrement = 0,
double keyCheckInterval = 0.1) {
double keyCheckInterval = 0.1,
uint64_t startNodeIdx = 0,
uint64_t endNodeIdx = 0) {
state std::vector<std::pair<uint64_t, uint64_t>> jobs;
state uint64_t startNode = (nodeCount * workload->clientId) / workload->clientCount;
state uint64_t endNode = (nodeCount * (workload->clientId + 1)) / workload->clientCount;
state uint64_t startNode = startNodeIdx ? startNodeIdx : (nodeCount * workload->clientId) / workload->clientCount;
state uint64_t endNode = endNodeIdx ? endNodeIdx : (nodeCount * (workload->clientId + 1)) / workload->clientCount;
state double start = now();

View File

@ -0,0 +1,342 @@
/*
* MiniCycle.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2021 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/workloads/BulkSetup.actor.h"
#include "flow/Arena.h"
#include "flow/IRandom.h"
#include "flow/Trace.h"
#include "flow/actorcompiler.h" // This must be the last #include.
#include "flow/serialize.h"
#include <cstring>
struct MiniCycleWorkload : TestWorkload {
int actorCount, nodeCount;
double testDuration, transactionsPerSecond, minExpectedTransactionsPerSecond, traceParentProbability;
Key keyPrefix;
FlowLock checkLock;
PerfIntCounter transactions, retries, tooOldRetries, commitFailedRetries;
PerfDoubleCounter totalLatency;
MiniCycleWorkload(WorkloadContext const& wcx)
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), tooOldRetries("Retries.too_old"),
commitFailedRetries("Retries.commit_failed"), totalLatency("Latency") {
testDuration = getOption(options, "testDuration"_sr, 10.0);
transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount;
actorCount = getOption(options, "actorsPerClient"_sr, transactionsPerSecond / 5);
nodeCount = getOption(options, "nodeCount"_sr, transactionsPerSecond * clientCount);
keyPrefix = unprintable(getOption(options, "keyPrefix"_sr, LiteralStringRef("")).toString());
traceParentProbability = getOption(options, "traceParentProbability "_sr, 0.01);
minExpectedTransactionsPerSecond = transactionsPerSecond * getOption(options, "expectedRate"_sr, 0.7);
}
std::string description() const override { return "MiniCycleWorkload"; }
Future<Void> setup(Database const& cx) override {
return bulkSetup(cx->clone(),
this,
cycleSize(clientId),
Promise<double>(),
false,
0.0,
1e12,
std::vector<uint64_t>(),
Promise<std::vector<std::pair<uint64_t, double>>>(),
0,
0.1,
beginKey(clientId),
endKey(clientId));
}
Future<Void> start(Database const& cx) override { return Void(); }
Future<bool> check(Database const& cx) override { return _check(cx->clone(), this); }
ACTOR Future<bool> _check(Database cx, MiniCycleWorkload* self) {
state std::vector<Future<Void>> cycleClients;
for (int c = 0; c < self->clientCount; c++)
cycleClients.push_back(
timeout(self->cycleClient(cx->clone(), self, self->actorCount / self->transactionsPerSecond),
self->testDuration,
Void()));
state Future<Void> end = delay(self->testDuration);
state bool ok = true;
loop {
choose {
when(bool ret = wait(self->_checkCycle(cx->clone(), self, ok))) {
ok = ret && ok;
if (!ok)
return false;
}
when(wait(end)) { break; }
}
}
// Check for errors in the cycle clients
int errors = 0;
for (int c = 0; c < cycleClients.size(); c++)
errors += cycleClients[c].isError();
if (errors || !ok)
TraceEvent(SevError, "TestFailure").detail("Reason", "There were check or cycle client errors.");
cycleClients.clear();
printf("Beginning full cycle check...");
bool ret = wait(self->_checkCycle(cx->clone(), self, ok));
return ret;
}
ACTOR Future<bool> _checkCycle(Database cx, MiniCycleWorkload* self, bool ok) {
state std::vector<Future<bool>> checkClients;
for (int c = 0; c < self->clientCount; c++)
checkClients.push_back(self->cycleCheckClient(cx->clone(), self, ok));
bool ret = wait(allTrue(checkClients));
// Check for errors in the cycle clients
int errors = 0;
for (int c = 0; c < checkClients.size(); c++)
errors += checkClients[c].isError();
if (errors)
TraceEvent(SevError, "TestFailure").detail("Reason", "There were checker errors.");
return ret;
}
void getMetrics(std::vector<PerfMetric>& m) override {
m.push_back(transactions.getMetric());
m.push_back(retries.getMetric());
m.push_back(tooOldRetries.getMetric());
m.push_back(commitFailedRetries.getMetric());
m.emplace_back("Avg Latency (ms)", 1000 * totalLatency.getValue() / transactions.getValue(), Averaged::True);
m.emplace_back("Read rows/simsec (approx)", transactions.getValue() * 3 / testDuration, Averaged::False);
m.emplace_back("Write rows/simsec (approx)", transactions.getValue() * 4 / testDuration, Averaged::False);
}
Key keyForIndex(int n) { return key(n); }
Key key(int n) { return doubleToTestKey(n, keyPrefix); }
Value value(int n) { return doubleToTestKey(n, keyPrefix); }
int fromValue(const ValueRef& v) { return testKeyToDouble(v, keyPrefix); }
// cycleSize returns the length of each mini-cycle besides the last,
// which is cycleSize + remainder nodes in length
int cycleSize(int clientId) {
// The remaining keys should go in the last cycle
int rem = (clientId == clientCount - 1) ? nodeCount % clientCount : 0;
return nodeCount / clientCount + rem;
}
// cycleOffset returns the node number at which clientId's mini-cycle begins
int cycleOffset(int clientId) { return clientId * cycleSize(0); }
int beginKey(int clientId) { return cycleOffset(clientId); }
int endKey(int clientId) { return cycleSize(clientId) + cycleOffset(clientId); }
Standalone<KeyValueRef> operator()(int n) {
const uint64_t val = (n + 1) % endKey(clientId) ? n + 1 : beginKey(clientId);
return KeyValueRef(key(n), value(val));
}
void badRead(const char* name, int r, Transaction& tr) {
TraceEvent(SevError, "MiniCycleBadRead")
.detail(name, r)
.detail("Key", printable(key(r)))
.detail("Version", tr.getReadVersion().get())
.detailf("From", "%016llx", debug_lastLoadBalanceResultEndpointToken);
}
ACTOR Future<Void> cycleClient(Database cx, MiniCycleWorkload* self, double delay) {
state double lastTime = now();
try {
loop {
wait(poisson(&lastTime, delay));
state double tstart = now();
state int r =
deterministicRandom()->randomInt(self->beginKey(self->clientId), self->endKey(self->clientId) - 1);
state Transaction tr(cx);
if (deterministicRandom()->random01() >= self->traceParentProbability) {
state Span span("MiniCycleClient"_loc);
TraceEvent("MiniCycleTracingTransaction", span.context).log();
tr.setOption(FDBTransactionOptions::SPAN_PARENT,
BinaryWriter::toValue(span.context, Unversioned()));
}
while (true) {
try {
// Reverse next and next^2 node
Optional<Value> v = wait(tr.get(self->key(r)));
if (!v.present())
self->badRead("KeyR", r, tr);
state int r2 = self->fromValue(v.get());
Optional<Value> v2 = wait(tr.get(self->key(r2)));
if (!v2.present())
self->badRead("KeyR2", r2, tr);
state int r3 = self->fromValue(v2.get());
Optional<Value> v3 = wait(tr.get(self->key(r3)));
if (!v3.present())
self->badRead("KeyR3", r3, tr);
int r4 = self->fromValue(v3.get());
tr.clear(self->key(r)); //< Shouldn't have an effect, but will break with wrong ordering
tr.set(self->key(r), self->value(r3));
tr.set(self->key(r2), self->value(r4));
tr.set(self->key(r3), self->value(r2));
// TraceEvent("CyclicTest").detail("Key", self->key(r).toString()).detail("Value", self->value(r3).toString());
// TraceEvent("CyclicTest").detail("Key", self->key(r2).toString()).detail("Value", self->value(r4).toString());
// TraceEvent("CyclicTest").detail("Key", self->key(r3).toString()).detail("Value", self->value(r2).toString());
wait(tr.commit());
// TraceEvent("MiniCycleCommit");
break;
} catch (Error& e) {
if (e.code() == error_code_transaction_too_old)
++self->tooOldRetries;
else if (e.code() == error_code_not_committed)
++self->commitFailedRetries;
wait(tr.onError(e));
}
++self->retries;
}
++self->transactions;
self->totalLatency += now() - tstart;
}
} catch (Error& e) {
TraceEvent(SevError, "MiniCycleClient").error(e);
throw;
}
}
void logTestData(const VectorRef<KeyValueRef>& data) {
TraceEvent("TestFailureDetail").log();
int index = 0;
for (auto& entry : data) {
TraceEvent("CurrentDataEntry")
.detail("Index", index)
.detail("Key", entry.key.toString())
.detail("Value", entry.value.toString());
index++;
}
}
bool cycleCheckData(const VectorRef<KeyValueRef>& data, Version v, int clientID) {
if (data.size() != cycleSize(clientId)) {
logTestData(data);
TraceEvent(SevError, "TestFailure")
.detail("Reason", "Node count changed")
.detail("Before", cycleSize(clientId))
.detail("After", data.size())
.detail("Version", v)
.detail("KeyPrefix", keyPrefix.printable());
TraceEvent(SevError, "TestFailureInfo")
.detail("DataSize", data.size())
.detail("CycleSize", cycleSize(clientId))
.detail("Workload", description());
return false;
}
int i = beginKey(clientId);
int iPrev = beginKey(clientId);
double d;
int c;
for (c = 0; c < cycleSize(clientId); c++) {
if (c && !(i - beginKey(clientId))) {
TraceEvent(SevError, "TestFailure")
.detail("Reason", "MiniCycle got shorter")
.detail("Before", cycleSize(clientId))
.detail("After", c)
.detail("KeyPrefix", keyPrefix.printable());
logTestData(data);
return false;
}
if (data[i - beginKey(clientId)].key != key(i)) {
TraceEvent(SevError, "TestFailure")
.detail("Reason", "Key changed")
.detail("KeyPrefix", keyPrefix.printable());
logTestData(data);
return false;
}
d = testKeyToDouble(data[i - beginKey(clientId)].value, keyPrefix);
iPrev = i;
i = (int)d;
if (i != d || i < beginKey(clientId) || i >= endKey(clientId)) {
TraceEvent(SevError, "TestFailure")
.detail("Reason", "Invalid value")
.detail("KeyPrefix", keyPrefix.printable());
logTestData(data);
return false;
}
}
if (i - beginKey(clientId) != 0) {
TraceEvent(SevError, "TestFailure")
.detail("Reason", "MiniCycle got longer")
.detail("KeyPrefix", keyPrefix.printable())
.detail("Key", key(i))
.detail("Value", data[i - beginKey(clientId)].value)
.detail("Iteration", c)
.detail("CycleSize", cycleSize(clientId))
.detail("Int", i)
.detail("Double", d)
.detail("ValuePrev", data[iPrev - beginKey(clientId)].value)
.detail("KeyPrev", data[iPrev - beginKey(clientId)].key);
logTestData(data);
return false;
}
return true;
}
ACTOR Future<bool> cycleCheckClient(Database cx, MiniCycleWorkload* self, bool ok) {
if (self->transactions.getMetric().value() < self->testDuration * self->minExpectedTransactionsPerSecond) {
TraceEvent(SevWarnAlways, "TestFailure")
.detail("Reason", "Rate below desired rate")
.detail("File", __FILE__)
.detail(
"Details",
format("%.2f",
self->transactions.getMetric().value() / (self->transactionsPerSecond * self->testDuration)))
.detail("TransactionsAchieved", self->transactions.getMetric().value())
.detail("MinTransactionsExpected", self->testDuration * self->minExpectedTransactionsPerSecond)
.detail("TransactionGoal", self->transactionsPerSecond * self->testDuration);
ok = false;
}
// One client checks the validity of the cycle at a time
wait(self->checkLock.take());
state FlowLock::Releaser releaser(self->checkLock);
state Transaction tr(cx);
state int retryCount = 0;
loop {
try {
state Version v = wait(tr.getReadVersion());
RangeResult data = wait(
tr.getRange(firstGreaterOrEqual(doubleToTestKey(self->beginKey(self->clientId), self->keyPrefix)),
firstGreaterOrEqual(doubleToTestKey(self->endKey(self->clientId), self->keyPrefix)),
self->cycleSize(self->clientId) + 1));
ok = self->cycleCheckData(data, v, self->clientId) && ok;
break;
} catch (Error& e) {
retryCount++;
TraceEvent(retryCount > 20 ? SevWarnAlways : SevWarn, "MiniCycleCheckError").error(e);
wait(tr.onError(e));
}
}
return ok;
}
};
WorkloadFactory<MiniCycleWorkload> MiniCycleWorkloadFactory("MiniCycle");

View File

@ -50,7 +50,7 @@ struct TransientMetricSample : MetricSample<T> {
int64_t addAndExpire(const T& key, int64_t metric, double expiration) {
int64_t x = add(key, metric);
if (x)
queue.push_back(std::make_tuple(expiration, *this->sample.find(key), -x));
queue.emplace_back(expiration, *this->sample.find(key), -x);
return x;
}
@ -116,7 +116,7 @@ struct TransientThresholdMetricSample : MetricSample<T> {
int64_t addAndExpire(T_&& key, int64_t metric, double expiration) {
int64_t x = add(std::forward<T_>(key), metric);
if (x)
queue.push_back(std::make_tuple(expiration, *this->sample.find(key), -x));
queue.emplace_back(expiration, *this->sample.find(key), -x);
return x;
}

View File

@ -1819,33 +1819,33 @@ ACTOR static Future<std::vector<NetworkAddress>> resolveTCPEndpoint_impl(Net2* s
Promise<std::vector<NetworkAddress>> promise;
state Future<std::vector<NetworkAddress>> result = promise.getFuture();
tcpResolver.async_resolve(
tcp::resolver::query(host, service), [=](const boost::system::error_code& ec, tcp::resolver::iterator iter) {
if (ec) {
promise.sendError(lookup_failed());
return;
}
tcpResolver.async_resolve(tcp::resolver::query(host, service),
[=](const boost::system::error_code& ec, tcp::resolver::iterator iter) {
if (ec) {
promise.sendError(lookup_failed());
return;
}
std::vector<NetworkAddress> addrs;
std::vector<NetworkAddress> addrs;
tcp::resolver::iterator end;
while (iter != end) {
auto endpoint = iter->endpoint();
auto addr = endpoint.address();
if (addr.is_v6()) {
addrs.push_back(NetworkAddress(IPAddress(addr.to_v6().to_bytes()), endpoint.port()));
} else {
addrs.push_back(NetworkAddress(addr.to_v4().to_ulong(), endpoint.port()));
}
++iter;
}
tcp::resolver::iterator end;
while (iter != end) {
auto endpoint = iter->endpoint();
auto addr = endpoint.address();
if (addr.is_v6()) {
addrs.emplace_back(IPAddress(addr.to_v6().to_bytes()), endpoint.port());
} else {
addrs.emplace_back(addr.to_v4().to_ulong(), endpoint.port());
}
++iter;
}
if (addrs.empty()) {
promise.sendError(lookup_failed());
} else {
promise.send(addrs);
}
});
if (addrs.empty()) {
promise.sendError(lookup_failed());
} else {
promise.send(addrs);
}
});
wait(ready(result));
tcpResolver.cancel();

View File

@ -747,7 +747,7 @@ public:
ThreadResult() : sav(0) {}
explicit ThreadResult(ThreadSingleAssignmentVar<T>* sav) : sav(sav) {
ASSERT(sav->isReadyUnsafe());
ASSERT(sav->isReady());
// sav->addref();
}
ThreadResult(const ThreadResult<T>& rhs) : sav(rhs.sav) {

View File

@ -1,111 +1,4 @@
# Overview
This directory provides various Docker images for running FoundationDB.
This directory includes two sets of images. The "release" images are based
on Ubuntu 18.04. The EKS images use Amazon Linux, which allows us to profile
FoundationDB when it is running inside of Amazon EKS.
# Build Configuration
The build scripts are configured using the following environment variables:
`TAG` is the base docker tag for this build. The sidecar tag will be this
string, with a "-1" appended to it. If you do not specify a tag, then the
scripts attempt to provide a reasonable default.
`ECR` is the name of the Docker registry the images should be published to.
It defaults to a private registry, so it is likely you will need to override this.
`STRIPPED` if true, the Docker images will contain stripped binaries without
debugging symbols. Debugging symbols add approximately 2GiB to the image size.
# Release Dockerfile arguments.
These arguments are set automatically by the build scripts, but are documented here
in case you need to invoke the release Dockerfiles directly.
### FDB_VERSION
The version of FoundationDB to install in the container. This is required.
### FDB_WEBSITE
The base URL for the FoundationDB website. The default is
`https://www.foundationdb.org`.
You can build the docker without talking to a webserver by using the URL
`file:///mnt/website` and mirroring the directory tree of the webserver
inside the `website` subdirectory.
### FDB_ADDITIONAL_VERSIONS
A list of additional client library versions to include in this image. These
libraries will be in a special multiversion library folder.
# Runtime Configuration
This image supports several environment variables for run-time configuration.
### FDB_PORT
The port that FoundationDB should bind to. The default is 4500.
### FDB_NETWORKING_MODE
A networking mode that controls what address FoundationDB listens on. If this
is `container` (the default), then the server will listen on its public IP
within the docker network, and will only be accessible from other containers.
If this is `host`, then the server will listen on `127.0.0.1`, and will not be
accessible from other containers. You should use `host` networking mode if you
want to access your container from your host machine, and you should also
map the port to the same port on your host machine when you run the container.
### FDB_COORDINATOR
A name of another FDB instance to use as a coordinator process. This can be
helpful when setting up a larger cluster inside a docker network, for instance
when using Docker Compose. The name you provide must be resolvable through the
DNS on the container you are running.
### FDB_COORDINATOR_PORT
The port to use for connecting to the FDB coordinator process. This should be
set by other processes in a multi-process cluster to the same value as the
`FDB_PORT` environment variable of the coordinator process. It will default
to 4500, which is also the default for `FDB_PORT`.
# Copying Into Other Images
You can also use this image to provide files for images that are clients of a
FoundationDB cluster, by using the `from` argument of the `COPY` command. Some
files you may want to copy are:
* `/usr/lib/libfdb_c.so`: The primary FoundationDB client library
* `/usr/lib/fdb/multiversion/libfdb_*.so`: Additional versions of the client
library, which you can use if you are setting up a multiversion client.
* `/var/fdb/scripts/create_cluster_file.bash`: A script for setting up the
cluster file based on an `FDB_COORDINATOR` environment variable.
* `/usr/bin/fdbcli`: The FoundationDB CLI.
If you are running FDB inside of a Kubernetes cluster, you should probably use
the sidecar image instead. It makes it easier to automatically copy a compatible
`libfdb_c.so` and cluster file into application containers.
TODO: Document sidecar.py
# Example Usages
### Build an Ubuntu-based image with a custom tag and unstripped binaries
```
# compile FDB, then:
cd ~/build_output/packages/docker/
TAG=my-custom-tag ./build-release-docker.sh
```
### Build an Amazon Linux-based image with a default tag and stripped binaries
```
# compile FDB, then:
cd ~/build_output/packages/docker/
STRIPPED=true ./build-eks-docker.sh
```
# packaging/docker
This directory contains the pieces for building FoundationDB docker images.
Read the `build-images.sh` shell script to learn more about how the images are
built and `Dockerfile{,.eks}` for what is contained in the images.

View File

@ -165,6 +165,9 @@ function build_and_push_images(){
for image in "${image_list[@]}"; do
logg "BUILDING ${image}"
image_tag="${tag_base}${image}:${fdb_version}"
if [ -n "${tag_postfix+x}" ]; then
image_tag="${tag_base}${image}:${fdb_version}-${tag_postfix}"
fi
if [ "${image}" == "foundationdb-kubernetes-sidecar" ]; then
image_tag="${image_tag}-1"
fi
@ -256,6 +259,7 @@ if [ -n "${OKTETO_NAMESPACE+x}" ]; then
fdb_library_versions=( "${fdb_version}" )
registry="${aws_account_id}.dkr.ecr.${aws_region}.amazonaws.com"
tag_base="${registry}/foundationdb/"
tag_postfix="${OKTETO_NAME:-dev}"
stripped_binaries_and_from_where="unstripped_local" # MUST BE ONE OF ( "unstripped_artifactory" "stripped_artifactory" "unstripped_local" "stripped_local" )
dockerfile_name="Dockerfile.eks"
use_development_java_bindings="true"

View File

@ -202,8 +202,8 @@ if(WITH_PYTHON)
TEST_FILES restarting/from_7.0.0/SnapIncrementalRestore-1.toml
restarting/from_7.0.0/SnapIncrementalRestore-2.toml)
add_fdb_test(
TEST_FILES restarting/from_7.0.0/ConfigureTestRestart-1.txt
restarting/from_7.0.0/ConfigureTestRestart-2.txt)
TEST_FILES restarting/from_7.0.0/ConfigureTestRestart-1.toml
restarting/from_7.0.0/ConfigureTestRestart-2.toml)
add_fdb_test(
TEST_FILES restarting/from_5.0.0_until_6.3.0/CycleTestRestart-1.txt
restarting/from_5.0.0_until_6.3.0/CycleTestRestart-2.txt)
@ -232,20 +232,20 @@ if(WITH_PYTHON)
TEST_FILES restarting/from_7.0.0/UpgradeAndBackupRestore-1.toml
restarting/from_7.0.0/UpgradeAndBackupRestore-2.toml)
add_fdb_test(
TEST_FILES restarting/to_7.0.0/CycleTestRestart-1.txt
restarting/to_7.0.0/CycleTestRestart-2.txt)
TEST_FILES restarting/to_7.0.0/CycleTestRestart-1.toml
restarting/to_7.0.0/CycleTestRestart-2.toml)
add_fdb_test(
TEST_FILES restarting/from_7.0.0/SnapTestAttrition-1.txt
restarting/from_7.0.0/SnapTestAttrition-2.txt)
TEST_FILES restarting/from_7.0.0/SnapTestAttrition-1.toml
restarting/from_7.0.0/SnapTestAttrition-2.toml)
add_fdb_test(
TEST_FILES restarting/from_7.0.0/SnapTestSimpleRestart-1.txt
restarting/from_7.0.0/SnapTestSimpleRestart-2.txt)
TEST_FILES restarting/from_7.0.0/SnapTestSimpleRestart-1.toml
restarting/from_7.0.0/SnapTestSimpleRestart-2.toml)
add_fdb_test(
TEST_FILES restarting/from_7.0.0/SnapTestRestart-1.txt
restarting/from_7.0.0/SnapTestRestart-2.txt)
TEST_FILES restarting/from_7.0.0/SnapTestRestart-1.toml
restarting/from_7.0.0/SnapTestRestart-2.toml)
add_fdb_test(
TEST_FILES restarting/from_7.0.0/SnapCycleRestart-1.txt
restarting/from_7.0.0/SnapCycleRestart-2.txt)
TEST_FILES restarting/from_7.0.0/SnapCycleRestart-1.toml
restarting/from_7.0.0/SnapCycleRestart-2.toml)
add_fdb_test(TEST_FILES slow/ApiCorrectness.toml)
add_fdb_test(TEST_FILES slow/ApiCorrectnessAtomicRestore.toml)
add_fdb_test(TEST_FILES slow/ApiCorrectnessSwitchover.toml)

View File

@ -0,0 +1,23 @@
[[test]]
testTitle='CloggedConfigureDatabaseTest'
clearAfterTest=false
[[test.workload]]
testName='ConfigureDatabase'
testDuration=30.0
allowDescriptorChange=false
[[test.workload]]
testName='RandomClogging'
testDuration=30.0
[[test.workload]]
testName='RandomClogging'
testDuration=30.0
scale=0.1
clogginess=2.0
[[test.workload]]
testName='SaveAndKill'
restartInfoLocation='simfdb/restartInfo.ini'
testDuration=30.0

View File

@ -1,18 +0,0 @@
testTitle=CloggedConfigureDatabaseTest
clearAfterTest=false
testName=ConfigureDatabase
testDuration=30.0
allowDescriptorChange=false
testName=RandomClogging
testDuration=30.0
testName=RandomClogging
testDuration=30.0
scale=0.1
clogginess=2.0
testName=SaveAndKill
restartInfoLocation=simfdb/restartInfo.ini
testDuration=30.0

View File

@ -0,0 +1,18 @@
[[test]]
testTitle='CloggedConfigureDatabaseTest'
runSetup=false
[[test.workload]]
testName='ConfigureDatabase'
testDuration=300.0
allowDescriptorChange=false
[[test.workload]]
testName='RandomClogging'
testDuration=300.0
[[test.workload]]
testName='RandomClogging'
testDuration=300.0
scale=0.1
clogginess=2.0

View File

@ -1,14 +0,0 @@
testTitle=CloggedConfigureDatabaseTest
runSetup=false
testName=ConfigureDatabase
testDuration=300.0
allowDescriptorChange=false
testName=RandomClogging
testDuration=300.0
testName=RandomClogging
testDuration=300.0
scale=0.1
clogginess=2.0

View File

@ -0,0 +1,30 @@
[configuration]
storageEngineExcludeTypes=[3]
# Take snap and do cycle test
[[test]]
testTitle='SnapCyclePre'
clearAfterTest=false
[[test.workload]]
testName='Cycle'
transactionsPerSecond=2500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=10.0
testID=1
# Save and shutdown
[[test]]
testTitle='SnapCycleShutdown'
[[test.workload]]
testName='SaveAndKill'
restartInfoLocation='simfdb/restartInfo.ini'
testDuration=10.0
isRestoring=1

View File

@ -1,23 +0,0 @@
storageEngineExcludeTypes=3
;Take snap and do cycle test
testTitle=SnapCyclePre
clearAfterTest=false
testName=Cycle
transactionsPerSecond=2500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
testName=SnapTest
numSnaps=1
maxSnapDelay=10.0
testID=1
;save and shutdown
testTitle=SnapCycleShutdown
testName=SaveAndKill
restartInfoLocation=simfdb/restartInfo.ini
testDuration=10.0
isRestoring=1

View File

@ -0,0 +1,15 @@
[configuration]
buggify=false
[[test]]
testTitle='SnapCycleRestore'
runSetup=false
# Post snap restore test
[[test.workload]]
testName='Cycle'
transactionsPerSecond=2500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
enableDD=true

View File

@ -1,12 +0,0 @@
buggify=off
testTitle=SnapCycleRestore
runSetup=false
;Post snap restore test
testName=Cycle
transactionsPerSecond=2500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
enableDD=true

View File

@ -0,0 +1,59 @@
# Write 1000 Keys ending with even numbers
[[test]]
testTitle='SnapTestPre'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=0
# Take snap and do read/write
[[test]]
testTitle='SnapTestTakeSnap'
clearAfterTest=false
[[test.workload]]
testName='ReadWrite'
testDuration=10.0
transactionsPerSecond=10000
writesPerTransactionA=0
readsPerTransactionA=10
writesPerTransactionB=10
readsPerTransactionB=1
alpha=0.5
nodeCount=100000
valueBytes=16
discardEdgeMeasurements=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=10.0
testID=1
[[test.workload]]
testName='Attrition'
testDuration=10.0
# Write 1000 Keys ending with odd numbers
[[test]]
testTitle='SnapTestPost'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=25.0
testID=2
# Save and shutdown
[[test]]
testTitle='SnapSimpleShutdown'
[[test.workload]]
testName='SaveAndKill'
restartInfoLocation='simfdb/restartInfo.ini'
testDuration=10.0
isRestoring=1

View File

@ -1,48 +0,0 @@
;write 1000 Keys ending with even numbers
testTitle=SnapTestPre
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=0
;Take snap and do read/write
testTitle=SnapTestTakeSnap
clearAfterTest=false
testName=ReadWrite
testDuration=10.0
transactionsPerSecond=10000
writesPerTransactionA=0
readsPerTransactionA=10
writesPerTransactionB=10
readsPerTransactionB=1
alpha=0.5
nodeCount=100000
valueBytes=16
discardEdgeMeasurements=false
testName=SnapTest
numSnaps=1
maxSnapDelay=10.0
testID=1
testName=Attrition
testDuration=10.0
;write 1000 Keys ending with odd numbers
testTitle=SnapTestPost
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=25.0
testID=2
; save and shutdown
testTitle=SnapSimpleShutdown
testName=SaveAndKill
restartInfoLocation=simfdb/restartInfo.ini
testDuration=10.0
isRestoring=1

View File

@ -0,0 +1,14 @@
[configuration]
buggify=false
# Verify all keys are even numbered
[[test]]
testTitle='SnapTestVerify'
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=3
restartInfoLocation='simfdb/restartInfo.ini'
enableDD=true

View File

@ -1,10 +0,0 @@
buggify=off
; verify all keys are even numbered
testTitle=SnapTestVerify
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=3
restartInfoLocation=simfdb/restartInfo.ini
enableDD=true

View File

@ -0,0 +1,55 @@
# Write 1000 Keys ending with even numbers
[[test]]
testTitle='SnapTestPre'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=0
# Take snap and do read/write
[[test]]
testTitle='SnapTestTakeSnap'
clearAfterTest=false
[[test.workload]]
testName='ReadWrite'
testDuration=10.0
transactionsPerSecond=10000
writesPerTransactionA=0
readsPerTransactionA=10
writesPerTransactionB=10
readsPerTransactionB=1
alpha=0.5
nodeCount=100000
valueBytes=16
discardEdgeMeasurements=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=10.0
testID=1
# Write 1000 Keys ending with odd numbers
[[test]]
testTitle='SnapTestPost'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=25.0
testID=2
# Save and shutdown
[[test]]
testTitle='SnapTestShutdown'
[[test.workload]]
testName='SaveAndKill'
restartInfoLocation='simfdb/restartInfo.ini'
testDuration=10.0
isRestoring=1

View File

@ -1,45 +0,0 @@
;write 1000 Keys ending with even numbers
testTitle=SnapTestPre
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=0
;Take snap and do read/write
testTitle=SnapTestTakeSnap
clearAfterTest=false
testName=ReadWrite
testDuration=10.0
transactionsPerSecond=10000
writesPerTransactionA=0
readsPerTransactionA=10
writesPerTransactionB=10
readsPerTransactionB=1
alpha=0.5
nodeCount=100000
valueBytes=16
discardEdgeMeasurements=false
testName=SnapTest
numSnaps=1
maxSnapDelay=10.0
testID=1
;write 1000 Keys ending with odd numbers
testTitle=SnapTestPost
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=25.0
testID=2
;save and shutdown
testTitle=SnapTestShutdown
testName=SaveAndKill
restartInfoLocation=simfdb/restartInfo.ini
testDuration=10.0
isRestoring=1

View File

@ -0,0 +1,13 @@
[configuration]
buggify=false
# Verify all keys are even numbered
[[test]]
testTitle='SnapTestVerify'
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=3
enableDD=true

View File

@ -1,9 +0,0 @@
buggify=off
; verify all keys are even numbered
testTitle=SnapTestVerify
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=3
enableDD=true

View File

@ -0,0 +1,52 @@
# Write 1000 Keys ending with even number
[[test]]
testTitle='SnapSimplePre'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=30.0
testID=0
# Take snap
[[test]]
testTitle='SnapSimpleTakeSnap'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=5.0
testID=1
# Write 1000 Keys ending with odd number
[[test]]
testTitle='SnapSimplePost'
clearAfterTest=false
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=2
# snapCreate with binary path that is not whitelisted
[[test]]
testTitle='SnapCreateNotWhitelistedBinaryPath'
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=4
# Save and shutdown
[[test]]
testTitle='SnapSimpleShutdown'
[[test.workload]]
testName='SaveAndKill'
restartInfoLocation='simfdb/restartInfo.ini'
testDuration=10.0
isRestoring=1

View File

@ -1,40 +0,0 @@
;write 1000 Keys ending with even number
testTitle=SnapSimplePre
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=30.0
testID=0
;take snap
testTitle=SnapSimpleTakeSnap
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=5.0
testID=1
;write 1000 Keys ending with odd number
testTitle=SnapSimplePost
clearAfterTest=false
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=2
;snapCreate with binary path that is not whitelisted
testTitle=SnapCreateNotWhitelistedBinaryPath
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=4
;save and shutdown
testTitle=SnapSimpleShutdown
testName=SaveAndKill
restartInfoLocation=simfdb/restartInfo.ini
testDuration=10.0
isRestoring=1

View File

@ -0,0 +1,13 @@
[configuration]
buggify=false
# Verify all keys are even numbered
[[test]]
testTitle='SnapSimpleVerify'
[[test.workload]]
testName='SnapTest'
numSnaps=1
maxSnapDelay=3.0
testID=3
enableDD=true

View File

@ -1,9 +0,0 @@
buggify=off
; verify all keys are even numbered
testTitle=SnapSimpleVerify
testName=SnapTest
numSnaps=1
maxSnapDelay=3.0
testID=3
enableDD=true

View File

@ -0,0 +1,43 @@
[configuration]
storageEngineExcludeTypes=[-1,-2]
maxTLogVersion=6
disableTss=true
[[test]]
testTitle='Clogged'
clearAfterTest=false
[[test.workload]]
testName='Cycle'
transactionsPerSecond=500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
[[test.workload]]
testName='RandomClogging'
testDuration=10.0
[[test.workload]]
testName='Rollback'
meanDelay=10.0
testDuration=10.0
[[test.workload]]
testName='Attrition'
machinesToKill=10
machinesToLeave=3
reboot=true
testDuration=10.0
[[test.workload]]
testName='Attrition'
machinesToKill=10
machinesToLeave=3
reboot=true
testDuration=10.0
[[test.workload]]
testName='SaveAndKill'
restartInfoLocation='simfdb/restartInfo.ini'
testDuration=10.0

View File

@ -1,33 +0,0 @@
storageEngineExcludeTypes=-1,-2
maxTLogVersion=6
disableTss=true
testTitle=Clogged
clearAfterTest=false
testName=Cycle
transactionsPerSecond=500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
testName=RandomClogging
testDuration=10.0
testName=Rollback
meanDelay=10.0
testDuration=10.0
testName=Attrition
machinesToKill=10
machinesToLeave=3
reboot=true
testDuration=10.0
testName=Attrition
machinesToKill=10
machinesToLeave=3
reboot=true
testDuration=10.0
testName=SaveAndKill
restartInfoLocation=simfdb/restartInfo.ini
testDuration=10.0

View File

@ -1,28 +1,37 @@
storageEngineExcludeTypes=-1,-2
[configuration]
storageEngineExcludeTypes=[-1,-2]
maxTLogVersion=6
disableTss=true
testTitle=Clogged
[[test]]
testTitle='Clogged'
[[test.workload]]
runSetup=false
testName=Cycle
testName='Cycle'
transactionsPerSecond=2500.0
nodeCount=2500
testDuration=10.0
expectedRate=0
testName=RandomClogging
[[test.workload]]
testName='RandomClogging'
testDuration=10.0
testName=Rollback
[[test.workload]]
testName='Rollback'
meanDelay=10.0
testDuration=10.0
testName=Attrition
[[test.workload]]
testName='Attrition'
machinesToKill=10
machinesToLeave=3
reboot=true
testDuration=10.0
testName=Attrition
[[test.workload]]
testName='Attrition'
machinesToKill=10
machinesToLeave=3
reboot=true

View File

@ -10,15 +10,15 @@ runSetup = true
[[test.workload]]
testName = 'ApiCorrectness'
numKeys = 5000
numKeys = 2500
onlyLowerCase = true
shortKeysRatio = 0.5
minShortKeyLength = 1
maxShortKeyLength = 3
minLongKeyLength = 1
maxLongKeyLength = 128
maxLongKeyLength = 64
minValueLength = 1
maxValueLength = 1000
maxValueLength = 200
numGets = 1000
numGetRanges = 100
numGetRangeSelectors = 100