mirror of
https://github.com/apple/foundationdb.git
synced 2025-06-02 19:25:52 +08:00
flow: switch from hard coded to ApiVersion like ProtocolVersion (#8071)
* flow: add ApiVersion to replace hard coding api version Instead of hard coding api value, let's rely on feature versions akin to ProtocolVersion. * ApiVersion: remove use of -1 for latest and use LATEST_VERSION
This commit is contained in:
parent
d3d1ff6023
commit
80a0816157
@ -18,6 +18,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "fmt/format.h"
|
||||
#include "fdbbackup/BackupTLSConfig.h"
|
||||
#include "fdbclient/JsonBuilder.h"
|
||||
@ -2314,7 +2315,7 @@ ACTOR Future<Void> runRestore(Database db,
|
||||
throw restore_error();
|
||||
}
|
||||
|
||||
origDb = Database::createDatabase(originalClusterFile, Database::API_VERSION_LATEST);
|
||||
origDb = Database::createDatabase(originalClusterFile, ApiVersion::LATEST_VERSION);
|
||||
Version v = wait(timeKeeperVersionFromDatetime(targetTimestamp, origDb.get()));
|
||||
fmt::print("Timestamp '{0}' resolves to version {1}\n", targetTimestamp, v);
|
||||
targetVersion = v;
|
||||
@ -2720,7 +2721,7 @@ ACTOR Future<Void> queryBackup(const char* name,
|
||||
return Void();
|
||||
}
|
||||
|
||||
Database origDb = Database::createDatabase(originalClusterFile, Database::API_VERSION_LATEST);
|
||||
Database origDb = Database::createDatabase(originalClusterFile, ApiVersion::LATEST_VERSION);
|
||||
Version v = wait(timeKeeperVersionFromDatetime(restoreTimestamp, origDb));
|
||||
result["restore_timestamp"] = restoreTimestamp;
|
||||
result["restore_timestamp_resolved_version"] = v;
|
||||
@ -3127,7 +3128,7 @@ Optional<Database> connectToCluster(std::string const& clusterFile,
|
||||
}
|
||||
|
||||
try {
|
||||
db = Database::createDatabase(ccf, -1, IsInternal::True, localities);
|
||||
db = Database::createDatabase(ccf, ApiVersion::LATEST_VERSION, IsInternal::True, localities);
|
||||
} catch (Error& e) {
|
||||
if (!quiet) {
|
||||
fprintf(stderr, "ERROR: %s\n", e.what());
|
||||
@ -4123,7 +4124,7 @@ int main(int argc, char* argv[]) {
|
||||
}
|
||||
|
||||
try {
|
||||
db = Database::createDatabase(restoreClusterFileDest, Database::API_VERSION_LATEST);
|
||||
db = Database::createDatabase(restoreClusterFileDest, ApiVersion::LATEST_VERSION);
|
||||
} catch (Error& e) {
|
||||
fprintf(stderr,
|
||||
"Restore destination cluster file '%s' invalid: %s\n",
|
||||
@ -4202,7 +4203,7 @@ int main(int argc, char* argv[]) {
|
||||
}
|
||||
|
||||
try {
|
||||
db = Database::createDatabase(restoreClusterFileDest, Database::API_VERSION_LATEST);
|
||||
db = Database::createDatabase(restoreClusterFileDest, ApiVersion::LATEST_VERSION);
|
||||
} catch (Error& e) {
|
||||
fprintf(stderr,
|
||||
"Restore destination cluster file '%s' invalid: %s\n",
|
||||
|
@ -728,8 +728,11 @@ void DLApi::init() {
|
||||
|
||||
loadClientFunction(&api->selectApiVersion, lib, fdbCPath, "fdb_select_api_version_impl", headerVersion >= 0);
|
||||
loadClientFunction(&api->getClientVersion, lib, fdbCPath, "fdb_get_client_version", headerVersion >= 410);
|
||||
loadClientFunction(
|
||||
&api->useFutureProtocolVersion, lib, fdbCPath, "fdb_use_future_protocol_version", headerVersion >= 720);
|
||||
loadClientFunction(&api->useFutureProtocolVersion,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_use_future_protocol_version",
|
||||
headerVersion >= ApiVersion::withFutureProtocolVersionApi().version());
|
||||
loadClientFunction(&api->setNetworkOption, lib, fdbCPath, "fdb_network_set_option", headerVersion >= 0);
|
||||
loadClientFunction(&api->setupNetwork, lib, fdbCPath, "fdb_setup_network", headerVersion >= 0);
|
||||
loadClientFunction(&api->runNetwork, lib, fdbCPath, "fdb_run_network", headerVersion >= 0);
|
||||
@ -739,7 +742,7 @@ void DLApi::init() {
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_create_database_from_connection_string",
|
||||
headerVersion >= 720);
|
||||
headerVersion >= ApiVersion::withCreateDBFromConnString().version());
|
||||
|
||||
loadClientFunction(&api->databaseOpenTenant, lib, fdbCPath, "fdb_database_open_tenant", headerVersion >= 710);
|
||||
loadClientFunction(
|
||||
@ -772,23 +775,39 @@ void DLApi::init() {
|
||||
fdbCPath,
|
||||
"fdb_database_wait_purge_granules_complete",
|
||||
headerVersion >= 710);
|
||||
loadClientFunction(&api->databaseBlobbifyRange, lib, fdbCPath, "fdb_database_blobbify_range", headerVersion >= 720);
|
||||
loadClientFunction(
|
||||
&api->databaseUnblobbifyRange, lib, fdbCPath, "fdb_database_unblobbify_range", headerVersion >= 720);
|
||||
loadClientFunction(
|
||||
&api->databaseListBlobbifiedRanges, lib, fdbCPath, "fdb_database_list_blobbified_ranges", headerVersion >= 720);
|
||||
loadClientFunction(
|
||||
&api->databaseVerifyBlobRange, lib, fdbCPath, "fdb_database_verify_blob_range", headerVersion >= 720);
|
||||
loadClientFunction(&api->databaseBlobbifyRange,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_database_blobbify_range",
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->databaseUnblobbifyRange,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_database_unblobbify_range",
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->databaseListBlobbifiedRanges,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_database_list_blobbified_ranges",
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->databaseVerifyBlobRange,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_database_verify_blob_range",
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
|
||||
loadClientFunction(
|
||||
&api->tenantCreateTransaction, lib, fdbCPath, "fdb_tenant_create_transaction", headerVersion >= 710);
|
||||
loadClientFunction(
|
||||
&api->tenantPurgeBlobGranules, lib, fdbCPath, "fdb_tenant_purge_blob_granules", headerVersion >= 720);
|
||||
loadClientFunction(&api->tenantPurgeBlobGranules,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_tenant_purge_blob_granules",
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->tenantWaitPurgeGranulesComplete,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_tenant_wait_purge_granules_complete",
|
||||
headerVersion >= 720);
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->tenantDestroy, lib, fdbCPath, "fdb_tenant_destroy", headerVersion >= 710);
|
||||
|
||||
loadClientFunction(&api->transactionSetOption, lib, fdbCPath, "fdb_transaction_set_option", headerVersion >= 0);
|
||||
@ -852,18 +871,22 @@ void DLApi::init() {
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_transaction_read_blob_granules_start",
|
||||
headerVersion >= 720);
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->transactionReadBlobGranulesFinish,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_transaction_read_blob_granules_finish",
|
||||
headerVersion >= 720);
|
||||
headerVersion >= ApiVersion::withBlobRangeApi().version());
|
||||
loadClientFunction(&api->futureGetInt64,
|
||||
lib,
|
||||
fdbCPath,
|
||||
headerVersion >= 620 ? "fdb_future_get_int64" : "fdb_future_get_version",
|
||||
headerVersion >= 0);
|
||||
loadClientFunction(&api->futureGetBool, lib, fdbCPath, "fdb_future_get_bool", headerVersion >= 720);
|
||||
loadClientFunction(&api->futureGetBool,
|
||||
lib,
|
||||
fdbCPath,
|
||||
"fdb_future_get_bool",
|
||||
headerVersion >= ApiVersion::withFutureGetBool().version());
|
||||
loadClientFunction(&api->futureGetUInt64, lib, fdbCPath, "fdb_future_get_uint64", headerVersion >= 700);
|
||||
loadClientFunction(&api->futureGetError, lib, fdbCPath, "fdb_future_get_error", headerVersion >= 0);
|
||||
loadClientFunction(&api->futureGetKey, lib, fdbCPath, "fdb_future_get_key", headerVersion >= 0);
|
||||
@ -1344,7 +1367,7 @@ void MultiVersionTransaction::setOption(FDBTransactionOptions::Option option, Op
|
||||
throw invalid_option();
|
||||
}
|
||||
|
||||
if (MultiVersionApi::apiVersionAtLeast(610) && itr->second.persistent) {
|
||||
if (MultiVersionApi::api->getApiVersion().hasPersistentOptions() && itr->second.persistent) {
|
||||
persistentOptions.emplace_back(option, value.castTo<Standalone<StringRef>>());
|
||||
}
|
||||
|
||||
@ -1862,7 +1885,7 @@ void MultiVersionDatabase::DatabaseState::protocolVersionChanged(ProtocolVersion
|
||||
.detail("OldProtocolVersion", dbProtocolVersion);
|
||||
// When the protocol version changes, clear the corresponding entry in the shared state map
|
||||
// so it can be re-initialized. Only do so if there was a valid previous protocol version.
|
||||
if (dbProtocolVersion.present() && MultiVersionApi::apiVersionAtLeast(710)) {
|
||||
if (dbProtocolVersion.present() && MultiVersionApi::api->getApiVersion().hasClusterSharedStateMap()) {
|
||||
MultiVersionApi::api->clearClusterSharedStateMapEntry(clusterId, dbProtocolVersion.get());
|
||||
}
|
||||
|
||||
@ -1891,7 +1914,7 @@ void MultiVersionDatabase::DatabaseState::protocolVersionChanged(ProtocolVersion
|
||||
return;
|
||||
}
|
||||
|
||||
if (client->external && !MultiVersionApi::apiVersionAtLeast(610)) {
|
||||
if (client->external && !MultiVersionApi::api->getApiVersion().hasInlineUpdateDatabase()) {
|
||||
// Old API versions return a future when creating the database, so we need to wait for it
|
||||
Reference<DatabaseState> self = Reference<DatabaseState>::addRef(this);
|
||||
dbReady = mapThreadFuture<Void, Void>(
|
||||
@ -1975,7 +1998,8 @@ void MultiVersionDatabase::DatabaseState::updateDatabase(Reference<IDatabase> ne
|
||||
.detail("ConnectionRecord", connectionRecord);
|
||||
}
|
||||
}
|
||||
if (db.isValid() && dbProtocolVersion.present() && MultiVersionApi::apiVersionAtLeast(710)) {
|
||||
if (db.isValid() && dbProtocolVersion.present() &&
|
||||
MultiVersionApi::api->getApiVersion().hasClusterSharedStateMap()) {
|
||||
Future<std::string> updateResult =
|
||||
MultiVersionApi::api->updateClusterSharedStateMap(connectionRecord, dbProtocolVersion.get(), db);
|
||||
sharedStateUpdater = map(errorOr(updateResult), [this](ErrorOr<std::string> result) {
|
||||
@ -2095,11 +2119,6 @@ void MultiVersionDatabase::LegacyVersionMonitor::close() {
|
||||
}
|
||||
|
||||
// MultiVersionApi
|
||||
bool MultiVersionApi::apiVersionAtLeast(int minVersion) {
|
||||
ASSERT_NE(MultiVersionApi::api->apiVersion, 0);
|
||||
return MultiVersionApi::api->apiVersion >= minVersion || MultiVersionApi::api->apiVersion < 0;
|
||||
}
|
||||
|
||||
void MultiVersionApi::runOnExternalClientsAllThreads(std::function<void(Reference<ClientInfo>)> func,
|
||||
bool runOnFailedClients) {
|
||||
for (int i = 0; i < threadCount; i++) {
|
||||
@ -2145,17 +2164,18 @@ Reference<ClientInfo> MultiVersionApi::getLocalClient() {
|
||||
}
|
||||
|
||||
void MultiVersionApi::selectApiVersion(int apiVersion) {
|
||||
ApiVersion newApiVersion(apiVersion);
|
||||
if (!localClient) {
|
||||
localClient = makeReference<ClientInfo>(getLocalClientAPI());
|
||||
ASSERT(localClient);
|
||||
}
|
||||
|
||||
if (this->apiVersion != 0 && this->apiVersion != apiVersion) {
|
||||
if (this->apiVersion.isValid() && this->apiVersion != newApiVersion) {
|
||||
throw api_version_already_set();
|
||||
}
|
||||
|
||||
localClient->api->selectApiVersion(apiVersion);
|
||||
this->apiVersion = apiVersion;
|
||||
this->apiVersion = newApiVersion;
|
||||
}
|
||||
|
||||
const char* MultiVersionApi::getClientVersion() {
|
||||
@ -2492,7 +2512,7 @@ void MultiVersionApi::setupNetwork() {
|
||||
if (!bypassMultiClientApi) {
|
||||
runOnExternalClientsAllThreads([this](Reference<ClientInfo> client) {
|
||||
TraceEvent("InitializingExternalClient").detail("LibraryPath", client->libPath);
|
||||
client->api->selectApiVersion(apiVersion);
|
||||
client->api->selectApiVersion(apiVersion.version());
|
||||
if (client->useFutureVersion) {
|
||||
client->api->useFutureProtocolVersion();
|
||||
}
|
||||
@ -2673,7 +2693,7 @@ ACTOR Future<std::string> updateClusterSharedStateMapImpl(MultiVersionApi* self,
|
||||
// The cluster ID will be the connection record string (either a filename or the connection string itself)
|
||||
// in API versions before we could read the cluster ID.
|
||||
state std::string clusterId = connectionRecord.toString();
|
||||
if (MultiVersionApi::apiVersionAtLeast(720)) {
|
||||
if (MultiVersionApi::api->getApiVersion().hasCreateDBFromConnString()) {
|
||||
state Reference<ITransaction> tr = db->createTransaction();
|
||||
loop {
|
||||
try {
|
||||
|
@ -1273,7 +1273,7 @@ void DatabaseContext::registerSpecialKeysImpl(SpecialKeySpace::MODULE module,
|
||||
std::unique_ptr<SpecialKeyRangeReadImpl>&& impl,
|
||||
int deprecatedVersion) {
|
||||
// if deprecated, add the implementation when the api version is less than the deprecated version
|
||||
if (deprecatedVersion == -1 || apiVersion < deprecatedVersion) {
|
||||
if (deprecatedVersion == -1 || apiVersion.version() < deprecatedVersion) {
|
||||
specialKeySpace->registerKeyRange(module, type, impl->getKeyRange(), impl.get());
|
||||
specialKeySpaceModules.push_back(std::move(impl));
|
||||
}
|
||||
@ -1426,7 +1426,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
||||
EnableLocalityLoadBalance enableLocalityLoadBalance,
|
||||
LockAware lockAware,
|
||||
IsInternal internal,
|
||||
int apiVersion,
|
||||
int _apiVersion,
|
||||
IsSwitchable switchable,
|
||||
Optional<TenantName> defaultTenant)
|
||||
: lockAware(lockAware), switchable(switchable), connectionRecord(connectionRecord), proxyProvisional(false),
|
||||
@ -1466,11 +1466,13 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
||||
bgGranulesPerRequest(1000), outstandingWatches(0), sharedStatePtr(nullptr), lastGrvTime(0.0), cachedReadVersion(0),
|
||||
lastRkBatchThrottleTime(0.0), lastRkDefaultThrottleTime(0.0), lastProxyRequestTime(0.0),
|
||||
transactionTracingSample(false), taskID(taskID), clientInfo(clientInfo), clientInfoMonitor(clientInfoMonitor),
|
||||
coordinator(coordinator), apiVersion(apiVersion), mvCacheInsertLocation(0), healthMetricsLastUpdated(0),
|
||||
coordinator(coordinator), mvCacheInsertLocation(0), healthMetricsLastUpdated(0),
|
||||
detailedHealthMetricsLastUpdated(0), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
specialKeySpace(std::make_unique<SpecialKeySpace>(specialKeys.begin, specialKeys.end, /* test */ false)),
|
||||
connectToDatabaseEventCacheHolder(format("ConnectToDatabase/%s", dbId.toString().c_str())) {
|
||||
|
||||
apiVersion = ApiVersion(_apiVersion);
|
||||
|
||||
dbId = deterministicRandom()->randomUniqueID();
|
||||
|
||||
TraceEvent("DatabaseContextCreated", dbId).backtrace();
|
||||
@ -1482,7 +1484,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
||||
metadataVersionCache.resize(CLIENT_KNOBS->METADATA_VERSION_CACHE_SIZE);
|
||||
maxOutstandingWatches = CLIENT_KNOBS->DEFAULT_MAX_OUTSTANDING_WATCHES;
|
||||
|
||||
snapshotRywEnabled = apiVersionAtLeast(300) ? 1 : 0;
|
||||
snapshotRywEnabled = apiVersion.hasSnapshotRYW() ? 1 : 0;
|
||||
|
||||
logger = databaseLogger(this) && tssLogger(this);
|
||||
locationCacheSize = g_network->isSimulated() ? CLIENT_KNOBS->LOCATION_CACHE_EVICTION_SIZE_SIM
|
||||
@ -1501,7 +1503,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
||||
smoothMidShardSize.reset(CLIENT_KNOBS->INIT_MID_SHARD_BYTES);
|
||||
globalConfig = std::make_unique<GlobalConfig>(this);
|
||||
|
||||
if (apiVersionAtLeast(720)) {
|
||||
if (apiVersion.hasTenantsV2()) {
|
||||
registerSpecialKeysImpl(
|
||||
SpecialKeySpace::MODULE::CLUSTERID,
|
||||
SpecialKeySpace::IMPLTYPE::READONLY,
|
||||
@ -1521,14 +1523,13 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
||||
SpecialKeySpace::MODULE::MANAGEMENT,
|
||||
SpecialKeySpace::IMPLTYPE::READWRITE,
|
||||
std::make_unique<TenantRangeImpl<true>>(SpecialKeySpace::getManagementApiCommandRange("tenant")));
|
||||
}
|
||||
if (apiVersionAtLeast(710) && !apiVersionAtLeast(720)) {
|
||||
} else if (apiVersion.hasTenantsV1()) {
|
||||
registerSpecialKeysImpl(
|
||||
SpecialKeySpace::MODULE::MANAGEMENT,
|
||||
SpecialKeySpace::IMPLTYPE::READWRITE,
|
||||
std::make_unique<TenantRangeImpl<false>>(SpecialKeySpace::getManagementApiCommandRange("tenantmap")));
|
||||
}
|
||||
if (apiVersionAtLeast(700)) {
|
||||
if (apiVersion.version() >= 700) {
|
||||
registerSpecialKeysImpl(SpecialKeySpace::MODULE::ERRORMSG,
|
||||
SpecialKeySpace::IMPLTYPE::READONLY,
|
||||
std::make_unique<SingleSpecialKeyImpl>(
|
||||
@ -1651,7 +1652,7 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<IClusterConnection
|
||||
std::make_unique<ActorProfilerConf>(
|
||||
SpecialKeySpace::getModuleRange(SpecialKeySpace::MODULE::ACTOR_PROFILER_CONF)));
|
||||
}
|
||||
if (apiVersionAtLeast(630)) {
|
||||
if (apiVersion.version() >= 630) {
|
||||
registerSpecialKeysImpl(SpecialKeySpace::MODULE::TRANSACTION,
|
||||
SpecialKeySpace::IMPLTYPE::READONLY,
|
||||
std::make_unique<ConflictingKeysImpl>(conflictingKeysRange));
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/Tenant.h"
|
||||
#include "libb64/encode.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/UnitTest.h"
|
||||
|
||||
Key TenantMapEntry::idToPrefix(int64_t id) {
|
||||
@ -127,7 +128,7 @@ std::string TenantMapEntry::toJson(int apiVersion) const {
|
||||
tenantEntry["id"] = id;
|
||||
tenantEntry["encrypted"] = encrypted;
|
||||
|
||||
if (apiVersion >= 720 || apiVersion == Database::API_VERSION_LATEST) {
|
||||
if (apiVersion >= ApiVersion::withTenantsV2().version()) {
|
||||
json_spirit::mObject prefixObject;
|
||||
std::string encodedPrefix = base64::encoder::from_string(prefix.toString());
|
||||
// Remove trailing newline
|
||||
|
@ -601,7 +601,7 @@ extern const char* getSourceVersion();
|
||||
ThreadSafeApi::ThreadSafeApi() : apiVersion(-1), transportId(0) {}
|
||||
|
||||
void ThreadSafeApi::selectApiVersion(int apiVersion) {
|
||||
this->apiVersion = apiVersion;
|
||||
this->apiVersion = ApiVersion(apiVersion);
|
||||
}
|
||||
|
||||
const char* ThreadSafeApi::getClientVersion() {
|
||||
@ -673,12 +673,12 @@ void ThreadSafeApi::stopNetwork() {
|
||||
|
||||
Reference<IDatabase> ThreadSafeApi::createDatabase(const char* clusterFilePath) {
|
||||
return Reference<IDatabase>(
|
||||
new ThreadSafeDatabase(ThreadSafeDatabase::ConnectionRecordType::FILE, clusterFilePath, apiVersion));
|
||||
new ThreadSafeDatabase(ThreadSafeDatabase::ConnectionRecordType::FILE, clusterFilePath, apiVersion.version()));
|
||||
}
|
||||
|
||||
Reference<IDatabase> ThreadSafeApi::createDatabaseFromConnectionString(const char* connectionString) {
|
||||
return Reference<IDatabase>(new ThreadSafeDatabase(
|
||||
ThreadSafeDatabase::ConnectionRecordType::CONNECTION_STRING, connectionString, apiVersion));
|
||||
ThreadSafeDatabase::ConnectionRecordType::CONNECTION_STRING, connectionString, apiVersion.version()));
|
||||
}
|
||||
|
||||
void ThreadSafeApi::addNetworkThreadCompletionHook(void (*hook)(void*), void* hookParameter) {
|
||||
|
@ -21,6 +21,7 @@
|
||||
#ifndef DatabaseContext_h
|
||||
#define DatabaseContext_h
|
||||
#include "fdbclient/Notified.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/FastAlloc.h"
|
||||
#include "flow/FastRef.h"
|
||||
#include "fdbclient/GlobalConfig.actor.h"
|
||||
@ -237,7 +238,7 @@ public:
|
||||
EnableLocalityLoadBalance,
|
||||
TaskPriority taskID = TaskPriority::DefaultEndpoint,
|
||||
LockAware = LockAware::False,
|
||||
int apiVersion = Database::API_VERSION_LATEST,
|
||||
int _apiVersion = ApiVersion::LATEST_VERSION,
|
||||
IsSwitchable = IsSwitchable::False);
|
||||
|
||||
~DatabaseContext();
|
||||
@ -253,7 +254,7 @@ public:
|
||||
enableLocalityLoadBalance,
|
||||
lockAware,
|
||||
internal,
|
||||
apiVersion,
|
||||
apiVersion.version(),
|
||||
switchable,
|
||||
defaultTenant));
|
||||
cx->globalConfig->init(Reference<AsyncVar<ClientDBInfo> const>(cx->clientInfo),
|
||||
@ -344,7 +345,7 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
int apiVersionAtLeast(int minVersion) const { return apiVersion < 0 || apiVersion >= minVersion; }
|
||||
int apiVersionAtLeast(int minVersion) const { return apiVersion.version() >= minVersion; }
|
||||
|
||||
Future<Void> onConnected(); // Returns after a majority of coordination servers are available and have reported a
|
||||
// leader. The cluster file therefore is valid, but the database might be unavailable.
|
||||
@ -402,7 +403,7 @@ public:
|
||||
EnableLocalityLoadBalance,
|
||||
LockAware,
|
||||
IsInternal = IsInternal::True,
|
||||
int apiVersion = Database::API_VERSION_LATEST,
|
||||
int _apiVersion = ApiVersion::LATEST_VERSION,
|
||||
IsSwitchable = IsSwitchable::False,
|
||||
Optional<TenantName> defaultTenant = Optional<TenantName>());
|
||||
|
||||
@ -595,7 +596,7 @@ public:
|
||||
Future<Void> statusLeaderMon;
|
||||
double lastStatusFetch;
|
||||
|
||||
int apiVersion;
|
||||
ApiVersion apiVersion;
|
||||
|
||||
int mvCacheInsertLocation;
|
||||
std::vector<std::pair<Version, Optional<Value>>> metadataVersionCache;
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include "fdbclient/FDBOptions.g.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbclient/IClientApi.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/ProtocolVersion.h"
|
||||
#include "flow/ThreadHelper.actor.h"
|
||||
|
||||
@ -1048,7 +1049,7 @@ public:
|
||||
};
|
||||
std::map<std::string, SharedStateInfo> clusterSharedStateMap;
|
||||
|
||||
static bool apiVersionAtLeast(int minVersion);
|
||||
ApiVersion getApiVersion() { return apiVersion; }
|
||||
|
||||
private:
|
||||
MultiVersionApi();
|
||||
@ -1075,7 +1076,7 @@ private:
|
||||
volatile bool networkSetup;
|
||||
volatile bool bypassMultiClientApi;
|
||||
volatile bool externalClient;
|
||||
int apiVersion;
|
||||
ApiVersion apiVersion;
|
||||
|
||||
int nextThread = 0;
|
||||
int threadCount;
|
||||
|
@ -82,8 +82,6 @@ struct NetworkOptions {
|
||||
|
||||
class Database {
|
||||
public:
|
||||
enum { API_VERSION_LATEST = -1 };
|
||||
|
||||
// Creates a database object that represents a connection to a cluster
|
||||
// This constructor uses a preallocated DatabaseContext that may have been created
|
||||
// on another thread
|
||||
|
@ -76,7 +76,7 @@ private:
|
||||
wait(TenantAPI::listTenantsTransaction(&ryw->getTransaction(), kr.begin, kr.end, limitsHint.rows));
|
||||
|
||||
for (auto tenant : tenants) {
|
||||
std::string jsonString = tenant.second.toJson(ryw->getDatabase()->apiVersion);
|
||||
std::string jsonString = tenant.second.toJson(ryw->getDatabase()->apiVersion.version());
|
||||
ValueRef tenantEntryBytes(results->arena(), jsonString);
|
||||
results->push_back(results->arena(),
|
||||
KeyValueRef(withTenantMapPrefix(tenant.first, results->arena()), tenantEntryBytes));
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
#ifndef FDBCLIENT_THREADSAFETRANSACTION_H
|
||||
#define FDBCLIENT_THREADSAFETRANSACTION_H
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/ProtocolVersion.h"
|
||||
#pragma once
|
||||
|
||||
@ -242,7 +243,7 @@ private:
|
||||
friend IClientApi* getLocalClientAPI();
|
||||
ThreadSafeApi();
|
||||
|
||||
int apiVersion;
|
||||
ApiVersion apiVersion;
|
||||
std::string clientVersion;
|
||||
uint64_t transportId;
|
||||
|
||||
|
@ -19,6 +19,7 @@
|
||||
*/
|
||||
|
||||
#include <cmath>
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/UnitTest.h"
|
||||
#include "flow/TDMetric.actor.h"
|
||||
#include "fdbclient/DatabaseContext.h"
|
||||
@ -417,7 +418,7 @@ TEST_CASE("/fdbserver/metrics/TraceEvents") {
|
||||
}
|
||||
fprintf(stdout, "Using environment variables METRICS_CONNFILE and METRICS_PREFIX.\n");
|
||||
|
||||
state Database metricsDb = Database::createDatabase(metricsConnFile, Database::API_VERSION_LATEST);
|
||||
state Database metricsDb = Database::createDatabase(metricsConnFile, ApiVersion::LATEST_VERSION);
|
||||
TDMetricCollection::getTDMetrics()->address = LiteralStringRef("0.0.0.0:0");
|
||||
state Future<Void> metrics = runMetrics(metricsDb, KeyRef(metricsPrefix));
|
||||
state int64_t x = 0;
|
||||
|
@ -31,6 +31,7 @@
|
||||
#include "fdbclient/ManagementAPI.actor.h"
|
||||
#include "fdbclient/MutationList.h"
|
||||
#include "fdbclient/BackupContainer.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/IAsyncFile.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "flow/genericactors.actor.h"
|
||||
@ -410,7 +411,7 @@ ACTOR Future<Void> restoreWorker(Reference<IClusterConnectionRecord> connRecord,
|
||||
LocalityData locality,
|
||||
std::string coordFolder) {
|
||||
try {
|
||||
Database cx = Database::createDatabase(connRecord, Database::API_VERSION_LATEST, IsInternal::True, locality);
|
||||
Database cx = Database::createDatabase(connRecord, ApiVersion::LATEST_VERSION, IsInternal::True, locality);
|
||||
wait(reportErrors(_restoreWorker(cx, locality), "RestoreWorker"));
|
||||
} catch (Error& e) {
|
||||
TraceEvent("FastRestoreWorker").detail("Error", e.what());
|
||||
|
@ -473,7 +473,7 @@ ACTOR Future<Void> runBackup(Reference<IClusterConnectionRecord> connRecord) {
|
||||
}
|
||||
|
||||
if (g_simulator.backupAgents == ISimulator::BackupAgentType::BackupToFile) {
|
||||
Database cx = Database::createDatabase(connRecord, -1);
|
||||
Database cx = Database::createDatabase(connRecord, ApiVersion::LATEST_VERSION);
|
||||
|
||||
state FileBackupAgent fileAgent;
|
||||
agentFutures.push_back(fileAgent.run(
|
||||
@ -501,11 +501,11 @@ ACTOR Future<Void> runDr(Reference<IClusterConnectionRecord> connRecord) {
|
||||
|
||||
if (g_simulator.drAgents == ISimulator::BackupAgentType::BackupToDB) {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
Database cx = Database::createDatabase(connRecord, -1);
|
||||
Database cx = Database::createDatabase(connRecord, ApiVersion::LATEST_VERSION);
|
||||
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
state Database drDatabase = Database::createDatabase(extraFile, -1);
|
||||
state Database drDatabase = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
|
||||
TraceEvent("StartingDrAgents")
|
||||
.detail("ConnectionString", connRecord->getConnectionString().toString())
|
||||
|
@ -291,7 +291,7 @@ struct ApiWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <unordered_map>
|
||||
|
||||
#include "fdbclient/BlobGranuleCommon.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "fmt/format.h"
|
||||
#include "fdbclient/CommitTransaction.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
@ -10359,7 +10360,7 @@ ACTOR Future<Void> memoryStoreRecover(IKeyValueStore* store, Reference<IClusterC
|
||||
}
|
||||
|
||||
// create a temp client connect to DB
|
||||
Database cx = Database::createDatabase(connRecord, Database::API_VERSION_LATEST);
|
||||
Database cx = Database::createDatabase(connRecord, ApiVersion::LATEST_VERSION);
|
||||
|
||||
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(cx);
|
||||
state int noCanRemoveCount = 0;
|
||||
|
@ -684,7 +684,7 @@ ACTOR Future<Void> testerServerWorkload(WorkloadRequest work,
|
||||
startRole(Role::TESTER, workIface.id(), UID(), details);
|
||||
|
||||
if (work.useDatabase) {
|
||||
cx = Database::createDatabase(ccr, -1, IsInternal::True, locality);
|
||||
cx = Database::createDatabase(ccr, ApiVersion::LATEST_VERSION, IsInternal::True, locality);
|
||||
cx->defaultTenant = work.defaultTenant.castTo<TenantName>();
|
||||
wait(delay(1.0));
|
||||
}
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include <boost/lexical_cast.hpp>
|
||||
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/IAsyncFile.h"
|
||||
#include "fdbrpc/Locality.h"
|
||||
#include "fdbclient/GlobalConfig.actor.h"
|
||||
@ -1663,7 +1664,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
||||
if (metricsConnFile.size() > 0) {
|
||||
try {
|
||||
state Database db =
|
||||
Database::createDatabase(metricsConnFile, Database::API_VERSION_LATEST, IsInternal::True, locality);
|
||||
Database::createDatabase(metricsConnFile, ApiVersion::LATEST_VERSION, IsInternal::True, locality);
|
||||
metricsLogger = runMetrics(db, KeyRef(metricsPrefix));
|
||||
db->globalConfig->trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
||||
} catch (Error& e) {
|
||||
|
@ -307,7 +307,7 @@ ACTOR Future<Void> chooseTransactionFactory(Database cx, std::vector<Transaction
|
||||
new TransactionFactory<ThreadTransactionWrapper, Reference<IDatabase>>(dbHandle, dbHandle, false));
|
||||
} else if (transactionType == MULTI_VERSION) {
|
||||
printf("client %d: Running Multi-Version Transactions\n", self->clientPrefixInt);
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion.version());
|
||||
Reference<IDatabase> threadSafeHandle =
|
||||
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(cx)));
|
||||
Reference<IDatabase> dbHandle = MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeHandle);
|
||||
|
@ -111,7 +111,7 @@ struct AtomicOpsWorkload : TestWorkload {
|
||||
|
||||
Future<Void> setup(Database const& cx) override {
|
||||
if (apiVersion500)
|
||||
cx->apiVersion = 500;
|
||||
cx->apiVersion = ApiVersion(500);
|
||||
|
||||
if (clientId != 0)
|
||||
return Void();
|
||||
|
@ -29,9 +29,9 @@ struct AtomicOpsApiCorrectnessWorkload : TestWorkload {
|
||||
uint32_t opType;
|
||||
|
||||
private:
|
||||
static int getApiVersion(const Database& cx) { return cx->apiVersion; }
|
||||
static int getApiVersion(const Database& cx) { return cx->apiVersion.version(); }
|
||||
|
||||
static void setApiVersion(Database* cx, int version) { (*cx)->apiVersion = version; }
|
||||
static void setApiVersion(Database* cx, int version) { (*cx)->apiVersion = ApiVersion(version); }
|
||||
|
||||
Key getTestKey(std::string prefix) {
|
||||
std::string key = prefix + std::to_string(clientId);
|
||||
|
@ -42,7 +42,7 @@ struct AtomicSwitchoverWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
}
|
||||
|
||||
std::string description() const override { return "AtomicSwitchover"; }
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "fdbclient/ManagementAPI.actor.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct BackupToDBAbort : TestWorkload {
|
||||
@ -39,7 +40,7 @@ struct BackupToDBAbort : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
|
||||
lockid = UID(0xbeeffeed, 0xdecaf00d);
|
||||
}
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "fdbclient/ClusterConnectionMemoryRecord.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
// A workload which test the correctness of backup and restore process. The
|
||||
@ -131,7 +132,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
|
||||
TraceEvent("BARW_Start").detail("Locked", locked);
|
||||
}
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
#include "fdbclient/ManagementAPI.actor.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
// A workload which test the correctness of upgrading DR from 5.1 to 5.2
|
||||
@ -79,7 +80,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
|
||||
TraceEvent("DRU_Start").log();
|
||||
}
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbclient/Schemas.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct ChangeConfigWorkload : TestWorkload {
|
||||
@ -85,7 +86,7 @@ struct ChangeConfigWorkload : TestWorkload {
|
||||
if (g_network->isSimulated()) {
|
||||
for (auto extraDatabase : g_simulator.extraDatabases) {
|
||||
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(extraDatabase));
|
||||
Database db = Database::createDatabase(extraFile, -1);
|
||||
Database db = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
futures.push_back(configureExtraDatabase(self, db));
|
||||
}
|
||||
}
|
||||
|
@ -24,6 +24,7 @@
|
||||
|
||||
#include <fmt/format.h>
|
||||
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // has to be last include
|
||||
|
||||
class WorkloadProcessState {
|
||||
@ -130,7 +131,7 @@ struct WorkloadProcess {
|
||||
try {
|
||||
child = childCreator(wcx);
|
||||
TraceEvent("ClientWorkloadOpenDatabase", id).detail("ClusterFileLocation", child->ccr->getLocation());
|
||||
cx = Database::createDatabase(child->ccr, -1);
|
||||
cx = Database::createDatabase(child->ccr, ApiVersion::LATEST_VERSION);
|
||||
desc = child->description();
|
||||
} catch (Error&) {
|
||||
throw;
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "fdbclient/RunTransaction.actor.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/genericactors.actor.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
@ -40,7 +41,7 @@ struct DifferentClustersSameRVWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 100.0);
|
||||
switchAfter = getOption(options, LiteralStringRef("switchAfter"), 50.0);
|
||||
keyToRead = getOption(options, LiteralStringRef("keyToRead"), LiteralStringRef("someKey"));
|
||||
|
@ -96,7 +96,7 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
||||
Reference<IDatabase> threadSafeHandle =
|
||||
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(cx)));
|
||||
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion.version());
|
||||
self->managementDb = MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeHandle);
|
||||
|
||||
ASSERT(g_simulator.extraDatabases.size() > 0);
|
||||
@ -104,7 +104,8 @@ struct MetaclusterManagementWorkload : TestWorkload {
|
||||
ClusterConnectionString ccs(connectionString);
|
||||
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(ccs);
|
||||
self->dataDbIndex.push_back(ClusterName(format("cluster_%08d", self->dataDbs.size())));
|
||||
self->dataDbs[self->dataDbIndex.back()] = DataClusterData(Database::createDatabase(extraFile, -1));
|
||||
self->dataDbs[self->dataDbIndex.back()] =
|
||||
DataClusterData(Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION));
|
||||
}
|
||||
|
||||
wait(success(MetaclusterAPI::createMetacluster(cx.getReference(), "management_cluster"_sr)));
|
||||
|
@ -96,7 +96,7 @@ struct TenantManagementConcurrencyWorkload : TestWorkload {
|
||||
Reference<IDatabase> threadSafeHandle =
|
||||
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(cx)));
|
||||
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion.version());
|
||||
self->mvDb = MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeHandle);
|
||||
|
||||
if (self->useMetacluster && self->clientId == 0) {
|
||||
@ -143,7 +143,7 @@ struct TenantManagementConcurrencyWorkload : TestWorkload {
|
||||
if (self->useMetacluster) {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(connectionString);
|
||||
self->dataDb = Database::createDatabase(extraFile, -1);
|
||||
self->dataDb = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
} else {
|
||||
self->dataDb = cx;
|
||||
}
|
||||
|
@ -36,6 +36,7 @@
|
||||
#include "fdbserver/workloads/TenantConsistency.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/Error.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/ThreadHelper.actor.h"
|
||||
@ -171,7 +172,7 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
Reference<IDatabase> threadSafeHandle =
|
||||
wait(unsafeThreadFutureToFuture(ThreadSafeDatabase::createFromExistingDatabase(cx)));
|
||||
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion.version());
|
||||
self->mvDb = MultiVersionDatabase::debugCreateFromExistingDatabase(threadSafeHandle);
|
||||
|
||||
if (self->useMetacluster && self->clientId == 0) {
|
||||
@ -219,7 +220,7 @@ struct TenantManagementWorkload : TestWorkload {
|
||||
if (self->useMetacluster) {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile = makeReference<ClusterConnectionMemoryRecord>(g_simulator.extraDatabases[0]);
|
||||
self->dataDb = Database::createDatabase(extraFile, -1);
|
||||
self->dataDb = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
} else {
|
||||
self->dataDb = cx;
|
||||
}
|
||||
|
@ -149,7 +149,7 @@ struct ThreadSafetyWorkload : TestWorkload {
|
||||
self->db = dbRef;
|
||||
|
||||
if (deterministicRandom()->coinflip()) {
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion);
|
||||
MultiVersionApi::api->selectApiVersion(cx->apiVersion.version());
|
||||
self->db = MultiVersionDatabase::debugCreateFromExistingDatabase(dbRef);
|
||||
}
|
||||
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include "fdbserver/workloads/BulkSetup.actor.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct VersionStampWorkload : TestWorkload {
|
||||
@ -72,13 +73,13 @@ struct VersionStampWorkload : TestWorkload {
|
||||
} else if (choice < 0.3) {
|
||||
apiVersion = 520;
|
||||
} else {
|
||||
apiVersion = Database::API_VERSION_LATEST;
|
||||
apiVersion = ApiVersion::LATEST_VERSION;
|
||||
}
|
||||
TraceEvent("VersionStampApiVersion").detail("ApiVersion", apiVersion);
|
||||
|
||||
allowMetadataVersionKey = apiVersion >= 610 || apiVersion == Database::API_VERSION_LATEST;
|
||||
allowMetadataVersionKey = apiVersion >= 610 || apiVersion == ApiVersion::LATEST_VERSION;
|
||||
|
||||
cx->apiVersion = apiVersion;
|
||||
cx->apiVersion = ApiVersion(apiVersion);
|
||||
if (clientId == 0)
|
||||
return _start(cx, this, 1 / transactionsPerSecond);
|
||||
return Void();
|
||||
@ -158,7 +159,7 @@ struct VersionStampWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
cx = Database::createDatabase(extraFile, -1);
|
||||
cx = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
}
|
||||
state ReadYourWritesTransaction tr(cx);
|
||||
// We specifically wish to grab the smalles read version that we can get and maintain it, to
|
||||
@ -318,7 +319,7 @@ struct VersionStampWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
}
|
||||
|
||||
state Future<Void> metadataWatch = Void();
|
||||
|
@ -27,6 +27,7 @@
|
||||
#include "flow/ActorCollection.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbclient/Atomic.h"
|
||||
#include "flow/ApiVersion.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct WriteDuringReadWorkload : TestWorkload {
|
||||
@ -93,7 +94,7 @@ struct WriteDuringReadWorkload : TestWorkload {
|
||||
ASSERT(g_simulator.extraDatabases.size() == 1);
|
||||
auto extraFile =
|
||||
makeReference<ClusterConnectionMemoryRecord>(ClusterConnectionString(g_simulator.extraDatabases[0]));
|
||||
extraDB = Database::createDatabase(extraFile, -1);
|
||||
extraDB = Database::createDatabase(extraFile, ApiVersion::LATEST_VERSION);
|
||||
useSystemKeys = false;
|
||||
}
|
||||
|
||||
|
72
flow/ApiVersion.h.cmake
Normal file
72
flow/ApiVersion.h.cmake
Normal file
@ -0,0 +1,72 @@
|
||||
/*
|
||||
* ProtocolVersion.h
|
||||
*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
#include "flow/Trace.h"
|
||||
#include <cstdint>
|
||||
|
||||
constexpr int noBackwardsCompatibility = 13;
|
||||
|
||||
// The first check second expression version doesn't need to change because it's just for earlier protocol versions.
|
||||
#define API_VERSION_FEATURE(v, x) \
|
||||
static_assert(v <= @FDB_AV_LATEST_VERSION@, "Feature protocol version too large"); \
|
||||
struct x { \
|
||||
static constexpr uint64_t apiVersion = v; \
|
||||
}; \
|
||||
constexpr bool has##x() const { return this->version() >= x ::apiVersion; } \
|
||||
static constexpr ApiVersion with##x() { return ApiVersion(x ::apiVersion); }
|
||||
|
||||
class ApiVersion {
|
||||
int _version;
|
||||
|
||||
public:
|
||||
// Statics.
|
||||
constexpr static int LATEST_VERSION = @FDB_AV_LATEST_VERSION@;
|
||||
constexpr static int FUTURE_VERSION = @FDB_AV_FUTURE_VERSION@;
|
||||
|
||||
constexpr explicit ApiVersion(int version) : _version(version) {}
|
||||
constexpr ApiVersion() : _version(0) {}
|
||||
|
||||
constexpr bool isValid() const {
|
||||
return version() > noBackwardsCompatibility && version() <= LATEST_VERSION;
|
||||
}
|
||||
|
||||
constexpr int version() const { return _version; }
|
||||
|
||||
// comparison operators
|
||||
constexpr bool operator==(const ApiVersion other) const { return version() == other.version(); }
|
||||
constexpr bool operator!=(const ApiVersion other) const { return version() != other.version(); }
|
||||
constexpr bool operator<=(const ApiVersion other) const { return version() <= other.version(); }
|
||||
constexpr bool operator>=(const ApiVersion other) const { return version() >= other.version(); }
|
||||
constexpr bool operator<(const ApiVersion other) const { return version() < other.version(); }
|
||||
constexpr bool operator>(const ApiVersion other) const { return version() > other.version(); }
|
||||
|
||||
public: // introduced features
|
||||
API_VERSION_FEATURE(@FDB_AV_SNAPSHOT_RYW@, SnapshotRYW);
|
||||
API_VERSION_FEATURE(@FDB_AV_INLINE_UPDATE_DATABASE@, InlineUpdateDatabase);
|
||||
API_VERSION_FEATURE(@FDB_AV_PERSISTENT_OPTIONS@, PersistentOptions);
|
||||
API_VERSION_FEATURE(@FDB_AV_CLUSTER_SHARED_STATE_MAP@, ClusterSharedStateMap);
|
||||
API_VERSION_FEATURE(@FDB_AV_TENANTS_V1@, TenantsV1);
|
||||
API_VERSION_FEATURE(@FDB_AV_BLOB_RANGE_API@, BlobRangeApi);
|
||||
API_VERSION_FEATURE(@FDB_AV_CREATE_DB_FROM_CONN_STRING@, CreateDBFromConnString);
|
||||
API_VERSION_FEATURE(@FDB_AV_FUTURE_GET_BOOL@, FutureGetBool);
|
||||
API_VERSION_FEATURE(@FDB_AV_FUTURE_PROTOCOL_VERSION_API@, FutureProtocolVersionApi);
|
||||
API_VERSION_FEATURE(@FDB_AV_TENANTS_V2@, TenantsV2);
|
||||
};
|
15
flow/ApiVersions.cmake
Normal file
15
flow/ApiVersions.cmake
Normal file
@ -0,0 +1,15 @@
|
||||
# API Versions
|
||||
set(FDB_AV_LATEST_VERSION "720")
|
||||
set(FDB_AV_FUTURE_VERSION "730")
|
||||
|
||||
# Features
|
||||
set(FDB_AV_SNAPSHOT_RYW "300")
|
||||
set(FDB_AV_INLINE_UPDATE_DATABASE "610")
|
||||
set(FDB_AV_PERSISTENT_OPTIONS "610")
|
||||
set(FDB_AV_CLUSTER_SHARED_STATE_MAP "710")
|
||||
set(FDB_AV_TENANTS_V1 "720")
|
||||
set(FDB_AV_BLOB_RANGE_API "720")
|
||||
set(FDB_AV_CREATE_DB_FROM_CONN_STRING "720")
|
||||
set(FDB_AV_FUTURE_GET_BOOL "720")
|
||||
set(FDB_AV_FUTURE_PROTOCOL_VERSION_API "720")
|
||||
set(FDB_AV_TENANTS_V2 "720")
|
@ -11,6 +11,9 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64")
|
||||
endif()
|
||||
|
||||
make_directory(${CMAKE_CURRENT_BINARY_DIR}/include/flow)
|
||||
set(FDB_API_VERSION_FILE "${CMAKE_CURRENT_SOURCE_DIR}/ApiVersions.cmake" CACHE STRING "Api version cmake file." FORCE)
|
||||
include(${FDB_API_VERSION_FILE})
|
||||
configure_file(${CMAKE_CURRENT_SOURCE_DIR}/ApiVersion.h.cmake ${CMAKE_CURRENT_BINARY_DIR}/include/flow/ApiVersion.h)
|
||||
set(FDB_PROTOCOL_VERSION_FILE "${CMAKE_CURRENT_SOURCE_DIR}/ProtocolVersions.cmake" CACHE STRING "Protocol version cmake file." FORCE)
|
||||
include(${FDB_PROTOCOL_VERSION_FILE})
|
||||
configure_file(${CMAKE_CURRENT_SOURCE_DIR}/ProtocolVersion.h.cmake ${CMAKE_CURRENT_BINARY_DIR}/include/flow/ProtocolVersion.h)
|
||||
|
Loading…
x
Reference in New Issue
Block a user