Merge branch 'master' into optimize-deque-copy-ctor

This commit is contained in:
Trevor Clinkenbeard 2020-06-10 14:38:30 -07:00 committed by GitHub
commit c2587bd87c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
82 changed files with 3207 additions and 931 deletions

View File

@ -158,15 +158,15 @@ namespace FDB {
void reset() override;
TransactionImpl() : tr(NULL) {}
TransactionImpl(TransactionImpl&& r) BOOST_NOEXCEPT {
tr = r.tr;
r.tr = NULL;
}
TransactionImpl& operator=(TransactionImpl&& r) BOOST_NOEXCEPT {
tr = r.tr;
r.tr = NULL;
TransactionImpl(TransactionImpl&& r) noexcept {
tr = r.tr;
r.tr = NULL;
}
TransactionImpl& operator=(TransactionImpl&& r) noexcept {
tr = r.tr;
r.tr = NULL;
return *this;
}
}
private:
FDBTransaction* tr;

View File

@ -35,6 +35,7 @@ import com.apple.foundationdb.Transaction;
*
*/
public class ByteArrayUtil extends FastByteComparisons {
private static final byte[] EMPTY_BYTES = new byte[0];
/**
* Joins a set of byte arrays into a larger array. The {@code interlude} is placed
@ -45,36 +46,46 @@ public class ByteArrayUtil extends FastByteComparisons {
* concatenated elements.
* @param parts the pieces to be joined. May be {@code null}, but does not allow
* for elements in the list to be {@code null}.
*
*
* @return a newly created concatenation of the input
*/
public static byte[] join(byte[] interlude, List<byte[]> parts) {
return interludeJoin(interlude, parts.toArray(new byte[0][]));
}
/**
* Joins a set of byte arrays into a larger array. The {@code interlude} is placed
* between each of the elements, but not at the beginning or end. In the case that
* the list is empty or {@code null}, a zero-length byte array will be returned.
*
* @param interlude can be {@code null} or zero length. Placed internally between
* concatenated elements.
* @param parts the pieces to be joined. May be {@code null}, but does not allow
* for elements in the array to be {@code null}.
*
* @return a newly created concatenation of the input
*/
public static byte[] interludeJoin(byte[] interlude, byte[][] parts) {
if(parts == null)
return new byte[0];
int partCount = parts.size();
int partCount = parts.length;
if(partCount == 0)
return new byte[0];
return EMPTY_BYTES;
if(interlude == null)
interlude = new byte[0];
interlude = EMPTY_BYTES;
int elementTotals = 0;
int interludeSize = interlude.length;
for(byte[] e : parts) {
elementTotals += e.length;
for (int i = 0; i < partCount; i++) {
elementTotals += parts[i].length;
}
byte[] dest = new byte[(interludeSize * (partCount - 1)) + elementTotals];
//System.out.println(" interlude -> " + ArrayUtils.printable(interlude));
int startByte = 0;
int index = 0;
for(byte[] part : parts) {
//System.out.println(" section -> " + ArrayUtils.printable(parts.get(i)));
int length = part.length;
for (int i = 0; i < partCount; i++) {
int length = parts[i].length;
if(length > 0) {
System.arraycopy(part, 0, dest, startByte, length);
System.arraycopy(parts[i], 0, dest, startByte, length);
startByte += length;
}
if(index < partCount - 1 && interludeSize > 0) {
@ -84,8 +95,6 @@ public class ByteArrayUtil extends FastByteComparisons {
}
index++;
}
//System.out.println(" complete -> " + ArrayUtils.printable(dest));
return dest;
}
@ -97,7 +106,7 @@ public class ByteArrayUtil extends FastByteComparisons {
* @return a newly created concatenation of the input
*/
public static byte[] join(byte[]... parts) {
return join(null, Arrays.asList(parts));
return interludeJoin(null, parts);
}
/**

View File

@ -74,6 +74,14 @@ services:
<<: *snapshot-bindings-cmake
snapshot-cmake: &snapshot-testpackages
<<: *build-setup
command: scl enable devtoolset-8 rh-python36 rh-ruby24 -- bash -c 'mkdir -p "$${BUILD_DIR}" && cd "$${BUILD_DIR}" && cmake -G "Ninja" -DFDB_RELEASE=0 /__this_is_some_very_long_name_dir_needed_to_fix_a_bug_with_debug_rpms__/foundationdb && ninja -v -j "$${MAKEJOBS}"'
prb-testpackages:
<<: *snapshot-testpackages
snapshot-ctest: &snapshot-ctest
<<: *build-setup
command: scl enable devtoolset-8 rh-python36 rh-ruby24 -- bash -c 'mkdir -p "$${BUILD_DIR}" && cd "$${BUILD_DIR}" && cmake -G "Ninja" -DFDB_RELEASE=1 /__this_is_some_very_long_name_dir_needed_to_fix_a_bug_with_debug_rpms__/foundationdb && ninja -v -j "$${MAKEJOBS}" && ctest -L fast -j "$${MAKEJOBS}" --output-on-failure'

View File

@ -286,6 +286,7 @@ else()
-fvisibility=hidden
-Wreturn-type
-fPIC)
add_compile_options($<$<COMPILE_LANGUAGE:CXX>:-Wclass-memaccess>)
if (GPERFTOOLS_FOUND AND GCC)
add_compile_options(
-fno-builtin-malloc

View File

@ -247,14 +247,11 @@ class FileBackupAgent : public BackupAgentBase {
public:
FileBackupAgent();
FileBackupAgent( FileBackupAgent&& r ) BOOST_NOEXCEPT :
subspace( std::move(r.subspace) ),
config( std::move(r.config) ),
lastRestorable( std::move(r.lastRestorable) ),
taskBucket( std::move(r.taskBucket) ),
futureBucket( std::move(r.futureBucket) ) {}
FileBackupAgent(FileBackupAgent&& r) noexcept
: subspace(std::move(r.subspace)), config(std::move(r.config)), lastRestorable(std::move(r.lastRestorable)),
taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)) {}
void operator=( FileBackupAgent&& r ) BOOST_NOEXCEPT {
void operator=(FileBackupAgent&& r) noexcept {
subspace = std::move(r.subspace);
config = std::move(r.config);
lastRestorable = std::move(r.lastRestorable),
@ -381,19 +378,13 @@ public:
DatabaseBackupAgent();
explicit DatabaseBackupAgent(Database src);
DatabaseBackupAgent( DatabaseBackupAgent&& r ) BOOST_NOEXCEPT :
subspace( std::move(r.subspace) ),
states( std::move(r.states) ),
config( std::move(r.config) ),
errors( std::move(r.errors) ),
ranges( std::move(r.ranges) ),
tagNames( std::move(r.tagNames) ),
taskBucket( std::move(r.taskBucket) ),
futureBucket( std::move(r.futureBucket) ),
sourceStates( std::move(r.sourceStates) ),
sourceTagNames( std::move(r.sourceTagNames) ) {}
DatabaseBackupAgent(DatabaseBackupAgent&& r) noexcept
: subspace(std::move(r.subspace)), states(std::move(r.states)), config(std::move(r.config)),
errors(std::move(r.errors)), ranges(std::move(r.ranges)), tagNames(std::move(r.tagNames)),
taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)),
sourceStates(std::move(r.sourceStates)), sourceTagNames(std::move(r.sourceTagNames)) {}
void operator=( DatabaseBackupAgent&& r ) BOOST_NOEXCEPT {
void operator=(DatabaseBackupAgent&& r) noexcept {
subspace = std::move(r.subspace);
states = std::move(r.states);
config = std::move(r.config);
@ -883,7 +874,7 @@ public:
}
TraceEvent t(SevWarn, "FileBackupError");
t.error(e).detail("BackupUID", uid).detail("Description", details).detail("TaskInstance", (uint64_t)taskInstance);
// These should not happen
// key_not_found could happen
if(e.code() == error_code_key_not_found)
t.backtrace();

View File

@ -20,6 +20,11 @@
#ifndef DatabaseContext_h
#define DatabaseContext_h
#include "flow/FastAlloc.h"
#include "flow/FastRef.h"
#include "fdbclient/StorageServerInterface.h"
#include "flow/genericactors.actor.h"
#include <vector>
#pragma once
#include "fdbclient/NativeAPI.actor.h"
@ -44,7 +49,25 @@ private:
StorageServerInfo( DatabaseContext *cx, StorageServerInterface const& interf, LocalityData const& locality ) : cx(cx), ReferencedInterface<StorageServerInterface>(interf, locality) {}
};
typedef MultiInterface<ReferencedInterface<StorageServerInterface>> LocationInfo;
struct LocationInfo : MultiInterface<ReferencedInterface<StorageServerInterface>>, FastAllocated<LocationInfo> {
using Locations = MultiInterface<ReferencedInterface<StorageServerInterface>>;
explicit LocationInfo(const std::vector<Reference<ReferencedInterface<StorageServerInterface>>>& v)
: Locations(v)
{}
LocationInfo(const std::vector<Reference<ReferencedInterface<StorageServerInterface>>>& v, bool hasCaches)
: Locations(v)
, hasCaches(hasCaches)
{}
LocationInfo(const LocationInfo&) = delete;
LocationInfo(LocationInfo&&) = delete;
LocationInfo& operator=(const LocationInfo&) = delete;
LocationInfo& operator=(LocationInfo&&) = delete;
bool hasCaches = false;
Reference<Locations> locations() {
return Reference<Locations>::addRef(this);
}
};
typedef ModelInterface<MasterProxyInterface> ProxyInfo;
class ClientTagThrottleData : NonCopyable {
@ -131,7 +154,7 @@ public:
Database clone() const { return Database(new DatabaseContext( connectionFile, clientInfo, clientInfoMonitor, taskID, clientLocality, enableLocalityLoadBalance, lockAware, internal, apiVersion, switchable )); }
std::pair<KeyRange,Reference<LocationInfo>> getCachedLocation( const KeyRef&, bool isBackward = false );
std::pair<KeyRange, Reference<LocationInfo>> getCachedLocation( const KeyRef&, bool isBackward = false );
bool getCachedLocations( const KeyRangeRef&, vector<std::pair<KeyRange,Reference<LocationInfo>>>&, int limit, bool reverse );
Reference<LocationInfo> setCachedLocation( const KeyRangeRef&, const vector<struct StorageServerInterface>& );
void invalidateCache( const KeyRef&, bool isBackward = false );
@ -232,7 +255,7 @@ public:
// Cache of location information
int locationCacheSize;
CoalescedKeyRangeMap< Reference<LocationInfo> > locationCache;
CoalescedKeyRangeMap<Reference<LocationInfo>> locationCache;
std::map< UID, StorageServerInfo* > server_interf;
@ -314,7 +337,8 @@ public:
double detailedHealthMetricsLastUpdated;
UniqueOrderedOptionList<FDBTransactionOptions> transactionDefaults;
Future<Void> cacheListMonitor;
AsyncTrigger updateCache;
std::vector<std::unique_ptr<SpecialKeyRangeBaseImpl>> specialKeySpaceModules;
std::unique_ptr<SpecialKeySpace> specialKeySpace;
void registerSpecialKeySpaceModule(SpecialKeySpace::MODULE module, std::unique_ptr<SpecialKeyRangeBaseImpl> impl);

View File

@ -26,6 +26,7 @@
#include <string>
#include <vector>
#include "flow/Arena.h"
#include "flow/flow.h"
#include "fdbclient/Knobs.h"
@ -77,6 +78,10 @@ struct Tag {
serializer(ar, locality, id);
}
};
template <>
struct flow_ref<Tag> : std::integral_constant<bool, false> {};
#pragma pack(pop)
template <class Ar> void load( Ar& ar, Tag& tag ) { tag.serialize_unversioned(ar); }

View File

@ -235,7 +235,7 @@ public:
}
TraceEvent t(SevWarn, "FileRestoreError");
t.error(e).detail("RestoreUID", uid).detail("Description", details).detail("TaskInstance", (uint64_t)taskInstance);
// These should not happen
// key_not_found could happen
if(e.code() == error_code_key_not_found)
t.backtrace();
@ -3580,33 +3580,38 @@ public:
// Parallel restore
ACTOR static Future<Void> parallelRestoreFinish(Database cx, UID randomUID) {
state ReadYourWritesTransaction tr(cx);
state Future<Void> watchForRestoreRequestDone;
state bool restoreDone = false;
state Optional<Value> restoreRequestDoneKeyValue;
TraceEvent("FastRestoreAgentWaitForRestoreToFinish").detail("DBLock", randomUID);
// TODO: register watch first and then check if the key exist
loop {
try {
tr.reset();
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
Optional<Value> restoreRequestDoneKeyValue = wait(tr.get(restoreRequestDoneKey));
Optional<Value> _restoreRequestDoneKeyValue = wait(tr.get(restoreRequestDoneKey));
restoreRequestDoneKeyValue = _restoreRequestDoneKeyValue;
// Restore may finish before restoreAgent waits on the restore finish event.
if (restoreRequestDoneKeyValue.present()) {
restoreDone = true; // In case commit clears the key but in unknown_state
tr.clear(restoreRequestDoneKey);
wait(tr.commit());
break;
} else if (!restoreDone) {
watchForRestoreRequestDone = tr.watch(restoreRequestDoneKey);
} else {
state Future<Void> watchForRestoreRequestDone = tr.watch(restoreRequestDoneKey);
wait(tr.commit());
wait(watchForRestoreRequestDone);
} else {
break;
}
} catch (Error& e) {
wait(tr.onError(e));
}
}
TraceEvent("FastRestoreAgentRestoreFinished")
.detail("ClearRestoreRequestDoneKey", restoreRequestDoneKeyValue.present());
// Only this agent can clear the restoreRequestDoneKey
wait(runRYWTransaction(cx, [](Reference<ReadYourWritesTransaction> tr) -> Future<Void> {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->clear(restoreRequestDoneKey);
return Void();
}));
TraceEvent("FastRestoreAgentRestoreFinished").detail("UnlockDBStart", randomUID);
try {
wait(unlockDatabase(cx, randomUID));
@ -3671,18 +3676,18 @@ public:
TraceEvent("FastRestoreAgentSubmitRestoreRequests").detail("DBIsLocked", randomUID);
break;
} catch (Error& e) {
TraceEvent("FastRestoreAgentSubmitRestoreRequests").detail("CheckLockError", e.what());
TraceEvent(numTries > 50 ? SevError : SevWarnAlways, "FastRestoreMayFail")
TraceEvent(numTries > 50 ? SevError : SevWarnAlways, "FastRestoreAgentSubmitRestoreRequestsMayFail")
.detail("Reason", "DB is not properly locked")
.detail("ExpectedLockID", randomUID);
.detail("ExpectedLockID", randomUID)
.error(e);
numTries++;
wait(delay(5.0));
wait(tr->onError(e));
}
}
// set up restore request
tr->reset();
loop {
tr->reset();
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
try {
@ -4444,7 +4449,10 @@ public:
return r;
}
ACTOR static Future<Version> restore(FileBackupAgent* backupAgent, Database cx, Optional<Database> cxOrig, Key tagName, Key url, Standalone<VectorRef<KeyRangeRef>> ranges, bool waitForComplete, Version targetVersion, bool verbose, Key addPrefix, Key removePrefix, bool lockDB, UID randomUid) {
ACTOR static Future<Version> restore(FileBackupAgent* backupAgent, Database cx, Optional<Database> cxOrig,
Key tagName, Key url, Standalone<VectorRef<KeyRangeRef>> ranges,
bool waitForComplete, Version targetVersion, bool verbose, Key addPrefix,
Key removePrefix, bool lockDB, UID randomUid) {
state Reference<IBackupContainer> bc = IBackupContainer::openContainer(url.toString());
state BackupDescription desc = wait(bc->describeBackup());

View File

@ -36,7 +36,10 @@ template <class Val, class Metric=int, class MetricFunc = ConstantMetric<Metric>
class KeyRangeMap : public RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>, NonCopyable, public ReferenceCounted<KeyRangeMap<Val>> {
public:
explicit KeyRangeMap(Val v=Val(), Key endKey = allKeys.end) : RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>(endKey, v), mapEnd(endKey) {}
void operator=(KeyRangeMap&& r) BOOST_NOEXCEPT { mapEnd = std::move(r.mapEnd); RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>::operator=(std::move(r)); }
void operator=(KeyRangeMap&& r) noexcept {
mapEnd = std::move(r.mapEnd);
RangeMap<Key, Val, KeyRangeRef, Metric, MetricFunc>::operator=(std::move(r));
}
void insert( const KeyRangeRef& keys, const Val& value ) { RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>::insert(keys, value); }
void insert( const KeyRef& key, const Val& value ) { RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>::insert( singleKeyRange(key), value); }
std::vector<KeyRangeWith<Val>> getAffectedRangesAfterInsertion( const KeyRangeRef& keys, const Val &insertionValue = Val());
@ -67,7 +70,10 @@ template <class Val, class Metric=int, class MetricFunc = ConstantMetric<Metric>
class CoalescedKeyRefRangeMap : public RangeMap<KeyRef,Val,KeyRangeRef,Metric,MetricFunc>, NonCopyable {
public:
explicit CoalescedKeyRefRangeMap(Val v=Val(), Key endKey = allKeys.end) : RangeMap<KeyRef,Val,KeyRangeRef,Metric,MetricFunc>(endKey, v), mapEnd(endKey) {}
void operator=(CoalescedKeyRefRangeMap&& r) BOOST_NOEXCEPT { mapEnd = std::move(r.mapEnd); RangeMap<KeyRef, Val, KeyRangeRef,Metric,MetricFunc>::operator=(std::move(r)); }
void operator=(CoalescedKeyRefRangeMap&& r) noexcept {
mapEnd = std::move(r.mapEnd);
RangeMap<KeyRef, Val, KeyRangeRef, Metric, MetricFunc>::operator=(std::move(r));
}
void insert( const KeyRangeRef& keys, const Val& value );
void insert( const KeyRef& key, const Val& value, Arena& arena );
Key mapEnd;
@ -77,7 +83,10 @@ template <class Val, class Metric=int, class MetricFunc = ConstantMetric<Metric>
class CoalescedKeyRangeMap : public RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>, NonCopyable {
public:
explicit CoalescedKeyRangeMap(Val v=Val(), Key endKey = allKeys.end) : RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>(endKey, v), mapEnd(endKey) {}
void operator=(CoalescedKeyRangeMap&& r) BOOST_NOEXCEPT { mapEnd = std::move(r.mapEnd); RangeMap<Key,Val,KeyRangeRef,Metric,MetricFunc>::operator=(std::move(r)); }
void operator=(CoalescedKeyRangeMap&& r) noexcept {
mapEnd = std::move(r.mapEnd);
RangeMap<Key, Val, KeyRangeRef, Metric, MetricFunc>::operator=(std::move(r));
}
void insert( const KeyRangeRef& keys, const Val& value );
void insert( const KeyRef& key, const Val& value );
Key mapEnd;

View File

@ -19,7 +19,12 @@
*/
#include <cinttypes>
#include <vector>
#include "flow/Arena.h"
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/SystemData.h"
@ -1858,6 +1863,69 @@ ACTOR Future<Void> waitForPrimaryDC( Database cx, StringRef dcId ) {
}
}
ACTOR Future<Void> changeCachedRange(Database cx, KeyRangeRef range, bool add) {
state ReadYourWritesTransaction tr(cx);
state KeyRange sysRange = KeyRangeRef(storageCacheKey(range.begin), storageCacheKey(range.end));
state KeyRange sysRangeClear = KeyRangeRef(storageCacheKey(range.begin), keyAfter(storageCacheKey(range.end)));
state KeyRange privateRange = KeyRangeRef(cacheKeysKey(0, range.begin), cacheKeysKey(0, range.end));
state Value trueValue = storageCacheValue(std::vector<uint16_t>{ 0 });
state Value falseValue = storageCacheValue(std::vector<uint16_t>{});
loop {
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
tr.clear(sysRangeClear);
tr.clear(privateRange);
tr.addReadConflictRange(privateRange);
Standalone<RangeResultRef> previous =
wait(tr.getRange(KeyRangeRef(storageCachePrefix, sysRange.begin), 1, true));
bool prevIsCached = false;
if (!previous.empty()) {
std::vector<uint16_t> prevVal;
decodeStorageCacheValue(previous[0].value, prevVal);
prevIsCached = !prevVal.empty();
}
if (prevIsCached && !add) {
// we need to uncache from here
tr.set(sysRange.begin, falseValue);
tr.set(privateRange.begin, serverKeysFalse);
} else if (!prevIsCached && add) {
// we need to cache, starting from here
tr.set(sysRange.begin, trueValue);
tr.set(privateRange.begin, serverKeysTrue);
}
Standalone<RangeResultRef> after =
wait(tr.getRange(KeyRangeRef(sysRange.end, storageCacheKeys.end), 1, false));
bool afterIsCached = false;
if (!after.empty()) {
std::vector<uint16_t> afterVal;
decodeStorageCacheValue(after[0].value, afterVal);
afterIsCached = afterVal.empty();
}
if (afterIsCached && !add) {
tr.set(sysRange.end, trueValue);
tr.set(privateRange.end, serverKeysTrue);
} else if (!afterIsCached && add) {
tr.set(sysRange.end, falseValue);
tr.set(privateRange.end, serverKeysFalse);
}
wait(tr.commit());
return Void();
} catch (Error& e) {
state Error err = e;
wait(tr.onError(err));
TraceEvent(SevDebug, "ChangeCachedRangeError").error(err);
}
}
}
Future<Void> addCachedRange(const Database& cx, KeyRangeRef range) {
return changeCachedRange(cx, range, true);
}
Future<Void> removeCachedRange(const Database& cx, KeyRangeRef range) {
return changeCachedRange(cx, range, false);
}
json_spirit::Value_type normJSONType(json_spirit::Value_type type) {
if (type == json_spirit::int_type)
return json_spirit::real_type;

View File

@ -201,5 +201,8 @@ bool schemaMatch( json_spirit::mValue const& schema, json_spirit::mValue const&
// storage nodes
ACTOR Future<Void> mgmtSnapCreate(Database cx, Standalone<StringRef> snapCmd, UID snapUID);
Future<Void> addCachedRange(const Database& cx, KeyRangeRef range);
Future<Void> removeCachedRange(const Database& cx, KeyRangeRef range);
#include "flow/unactorcompiler.h"
#endif

View File

@ -20,9 +20,17 @@
#include "fdbclient/NativeAPI.actor.h"
#include <algorithm>
#include <iterator>
#include <regex>
#include <unordered_set>
#include <tuple>
#include <utility>
#include <vector>
#include "fdbclient/FDBTypes.h"
#include "fdbrpc/FailureMonitor.h"
#include "fdbrpc/MultiInterface.h"
#include "fdbclient/Atomic.h"
#include "fdbclient/ClusterInterface.h"
@ -41,13 +49,19 @@
#include "fdbrpc/LoadBalance.h"
#include "fdbrpc/Net2FileSystem.h"
#include "fdbrpc/simulator.h"
#include "flow/Arena.h"
#include "flow/ActorCollection.h"
#include "flow/DeterministicRandom.h"
#include "flow/Error.h"
#include "flow/flow.h"
#include "flow/genericactors.actor.h"
#include "flow/Knobs.h"
#include "flow/Platform.h"
#include "flow/SystemMonitor.h"
#include "flow/TLSConfig.actor.h"
#include "flow/Trace.h"
#include "flow/UnitTest.h"
#include "flow/serialize.h"
#include "fdbclient/versions.h"
@ -67,6 +81,33 @@ using std::max;
using std::min;
using std::pair;
namespace {
ACTOR template <class T, class Fun>
Future<T> runAfter(Future<T> in, Fun func) {
T res = wait(in);
return func(res);
}
template <class Interface, class Request>
Future<REPLY_TYPE(Request)> loadBalance(
DatabaseContext* ctx, const Reference<LocationInfo> alternatives, RequestStream<Request> Interface::*channel,
const Request& request = Request(), TaskPriority taskID = TaskPriority::DefaultPromiseEndpoint,
bool atMostOnce = false, // if true, throws request_maybe_delivered() instead of retrying automatically
QueueModel* model = NULL) {
if (alternatives->hasCaches) {
return loadBalance(alternatives->locations(), channel, request, taskID, atMostOnce, model);
}
return runAfter(loadBalance(alternatives->locations(), channel, request, taskID, atMostOnce, model),
[ctx](auto res) {
if (res.cached) {
ctx->updateCache.trigger();
}
return res;
});
}
} // namespace
NetworkOptions networkOptions;
TLSConfig tlsConfig(TLSEndpointType::CLIENT);
@ -454,6 +495,166 @@ ACTOR static Future<Void> monitorMasterProxiesChange(Reference<AsyncVar<ClientDB
}
}
void updateLocationCacheWithCaches(DatabaseContext* self, const std::map<UID, StorageServerInterface>& removed,
const std::map<UID, StorageServerInterface>& added) {
// TODO: this needs to be more clever in the future
auto ranges = self->locationCache.ranges();
for (auto iter = ranges.begin(); iter != ranges.end(); ++iter) {
if (iter->value() && iter->value()->hasCaches) {
auto& val = iter->value();
std::vector<Reference<ReferencedInterface<StorageServerInterface>>> interfaces;
interfaces.reserve(val->size() - removed.size() + added.size());
for (int i = 0; i < val->size(); ++i) {
const auto& interf = (*val)[i];
if (removed.count(interf->interf.id()) == 0) {
interfaces.emplace_back(interf);
}
}
for (const auto& p : added) {
interfaces.emplace_back(Reference<ReferencedInterface<StorageServerInterface>>{new ReferencedInterface<StorageServerInterface>{p.second}});
}
iter->value() = Reference<LocationInfo>{ new LocationInfo(interfaces, true) };
}
}
}
Reference<LocationInfo> addCaches(const Reference<LocationInfo>& loc,
const std::vector<Reference<ReferencedInterface<StorageServerInterface>>>& other) {
std::vector<Reference<ReferencedInterface<StorageServerInterface>>> interfaces;
interfaces.reserve(loc->size() + other.size());
for (int i = 0; i < loc->size(); ++i) {
interfaces.emplace_back((*loc)[i]);
}
interfaces.insert(interfaces.end(), other.begin(), other.end());
return Reference<LocationInfo>{ new LocationInfo{ interfaces, true } };
}
ACTOR Future<Void> updateCachedRanges(DatabaseContext* self, std::map<UID, StorageServerInterface>* cacheServers) {
state Database db(self);
state ReadYourWritesTransaction tr(db);
state Value trueValue = storageCacheValue(std::vector<uint16_t>{ 0 });
state Value falseValue = storageCacheValue(std::vector<uint16_t>{});
try {
loop {
wait(self->updateCache.onTrigger());
tr.reset();
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::READ_LOCK_AWARE);
try {
Standalone<RangeResultRef> range = wait(tr.getRange(storageCacheKeys, CLIENT_KNOBS->TOO_MANY));
ASSERT(!range.more);
std::vector<Reference<ReferencedInterface<StorageServerInterface>>> cacheInterfaces;
cacheInterfaces.reserve(cacheServers->size());
for (const auto& p : *cacheServers) {
cacheInterfaces.emplace_back(Reference<ReferencedInterface<StorageServerInterface>>{
new ReferencedInterface<StorageServerInterface>{ p.second } });
}
bool currCached = false;
KeyRef begin, end;
for (const auto& kv : range) {
// These booleans have to flip consistently
ASSERT(currCached == (kv.value == falseValue));
if (kv.value == trueValue) {
begin = kv.key.substr(storageCacheKeys.begin.size());
currCached = true;
} else {
currCached = false;
end = kv.key.substr(storageCacheKeys.begin.size());
KeyRangeRef cachedRange{begin, end};
auto ranges = self->locationCache.containedRanges(cachedRange);
KeyRef containedRangesBegin, containedRangesEnd, prevKey;
if (!ranges.empty()) {
containedRangesBegin = ranges.begin().range().begin;
}
for (auto iter = ranges.begin(); iter != ranges.end(); ++iter) {
// We probably don't want to do the code below? Otherwise we would never
// fetch the corresponding storages - which would give us a different semantics
//if (containedRangesEnd > iter->range().begin) {
// self->locationCache.insert(
// KeyRangeRef{ containedRangesEnd, iter->range().begin },
// Reference<LocationInfo>{ new LocationInfo{ cacheInterfaces, true } });
//}
containedRangesEnd = iter->range().end;
if (iter->value() && !iter->value()->hasCaches) {
iter->value() = addCaches(iter->value(), cacheInterfaces);
}
}
auto iter = self->locationCache.rangeContaining(begin);
if (iter->value() && !iter->value()->hasCaches) {
if (end>=iter->range().end) {
self->locationCache.insert(KeyRangeRef{ begin, iter->range().end },
addCaches(iter->value(), cacheInterfaces));
} else {
self->locationCache.insert(KeyRangeRef{ begin, end },
addCaches(iter->value(), cacheInterfaces));
}
}
iter = self->locationCache.rangeContainingKeyBefore(end);
if (iter->value() && !iter->value()->hasCaches) {
self->locationCache.insert(KeyRangeRef{iter->range().begin, end}, addCaches(iter->value(), cacheInterfaces));
}
}
}
wait(delay(2.0)); // we want to wait at least some small amount of time before
// updating this list again
} catch (Error& e) {
wait(tr.onError(e));
}
}
} catch (Error& e) {
TraceEvent(SevError, "UpdateCachedRangesFailed")
.error(e);
throw;
}
}
ACTOR Future<Void> monitorCacheList(DatabaseContext* self) {
state Database db(self);
state Transaction tr(db);
state std::map<UID, StorageServerInterface> cacheServerMap;
state Future<Void> updateRanges = updateCachedRanges(self, &cacheServerMap);
// if no caches are configured, we don't want to run this actor at all
// so we just wait for the first trigger from a storage server
wait(self->updateCache.onTrigger());
try {
loop {
tr.reset();
try {
Standalone<RangeResultRef> cacheList =
wait(tr.getRange(storageCacheServerKeys, CLIENT_KNOBS->TOO_MANY));
ASSERT(!cacheList.more);
bool hasChanges = false;
std::map<UID, StorageServerInterface> allCacheServers;
for (auto kv : cacheList) {
auto ssi = BinaryReader::fromStringRef<StorageServerInterface>(kv.value, IncludeVersion());
allCacheServers.emplace(ssi.id(), ssi);
}
std::map<UID, StorageServerInterface> newCacheServers;
std::map<UID, StorageServerInterface> deletedCacheServers;
std::set_difference(allCacheServers.begin(), allCacheServers.end(), cacheServerMap.begin(),
cacheServerMap.end(),
std::insert_iterator<std::map<UID, StorageServerInterface>>(
newCacheServers, newCacheServers.begin()));
std::set_difference(cacheServerMap.begin(), cacheServerMap.end(), allCacheServers.begin(),
allCacheServers.end(),
std::insert_iterator<std::map<UID, StorageServerInterface>>(
deletedCacheServers, deletedCacheServers.begin()));
hasChanges = !(newCacheServers.empty() && deletedCacheServers.empty());
if (hasChanges) {
updateLocationCacheWithCaches(self, deletedCacheServers, newCacheServers);
}
cacheServerMap = std::move(allCacheServers);
wait(delay(5.0));
} catch (Error& e) {
wait(tr.onError(e));
}
}
} catch (Error& e) {
TraceEvent(SevError, "MonitorCacheListFailed").error(e);
throw;
}
}
ACTOR static Future<HealthMetrics> getHealthMetricsActor(DatabaseContext *cx, bool detailed) {
if (now() - cx->healthMetricsLastUpdated < CLIENT_KNOBS->AGGREGATE_HEALTH_METRICS_MAX_STALENESS) {
if (detailed) {
@ -600,6 +801,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
monitorMasterProxiesInfoChange = monitorMasterProxiesChange(clientInfo, &masterProxiesChangeTrigger);
clientStatusUpdater.actor = clientStatusUpdateActor(this);
cacheListMonitor = monitorCacheList(this);
if (apiVersionAtLeast(630)) {
registerSpecialKeySpaceModule(SpecialKeySpace::MODULE::TRANSACTION, std::make_unique<ConflictingKeysImpl>(conflictingKeysRange));
registerSpecialKeySpaceModule(SpecialKeySpace::MODULE::TRANSACTION, std::make_unique<ReadConflictRangeImpl>(readConflictRangeKeysRange));
@ -681,14 +883,15 @@ Database DatabaseContext::create(Reference<AsyncVar<ClientDBInfo>> clientInfo, F
}
DatabaseContext::~DatabaseContext() {
cacheListMonitor.cancel();
monitorMasterProxiesInfoChange.cancel();
for(auto it = server_interf.begin(); it != server_interf.end(); it = server_interf.erase(it))
it->second->notifyContextDestroyed();
ASSERT_ABORT( server_interf.empty() );
locationCache.insert( allKeys, Reference<LocationInfo>() );
locationCache.insert(allKeys, Reference<LocationInfo>());
}
pair<KeyRange,Reference<LocationInfo>> DatabaseContext::getCachedLocation( const KeyRef& key, bool isBackward ) {
pair<KeyRange, Reference<LocationInfo>> DatabaseContext::getCachedLocation( const KeyRef& key, bool isBackward ) {
if( isBackward ) {
auto range = locationCache.rangeContainingKeyBefore(key);
return std::make_pair(range->range(), range->value());
@ -740,23 +943,24 @@ Reference<LocationInfo> DatabaseContext::setCachedLocation( const KeyRangeRef& k
attempts++;
auto r = locationCache.randomRange();
Key begin = r.begin(), end = r.end(); // insert invalidates r, so can't be passed a mere reference into it
locationCache.insert( KeyRangeRef(begin, end), Reference<LocationInfo>() );
locationCache.insert(KeyRangeRef(begin, end), Reference<LocationInfo>());
}
locationCache.insert( keys, loc );
return loc;
}
void DatabaseContext::invalidateCache( const KeyRef& key, bool isBackward ) {
if( isBackward )
if( isBackward ) {
locationCache.rangeContainingKeyBefore(key)->value() = Reference<LocationInfo>();
else
} else {
locationCache.rangeContaining(key)->value() = Reference<LocationInfo>();
}
}
void DatabaseContext::invalidateCache( const KeyRangeRef& keys ) {
auto rs = locationCache.intersectingRanges(keys);
Key begin = rs.begin().begin(), end = rs.end().begin(); // insert invalidates rs, so can't be passed a mere reference into it
locationCache.insert( KeyRangeRef(begin, end), Reference<LocationInfo>() );
locationCache.insert(KeyRangeRef(begin, end), Reference<LocationInfo>());
}
Future<Void> DatabaseContext::onMasterProxiesChanged() {
@ -1362,7 +1566,11 @@ ACTOR Future< pair<KeyRange,Reference<LocationInfo>> > getKeyLocation_internal(
}
template <class F>
Future<pair<KeyRange, Reference<LocationInfo>>> getKeyLocation( Database const& cx, Key const& key, F StorageServerInterface::*member, TransactionInfo const& info, bool isBackward = false ) {
Future<pair<KeyRange, Reference<LocationInfo>>> getKeyLocation(Database const& cx, Key const& key,
F StorageServerInterface::*member,
TransactionInfo const& info,
bool isBackward = false) {
// we first check whether this range is cached
auto ssi = cx->getCachedLocation( key, isBackward );
if (!ssi.second) {
return getKeyLocation_internal( cx, key, info, isBackward );
@ -1511,7 +1719,7 @@ ACTOR Future<Optional<Value>> getValue( Future<Version> version, Key key, Databa
choose {
when(wait(cx->connectionFileChanged())) { throw transaction_too_old(); }
when(GetValueReply _reply =
wait(loadBalance(ssi.second, &StorageServerInterface::getValue,
wait(loadBalance(cx.getPtr(), ssi.second, &StorageServerInterface::getValue,
GetValueRequest(key, ver, cx->sampleReadTags() ? tags : Optional<TagSet>(), getValueID), TaskPriority::DefaultPromiseEndpoint, false,
cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) {
reply = _reply;
@ -1599,7 +1807,7 @@ ACTOR Future<Key> getKey( Database cx, KeySelector k, Future<Version> version, T
choose {
when(wait(cx->connectionFileChanged())) { throw transaction_too_old(); }
when(GetKeyReply _reply =
wait(loadBalance(ssi.second, &StorageServerInterface::getKey, GetKeyRequest(k, version.get(), cx->sampleReadTags() ? tags : Optional<TagSet>(), getKeyID),
wait(loadBalance(cx.getPtr(), ssi.second, &StorageServerInterface::getKey, GetKeyRequest(k, version.get(), cx->sampleReadTags() ? tags : Optional<TagSet>(), getKeyID),
TaskPriority::DefaultPromiseEndpoint, false,
cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) {
reply = _reply;
@ -1689,7 +1897,7 @@ ACTOR Future<Void> watchValue(Future<Version> version, Key key, Optional<Value>
}
state WatchValueReply resp;
choose {
when(WatchValueReply r = wait(loadBalance(ssi.second, &StorageServerInterface::watchValue,
when(WatchValueReply r = wait(loadBalance(cx.getPtr(), ssi.second, &StorageServerInterface::watchValue,
WatchValueRequest(key, value, ver, cx->sampleReadTags() ? tags : Optional<TagSet>(), watchValueID),
TaskPriority::DefaultPromiseEndpoint))) {
resp = r;
@ -1793,10 +2001,10 @@ ACTOR Future<Standalone<RangeResultRef>> getExactRange( Database cx, Version ver
try {
choose {
when(wait(cx->connectionFileChanged())) { throw transaction_too_old(); }
when(GetKeyValuesReply _rep =
wait(loadBalance(locations[shard].second, &StorageServerInterface::getKeyValues, req,
TaskPriority::DefaultPromiseEndpoint, false,
cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) {
when(GetKeyValuesReply _rep = wait(
loadBalance(cx.getPtr(), locations[shard].second, &StorageServerInterface::getKeyValues,
req, TaskPriority::DefaultPromiseEndpoint, false,
cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) {
rep = _rep;
}
}
@ -2092,7 +2300,10 @@ ACTOR Future<Standalone<RangeResultRef>> getRange( Database cx, Reference<Transa
transaction_too_old(), future_version()
});
}
GetKeyValuesReply _rep = wait( loadBalance(beginServer.second, &StorageServerInterface::getKeyValues, req, TaskPriority::DefaultPromiseEndpoint, false, cx->enableLocalityLoadBalance ? &cx->queueModel : NULL ) );
GetKeyValuesReply _rep =
wait(loadBalance(cx.getPtr(), beginServer.second, &StorageServerInterface::getKeyValues, req,
TaskPriority::DefaultPromiseEndpoint, false,
cx->enableLocalityLoadBalance ? &cx->queueModel : NULL));
rep = _rep;
++cx->transactionPhysicalReadsCompleted;
} catch(Error&) {
@ -2268,7 +2479,7 @@ Transaction::~Transaction() {
cancelWatches();
}
void Transaction::operator=(Transaction&& r) BOOST_NOEXCEPT {
void Transaction::operator=(Transaction&& r) noexcept {
flushTrLogsIfEnabled();
cx = std::move(r.cx);
tr = std::move(r.tr);
@ -3660,7 +3871,7 @@ ACTOR Future<StorageMetrics> doGetStorageMetrics(Database cx, KeyRangeRef keys,
req.min.bytes = 0;
req.max.bytes = -1;
StorageMetrics m = wait(
loadBalance(locationInfo, &StorageServerInterface::waitMetrics, req, TaskPriority::DataDistribution));
loadBalance(locationInfo->locations(), &StorageServerInterface::waitMetrics, req, TaskPriority::DataDistribution));
return m;
} catch (Error& e) {
if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) {
@ -3702,8 +3913,8 @@ ACTOR Future<Void> trackBoundedStorageMetrics(
try {
loop {
WaitMetricsRequest req( keys, x - halfError, x + halfError );
StorageMetrics nextX = wait( loadBalance( location, &StorageServerInterface::waitMetrics, req ) );
deltaStream.send( nextX - x );
StorageMetrics nextX = wait(loadBalance(location->locations(), &StorageServerInterface::waitMetrics, req));
deltaStream.send(nextX - x);
x = nextX;
}
} catch (Error& e) {
@ -3728,8 +3939,8 @@ ACTOR Future<StorageMetrics> waitStorageMetricsMultipleLocations(
WaitMetricsRequest req(locations[i].first, StorageMetrics(), StorageMetrics());
req.min.bytes = 0;
req.max.bytes = -1;
fx[i] =
loadBalance(locations[i].second, &StorageServerInterface::waitMetrics, req, TaskPriority::DataDistribution);
fx[i] = loadBalance(locations[i].second->locations(), &StorageServerInterface::waitMetrics, req,
TaskPriority::DataDistribution);
}
wait(waitForAll(fx));
@ -3777,7 +3988,7 @@ ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> getReadHotRanges(Database cx, K
state vector<Future<ReadHotSubRangeReply>> fReplies(nLocs);
for (int i = 0; i < nLocs; i++) {
ReadHotSubRangeRequest req(locations[i].first);
fReplies[i] = loadBalance(locations[i].second, &StorageServerInterface::getReadHotRanges, req,
fReplies[i] = loadBalance(locations[i].second->locations(), &StorageServerInterface::getReadHotRanges, req,
TaskPriority::DataDistribution);
}
@ -3823,7 +4034,8 @@ ACTOR Future< std::pair<Optional<StorageMetrics>, int> > waitStorageMetrics(
fx = waitStorageMetricsMultipleLocations(locations, min, max, permittedError);
} else {
WaitMetricsRequest req( keys, min, max );
fx = loadBalance( locations[0].second, &StorageServerInterface::waitMetrics, req, TaskPriority::DataDistribution );
fx = loadBalance(locations[0].second->locations(), &StorageServerInterface::waitMetrics, req,
TaskPriority::DataDistribution);
}
StorageMetrics x = wait(fx);
return std::make_pair(x,-1);
@ -3911,8 +4123,12 @@ ACTOR Future< Standalone<VectorRef<KeyRef>> > splitStorageMetrics( Database cx,
state int i = 0;
for(; i<locations.size(); i++) {
SplitMetricsRequest req( locations[i].first, limit, used, estimated, i == locations.size() - 1 );
SplitMetricsReply res = wait( loadBalance( locations[i].second, &StorageServerInterface::splitMetrics, req, TaskPriority::DataDistribution ) );
if( res.splits.size() && res.splits[0] <= results.back() ) { // split points are out of order, possibly because of moving data, throw error to retry
SplitMetricsReply res =
wait(loadBalance(locations[i].second->locations(), &StorageServerInterface::splitMetrics, req,
TaskPriority::DataDistribution));
if (res.splits.size() &&
res.splits[0] <= results.back()) { // split points are out of order, possibly because of moving
// data, throw error to retry
ASSERT_WE_THINK(false); // FIXME: This seems impossible and doesn't seem to be covered by testing
throw all_alternatives_failed();
}

View File

@ -77,8 +77,8 @@ public:
Database() {} // an uninitialized database can be destructed or reassigned safely; that's it
void operator= ( Database const& rhs ) { db = rhs.db; }
Database( Database const& rhs ) : db(rhs.db) {}
Database(Database&& r) BOOST_NOEXCEPT : db(std::move(r.db)) {}
void operator= (Database&& r) BOOST_NOEXCEPT { db = std::move(r.db); }
Database(Database&& r) noexcept : db(std::move(r.db)) {}
void operator=(Database&& r) noexcept { db = std::move(r.db); }
// For internal use by the native client:
explicit Database(Reference<DatabaseContext> cx) : db(cx) {}
@ -279,7 +279,7 @@ public:
// These are to permit use as state variables in actors:
Transaction() : info( TaskPriority::DefaultEndpoint ) {}
void operator=(Transaction&& r) BOOST_NOEXCEPT;
void operator=(Transaction&& r) noexcept;
void reset();
void fullReset();

View File

@ -73,8 +73,8 @@ struct Notified {
void operator=(const ValueType& v) { set(v); }
Notified(Notified&& r) BOOST_NOEXCEPT : waiting(std::move(r.waiting)), val(std::move(r.val)) {}
void operator=(Notified&& r) BOOST_NOEXCEPT {
Notified(Notified&& r) noexcept : waiting(std::move(r.waiting)), val(std::move(r.val)) {}
void operator=(Notified&& r) noexcept {
waiting = std::move(r.waiting);
val = std::move(r.val);
}

View File

@ -1119,8 +1119,7 @@ public:
}
bool retry_limit_hit = ryw->options.maxRetries != -1 && ryw->retries >= ryw->options.maxRetries;
if (ryw->retries < std::numeric_limits<int>::max())
ryw->retries++;
if (ryw->retries < std::numeric_limits<int>::max()) ryw->retries++;
if(retry_limit_hit) {
throw e;
}
@ -1130,7 +1129,7 @@ public:
ryw->debugLogRetries(e);
ryw->resetRyow();
return Void();
return Void();
} catch( Error &e ) {
if ( !ryw->resetPromise.isSet() ) {
if(ryw->tr.apiVersionAtLeast(610)) {
@ -2025,7 +2024,7 @@ void ReadYourWritesTransaction::setOptionImpl( FDBTransactionOptions::Option opt
tr.setOption( option, value );
}
void ReadYourWritesTransaction::operator=(ReadYourWritesTransaction&& r) BOOST_NOEXCEPT {
void ReadYourWritesTransaction::operator=(ReadYourWritesTransaction&& r) noexcept {
cache = std::move( r.cache );
writes = std::move( r.writes );
arena = std::move( r.arena );
@ -2051,21 +2050,12 @@ void ReadYourWritesTransaction::operator=(ReadYourWritesTransaction&& r) BOOST_N
versionStampKeys = std::move(r.versionStampKeys);
}
ReadYourWritesTransaction::ReadYourWritesTransaction(ReadYourWritesTransaction&& r) BOOST_NOEXCEPT :
cache( std::move(r.cache) ),
writes( std::move(r.writes) ),
arena( std::move(r.arena) ),
reading( std::move(r.reading) ),
retries( r.retries ),
approximateSize(r.approximateSize),
creationTime( r.creationTime ),
deferredError( std::move(r.deferredError) ),
timeoutActor( std::move(r.timeoutActor) ),
resetPromise( std::move(r.resetPromise) ),
commitStarted( r.commitStarted ),
options( r.options ),
transactionDebugInfo( r.transactionDebugInfo )
{
ReadYourWritesTransaction::ReadYourWritesTransaction(ReadYourWritesTransaction&& r) noexcept
: cache(std::move(r.cache)), writes(std::move(r.writes)), arena(std::move(r.arena)), reading(std::move(r.reading)),
retries(r.retries), approximateSize(r.approximateSize), creationTime(r.creationTime),
deferredError(std::move(r.deferredError)), timeoutActor(std::move(r.timeoutActor)),
resetPromise(std::move(r.resetPromise)), commitStarted(r.commitStarted), options(r.options),
transactionDebugInfo(r.transactionDebugInfo) {
cache.arena = &arena;
writes.arena = &arena;
tr = std::move( r.tr );

View File

@ -110,8 +110,8 @@ public:
// These are to permit use as state variables in actors:
ReadYourWritesTransaction() : cache(&arena), writes(&arena) {}
void operator=(ReadYourWritesTransaction&& r) BOOST_NOEXCEPT;
ReadYourWritesTransaction(ReadYourWritesTransaction&& r) BOOST_NOEXCEPT;
void operator=(ReadYourWritesTransaction&& r) noexcept;
ReadYourWritesTransaction(ReadYourWritesTransaction&& r) noexcept;
virtual void addref() { ReferenceCounted<ReadYourWritesTransaction>::addref(); }
virtual void delref() { ReferenceCounted<ReadYourWritesTransaction>::delref(); }

View File

@ -292,8 +292,12 @@ public:
entries.insert( Entry( allKeys.end, afterAllKeys, VectorRef<KeyValueRef>() ), NoMetric(), true );
}
// Visual Studio refuses to generate these, apparently despite the standard
SnapshotCache(SnapshotCache&& r) BOOST_NOEXCEPT : entries(std::move(r.entries)), arena(r.arena) {}
SnapshotCache& operator=(SnapshotCache&& r) BOOST_NOEXCEPT { entries = std::move(r.entries); arena = r.arena; return *this; }
SnapshotCache(SnapshotCache&& r) noexcept : entries(std::move(r.entries)), arena(r.arena) {}
SnapshotCache& operator=(SnapshotCache&& r) noexcept {
entries = std::move(r.entries);
arena = r.arena;
return *this;
}
bool empty() const {
// Returns true iff anything is known about the contents of the snapshot

View File

@ -72,9 +72,9 @@ struct StorageServerInterface {
RequestStream<ReplyPromise<KeyValueStoreType>> getKeyValueStoreType;
RequestStream<struct WatchValueRequest> watchValue;
RequestStream<struct ReadHotSubRangeRequest> getReadHotRanges;
explicit StorageServerInterface(UID uid) : uniqueID( uid ) {}
StorageServerInterface() : uniqueID( deterministicRandom()->randomUniqueID() ) {}
bool isCacheServer;
explicit StorageServerInterface(UID uid) : uniqueID( uid ), isCacheServer(false) {}
StorageServerInterface() : uniqueID( deterministicRandom()->randomUniqueID() ), isCacheServer(false) {}
NetworkAddress address() const { return getValue.getEndpoint().getPrimaryAddress(); }
NetworkAddress stableAddress() const { return getValue.getEndpoint().getStableAddress(); }
Optional<NetworkAddress> secondaryAddress() const { return getValue.getEndpoint().addresses.secondaryAddress; }
@ -86,7 +86,7 @@ struct StorageServerInterface {
//To change this serialization, ProtocolVersion::ServerListValue must be updated, and downgrades need to be considered
if (ar.protocolVersion().hasSmallEndpoints()) {
serializer(ar, uniqueID, locality, getValue);
serializer(ar, uniqueID, locality, getValue, isCacheServer);
if( Ar::isDeserializing ) {
getKey = RequestStream<struct GetKeyRequest>( getValue.getEndpoint().getAdjustedEndpoint(1) );
getKeyValues = RequestStream<struct GetKeyValuesRequest>( getValue.getEndpoint().getAdjustedEndpoint(2) );
@ -108,6 +108,7 @@ struct StorageServerInterface {
serializer(ar, uniqueID, locality, getValue, getKey, getKeyValues, getShardState, waitMetrics,
splitMetrics, getStorageMetrics, waitFailure, getQueuingMetrics, getKeyValueStoreType);
if (ar.protocolVersion().hasWatches()) serializer(ar, watchValue);
if (ar.protocolVersion().hasCacheRole()) serializer(ar, isCacheServer);
}
}
bool operator == (StorageServerInterface const& s) const { return uniqueID == s.uniqueID; }
@ -157,13 +158,14 @@ struct ServerCacheInfo {
struct GetValueReply : public LoadBalancedReply {
constexpr static FileIdentifier file_identifier = 1378929;
Optional<Value> value;
bool cached;
GetValueReply() {}
GetValueReply(Optional<Value> value) : value(value) {}
GetValueReply() : cached(false) {}
GetValueReply(Optional<Value> value, bool cached) : value(value), cached(cached) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, value);
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, value, cached);
}
};
@ -188,12 +190,13 @@ struct WatchValueReply {
constexpr static FileIdentifier file_identifier = 3;
Version version;
bool cached = false;
WatchValueReply() = default;
explicit WatchValueReply(Version version) : version(version) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version);
serializer(ar, version, cached);
}
};
@ -221,13 +224,13 @@ struct GetKeyValuesReply : public LoadBalancedReply {
VectorRef<KeyValueRef, VecSerStrategy::String> data;
Version version; // useful when latestVersion was requested
bool more;
bool cached;
bool cached = false;
GetKeyValuesReply() : version(invalidVersion), more(false), cached(false) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, data, version, more, arena);
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, data, version, more, cached, arena);
}
};
@ -252,13 +255,14 @@ struct GetKeyValuesRequest : TimedRequest {
struct GetKeyReply : public LoadBalancedReply {
constexpr static FileIdentifier file_identifier = 11226513;
KeySelector sel;
bool cached;
GetKeyReply() {}
GetKeyReply(KeySelector sel) : sel(sel) {}
GetKeyReply() : cached(false) {}
GetKeyReply(KeySelector sel, bool cached) : sel(sel), cached(cached) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, sel);
serializer(ar, LoadBalancedReply::penalty, LoadBalancedReply::error, sel, cached);
}
};

View File

@ -19,10 +19,12 @@
*/
#include "fdbclient/SystemData.h"
#include "fdbclient/StorageServerInterface.h"
#include "flow/TDMetric.actor.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/StorageServerInterface.h"
#include "flow/Arena.h"
#include "flow/TDMetric.actor.h"
#include "flow/serialize.h"
const KeyRef systemKeysPrefix = LiteralStringRef("\xff");
const KeyRangeRef normalKeys(KeyRef(), systemKeysPrefix);
@ -200,6 +202,29 @@ const KeyRangeRef writeConflictRangeKeysRange =
KeyRangeRef(LiteralStringRef("\xff\xff/transaction/write_conflict_range/"),
LiteralStringRef("\xff\xff/transaction/write_conflict_range/\xff\xff"));
// "\xff/cacheServer/[[UID]] := StorageServerInterface"
// This will be added by the cache server on initialization and removed by DD
// TODO[mpilman]: We will need a way to map uint16_t ids to UIDs in a future
// versions. For now caches simply cache everything so the ids
// are not yet meaningful.
const KeyRangeRef storageCacheServerKeys(LiteralStringRef("\xff/cacheServer/"),
LiteralStringRef("\xff/cacheServer0"));
const KeyRef storageCacheServersPrefix = storageCacheServerKeys.begin;
const KeyRef storageCacheServersEnd = storageCacheServerKeys.end;
const Key storageCacheServerKey(UID id) {
BinaryWriter wr(Unversioned());
wr.serializeBytes(storageCacheServersPrefix);
wr << id;
return wr.toValue();
}
const Value storageCacheServerValue(const StorageServerInterface& ssi) {
BinaryWriter wr(IncludeVersion());
wr << ssi;
return wr.toValue();
}
const KeyRangeRef ddStatsRange = KeyRangeRef(LiteralStringRef("\xff\xff/metrics/data_distribution_stats/"),
LiteralStringRef("\xff\xff/metrics/data_distribution_stats/\xff\xff"));
@ -526,6 +551,7 @@ StorageServerInterface decodeServerListValue( ValueRef const& value ) {
return s;
}
// processClassKeys.contains(k) iff k.startsWith( processClassKeys.begin ) because '/'+1 == '0'
const KeyRangeRef processClassKeys(
LiteralStringRef("\xff/processClass/"),

View File

@ -62,6 +62,12 @@ void decodeKeyServersValue( Standalone<RangeResultRef> result, const ValueRef& v
void decodeKeyServersValue( std::map<Tag, UID> const& tag_uid, const ValueRef& value,
std::vector<UID>& src, std::vector<UID>& dest );
// "\xff/storageCacheServer/[[UID]] := StorageServerInterface"
extern const KeyRangeRef storageCacheServerKeys;
extern const KeyRef storageCacheServersPrefix, storageCacheServersEnd;
const Key storageCacheServerKey(UID id);
const Value storageCacheServerValue(const StorageServerInterface& ssi);
// "\xff/storageCache/[[begin]]" := "[[vector<uint16_t>]]"
extern const KeyRangeRef storageCacheKeys;
extern const KeyRef storageCachePrefix;

View File

@ -326,12 +326,12 @@ ThreadFuture<Void> ThreadSafeTransaction::onError( Error const& e ) {
return onMainThread( [tr, e](){ return tr->onError(e); } );
}
void ThreadSafeTransaction::operator=(ThreadSafeTransaction&& r) BOOST_NOEXCEPT {
void ThreadSafeTransaction::operator=(ThreadSafeTransaction&& r) noexcept {
tr = r.tr;
r.tr = NULL;
}
ThreadSafeTransaction::ThreadSafeTransaction(ThreadSafeTransaction&& r) BOOST_NOEXCEPT {
ThreadSafeTransaction::ThreadSafeTransaction(ThreadSafeTransaction&& r) noexcept {
tr = r.tr;
r.tr = NULL;
}

View File

@ -97,8 +97,8 @@ public:
// These are to permit use as state variables in actors:
ThreadSafeTransaction() : tr(NULL) {}
void operator=(ThreadSafeTransaction&& r) BOOST_NOEXCEPT;
ThreadSafeTransaction(ThreadSafeTransaction&& r) BOOST_NOEXCEPT;
void operator=(ThreadSafeTransaction&& r) noexcept;
ThreadSafeTransaction(ThreadSafeTransaction&& r) noexcept;
void reset() override;

View File

@ -538,7 +538,8 @@ namespace PTreeImpl {
return;
}
if (p->updated && p->lastUpdateVersion <= newOldestVersion) {
/* If the node has been updated, figure out which pointer was repalced. And delete that pointer */
/* If the node has been updated, figure out which pointer was replaced. And replace that pointer with the updated pointer.
Then we can get rid of the updated child pointer and then make room in the node for future updates */
auto which = p->replacedPointer;
p->pointer[which] = p->pointer[2];
p->updated = false;
@ -611,9 +612,9 @@ public:
VersionedMap() : oldestVersion(0), latestVersion(0) {
roots.emplace_back(0, Tree());
}
VersionedMap( VersionedMap&& v ) BOOST_NOEXCEPT : oldestVersion(v.oldestVersion), latestVersion(v.latestVersion), roots(std::move(v.roots)) {
}
void operator = (VersionedMap && v) BOOST_NOEXCEPT {
VersionedMap(VersionedMap&& v) noexcept
: oldestVersion(v.oldestVersion), latestVersion(v.latestVersion), roots(std::move(v.roots)) {}
void operator=(VersionedMap&& v) noexcept {
oldestVersion = v.oldestVersion;
latestVersion = v.latestVersion;
roots = std::move(v.roots);

View File

@ -145,8 +145,17 @@ public:
PTreeImpl::insert( writes, ver, WriteMapEntry( afterAllKeys, OperationStack(), false, false, false, false, false ) );
}
WriteMap(WriteMap&& r) BOOST_NOEXCEPT : writeMapEmpty(r.writeMapEmpty), writes(std::move(r.writes)), ver(r.ver), scratch_iterator(std::move(r.scratch_iterator)), arena(r.arena) {}
WriteMap& operator=(WriteMap&& r) BOOST_NOEXCEPT { writeMapEmpty = r.writeMapEmpty; writes = std::move(r.writes); ver = r.ver; scratch_iterator = std::move(r.scratch_iterator); arena = r.arena; return *this; }
WriteMap(WriteMap&& r) noexcept
: writeMapEmpty(r.writeMapEmpty), writes(std::move(r.writes)), ver(r.ver),
scratch_iterator(std::move(r.scratch_iterator)), arena(r.arena) {}
WriteMap& operator=(WriteMap&& r) noexcept {
writeMapEmpty = r.writeMapEmpty;
writes = std::move(r.writes);
ver = r.ver;
scratch_iterator = std::move(r.scratch_iterator);
arena = r.arena;
return *this;
}
//a write with addConflict false on top of an existing write with a conflict range will not remove the conflict
void mutate( KeyRef key, MutationRef::Type operation, ValueRef param, bool addConflict ) {

View File

@ -132,7 +132,7 @@ struct OpenFileInfo : NonCopyable {
Future<Reference<IAsyncFile>> opened; // Only valid until the file is fully opened
OpenFileInfo() : f(0) {}
OpenFileInfo(OpenFileInfo && r) BOOST_NOEXCEPT : f(r.f), opened(std::move(r.opened)) { r.f = 0; }
OpenFileInfo(OpenFileInfo&& r) noexcept : f(r.f), opened(std::move(r.opened)) { r.f = 0; }
Future<Reference<IAsyncFile>> get() {
if (f) return Reference<IAsyncFile>::addRef(f);

View File

@ -22,6 +22,11 @@
#define FLOW_MULTIINTERFACE_H
#pragma once
#include "flow/FastRef.h"
#include "fdbrpc/Locality.h"
#include <vector>
extern uint64_t debug_lastLoadBalanceResultEndpointToken;
template <class K, class V>
@ -168,7 +173,7 @@ class MultiInterface : public ReferenceCounted<MultiInterface<T>> {
template <class T>
class MultiInterface<ReferencedInterface<T>> : public ReferenceCounted<MultiInterface<ReferencedInterface<T>>> {
public:
MultiInterface( const vector<Reference<ReferencedInterface<T>>>& v ) : alternatives(v), bestCount(0) {
MultiInterface( const std::vector<Reference<ReferencedInterface<T>>>& v ) : alternatives(v), bestCount(0) {
deterministicRandom()->randomShuffle(alternatives);
if ( LBLocalityData<T>::Present ) {
std::stable_sort( alternatives.begin(), alternatives.end(), ReferencedInterface<T>::sort_by_distance );
@ -204,6 +209,18 @@ public:
T const& getInterface(int index) { return alternatives[index]->interf; }
UID getId( int index ) const { return alternatives[index]->interf.id(); }
bool hasInterface(UID id) const {
for (const auto& ref : alternatives) {
if (ref->interf.id() == id) {
return true;
}
}
return false;
}
Reference<ReferencedInterface<T>>& operator[](int i) { return alternatives[i]; }
const Reference<ReferencedInterface<T>>& operator[](int i) const { return alternatives[i]; }
virtual ~MultiInterface() {}
@ -211,7 +228,7 @@ public:
return describe( alternatives );
}
private:
vector<Reference<ReferencedInterface<T>>> alternatives;
std::vector<Reference<ReferencedInterface<T>>> alternatives;
int16_t bestCount;
};

View File

@ -150,7 +150,7 @@ public:
void coalesce( const Range& k );
void validateCoalesced();
void operator=(RangeMap&& r) BOOST_NOEXCEPT { map = std::move(r.map); }
void operator=(RangeMap&& r) noexcept { map = std::move(r.map); }
//void clear( const Val& value ) { ranges.clear(); ranges.insert(std::make_pair(Key(),value)); }
void insert( const Range& keys, const Val& value );

View File

@ -121,7 +121,7 @@ public:
bool isValid() const { return sav != NULL; }
ReplyPromise() : sav(new NetSAV<T>(0, 1)) {}
ReplyPromise(const ReplyPromise& rhs) : sav(rhs.sav) { sav->addPromiseRef(); }
ReplyPromise(ReplyPromise&& rhs) BOOST_NOEXCEPT : sav(rhs.sav) { rhs.sav = 0; }
ReplyPromise(ReplyPromise&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; }
~ReplyPromise() { if (sav) sav->delPromiseRef(); }
ReplyPromise(const Endpoint& endpoint) : sav(new NetSAV<T>(0, 1, endpoint)) {}
@ -132,7 +132,7 @@ public:
if (sav) sav->delPromiseRef();
sav = rhs.sav;
}
void operator=(ReplyPromise && rhs) BOOST_NOEXCEPT {
void operator=(ReplyPromise&& rhs) noexcept {
if (sav != rhs.sav) {
if (sav) sav->delPromiseRef();
sav = rhs.sav;
@ -363,13 +363,13 @@ public:
FutureStream<T> getFuture() const { queue->addFutureRef(); return FutureStream<T>(queue); }
RequestStream() : queue(new NetNotifiedQueue<T>(0, 1)) {}
RequestStream(const RequestStream& rhs) : queue(rhs.queue) { queue->addPromiseRef(); }
RequestStream(RequestStream&& rhs) BOOST_NOEXCEPT : queue(rhs.queue) { rhs.queue = 0; }
RequestStream(RequestStream&& rhs) noexcept : queue(rhs.queue) { rhs.queue = 0; }
void operator=(const RequestStream& rhs) {
rhs.queue->addPromiseRef();
if (queue) queue->delPromiseRef();
queue = rhs.queue;
}
void operator=(RequestStream&& rhs) BOOST_NOEXCEPT {
void operator=(RequestStream&& rhs) noexcept {
if (queue != rhs.queue) {
if (queue) queue->delPromiseRef();
queue = rhs.queue;

View File

@ -1630,10 +1630,18 @@ public:
Promise<Void> action;
Task( double time, TaskPriority taskID, uint64_t stable, ProcessInfo* machine, Promise<Void>&& action ) : time(time), taskID(taskID), stable(stable), machine(machine), action(std::move(action)) {}
Task( double time, TaskPriority taskID, uint64_t stable, ProcessInfo* machine, Future<Void>& future ) : time(time), taskID(taskID), stable(stable), machine(machine) { future = action.getFuture(); }
Task(Task&& rhs) BOOST_NOEXCEPT : time(rhs.time), taskID(rhs.taskID), stable(rhs.stable), machine(rhs.machine), action(std::move(rhs.action)) {}
Task(Task&& rhs) noexcept
: time(rhs.time), taskID(rhs.taskID), stable(rhs.stable), machine(rhs.machine),
action(std::move(rhs.action)) {}
void operator= ( Task const& rhs ) { taskID = rhs.taskID; time = rhs.time; stable = rhs.stable; machine = rhs.machine; action = rhs.action; }
Task( Task const& rhs ) : taskID(rhs.taskID), time(rhs.time), stable(rhs.stable), machine(rhs.machine), action(rhs.action) {}
void operator= (Task&& rhs) BOOST_NOEXCEPT { time = rhs.time; taskID = rhs.taskID; stable = rhs.stable; machine = rhs.machine; action = std::move(rhs.action); }
void operator=(Task&& rhs) noexcept {
time = rhs.time;
taskID = rhs.taskID;
stable = rhs.stable;
machine = rhs.machine;
action = std::move(rhs.action);
}
bool operator < (Task const& rhs) const {
// Ordering is reversed for priority_queue

View File

@ -144,7 +144,7 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef<MutationRe
{
MutationRef privatized = m;
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivateMutation", dbgid).detail("Original", m.toString()).detail("Privatized", privatized.toString());
//TraceEvent(SevDebug, "SendingPrivateMutation", dbgid).detail("Original", m.toString()).detail("Privatized", privatized.toString());
cachedRangeInfo[k] = privatized;
}
if(k != allKeys.end) {
@ -161,7 +161,7 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef<MutationRe
if(toCommit) {
MutationRef privatized = m;
privatized.param1 = m.param1.withPrefix(systemKeys.begin, arena);
TraceEvent(SevDebug, "SendingPrivateMutation", dbgid).detail("Original", m.toString()).detail("Privatized", privatized.toString());
//TraceEvent(SevDebug, "SendingPrivateMutation", dbgid).detail("Original", m.toString()).detail("Privatized", privatized.toString());
toCommit->addTag( cacheTag );
toCommit->addTypedMessage(privatized);
}
@ -276,6 +276,7 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef<MutationRe
allTags.insert(decodeServerTagValue(kv.value));
}
}
allTags.insert(cacheTag);
if (m.param1 == lastEpochEndKey) {
toCommit->addTags(allTags);
@ -494,14 +495,24 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef<MutationRe
keyBegin = itr->first;
mutationBegin = itr->second;
++itr;
keyEnd = itr->first;
mutationEnd = itr->second;
if (itr != cachedRangeInfo.end()) {
keyEnd = itr->first;
mutationEnd = itr->second;
} else {
//TraceEvent(SevDebug, "EndKeyNotFound", dbgid).detail("KeyBegin", keyBegin.toString());
break;
}
} else {
keyEnd = itr->first;
mutationEnd = itr->second;
++itr;
keyBegin = itr->first;
mutationBegin = itr->second;
if (itr != cachedRangeInfo.end()) {
keyBegin = itr->first;
mutationBegin = itr->second;
} else {
//TraceEvent(SevDebug, "BeginKeyNotFound", dbgid).detail("KeyEnd", keyEnd.toString());
break;
}
}
// Now get all the storage server tags for the cached key-ranges

View File

@ -124,6 +124,7 @@ set(FDBSERVER_SRCS
workloads/BackupToDBUpgrade.actor.cpp
workloads/BulkLoad.actor.cpp
workloads/BulkSetup.actor.h
workloads/Cache.actor.cpp
workloads/ChangeConfig.actor.cpp
workloads/ClientTransactionProfileCorrectness.actor.cpp
workloads/TriggerRecovery.actor.cpp

View File

@ -61,17 +61,17 @@ struct WorkerInfo : NonCopyable {
WorkerDetails details;
Future<Void> haltRatekeeper;
Future<Void> haltDistributor;
Optional<uint16_t> storageCacheInfo;
Standalone<VectorRef<StringRef>> issues;
WorkerInfo() : gen(-1), reboots(0), priorityInfo(ProcessClass::UnsetFit, false, ClusterControllerPriorityInfo::FitnessUnknown) {}
WorkerInfo( Future<Void> watcher, ReplyPromise<RegisterWorkerReply> reply, Generation gen, WorkerInterface interf, ProcessClass initialClass, ProcessClass processClass, ClusterControllerPriorityInfo priorityInfo, bool degraded, Standalone<VectorRef<StringRef>> issues ) :
watcher(watcher), reply(reply), gen(gen), reboots(0), initialClass(initialClass), priorityInfo(priorityInfo), details(interf, processClass, degraded), issues(issues) {}
WorkerInfo( WorkerInfo&& r ) BOOST_NOEXCEPT : watcher(std::move(r.watcher)), reply(std::move(r.reply)), gen(r.gen),
reboots(r.reboots), initialClass(r.initialClass), priorityInfo(r.priorityInfo), details(std::move(r.details)),
haltRatekeeper(r.haltRatekeeper), haltDistributor(r.haltDistributor), storageCacheInfo(r.storageCacheInfo), issues(r.issues) {}
void operator=( WorkerInfo&& r ) BOOST_NOEXCEPT {
WorkerInfo(WorkerInfo&& r) noexcept
: watcher(std::move(r.watcher)), reply(std::move(r.reply)), gen(r.gen), reboots(r.reboots),
initialClass(r.initialClass), priorityInfo(r.priorityInfo), details(std::move(r.details)),
haltRatekeeper(r.haltRatekeeper), haltDistributor(r.haltDistributor), issues(r.issues) {}
void operator=(WorkerInfo&& r) noexcept {
watcher = std::move(r.watcher);
reply = std::move(r.reply);
gen = r.gen;
@ -81,7 +81,6 @@ struct WorkerInfo : NonCopyable {
details = std::move(r.details);
haltRatekeeper = r.haltRatekeeper;
haltDistributor = r.haltDistributor;
storageCacheInfo = r.storageCacheInfo;
issues = r.issues;
}
};
@ -111,7 +110,6 @@ public:
Database db;
int unfinishedRecoveries;
int logGenerations;
std::map<uint16_t, std::pair<Optional<StorageServerInterface>, Optional<Key>>> cacheInterfaces;
bool cachePopulated;
std::map<NetworkAddress, std::pair<double, OpenDatabaseRequest>> clientStatus;
@ -138,28 +136,6 @@ public:
serverInfo->set( newInfo );
}
void setStorageCache(uint16_t id, const StorageServerInterface& interf) {
auto newInfo = serverInfo->get();
bool found = false;
for(auto& it : newInfo.storageCaches) {
if(it.first == id) {
if(it.second != interf) {
newInfo.id = deterministicRandom()->randomUniqueID();
newInfo.infoGeneration = ++dbInfoCount;
it.second = interf;
}
found = true;
break;
}
}
if(!found) {
newInfo.id = deterministicRandom()->randomUniqueID();
newInfo.infoGeneration = ++dbInfoCount;
newInfo.storageCaches.push_back(std::make_pair(id, interf));
}
serverInfo->set( newInfo );
}
void clearInterf(ProcessClass::ClassType t) {
auto newInfo = serverInfo->get();
newInfo.id = deterministicRandom()->randomUniqueID();
@ -172,18 +148,6 @@ public:
serverInfo->set( newInfo );
}
void clearStorageCache(uint16_t id) {
auto newInfo = serverInfo->get();
for(auto it = newInfo.storageCaches.begin(); it != newInfo.storageCaches.end(); ++it) {
if(it->first == id) {
newInfo.id = deterministicRandom()->randomUniqueID();
newInfo.infoGeneration = ++dbInfoCount;
newInfo.storageCaches.erase(it);
break;
}
}
serverInfo->set( newInfo );
}
};
struct UpdateWorkerList {
@ -365,7 +329,7 @@ public:
logServerMap->add(worker.interf.locality, &worker);
}
}
if (logServerSet->size() < (addingDegraded == 0 ? desired : required)) {
}
else if (logServerSet->size() == required || logServerSet->size() <= desired) {
@ -1441,7 +1405,6 @@ ACTOR Future<Void> clusterWatchDatabase( ClusterControllerData* cluster, Cluster
dbInfo.clusterInterface = db->serverInfo->get().clusterInterface;
dbInfo.distributor = db->serverInfo->get().distributor;
dbInfo.ratekeeper = db->serverInfo->get().ratekeeper;
dbInfo.storageCaches = db->serverInfo->get().storageCaches;
dbInfo.latencyBandConfig = db->serverInfo->get().latencyBandConfig;
TraceEvent("CCWDB", cluster->id).detail("Lifetime", dbInfo.masterLifetime.toString()).detail("ChangeID", dbInfo.id);
@ -1496,7 +1459,7 @@ ACTOR Future<Void> clusterOpenDatabase(ClusterControllerData::DBInfo* db, OpenDa
if(db->clientStatus.size() > 10000) {
TraceEvent(SevWarnAlways, "TooManyClientStatusEntries").suppressFor(1.0);
}
while (db->clientInfo->get().id == req.knownClientInfoID) {
choose {
when (wait( db->clientInfo->onChange() )) {}
@ -1747,27 +1710,9 @@ ACTOR Future<Void> workerAvailabilityWatch( WorkerInterface worker, ProcessClass
}
when( wait( failed ) ) { // remove workers that have failed
WorkerInfo& failedWorkerInfo = cluster->id_worker[ worker.locality.processId() ];
if(failedWorkerInfo.storageCacheInfo.present()) {
bool found = false;
for(auto& it : cluster->id_worker) {
if(!it.second.storageCacheInfo.present() && it.second.details.processClass == ProcessClass::StorageCacheClass) {
found = true;
it.second.storageCacheInfo = failedWorkerInfo.storageCacheInfo;
cluster->db.cacheInterfaces[failedWorkerInfo.storageCacheInfo.get()] = std::make_pair(Optional<StorageServerInterface>(), it.first);
if(!it.second.reply.isSet()) {
it.second.reply.send( RegisterWorkerReply(it.second.details.processClass, it.second.priorityInfo, failedWorkerInfo.storageCacheInfo) );
}
break;
}
}
if(!found) {
cluster->db.cacheInterfaces[failedWorkerInfo.storageCacheInfo.get()] = std::make_pair(Optional<StorageServerInterface>(), Optional<Key>());
}
cluster->db.clearStorageCache(failedWorkerInfo.storageCacheInfo.get());
}
if (!failedWorkerInfo.reply.isSet()) {
failedWorkerInfo.reply.send( RegisterWorkerReply(failedWorkerInfo.details.processClass, failedWorkerInfo.priorityInfo, Optional<uint16_t>()) );
failedWorkerInfo.reply.send( RegisterWorkerReply(failedWorkerInfo.details.processClass, failedWorkerInfo.priorityInfo) );
}
if (worker.locality.processId() == cluster->masterProcessId) {
cluster->masterProcessId = Optional<Key>();
@ -2055,7 +2000,7 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
if ( it.second.priorityInfo.isExcluded != isExcludedFromConfig ) {
it.second.priorityInfo.isExcluded = isExcludedFromConfig;
if( !it.second.reply.isSet() ) {
it.second.reply.send( RegisterWorkerReply( it.second.details.processClass, it.second.priorityInfo, it.second.storageCacheInfo ) );
it.second.reply.send( RegisterWorkerReply( it.second.details.processClass, it.second.priorityInfo ) );
}
}
}
@ -2228,56 +2173,10 @@ void registerWorker( RegisterWorkerRequest req, ClusterControllerData *self ) {
}
}
}
Optional<uint16_t> newStorageCache = req.storageCacheInterf.present() ? req.storageCacheInterf.get().first : Optional<uint16_t>();
auto& cacheInfo = self->id_worker[w.locality.processId()].storageCacheInfo;
if (req.storageCacheInterf.present()) {
auto it = self->db.cacheInterfaces.find(req.storageCacheInterf.get().first);
if(it == self->db.cacheInterfaces.end()) {
if(self->db.cachePopulated) {
if(cacheInfo.present()) {
self->db.clearStorageCache(cacheInfo.get());
}
newStorageCache = Optional<uint16_t>();
cacheInfo = Optional<uint16_t>();
} else {
self->db.setStorageCache(req.storageCacheInterf.get().first, req.storageCacheInterf.get().second);
self->db.cacheInterfaces[req.storageCacheInterf.get().first] = std::make_pair(req.storageCacheInterf.get().second, w.locality.processId());
cacheInfo = req.storageCacheInterf.get().first;
}
} else {
if(!it->second.second.present() || (cacheInfo.present() && cacheInfo.get() == it->first) ) {
self->db.setStorageCache(req.storageCacheInterf.get().first, req.storageCacheInterf.get().second);
it->second = std::make_pair(req.storageCacheInterf.get().second, w.locality.processId());
cacheInfo = req.storageCacheInterf.get().first;
}
else {
if(cacheInfo.present()) {
self->db.clearStorageCache(cacheInfo.get());
}
newStorageCache = Optional<uint16_t>();
cacheInfo = Optional<uint16_t>();
}
}
} else {
newStorageCache = cacheInfo;
}
if(self->gotProcessClasses && newProcessClass == ProcessClass::StorageCacheClass && !newStorageCache.present()) {
for(auto& it : self->db.cacheInterfaces) {
if(!it.second.second.present()) {
it.second.second = w.locality.processId();
self->id_worker[w.locality.processId()].storageCacheInfo = it.first;
newStorageCache = it.first;
break;
}
}
}
// Notify the worker to register again with new process class/exclusive property
if ( !req.reply.isSet() && ( newPriorityInfo != req.priorityInfo ||
newStorageCache.present() != req.storageCacheInterf.present() ||
(newStorageCache.present() && newStorageCache.get() != req.storageCacheInterf.get().first) ) ) {
req.reply.send( RegisterWorkerReply(newProcessClass, newPriorityInfo, newStorageCache) );
if ( !req.reply.isSet() && newPriorityInfo != req.priorityInfo ) {
req.reply.send( RegisterWorkerReply(newProcessClass, newPriorityInfo) );
}
}
@ -2504,7 +2403,7 @@ ACTOR Future<Void> monitorProcessClasses(ClusterControllerData *self) {
w.second.details.processClass = newProcessClass;
w.second.priorityInfo.processClassFitness = newProcessClass.machineClassFitness(ProcessClass::ClusterController);
if (!w.second.reply.isSet()) {
w.second.reply.send( RegisterWorkerReply(w.second.details.processClass, w.second.priorityInfo, w.second.storageCacheInfo) );
w.second.reply.send( RegisterWorkerReply(w.second.details.processClass, w.second.priorityInfo) );
}
}
}
@ -2558,81 +2457,7 @@ ACTOR Future<Void> monitorServerInfoConfig(ClusterControllerData::DBInfo* db) {
break;
}
catch (Error &e) {
wait(tr.onError(e));
}
}
}
}
ACTOR Future<Void> monitorStorageCache(ClusterControllerData* self) {
loop {
state ReadYourWritesTransaction tr(self->db.db);
loop {
try {
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::READ_LOCK_AWARE);
Optional<Value> changeVal = wait(tr.get(cacheChangeKey));
Standalone<RangeResultRef> changeKeys = wait(tr.getRange(cacheChangeKeys, CLIENT_KNOBS->TOO_MANY));
ASSERT( !changeKeys.more && changeKeys.size() < CLIENT_KNOBS->TOO_MANY );
std::set<uint16_t> changeIDs;
for(auto& it : changeKeys) {
changeIDs.insert(cacheChangeKeyDecodeIndex(it.key));
}
for(auto& it : changeIDs) {
if(!self->db.cacheInterfaces.count(it)) {
self->db.cacheInterfaces[it] = std::make_pair(Optional<StorageServerInterface>(), Optional<Key>());
}
}
std::vector<uint16_t> removeIDs;
for(auto& it : self->db.cacheInterfaces) {
if(!changeIDs.count(it.first)) {
removeIDs.push_back(it.first);
if(it.second.second.present()) {
self->id_worker[it.second.second.get()].storageCacheInfo = Optional<uint16_t>();
}
self->db.clearStorageCache(it.first);
}
}
for(auto& it : removeIDs) {
self->db.cacheInterfaces.erase(it);
}
for(auto& c : self->db.cacheInterfaces) {
if(!c.second.second.present()) {
bool found = false;
for(auto& it : self->id_worker) {
if(!it.second.storageCacheInfo.present() && it.second.details.processClass == ProcessClass::StorageCacheClass) {
found = true;
it.second.storageCacheInfo = c.first;
c.second.second = it.first;
if(!it.second.reply.isSet()) {
it.second.reply.send( RegisterWorkerReply(it.second.details.processClass, it.second.priorityInfo, c.first) );
}
break;
}
}
if(!found) {
break;
}
}
}
state Future<Void> configChangeFuture = tr.watch(cacheChangeKey);
self->db.cachePopulated = true;
wait(tr.commit());
wait(configChangeFuture);
break;
}
catch (Error &e) {
wait(tr.onError(e));
wait(tr.onError(e));
}
}
}
@ -2688,7 +2513,7 @@ ACTOR Future<Void> updatedChangingDatacenters(ClusterControllerData *self) {
if ( worker.priorityInfo.dcFitness > newFitness ) {
worker.priorityInfo.dcFitness = newFitness;
if(!worker.reply.isSet()) {
worker.reply.send( RegisterWorkerReply( worker.details.processClass, worker.priorityInfo, worker.storageCacheInfo ) );
worker.reply.send( RegisterWorkerReply( worker.details.processClass, worker.priorityInfo ) );
}
} else {
state int currentFit = ProcessClass::BestFit;
@ -2701,7 +2526,7 @@ ACTOR Future<Void> updatedChangingDatacenters(ClusterControllerData *self) {
updated = true;
it.second.priorityInfo.dcFitness = fitness;
if(!it.second.reply.isSet()) {
it.second.reply.send( RegisterWorkerReply( it.second.details.processClass, it.second.priorityInfo, it.second.storageCacheInfo ) );
it.second.reply.send( RegisterWorkerReply( it.second.details.processClass, it.second.priorityInfo ) );
}
}
}
@ -2740,7 +2565,7 @@ ACTOR Future<Void> updatedChangedDatacenters(ClusterControllerData *self) {
if( worker.priorityInfo.dcFitness != newFitness ) {
worker.priorityInfo.dcFitness = newFitness;
if(!worker.reply.isSet()) {
worker.reply.send( RegisterWorkerReply( worker.details.processClass, worker.priorityInfo, worker.storageCacheInfo ) );
worker.reply.send( RegisterWorkerReply( worker.details.processClass, worker.priorityInfo ) );
}
}
} else {
@ -2754,7 +2579,7 @@ ACTOR Future<Void> updatedChangedDatacenters(ClusterControllerData *self) {
updated = true;
it.second.priorityInfo.dcFitness = fitness;
if(!it.second.reply.isSet()) {
it.second.reply.send( RegisterWorkerReply( it.second.details.processClass, it.second.priorityInfo, it.second.storageCacheInfo ) );
it.second.reply.send( RegisterWorkerReply( it.second.details.processClass, it.second.priorityInfo ) );
}
}
}
@ -2908,7 +2733,7 @@ ACTOR Future<DataDistributorInterface> startDataDistributor( ClusterControllerDa
if (self->onMasterIsBetter(worker, ProcessClass::DataDistributor)) {
worker = self->id_worker[self->masterProcessId.get()].details;
}
InitializeDataDistributorRequest req(deterministicRandom()->randomUniqueID());
TraceEvent("CCDataDistributorRecruit", self->id).detail("Addr", worker.interf.address());
@ -3091,7 +2916,6 @@ ACTOR Future<Void> clusterControllerCore( ClusterControllerFullInterface interf,
self.addActor.send( handleForcedRecoveries(&self, interf) );
self.addActor.send( monitorDataDistributor(&self) );
self.addActor.send( monitorRatekeeper(&self) );
self.addActor.send( monitorStorageCache(&self) );
self.addActor.send( dbInfoUpdater(&self) );
self.addActor.send( traceCounters("ClusterControllerMetrics", self.id, SERVER_KNOBS->STORAGE_LOGGING_DELAY, &self.clusterControllerMetrics, self.id.toString() + "/ClusterControllerMetrics") );
self.addActor.send( traceRole(Role::CLUSTER_CONTROLLER, interf.id()) );

View File

@ -20,6 +20,10 @@
#include <set>
#include <sstream>
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/ManagementAPI.actor.h"
@ -35,9 +39,11 @@
#include "fdbserver/TLogInterface.h"
#include "fdbserver/WaitFailure.h"
#include "flow/ActorCollection.h"
#include "flow/Arena.h"
#include "flow/Trace.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // This must be the last #include.
#include "flow/serialize.h"
class TCTeamInfo;
struct TCMachineInfo;
@ -4853,6 +4859,56 @@ ACTOR Future<Void> ddExclusionSafetyCheck(DistributorExclusionSafetyCheckRequest
return Void();
}
ACTOR Future<Void> waitFailCacheServer(Database* db, StorageServerInterface ssi) {
state Transaction tr(*db);
state Key key = storageCacheServerKey(ssi.id());
wait(waitFailureClient(ssi.waitFailure));
loop {
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
tr.addReadConflictRange(storageCacheServerKeys);
tr.clear(key);
wait(tr.commit());
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
return Void();
}
ACTOR Future<Void> cacheServerWatcher(Database* db) {
state Transaction tr(*db);
state ActorCollection actors(false);
state std::set<UID> knownCaches;
loop {
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
try {
Standalone<RangeResultRef> range = wait(tr.getRange(storageCacheServerKeys, CLIENT_KNOBS->TOO_MANY));
ASSERT(!range.more);
std::set<UID> caches;
for (auto& kv : range) {
UID id;
BinaryReader reader{kv.key.removePrefix(storageCacheServersPrefix), Unversioned()};
reader >> id;
caches.insert(id);
if (knownCaches.find(id) == knownCaches.end()) {
StorageServerInterface ssi;
BinaryReader reader{kv.value, IncludeVersion()};
reader >> ssi;
actors.add(waitFailCacheServer(db, ssi));
}
}
knownCaches = std::move(caches);
tr.reset();
wait(delay(5.0) || actors.getResult());
ASSERT(!actors.getResult().isReady());
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<Void> dataDistributor(DataDistributorInterface di, Reference<AsyncVar<struct ServerDBInfo>> db ) {
state Reference<DataDistributorData> self( new DataDistributorData(db, di.id()) );
state Future<Void> collection = actorCollection( self->addActor.getFuture() );
@ -4865,6 +4921,7 @@ ACTOR Future<Void> dataDistributor(DataDistributorInterface di, Reference<AsyncV
try {
TraceEvent("DataDistributorRunning", di.id());
self->addActor.send( waitFailureServer(di.waitFailure.getFuture()) );
self->addActor.send(cacheServerWatcher(&cx));
state Future<Void> distributor = reportErrorsExcept( dataDistribution(self, getShardMetricsList), "DataDistribution", di.id(), &normalDataDistributorErrors() );
loop choose {

View File

@ -1013,7 +1013,7 @@ private:
ASSERT( nextPageSeq%sizeof(Page)==0 );
auto& p = backPage();
memset(&p, 0, sizeof(Page)); // FIXME: unnecessary?
memset(static_cast<void*>(&p), 0, sizeof(Page)); // FIXME: unnecessary?
p.magic = 0xFDB;
switch (diskQueueVersion) {
case DiskQueueVersion::V0:

View File

@ -629,6 +629,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( REDWOOD_DEFAULT_PAGE_SIZE, 4096 );
init( REDWOOD_KVSTORE_CONCURRENT_READS, 64 );
init( REDWOOD_COMMIT_CONCURRENT_READS, 64 );
init( REDWOOD_PAGE_REBUILD_FILL_FACTOR, 0.66 );
init( REDWOOD_LAZY_CLEAR_BATCH_SIZE_PAGES, 10 );
init( REDWOOD_LAZY_CLEAR_MIN_PAGES, 0 );

View File

@ -561,6 +561,7 @@ public:
int REDWOOD_DEFAULT_PAGE_SIZE; // Page size for new Redwood files
int REDWOOD_KVSTORE_CONCURRENT_READS; // Max number of simultaneous point or range reads in progress.
int REDWOOD_COMMIT_CONCURRENT_READS; // Max number of concurrent reads done to support commit operations
double REDWOOD_PAGE_REBUILD_FILL_FACTOR; // When rebuilding pages, start a new page after this capacity
int REDWOOD_LAZY_CLEAR_BATCH_SIZE_PAGES; // Number of pages to try to pop from the lazy delete queue and process at once
int REDWOOD_LAZY_CLEAR_MIN_PAGES; // Minimum number of pages to free before ending a lazy clear cycle, unless the queue is empty

View File

@ -42,8 +42,10 @@ struct LogRouterData {
TagData( Tag tag, Version popped, Version durableKnownCommittedVersion ) : tag(tag), popped(popped), durableKnownCommittedVersion(durableKnownCommittedVersion) {}
TagData(TagData&& r) BOOST_NOEXCEPT : version_messages(std::move(r.version_messages)), tag(r.tag), popped(r.popped), durableKnownCommittedVersion(r.durableKnownCommittedVersion) {}
void operator= (TagData&& r) BOOST_NOEXCEPT {
TagData(TagData&& r) noexcept
: version_messages(std::move(r.version_messages)), tag(r.tag), popped(r.popped),
durableKnownCommittedVersion(r.durableKnownCommittedVersion) {}
void operator=(TagData&& r) noexcept {
version_messages = std::move(r.version_messages);
tag = r.tag;
popped = r.popped;

View File

@ -738,10 +738,10 @@ void ILogSystem::SetPeekCursor::advanceTo(LogMessageVersion n) {
ACTOR Future<Void> setPeekGetMore(ILogSystem::SetPeekCursor* self, LogMessageVersion startVersion, TaskPriority taskID) {
loop {
//TraceEvent("LPC_GetMore1", self->randomID).detail("Start", startVersion.toString()).detail("Tag", self->tag);
//TraceEvent("LPC_GetMore1", self->randomID).detail("Start", startVersion.toString()).detail("Tag", self->tag.toString());
if(self->bestServer >= 0 && self->bestSet >= 0 && self->serverCursors[self->bestSet][self->bestServer]->isActive()) {
ASSERT(!self->serverCursors[self->bestSet][self->bestServer]->hasMessage());
//TraceEvent("LPC_GetMore2", self->randomID).detail("Start", startVersion.toString()).detail("Tag", self->tag);
//TraceEvent("LPC_GetMore2", self->randomID).detail("Start", startVersion.toString()).detail("Tag", self->tag.toString());
wait( self->serverCursors[self->bestSet][self->bestServer]->getMore(taskID) || self->serverCursors[self->bestSet][self->bestServer]->onFailed() );
self->useBestSet = true;
} else {
@ -778,7 +778,7 @@ ACTOR Future<Void> setPeekGetMore(ILogSystem::SetPeekCursor* self, LogMessageVer
} else {
//FIXME: this will peeking way too many cursors when satellites exist, and does not need to peek bestSet cursors since we cannot get anymore data from them
vector<Future<Void>> q;
//TraceEvent("LPC_GetMore4", self->randomID).detail("Start", startVersion.toString()).detail("Tag", self->tag);
//TraceEvent("LPC_GetMore4", self->randomID).detail("Start", startVersion.toString()).detail("Tag", self->tag.toString());
for(auto& cursors : self->serverCursors) {
for (auto& c :cursors) {
if (!c->hasMessage()) {

View File

@ -324,17 +324,19 @@ namespace oldTLog_4_6 {
TagData( Version popped, bool nothing_persistent, bool popped_recently, OldTag tag ) : nothing_persistent(nothing_persistent), popped(popped), popped_recently(popped_recently), update_version_sizes(tag != txsTagOld) {}
TagData(TagData&& r) BOOST_NOEXCEPT : version_messages(std::move(r.version_messages)), nothing_persistent(r.nothing_persistent), popped_recently(r.popped_recently), popped(r.popped), update_version_sizes(r.update_version_sizes) {}
void operator= (TagData&& r) BOOST_NOEXCEPT {
version_messages = std::move(r.version_messages);
nothing_persistent = r.nothing_persistent;
TagData(TagData&& r) noexcept
: version_messages(std::move(r.version_messages)), nothing_persistent(r.nothing_persistent),
popped_recently(r.popped_recently), popped(r.popped), update_version_sizes(r.update_version_sizes) {}
void operator=(TagData&& r) noexcept {
version_messages = std::move(r.version_messages);
nothing_persistent = r.nothing_persistent;
popped_recently = r.popped_recently;
popped = r.popped;
update_version_sizes = r.update_version_sizes;
}
}
// Erase messages not needed to update *from* versions >= before (thus, messages with toversion <= before)
ACTOR Future<Void> eraseMessagesBefore( TagData *self, Version before, int64_t* gBytesErased, Reference<LogData> tlogData, TaskPriority taskID ) {
// Erase messages not needed to update *from* versions >= before (thus, messages with toversion <= before)
ACTOR Future<Void> eraseMessagesBefore( TagData *self, Version before, int64_t* gBytesErased, Reference<LogData> tlogData, TaskPriority taskID ) {
while(!self->version_messages.empty() && self->version_messages.front().first < before) {
Version version = self->version_messages.front().first;
std::pair<int, int> &sizes = tlogData->version_sizes[version];

View File

@ -310,8 +310,10 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
TagData( Tag tag, Version popped, bool nothingPersistent, bool poppedRecently, bool unpoppedRecovered ) : tag(tag), nothingPersistent(nothingPersistent), popped(popped), poppedRecently(poppedRecently), unpoppedRecovered(unpoppedRecovered) {}
TagData(TagData&& r) BOOST_NOEXCEPT : versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent), poppedRecently(r.poppedRecently), popped(r.popped), tag(r.tag), unpoppedRecovered(r.unpoppedRecovered) {}
void operator= (TagData&& r) BOOST_NOEXCEPT {
TagData(TagData&& r) noexcept
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
poppedRecently(r.poppedRecently), popped(r.popped), tag(r.tag), unpoppedRecovered(r.unpoppedRecovered) {}
void operator=(TagData&& r) noexcept {
versionMessages = std::move(r.versionMessages);
nothingPersistent = r.nothingPersistent;
poppedRecently = r.poppedRecently;

View File

@ -375,8 +375,12 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
TagData( Tag tag, Version popped, IDiskQueue::location poppedLocation, bool nothingPersistent, bool poppedRecently, bool unpoppedRecovered ) : tag(tag), nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped), persistentPopped(0), versionForPoppedLocation(0), poppedLocation(poppedLocation), unpoppedRecovered(unpoppedRecovered) {}
TagData(TagData&& r) BOOST_NOEXCEPT : versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent), poppedRecently(r.poppedRecently), popped(r.popped), persistentPopped(r.persistentPopped), versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation), tag(r.tag), unpoppedRecovered(r.unpoppedRecovered) {}
void operator= (TagData&& r) BOOST_NOEXCEPT {
TagData(TagData&& r) noexcept
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
poppedRecently(r.poppedRecently), popped(r.popped), persistentPopped(r.persistentPopped),
versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation), tag(r.tag),
unpoppedRecovered(r.unpoppedRecovered) {}
void operator=(TagData&& r) noexcept {
versionMessages = std::move(r.versionMessages);
nothingPersistent = r.nothingPersistent;
poppedRecently = r.poppedRecently;

View File

@ -22,7 +22,10 @@
#define FDBSERVER_RESOLVERINTERFACE_H
#pragma once
#include "fdbrpc/Locality.h"
#include "fdbrpc/fdbrpc.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/CommitTransaction.h"
struct ResolverInterface {
constexpr static FileIdentifier file_identifier = 1755944;

View File

@ -83,6 +83,7 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
updateProcessStats(self);
updateProcessStatsTimer = delay(SERVER_KNOBS->FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL);
}
when(wait(actors.getResult())) {}
when(wait(exitRole)) {
TraceEvent("RestoreApplierCoreExitRole", self->id());
break;
@ -92,6 +93,7 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
TraceEvent(SevWarn, "FastRestoreApplierError", self->id())
.detail("RequestType", requestTypeStr)
.error(e, true);
actors.clear(false);
break;
}
}
@ -179,7 +181,6 @@ ACTOR static Future<Void> applyClearRangeMutations(Standalone<VectorRef<KeyRange
.detail("DelayTime", delayTime);
loop {
try {
tr->reset();
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
for (auto& range : ranges) {
@ -216,47 +217,50 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
std::map<Key, std::map<Key, StagingKey>::iterator> incompleteStagingKeys, double delayTime, Database cx,
UID applierID, int batchIndex) {
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
state std::vector<Future<Optional<Value>>> fValues;
state std::vector<Future<Optional<Value>>> fValues(incompleteStagingKeys.size(), Never());
state int retries = 0;
state UID randomID = deterministicRandom()->randomUniqueID();
wait(delay(delayTime + deterministicRandom()->random01() * delayTime));
TraceEvent("FastRestoreApplierGetAndComputeStagingKeysStart", applierID)
.detail("RandomUID", randomID)
.detail("BatchIndex", batchIndex)
.detail("GetKeys", incompleteStagingKeys.size())
.detail("DelayTime", delayTime);
loop {
try {
tr->reset();
int i = 0;
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
for (auto& key : incompleteStagingKeys) {
fValues.push_back(tr->get(key.first));
fValues[i++] = tr->get(key.first);
}
wait(waitForAll(fValues));
break;
} catch (Error& e) {
if (retries++ > 10) { // TODO: Can we stop retry at the first error?
TraceEvent(SevWarn, "FastRestoreApplierGetAndComputeStagingKeysGetKeysStuck", applierID)
if (retries++ > incompleteStagingKeys.size()) {
TraceEvent(SevWarnAlways, "GetAndComputeStagingKeys", applierID)
.suppressFor(1.0)
.detail("RandomUID", randomID)
.detail("BatchIndex", batchIndex)
.detail("GetKeys", incompleteStagingKeys.size())
.error(e);
break;
}
wait(tr->onError(e));
fValues.clear();
}
}
ASSERT(fValues.size() == incompleteStagingKeys.size());
int i = 0;
for (auto& key : incompleteStagingKeys) {
if (!fValues[i].get().present()) { // Debug info to understand which key does not exist in DB
if (!fValues[i].get().present()) { // Key not exist in DB
// if condition: fValues[i].Valid() && fValues[i].isReady() && !fValues[i].isError() &&
TraceEvent(SevWarn, "FastRestoreApplierGetAndComputeStagingKeysNoBaseValueInDB", applierID)
.detail("BatchIndex", batchIndex)
.detail("Key", key.first)
.detail("Reason", "Not found in DB")
.detail("IsReady", fValues[i].isReady())
.detail("PendingMutations", key.second->second.pendingMutations.size())
.detail("StagingKeyType", (int)key.second->second.type);
.detail("StagingKeyType", getTypeString(key.second->second.type));
for (auto& vm : key.second->second.pendingMutations) {
TraceEvent(SevWarn, "FastRestoreApplierGetAndComputeStagingKeysNoBaseValueInDB")
.detail("PendingMutationVersion", vm.first.toString())
@ -274,8 +278,10 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
}
TraceEvent("FastRestoreApplierGetAndComputeStagingKeysDone", applierID)
.detail("RandomUID", randomID)
.detail("BatchIndex", batchIndex)
.detail("GetKeys", incompleteStagingKeys.size());
.detail("GetKeys", incompleteStagingKeys.size())
.detail("DelayTime", delayTime);
return Void();
}
@ -404,7 +410,6 @@ ACTOR static Future<Void> applyStagingKeysBatch(std::map<Key, StagingKey>::itera
TraceEvent("FastRestoreApplierPhaseApplyStagingKeysBatch", applierID).detail("Begin", begin->first);
loop {
try {
tr->reset();
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
std::map<Key, StagingKey>::iterator iter = begin;
@ -502,6 +507,7 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
.detail("FinishedBatch", self->finishedBatch.get());
// Ensure batch (i-1) is applied before batch i
// TODO: Add a counter to warn when too many requests are waiting on the actor
wait(self->finishedBatch.whenAtLeast(req.batchIndex - 1));
state bool isDuplicated = true;
@ -523,6 +529,8 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
}
ASSERT(batchData->dbApplier.present());
ASSERT(!batchData->dbApplier.get().isError()); // writeMutationsToDB actor cannot have error.
// We cannot blindly retry because it is not idempodent
wait(batchData->dbApplier.get());
@ -578,4 +586,4 @@ Value applyAtomicOp(Optional<StringRef> existingValue, Value value, MutationRef:
ASSERT(false);
}
return Value();
}
}

View File

@ -123,7 +123,8 @@ struct StagingKey {
.detail("Value", val)
.detail("MType", type < MutationRef::MAX_ATOMIC_OP ? getTypeString(type) : "[Unset]")
.detail("LargestPendingVersion",
(pendingMutations.empty() ? "[none]" : pendingMutations.rbegin()->first.toString()));
(pendingMutations.empty() ? "[none]" : pendingMutations.rbegin()->first.toString()))
.detail("PendingMutations", pendingMutations.size());
std::map<LogMessageVersion, Standalone<MutationRef>>::iterator lb = pendingMutations.lower_bound(version);
if (lb == pendingMutations.end()) {
return;

View File

@ -122,7 +122,7 @@ Future<Void> RestoreConfigFR::logError(Database cx, Error e, std::string const&
}
TraceEvent t(SevWarn, "FileRestoreError");
t.error(e).detail("RestoreUID", uid).detail("Description", details).detail("TaskInstance", (uint64_t)taskInstance);
// These should not happen
// key_not_found could happen
if (e.code() == error_code_key_not_found) t.backtrace();
return updateErrorInfo(cx, e, details);

View File

@ -111,13 +111,17 @@ ACTOR Future<Void> restoreLoaderCore(RestoreLoaderInterface loaderInterf, int no
updateProcessStats(self);
updateProcessStatsTimer = delay(SERVER_KNOBS->FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL);
}
when(wait(actors.getResult())) {}
when(wait(exitRole)) {
TraceEvent("FastRestoreLoaderCoreExitRole", self->id());
break;
}
}
} catch (Error& e) {
TraceEvent(SevWarn, "FastRestoreLoader", self->id()).detail("RequestType", requestTypeStr).error(e, true);
TraceEvent(SevWarn, "FastRestoreLoaderError", self->id())
.detail("RequestType", requestTypeStr)
.error(e, true);
actors.clear(false);
break;
}
}

View File

@ -198,17 +198,25 @@ ACTOR Future<Void> startProcessRestoreRequests(Reference<RestoreMasterData> self
TraceEvent("FastRestoreMasterWaitOnRestoreRequests", self->id()).detail("RestoreRequests", restoreRequests.size());
// DB has been locked where restore request is submitted
wait(clearDB(cx));
// TODO: Sanity check restoreRequests' key ranges do not overlap
// Step: Perform the restore requests
try {
for (restoreIndex = 0; restoreIndex < restoreRequests.size(); restoreIndex++) {
RestoreRequest& request = restoreRequests[restoreIndex];
state RestoreRequest request = restoreRequests[restoreIndex];
TraceEvent("FastRestoreMasterProcessRestoreRequests", self->id())
.detail("RestoreRequestInfo", request.toString());
// TODO: Initialize MasterData and all loaders and appliers' data for each restore request!
self->resetPerRestoreRequest();
// clear the key range that will be restored
wait(runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) -> Future<Void> {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->clear(request.range);
return Void();
}));
wait(success(processRestoreRequest(self, cx, request)));
wait(notifyRestoreCompleted(self, false));
}
@ -637,7 +645,6 @@ ACTOR static Future<Standalone<VectorRef<RestoreRequest>>> collectRestoreRequest
loop {
try {
TraceEvent("FastRestoreMasterPhaseCollectRestoreRequestsWait");
tr.reset();
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
@ -866,6 +873,7 @@ ACTOR static Future<Void> notifyApplierToApplyMutations(Reference<MasterBatchDat
}
ASSERT(batchData->applyToDB.present());
ASSERT(!batchData->applyToDB.get().isError());
wait(batchData->applyToDB.get());
// Sanity check all appliers have applied data to destination DB
@ -943,7 +951,7 @@ ACTOR static Future<Void> notifyRestoreCompleted(Reference<RestoreMasterData> se
ACTOR static Future<Void> signalRestoreCompleted(Reference<RestoreMasterData> self, Database cx) {
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
wait(notifyRestoreCompleted(self, true));
wait(notifyRestoreCompleted(self, true)); // notify workers the restore has completed
wait(delay(5.0)); // Give some time for loaders and appliers to exit

View File

@ -249,7 +249,6 @@ ACTOR Future<Void> startRestoreWorker(Reference<RestoreWorkerData> self, Restore
}
ACTOR static Future<Void> waitOnRestoreRequests(Database cx, UID nodeID = UID()) {
state Future<Void> watch4RestoreRequest;
state ReadYourWritesTransaction tr(cx);
state Optional<Value> numRequests;
@ -263,10 +262,10 @@ ACTOR static Future<Void> waitOnRestoreRequests(Database cx, UID nodeID = UID())
Optional<Value> _numRequests = wait(tr.get(restoreRequestTriggerKey));
numRequests = _numRequests;
if (!numRequests.present()) {
watch4RestoreRequest = tr.watch(restoreRequestTriggerKey);
state Future<Void> watchForRestoreRequest = tr.watch(restoreRequestTriggerKey);
wait(tr.commit());
TraceEvent(SevInfo, "FastRestoreWaitOnRestoreRequestTriggerKey", nodeID);
wait(watch4RestoreRequest);
wait(watchForRestoreRequest);
TraceEvent(SevInfo, "FastRestoreDetectRestoreRequestTriggerKeyChanged", nodeID);
} else {
TraceEvent(SevInfo, "FastRestoreRestoreRequestTriggerKey", nodeID)

View File

@ -55,7 +55,6 @@ struct ServerDBInfo {
LogSystemConfig logSystemConfig;
std::vector<UID> priorCommittedLogServers; // If !fullyRecovered and logSystemConfig refers to a new log system which may not have been committed to the coordinated state yet, then priorCommittedLogServers are the previous, fully committed generation which need to stay alive in case this recovery fails
Optional<LatencyBandConfig> latencyBandConfig;
std::vector<std::pair<uint16_t,StorageServerInterface>> storageCaches;
int64_t infoGeneration;
ServerDBInfo() : recoveryCount(0), recoveryState(RecoveryState::UNINITIALIZED), logSystemConfig(0), infoGeneration(0) {}
@ -65,7 +64,7 @@ struct ServerDBInfo {
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, id, clusterInterface, client, distributor, master, ratekeeper, resolvers, recoveryCount, recoveryState, masterLifetime, logSystemConfig, priorCommittedLogServers, latencyBandConfig, storageCaches, infoGeneration);
serializer(ar, id, clusterInterface, client, distributor, master, ratekeeper, resolvers, recoveryCount, recoveryState, masterLifetime, logSystemConfig, priorCommittedLogServers, latencyBandConfig, infoGeneration);
}
};

View File

@ -1220,9 +1220,9 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
//FIXME: temporarily code to test storage cache
//TODO: caching disabled for this merge
//if(dc==0) {
// machines++;
//}
if(dc==0) {
machines++;
}
int useSeedForMachine = deterministicRandom()->randomInt(0, machines);
Standalone<StringRef> zoneId;
@ -1249,10 +1249,10 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
//FIXME: temporarily code to test storage cache
//TODO: caching disabled for this merge
//if(machine==machines-1 && dc==0) {
// processClass = ProcessClass(ProcessClass::StorageCacheClass, ProcessClass::CommandLineSource);
// nonVersatileMachines++;
//}
if(machine==machines-1 && dc==0) {
processClass = ProcessClass(ProcessClass::StorageCacheClass, ProcessClass::CommandLineSource);
nonVersatileMachines++;
}
std::vector<IPAddress> ips;
for (int i = 0; i < processesPerMachine; i++) {

View File

@ -393,8 +393,8 @@ public:
}
}
~SkipList() { destroy(); }
SkipList(SkipList&& other) BOOST_NOEXCEPT : header(other.header) { other.header = NULL; }
void operator=(SkipList&& other) BOOST_NOEXCEPT {
SkipList(SkipList&& other) noexcept : header(other.header) { other.header = NULL; }
void operator=(SkipList&& other) noexcept {
destroy();
header = other.header;
other.header = NULL;

File diff suppressed because it is too large Load Diff

View File

@ -28,6 +28,7 @@
#include "fdbclient/SystemData.h"
#include "fdbclient/FDBTypes.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "fdbserver/LogProtocolMessage.h"
#include "fdbserver/TLogInterface.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/IKeyValueStore.h"
@ -373,8 +374,12 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
TagData( Tag tag, Version popped, IDiskQueue::location poppedLocation, bool nothingPersistent, bool poppedRecently, bool unpoppedRecovered ) : tag(tag), nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped), persistentPopped(0), versionForPoppedLocation(0), poppedLocation(poppedLocation), unpoppedRecovered(unpoppedRecovered) {}
TagData(TagData&& r) BOOST_NOEXCEPT : versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent), poppedRecently(r.poppedRecently), popped(r.popped), persistentPopped(r.persistentPopped), versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation), tag(r.tag), unpoppedRecovered(r.unpoppedRecovered) {}
void operator= (TagData&& r) BOOST_NOEXCEPT {
TagData(TagData&& r) noexcept
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
poppedRecently(r.poppedRecently), popped(r.popped), persistentPopped(r.persistentPopped),
versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation), tag(r.tag),
unpoppedRecovered(r.unpoppedRecovered) {}
void operator=(TagData&& r) noexcept {
versionMessages = std::move(r.versionMessages);
nothingPersistent = r.nothingPersistent;
poppedRecently = r.poppedRecently;
@ -1677,7 +1682,10 @@ ACTOR Future<Void> tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere
reply.end = endVersion;
reply.onlySpilled = onlySpilled;
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress());
//TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()).
// detail("BeginVer", req.begin).detail("EndVer", reply.end).
// detail("MsgBytes", reply.messages.expectedSize()).
// detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress());
if(req.sequence.present()) {
auto& trackerData = logData->peekTracker[peekId];

View File

@ -28,6 +28,7 @@
#include "fdbrpc/Replication.h"
#include "fdbrpc/ReplicationUtils.h"
#include "fdbserver/RecoveryState.h"
#include "fdbserver/LogProtocolMessage.h"
#include "flow/actorcompiler.h" // This must be the last #include.
ACTOR Future<Version> minVersionWhenReady(Future<Void> f, std::vector<Future<TLogCommitReply>> replies) {
@ -690,10 +691,10 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
Reference<IPeekCursor> peekLocal( UID dbgid, Tag tag, Version begin, Version end, bool useMergePeekCursors, int8_t peekLocality = tagLocalityInvalid ) {
if(tag.locality >= 0 || tag.locality == tagLocalityUpgraded) {
if(tag.locality >= 0 || tag.locality == tagLocalityUpgraded || tag.locality == tagLocalitySpecial) {
peekLocality = tag.locality;
}
ASSERT(peekLocality >= 0 || peekLocality == tagLocalityUpgraded);
ASSERT(peekLocality >= 0 || peekLocality == tagLocalityUpgraded || tag.locality == tagLocalitySpecial);
int bestSet = -1;
bool foundSpecial = false;
@ -702,7 +703,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
if(tLogs[t]->logServers.size() && tLogs[t]->locality != tagLocalitySatellite) {
logCount++;
}
if(tLogs[t]->logServers.size() && (tLogs[t]->locality == tagLocalitySpecial || tLogs[t]->locality == tagLocalityUpgraded || tLogs[t]->locality == peekLocality || peekLocality == tagLocalityUpgraded)) {
if(tLogs[t]->logServers.size() && (tLogs[t]->locality == tagLocalitySpecial || tLogs[t]->locality == tagLocalityUpgraded || tLogs[t]->locality == peekLocality || peekLocality == tagLocalityUpgraded || peekLocality == tagLocalitySpecial)) {
if( tLogs[t]->locality == tagLocalitySpecial || tLogs[t]->locality == tagLocalityUpgraded ) {
foundSpecial = true;
}
@ -757,7 +758,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
if(oldLogData[i].tLogs[t]->logServers.size() && oldLogData[i].tLogs[t]->locality != tagLocalitySatellite) {
logCount++;
}
if(oldLogData[i].tLogs[t]->logServers.size() && (oldLogData[i].tLogs[t]->locality == tagLocalitySpecial || oldLogData[i].tLogs[t]->locality == tagLocalityUpgraded || oldLogData[i].tLogs[t]->locality == peekLocality || peekLocality == tagLocalityUpgraded)) {
if(oldLogData[i].tLogs[t]->logServers.size() && (oldLogData[i].tLogs[t]->locality == tagLocalitySpecial || oldLogData[i].tLogs[t]->locality == tagLocalityUpgraded || oldLogData[i].tLogs[t]->locality == peekLocality ||
peekLocality == tagLocalityUpgraded || peekLocality == tagLocalitySpecial)) {
if( oldLogData[i].tLogs[t]->locality == tagLocalitySpecial || oldLogData[i].tLogs[t]->locality == tagLocalityUpgraded ) {
nextFoundSpecial = true;
}
@ -783,8 +785,9 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
Version thisBegin = std::max(oldLogData[i].tLogs[bestOldSet]->startVersion, begin);
if(thisBegin < lastBegin) {
if(thisBegin < end) {
TraceEvent("TLogPeekLocalAddingOldBest", dbgid).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end)
TraceEvent("TLogPeekLocalAddingOldBest", dbgid).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).detail("BestOldSet", bestOldSet)
.detail("LogServers", oldLogData[i].tLogs[bestOldSet]->logServerString()).detail("ThisBegin", thisBegin).detail("LastBegin", lastBegin);
//detail("LogId", oldLogData[i].tLogs[bestOldSet]->logServers[tLogs[bestOldSet]->bestLocationFor( tag )]->get().id());
cursors.emplace_back(new ILogSystem::MergedPeekCursor( oldLogData[i].tLogs[bestOldSet]->logServers, oldLogData[i].tLogs[bestOldSet]->bestLocationFor( tag ), oldLogData[i].tLogs[bestOldSet]->logServers.size() + 1 - oldLogData[i].tLogs[bestOldSet]->tLogReplicationFactor, tag,
thisBegin, std::min(lastBegin, end), useMergePeekCursors, oldLogData[i].tLogs[bestOldSet]->tLogLocalities, oldLogData[i].tLogs[bestOldSet]->tLogPolicy, oldLogData[i].tLogs[bestOldSet]->tLogReplicationFactor));
epochEnds.emplace_back(std::min(lastBegin, end));

View File

@ -531,7 +531,7 @@ static int asyncOpen(
if (flags & SQLITE_OPEN_WAL) oflags |= IAsyncFile::OPEN_LARGE_PAGES;
oflags |= IAsyncFile::OPEN_LOCK;
memset(p, 0, sizeof(VFSAsyncFile));
memset(static_cast<void*>(p), 0, sizeof(VFSAsyncFile));
new (p) VFSAsyncFile(zName, flags);
try {
// Note that SQLiteDB::open also opens the db file, so its flags and modes are important, too

View File

@ -2204,6 +2204,7 @@ struct SplitStringRef {
// A BTree "page id" is actually a list of LogicalPageID's whose contents should be concatenated together.
// NOTE: Uses host byte order
typedef VectorRef<LogicalPageID> BTreePageIDRef;
constexpr LogicalPageID maxPageID = (LogicalPageID)-1;
std::string toString(BTreePageIDRef id) {
return std::string("BTreePageID") + toString(id.begin(), id.end());
@ -2246,6 +2247,10 @@ struct RedwoodRecordRef {
inline RedwoodRecordRef withoutValue() const { return RedwoodRecordRef(key, version); }
inline RedwoodRecordRef withMaxPageID() const {
return RedwoodRecordRef(key, version, StringRef((uint8_t *)&maxPageID, sizeof(maxPageID)));
}
// Truncate (key, version, part) tuple to len bytes.
void truncate(int len) {
ASSERT(len <= key.size());
@ -2988,7 +2993,8 @@ public:
VersionedBTree(IPager2* pager, std::string name)
: m_pager(pager), m_writeVersion(invalidVersion), m_lastCommittedVersion(invalidVersion), m_pBuffer(nullptr),
m_name(name) {
m_commitReadLock(SERVER_KNOBS->REDWOOD_COMMIT_CONCURRENT_READS), m_name(name) {
m_lazyClearActor = 0;
m_init = init_impl(this);
m_latestCommit = m_init;
@ -3435,6 +3441,7 @@ private:
Version m_writeVersion;
Version m_lastCommittedVersion;
Version m_newOldestVersion;
FlowLock m_commitReadLock;
Future<Void> m_latestCommit;
Future<Void> m_init;
std::string m_name;
@ -3872,7 +3879,7 @@ private:
// If the decode upper boundary is the subtree upper boundary the pointers will be the same
// For the lower boundary, if the pointers are not the same there is still a possibility
// that the keys are the same. This happens for the first remaining subtree of an internal page
// after the previous first subtree was cleared.
// after the prior subtree(s) were cleared.
return (decodeUpperBound == subtreeUpperBound) &&
(decodeLowerBound == subtreeLowerBound || decodeLowerBound->sameExceptValue(*subtreeLowerBound));
}
@ -4126,8 +4133,13 @@ private:
}
state Version writeVersion = self->getLastCommittedVersion() + 1;
wait(self->m_commitReadLock.take());
state FlowLock::Releaser readLock(self->m_commitReadLock);
state Reference<const IPage> page =
wait(readPage(snapshot, rootID, update->decodeLowerBound, update->decodeUpperBound));
readLock.release();
state BTreePage* btPage = (BTreePage*)page->begin();
ASSERT(isLeaf == btPage->isLeaf());
g_redwoodMetrics.level(btPage->height).pageCommitStart += 1;
@ -4984,6 +4996,246 @@ public:
Future<bool> moveLast() { return move_end(this, false); }
};
// Cursor designed for short lifespans.
// Holds references to all pages touched.
// All record references returned from it are valid until the cursor is destroyed.
class BTreeCursor {
Arena arena;
Reference<IPagerSnapshot> pager;
std::unordered_map<LogicalPageID, Reference<const IPage>> pages;
VersionedBTree* btree;
bool valid;
struct PathEntry {
BTreePage* btPage;
BTreePage::BinaryTree::Cursor cursor;
};
VectorRef<PathEntry> path;
public:
BTreeCursor() {}
bool isValid() const { return valid; }
std::string toString() const {
std::string r;
for (int i = 0; i < path.size(); ++i) {
r += format("[%d/%d: %s] ", i + 1, path.size(),
path[i].cursor.valid() ? path[i].cursor.get().toString(path[i].btPage->isLeaf()).c_str()
: "<invalid>");
}
if (!valid) {
r += " (invalid) ";
}
return r;
}
const RedwoodRecordRef& get() { return path.back().cursor.get(); }
bool inRoot() const { return path.size() == 1; }
// Pop and return the page cursor at the end of the path.
// This is meant to enable range scans to consume the contents of a leaf page more efficiently.
// Can only be used when inRoot() is true.
BTreePage::BinaryTree::Cursor popPath() {
BTreePage::BinaryTree::Cursor c = path.back().cursor;
path.pop_back();
return c;
}
Future<Void> pushPage(BTreePageIDRef id, const RedwoodRecordRef& lowerBound,
const RedwoodRecordRef& upperBound) {
Reference<const IPage>& page = pages[id.front()];
if (page.isValid()) {
path.push_back(arena, { (BTreePage*)page->begin(), getCursor(page) });
return Void();
}
return map(readPage(pager, id, &lowerBound, &upperBound), [this, &page, id](Reference<const IPage> p) {
page = p;
path.push_back(arena, { (BTreePage*)p->begin(), getCursor(p) });
return Void();
});
}
Future<Void> pushPage(BTreePage::BinaryTree::Cursor c) {
const RedwoodRecordRef& rec = c.get();
auto next = c;
next.moveNext();
BTreePageIDRef id = rec.getChildPage();
return pushPage(id, rec, next.getOrUpperBound());
}
Future<Void> init(VersionedBTree* btree_in, Reference<IPagerSnapshot> pager_in, BTreePageIDRef root) {
btree = btree_in;
pager = pager_in;
path.reserve(arena, 6);
valid = false;
return pushPage(root, dbBegin, dbEnd);
}
// Seeks cursor to query if it exists, the record before or after it, or an undefined and invalid
// position between those records
// If 0 is returned, then
// If the cursor is valid then it points to query
// If the cursor is not valid then the cursor points to some place in the btree such that
// If there is a record in the tree < query then movePrev() will move to it, and
// If there is a record in the tree > query then moveNext() will move to it.
// If non-zero is returned then the cursor is valid and the return value is logically equivalent
// to query.compare(cursor.get())
ACTOR Future<int> seek_impl(BTreeCursor* self, RedwoodRecordRef query, int prefetchBytes) {
state RedwoodRecordRef internalPageQuery = query.withMaxPageID();
self->path = self->path.slice(0, 1);
debug_printf("seek(%s, %d) start cursor = %s\n", query.toString().c_str(), prefetchBytes,
self->toString().c_str());
loop {
auto& entry = self->path.back();
if (entry.btPage->isLeaf()) {
int cmp = entry.cursor.seek(query);
self->valid = entry.cursor.valid() && !entry.cursor.node->isDeleted();
debug_printf("seek(%s, %d) loop exit cmp=%d cursor=%s\n", query.toString().c_str(), prefetchBytes,
cmp, self->toString().c_str());
return self->valid ? cmp : 0;
}
// Internal page, so seek to the branch where query must be
// Currently, after a subtree deletion internal page boundaries are still strictly adhered
// to and will be updated if anything is inserted into the cleared range, so if the seek fails
// or it finds an entry with a null child page then query does not exist in the BTree.
if (entry.cursor.seekLessThan(internalPageQuery) && entry.cursor.get().value.present()) {
debug_printf("seek(%s, %d) loop seek success cursor=%s\n", query.toString().c_str(), prefetchBytes,
self->toString().c_str());
Future<Void> f = self->pushPage(entry.cursor);
// Prefetch siblings, at least prefetchBytes, at level 2 but without jumping to another level 2
// sibling
if (prefetchBytes != 0 && entry.btPage->height == 2) {
auto c = entry.cursor;
bool fwd = prefetchBytes > 0;
prefetchBytes = abs(prefetchBytes);
// While we should still preload more bytes and a move in the target direction is successful
while (prefetchBytes > 0 && (fwd ? c.moveNext() : c.movePrev())) {
// If there is a page link, preload it.
if (c.get().value.present()) {
BTreePageIDRef childPage = c.get().getChildPage();
preLoadPage(self->pager.getPtr(), childPage);
prefetchBytes -= self->btree->m_blockSize * childPage.size();
}
}
}
wait(f);
} else {
self->valid = false;
debug_printf("seek(%s, %d) loop exit cmp=0 cursor=%s\n", query.toString().c_str(), prefetchBytes,
self->toString().c_str());
return 0;
}
}
}
Future<int> seek(RedwoodRecordRef query, int prefetchBytes) { return seek_impl(this, query, prefetchBytes); }
ACTOR Future<Void> seekGTE_impl(BTreeCursor* self, RedwoodRecordRef query, int prefetchBytes) {
debug_printf("seekGTE(%s, %d) start\n", query.toString().c_str(), prefetchBytes);
int cmp = wait(self->seek(query, prefetchBytes));
if (cmp > 0 || (cmp == 0 && !self->isValid())) {
wait(self->moveNext());
}
return Void();
}
Future<Void> seekGTE(RedwoodRecordRef query, int prefetchBytes) {
return seekGTE_impl(this, query, prefetchBytes);
}
ACTOR Future<Void> seekLT_impl(BTreeCursor* self, RedwoodRecordRef query, int prefetchBytes) {
debug_printf("seekLT(%s, %d) start\n", query.toString().c_str(), prefetchBytes);
int cmp = wait(self->seek(query, prefetchBytes));
if (cmp <= 0) {
wait(self->movePrev());
}
return Void();
}
Future<Void> seekLT(RedwoodRecordRef query, int prefetchBytes) {
return seekLT_impl(this, query, -prefetchBytes);
}
ACTOR Future<Void> move_impl(BTreeCursor* self, bool forward) {
// Try to the move cursor at the end of the path in the correct direction
debug_printf("move%s() start cursor=%s\n", forward ? "Next" : "Prev", self->toString().c_str());
while (1) {
debug_printf("move%s() first loop cursor=%s\n", forward ? "Next" : "Prev", self->toString().c_str());
auto& entry = self->path.back();
bool success;
if(entry.cursor.valid()) {
success = forward ? entry.cursor.moveNext() : entry.cursor.movePrev();
} else {
success = forward ? entry.cursor.moveFirst() : false;
}
// Skip over internal page entries that do not link to child pages. There should never be two in a row.
if (success && !entry.btPage->isLeaf() && !entry.cursor.get().value.present()) {
success = forward ? entry.cursor.moveNext() : entry.cursor.movePrev();
ASSERT(!success || entry.cursor.get().value.present());
}
// Stop if successful
if (success) {
break;
}
if (self->path.size() == 1) {
self->valid = false;
return Void();
}
// Move to parent
self->path = self->path.slice(0, self->path.size() - 1);
}
// While not on a leaf page, move down to get to one.
while (1) {
debug_printf("move%s() second loop cursor=%s\n", forward ? "Next" : "Prev", self->toString().c_str());
auto& entry = self->path.back();
if (entry.btPage->isLeaf()) {
break;
}
// The last entry in an internal page could be a null link, if so move back
if (!forward && !entry.cursor.get().value.present()) {
ASSERT(entry.cursor.movePrev());
ASSERT(entry.cursor.get().value.present());
}
wait(self->pushPage(entry.cursor));
auto& newEntry = self->path.back();
ASSERT(forward ? newEntry.cursor.moveFirst() : newEntry.cursor.moveLast());
}
self->valid = true;
debug_printf("move%s() exit cursor=%s\n", forward ? "Next" : "Prev", self->toString().c_str());
return Void();
}
Future<Void> moveNext() { return move_impl(this, true); }
Future<Void> movePrev() { return move_impl(this, false); }
};
Future<Void> initBTreeCursor(BTreeCursor* cursor, Version snapshotVersion) {
// Only committed versions can be read.
ASSERT(snapshotVersion <= m_lastCommittedVersion);
Reference<IPagerSnapshot> snapshot = m_pager->getReadSnapshot(snapshotVersion);
// This is a ref because snapshot will continue to hold the metakey value memory
KeyRef m = snapshot->getMetaKey();
return cursor->init(this, snapshot, ((MetaKey*)m.begin())->root.get());
}
// Cursor is for reading and interating over user visible KV pairs at a specific version
// KeyValueRefs returned become invalid once the cursor is moved
class Cursor : public IStoreCursor, public ReferenceCounted<Cursor>, public FastAllocated<Cursor>, NonCopyable {
@ -5264,10 +5516,13 @@ public:
ACTOR static Future<Standalone<RangeResultRef>> readRange_impl(KeyValueStoreRedwoodUnversioned* self, KeyRange keys,
int rowLimit, int byteLimit) {
state VersionedBTree::BTreeCursor cur;
wait(self->m_tree->initBTreeCursor(&cur, self->m_tree->getLastCommittedVersion()));
wait(self->m_concurrentReads.take());
state FlowLock::Releaser releaser(self->m_concurrentReads);
++g_redwoodMetrics.opGetRange;
state Standalone<RangeResultRef> result;
state int accumulatedBytes = 0;
ASSERT(byteLimit > 0);
@ -5276,33 +5531,58 @@ public:
return result;
}
state Reference<IStoreCursor> cur = self->m_tree->readAtVersion(self->m_tree->getLastCommittedVersion());
// Prefetch is currently only done in the forward direction
state int prefetchBytes = rowLimit > 1 ? byteLimit : 0;
// Prefetch is disabled for now pending some decent logic for deciding how much to fetch
state int prefetchBytes = 0;
if (rowLimit > 0) {
wait(cur->findFirstEqualOrGreater(keys.begin, prefetchBytes));
while (cur->isValid() && cur->getKey() < keys.end) {
KeyValueRef kv(KeyRef(result.arena(), cur->getKey()), ValueRef(result.arena(), cur->getValue()));
accumulatedBytes += kv.expectedSize();
result.push_back(result.arena(), kv);
if (--rowLimit == 0 || accumulatedBytes >= byteLimit) {
wait(cur.seekGTE(keys.begin, prefetchBytes));
while (cur.isValid()) {
// Read page contents without using waits
bool isRoot = cur.inRoot();
BTreePage::BinaryTree::Cursor leafCursor = cur.popPath();
while(leafCursor.valid()) {
KeyValueRef kv = leafCursor.get().toKeyValueRef();
if(kv.key >= keys.end) {
break;
}
accumulatedBytes += kv.expectedSize();
result.push_back_deep(result.arena(), kv);
if (--rowLimit == 0 || accumulatedBytes >= byteLimit) {
break;
}
leafCursor.moveNext();
}
// Stop if the leaf cursor is still valid which means we hit a key or size limit or
// if we started in the root page
if(leafCursor.valid() || isRoot) {
break;
}
wait(cur->next());
wait(cur.moveNext());
}
} else {
wait(cur->findLastLessOrEqual(keys.end));
if (cur->isValid() && cur->getKey() == keys.end) wait(cur->prev());
while (cur->isValid() && cur->getKey() >= keys.begin) {
KeyValueRef kv(KeyRef(result.arena(), cur->getKey()), ValueRef(result.arena(), cur->getValue()));
accumulatedBytes += kv.expectedSize();
result.push_back(result.arena(), kv);
if (++rowLimit == 0 || accumulatedBytes >= byteLimit) {
wait(cur.seekLT(keys.end, prefetchBytes));
while (cur.isValid()) {
// Read page contents without using waits
bool isRoot = cur.inRoot();
BTreePage::BinaryTree::Cursor leafCursor = cur.popPath();
while(leafCursor.valid()) {
KeyValueRef kv = leafCursor.get().toKeyValueRef();
if(kv.key < keys.begin) {
break;
}
accumulatedBytes += kv.expectedSize();
result.push_back_deep(result.arena(), kv);
if (++rowLimit == 0 || accumulatedBytes >= byteLimit) {
break;
}
leafCursor.movePrev();
}
// Stop if the leaf cursor is still valid which means we hit a key or size limit or
// if we started in the root page
if(leafCursor.valid() || isRoot) {
break;
}
wait(cur->prev());
wait(cur.movePrev());
}
}
@ -5316,15 +5596,16 @@ public:
ACTOR static Future<Optional<Value>> readValue_impl(KeyValueStoreRedwoodUnversioned* self, Key key,
Optional<UID> debugID) {
state VersionedBTree::BTreeCursor cur;
wait(self->m_tree->initBTreeCursor(&cur, self->m_tree->getLastCommittedVersion()));
wait(self->m_concurrentReads.take());
state FlowLock::Releaser releaser(self->m_concurrentReads);
++g_redwoodMetrics.opGet;
state Reference<IStoreCursor> cur = self->m_tree->readAtVersion(self->m_tree->getLastCommittedVersion());
wait(cur->findEqual(key));
if (cur->isValid()) {
return cur->getValue();
wait(cur.seekGTE(key, 0));
if (cur.isValid() && cur.get().key == key) {
return cur.get().value.get();
}
return Optional<Value>();
}
@ -5335,18 +5616,20 @@ public:
ACTOR static Future<Optional<Value>> readValuePrefix_impl(KeyValueStoreRedwoodUnversioned* self, Key key,
int maxLength, Optional<UID> debugID) {
state VersionedBTree::BTreeCursor cur;
wait(self->m_tree->initBTreeCursor(&cur, self->m_tree->getLastCommittedVersion()));
wait(self->m_concurrentReads.take());
state FlowLock::Releaser releaser(self->m_concurrentReads);
++g_redwoodMetrics.opGet;
state Reference<IStoreCursor> cur = self->m_tree->readAtVersion(self->m_tree->getLastCommittedVersion());
wait(cur->findEqual(key));
if (cur->isValid()) {
Value v = cur->getValue();
wait(cur.seekGTE(key, 0));
if (cur.isValid() && cur.get().key == key) {
Value v = cur.get().value.get();
int len = std::min(v.size(), maxLength);
return Value(cur->getValue().substr(0, len));
return Value(v.substr(0, len));
}
return Optional<Value>();
}
@ -5411,6 +5694,157 @@ KeyValue randomKV(int maxKeySize = 10, int maxValueSize = 5) {
return kv;
}
// Verify a range using a BTreeCursor.
// Assumes that the BTree holds a single data version and the version is 0.
ACTOR Future<int> verifyRangeBTreeCursor(VersionedBTree* btree, Key start, Key end, Version v,
std::map<std::pair<std::string, Version>, Optional<std::string>>* written,
int* pErrorCount) {
state int errors = 0;
if (end <= start) end = keyAfter(start);
state std::map<std::pair<std::string, Version>, Optional<std::string>>::const_iterator i =
written->lower_bound(std::make_pair(start.toString(), 0));
state std::map<std::pair<std::string, Version>, Optional<std::string>>::const_iterator iEnd =
written->upper_bound(std::make_pair(end.toString(), 0));
state std::map<std::pair<std::string, Version>, Optional<std::string>>::const_iterator iLast;
state VersionedBTree::BTreeCursor cur;
wait(btree->initBTreeCursor(&cur, v));
debug_printf("VerifyRange(@%" PRId64 ", %s, %s): Start\n", v, start.printable().c_str(), end.printable().c_str());
// Randomly use the cursor for something else first.
if (deterministicRandom()->coinflip()) {
state Key randomKey = randomKV().key;
debug_printf("VerifyRange(@%" PRId64 ", %s, %s): Dummy seek to '%s'\n", v, start.printable().c_str(),
end.printable().c_str(), randomKey.toString().c_str());
wait(success(cur.seek(randomKey, 0)));
}
debug_printf("VerifyRange(@%" PRId64 ", %s, %s): Actual seek\n", v, start.printable().c_str(),
end.printable().c_str());
wait(cur.seekGTE(start, 0));
state std::vector<KeyValue> results;
while (cur.isValid() && cur.get().key < end) {
// Find the next written kv pair that would be present at this version
while (1) {
iLast = i;
if (i == iEnd) break;
++i;
if (iLast->first.second <= v && iLast->second.present() &&
(i == iEnd || i->first.first != iLast->first.first || i->first.second > v)) {
debug_printf("VerifyRange(@%" PRId64 ", %s, %s) Found key in written map: %s\n", v,
start.printable().c_str(), end.printable().c_str(), iLast->first.first.c_str());
break;
}
}
if (iLast == iEnd) {
++errors;
++*pErrorCount;
printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v,
start.printable().c_str(), end.printable().c_str(), cur.get().key.toString().c_str());
break;
}
if (cur.get().key != iLast->first.first) {
++errors;
++*pErrorCount;
printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' but expected '%s'\n", v,
start.printable().c_str(), end.printable().c_str(), cur.get().key.toString().c_str(),
iLast->first.first.c_str());
break;
}
if (cur.get().value.get() != iLast->second.get()) {
++errors;
++*pErrorCount;
printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' has tree value '%s' but expected '%s'\n", v,
start.printable().c_str(), end.printable().c_str(), cur.get().key.toString().c_str(),
cur.get().value.get().toString().c_str(), iLast->second.get().c_str());
break;
}
ASSERT(errors == 0);
results.push_back(KeyValue(KeyValueRef(cur.get().key, cur.get().value.get())));
wait(cur.moveNext());
}
// Make sure there are no further written kv pairs that would be present at this version.
while (1) {
iLast = i;
if (i == iEnd) break;
++i;
if (iLast->first.second <= v && iLast->second.present() &&
(i == iEnd || i->first.first != iLast->first.first || i->first.second > v))
break;
}
if (iLast != iEnd) {
++errors;
++*pErrorCount;
printf("VerifyRange(@%" PRId64 ", %s, %s) ERROR: Tree range ended but written has @%" PRId64 " '%s'\n", v,
start.printable().c_str(), end.printable().c_str(), iLast->first.second, iLast->first.first.c_str());
}
debug_printf("VerifyRangeReverse(@%" PRId64 ", %s, %s): start\n", v, start.printable().c_str(),
end.printable().c_str());
// Randomly use a new cursor at the same version for the reverse range read, if the version is still available for
// opening new cursors
if (v >= btree->getOldestVersion() && deterministicRandom()->coinflip()) {
cur = VersionedBTree::BTreeCursor();
wait(btree->initBTreeCursor(&cur, v));
}
// Now read the range from the tree in reverse order and compare to the saved results
wait(cur.seekLT(end, 0));
state std::vector<KeyValue>::const_reverse_iterator r = results.rbegin();
while (cur.isValid() && cur.get().key >= start) {
if (r == results.rend()) {
++errors;
++*pErrorCount;
printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' vs nothing in written map.\n", v,
start.printable().c_str(), end.printable().c_str(), cur.get().key.toString().c_str());
break;
}
if (cur.get().key != r->key) {
++errors;
++*pErrorCount;
printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree key '%s' but expected '%s'\n", v,
start.printable().c_str(), end.printable().c_str(), cur.get().key.toString().c_str(),
r->key.toString().c_str());
break;
}
if (cur.get().value.get() != r->value) {
++errors;
++*pErrorCount;
printf("VerifyRangeReverse(@%" PRId64
", %s, %s) ERROR: Tree key '%s' has tree value '%s' but expected '%s'\n",
v, start.printable().c_str(), end.printable().c_str(), cur.get().key.toString().c_str(),
cur.get().value.get().toString().c_str(), r->value.toString().c_str());
break;
}
++r;
wait(cur.movePrev());
}
if (r != results.rend()) {
++errors;
++*pErrorCount;
printf("VerifyRangeReverse(@%" PRId64 ", %s, %s) ERROR: Tree range ended but written has '%s'\n", v,
start.printable().c_str(), end.printable().c_str(), r->key.toString().c_str());
}
return errors;
}
ACTOR Future<int> verifyRange(VersionedBTree* btree, Key start, Key end, Version v,
std::map<std::pair<std::string, Version>, Optional<std::string>>* written,
int* pErrorCount) {
@ -5607,6 +6041,58 @@ ACTOR Future<int> seekAll(VersionedBTree* btree, Version v,
return errors;
}
// Verify the result of point reads for every set or cleared key at the given version
ACTOR Future<int> seekAllBTreeCursor(VersionedBTree* btree, Version v,
std::map<std::pair<std::string, Version>, Optional<std::string>>* written, int* pErrorCount) {
state std::map<std::pair<std::string, Version>, Optional<std::string>>::const_iterator i = written->cbegin();
state std::map<std::pair<std::string, Version>, Optional<std::string>>::const_iterator iEnd = written->cend();
state int errors = 0;
state VersionedBTree::BTreeCursor cur;
wait(btree->initBTreeCursor(&cur, v));
while (i != iEnd) {
state std::string key = i->first.first;
state Version ver = i->first.second;
if (ver == v) {
state Optional<std::string> val = i->second;
debug_printf("Verifying @%" PRId64 " '%s'\n", ver, key.c_str());
state Arena arena;
wait(cur.seekGTE(RedwoodRecordRef(KeyRef(arena, key), 0), 0));
bool foundKey = cur.isValid() && cur.get().key == key;
bool hasValue = foundKey && cur.get().value.present();
if (val.present()) {
bool valueMatch = hasValue && cur.get().value.get() == val.get();
if (!foundKey || !hasValue || !valueMatch) {
++errors;
++*pErrorCount;
if (!foundKey) {
printf("Verify ERROR: key_not_found: '%s' -> '%s' @%" PRId64 "\n", key.c_str(),
val.get().c_str(), ver);
}
else if (!hasValue) {
printf("Verify ERROR: value_not_found: '%s' -> '%s' @%" PRId64 "\n", key.c_str(),
val.get().c_str(), ver);
}
else if (!valueMatch) {
printf("Verify ERROR: value_incorrect: for '%s' found '%s' expected '%s' @%" PRId64 "\n",
key.c_str(), cur.get().value.get().toString().c_str(), val.get().c_str(),
ver);
}
}
} else if (foundKey && hasValue) {
++errors;
++*pErrorCount;
printf("Verify ERROR: cleared_key_found: '%s' -> '%s' @%" PRId64 "\n", key.c_str(),
cur.get().value.get().toString().c_str(), ver);
}
}
++i;
}
return errors;
}
ACTOR Future<Void> verify(VersionedBTree* btree, FutureStream<Version> vStream,
std::map<std::pair<std::string, Version>, Optional<std::string>>* written, int* pErrorCount,
bool serial) {
@ -5637,7 +6123,13 @@ ACTOR Future<Void> verify(VersionedBTree* btree, FutureStream<Version> vStream,
state Reference<IStoreCursor> cur = btree->readAtVersion(v);
debug_printf("Verifying entire key range at version %" PRId64 "\n", v);
fRangeAll = verifyRange(btree, LiteralStringRef(""), LiteralStringRef("\xff\xff"), v, written, pErrorCount);
if(deterministicRandom()->coinflip()) {
fRangeAll = verifyRange(btree, LiteralStringRef(""), LiteralStringRef("\xff\xff"), v, written,
pErrorCount);
} else {
fRangeAll = verifyRangeBTreeCursor(btree, LiteralStringRef(""), LiteralStringRef("\xff\xff"), v, written,
pErrorCount);
}
if (serial) {
wait(success(fRangeAll));
}
@ -5646,13 +6138,21 @@ ACTOR Future<Void> verify(VersionedBTree* btree, FutureStream<Version> vStream,
Key end = randomKV().key;
debug_printf("Verifying range (%s, %s) at version %" PRId64 "\n", toString(begin).c_str(),
toString(end).c_str(), v);
fRangeRandom = verifyRange(btree, begin, end, v, written, pErrorCount);
if(deterministicRandom()->coinflip()) {
fRangeRandom = verifyRange(btree, begin, end, v, written, pErrorCount);
} else {
fRangeRandom = verifyRangeBTreeCursor(btree, begin, end, v, written, pErrorCount);
}
if (serial) {
wait(success(fRangeRandom));
}
debug_printf("Verifying seeks to each changed key at version %" PRId64 "\n", v);
fSeekAll = seekAll(btree, v, written, pErrorCount);
if(deterministicRandom()->coinflip()) {
fSeekAll = seekAll(btree, v, written, pErrorCount);
} else {
fSeekAll = seekAllBTreeCursor(btree, v, written, pErrorCount);
}
if (serial) {
wait(success(fSeekAll));
}
@ -6485,11 +6985,11 @@ TEST_CASE("!/redwood/correctness/btree") {
state int maxKeySize = deterministicRandom()->randomInt(1, pageSize * 2);
state int maxValueSize = randomSize(pageSize * 25);
state int maxCommitSize = shortTest ? 1000 : randomSize(std::min<int>((maxKeySize + maxValueSize) * 20000, 10e6));
state int mutationBytesTarget = shortTest ? 100000 : randomSize(std::min<int>(maxCommitSize * 100, 100e6));
state int mutationBytesTarget = shortTest ? 100000 : randomSize(std::min<int>(maxCommitSize * 100, pageSize * 100000));
state double clearProbability = deterministicRandom()->random01() * .1;
state double clearSingleKeyProbability = deterministicRandom()->random01();
state double clearPostSetProbability = deterministicRandom()->random01() * .1;
state double coldStartProbability = pagerMemoryOnly ? 0 : deterministicRandom()->random01();
state double coldStartProbability = pagerMemoryOnly ? 0 : (deterministicRandom()->random01() * 0.3);
state double advanceOldVersionProbability = deterministicRandom()->random01();
state double maxDuration = 60;
state int64_t cacheSizeBytes =

View File

@ -163,14 +163,13 @@ struct RegisterWorkerReply {
constexpr static FileIdentifier file_identifier = 16475696;
ProcessClass processClass;
ClusterControllerPriorityInfo priorityInfo;
Optional<uint16_t> storageCache;
RegisterWorkerReply() : priorityInfo(ProcessClass::UnsetFit, false, ClusterControllerPriorityInfo::FitnessUnknown) {}
RegisterWorkerReply(ProcessClass processClass, ClusterControllerPriorityInfo priorityInfo, Optional<uint16_t> storageCache) : processClass(processClass), priorityInfo(priorityInfo), storageCache(storageCache) {}
RegisterWorkerReply(ProcessClass processClass, ClusterControllerPriorityInfo priorityInfo) : processClass(processClass), priorityInfo(priorityInfo) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, processClass, priorityInfo, storageCache);
serializer(ar, processClass, priorityInfo);
}
};
@ -302,19 +301,18 @@ struct RegisterWorkerRequest {
Generation generation;
Optional<DataDistributorInterface> distributorInterf;
Optional<RatekeeperInterface> ratekeeperInterf;
Optional<std::pair<uint16_t,StorageServerInterface>> storageCacheInterf;
Standalone<VectorRef<StringRef>> issues;
std::vector<NetworkAddress> incompatiblePeers;
ReplyPromise<RegisterWorkerReply> reply;
bool degraded;
RegisterWorkerRequest() : priorityInfo(ProcessClass::UnsetFit, false, ClusterControllerPriorityInfo::FitnessUnknown), degraded(false) {}
RegisterWorkerRequest(WorkerInterface wi, ProcessClass initialClass, ProcessClass processClass, ClusterControllerPriorityInfo priorityInfo, Generation generation, Optional<DataDistributorInterface> ddInterf, Optional<RatekeeperInterface> rkInterf, Optional<std::pair<uint16_t,StorageServerInterface>> storageCacheInterf, bool degraded) :
wi(wi), initialClass(initialClass), processClass(processClass), priorityInfo(priorityInfo), generation(generation), distributorInterf(ddInterf), ratekeeperInterf(rkInterf), storageCacheInterf(storageCacheInterf), degraded(degraded) {}
RegisterWorkerRequest(WorkerInterface wi, ProcessClass initialClass, ProcessClass processClass, ClusterControllerPriorityInfo priorityInfo, Generation generation, Optional<DataDistributorInterface> ddInterf, Optional<RatekeeperInterface> rkInterf, bool degraded) :
wi(wi), initialClass(initialClass), processClass(processClass), priorityInfo(priorityInfo), generation(generation), distributorInterf(ddInterf), ratekeeperInterf(rkInterf), degraded(degraded) {}
template <class Ar>
void serialize( Ar& ar ) {
serializer(ar, wi, initialClass, processClass, priorityInfo, generation, distributorInterf, ratekeeperInterf, storageCacheInterf, issues, incompatiblePeers, reply, degraded);
serializer(ar, wi, initialClass, processClass, priorityInfo, generation, distributorInterf, ratekeeperInterf, issues, incompatiblePeers, reply, degraded);
}
};
@ -712,7 +710,7 @@ ACTOR Future<Void> logRouter(TLogInterface interf, InitializeLogRouterRequest re
Reference<AsyncVar<ServerDBInfo>> db);
ACTOR Future<Void> dataDistributor(DataDistributorInterface ddi, Reference<AsyncVar<ServerDBInfo>> db);
ACTOR Future<Void> ratekeeper(RatekeeperInterface rki, Reference<AsyncVar<ServerDBInfo>> db);
ACTOR Future<Void> storageCache(StorageServerInterface interf, uint16_t id, Reference<AsyncVar<ServerDBInfo>> db);
ACTOR Future<Void> storageCacheServer(StorageServerInterface interf, uint16_t id, Reference<AsyncVar<ServerDBInfo>> db);
ACTOR Future<Void> backupWorker(BackupInterface bi, InitializeBackupRequest req, Reference<AsyncVar<ServerDBInfo>> db);
void registerThreadForProfiling();

View File

@ -53,8 +53,12 @@ struct ProxyVersionReplies {
std::map<uint64_t, GetCommitVersionReply> replies;
NotifiedVersion latestRequestNum;
ProxyVersionReplies(ProxyVersionReplies&& r) BOOST_NOEXCEPT : replies(std::move(r.replies)), latestRequestNum(std::move(r.latestRequestNum)) {}
void operator=(ProxyVersionReplies&& r) BOOST_NOEXCEPT { replies = std::move(r.replies); latestRequestNum = std::move(r.latestRequestNum); }
ProxyVersionReplies(ProxyVersionReplies&& r) noexcept
: replies(std::move(r.replies)), latestRequestNum(std::move(r.latestRequestNum)) {}
void operator=(ProxyVersionReplies&& r) noexcept {
replies = std::move(r.replies);
latestRequestNum = std::move(r.latestRequestNum);
}
ProxyVersionReplies() : latestRequestNum(0) {}
};
@ -157,7 +161,7 @@ private:
} else {
self->fullyRecovered.send(Void());
}
return Void();
}
};
@ -373,7 +377,7 @@ ACTOR Future<Void> newSeedServers( Reference<MasterData> self, RecruitFromConfig
dcId_tags[recruits.storageServers[idx].locality.dcId()] = Tag(nextLocality, 0);
nextLocality++;
}
Tag& tag = dcId_tags[recruits.storageServers[idx].locality.dcId()];
tag.id++;
idx++;
@ -582,7 +586,7 @@ ACTOR Future<vector<Standalone<CommitTransactionRef>>> recruitEverything( Refere
.detail("DesiredResolvers", self->configuration.getDesiredResolvers())
.detail("StoreType", self->configuration.storageServerStoreType)
.trackLatest("MasterRecoveryState");
//FIXME: we only need log routers for the same locality as the master
int maxLogRouters = self->cstate.prevDBState.logRouterTags;
for(auto& old : self->cstate.prevDBState.oldTLogData) {
@ -1505,15 +1509,6 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
tr.set(recoveryCommitRequest.arena, coordinatorsKey, self->coordinators.ccf->getConnectionString().toString());
tr.set(recoveryCommitRequest.arena, logsKey, self->logSystem->getLogsValue());
tr.set(recoveryCommitRequest.arena, primaryDatacenterKey, self->myInterface.locality.dcId().present() ? self->myInterface.locality.dcId().get() : StringRef());
//FIXME: remove this code, caching the entire normal keyspace as a test of functionality
//TODO: caching disabled for this merge
//tr.set(recoveryCommitRequest.arena, storageCacheKey(normalKeys.begin), storageCacheValue({0}));
//tr.set(recoveryCommitRequest.arena, storageCacheKey(normalKeys.end), storageCacheValue({}));
//tr.set(recoveryCommitRequest.arena, cacheKeysKey(0, normalKeys.begin), serverKeysTrue);
//tr.set(recoveryCommitRequest.arena, cacheKeysKey(0, normalKeys.end), serverKeysFalse);
//tr.set(recoveryCommitRequest.arena, cacheChangeKeyFor(0), BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned()));
//tr.set(recoveryCommitRequest.arena, cacheChangeKey, BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned()));
tr.clear(recoveryCommitRequest.arena, tLogDatacentersKeys);
for(auto& dc : self->primaryDcId) {
@ -1654,7 +1649,7 @@ ACTOR Future<Void> masterServer( MasterInterface mi, Reference<AsyncVar<ServerDB
while(!self->addActor.isEmpty()) {
self->addActor.getFuture().pop();
}
TEST(err.code() == error_code_master_tlog_failed); // Master: terminated because of a tLog failure
TEST(err.code() == error_code_master_proxy_failed); // Master: terminated because of a proxy failure
TEST(err.code() == error_code_master_resolver_failed); // Master: terminated because of a resolver failure

View File

@ -982,7 +982,12 @@ ACTOR Future<Void> getValueQ( StorageServer* data, GetValueRequest req ) {
if( req.debugID.present() )
g_traceBatch.addEvent("GetValueDebug", req.debugID.get().first(), "getValueQ.AfterRead"); //.detail("TaskID", g_network->getCurrentTask());
GetValueReply reply(v);
// Check if the desired key might be cached
auto cached = data->cachedRangeMap[req.key];
//if (cached)
// TraceEvent(SevDebug, "SSGetValueCached").detail("Key", req.key);
GetValueReply reply(v, cached);
reply.penalty = data->getPenalty();
req.reply.send(reply);
} catch (Error& e) {
@ -1190,11 +1195,14 @@ ACTOR Future<GetKeyValuesReply> readRange( StorageServer* data, Version version,
state int pos = 0;
// Check if the desired key-range intersects the cached key-ranges
// TODO Find a more efficient way to do it
// TODO Also need this check in single key/value lookup
auto cached = data->cachedRangeMap.intersectingRanges(range);
result.cached = (cached.begin() != cached.end());
// Check if the desired key-range is cached
auto containingRange = data->cachedRangeMap.rangeContaining(range.begin);
if (containingRange.value() && containingRange->range().end >= range.end) {
//TraceEvent(SevDebug, "SSReadRangeCached").detail("Size",data->cachedRangeMap.size()).detail("ContainingRangeBegin",containingRange->range().begin).detail("ContainingRangeEnd",containingRange->range().end).
// detail("Begin", range.begin).detail("End",range.end);
result.cached = true;
} else
result.cached = false;
// if (limit >= 0) we are reading forward, else backward
if (limit >= 0) {
@ -1610,8 +1618,14 @@ ACTOR Future<Void> getKeyQ( StorageServer* data, GetKeyRequest req ) {
data->counters.bytesQueried += resultSize;
++data->counters.rowsQueried;
GetKeyReply reply(updated);
// Check if the desired key might be cached
auto cached = data->cachedRangeMap[k];
//if (cached)
// TraceEvent(SevDebug, "SSGetKeyCached").detail("Key", k).detail("Begin", shard.begin.printable()).detail("End", shard.end.printable());
GetKeyReply reply(updated, cached);
reply.penalty = data->getPenalty();
req.reply.send(reply);
}
catch (Error& e) {
@ -2584,7 +2598,6 @@ public:
if ((m.type == MutationRef::SetValue) && m.param1.substr(1).startsWith(storageCachePrefix))
applyPrivateCacheData( data, m);
else {
//TraceEvent("PrivateData", data->thisServerID).detail("Mutation", m.toString()).detail("Version", ver);
applyPrivateData( data, m );
}
} else {
@ -2673,7 +2686,7 @@ private:
}
void applyPrivateCacheData( StorageServer* data, MutationRef const& m ) {
TraceEvent(SevDebug, "SSPrivateCacheMutation", data->thisServerID).detail("Mutation", m.toString());
//TraceEvent(SevDebug, "SSPrivateCacheMutation", data->thisServerID).detail("Mutation", m.toString());
if (processedCacheStartKey) {
// Because of the implementation of the krm* functions, we expect changes in pairs, [begin,end)
@ -2681,17 +2694,16 @@ private:
KeyRangeRef keys( cacheStartKey.removePrefix(systemKeys.begin).removePrefix( storageCachePrefix ),
m.param1.removePrefix(systemKeys.begin).removePrefix( storageCachePrefix ));
data->cachedRangeMap.insert(keys, true);
//TraceEvent(SevDebug, "SSPrivateCacheMutation", data->thisServerID).detail("Begin", keys.begin).detail("End", keys.end);
//fprintf(stderr, "applyPrivateCacheData : begin: %s, end: %s\n", printable(keys.begin).c_str(), printable(keys.end).c_str());
//Figure out the affected shard ranges and maintain the cached key-range information in the in-memory map
// TODO revisit- we are not splitting the cached ranges based on shards as of now.
if (0) {
auto cachedRanges = data->shards.intersectingRanges(keys);
for(auto shard = cachedRanges.begin(); shard != cachedRanges.end(); ++shard) {
KeyRangeRef intersectingRange = shard.range() & keys;
data->cachedRangeMap.insert(KeyRangeRef(intersectingRange.begin, intersectingRange.end), true);
}
auto cachedRanges = data->shards.intersectingRanges(keys);
for(auto shard = cachedRanges.begin(); shard != cachedRanges.end(); ++shard) {
KeyRangeRef intersectingRange = shard.range() & keys;
TraceEvent(SevDebug, "SSPrivateCacheMutationInsertUnexpected", data->thisServerID).detail("Begin", intersectingRange.begin).detail("End", intersectingRange.end);
data->cachedRangeMap.insert(intersectingRange, true);
}
}
processedStartKey = false;
} else if ((m.type == MutationRef::SetValue) && m.param1.substr(1).startsWith(storageCachePrefix)) {
@ -2728,7 +2740,6 @@ ACTOR Future<Void> update( StorageServer* data, bool* pReceivedUpdate )
}
state Reference<ILogSystem::IPeekCursor> cursor = data->logCursor;
//TraceEvent("SSUpdatePeeking", data->thisServerID).detail("MyVer", data->version.get()).detail("Epoch", data->updateEpoch).detail("Seq", data->updateSequence);
loop {
wait( cursor->getMore() );
@ -2775,12 +2786,14 @@ ACTOR Future<Void> update( StorageServer* data, bool* pReceivedUpdate )
if (LogProtocolMessage::isNextIn(cloneReader)) {
LogProtocolMessage lpm;
cloneReader >> lpm;
//TraceEvent(SevDebug, "SSReadingLPM", data->thisServerID).detail("Mutation", lpm.toString());
dbgLastMessageWasProtocol = true;
cloneCursor1->setProtocolVersion(cloneReader.protocolVersion());
}
else {
MutationRef msg;
cloneReader >> msg;
//TraceEvent(SevDebug, "SSReadingLog", data->thisServerID).detail("Mutation", msg.toString());
if (firstMutation && msg.param1.startsWith(systemKeys.end))
hasPrivateData = true;
@ -2844,7 +2857,6 @@ ACTOR Future<Void> update( StorageServer* data, bool* pReceivedUpdate )
state Version ver = invalidVersion;
cloneCursor2->setProtocolVersion(data->logProtocol);
//TraceEvent("SSUpdatePeeked", data->thisServerID).detail("FromEpoch", data->updateEpoch).detail("FromSeq", data->updateSequence).detail("ToEpoch", results.end_epoch).detail("ToSeq", results.end_seq).detail("MsgSize", results.messages.size());
for (;cloneCursor2->hasMessage(); cloneCursor2->nextMessage()) {
if(mutationBytes > SERVER_KNOBS->DESIRED_UPDATE_BYTES) {
mutationBytes = 0;

View File

@ -21,6 +21,8 @@
#include <tuple>
#include <boost/lexical_cast.hpp>
#include "fdbrpc/Locality.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbserver/Knobs.h"
#include "flow/ActorCollection.h"
#include "flow/SystemMonitor.h"
@ -453,7 +455,7 @@ ACTOR Future<Void> registrationClient(
state Future<Void> cacheErrorsFuture;
state Optional<double> incorrectTime;
loop {
RegisterWorkerRequest request(interf, initialClass, processClass, asyncPriorityInfo->get(), requestGeneration++, ddInterf->get(), rkInterf->get(), scInterf->get(), degraded->get());
RegisterWorkerRequest request(interf, initialClass, processClass, asyncPriorityInfo->get(), requestGeneration++, ddInterf->get(), rkInterf->get(), degraded->get());
for (auto const& i : issues->get()) {
request.issues.push_back_deep(request.issues.arena(), i);
}
@ -491,41 +493,10 @@ ACTOR Future<Void> registrationClient(
when ( RegisterWorkerReply reply = wait( registrationReply )) {
processClass = reply.processClass;
asyncPriorityInfo->set( reply.priorityInfo );
if(!reply.storageCache.present()) {
cacheProcessFuture.cancel();
scInterf->set(Optional<std::pair<uint16_t,StorageServerInterface>>());
} else if (!scInterf->get().present() || scInterf->get().get().first != reply.storageCache.get()) {
StorageServerInterface recruited;
recruited.locality = locality;
recruited.initEndpoints();
std::map<std::string, std::string> details;
startRole( Role::STORAGE_CACHE, recruited.id(), interf.id(), details );
//DUMPTOKEN(recruited.getVersion);
DUMPTOKEN(recruited.getValue);
DUMPTOKEN(recruited.getKey);
DUMPTOKEN(recruited.getKeyValues);
DUMPTOKEN(recruited.getShardState);
DUMPTOKEN(recruited.waitMetrics);
DUMPTOKEN(recruited.splitMetrics);
DUMPTOKEN(recruited.getReadHotRanges);
DUMPTOKEN(recruited.getStorageMetrics);
DUMPTOKEN(recruited.waitFailure);
DUMPTOKEN(recruited.getQueuingMetrics);
DUMPTOKEN(recruited.getKeyValueStoreType);
DUMPTOKEN(recruited.watchValue);
cacheProcessFuture = storageCache( recruited, reply.storageCache.get(), dbInfo );
cacheErrorsFuture = forwardError(errors, Role::STORAGE_CACHE, recruited.id(), setWhenDoneOrError(cacheProcessFuture, scInterf, Optional<std::pair<uint16_t,StorageServerInterface>>()));
scInterf->set(std::make_pair(reply.storageCache.get(), recruited));
}
}
when ( wait( ccInterface->onChange() )) {}
when ( wait( ddInterf->onChange() ) ) {}
when ( wait( rkInterf->onChange() ) ) {}
when ( wait( scInterf->onChange() ) ) {}
when ( wait( degraded->onChange() ) ) {}
when ( wait( FlowTransport::transport().onIncompatibleChanged() ) ) {}
when ( wait( issues->onChange() ) ) {}
@ -712,6 +683,41 @@ ACTOR Future<Void> storageServerRollbackRebooter( Future<Void> prevStorageServer
}
}
ACTOR Future<Void> storageCacheRollbackRebooter( Future<Void> prevStorageCache, UID id, LocalityData locality, Reference<AsyncVar<ServerDBInfo>> db) {
loop {
ErrorOr<Void> e = wait( errorOr( prevStorageCache) );
if (!e.isError()) {
TraceEvent("StorageCacheRequestedReboot1", id);
return Void();
}
else if (e.getError().code() != error_code_please_reboot && e.getError().code() != error_code_worker_removed) {
TraceEvent("StorageCacheRequestedReboot2", id).detail("Code",e.getError().code());
throw e.getError();
}
TraceEvent("StorageCacheRequestedReboot", id);
StorageServerInterface recruited;
recruited.uniqueID = deterministicRandom()->randomUniqueID();// id;
recruited.locality = locality;
recruited.initEndpoints();
DUMPTOKEN(recruited.getValue);
DUMPTOKEN(recruited.getKey);
DUMPTOKEN(recruited.getKeyValues);
DUMPTOKEN(recruited.getShardState);
DUMPTOKEN(recruited.waitMetrics);
DUMPTOKEN(recruited.splitMetrics);
DUMPTOKEN(recruited.getStorageMetrics);
DUMPTOKEN(recruited.waitFailure);
DUMPTOKEN(recruited.getQueuingMetrics);
DUMPTOKEN(recruited.getKeyValueStoreType);
DUMPTOKEN(recruited.watchValue);
prevStorageCache = storageCacheServer(recruited, 0, db);
}
}
// FIXME: This will not work correctly in simulation as all workers would share the same roles map
std::set<std::pair<std::string, std::string>> g_roles;
@ -1049,10 +1055,40 @@ ACTOR Future<Void> workerServer(
}
}
bool hasCache = false;
// start cache role if we have the right process class
if (initialClass.classType() == ProcessClass::StorageCacheClass) {
hasCache = true;
StorageServerInterface recruited;
recruited.locality = locality;
recruited.initEndpoints();
std::map<std::string, std::string> details;
startRole(Role::STORAGE_CACHE, recruited.id(), interf.id(), details);
// DUMPTOKEN(recruited.getVersion);
DUMPTOKEN(recruited.getValue);
DUMPTOKEN(recruited.getKey);
DUMPTOKEN(recruited.getKeyValues);
DUMPTOKEN(recruited.getShardState);
DUMPTOKEN(recruited.waitMetrics);
DUMPTOKEN(recruited.splitMetrics);
DUMPTOKEN(recruited.getStorageMetrics);
DUMPTOKEN(recruited.waitFailure);
DUMPTOKEN(recruited.getQueuingMetrics);
DUMPTOKEN(recruited.getKeyValueStoreType);
DUMPTOKEN(recruited.watchValue);
auto f = storageCacheServer(recruited, 0, dbInfo);
f = storageCacheRollbackRebooter( f, recruited.id(), recruited.locality, dbInfo);
errorForwarders.add(forwardError(errors, Role::STORAGE_CACHE, recruited.id(), f));
}
std::map<std::string, std::string> details;
details["Locality"] = locality.toString();
details["DataFolder"] = folder;
details["StoresPresent"] = format("%d", stores.size());
details["CachePresent"] = hasCache ? "true" : "false";
startRole( Role::WORKER, interf.id(), interf.id(), details );
errorForwarders.add(traceRole(Role::WORKER, interf.id()));
@ -1347,7 +1383,7 @@ ACTOR Future<Void> workerServer(
DUMPTOKEN( recruited.getQueuingMetrics );
DUMPTOKEN( recruited.confirmRunning );
errorForwarders.add( zombie(recruited, forwardError( errors, Role::LOG_ROUTER, recruited.id(),
errorForwarders.add( zombie(recruited, forwardError( errors, Role::LOG_ROUTER, recruited.id(),
logRouter( recruited, req, dbInfo ) ) ) );
req.reply.send(recruited);
}

View File

@ -507,9 +507,9 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
}
// Wait for parallel restore to finish before we can proceed
TraceEvent("FastRestore").detail("BackupAndParallelRestore", "WaitForRestoreToFinish");
TraceEvent("FastRestoreWorkload").detail("WaitForRestoreToFinish", randomID);
wait(backupAgent.parallelRestoreFinish(cx, randomID));
TraceEvent("FastRestore").detail("BackupAndParallelRestore", "RestoreFinished");
TraceEvent("FastRestoreWorkload").detail("RestoreFinished", randomID);
for (auto& restore : restores) {
ASSERT(!restore.isError());
@ -668,7 +668,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
g_simulator.backupAgents = ISimulator::NoBackupAgents;
}
} catch (Error& e) {
TraceEvent(SevError, "BackupAndRestoreCorrectness").error(e).GetLastError();
TraceEvent(SevError, "BackupAndParallelRestoreCorrectness").error(e).GetLastError();
throw;
}
return Void();

View File

@ -0,0 +1,33 @@
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
struct CacheWorkload : TestWorkload {
Key keyPrefix;
CacheWorkload(WorkloadContext const& wcx)
: TestWorkload(wcx)
{
keyPrefix = unprintable( getOption(options, LiteralStringRef("keyPrefix"), LiteralStringRef("")).toString() );
}
virtual std::string description() { return "CacheWorkload"; }
virtual Future<Void> setup( Database const& cx ) {
if (clientId == 0) {
//Call management API to cache keys under the given prefix
return addCachedRange(cx, prefixRange(keyPrefix));
}
return Void();
}
virtual Future<Void> start( Database const& cx ) {
return Void();
}
virtual Future<bool> check( Database const& cx ) {
return true;
}
virtual void getMetrics( vector<PerfMetric>& m ) {
}
};
WorkloadFactory<CacheWorkload> CacheWorkloadFactory("Cache");

View File

@ -708,6 +708,7 @@ struct ConsistencyCheckWorkload : TestWorkload
state vector<UID> storageServers = (isRelocating) ? destStorageServers : sourceStorageServers;
state vector<StorageServerInterface> storageServerInterfaces;
//TraceEvent("ConsistencyCheck_GetStorageInfo").detail("StorageServers", storageServers.size());
loop {
try {
vector< Future< Optional<Value> > > serverListEntries;
@ -720,6 +721,7 @@ struct ConsistencyCheckWorkload : TestWorkload
else if (self->performQuiescentChecks)
self->testFailure("/FF/serverList changing in a quiescent database");
}
break;
}
catch(Error &e) {
@ -917,7 +919,7 @@ struct ConsistencyCheckWorkload : TestWorkload
else if(!isRelocating)
{
TraceEvent("ConsistencyCheck_StorageServerUnavailable").suppressFor(1.0).detail("StorageServer", storageServers[j]).detail("ShardBegin", printable(range.begin)).detail("ShardEnd", printable(range.end))
.detail("Address", storageServerInterfaces[j].address()).detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token);
.detail("Address", storageServerInterfaces[j].address()).detail("UID", storageServerInterfaces[j].id()).detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token);
//All shards should be available in quiscence
if(self->performQuiescentChecks)

View File

@ -115,9 +115,9 @@ struct CycleWorkload : TestWorkload {
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("CyclicTestMX").detail("Key", self->key(r).toString()).detail("Value", self->value(r3).toString());
// TraceEvent("CyclicTestMX").detail("Key", self->key(r2).toString()).detail("Value", self->value(r4).toString());
// TraceEvent("CyclicTestMX").detail("Key", self->key(r3).toString()).detail("Value", self->value(r2).toString());
//TraceEvent("CyclicTestMX1").detail("Key", self->key(r).toString()).detail("Value", self->value(r3).toString());
//TraceEvent("CyclicTestMX2").detail("Key", self->key(r2).toString()).detail("Value", self->value(r4).toString());
//TraceEvent("CyclicTestMX3").detail("Key", self->key(r3).toString()).detail("Value", self->value(r2).toString());
wait( tr.commit() );
// TraceEvent("CycleCommit");
@ -161,7 +161,10 @@ struct CycleWorkload : TestWorkload {
return false;
}
int i=0;
for(int c=0; c<nodeCount; c++) {
int iPrev=0;
double d;
int c;
for(c=0; c<nodeCount; c++) {
if (c && !i) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Cycle got shorter").detail("Before", nodeCount).detail("After", c).detail("KeyPrefix", keyPrefix.printable());
logTestData(data);
@ -172,7 +175,8 @@ struct CycleWorkload : TestWorkload {
logTestData(data);
return false;
}
double d = testKeyToDouble(data[i].value, keyPrefix);
d = testKeyToDouble(data[i].value, keyPrefix);
iPrev = i;
i = (int)d;
if ( i != d || i<0 || i>=nodeCount) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Invalid value").detail("KeyPrefix", keyPrefix.printable());
@ -181,7 +185,8 @@ struct CycleWorkload : TestWorkload {
}
}
if (i != 0) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Cycle got longer").detail("KeyPrefix", keyPrefix.printable());
TraceEvent(SevError, "TestFailure").detail("Reason", "Cycle got longer").detail("KeyPrefix", keyPrefix.printable()).detail("Key", key(i)).detail("Value", data[i].value).
detail("Iteration", c).detail("Nodecount", nodeCount).detail("Int", i).detail("Double", d).detail("ValuePrev", data[iPrev].value).detail("KeyPrev", data[iPrev].key);
logTestData(data);
return false;
}

View File

@ -59,7 +59,9 @@ struct ExceptionContract {
e.code() == error_code_transaction_cancelled ||
e.code() == error_code_key_too_large ||
e.code() == error_code_value_too_large ||
e.code() == error_code_process_behind)
e.code() == error_code_process_behind ||
e.code() == error_code_batch_transaction_throttled ||
e.code() == error_code_tag_throttled)
{
return;
}

View File

@ -226,15 +226,21 @@ struct ReadWriteWorkload : KVWorkload {
ACTOR static Future<bool> traceDumpWorkers( Reference<AsyncVar<ServerDBInfo>> db ) {
try {
loop {
ErrorOr<std::vector<WorkerDetails>> workerList = wait( db->get().clusterInterface.getWorkers.tryGetReply( GetWorkersRequest() ) );
if( workerList.present() ) {
std::vector<Future<ErrorOr<Void>>> dumpRequests;
for( int i = 0; i < workerList.get().size(); i++)
dumpRequests.push_back( workerList.get()[i].interf.traceBatchDumpRequest.tryGetReply( TraceBatchDumpRequest() ) );
wait( waitForAll( dumpRequests ) );
return true;
choose {
when( wait( db->onChange() ) ) {}
when (ErrorOr<std::vector<WorkerDetails>> workerList = wait( db->get().clusterInterface.getWorkers.tryGetReply( GetWorkersRequest() ) );)
{
if( workerList.present() ) {
std::vector<Future<ErrorOr<Void>>> dumpRequests;
for( int i = 0; i < workerList.get().size(); i++)
dumpRequests.push_back( workerList.get()[i].interf.traceBatchDumpRequest.tryGetReply( TraceBatchDumpRequest() ) );
wait( waitForAll( dumpRequests ) );
return true;
}
wait( delay( 1.0 ) );
}
}
wait( delay( 1.0 ) );
}
} catch( Error &e ) {
TraceEvent(SevError, "FailedToDumpWorkers").error(e);

View File

@ -95,9 +95,9 @@ public:
inline explicit Arena( size_t reservedSize );
//~Arena();
Arena(const Arena&);
Arena(Arena && r) BOOST_NOEXCEPT;
Arena(Arena&& r) noexcept;
Arena& operator=(const Arena&);
Arena& operator=(Arena&&) BOOST_NOEXCEPT;
Arena& operator=(Arena&&) noexcept;
inline void dependsOn( const Arena& p );
inline size_t getSize() const;
@ -173,12 +173,12 @@ inline Arena::Arena(size_t reservedSize) : impl( 0 ) {
ArenaBlock::create((int)reservedSize,impl);
}
inline Arena::Arena( const Arena& r ) : impl( r.impl ) {}
inline Arena::Arena(Arena && r) BOOST_NOEXCEPT : impl(std::move(r.impl)) {}
inline Arena::Arena(Arena&& r) noexcept : impl(std::move(r.impl)) {}
inline Arena& Arena::operator=(const Arena& r) {
impl = r.impl;
return *this;
}
inline Arena& Arena::operator=(Arena&& r) BOOST_NOEXCEPT {
inline Arena& Arena::operator=(Arena&& r) noexcept {
impl = std::move(r.impl);
return *this;
}
@ -380,12 +380,11 @@ public:
}
#else
Standalone( const T& t, const Arena& arena ) : Arena( arena ), T( t ) {}
Standalone( const Standalone<T> & t ) : Arena((Arena const&)t), T((T const&)t) {}
Standalone<T>& operator=( const Standalone<T> & t ) {
*(Arena*)this = (Arena const&)t;
*(T*)this = (T const&)t;
return *this;
}
Standalone(const Standalone<T>&) = default;
Standalone<T>& operator=(const Standalone<T>&) = default;
Standalone(Standalone<T>&&) = default;
Standalone<T>& operator=(Standalone<T>&&) = default;
~Standalone() = default;
#endif
template <class U> Standalone<U> castTo() const {
@ -710,15 +709,20 @@ inline bool operator != (const StringRef& lhs, const StringRef& rhs ) { return !
inline bool operator <= ( const StringRef& lhs, const StringRef& rhs ) { return !(lhs>rhs); }
inline bool operator >= ( const StringRef& lhs, const StringRef& rhs ) { return !(lhs<rhs); }
// This trait is used by VectorRef to determine if it should just memcpy the vector contents.
// FIXME: VectorRef really should use std::is_trivially_copyable for this BUT that is not implemented
// in gcc c++0x so instead we will use this custom trait which defaults to std::is_trivial, which
// handles most situations but others will have to be specialized.
// This trait is used by VectorRef to determine if deep copy constructor should recursively
// call deep copies of each element.
//
// TODO: There should be an easier way to identify the difference between flow_ref and non-flow_ref types.
// std::is_trivially_copyable does not work because some flow_ref types are trivially copyable
// and some non-flow_ref types are not trivially copyable.
template <typename T>
struct memcpy_able : std::is_trivial<T> {};
struct flow_ref : std::integral_constant<bool, !std::is_fundamental_v<T>> {};
template <>
struct memcpy_able<UID> : std::integral_constant<bool, true> {};
struct flow_ref<UID> : std::integral_constant<bool, false> {};
template <class A, class B>
struct flow_ref<std::pair<A, B>> : std::integral_constant<bool, false> {};
template<class T>
struct string_serialized_traits : std::false_type {
@ -794,7 +798,7 @@ public:
using value_type = T;
static_assert(SerStrategy == VecSerStrategy::FlatBuffers || string_serialized_traits<T>::value);
// T must be trivially destructible (and copyable)!
// T must be trivially destructible!
VectorRef() : data(0), m_size(0), m_capacity(0) {}
template <VecSerStrategy S>
@ -809,19 +813,19 @@ public:
return *this;
}
// Arena constructor for non-Ref types, identified by memcpy_able
// Arena constructor for non-Ref types, identified by !flow_ref
template <class T2 = T, VecSerStrategy S>
VectorRef(Arena& p, const VectorRef<T, S>& toCopy, typename std::enable_if<memcpy_able<T2>::value, int>::type = 0)
VectorRef(Arena& p, const VectorRef<T, S>& toCopy, typename std::enable_if<!flow_ref<T2>::value, int>::type = 0)
: VPS(toCopy), data((T*)new (p) uint8_t[sizeof(T) * toCopy.size()]), m_size(toCopy.size()),
m_capacity(toCopy.size()) {
if (m_size > 0) {
memcpy(data, toCopy.data, m_size * sizeof(T));
std::copy(toCopy.data, toCopy.data + m_size, data);
}
}
// Arena constructor for Ref types, which must have an Arena constructor
template <class T2 = T, VecSerStrategy S>
VectorRef(Arena& p, const VectorRef<T, S>& toCopy, typename std::enable_if<!memcpy_able<T2>::value, int>::type = 0)
VectorRef(Arena& p, const VectorRef<T, S>& toCopy, typename std::enable_if<flow_ref<T2>::value, int>::type = 0)
: VPS(), data((T*)new (p) uint8_t[sizeof(T) * toCopy.size()]), m_size(toCopy.size()), m_capacity(toCopy.size()) {
for (int i = 0; i < m_size; i++) {
auto ptr = new (&data[i]) T(p, toCopy[i]);
@ -917,7 +921,7 @@ public:
if (m_size + count > m_capacity) reallocate(p, m_size + count);
VPS::invalidate();
if (count > 0) {
memcpy(data + m_size, begin, sizeof(T) * count);
std::copy(begin, begin + count, data + m_size);
}
m_size += count;
}
@ -957,15 +961,15 @@ public:
if (size > m_capacity) reallocate(p, size);
}
// expectedSize() for non-Ref types, identified by memcpy_able
// expectedSize() for non-Ref types, identified by !flow_ref
template <class T2 = T>
typename std::enable_if<memcpy_able<T2>::value, size_t>::type expectedSize() const {
typename std::enable_if<!flow_ref<T2>::value, size_t>::type expectedSize() const {
return sizeof(T) * m_size;
}
// expectedSize() for Ref types, which must in turn have expectedSize() implemented.
template <class T2 = T>
typename std::enable_if<!memcpy_able<T2>::value, size_t>::type expectedSize() const {
typename std::enable_if<flow_ref<T2>::value, size_t>::type expectedSize() const {
size_t t = sizeof(T) * m_size;
for (int i = 0; i < m_size; i++) t += data[i].expectedSize();
return t;
@ -982,9 +986,9 @@ private:
void reallocate(Arena& p, int requiredCapacity) {
requiredCapacity = std::max(m_capacity * 2, requiredCapacity);
// SOMEDAY: Maybe we are right at the end of the arena and can expand cheaply
T* newData = (T*)new (p) uint8_t[requiredCapacity * sizeof(T)];
T* newData = new (p) T[requiredCapacity];
if (m_size > 0) {
memcpy(newData, data, m_size * sizeof(T));
std::move(data, data + m_size, newData);
}
data = newData;
m_capacity = requiredCapacity;

View File

@ -77,13 +77,13 @@ public:
}
}
Deque(Deque&& r) BOOST_NOEXCEPT : begin(r.begin), end(r.end), mask(r.mask), arr(r.arr) {
Deque(Deque&& r) noexcept : begin(r.begin), end(r.end), mask(r.mask), arr(r.arr) {
r.arr = nullptr;
r.begin = r.end = 0;
r.mask = -1;
}
void operator=(Deque&& r) BOOST_NOEXCEPT {
void operator=(Deque&& r) noexcept {
cleanup();
begin = r.begin;

View File

@ -104,7 +104,7 @@ public:
static Reference<P> addRef( P* ptr ) { ptr->addref(); return Reference(ptr); }
Reference(const Reference& r) : ptr(r.getPtr()) { if (ptr) addref(ptr); }
Reference(Reference && r) BOOST_NOEXCEPT : ptr(r.getPtr()) { r.ptr = NULL; }
Reference(Reference&& r) noexcept : ptr(r.getPtr()) { r.ptr = NULL; }
template <class Q>
Reference(const Reference<Q>& r) : ptr(r.getPtr()) { if (ptr) addref(ptr); }
@ -122,7 +122,7 @@ public:
}
return *this;
}
Reference& operator=(Reference&& r) BOOST_NOEXCEPT {
Reference& operator=(Reference&& r) noexcept {
P* oldPtr = ptr;
P* newPtr = r.ptr;
if (oldPtr != newPtr) {

View File

@ -69,23 +69,33 @@ bool operator<(CompatibleWithKey const& l, KeyValueMapPair const& r) {
class IKeyValueContainer {
public:
typedef typename IndexedSet<KeyValueMapPair, uint64_t>::iterator iterator;
using const_iterator = IndexedSet<KeyValueMapPair, uint64_t>::const_iterator;
using iterator = IndexedSet<KeyValueMapPair, uint64_t>::iterator;
IKeyValueContainer() = default;
~IKeyValueContainer() = default;
bool empty() { return data.empty(); }
bool empty() const { return data.empty(); }
void clear() { return data.clear(); }
std::tuple<size_t, size_t, size_t> size() { return std::make_tuple(0, 0, 0); }
std::tuple<size_t, size_t, size_t> size() const { return std::make_tuple(0, 0, 0); }
const_iterator find(const StringRef& key) const { return data.find(key); }
iterator find(const StringRef& key) { return data.find(key); }
const_iterator begin() const { return data.begin(); }
iterator begin() { return data.begin(); }
const_iterator cbegin() const { return begin(); }
const_iterator end() const { return data.end(); }
iterator end() { return data.end(); }
const_iterator cend() const { return end(); }
const_iterator lower_bound(const StringRef& key) const { return data.lower_bound(key); }
iterator lower_bound(const StringRef& key) { return data.lower_bound(key); }
const_iterator upper_bound(const StringRef& key) const { return data.upper_bound(key); }
iterator upper_bound(const StringRef& key) { return data.upper_bound(key); }
iterator previous(iterator i) const { return data.previous(i); }
const_iterator previous(const_iterator i) const { return data.previous(i); }
const_iterator previous(iterator i) const { return data.previous(const_iterator{ i }); }
iterator previous(iterator i) { return data.previous(i); }
void erase(iterator begin, iterator end) { data.erase(begin, end); }
iterator insert(const StringRef& key, const StringRef& val, bool replaceExisting = true) {
@ -96,7 +106,8 @@ public:
return data.insert(pairs, replaceExisting);
}
uint64_t sumTo(iterator to) { return data.sumTo(to); }
uint64_t sumTo(const_iterator to) const { return data.sumTo(to); }
uint64_t sumTo(iterator to) const { return data.sumTo(const_iterator{ to }); }
static int getElementBytes() { return IndexedSet<KeyValueMapPair, uint64_t>::getElementBytes(); }

View File

@ -31,6 +31,7 @@
#include <cstring>
#include <deque>
#include <random>
#include <type_traits>
#include "flow/TreeBenchmark.h"
#include "flow/UnitTest.h"
template <class Node>
@ -204,18 +205,25 @@ TEST_CASE("/flow/IndexedSet/strings") {
template <typename K>
struct IndexedSetHarness {
using map = IndexedSet<K, int>;
using const_result = typename map::const_iterator;
using result = typename map::iterator;
using key_type = K;
map s;
void insert(K const& k) { s.insert(K(k), 1); }
result find(K const& k) const { return s.find(k); }
result not_found() const { return s.end(); }
result begin() const { return s.begin(); }
result end() const { return s.end(); }
result lower_bound(K const& k) const { return s.lower_bound(k); }
result upper_bound(K const& k) const { return s.upper_bound(k); }
const_result find(K const& k) const { return s.find(k); }
result find(K const& k) { return s.find(k); }
const_result not_found() const { return s.end(); }
result not_found() { return s.end(); }
const_result begin() const { return s.begin(); }
result begin() { return s.begin(); }
const_result end() const { return s.end(); }
result end() { return s.end(); }
const_result lower_bound(K const& k) const { return s.lower_bound(k); }
result lower_bound(K const& k) { return s.lower_bound(k); }
const_result upper_bound(K const& k) const { return s.upper_bound(k); }
result upper_bound(K const& k) { return s.upper_bound(k); }
void erase(K const& k) { s.erase(k); }
};
@ -494,4 +502,60 @@ TEST_CASE("/flow/IndexedSet/all numbers") {
return Void();
}
template <class T>
static constexpr bool is_const_ref_v = std::is_const_v<typename std::remove_reference_t<T>>;
TEST_CASE("/flow/IndexedSet/const_iterator") {
struct Key {
int key;
explicit Key(int key) : key(key) {}
};
struct Metric {
int metric;
explicit Metric(int metric) : metric(metric) {}
};
IndexedSet<int, int64_t> is;
for (int i = 0; i < 10; ++i) is.insert(i, 1);
IndexedSet<int, int64_t>& ncis = is;
static_assert(!is_const_ref_v<decltype(ncis)>);
static_assert(!is_const_ref_v<decltype(*ncis.begin())>);
static_assert(is_const_ref_v<decltype(*ncis.cbegin())>);
static_assert(!is_const_ref_v<decltype(*ncis.previous(ncis.end()))>);
static_assert(is_const_ref_v<decltype(*ncis.previous(ncis.cend()))>);
static_assert(!is_const_ref_v<decltype(*ncis.index(Metric{ 5 }))>);
static_assert(!is_const_ref_v<decltype(*ncis.find(Key{ 5 }))>);
static_assert(!is_const_ref_v<decltype(*ncis.upper_bound(Key{ 5 }))>);
static_assert(!is_const_ref_v<decltype(*ncis.lower_bound(Key{ 5 }))>);
static_assert(!is_const_ref_v<decltype(*ncis.lastLessOrEqual(Key{ 5 }))>);
static_assert(!is_const_ref_v<decltype(*ncis.lastItem())>);
const IndexedSet<int, int64_t>& cis = is;
static_assert(is_const_ref_v<decltype(cis)>);
static_assert(is_const_ref_v<decltype(*cis.begin())>);
static_assert(is_const_ref_v<decltype(*cis.cbegin())>);
static_assert(is_const_ref_v<decltype(*cis.previous(cis.end()))>);
static_assert(is_const_ref_v<decltype(*cis.previous(cis.cend()))>);
static_assert(is_const_ref_v<decltype(*cis.previous(ncis.end()))>);
static_assert(is_const_ref_v<decltype(*cis.previous(ncis.cend()))>);
static_assert(is_const_ref_v<decltype(*cis.index(Metric{ 5 }))>);
static_assert(is_const_ref_v<decltype(*cis.find(Key{ 5 }))>);
static_assert(is_const_ref_v<decltype(*cis.upper_bound(Key{ 5 }))>);
static_assert(is_const_ref_v<decltype(*cis.lower_bound(Key{ 5 }))>);
static_assert(is_const_ref_v<decltype(*cis.lastLessOrEqual(Key{ 5 }))>);
static_assert(is_const_ref_v<decltype(*cis.lastItem())>);
for (auto& val : ncis) {
static_assert(!is_const_ref_v<decltype(val)>);
}
for (const auto& val : ncis) {
static_assert(is_const_ref_v<decltype(val)>);
}
for (auto& val : cis) {
static_assert(is_const_ref_v<decltype(val)>);
}
return Void();
}
void forceLinkIndexedSetTests() {}

View File

@ -29,6 +29,7 @@
#include "flow/Error.h"
#include <deque>
#include <type_traits>
#include <vector>
// IndexedSet<T, Metric> is similar to a std::set<T>, with the following additional features:
@ -39,7 +40,6 @@
// - Search functions (find(), lower_bound(), etc) can accept a type comparable to T instead of T
// (e.g. StringRef when T is std::string or Standalone<StringRef>). This can save a lot of needless
// copying at query time for read-mostly sets with string keys.
// - iterators are not const; the responsibility of not changing the order lies with the caller
// - the size() function is missing; if the metric being used is a count sumTo(end()) will do instead
// A number of STL compatibility features are missing and should be added as needed.
// T must define operator <, which must define a total order. Unlike std::set,
@ -70,8 +70,10 @@ private: // Forward-declare IndexedSet::Node because Clang is much stricter abou
// combinations, but still take advantage of move constructors when available (or required).
template <class T_, class Metric_>
Node(T_&& data, Metric_&& m, Node* parent=0) : data(std::forward<T_>(data)), total(std::forward<Metric_>(m)), parent(parent), balance(0) {
child[0] = child[1] = NULL;
child[0] = child[1] = nullptr;
}
Node(Node const&) = delete;
Node& operator=(Node const&) = delete;
~Node(){
delete child[0];
delete child[1];
@ -84,35 +86,98 @@ private: // Forward-declare IndexedSet::Node because Clang is much stricter abou
Node *parent;
};
public:
struct iterator{
typename IndexedSet::Node *i;
iterator() : i(0) {};
iterator(typename IndexedSet::Node *n) : i(n) {};
T& operator*() { return i->data; };
T* operator->() { return &i->data; }
template <bool isConst>
struct IteratorImpl {
typename std::conditional_t<isConst, const IndexedSet::Node, IndexedSet::Node>* node;
explicit IteratorImpl<isConst>(const IteratorImpl<!isConst>& nonConstIter) : node(nonConstIter.node) {
static_assert(isConst);
}
explicit IteratorImpl(decltype(node) n = nullptr) : node(n){};
typename std::conditional_t<isConst, const T, T>& operator*() const { return node->data; }
typename std::conditional_t<isConst, const T, T>* operator->() const { return &node->data; }
void operator++();
void decrementNonEnd();
bool operator == ( const iterator& r ) const { return i == r.i; }
bool operator != ( const iterator& r ) const { return i != r.i; }
bool operator==(const IteratorImpl<isConst>& r) const { return node == r.node; }
bool operator!=(const IteratorImpl<isConst>& r) const { return node != r.node; }
// following two methods are for memory storage engine(KeyValueStoreMemory class) use only
// in order to have same interface as radixtree
StringRef& getKey(uint8_t* dummyContent) const { return i->data.key; }
StringRef& getValue() const { return i->data.value; }
typename std::conditional_t<isConst, const StringRef, StringRef>& getKey(uint8_t* dummyContent) const {
return node->data.key;
}
typename std::conditional_t<isConst, const StringRef, StringRef>& getValue() const { return node->data.value; }
};
IndexedSet() : root(NULL) {};
~IndexedSet() { delete root; }
IndexedSet(IndexedSet&& r) BOOST_NOEXCEPT : root(r.root) { r.root = NULL; }
IndexedSet& operator=(IndexedSet&& r) BOOST_NOEXCEPT { delete root; root = r.root; r.root = 0; return *this; }
template <bool isConst>
struct Impl {
using NodeT = std::conditional_t<isConst, const Node, Node>;
using IteratorT = IteratorImpl<isConst>;
using SetT = std::conditional_t<isConst, const IndexedSet<T, Metric>, IndexedSet<T, Metric>>;
iterator begin() const;
iterator end() const { return iterator(); }
iterator previous(iterator i) const;
iterator lastItem() const;
static IteratorT begin(SetT&);
template <bool constIterator>
static IteratorImpl<isConst || constIterator> previous(SetT&, IteratorImpl<constIterator>);
template <class M>
static IteratorT index(SetT&, const M&);
template <class Key>
static IteratorT find(SetT&, const Key&);
template <class Key>
static IteratorT upper_bound(SetT&, const Key&);
template <class Key>
static IteratorT lower_bound(SetT&, const Key&);
template <class Key>
static IteratorT lastLessOrEqual(SetT&, const Key&);
static IteratorT lastItem(SetT&);
};
using ConstImpl = Impl<true>;
using NonConstImpl = Impl<false>;
public:
using iterator = IteratorImpl<false>;
using const_iterator = IteratorImpl<true>;
IndexedSet() : root(nullptr){};
~IndexedSet() { delete root; }
IndexedSet(IndexedSet&& r) noexcept : root(r.root) { r.root = nullptr; }
IndexedSet& operator=(IndexedSet&& r) noexcept {
delete root;
root = r.root;
r.root = 0;
return *this;
}
const_iterator begin() const { return ConstImpl::begin(*this); };
iterator begin() { return NonConstImpl::begin(*this); };
const_iterator cbegin() const { return begin(); }
const_iterator end() const { return const_iterator{}; }
iterator end() { return iterator{}; }
const_iterator cend() const { return end(); }
const_iterator previous(const_iterator i) const { return ConstImpl::previous(*this, i); }
const_iterator previous(iterator i) const { return ConstImpl::previous(*this, const_iterator{ i }); }
iterator previous(iterator i) { return NonConstImpl::previous(*this, i); }
const_iterator lastItem() const { return ConstImpl::lastItem(*this); }
iterator lastItem() { return NonConstImpl::lastItem(*this); }
bool empty() const { return !root; }
void clear() { delete root; root = NULL; }
void clear() {
delete root;
root = nullptr;
}
void swap( IndexedSet& r ) { std::swap( root, r.root ); }
// Place data in the set with the given metric. If an item equal to data is already in the set and,
@ -159,36 +224,78 @@ public:
// Returns x such that key==*x, or end()
template <class Key>
iterator find(const Key &key) const;
const_iterator find(const Key& key) const {
return ConstImpl::find(*this, key);
}
template <class Key>
iterator find(const Key& key) {
return NonConstImpl::find(*this, key);
}
// Returns the smallest x such that *x>=key, or end()
template <class Key>
iterator lower_bound(const Key &key) const;
const_iterator lower_bound(const Key& key) const {
return ConstImpl::lower_bound(*this, key);
}
template <class Key>
iterator lower_bound(const Key& key) {
return NonConstImpl::lower_bound(*this, key);
};
// Returns the smallest x such that *x>key, or end()
template <class Key>
iterator upper_bound(const Key &key) const;
const_iterator upper_bound(const Key& key) const {
return ConstImpl::upper_bound(*this, key);
}
template <class Key>
iterator upper_bound(const Key& key) {
return NonConstImpl::upper_bound(*this, key);
};
// Returns the largest x such that *x<=key, or end()
template <class Key>
iterator lastLessOrEqual( const Key &key ) const;
const_iterator lastLessOrEqual(const Key& key) const {
return ConstImpl::lastLessOrEqual(*this, key);
};
template <class Key>
iterator lastLessOrEqual(const Key& key) {
return NonConstImpl::lastLessOrEqual(*this, key);
}
// Returns smallest x such that sumTo(x+1) > metric, or end()
template <class M>
iterator index( M const& metric ) const;
const_iterator index(M const& metric) const {
return ConstImpl::index(*this, metric);
};
template <class M>
iterator index(M const& metric) {
return NonConstImpl::index(*this, metric);
}
// Return the metric inserted with item x
Metric getMetric(iterator x) const;
Metric getMetric(const_iterator x) const;
Metric getMetric(iterator x) const { return getMetric(const_iterator{ x }); }
// Return the sum of getMetric(x) for begin()<=x<to
Metric sumTo(iterator to) const;
Metric sumTo(const_iterator to) const;
Metric sumTo(iterator to) const { return sumTo(const_iterator{ to }); }
// Return the sum of getMetric(x) for begin<=x<end
Metric sumRange(iterator begin, iterator end) const { return sumTo(end) - sumTo(begin); }
Metric sumRange(const_iterator begin, const_iterator end) const { return sumTo(end) - sumTo(begin); }
Metric sumRange(iterator begin, iterator end) const {
return sumTo(const_iterator{ end }) - sumTo(const_iterator{ begin });
}
// Return the sum of getMetric(x) for all x s.t. begin <= *x && *x < end
template <class Key>
Metric sumRange(const Key& begin, const Key& end) const { return sumRange(lower_bound(begin), lower_bound(end)); }
template <class Key>
Metric sumRange(const Key& begin, const Key& end) const {
return sumRange(lower_bound(begin), lower_bound(end));
}
// Return the amount of memory used by an entry in the IndexedSet
static int getElementBytes() { return sizeof(Node); }
@ -212,18 +319,25 @@ private:
newNode->parent = oldNode->parent;
}
template <int direction, bool isConst>
static void moveIteratorImpl(std::conditional_t<isConst, const Node, Node>*& node) {
if (node->child[0 ^ direction]) {
node = node->child[0 ^ direction];
while (node->child[1 ^ direction]) node = node->child[1 ^ direction];
} else {
while (node->parent && node->parent->child[0 ^ direction] == node) node = node->parent;
node = node->parent;
}
}
// direction 0 = left, 1 = right
template <int direction>
static void moveIterator(Node* &i){
if (i->child[0^direction]) {
i = i->child[0^direction];
while (i->child[1^direction])
i = i->child[1^direction];
} else {
while (i->parent && i->parent->child[0^direction] == i)
i = i->parent;
i = i->parent;
}
static void moveIterator(Node const*& node) {
moveIteratorImpl<direction, true>(node);
}
template <int direction>
static void moveIterator(Node*& node) {
moveIteratorImpl<direction, false>(node);
}
public: // but testonly
@ -250,8 +364,11 @@ public:
void operator= ( MapPair const& rhs ) { key = rhs.key; value = rhs.value; }
MapPair( MapPair const& rhs ) : key(rhs.key), value(rhs.value) {}
MapPair(MapPair&& r) BOOST_NOEXCEPT : key(std::move(r.key)), value(std::move(r.value)) {}
void operator=(MapPair&& r) BOOST_NOEXCEPT { key = std::move(r.key); value = std::move(r.value); }
MapPair(MapPair&& r) noexcept : key(std::move(r.key)), value(std::move(r.value)) {}
void operator=(MapPair&& r) noexcept {
key = std::move(r.key);
value = std::move(r.value);
}
int compare(MapPair<Key, Value> const& r) const { return ::compare(key, r.key); }
template <class CompatibleWithKey>
@ -284,12 +401,19 @@ template <class Key, class Value, class Pair = MapPair<Key,Value>, class Metric=
class Map {
public:
typedef typename IndexedSet<Pair,Metric>::iterator iterator;
typedef typename IndexedSet<Pair, Metric>::const_iterator const_iterator;
Map() {}
iterator begin() const { return set.begin(); }
iterator end() const { return set.end(); }
iterator lastItem() const { return set.lastItem(); }
iterator previous(iterator i) const { return set.previous(i); }
const_iterator begin() const { return set.begin(); }
iterator begin() { return set.begin(); }
const_iterator cbegin() const { return begin(); }
const_iterator end() const { return set.end(); }
iterator end() { return set.end(); }
const_iterator cend() const { return end(); }
const_iterator lastItem() const { return set.lastItem(); }
iterator lastItem() { return set.lastItem(); }
const_iterator previous(const_iterator i) const { return set.previous(i); }
iterator previous(iterator i) { return set.previous(i); }
bool empty() const { return set.empty(); }
Value& operator[]( const Key& key ) {
@ -317,25 +441,65 @@ public:
}
template <class KeyCompatible>
iterator find( KeyCompatible const& k ) const { return set.find(k); }
const_iterator find(KeyCompatible const& k) const {
return set.find(k);
}
template <class KeyCompatible>
iterator lower_bound( KeyCompatible const& k ) const { return set.lower_bound(k); }
iterator find(KeyCompatible const& k) {
return set.find(k);
}
template <class KeyCompatible>
iterator upper_bound( KeyCompatible const& k ) const { return set.upper_bound(k); }
const_iterator lower_bound(KeyCompatible const& k) const {
return set.lower_bound(k);
}
template <class KeyCompatible>
iterator lastLessOrEqual( KeyCompatible const& k ) const { return set.lastLessOrEqual(k); }
template <class M>
iterator index( M const& metric ) const { return set.index(metric); }
Metric getMetric(iterator x) const { return set.getMetric(x); }
Metric sumTo(iterator to) const { return set.sumTo(to); }
Metric sumRange(iterator begin, iterator end) const { return set.sumRange(begin,end); }
iterator lower_bound(KeyCompatible const& k) {
return set.lower_bound(k);
}
template <class KeyCompatible>
const_iterator upper_bound(KeyCompatible const& k) const {
return set.upper_bound(k);
}
template <class KeyCompatible>
iterator upper_bound(KeyCompatible const& k) {
return set.upper_bound(k);
}
template <class KeyCompatible>
const_iterator lastLessOrEqual(KeyCompatible const& k) const {
return set.lastLessOrEqual(k);
}
template <class KeyCompatible>
iterator lastLessOrEqual(KeyCompatible const& k) {
return set.lastLessOrEqual(k);
}
template <class M>
const_iterator index(M const& metric) const {
return set.index(metric);
}
template <class M>
iterator index(M const& metric) {
return set.index(metric);
}
Metric getMetric(const_iterator x) const { return set.getMetric(x); }
Metric getMetric(iterator x) const { return getMetric(const_iterator{ x }); }
Metric sumTo(const_iterator to) const { return set.sumTo(to); }
Metric sumTo(iterator to) const { return sumTo(const_iterator{ to }); }
Metric sumRange(const_iterator begin, const_iterator end) const { return set.sumRange(begin, end); }
Metric sumRange(iterator begin, iterator end) const { return set.sumRange(begin, end); }
template <class KeyCompatible>
Metric sumRange(const KeyCompatible& begin, const KeyCompatible& end) const { return set.sumRange(begin,end); }
static int getElementBytes() { return IndexedSet< Pair, Metric >::getElementBytes(); }
Map(Map&& r) BOOST_NOEXCEPT : set(std::move(r.set)) {}
void operator=(Map&& r) BOOST_NOEXCEPT { set = std::move(r.set); }
Map(Map&& r) noexcept : set(std::move(r.set)) {}
void operator=(Map&& r) noexcept { set = std::move(r.set); }
private:
Map( Map<Key,Value,Pair> const& ); // unimplemented
@ -347,13 +511,15 @@ private:
/////////////////////// implementation //////////////////////////
template <class T, class Metric>
void IndexedSet<T,Metric>::iterator::operator++(){
moveIterator<1>(i);
template <bool isConst>
void IndexedSet<T, Metric>::IteratorImpl<isConst>::operator++() {
moveIterator<1>(node);
}
template <class T, class Metric>
void IndexedSet<T,Metric>::iterator::decrementNonEnd(){
moveIterator<0>(i);
template <bool isConst>
void IndexedSet<T, Metric>::IteratorImpl<isConst>::decrementNonEnd() {
moveIterator<0>(node);
}
template <class Node>
@ -578,28 +744,33 @@ Node* ISCommonSubtreeRoot(Node* first, Node* last) {
}
template <class T, class Metric>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::begin() const {
Node *x = root;
while (x && x->child[0])
x = x->child[0];
return x;
template <bool isConst>
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::begin(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self) {
NodeT* x = self.root;
while (x && x->child[0]) x = x->child[0];
return IteratorT{ x };
}
template <class T, class Metric>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::previous(typename IndexedSet<T,Metric>::iterator i) const {
if (i==end())
return lastItem();
template <bool isConst>
template <bool constIterator>
typename IndexedSet<T, Metric>::template IteratorImpl<isConst || constIterator>
IndexedSet<T, Metric>::Impl<isConst>::previous(IndexedSet<T, Metric>::Impl<isConst>::SetT& self,
IndexedSet<T, Metric>::IteratorImpl<constIterator> iter) {
if (iter == self.end()) return self.lastItem();
moveIterator<0>(i.i);
return i;
moveIterator<0>(iter.node);
return iter;
}
template <class T, class Metric>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::lastItem() const {
Node *x = root;
while (x && x->child[1])
x = x->child[1];
return x;
template <bool isConst>
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::lastItem(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self) {
NodeT* x = self.root;
while (x && x->child[1]) x = x->child[1];
return IteratorT{ x };
}
template <class T, class Metric> template<class T_, class Metric_>
@ -617,9 +788,9 @@ Metric IndexedSet<T,Metric>::addMetric(T_&& data, Metric_&& metric){
template <class T, class Metric> template<class T_, class Metric_>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::insert(T_&& data, Metric_&& metric, bool replaceExisting){
if (root == NULL){
if (root == nullptr) {
root = new Node(std::forward<T_>(data), std::forward<Metric_>(metric));
return root;
return iterator{ root };
}
Node *t = root;
int d; // direction
@ -642,7 +813,7 @@ typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::insert(T_&& data,
}
}
return returnNode;
return iterator{ returnNode };
}
d = cmp > 0;
Node *nextT = t->child[d];
@ -685,23 +856,23 @@ typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::insert(T_&& data,
t->total = t->total + metric;
}
return newNode;
return iterator{ newNode };
}
template <class T, class Metric>
int IndexedSet<T,Metric>::insert(const std::vector<std::pair<T,Metric>>& dataVector, bool replaceExisting) {
int num_inserted = 0;
Node *blockStart = NULL;
Node *blockEnd = NULL;
Node* blockStart = nullptr;
Node* blockEnd = nullptr;
for(int i = 0; i < dataVector.size(); ++i) {
Metric metric = dataVector[i].second;
T data = std::move(dataVector[i].first);
int d = 1; // direction
if(blockStart == NULL || (blockEnd != NULL && data >= blockEnd->data)) {
blockEnd = NULL;
if (root == NULL) {
if (blockStart == nullptr || (blockEnd != nullptr && data >= blockEnd->data)) {
blockEnd = nullptr;
if (root == nullptr) {
root = new Node(std::move(data), metric);
num_inserted++;
blockStart = root;
@ -842,8 +1013,8 @@ Metric IndexedSet<T, Metric>::eraseHalf(Node* start, Node* end, int eraseDir, in
metricDelta = metricDelta - n->total;
n->parent = start->parent;
}
start->child[fromDir] = NULL;
start->child[fromDir] = nullptr;
toFree.push_back( start );
}
@ -874,13 +1045,13 @@ void IndexedSet<T,Metric>::erase( typename IndexedSet<T,Metric>::iterator begin,
// Removes all nodes in the set between first and last, inclusive.
// toFree is extended with the roots of completely removed subtrees.
ASSERT(!end.i || (begin.i && (::compare(*begin, *end) <= 0)));
ASSERT(!end.node || (begin.node && (::compare(*begin, *end) <= 0)));
if(begin == end)
return;
IndexedSet<T,Metric>::Node* first = begin.i;
IndexedSet<T,Metric>::Node* last = previous(end).i;
IndexedSet<T, Metric>::Node* first = begin.node;
IndexedSet<T, Metric>::Node* last = previous(end).node;
IndexedSet<T,Metric>::Node* subRoot = ISCommonSubtreeRoot(first, last);
@ -897,7 +1068,7 @@ void IndexedSet<T,Metric>::erase( typename IndexedSet<T,Metric>::iterator begin,
int heightDelta = leftHeightDelta + rightHeightDelta;
// Rebalance and update metrics for all nodes from subRoot up to the root
for(auto p = subRoot; p != NULL; p = p->parent) {
for (auto p = subRoot; p != nullptr; p = p->parent) {
p->total = p->total - metricDelta;
auto& pc = p->parent ? p->parent->child[p->parent->child[1]==p] : root;
@ -925,7 +1096,7 @@ void IndexedSet<T,Metric>::erase(iterator toErase) {
{
// Find the node to erase
Node* t = toErase.i;
Node* t = toErase.node;
if (!t) return;
if (!t->child[0] || !t->child[1]) {
@ -1005,101 +1176,106 @@ void IndexedSet<T,Metric>::erase(iterator toErase) {
// Returns x such that key==*x, or end()
template <class T, class Metric>
template <bool isConst>
template <class Key>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::find(const Key &key) const {
Node* t = root;
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::find(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self, const Key& key) {
NodeT* t = self.root;
while (t){
int cmp = compare(key, t->data);
if (cmp == 0) return iterator(t);
if (cmp == 0) return IteratorT{ t };
t = t->child[cmp > 0];
}
return end();
return self.end();
}
// Returns the smallest x such that *x>=key, or end()
template <class T, class Metric>
template <bool isConst>
template <class Key>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::lower_bound(const Key &key) const {
Node* t = root;
if (!t) return iterator();
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::lower_bound(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self, const Key& key) {
NodeT* t = self.root;
if (!t) return self.end();
bool less;
while (true) {
less = t->data < key;
Node* n = t->child[less];
NodeT* n = t->child[less];
if (!n) break;
t = n;
}
if (less) moveIterator<1>(t);
return iterator(t);
return IteratorT{ t };
}
// Returns the smallest x such that *x>key, or end()
template <class T, class Metric>
template <bool isConst>
template <class Key>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::upper_bound(const Key &key) const {
Node* t = root;
if (!t) return iterator();
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::upper_bound(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self, const Key& key) {
NodeT* t = self.root;
if (!t) return self.end();
bool not_less;
while (true) {
not_less = !(key < t->data);
Node* n = t->child[not_less];
NodeT* n = t->child[not_less];
if (!n) break;
t = n;
}
if (not_less) moveIterator<1>(t);
return iterator(t);
return IteratorT{ t };
}
template <class T, class Metric>
template <bool isConst>
template <class Key>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::lastLessOrEqual(const Key &key) const {
iterator i = upper_bound(key);
if (i == begin()) return end();
return previous(i);
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::lastLessOrEqual(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self, const Key& key) {
auto i = self.upper_bound(key);
if (i == self.begin()) return self.end();
return self.previous(i);
}
// Returns first x such that metric < sum(begin(), x+1), or end()
template <class T, class Metric>
template <bool isConst>
template <class M>
typename IndexedSet<T,Metric>::iterator IndexedSet<T,Metric>::index( M const& metric ) const
{
typename IndexedSet<T, Metric>::template Impl<isConst>::IteratorT IndexedSet<T, Metric>::Impl<isConst>::index(
IndexedSet<T, Metric>::Impl<isConst>::SetT& self, const M& metric) {
M m = metric;
Node* t = root;
NodeT* t = self.root;
while (t) {
if (t->child[0] && m < t->child[0]->total)
t = t->child[0];
else {
m = m - t->total;
if (t->child[1])
m = m + t->child[1]->total;
if (m < M())
return iterator(t);
if (t->child[1]) m = m + t->child[1]->total;
if (m < M()) return IteratorT{ t };
t = t->child[1];
}
}
return end();
return self.end();
}
template <class T, class Metric>
Metric IndexedSet<T,Metric>::getMetric(typename IndexedSet<T,Metric>::iterator x) const {
Metric m = x.i->total;
Metric IndexedSet<T, Metric>::getMetric(typename IndexedSet<T, Metric>::const_iterator x) const {
Metric m = x.node->total;
for(int i=0; i<2; i++)
if (x.i->child[i])
m = m - x.i->child[i]->total;
if (x.node->child[i]) m = m - x.node->child[i]->total;
return m;
}
template <class T, class Metric>
Metric IndexedSet<T,Metric>::sumTo(typename IndexedSet<T,Metric>::iterator end) const {
if (!end.i)
return root ? root->total : Metric();
Metric IndexedSet<T, Metric>::sumTo(typename IndexedSet<T, Metric>::const_iterator end) const {
if (!end.node) return root ? root->total : Metric();
Metric m = end.i->child[0] ? end.i->child[0]->total : Metric();
for(Node* p = end.i; p->parent; p=p->parent) {
Metric m = end.node->child[0] ? end.node->child[0]->total : Metric();
for (const Node* p = end.node; p->parent; p = p->parent) {
if (p->parent->child[1] == p) {
m = m - p->total;
m = m + p->parent->total;

View File

@ -264,7 +264,7 @@ class BindPromise {
public:
BindPromise( const char* errContext, UID errID ) : errContext(errContext), errID(errID) {}
BindPromise( BindPromise const& r ) : p(r.p), errContext(r.errContext), errID(r.errID) {}
BindPromise(BindPromise&& r) BOOST_NOEXCEPT : p(std::move(r.p)), errContext(r.errContext), errID(r.errID) {}
BindPromise(BindPromise&& r) noexcept : p(std::move(r.p)), errContext(r.errContext), errID(r.errID) {}
Future<Void> getFuture() { return p.getFuture(); }
@ -841,7 +841,7 @@ private:
struct PromiseTask : public Task, public FastAllocated<PromiseTask> {
Promise<Void> promise;
PromiseTask() {}
explicit PromiseTask( Promise<Void>&& promise ) BOOST_NOEXCEPT : promise(std::move(promise)) {}
explicit PromiseTask(Promise<Void>&& promise) noexcept : promise(std::move(promise)) {}
virtual void operator()() {
promise.send(Void());

View File

@ -621,9 +621,6 @@ inline static int clz( uint32_t value ) {
#define clz __builtin_clz
#endif
#include <boost/config.hpp>
// The formerly existing BOOST_NOEXCEPT is now BOOST_NOEXCEPT
// These return thread local counts
int64_t getNumProfilesDeferred();
int64_t getNumProfilesOverflowed();

View File

@ -122,6 +122,7 @@ public: // introduced features
PROTOCOL_VERSION_FEATURE(0x0FDB00B063010000LL, BackupWorker);
PROTOCOL_VERSION_FEATURE(0x0FDB00B063010000LL, ReportConflictingKeys);
PROTOCOL_VERSION_FEATURE(0x0FDB00B063010000LL, SmallEndpoints);
PROTOCOL_VERSION_FEATURE(0x0FDB00B063010000LL, CacheRole);
};
// These impact both communications and the deserialization of certain database and IKeyValueStore keys.

View File

@ -227,14 +227,10 @@ struct MetricData {
appendStart(appendStart) {
}
MetricData( MetricData&& r ) BOOST_NOEXCEPT :
start(r.start),
rollTime(r.rollTime),
appendStart(r.appendStart),
writer(std::move(r.writer)) {
}
MetricData(MetricData&& r) noexcept
: start(r.start), rollTime(r.rollTime), appendStart(r.appendStart), writer(std::move(r.writer)) {}
void operator=( MetricData&& r ) BOOST_NOEXCEPT {
void operator=(MetricData&& r) noexcept {
start = r.start; rollTime = r.rollTime; appendStart = r.appendStart; writer = std::move(r.writer);
}
@ -645,11 +641,9 @@ template <class T, class Descriptor = NullDescriptor, class FieldLevelType = Fie
struct EventField : public Descriptor {
std::vector<FieldLevelType> levels;
EventField( EventField&& r ) BOOST_NOEXCEPT : Descriptor(r), levels(std::move(r.levels)) {}
EventField(EventField&& r) noexcept : Descriptor(r), levels(std::move(r.levels)) {}
void operator=( EventField&& r ) BOOST_NOEXCEPT {
levels = std::move(r.levels);
}
void operator=(EventField&& r) noexcept { levels = std::move(r.levels); }
EventField(Descriptor d = Descriptor()) : Descriptor(d) {
}

View File

@ -458,9 +458,7 @@ public:
ThreadFuture( const ThreadFuture<T>& rhs ) : sav(rhs.sav) {
if (sav) sav->addref();
}
ThreadFuture(ThreadFuture<T>&& rhs) BOOST_NOEXCEPT : sav(rhs.sav) {
rhs.sav = 0;
}
ThreadFuture(ThreadFuture<T>&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; }
ThreadFuture( const T& presentValue )
: sav(new ThreadSingleAssignmentVar<T>())
{
@ -483,7 +481,7 @@ public:
if (sav) sav->delref();
sav = rhs.sav;
}
void operator=(ThreadFuture<T>&& rhs) BOOST_NOEXCEPT {
void operator=(ThreadFuture<T>&& rhs) noexcept {
if (sav != rhs.sav) {
if (sav) sav->delref();
sav = rhs.sav;

View File

@ -35,6 +35,7 @@ ERROR( end_of_stream, 1, "End of stream" )
ERROR( operation_failed, 1000, "Operation failed")
ERROR( wrong_shard_server, 1001, "Shard is not available from this server")
ERROR( operation_obsolete, 1002, "Operation result no longer necessary")
ERROR( cold_cache_server, 1003, "Cache server is not warm for this range")
ERROR( timed_out, 1004, "Operation timed out" )
ERROR( coordinated_state_conflict, 1005, "Conflict occurred while changing coordination information" )
ERROR( all_alternatives_failed, 1006, "All alternatives failed" )

View File

@ -677,7 +677,7 @@ public:
if (sav) sav->addFutureRef();
//if (sav->endpoint.isValid()) cout << "Future copied for " << sav->endpoint.key << endl;
}
Future(Future<T>&& rhs) BOOST_NOEXCEPT : sav(rhs.sav) {
Future(Future<T>&& rhs) noexcept : sav(rhs.sav) {
rhs.sav = 0;
//if (sav->endpoint.isValid()) cout << "Future moved for " << sav->endpoint.key << endl;
}
@ -716,7 +716,7 @@ public:
if (sav) sav->delFutureRef();
sav = rhs.sav;
}
void operator=(Future<T>&& rhs) BOOST_NOEXCEPT {
void operator=(Future<T>&& rhs) noexcept {
if (sav != rhs.sav) {
if (sav) sav->delFutureRef();
sav = rhs.sav;
@ -792,7 +792,7 @@ public:
bool isValid() const { return sav != NULL; }
Promise() : sav(new SAV<T>(0, 1)) {}
Promise(const Promise& rhs) : sav(rhs.sav) { sav->addPromiseRef(); }
Promise(Promise&& rhs) BOOST_NOEXCEPT : sav(rhs.sav) { rhs.sav = 0; }
Promise(Promise&& rhs) noexcept : sav(rhs.sav) { rhs.sav = 0; }
~Promise() { if (sav) sav->delPromiseRef(); }
void operator=(const Promise& rhs) {
@ -800,7 +800,7 @@ public:
if (sav) sav->delPromiseRef();
sav = rhs.sav;
}
void operator=(Promise && rhs) BOOST_NOEXCEPT {
void operator=(Promise&& rhs) noexcept {
if (sav != rhs.sav) {
if (sav) sav->delPromiseRef();
sav = rhs.sav;
@ -845,14 +845,14 @@ public:
}
FutureStream() : queue(NULL) {}
FutureStream(const FutureStream& rhs) : queue(rhs.queue) { queue->addFutureRef(); }
FutureStream(FutureStream&& rhs) BOOST_NOEXCEPT : queue(rhs.queue) { rhs.queue = 0; }
FutureStream(FutureStream&& rhs) noexcept : queue(rhs.queue) { rhs.queue = 0; }
~FutureStream() { if (queue) queue->delFutureRef(); }
void operator=(const FutureStream& rhs) {
rhs.queue->addFutureRef();
if (queue) queue->delFutureRef();
queue = rhs.queue;
}
void operator=(FutureStream&& rhs) BOOST_NOEXCEPT {
void operator=(FutureStream&& rhs) noexcept {
if (rhs.queue != queue) {
if (queue) queue->delFutureRef();
queue = rhs.queue;
@ -949,13 +949,13 @@ public:
FutureStream<T> getFuture() const { queue->addFutureRef(); return FutureStream<T>(queue); }
PromiseStream() : queue(new NotifiedQueue<T>(0, 1)) {}
PromiseStream(const PromiseStream& rhs) : queue(rhs.queue) { queue->addPromiseRef(); }
PromiseStream(PromiseStream&& rhs) BOOST_NOEXCEPT : queue(rhs.queue) { rhs.queue = 0; }
PromiseStream(PromiseStream&& rhs) noexcept : queue(rhs.queue) { rhs.queue = 0; }
void operator=(const PromiseStream& rhs) {
rhs.queue->addPromiseRef();
if (queue) queue->delPromiseRef();
queue = rhs.queue;
}
void operator=(PromiseStream&& rhs) BOOST_NOEXCEPT {
void operator=(PromiseStream&& rhs) noexcept {
if (queue != rhs.queue) {
if (queue) queue->delPromiseRef();
queue = rhs.queue;

View File

@ -1251,7 +1251,7 @@ struct FlowLock : NonCopyable, public ReferenceCounted<FlowLock> {
int remaining;
Releaser() : lock(0), remaining(0) {}
Releaser( FlowLock& lock, int64_t amount = 1 ) : lock(&lock), remaining(amount) {}
Releaser(Releaser&& r) BOOST_NOEXCEPT : lock(r.lock), remaining(r.remaining) { r.remaining = 0; }
Releaser(Releaser&& r) noexcept : lock(r.lock), remaining(r.remaining) { r.remaining = 0; }
void operator=(Releaser&& r) { if (remaining) lock->release(remaining); lock = r.lock; remaining = r.remaining; r.remaining = 0; }
void release( int64_t amount = -1 ) {
@ -1394,8 +1394,11 @@ struct NotifiedInt {
set( v );
}
NotifiedInt(NotifiedInt&& r) BOOST_NOEXCEPT : waiting(std::move(r.waiting)), val(r.val) {}
void operator=(NotifiedInt&& r) BOOST_NOEXCEPT { waiting = std::move(r.waiting); val = r.val; }
NotifiedInt(NotifiedInt&& r) noexcept : waiting(std::move(r.waiting)), val(r.val) {}
void operator=(NotifiedInt&& r) noexcept {
waiting = std::move(r.waiting);
val = r.val;
}
private:
typedef std::pair<int64_t,Promise<Void>> Item;
@ -1416,7 +1419,7 @@ struct BoundedFlowLock : NonCopyable, public ReferenceCounted<BoundedFlowLock> {
int64_t permitNumber;
Releaser() : lock(nullptr), permitNumber(0) {}
Releaser( BoundedFlowLock* lock, int64_t permitNumber ) : lock(lock), permitNumber(permitNumber) {}
Releaser(Releaser&& r) BOOST_NOEXCEPT : lock(r.lock), permitNumber(r.permitNumber) { r.permitNumber = 0; }
Releaser(Releaser&& r) noexcept : lock(r.lock), permitNumber(r.permitNumber) { r.permitNumber = 0; }
void operator=(Releaser&& r) { if (permitNumber) lock->release(permitNumber); lock = r.lock; permitNumber = r.permitNumber; r.permitNumber = 0; }
void release() {
@ -1592,9 +1595,7 @@ public:
futures = f.futures;
}
AndFuture(AndFuture&& f) BOOST_NOEXCEPT {
futures = std::move(f.futures);
}
AndFuture(AndFuture&& f) noexcept { futures = std::move(f.futures); }
AndFuture(Future<Void> const& f) {
futures.push_back(f);
@ -1612,9 +1613,7 @@ public:
futures = f.futures;
}
void operator=(AndFuture&& f) BOOST_NOEXCEPT {
futures = std::move(f.futures);
}
void operator=(AndFuture&& f) noexcept { futures = std::move(f.futures); }
void operator=(Future<Void> const& f) {
futures.push_back(f);

View File

@ -110,6 +110,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES fast/BackupCorrectnessClean.txt)
add_fdb_test(TEST_FILES fast/BackupToDBCorrectness.txt)
add_fdb_test(TEST_FILES fast/BackupToDBCorrectnessClean.txt)
add_fdb_test(TEST_FILES fast/CacheTest.txt)
add_fdb_test(TEST_FILES fast/CloggedSideband.txt)
add_fdb_test(TEST_FILES fast/ConfigureLocked.txt)
add_fdb_test(TEST_FILES fast/ConstrainedRandomSelector.txt)

11
tests/fast/CacheTest.txt Normal file
View File

@ -0,0 +1,11 @@
testTitle=Cached
testName=Cache
keyPrefix=foo/
testTitle=Cycle
testName=Cycle
transactionsPerSecond=2500.0
testDuration=10.0
expectedRate=0.80
keyPrefix=foo/