Merge pull request #2045 from atn34/disallow-scalar-network-messages

Disallow scalar network messages
This commit is contained in:
Evan Tschannen 2019-08-30 13:38:54 -07:00 committed by GitHub
commit a7237c4302
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 229 additions and 146 deletions

View File

@ -1474,11 +1474,11 @@ ACTOR Future<Void> watchValue(Future<Version> version, Key key, Optional<Value>
g_traceBatch.addAttach("WatchValueAttachID", info.debugID.get().first(), watchValueID.get().first());
g_traceBatch.addEvent("WatchValueDebug", watchValueID.get().first(), "NativeAPI.watchValue.Before"); //.detail("TaskID", g_network->getCurrentTask());
}
state Version resp;
state WatchValueReply resp;
choose {
when(Version r = wait(loadBalance(ssi.second, &StorageServerInterface::watchValue,
WatchValueRequest(key, value, ver, watchValueID),
TaskPriority::DefaultPromiseEndpoint))) {
when(WatchValueReply r = wait(loadBalance(ssi.second, &StorageServerInterface::watchValue,
WatchValueRequest(key, value, ver, watchValueID),
TaskPriority::DefaultPromiseEndpoint))) {
resp = r;
}
when(wait(cx->connectionFile ? cx->connectionFile->onChange() : Never())) { wait(Never()); }
@ -1489,12 +1489,13 @@ ACTOR Future<Void> watchValue(Future<Version> version, Key key, Optional<Value>
//FIXME: wait for known committed version on the storage server before replying,
//cannot do this until the storage server is notified on knownCommittedVersion changes from tlog (faster than the current update loop)
Version v = wait( waitForCommittedVersion( cx, resp ) );
Version v = wait(waitForCommittedVersion(cx, resp.version));
//TraceEvent("WatcherCommitted").detail("CommittedVersion", v).detail("WatchVersion", resp).detail("Key", key ).detail("Value", value);
//TraceEvent("WatcherCommitted").detail("CommittedVersion", v).detail("WatchVersion", resp.version).detail("Key", key ).detail("Value", value);
if( v - resp < 50000000 ) // False if there is a master failure between getting the response and getting the committed version, Dependent on SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT
return Void();
// False if there is a master failure between getting the response and getting the committed version,
// Dependent on SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT
if (v - resp.version < 50000000) return Void();
ver = v;
} catch (Error& e) {
if (e.code() == error_code_wrong_shard_server || e.code() == error_code_all_alternatives_failed) {

View File

@ -30,6 +30,20 @@
#include "flow/Stats.h"
#include "fdbrpc/TimedRequest.h"
// Dead code, removed in the next protocol version
struct VersionReply {
constexpr static FileIdentifier file_identifier = 3;
Version version;
VersionReply() = default;
explicit VersionReply(Version version) : version(version) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version);
}
};
struct StorageServerInterface {
constexpr static FileIdentifier file_identifier = 15302073;
enum { BUSY_ALLOWED = 0, BUSY_FORCE = 1, BUSY_LOCAL = 2 };
@ -40,7 +54,7 @@ struct StorageServerInterface {
LocalityData locality;
UID uniqueID;
RequestStream<ReplyPromise<Version>> getVersion;
RequestStream<ReplyPromise<VersionReply>> getVersion;
RequestStream<struct GetValueRequest> getValue;
RequestStream<struct GetKeyRequest> getKey;
@ -140,14 +154,27 @@ struct GetValueRequest : TimedRequest {
}
};
struct WatchValueReply {
constexpr static FileIdentifier file_identifier = 3;
Version version;
WatchValueReply() = default;
explicit WatchValueReply(Version version) : version(version) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version);
}
};
struct WatchValueRequest {
constexpr static FileIdentifier file_identifier = 14747733;
Key key;
Optional<Value> value;
Version version;
Optional<UID> debugID;
ReplyPromise< Version > reply;
ReplyPromise<WatchValueReply> reply;
WatchValueRequest(){}
WatchValueRequest(const Key& key, Optional<Value> value, Version ver, Optional<UID> debugID) : key(key), value(value), version(ver), debugID(debugID) {}
@ -219,6 +246,20 @@ struct GetKeyRequest : TimedRequest {
}
};
struct GetShardStateReply {
constexpr static FileIdentifier file_identifier = 0;
Version first;
Version second;
GetShardStateReply() = default;
GetShardStateReply(Version first, Version second) : first(first), second(second) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, first, second);
}
};
struct GetShardStateRequest {
constexpr static FileIdentifier file_identifier = 15860168;
enum waitMode {
@ -229,7 +270,7 @@ struct GetShardStateRequest {
KeyRange keys;
int32_t mode;
ReplyPromise< std::pair<Version,Version> > reply;
ReplyPromise<GetShardStateReply> reply;
GetShardStateRequest() {}
GetShardStateRequest( KeyRange const& keys, waitMode mode ) : keys(keys), mode(mode) {}

View File

@ -268,6 +268,20 @@ TEST_CASE("/flow/flow/cancel2")
return Void();
}
namespace {
// Simple message for flatbuffers unittests
struct Int {
constexpr static FileIdentifier file_identifier = 12345;
uint32_t value;
Int() = default;
Int(uint32_t value) : value(value) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, value);
}
};
} // namespace
TEST_CASE("/flow/flow/nonserializable futures")
{
// Types no longer need to be statically serializable to make futures, promises, actors
@ -283,20 +297,20 @@ TEST_CASE("/flow/flow/nonserializable futures")
// ReplyPromise can be used like a normal promise
{
ReplyPromise<int> rpInt;
Future<int> f = rpInt.getFuture();
ReplyPromise<Int> rpInt;
Future<Int> f = rpInt.getFuture();
ASSERT(!f.isReady());
rpInt.send(123);
ASSERT(f.get() == 123);
ASSERT(f.get().value == 123);
}
{
RequestStream<int> rsInt;
FutureStream<int> f = rsInt.getFuture();
RequestStream<Int> rsInt;
FutureStream<Int> f = rsInt.getFuture();
rsInt.send(1);
rsInt.send(2);
ASSERT(f.pop() == 1);
ASSERT(f.pop() == 2);
ASSERT(f.pop().value == 1);
ASSERT(f.pop().value == 2);
}
return Void();
@ -306,14 +320,14 @@ TEST_CASE("/flow/flow/networked futures")
{
// RequestStream can be serialized
{
RequestStream<int> locInt;
RequestStream<Int> locInt;
BinaryWriter wr(IncludeVersion());
wr << locInt;
ASSERT(locInt.getEndpoint().isValid() && locInt.getEndpoint().isLocal() && locInt.getEndpoint().getPrimaryAddress() == FlowTransport::transport().getLocalAddress());
BinaryReader rd(wr.toValue(), IncludeVersion());
RequestStream<int> remoteInt;
RequestStream<Int> remoteInt;
rd >> remoteInt;
ASSERT(remoteInt.getEndpoint() == locInt.getEndpoint());
@ -323,14 +337,14 @@ TEST_CASE("/flow/flow/networked futures")
// ReplyPromise can be serialized
// TODO: This needs to fiddle with g_currentDeliveryPeerAddress
if (0) {
ReplyPromise<int> locInt;
ReplyPromise<Int> locInt;
BinaryWriter wr(IncludeVersion());
wr << locInt;
ASSERT(locInt.getEndpoint().isValid() && locInt.getEndpoint().isLocal());
BinaryReader rd(wr.toValue(), IncludeVersion());
ReplyPromise<int> remoteInt;
ReplyPromise<Int> remoteInt;
rd >> remoteInt;
ASSERT(remoteInt.getEndpoint() == locInt.getEndpoint());

View File

@ -302,7 +302,8 @@ ACTOR Future<Void> leaderRegister(LeaderElectionRegInterface interf, Key key) {
//TODO: use notify to only send a heartbeat once per interval
availableLeaders.erase( LeaderInfo(req.prevChangeID) );
availableLeaders.insert( req.myInfo );
req.reply.send( currentNominee.present() && currentNominee.get().equalInternalId(req.myInfo) );
req.reply.send(
LeaderHeartbeatReply{ currentNominee.present() && currentNominee.get().equalInternalId(req.myInfo) });
}
when (ForwardRequest req = waitNext( interf.forward.getFuture() ) ) {
LeaderInfo newInfo;
@ -499,7 +500,7 @@ ACTOR Future<Void> leaderServer(LeaderElectionRegInterface interf, OnDemandStore
when ( LeaderHeartbeatRequest req = waitNext( interf.leaderHeartbeat.getFuture() ) ) {
Optional<LeaderInfo> forward = regs.getForward(req.key);
if( forward.present() )
req.reply.send( false );
req.reply.send(LeaderHeartbeatReply{ false });
else
regs.getInterface(req.key, id).leaderHeartbeat.send(req);
}

View File

@ -136,12 +136,29 @@ struct CandidacyRequest {
}
};
struct LeaderHeartbeatReply {
constexpr static FileIdentifier file_identifier = 11;
bool value = false;
LeaderHeartbeatReply() = default;
explicit LeaderHeartbeatReply(bool value) : value(value) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, value);
}
};
inline bool operator==(const LeaderHeartbeatReply& lhs, const LeaderHeartbeatReply& rhs) {
return lhs.value == rhs.value;
}
struct LeaderHeartbeatRequest {
constexpr static FileIdentifier file_identifier = 9495992;
Key key;
LeaderInfo myInfo;
UID prevChangeID;
ReplyPromise<bool> reply;
ReplyPromise<LeaderHeartbeatReply> reply;
LeaderHeartbeatRequest() {}
explicit LeaderHeartbeatRequest( Key key, LeaderInfo const& myInfo, UID prevChangeID ) : key(key), myInfo(myInfo), prevChangeID(prevChangeID) {}

View File

@ -183,9 +183,11 @@ ACTOR Future<Void> tryBecomeLeaderInternal(ServerCoordinators coordinators, Valu
state vector<Future<Void>> true_heartbeats;
state vector<Future<Void>> false_heartbeats;
for(int i=0; i<coordinators.leaderElectionServers.size(); i++) {
Future<bool> hb = retryBrokenPromise( coordinators.leaderElectionServers[i].leaderHeartbeat, LeaderHeartbeatRequest( coordinators.clusterKey, myInfo, prevChangeID ), TaskPriority::CoordinationReply );
true_heartbeats.push_back( onEqual(hb, true) );
false_heartbeats.push_back( onEqual(hb, false) );
Future<LeaderHeartbeatReply> hb = retryBrokenPromise(
coordinators.leaderElectionServers[i].leaderHeartbeat,
LeaderHeartbeatRequest(coordinators.clusterKey, myInfo, prevChangeID), TaskPriority::CoordinationReply);
true_heartbeats.push_back(onEqual(hb, LeaderHeartbeatReply{ true }));
false_heartbeats.push_back(onEqual(hb, LeaderHeartbeatReply{ false }));
}
state Future<Void> rate = delay( SERVER_KNOBS->HEARTBEAT_FREQUENCY, TaskPriority::CoordinationReply ) || asyncPriorityInfo->onChange(); // SOMEDAY: Move to server side?

View File

@ -55,10 +55,25 @@ struct MasterInterface {
}
};
struct TLogRejoinReply {
constexpr static FileIdentifier file_identifier = 11;
// false means someone else registered, so we should re-register. true means this master is recovered, so don't
// send again to the same master.
bool masterIsRecovered;
TLogRejoinReply() = default;
explicit TLogRejoinReply(bool masterIsRecovered) : masterIsRecovered(masterIsRecovered) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, masterIsRecovered);
}
};
struct TLogRejoinRequest {
constexpr static FileIdentifier file_identifier = 15692200;
TLogInterface myInterface;
ReplyPromise<bool> reply; // false means someone else registered, so we should re-register. true means this master is recovered, so don't send again to the same master.
ReplyPromise<TLogRejoinReply> reply;
TLogRejoinRequest() { }
explicit TLogRejoinRequest(const TLogInterface &interf) : myInterface(interf) { }

View File

@ -139,8 +139,8 @@ Future<Void> checkMoveKeysLockReadOnly( Transaction* tr, MoveKeysLock lock ) {
return checkMoveKeysLock(tr, lock, false);
}
ACTOR Future<Optional<UID>> checkReadWrite( Future< ErrorOr<std::pair<Version,Version>> > fReply, UID uid, Version version ) {
ErrorOr<std::pair<Version,Version>> reply = wait( fReply );
ACTOR Future<Optional<UID>> checkReadWrite(Future<ErrorOr<GetShardStateReply>> fReply, UID uid, Version version) {
ErrorOr<GetShardStateReply> reply = wait(fReply);
if (!reply.present() || reply.get().first < version)
return Optional<UID>();
return Optional<UID>(uid);
@ -443,7 +443,8 @@ ACTOR Future<Void> startMoveKeys( Database occ, KeyRange keys, vector<UID> serve
ACTOR Future<Void> waitForShardReady( StorageServerInterface server, KeyRange keys, Version minVersion, GetShardStateRequest::waitMode mode ) {
loop {
try {
std::pair<Version,Version> rep = wait( server.getShardState.getReply( GetShardStateRequest(keys, mode), TaskPriority::MoveKeys ) );
GetShardStateReply rep =
wait(server.getShardState.getReply(GetShardStateRequest(keys, mode), TaskPriority::MoveKeys));
if (rep.first >= minVersion) {
return Void();
}

View File

@ -1119,11 +1119,11 @@ namespace oldTLog_4_6 {
req.myInterface = tli;
TraceEvent("TLogRejoining", self->dbgid).detail("Master", self->dbInfo->get().master.id());
choose {
when ( bool success = wait( brokenPromiseToNever( self->dbInfo->get().master.tlogRejoin.getReply( req ) ) ) ) {
if (success)
lastMasterID = self->dbInfo->get().master.id();
}
when ( wait( self->dbInfo->onChange() ) ) { }
when(TLogRejoinReply rep =
wait(brokenPromiseToNever(self->dbInfo->get().master.tlogRejoin.getReply(req)))) {
if (rep.masterIsRecovered) lastMasterID = self->dbInfo->get().master.id();
}
when ( wait( self->dbInfo->onChange() ) ) { }
}
} else {
wait( self->dbInfo->onChange() );

View File

@ -1477,9 +1477,9 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
TLogRejoinRequest req(tli);
TraceEvent("TLogRejoining", self->dbgid).detail("Master", self->dbInfo->get().master.id());
choose {
when ( bool success = wait( brokenPromiseToNever( self->dbInfo->get().master.tlogRejoin.getReply( req ) ) ) ) {
if (success)
lastMasterID = self->dbInfo->get().master.id();
when(TLogRejoinReply rep =
wait(brokenPromiseToNever(self->dbInfo->get().master.tlogRejoin.getReply(req)))) {
if (rep.masterIsRecovered) lastMasterID = self->dbInfo->get().master.id();
}
when ( wait( self->dbInfo->onChange() ) ) { }
}

View File

@ -103,9 +103,22 @@ struct ResolveTransactionBatchRequest {
}
};
struct ResolutionMetricsReply {
constexpr static FileIdentifier file_identifier = 3;
int64_t value;
ResolutionMetricsReply() = default;
explicit ResolutionMetricsReply(int64_t value) : value(value) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, value);
}
};
struct ResolutionMetricsRequest {
constexpr static FileIdentifier file_identifier = 11663527;
ReplyPromise<int64_t> reply;
ReplyPromise<ResolutionMetricsReply> reply;
template <class Archive>
void serialize(Archive& ar) {

View File

@ -216,6 +216,19 @@ struct TagMessagesRef {
}
};
struct TLogCommitReply {
constexpr static FileIdentifier file_identifier = 3;
Version version;
TLogCommitReply() = default;
explicit TLogCommitReply(Version version) : version(version) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, version);
}
};
struct TLogCommitRequest {
constexpr static FileIdentifier file_identifier = 4022206;
Arena arena;
@ -223,7 +236,7 @@ struct TLogCommitRequest {
StringRef messages;// Each message prefixed by a 4-byte length
ReplyPromise<Version> reply;
ReplyPromise<TLogCommitReply> reply;
Optional<UID> debugID;
TLogCommitRequest() {}

View File

@ -1853,9 +1853,9 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
TLogRejoinRequest req(tli);
TraceEvent("TLogRejoining", self->dbgid).detail("Master", self->dbInfo->get().master.id());
choose {
when ( bool success = wait( brokenPromiseToNever( self->dbInfo->get().master.tlogRejoin.getReply( req ) ) ) ) {
if (success)
lastMasterID = self->dbInfo->get().master.id();
when(TLogRejoinReply rep =
wait(brokenPromiseToNever(self->dbInfo->get().master.tlogRejoin.getReply(req)))) {
if (rep.masterIsRecovered) lastMasterID = self->dbInfo->get().master.id();
}
when ( wait( self->dbInfo->onChange() ) ) { }
}

View File

@ -30,12 +30,12 @@
#include "fdbserver/RecoveryState.h"
#include "flow/actorcompiler.h" // This must be the last #include.
ACTOR Future<Version> minVersionWhenReady( Future<Void> f, std::vector<Future<Version>> replies) {
ACTOR Future<Version> minVersionWhenReady(Future<Void> f, std::vector<Future<TLogCommitReply>> replies) {
wait(f);
Version minVersion = std::numeric_limits<Version>::max();
for(auto& reply : replies) {
if(reply.isReady() && !reply.isError()) {
minVersion = std::min(minVersion, reply.get());
minVersion = std::min(minVersion, reply.get().version);
}
}
return minVersion;
@ -429,7 +429,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
virtual Future<Version> push( Version prevVersion, Version version, Version knownCommittedVersion, Version minKnownCommittedVersion, LogPushData& data, Optional<UID> debugID ) {
// FIXME: Randomize request order as in LegacyLogSystem?
vector<Future<Void>> quorumResults;
vector<Future<Version>> allReplies;
vector<Future<TLogCommitReply>> allReplies;
int location = 0;
for(auto& it : tLogs) {
if(it->isLocal && it->logServers.size()) {
@ -2271,7 +2271,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
ACTOR static Future<Void> trackRejoins( UID dbgid, std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> logServers, FutureStream< struct TLogRejoinRequest > rejoinRequests ) {
state std::map<UID,ReplyPromise<bool>> lastReply;
state std::map<UID, ReplyPromise<TLogRejoinReply>> lastReply;
try {
loop {
@ -2287,7 +2287,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
TraceEvent("TLogJoinedMe", dbgid).detail("TLog", req.myInterface.id()).detail("Address", req.myInterface.commit.getEndpoint().getPrimaryAddress().toString());
if( !logServers[pos]->get().present() || req.myInterface.commit.getEndpoint() != logServers[pos]->get().interf().commit.getEndpoint())
logServers[pos]->setUnconditional( OptionalInterface<TLogInterface>(req.myInterface) );
lastReply[req.myInterface.id()].send(false);
lastReply[req.myInterface.id()].send(TLogRejoinReply{ false });
lastReply[req.myInterface.id()] = req.reply;
}
else {
@ -2296,8 +2296,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
} catch (...) {
for( auto it = lastReply.begin(); it != lastReply.end(); ++it)
it->second.send(true);
for (auto it = lastReply.begin(); it != lastReply.end(); ++it) it->second.send(TLogRejoinReply{ true });
throw;
}
}

View File

@ -31,12 +31,22 @@
#include "fdbrpc/PerfMetric.h"
#include "fdbclient/NativeAPI.actor.h"
#include "flow/actorcompiler.h" // has to be last include
struct CheckReply {
constexpr static FileIdentifier file_identifier = 11;
bool value = false;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, value);
}
};
struct WorkloadInterface {
constexpr static FileIdentifier file_identifier = 4454551;
RequestStream<ReplyPromise<Void>> setup;
RequestStream<ReplyPromise<Void>> start;
RequestStream<ReplyPromise<bool>> check;
RequestStream<ReplyPromise<CheckReply>> check;
RequestStream<ReplyPromise< std::vector<PerfMetric> > > metrics;
RequestStream<ReplyPromise<Void>> stop;

View File

@ -1018,7 +1018,7 @@ ACTOR Future<Void> resolutionBalancing(Reference<MasterData> self) {
wait(delay(SERVER_KNOBS->MIN_BALANCE_TIME, TaskPriority::ResolutionMetrics));
while(self->resolverChanges.get().size())
wait(self->resolverChanges.onChange());
state std::vector<Future<int64_t>> futures;
state std::vector<Future<ResolutionMetricsReply>> futures;
for (auto& p : self->resolvers)
futures.push_back(brokenPromiseToNever(p.metrics.getReply(ResolutionMetricsRequest(), TaskPriority::ResolutionMetrics)));
wait( waitForAll(futures) );
@ -1026,8 +1026,8 @@ ACTOR Future<Void> resolutionBalancing(Reference<MasterData> self) {
int64_t total = 0;
for (int i = 0; i < futures.size(); i++) {
total += futures[i].get();
metrics.insert(std::make_pair(futures[i].get(), i), NoMetric());
total += futures[i].get().value;
metrics.insert(std::make_pair(futures[i].get().value, i), NoMetric());
//TraceEvent("ResolverMetric").detail("I", i).detail("Metric", futures[i].get());
}
if( metrics.lastItem()->first - metrics.begin()->first > SERVER_KNOBS->MIN_BALANCE_DIFFERENCE ) {

View File

@ -934,7 +934,7 @@ ACTOR Future<Void> watchValue_impl( StorageServer* data, WatchValueRequest req )
g_traceBatch.addEvent("WatchValueDebug", req.debugID.get().first(), "watchValueQ.AfterRead"); //.detail("TaskID", g_network->getCurrentTask());
if( reply.value != req.value ) {
req.reply.send( latest );
req.reply.send(WatchValueReply{ latest });
return Void();
}
@ -1012,7 +1012,7 @@ ACTOR Future<Void> getShardState_impl( StorageServer* data, GetShardStateRequest
}
if( !onChange.size() ) {
req.reply.send(std::make_pair(data->version.get(), data->durableVersion.get()));
req.reply.send(GetShardStateReply{ data->version.get(), data->durableVersion.get() });
return Void();
}
@ -3533,7 +3533,7 @@ ACTOR Future<Void> storageServerCore( StorageServer* self, StorageServerInterfac
when (GetShardStateRequest req = waitNext(ssi.getShardState.getFuture()) ) {
if (req.mode == GetShardStateRequest::NO_WAIT ) {
if( self->isReadable( req.keys ) )
req.reply.send(std::make_pair(self->version.get(),self->durableVersion.get()));
req.reply.send(GetShardStateReply{ self->version.get(), self->durableVersion.get() });
else
req.reply.sendError(wrong_shard_server());
} else {
@ -3543,7 +3543,7 @@ ACTOR Future<Void> storageServerCore( StorageServer* self, StorageServerInterfac
when (StorageQueuingMetricsRequest req = waitNext(ssi.getQueuingMetrics.getFuture())) {
getQueuingMetrics(self, req);
}
when( ReplyPromise<Version> reply = waitNext(ssi.getVersion.getFuture()) ) {
when(ReplyPromise<VersionReply> reply = waitNext(ssi.getVersion.getFuture())) {
reply.send( self->version.get() );
}
when( ReplyPromise<KeyValueStoreType> reply = waitNext(ssi.getKeyValueStoreType.getFuture()) ) {

View File

@ -404,10 +404,10 @@ ACTOR Future<Void> runWorkloadAsync( Database cx, WorkloadInterface workIface, T
state unique_ptr<TestWorkload> delw(workload);
state Optional<ErrorOr<Void>> setupResult;
state Optional<ErrorOr<Void>> startResult;
state Optional<ErrorOr<bool>> checkResult;
state Optional<ErrorOr<CheckReply>> checkResult;
state ReplyPromise<Void> setupReq;
state ReplyPromise<Void> startReq;
state ReplyPromise<bool> checkReq;
state ReplyPromise<CheckReply> checkReq;
TraceEvent("TestBeginAsync", workIface.id()).detail("Workload", workload->description()).detail("DatabasePingDelay", databasePingDelay);
@ -452,12 +452,12 @@ ACTOR Future<Void> runWorkloadAsync( Database cx, WorkloadInterface workIface, T
}
sendResult( startReq, startResult );
}
when( ReplyPromise<bool> req = waitNext( workIface.check.getFuture() ) ) {
when(ReplyPromise<CheckReply> req = waitNext(workIface.check.getFuture())) {
checkReq = req;
if (!checkResult.present()) {
try {
bool check = wait( timeoutError( workload->check(cx), workload->getCheckTimeout() ) );
checkResult = (!startResult.present() || !startResult.get().isError()) && check;
checkResult = CheckReply{ (!startResult.present() || !startResult.get().isError()) && check };
} catch (Error& e) {
checkResult = operation_failed(); // was: checkResult = false;
if( e.code() == error_code_please_reboot || e.code() == error_code_please_reboot_delete) throw;
@ -693,16 +693,16 @@ ACTOR Future<DistributedTestResults> runWorkload( Database cx, std::vector< Test
wait( delay(3.0) );
}
state std::vector< Future<ErrorOr<bool>> > checks;
state std::vector<Future<ErrorOr<CheckReply>>> checks;
TraceEvent("CheckingResults");
printf("checking test (%s)...\n", printable(spec.title).c_str());
for(int i= 0; i < workloads.size(); i++)
checks.push_back( workloads[i].check.template getReplyUnlessFailedFor<bool>(waitForFailureTime, 0) );
checks.push_back(workloads[i].check.template getReplyUnlessFailedFor<CheckReply>(waitForFailureTime, 0));
wait( waitForAll( checks ) );
throwIfError(checks, "CheckFailedForWorkload" + printable(spec.title));
for(int i = 0; i < checks.size(); i++) {
if(checks[i].get().get())
if (checks[i].get().get().value)
success++;
else
failure++;

View File

@ -72,68 +72,3 @@ template <class T, uint32_t B>
struct ComposedIdentifierExternal<T, B, true> {
static constexpr FileIdentifier value = ComposedIdentifier<T, B>::file_identifier;
};
template <>
struct FileIdentifierFor<int> {
constexpr static FileIdentifier value = 1;
};
template <>
struct FileIdentifierFor<unsigned> {
constexpr static FileIdentifier value = 2;
};
template <>
struct FileIdentifierFor<long> {
constexpr static FileIdentifier value = 3;
};
template <>
struct FileIdentifierFor<unsigned long> {
constexpr static FileIdentifier value = 4;
};
template <>
struct FileIdentifierFor<long long> {
constexpr static FileIdentifier value = 5;
};
template <>
struct FileIdentifierFor<unsigned long long> {
constexpr static FileIdentifier value = 6;
};
template <>
struct FileIdentifierFor<short> {
constexpr static FileIdentifier value = 7;
};
template <>
struct FileIdentifierFor<unsigned short> {
constexpr static FileIdentifier value = 8;
};
template <>
struct FileIdentifierFor<signed char> {
constexpr static FileIdentifier value = 9;
};
template <>
struct FileIdentifierFor<unsigned char> {
constexpr static FileIdentifier value = 10;
};
template <>
struct FileIdentifierFor<bool> {
constexpr static FileIdentifier value = 11;
};
template <>
struct FileIdentifierFor<float> {
constexpr static FileIdentifier value = 7266212;
};
template <>
struct FileIdentifierFor<double> {
constexpr static FileIdentifier value = 9348150;
};

View File

@ -1140,12 +1140,19 @@ inline FileIdentifier read_file_identifier(const uint8_t* in) {
return result;
}
namespace detail {
template <class T>
struct YesFileIdentifier {
constexpr static FileIdentifier file_identifier = FileIdentifierFor<T>::value;
};
struct NoFileIdentifier {};
}; // namespace detail
// members of unions must be tables in flatbuffers, so you can use this to
// introduce the indirection only when necessary.
template <class T>
struct EnsureTable {
static_assert(HasFileIdentifier<T>::value);
constexpr static FileIdentifier file_identifier = FileIdentifierFor<T>::value;
struct EnsureTable
: std::conditional_t<HasFileIdentifier<T>::value, detail::YesFileIdentifier<T>, detail::NoFileIdentifier> {
EnsureTable() = default;
EnsureTable(const T& t) : t(t) {}
template <class Archive>

View File

@ -249,10 +249,24 @@ void enableBuggify(bool enabled, BuggifyType type) {
buggifyActivated[int(type)] = enabled;
}
namespace {
// Simple message for flatbuffers unittests
struct Int {
constexpr static FileIdentifier file_identifier = 12345;
uint32_t value;
Int() = default;
Int(uint32_t value) : value(value) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, value);
}
};
} // namespace
TEST_CASE("/flow/FlatBuffers/ErrorOr") {
{
ErrorOr<int> in(worker_removed());
ErrorOr<int> out;
ErrorOr<Int> in(worker_removed());
ErrorOr<Int> out;
ObjectWriter writer(Unversioned());
writer.serialize(in);
Standalone<StringRef> copy = writer.toStringRef();
@ -262,23 +276,23 @@ TEST_CASE("/flow/FlatBuffers/ErrorOr") {
ASSERT(out.getError().code() == in.getError().code());
}
{
ErrorOr<uint32_t> in(deterministicRandom()->randomUInt32());
ErrorOr<uint32_t> out;
ErrorOr<Int> in(deterministicRandom()->randomUInt32());
ErrorOr<Int> out;
ObjectWriter writer(Unversioned());
writer.serialize(in);
Standalone<StringRef> copy = writer.toStringRef();
ArenaObjectReader reader(copy.arena(), copy, Unversioned());
reader.deserialize(out);
ASSERT(!out.isError());
ASSERT(out.get() == in.get());
ASSERT(out.get().value == in.get().value);
}
return Void();
}
TEST_CASE("/flow/FlatBuffers/Optional") {
{
Optional<int> in;
Optional<int> out;
Optional<Int> in;
Optional<Int> out;
ObjectWriter writer(Unversioned());
writer.serialize(in);
Standalone<StringRef> copy = writer.toStringRef();
@ -287,15 +301,15 @@ TEST_CASE("/flow/FlatBuffers/Optional") {
ASSERT(!out.present());
}
{
Optional<uint32_t> in(deterministicRandom()->randomUInt32());
Optional<uint32_t> out;
Optional<Int> in(deterministicRandom()->randomUInt32());
Optional<Int> out;
ObjectWriter writer(Unversioned());
writer.serialize(in);
Standalone<StringRef> copy = writer.toStringRef();
ArenaObjectReader reader(copy.arena(), copy, Unversioned());
reader.deserialize(out);
ASSERT(out.present());
ASSERT(out.get() == in.get());
ASSERT(out.get().value == in.get().value);
}
return Void();
}