Merge pull request #9851 from sfc-gh-ajbeamon/improve-boolean-param

Allow boolean parameters to be nested inside of namespaces or classes
This commit is contained in:
A.J. Beamon 2023-03-30 16:06:03 -07:00 committed by GitHub
commit 3164cadc6f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
55 changed files with 119 additions and 264 deletions

View File

@ -37,24 +37,6 @@
#include "flow/actorcompiler.h" // has to be last include
#include "flow/network.h"
FDB_DEFINE_BOOLEAN_PARAM(LockDB);
FDB_DEFINE_BOOLEAN_PARAM(UnlockDB);
FDB_DEFINE_BOOLEAN_PARAM(StopWhenDone);
FDB_DEFINE_BOOLEAN_PARAM(Verbose);
FDB_DEFINE_BOOLEAN_PARAM(WaitForComplete);
FDB_DEFINE_BOOLEAN_PARAM(ForceAction);
FDB_DEFINE_BOOLEAN_PARAM(Terminator);
FDB_DEFINE_BOOLEAN_PARAM(UsePartitionedLog);
FDB_DEFINE_BOOLEAN_PARAM(InconsistentSnapshotOnly);
FDB_DEFINE_BOOLEAN_PARAM(ShowErrors);
FDB_DEFINE_BOOLEAN_PARAM(AbortOldBackup);
FDB_DEFINE_BOOLEAN_PARAM(DstOnly);
FDB_DEFINE_BOOLEAN_PARAM(WaitForDestUID);
FDB_DEFINE_BOOLEAN_PARAM(CheckBackupUID);
FDB_DEFINE_BOOLEAN_PARAM(DeleteData);
FDB_DEFINE_BOOLEAN_PARAM(SetValidation);
FDB_DEFINE_BOOLEAN_PARAM(PartialBackup);
std::string BackupAgentBase::formatTime(int64_t epochs) {
time_t curTime = (time_t)epochs;
char buffer[30];

View File

@ -36,8 +36,6 @@
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(IncludeKeyRangeMap);
class BackupContainerFileSystemImpl {
public:
// TODO: Do this more efficiently, as the range file list for a snapshot could potentially be hundreds of

View File

@ -1,30 +0,0 @@
/*
* ClientBooleanParams.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/ClientBooleanParams.h"
FDB_DEFINE_BOOLEAN_PARAM(EnableLocalityLoadBalance);
FDB_DEFINE_BOOLEAN_PARAM(LockAware);
FDB_DEFINE_BOOLEAN_PARAM(Reverse);
FDB_DEFINE_BOOLEAN_PARAM(Snapshot);
FDB_DEFINE_BOOLEAN_PARAM(IsInternal);
FDB_DEFINE_BOOLEAN_PARAM(AddConflictRange);
FDB_DEFINE_BOOLEAN_PARAM(UseMetrics);
FDB_DEFINE_BOOLEAN_PARAM(IsSwitchable);

View File

@ -65,10 +65,6 @@
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(IncrementalBackupOnly);
FDB_DEFINE_BOOLEAN_PARAM(OnlyApplyMutationLogs);
FDB_DEFINE_BOOLEAN_PARAM(SnapshotBackupUseTenantCache);
#define SevFRTestInfo SevVerbose
// #define SevFRTestInfo SevInfo

View File

@ -23,19 +23,6 @@
#include "libb64/decode.h"
#include "libb64/encode.h"
FDB_DEFINE_BOOLEAN_PARAM(ApplyManagementClusterUpdates);
FDB_DEFINE_BOOLEAN_PARAM(RemoveMissingTenants);
FDB_DEFINE_BOOLEAN_PARAM(AssignClusterAutomatically);
FDB_DEFINE_BOOLEAN_PARAM(GroupAlreadyExists);
FDB_DEFINE_BOOLEAN_PARAM(IsRestoring);
FDB_DEFINE_BOOLEAN_PARAM(RunOnDisconnectedCluster);
FDB_DEFINE_BOOLEAN_PARAM(RunOnMismatchedCluster);
FDB_DEFINE_BOOLEAN_PARAM(RestoreDryRun);
FDB_DEFINE_BOOLEAN_PARAM(ForceJoin);
FDB_DEFINE_BOOLEAN_PARAM(ForceReuseTenantIdPrefix);
FDB_DEFINE_BOOLEAN_PARAM(ForceRemove);
FDB_DEFINE_BOOLEAN_PARAM(IgnoreCapacityLimit);
namespace MetaclusterAPI {
std::string tenantStateToString(TenantState tenantState) {

View File

@ -51,8 +51,6 @@ std::string trim(std::string const& connectionString) {
} // namespace
FDB_DEFINE_BOOLEAN_PARAM(ConnectionStringNeedsPersisted);
// Returns the connection string currently held in this object. This may not match the stored record if it hasn't
// been persisted or if the persistent storage for the record has been modified externally.
ClusterConnectionString& IClusterConnectionRecord::getConnectionString() {

View File

@ -114,8 +114,6 @@
template class RequestStream<OpenDatabaseRequest, false>;
template struct NetNotifiedQueue<OpenDatabaseRequest, false>;
FDB_DEFINE_BOOLEAN_PARAM(CacheResult);
extern const char* getSourceVersion();
namespace {
@ -148,10 +146,6 @@ Future<REPLY_TYPE(Request)> loadBalance(
} // namespace
FDB_BOOLEAN_PARAM(TransactionRecordLogInfo);
FDB_DEFINE_BOOLEAN_PARAM(UseProvisionalProxies);
// Used to determine whether or not client will load balance based on the number of GRVs released by each proxy
FDB_DEFINE_BOOLEAN_PARAM(BalanceOnRequests);
// Whether or not a request should include the tenant name
FDB_BOOLEAN_PARAM(UseTenant);
@ -3401,9 +3395,6 @@ Future<int64_t> DatabaseContext::lookupTenant(TenantName tenant) {
return lookupTenantImpl(this, tenant);
}
FDB_DEFINE_BOOLEAN_PARAM(AllowInvalidTenantID);
FDB_DEFINE_BOOLEAN_PARAM(ResolveDefaultTenant);
TransactionState::TransactionState(Database cx,
Optional<Reference<Tenant>> tenant,
TaskPriority taskID,

View File

@ -27,10 +27,6 @@
#include "flow/serialize.h"
#include "flow/UnitTest.h"
FDB_DEFINE_BOOLEAN_PARAM(AssignEmptyRange);
FDB_DEFINE_BOOLEAN_PARAM(UnassignShard);
FDB_DEFINE_BOOLEAN_PARAM(EnablePhysicalShardMove);
const KeyRef systemKeysPrefix = "\xff"_sr;
const KeyRangeRef normalKeys(KeyRef(), systemKeysPrefix);
const KeyRangeRef systemKeys(systemKeysPrefix, "\xff\xff"_sr);

View File

@ -165,9 +165,6 @@ ThrottleApi::TagQuotaValue ThrottleApi::TagQuotaValue::fromValue(ValueRef value)
return result;
}
FDB_DEFINE_BOOLEAN_PARAM(ContainsRecommended);
FDB_DEFINE_BOOLEAN_PARAM(Capitalize);
TEST_CASE("TagSet/toString") {
{
TagSet tagSet;

View File

@ -23,11 +23,6 @@
#include "fdbclient/ReadYourWrites.h"
#include "flow/actorcompiler.h" // has to be last include
FDB_DEFINE_BOOLEAN_PARAM(AccessSystemKeys);
FDB_DEFINE_BOOLEAN_PARAM(PriorityBatch);
FDB_DEFINE_BOOLEAN_PARAM(VerifyTask);
FDB_DEFINE_BOOLEAN_PARAM(UpdateParams);
Reference<TaskFuture> Task::getDoneFuture(Reference<FutureBucket> fb) {
return fb->unpack(params[reservedTaskParamKeyDone]);
}

View File

@ -29,8 +29,6 @@
#include "flow/ApiVersion.h"
#include "flow/UnitTest.h"
FDB_DEFINE_BOOLEAN_PARAM(EnforceValidTenantId);
namespace TenantAPI {
KeyRef idToPrefix(Arena& p, int64_t id) {

View File

@ -36,26 +36,26 @@
#include "fdbclient/BackupContainer.h"
#include "flow/actorcompiler.h" // has to be last include
FDB_DECLARE_BOOLEAN_PARAM(LockDB);
FDB_DECLARE_BOOLEAN_PARAM(UnlockDB);
FDB_DECLARE_BOOLEAN_PARAM(StopWhenDone);
FDB_DECLARE_BOOLEAN_PARAM(Verbose);
FDB_DECLARE_BOOLEAN_PARAM(WaitForComplete);
FDB_DECLARE_BOOLEAN_PARAM(ForceAction);
FDB_DECLARE_BOOLEAN_PARAM(Terminator);
FDB_DECLARE_BOOLEAN_PARAM(IncrementalBackupOnly);
FDB_DECLARE_BOOLEAN_PARAM(UsePartitionedLog);
FDB_DECLARE_BOOLEAN_PARAM(OnlyApplyMutationLogs);
FDB_DECLARE_BOOLEAN_PARAM(SnapshotBackupUseTenantCache);
FDB_DECLARE_BOOLEAN_PARAM(InconsistentSnapshotOnly);
FDB_DECLARE_BOOLEAN_PARAM(ShowErrors);
FDB_DECLARE_BOOLEAN_PARAM(AbortOldBackup);
FDB_DECLARE_BOOLEAN_PARAM(DstOnly); // TODO: More descriptive name?
FDB_DECLARE_BOOLEAN_PARAM(WaitForDestUID);
FDB_DECLARE_BOOLEAN_PARAM(CheckBackupUID);
FDB_DECLARE_BOOLEAN_PARAM(DeleteData);
FDB_DECLARE_BOOLEAN_PARAM(SetValidation);
FDB_DECLARE_BOOLEAN_PARAM(PartialBackup);
FDB_BOOLEAN_PARAM(LockDB);
FDB_BOOLEAN_PARAM(UnlockDB);
FDB_BOOLEAN_PARAM(StopWhenDone);
FDB_BOOLEAN_PARAM(Verbose);
FDB_BOOLEAN_PARAM(WaitForComplete);
FDB_BOOLEAN_PARAM(ForceAction);
FDB_BOOLEAN_PARAM(Terminator);
FDB_BOOLEAN_PARAM(IncrementalBackupOnly);
FDB_BOOLEAN_PARAM(UsePartitionedLog);
FDB_BOOLEAN_PARAM(OnlyApplyMutationLogs);
FDB_BOOLEAN_PARAM(SnapshotBackupUseTenantCache);
FDB_BOOLEAN_PARAM(InconsistentSnapshotOnly);
FDB_BOOLEAN_PARAM(ShowErrors);
FDB_BOOLEAN_PARAM(AbortOldBackup);
FDB_BOOLEAN_PARAM(DstOnly); // TODO: More descriptive name?
FDB_BOOLEAN_PARAM(WaitForDestUID);
FDB_BOOLEAN_PARAM(CheckBackupUID);
FDB_BOOLEAN_PARAM(DeleteData);
FDB_BOOLEAN_PARAM(SetValidation);
FDB_BOOLEAN_PARAM(PartialBackup);
class BackupAgentBase : NonCopyable {
public:

View File

@ -29,7 +29,7 @@
#include "fdbclient/ReadYourWrites.h"
#include <vector>
FDB_DECLARE_BOOLEAN_PARAM(IncludeKeyRangeMap);
FDB_BOOLEAN_PARAM(IncludeKeyRangeMap);
class ReadYourWritesTransaction;

View File

@ -22,11 +22,11 @@
#include "flow/BooleanParam.h"
FDB_DECLARE_BOOLEAN_PARAM(EnableLocalityLoadBalance);
FDB_DECLARE_BOOLEAN_PARAM(LockAware);
FDB_DECLARE_BOOLEAN_PARAM(Reverse);
FDB_DECLARE_BOOLEAN_PARAM(Snapshot);
FDB_DECLARE_BOOLEAN_PARAM(IsInternal);
FDB_DECLARE_BOOLEAN_PARAM(AddConflictRange);
FDB_DECLARE_BOOLEAN_PARAM(UseMetrics);
FDB_DECLARE_BOOLEAN_PARAM(IsSwitchable);
FDB_BOOLEAN_PARAM(EnableLocalityLoadBalance);
FDB_BOOLEAN_PARAM(LockAware);
FDB_BOOLEAN_PARAM(Reverse);
FDB_BOOLEAN_PARAM(Snapshot);
FDB_BOOLEAN_PARAM(IsInternal);
FDB_BOOLEAN_PARAM(AddConflictRange);
FDB_BOOLEAN_PARAM(UseMetrics);
FDB_BOOLEAN_PARAM(IsSwitchable);

View File

@ -26,8 +26,8 @@
#include "flow/Knobs.h"
#include "flow/flow.h"
FDB_DECLARE_BOOLEAN_PARAM(Randomize);
FDB_DECLARE_BOOLEAN_PARAM(IsSimulated);
FDB_BOOLEAN_PARAM(Randomize);
FDB_BOOLEAN_PARAM(IsSimulated);
class ClientKnobs : public KnobsImpl<ClientKnobs> {
public:

View File

@ -109,7 +109,7 @@ public:
}
};
FDB_DECLARE_BOOLEAN_PARAM(ConnectionStringNeedsPersisted);
FDB_BOOLEAN_PARAM(ConnectionStringNeedsPersisted);
// A record that stores the connection string used to connect to a cluster. This record can be updated when a cluster
// notifies a connected party that the connection string has changed.

View File

@ -1643,7 +1643,7 @@ struct StorageWiggleValue {
enum class ReadType { EAGER = 0, FETCH = 1, LOW = 2, NORMAL = 3, HIGH = 4, MIN = EAGER, MAX = HIGH };
FDB_DECLARE_BOOLEAN_PARAM(CacheResult);
FDB_BOOLEAN_PARAM(CacheResult);
// store options for storage engine read
// ReadType describes the usage and priority of the read

View File

@ -92,18 +92,18 @@ struct DataClusterMetadata {
}
};
FDB_DECLARE_BOOLEAN_PARAM(ApplyManagementClusterUpdates);
FDB_DECLARE_BOOLEAN_PARAM(RemoveMissingTenants);
FDB_DECLARE_BOOLEAN_PARAM(AssignClusterAutomatically);
FDB_DECLARE_BOOLEAN_PARAM(GroupAlreadyExists);
FDB_DECLARE_BOOLEAN_PARAM(IsRestoring);
FDB_DECLARE_BOOLEAN_PARAM(RunOnDisconnectedCluster);
FDB_DECLARE_BOOLEAN_PARAM(RunOnMismatchedCluster);
FDB_DECLARE_BOOLEAN_PARAM(RestoreDryRun);
FDB_DECLARE_BOOLEAN_PARAM(ForceJoin);
FDB_DECLARE_BOOLEAN_PARAM(ForceReuseTenantIdPrefix);
FDB_DECLARE_BOOLEAN_PARAM(ForceRemove);
FDB_DECLARE_BOOLEAN_PARAM(IgnoreCapacityLimit);
FDB_BOOLEAN_PARAM(ApplyManagementClusterUpdates);
FDB_BOOLEAN_PARAM(RemoveMissingTenants);
FDB_BOOLEAN_PARAM(AssignClusterAutomatically);
FDB_BOOLEAN_PARAM(GroupAlreadyExists);
FDB_BOOLEAN_PARAM(IsRestoring);
FDB_BOOLEAN_PARAM(RunOnDisconnectedCluster);
FDB_BOOLEAN_PARAM(RunOnMismatchedCluster);
FDB_BOOLEAN_PARAM(RestoreDryRun);
FDB_BOOLEAN_PARAM(ForceJoin);
FDB_BOOLEAN_PARAM(ForceReuseTenantIdPrefix);
FDB_BOOLEAN_PARAM(ForceRemove);
FDB_BOOLEAN_PARAM(IgnoreCapacityLimit);
namespace MetaclusterAPI {

View File

@ -47,7 +47,7 @@
(getSBVar(__FILE__, __LINE__, BuggifyType::Client) && deterministicRandom()->random01() < (x))
#define CLIENT_BUGGIFY CLIENT_BUGGIFY_WITH_PROB(P_BUGGIFIED_SECTION_FIRES[int(BuggifyType::Client)])
FDB_DECLARE_BOOLEAN_PARAM(UseProvisionalProxies);
FDB_BOOLEAN_PARAM(UseProvisionalProxies);
// Incomplete types that are reference counted
class DatabaseContext;
@ -266,8 +266,8 @@ struct Traceable<Tenant> : std::true_type {
static std::string toString(const Tenant& tenant) { return printable(tenant.description()); }
};
FDB_DECLARE_BOOLEAN_PARAM(AllowInvalidTenantID);
FDB_DECLARE_BOOLEAN_PARAM(ResolveDefaultTenant);
FDB_BOOLEAN_PARAM(AllowInvalidTenantID);
FDB_BOOLEAN_PARAM(ResolveDefaultTenant);
struct TransactionState : ReferenceCounted<TransactionState> {
Database cx;

View File

@ -33,7 +33,7 @@
#include "fdbclient/RunTransaction.actor.h"
#include "flow/actorcompiler.h"
FDB_DECLARE_BOOLEAN_PARAM(PrimaryRegion);
FDB_BOOLEAN_PARAM(PrimaryRegion);
struct StorageWiggleMetrics {
constexpr static FileIdentifier file_identifier = 4728961;

View File

@ -34,9 +34,9 @@
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wunused-variable"
FDB_DECLARE_BOOLEAN_PARAM(AssignEmptyRange);
FDB_DECLARE_BOOLEAN_PARAM(UnassignShard);
FDB_DECLARE_BOOLEAN_PARAM(EnablePhysicalShardMove);
FDB_BOOLEAN_PARAM(AssignEmptyRange);
FDB_BOOLEAN_PARAM(UnassignShard);
FDB_BOOLEAN_PARAM(EnablePhysicalShardMove);
struct RestoreLoaderInterface;
struct RestoreApplierInterface;

View File

@ -40,8 +40,8 @@
typedef StringRef TransactionTagRef;
typedef Standalone<TransactionTagRef> TransactionTag;
FDB_DECLARE_BOOLEAN_PARAM(ContainsRecommended);
FDB_DECLARE_BOOLEAN_PARAM(Capitalize);
FDB_BOOLEAN_PARAM(ContainsRecommended);
FDB_BOOLEAN_PARAM(Capitalize);
class TagSet {
public:

View File

@ -35,10 +35,10 @@
class FutureBucket;
class TaskFuture;
FDB_DECLARE_BOOLEAN_PARAM(AccessSystemKeys);
FDB_DECLARE_BOOLEAN_PARAM(PriorityBatch);
FDB_DECLARE_BOOLEAN_PARAM(VerifyTask);
FDB_DECLARE_BOOLEAN_PARAM(UpdateParams);
FDB_BOOLEAN_PARAM(AccessSystemKeys);
FDB_BOOLEAN_PARAM(PriorityBatch);
FDB_BOOLEAN_PARAM(VerifyTask);
FDB_BOOLEAN_PARAM(UpdateParams);
// A Task is a set of key=value parameters that constitute a unit of work for a TaskFunc to perform.
// The parameter keys are specific to the TaskFunc that the Task is for, except for a set of reserved

View File

@ -30,7 +30,7 @@
#include "flow/BooleanParam.h"
#include "flow/flat_buffers.h"
FDB_DECLARE_BOOLEAN_PARAM(EnforceValidTenantId);
FDB_BOOLEAN_PARAM(EnforceValidTenantId);
namespace TenantAPI {
KeyRef idToPrefix(Arena& p, int64_t id);

View File

@ -22,9 +22,6 @@
#include "flow/flow.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(AtMostOnce);
FDB_DEFINE_BOOLEAN_PARAM(TriedAllOptions);
// Throwing all_alternatives_failed will cause the client to issue a GetKeyLocationRequest to the proxy, so this actor
// attempts to limit the number of these errors thrown by a single client to prevent it from saturating the proxies with
// these requests

View File

@ -1,23 +0,0 @@
#/*
* PerfMetric.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbrpc/PerfMetric.h"
FDB_DEFINE_BOOLEAN_PARAM(Averaged);

View File

@ -174,8 +174,6 @@ void LatencyBands::insertBand(double value) {
bands.emplace(std::make_pair(value, std::make_unique<Counter>(format("Band%f", value), *cc)));
}
FDB_DEFINE_BOOLEAN_PARAM(Filtered);
LatencyBands::LatencyBands(std::string const& name,
UID id,
double loggingInterval,

View File

@ -240,8 +240,8 @@ Future<Void> tssComparison(Req req,
return Void();
}
FDB_DECLARE_BOOLEAN_PARAM(AtMostOnce);
FDB_DECLARE_BOOLEAN_PARAM(TriedAllOptions);
FDB_BOOLEAN_PARAM(AtMostOnce);
FDB_BOOLEAN_PARAM(TriedAllOptions);
// Stores state for a request made by the load balancer
template <class Request, class Interface, class Multi, bool P>

View File

@ -91,7 +91,7 @@ struct AlternativeInfo {
bool operator==(double const& r) const { return cumulativeProbability == r; }
};
FDB_DECLARE_BOOLEAN_PARAM(BalanceOnRequests);
FDB_BOOLEAN_PARAM(BalanceOnRequests);
template <class T>
class ModelInterface : public ReferenceCounted<ModelInterface<T>> {

View File

@ -27,7 +27,7 @@
#include "flow/BooleanParam.h"
#include "flow/flow.h"
FDB_DECLARE_BOOLEAN_PARAM(Averaged);
FDB_BOOLEAN_PARAM(Averaged);
struct PerfMetric {
constexpr static FileIdentifier file_identifier = 5980618;

View File

@ -192,7 +192,7 @@ static void specialCounter(CounterCollection& collection, std::string const& nam
new SpecialCounter<F>(collection, name, std::move(f));
}
FDB_DECLARE_BOOLEAN_PARAM(Filtered);
FDB_BOOLEAN_PARAM(Filtered);
class LatencyBands {
std::map<double, std::unique_ptr<Counter>> bands;

View File

@ -1490,13 +1490,6 @@ Future<Void> DataDistributionTracker::run(Reference<DataDistributionTracker> sel
return holdWhile(self, DataDistributionTrackerImpl::run(self.getPtr(), initData));
}
// Methods for PhysicalShardCollection
FDB_DEFINE_BOOLEAN_PARAM(InAnonymousPhysicalShard);
FDB_DEFINE_BOOLEAN_PARAM(PhysicalShardHasMoreThanKeyRange);
FDB_DEFINE_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_DEFINE_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_DEFINE_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
// Tracks storage metrics for `keys` and updates `physicalShardStats` which is the stats for the physical shard owning
// this key range. This function is similar to `trackShardMetrics()` and altered for physical shard. This meant to be
// temporary. Eventually, we want a new interface to track physical shard metrics more efficiently.

View File

@ -25,12 +25,6 @@
#include "flow/actorcompiler.h" // This must be the last #include.
#include <climits>
FDB_DEFINE_BOOLEAN_PARAM(IsPrimary);
FDB_DEFINE_BOOLEAN_PARAM(IsInitialTeam);
FDB_DEFINE_BOOLEAN_PARAM(IsRedundantTeam);
FDB_DEFINE_BOOLEAN_PARAM(IsBadTeam);
FDB_DEFINE_BOOLEAN_PARAM(WaitWiggle);
namespace {
// Helper function for STL containers, with flow-friendly error handling
@ -43,14 +37,6 @@ auto get(MapContainer& m, K const& k) -> decltype(m.at(k)) {
} // namespace
FDB_DEFINE_BOOLEAN_PARAM(WantNewServers);
FDB_DEFINE_BOOLEAN_PARAM(WantTrueBest);
FDB_DEFINE_BOOLEAN_PARAM(PreferLowerDiskUtil);
FDB_DEFINE_BOOLEAN_PARAM(TeamMustHaveShards);
FDB_DEFINE_BOOLEAN_PARAM(ForReadBalance);
FDB_DEFINE_BOOLEAN_PARAM(PreferLowerReadUtil);
FDB_DEFINE_BOOLEAN_PARAM(FindTeamByServers);
class DDTeamCollectionImpl {
ACTOR static Future<Void> checkAndRemoveInvalidLocalityAddr(DDTeamCollection* self) {
state double start = now();

View File

@ -25,8 +25,6 @@
#include "fdbclient/DatabaseContext.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(SkipDDModeCheck);
class DDTxnProcessorImpl {
friend class DDTxnProcessor;

View File

@ -31,8 +31,6 @@
typedef bool (*compare_pages)(void*, void*);
typedef int64_t loc_t;
FDB_DEFINE_BOOLEAN_PARAM(CheckHashes);
// 0 -> 0
// 1 -> 4k
// 4k -> 4k

View File

@ -27,8 +27,6 @@
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(IsTest);
namespace {
const KeyRef configPathKey = "configPath"_sr;

View File

@ -32,8 +32,6 @@
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(CancelConflictingDataMoves);
namespace {
struct Shard {
Shard() = default;

View File

@ -44,8 +44,6 @@
#include "flow/actorcompiler.h" // has to be last include
FDB_DEFINE_BOOLEAN_PARAM(CheckpointAsKeyValues);
#ifdef SSD_ROCKSDB_EXPERIMENTAL
// Enforcing rocksdb version to be 7.7.3.
static_assert((ROCKSDB_MAJOR == 7 && ROCKSDB_MINOR == 7 && ROCKSDB_PATCH == 3),

View File

@ -168,11 +168,11 @@ public:
};
typedef AsyncMap<UID, ServerStatus> ServerStatusMap;
FDB_DECLARE_BOOLEAN_PARAM(IsPrimary);
FDB_DECLARE_BOOLEAN_PARAM(IsInitialTeam);
FDB_DECLARE_BOOLEAN_PARAM(IsRedundantTeam);
FDB_DECLARE_BOOLEAN_PARAM(IsBadTeam);
FDB_DECLARE_BOOLEAN_PARAM(WaitWiggle);
FDB_BOOLEAN_PARAM(IsPrimary);
FDB_BOOLEAN_PARAM(IsInitialTeam);
FDB_BOOLEAN_PARAM(IsRedundantTeam);
FDB_BOOLEAN_PARAM(IsBadTeam);
FDB_BOOLEAN_PARAM(WaitWiggle);
// send request/signal to DDTeamCollection through interface
// call synchronous method from components outside DDTeamCollection

View File

@ -26,7 +26,7 @@
#include "fdbserver/MoveKeys.actor.h"
#include "fdbserver/MockGlobalState.h"
FDB_DECLARE_BOOLEAN_PARAM(SkipDDModeCheck);
FDB_BOOLEAN_PARAM(SkipDDModeCheck);
struct InitialDataDistribution;
struct DDShardInfo;

View File

@ -242,11 +242,11 @@ struct GetMetricsListRequest {
// For each shard (key-range) move, PhysicalShardCollection decides which physical shard and corresponding team(s) to
// move The current design of PhysicalShardCollection assumes that there exists at most two teamCollections
// TODO: unit test needed
FDB_DECLARE_BOOLEAN_PARAM(InAnonymousPhysicalShard);
FDB_DECLARE_BOOLEAN_PARAM(PhysicalShardHasMoreThanKeyRange);
FDB_DECLARE_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_DECLARE_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_DECLARE_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
FDB_BOOLEAN_PARAM(InAnonymousPhysicalShard);
FDB_BOOLEAN_PARAM(PhysicalShardHasMoreThanKeyRange);
FDB_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
struct ShardMetrics {
StorageMetrics metrics;

View File

@ -61,13 +61,13 @@ struct IDataDistributionTeam {
}
};
FDB_DECLARE_BOOLEAN_PARAM(WantNewServers);
FDB_DECLARE_BOOLEAN_PARAM(WantTrueBest);
FDB_DECLARE_BOOLEAN_PARAM(PreferLowerDiskUtil);
FDB_DECLARE_BOOLEAN_PARAM(TeamMustHaveShards);
FDB_DECLARE_BOOLEAN_PARAM(ForReadBalance);
FDB_DECLARE_BOOLEAN_PARAM(PreferLowerReadUtil);
FDB_DECLARE_BOOLEAN_PARAM(FindTeamByServers);
FDB_BOOLEAN_PARAM(WantNewServers);
FDB_BOOLEAN_PARAM(WantTrueBest);
FDB_BOOLEAN_PARAM(PreferLowerDiskUtil);
FDB_BOOLEAN_PARAM(TeamMustHaveShards);
FDB_BOOLEAN_PARAM(ForReadBalance);
FDB_BOOLEAN_PARAM(PreferLowerReadUtil);
FDB_BOOLEAN_PARAM(FindTeamByServers);
class TeamSelect {
public:

View File

@ -26,7 +26,7 @@
#include "fdbserver/IKeyValueStore.h"
#include "flow/BooleanParam.h"
FDB_DECLARE_BOOLEAN_PARAM(CheckHashes);
FDB_BOOLEAN_PARAM(CheckHashes);
class IDiskQueue : public IClosable {
public:

View File

@ -30,7 +30,7 @@
#include "flow/Arena.h"
#include "flow/Knobs.h"
FDB_DECLARE_BOOLEAN_PARAM(IsTest);
FDB_BOOLEAN_PARAM(IsTest);
/*
* Each worker maintains a LocalConfiguration object used to update its knob collection.

View File

@ -32,7 +32,7 @@
#include "flow/BooleanParam.h"
#include "flow/actorcompiler.h"
FDB_DECLARE_BOOLEAN_PARAM(CancelConflictingDataMoves);
FDB_BOOLEAN_PARAM(CancelConflictingDataMoves);
struct MoveKeysLock {
UID prevOwner, myOwner, prevWrite;

View File

@ -31,7 +31,7 @@
#include "flow/actorcompiler.h" // has to be last include
FDB_DECLARE_BOOLEAN_PARAM(CheckpointAsKeyValues);
FDB_BOOLEAN_PARAM(CheckpointAsKeyValues);
class ICheckpointIterator {
public:

View File

@ -39,7 +39,7 @@
#include "fdbserver/workloads/TenantConsistency.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DECLARE_BOOLEAN_PARAM(AllowPartialMetaclusterOperations);
FDB_BOOLEAN_PARAM(AllowPartialMetaclusterOperations);
template <class DB>
class MetaclusterConsistencyCheck {

View File

@ -234,7 +234,7 @@ struct IWorkloadFactory : ReferenceCounted<IWorkloadFactory> {
virtual Reference<TestWorkload> create(WorkloadContext const& wcx) = 0;
};
FDB_DECLARE_BOOLEAN_PARAM(UntrustedMode);
FDB_BOOLEAN_PARAM(UntrustedMode);
template <class Workload>
struct WorkloadFactory : IWorkloadFactory {

View File

@ -59,8 +59,6 @@
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(UntrustedMode);
WorkloadContext::WorkloadContext() {}
WorkloadContext::WorkloadContext(const WorkloadContext& r)

View File

@ -40,8 +40,6 @@
#include "flow/flow.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(AllowPartialMetaclusterOperations);
struct MetaclusterManagementWorkload : TestWorkload {
static constexpr auto NAME = "MetaclusterManagement";

View File

@ -135,9 +135,6 @@ void* Arena::allocate4kAlignedBuffer(uint32_t size) {
return ArenaBlock::dependOn4kAlignedBuffer(impl, size);
}
FDB_DEFINE_BOOLEAN_PARAM(FastInaccurateEstimate);
FDB_DEFINE_BOOLEAN_PARAM(IsSecureMem);
size_t Arena::getSize(FastInaccurateEstimate fastInaccurateEstimate) const {
if (impl) {
allowAccess(impl.getPtr());

View File

@ -93,7 +93,7 @@ protected:
NonCopyable& operator=(const NonCopyable&) = delete;
};
FDB_DECLARE_BOOLEAN_PARAM(FastInaccurateEstimate);
FDB_BOOLEAN_PARAM(FastInaccurateEstimate);
// Tag struct to indicate that the block containing allocated memory needs to be zero-ed out after use
struct WipeAfterUse {};
@ -162,7 +162,7 @@ struct ArenaBlockRef {
uint32_t nextBlockOffset;
};
FDB_DECLARE_BOOLEAN_PARAM(IsSecureMem);
FDB_BOOLEAN_PARAM(IsSecureMem);
struct ArenaBlock : NonCopyable, ThreadSafeReferenceCounted<ArenaBlock> {
enum {

View File

@ -22,25 +22,40 @@
#include "flow/Trace.h"
class BooleanParam {
bool value;
public:
explicit constexpr BooleanParam(bool value) : value(value) {}
constexpr operator bool() const { return value; }
constexpr void set(bool value) { this->value = value; }
};
template <class BooleanParamSub>
struct Traceable<BooleanParamSub, std::enable_if_t<std::is_base_of_v<BooleanParam, BooleanParamSub>>> : std::true_type {
static std::string toString(BooleanParamSub const& value) { return Traceable<bool>::toString(value); }
};
// Declares a boolean parametr with the desired name. This declaration can be nested inside of a namespace or another
// class. This macro should not be used directly unless this boolean parameter is going to be defined as a nested class.
#define FDB_DECLARE_BOOLEAN_PARAM(ParamName) \
class ParamName { \
bool value; \
\
class ParamName : public BooleanParam { \
public: \
explicit constexpr ParamName(bool value) : value(value) {} \
constexpr operator bool() const { return value; } \
static ParamName const True, False; \
constexpr void set(bool value) { this->value = value; } \
}; \
template <> \
struct Traceable<ParamName> : std::true_type { \
static std::string toString(ParamName const& value) { return Traceable<bool>::toString(value); } \
explicit constexpr ParamName(bool value) : BooleanParam(value) {} \
static ParamName const True; \
static ParamName const False; \
}
// Defines the static members True and False of a boolean parameter.
// This macro should not be used directly unless this boolean parameter is going to be defined as a nested class.
// For a nested class, it is necessary to specify the fully qualified name for the boolean param.
#define FDB_DEFINE_BOOLEAN_PARAM(ParamName) \
ParamName const ParamName::True = ParamName(true); \
ParamName const ParamName::False = ParamName(false)
inline ParamName const ParamName::True = ParamName(true); \
inline ParamName const ParamName::False = ParamName(false)
// Declares and defines a boolean parameter. Use this macro unless this parameter will be nested inside of another
// class. In that case, declare the boolean parameter inside of the class using FDB_DECLARE_BOOLEAN_PARAM and define it
// outside the class using FDB_DEFINE_BOOLEAN_PARAM with a fully-qualified name.
#define FDB_BOOLEAN_PARAM(ParamName) \
FDB_DECLARE_BOOLEAN_PARAM(ParamName); \
FDB_DEFINE_BOOLEAN_PARAM(ParamName)

View File

@ -25,7 +25,7 @@
#include "flow/Trace.h"
#include "flow/IPAddress.h"
FDB_DECLARE_BOOLEAN_PARAM(NetworkAddressFromHostname);
FDB_BOOLEAN_PARAM(NetworkAddressFromHostname);
struct NetworkAddress {
constexpr static FileIdentifier file_identifier = 14155727;

View File

@ -148,8 +148,6 @@ bool IPAddress::isValid() const {
return std::get<uint32_t>(addr) != 0;
}
FDB_DEFINE_BOOLEAN_PARAM(NetworkAddressFromHostname);
NetworkAddress NetworkAddress::parse(std::string const& s) {
if (s.empty()) {
throw connection_string_invalid();