Merge branch 'master' of https://github.com/apple/foundationdb into redwood-header-changes
This commit is contained in:
commit
702c26d8ab
|
@ -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()
|
||||
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());
|
||||
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()
|
||||
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());
|
||||
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()
|
||||
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());
|
||||
return (FDBFuture*)(TXN(tr)->getBlobGranuleRanges(range).extractPtr()););
|
||||
}
|
||||
|
||||
extern "C" DLLEXPORT FDBResult* fdb_transaction_read_blob_granules(FDBTransaction* tr,
|
||||
|
@ -765,7 +792,8 @@ 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()
|
||||
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?
|
||||
|
@ -777,11 +805,9 @@ extern "C" DLLEXPORT FDBResult* fdb_transaction_read_blob_granules(FDBTransactio
|
|||
context.debugNoMaterialize = granule_context.debugNoMaterialize;
|
||||
|
||||
Optional<Version> rv;
|
||||
if (readVersion != invalidVersion) {
|
||||
rv = readVersion;
|
||||
}
|
||||
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"
|
||||
|
|
|
@ -82,6 +82,37 @@ FILE* debugme; /* descriptor used for debug messages */
|
|||
} \
|
||||
} 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; \
|
||||
} \
|
||||
if (err == 1020 /* not_committed */) { \
|
||||
return FDB_ERROR_CONFLICT; \
|
||||
} \
|
||||
return FDB_ERROR_RETRY; \
|
||||
} \
|
||||
} while (0)
|
||||
|
||||
fdb_error_t wait_future(FDBFuture* f) {
|
||||
fdb_error_t err;
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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>`_
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -512,6 +512,9 @@ private:
|
|||
template <class T>
|
||||
ThreadFuture<T> makeTimeout();
|
||||
|
||||
template <class T>
|
||||
ThreadResult<T> abortableTimeoutResult(ThreadFuture<Void> abortSignal);
|
||||
|
||||
TransactionInfo transaction;
|
||||
|
||||
TransactionInfo getTransaction();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
// 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 {
|
||||
|
|
|
@ -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()),
|
||||
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;
|
||||
};
|
||||
|
|
|
@ -267,10 +267,12 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
a.done.sendError(statusToError(status));
|
||||
} else {
|
||||
TraceEvent(SevInfo, "RocksDB").detail("Path", a.path).detail("Method", "Open");
|
||||
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 avoiding the
|
||||
// metric logger in simulation.
|
||||
if (!g_network->isSimulated()) {
|
||||
// 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);
|
||||
|
|
|
@ -4892,14 +4892,14 @@ public:
|
|||
}
|
||||
|
||||
// Start reading the page, without caching
|
||||
entries.push_back(std::make_pair(q.get(),
|
||||
entries.emplace_back(q.get(),
|
||||
self->readPage(PagerEventReasons::LazyClear,
|
||||
q.get().height,
|
||||
snapshot,
|
||||
q.get().pageID,
|
||||
ioLeafPriority,
|
||||
true,
|
||||
false)));
|
||||
false));
|
||||
--toPop;
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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");
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -1819,8 +1819,8 @@ 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) {
|
||||
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;
|
||||
|
@ -1833,9 +1833,9 @@ ACTOR static Future<std::vector<NetworkAddress>> resolveTCPEndpoint_impl(Net2* s
|
|||
auto endpoint = iter->endpoint();
|
||||
auto addr = endpoint.address();
|
||||
if (addr.is_v6()) {
|
||||
addrs.push_back(NetworkAddress(IPAddress(addr.to_v6().to_bytes()), endpoint.port()));
|
||||
addrs.emplace_back(IPAddress(addr.to_v6().to_bytes()), endpoint.port());
|
||||
} else {
|
||||
addrs.push_back(NetworkAddress(addr.to_v4().to_ulong(), endpoint.port()));
|
||||
addrs.emplace_back(addr.to_v4().to_ulong(), endpoint.port());
|
||||
}
|
||||
++iter;
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue