1
0
Fork 0

issue 373.1: move globally-unique id generation from collection into data-source (#5182)

This commit is contained in:
Vasiliy 2018-05-07 22:14:40 +03:00 committed by Andrey Abramov
parent 39a4cbf0d6
commit 2ce20e86d7
20 changed files with 347 additions and 234 deletions

View File

@ -43,6 +43,12 @@ typedef irs::async_utils::read_write_mutex::write_mutex WriteMutex;
////////////////////////////////////////////////////////////////////////////////
const std::string DELETED_FIELD("deleted");
////////////////////////////////////////////////////////////////////////////////
/// @brief the name of the field in the IResearch View definition denoting the
/// view globaly-unique id (from LogicalView.cpp)
////////////////////////////////////////////////////////////////////////////////
const std::string GLOBALLY_UNIQUE_ID_FIELD("globallyUniqueId");
////////////////////////////////////////////////////////////////////////////////
/// @brief the name of the field in the IResearch View definition denoting the
/// view id (from LogicalView.cpp)
@ -491,6 +497,7 @@ void IResearchViewDBServer::toVelocyPack(
if (includeSystem) {
result.add(DELETED_FIELD, arangodb::velocypack::Value(deleted()));
result.add(GLOBALLY_UNIQUE_ID_FIELD, arangodb::velocypack::Value(guid()));
result.add(IS_SYSTEM_FIELD, arangodb::velocypack::Value(system()));
result.add(PLAN_ID_FIELD, arangodb::velocypack::Value(std::to_string(planId())));
}

View File

@ -1014,7 +1014,7 @@ arangodb::Result MMFilesEngine::dropCollection(
builder.openObject();
builder.add("id", VPackValue(std::to_string(collection->id())));
builder.add("name", VPackValue(collection->name()));
builder.add("cuid", VPackValue(collection->globallyUniqueId()));
builder.add("cuid", VPackValue(collection->guid()));
builder.close();
MMFilesCollectionMarker marker(

View File

@ -306,31 +306,40 @@ struct MMFilesWalAccessContext : WalAccessContext {
if (type == TRI_DF_MARKER_VPACK_DROP_DATABASE) {
VPackSlice slice(reinterpret_cast<char const*>(marker) +
MMFilesDatafileHelper::VPackOffset(type));
_builder.add("db", slice.get("name"));
} else if (type == TRI_DF_MARKER_VPACK_DROP_COLLECTION) {
TRI_ASSERT(databaseId != 0);
TRI_vocbase_t* vocbase = loadVocbase(databaseId);
if (vocbase == nullptr) {
// ignore markers from dropped dbs
return TRI_ERROR_NO_ERROR;
}
VPackSlice slice(reinterpret_cast<char const*>(marker) +
MMFilesDatafileHelper::VPackOffset(type));
_builder.add("db", VPackValue(vocbase->name()));
_builder.add("cuid", slice.get("cuid"));
} else {
TRI_ASSERT(databaseId != 0);
TRI_vocbase_t* vocbase = loadVocbase(databaseId);
if (vocbase == nullptr) {
return TRI_ERROR_NO_ERROR; // ignore dropped dbs
}
_builder.add("db", VPackValue(vocbase->name()));
if (collectionId > 0) {
LogicalCollection* col = loadCollection(databaseId, collectionId);
if (col == nullptr) {
return TRI_ERROR_NO_ERROR; // ignore dropped collections
}
_builder.add("cuid", VPackValue(col->globallyUniqueId()));
_builder.add("cuid", VPackValue(col->guid()));
}
}

View File

@ -811,9 +811,9 @@ Result DatabaseInitialSyncer::handleCollection(VPackSlice const& parameters,
// not found...
col = vocbase()->lookupCollection(masterName).get();
if (col != nullptr && (col->name() != masterName ||
(!masterUuid.empty() && col->globallyUniqueId() != masterUuid))) {
if (col != nullptr
&& (col->name() != masterName
|| (!masterUuid.empty() && col->guid() != masterUuid))) {
// found another collection with the same name locally.
// in this case we must drop it because we will run into duplicate
// name conflicts otherwise

View File

@ -254,18 +254,20 @@ Result GlobalInitialSyncer::updateServerInventory(VPackSlice const& masterDataba
// somehow invalid
continue;
}
VPackSlice const params = coll.get("parameters");
auto existingCollection = resolveCollection(vocbase, params);
if (existingCollection != nullptr) {
survivingCollections.emplace(existingCollection->globallyUniqueId());
survivingCollections.emplace(existingCollection->guid());
}
}
std::vector<arangodb::LogicalCollection*> toDrop;
// drop all collections that do not exist (anymore) on the master
vocbase->processCollections([&survivingCollections, &toDrop](arangodb::LogicalCollection* collection) {
if (survivingCollections.find(collection->globallyUniqueId()) != survivingCollections.end()) {
if (survivingCollections.find(collection->guid()) != survivingCollections.end()) {
// collection should surive
return;
}

View File

@ -585,7 +585,7 @@ Result Syncer::createCollection(TRI_vocbase_t* vocbase,
if (col != nullptr) {
if (col->system()) {
TRI_ASSERT(!simulate32Client() || col->globallyUniqueId() == col->name());
TRI_ASSERT(!simulate32Client() || col->guid() == col->name());
SingleCollectionTransaction trx(
transaction::StandaloneContext::Create(vocbase),
col->id(),
@ -621,12 +621,15 @@ Result Syncer::createCollection(TRI_vocbase_t* vocbase,
s.add("id", VPackSlice::nullSlice());
s.add("cid", VPackSlice::nullSlice());
}
s.close();
VPackBuilder merged = VPackCollection::merge(slice, s.slice(), /*mergeValues*/true,
/*nullMeansRemove*/true);
// we need to remove every occurence of objectId as a key
auto stripped = rocksutils::stripObjectIds(merged.slice());
try {
col = vocbase->createCollection(stripped.first);
} catch (basics::Exception const& ex) {
@ -638,12 +641,12 @@ Result Syncer::createCollection(TRI_vocbase_t* vocbase,
}
TRI_ASSERT(col != nullptr);
TRI_ASSERT(!uuid.isString() ||
uuid.compareString(col->globallyUniqueId()) == 0);
TRI_ASSERT(!uuid.isString() || uuid.compareString(col->guid()) == 0);
if (dst != nullptr) {
*dst = col;
}
return Result();
}

View File

@ -495,7 +495,7 @@ void RestCollectionHandler::collectionRepresentation(
if (!showProperties) {
builder.add("isSystem", VPackValue(coll->system()));
builder.add("globallyUniqueId", VPackValue(coll->globallyUniqueId()));
builder.add("globallyUniqueId", VPackValue(coll->guid()));
} else {
Result res = methods::Collections::properties(coll, builder);

View File

@ -1107,7 +1107,7 @@ arangodb::Result RocksDBEngine::dropCollection(
// Prepare collection remove batch
RocksDBLogValue logValue = RocksDBLogValue::CollectionDrop(
vocbase.id(), collection->id(), StringRef(collection->globallyUniqueId())
vocbase.id(), collection->id(), StringRef(collection->guid())
);
rocksdb::WriteBatch batch;
batch.PutLogData(logValue.slice());

View File

@ -183,30 +183,32 @@ class WALParser : public rocksdb::WriteBatch::Handler {
}
break;
}
case RocksDBLogType::IndexCreate: {
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
TRI_ASSERT(_vocbase->id() == dbid);
LogicalCollection* coll = loadCollection(cid);
TRI_ASSERT(coll != nullptr);
VPackSlice indexDef = RocksDBLogValue::indexSlice(blob);
auto stripped = rocksutils::stripObjectIds(indexDef);
uint64_t tick = _currentSequence + (_startOfBatch ? 0 : 1);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(tick)));
_builder.add("type", VPackValue(REPLICATION_INDEX_CREATE));
_builder.add("database", VPackValue(std::to_string(dbid)));
_builder.add("cid", VPackValue(std::to_string(cid)));
_builder.add("cuid", VPackValue(coll->globallyUniqueId()));
_builder.add("cuid", VPackValue(coll->guid()));
_builder.add("cname", VPackValue(coll->name()));
_builder.add("data", stripped.first);
_builder.close();
updateLastEmittedTick(tick);
}
break;
}
case RocksDBLogType::IndexDrop: {

View File

@ -184,8 +184,10 @@ class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext
}
case RocksDBLogType::IndexCreate: {
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
// only print markers from this collection if it is set
if (shouldHandleCollection(dbid, cid)) {
TRI_vocbase_t* vocbase = loadVocbase(dbid);
@ -193,45 +195,57 @@ class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext
TRI_ASSERT(vocbase != nullptr && coll != nullptr);
VPackSlice indexDef = RocksDBLogValue::indexSlice(blob);
auto stripped = rocksutils::stripObjectIds(indexDef);
{
uint64_t tick = _currentSequence + (_startOfBatch ? 0 : 1);
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(tick)));
marker->add("type", VPackValue(rocksutils::convertLogType(type)));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValue(coll->globallyUniqueId()));
marker->add("cuid", VPackValue(coll->guid()));
marker->add("data", stripped.first);
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
break;
}
case RocksDBLogType::IndexDrop: {
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
TRI_idx_iid_t iid = RocksDBLogValue::indexId(blob);
// only print markers from this collection if it is set
if (shouldHandleCollection(dbid, cid)) {
TRI_vocbase_t* vocbase = loadVocbase(dbid);
LogicalCollection* col = loadCollection(dbid, cid);
TRI_ASSERT(vocbase != nullptr && col != nullptr);
{
uint64_t tick = _currentSequence + (_startOfBatch ? 0 : 1);
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(tick)));
marker->add("type", VPackValue(rocksutils::convertLogType(type)));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValue(col->globallyUniqueId()));
marker->add("cuid", VPackValue(col->guid()));
VPackObjectBuilder data(&_builder, "data", true);
data->add("id", VPackValue(std::to_string(iid)));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
break;
}
case RocksDBLogType::ViewCreate:
@ -373,35 +387,43 @@ class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext
_builder.clear();
} // ignore Put in any other case
} else if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
TRI_voc_tick_t dbid = RocksDBKey::databaseId(key);
TRI_voc_cid_t cid = RocksDBKey::collectionId(key);
if (shouldHandleCollection(dbid, cid) && (_state == COLLECTION_CREATE ||
_state == COLLECTION_RENAME ||
_state == COLLECTION_CHANGE)) {
TRI_vocbase_t* vocbase = loadVocbase(dbid);
LogicalCollection* col = loadCollection(dbid, cid);
TRI_ASSERT(vocbase != nullptr && col != nullptr);
{
VPackSlice collectionDef = RocksDBValue::data(value);
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValue(col->globallyUniqueId()));
marker->add("cuid", VPackValue(col->guid()));
if (_state == COLLECTION_CREATE) {
auto stripped = rocksutils::stripObjectIds(collectionDef);
marker->add("type", VPackValue(REPLICATION_COLLECTION_CREATE));
marker->add("data", stripped.first);
} else if (_state == COLLECTION_RENAME) {
marker->add("type", VPackValue(REPLICATION_COLLECTION_RENAME));
VPackObjectBuilder data(&_builder, "data", true);
data->add("name", VPackValue(col->name()));
} else if (_state == COLLECTION_CHANGE) {
auto stripped = rocksutils::stripObjectIds(collectionDef);
marker->add("type", VPackValue(REPLICATION_COLLECTION_CHANGE));
marker->add("data", stripped.first);
}
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
@ -421,28 +443,33 @@ class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext
TRI_ASSERT(_state != TRANSACTION || _trxDbId != 0);
TRI_ASSERT(_removedDocRid == 0);
_removedDocRid = 0;
uint64_t objectId = RocksDBKey::objectId(key);
auto dbCollPair = rocksutils::mapObjectToCollection(objectId);
TRI_voc_tick_t const dbid = dbCollPair.first;
TRI_voc_cid_t const cid = dbCollPair.second;
if (!shouldHandleCollection(dbid, cid)) {
return rocksdb::Status(); // no reset here
}
TRI_ASSERT(_state != TRANSACTION || _trxDbId == dbid);
TRI_vocbase_t* vocbase = loadVocbase(dbid);
LogicalCollection* col = loadCollection(dbid, cid);
TRI_ASSERT(vocbase != nullptr && col != nullptr);
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type", VPackValue(REPLICATION_MARKER_DOCUMENT));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValue(col->globallyUniqueId()));
marker->add("cuid", VPackValue(col->guid()));
marker->add("tid", VPackValue(std::to_string(_currentTrxId)));
marker->add("data", RocksDBValue::data(value));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
@ -451,6 +478,7 @@ class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext
resetTransientState(); // always reset after single op
}
}
return rocksdb::Status();
}
@ -471,36 +499,45 @@ class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext
auto triple = rocksutils::mapObjectToIndex(objectId);
TRI_voc_tick_t const dbid = std::get<0>(triple);
TRI_voc_cid_t const cid = std::get<1>(triple);
if (!shouldHandleCollection(dbid, cid)) {
_removedDocRid = 0; // ignore rid too
return rocksdb::Status(); // no reset here
}
StringRef docKey = RocksDBKey::primaryKey(key);
TRI_ASSERT(_state != TRANSACTION || _trxDbId == dbid);
TRI_vocbase_t* vocbase = loadVocbase(dbid);
LogicalCollection* col = loadCollection(dbid, cid);
TRI_ASSERT(vocbase != nullptr && col != nullptr);
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type", VPackValue(REPLICATION_MARKER_REMOVE));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValue(col->globallyUniqueId()));
marker->add("cuid", VPackValue(col->guid()));
marker->add("tid", VPackValue(std::to_string(_currentTrxId)));
VPackObjectBuilder data(&_builder, "data", true);
data->add(StaticStrings::KeyString, VPackValuePair(docKey.data(), docKey.size(),
VPackValueType::String));
data->add(StaticStrings::RevString, VPackValue(TRI_RidToString(_removedDocRid)));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
_removedDocRid = 0; // always reset
if (_state == SINGLE_REMOVE) {
resetTransientState();
}
return rocksdb::Status();
}

View File

@ -2349,7 +2349,7 @@ static void JS_GloballyUniqueIdVocbaseCol(
TRI_V8_THROW_EXCEPTION_INTERNAL("cannot extract collection");
}
std::string uniqueId = collection->globallyUniqueId();
auto& uniqueId = collection->guid();
TRI_V8_RETURN(TRI_V8_ASCII_STD_STRING(isolate, uniqueId));
TRI_V8_TRY_CATCH_END

View File

@ -24,46 +24,27 @@
#include "LogicalCollection.h"
#include "Aql/PlanCache.h"
#include "Aql/QueryCache.h"
#include "Basics/conversions.h"
#include "Basics/fasthash.h"
#include "Basics/LocalTaskQueue.h"
#include "Basics/PerformanceLogScope.h"
#include "Basics/ReadLocker.h"
#include "Basics/Result.h"
#include "Basics/StaticStrings.h"
#include "Basics/StringUtils.h"
#include "Basics/StringRef.h"
#include "Basics/VelocyPackHelper.h"
#include "Basics/WriteLocker.h"
#include "Basics/encoding.h"
#include "Cluster/ClusterInfo.h"
#include "Cluster/ClusterMethods.h"
#include "Cluster/FollowerInfo.h"
#include "Cluster/ServerState.h"
#include "Indexes/Index.h"
#include "Indexes/IndexIterator.h"
#include "RestServer/DatabaseFeature.h"
#include "RestServer/ServerIdFeature.h"
#include "Scheduler/Scheduler.h"
#include "Scheduler/SchedulerFeature.h"
#include "StorageEngine/EngineSelectorFeature.h"
#include "StorageEngine/PhysicalCollection.h"
#include "StorageEngine/StorageEngine.h"
#include "StorageEngine/TransactionState.h"
#include "Transaction/Helpers.h"
#include "Transaction/StandaloneContext.h"
#include "Utils/CollectionNameResolver.h"
#include "Utils/OperationOptions.h"
#include "Utils/SingleCollectionTransaction.h"
#include "Utils/VersionTracker.h"
#include "VocBase/KeyGenerator.h"
#include "VocBase/ManagedDocumentResult.h"
#include "VocBase/ticks.h"
#include "VocBase/Methods/Indexes.h"
#include <velocypack/Collection.h>
#include <velocypack/Iterator.h>
#include <velocypack/velocypack-aliases.h>
using namespace arangodb;
@ -90,46 +71,32 @@ static std::string translateStatus(TRI_vocbase_col_status_e status) {
}
}
static TRI_voc_cid_t ReadCid(VPackSlice info) {
if (!info.isObject()) {
// ERROR CASE
return 0;
std::string ReadGloballyUniqueId(arangodb::velocypack::Slice info) {
static const std::string empty;
auto guid = arangodb::basics::VelocyPackHelper::getStringValue(
info,
"globallyUniqueId",
empty
);
if (!guid.empty()) {
return guid;
}
// Somehow the cid is now propagated to dbservers
TRI_voc_cid_t cid = Helper::extractIdValue(info);
auto version = arangodb::basics::VelocyPackHelper::readNumericValue<uint32_t>(
info,
"version",
LogicalCollection::currentVersion()
);
if (cid == 0) {
if (ServerState::instance()->isDBServer()) {
cid = ClusterInfo::instance()->uniqid(1);
} else if (ServerState::instance()->isCoordinator()) {
cid = ClusterInfo::instance()->uniqid(1);
} else {
cid = TRI_NewTickServer();
}
}
return cid;
}
static TRI_voc_cid_t ReadPlanId(VPackSlice info, TRI_voc_cid_t cid) {
if (!info.isObject()) {
// ERROR CASE
return 0;
}
VPackSlice id = info.get("planId");
if (id.isNone()) {
return cid;
// predictable UUID for legacy collections
if (version < LogicalCollection::CollectionVersions::VERSION_33) {
return info.isObject()
? arangodb::basics::VelocyPackHelper::getStringValue(info, "name", empty)
: empty;
}
if (id.isString()) {
// string cid, e.g. "9988488"
return arangodb::basics::StringUtils::uint64(id.copyString());
} else if (id.isNumber()) {
// numeric cid, e.g. 9988488
return id.getNumericValue<uint64_t>();
}
// TODO Throw error for invalid type?
return cid;
return empty;
}
std::string ReadStringValue(
@ -191,17 +158,14 @@ LogicalCollection::LogicalCollection(LogicalCollection const& other)
_shardIds(new ShardMap()), // Not needed
_keyOptions(other._keyOptions),
_keyGenerator(KeyGenerator::factory(VPackSlice(keyOptions()))),
_globallyUniqueId(other._globallyUniqueId),
_physical(other.getPhysical()->clone(this)),
_clusterEstimateTTL(0) {
TRI_ASSERT(_physical != nullptr);
if (ServerState::instance()->isDBServer() ||
!ServerState::instance()->isRunningInCluster()) {
_followers.reset(new FollowerInfo(this));
}
TRI_ASSERT(!_globallyUniqueId.empty());
}
// The Slice contains the part of the plan that
@ -215,8 +179,9 @@ LogicalCollection::LogicalCollection(
category(),
ReadType(info, "type", TRI_COL_TYPE_UNKNOWN),
vocbase,
ReadCid(info),
ReadPlanId(info, 0),
arangodb::basics::VelocyPackHelper::extractIdValue(info),
ReadGloballyUniqueId(info),
arangodb::basics::VelocyPackHelper::stringUInt64(info.get("planId")),
ReadStringValue(info, "name", ""),
planVersion,
TRI_vocbase_t::IsSystemName(ReadStringValue(info, "name", ""))
@ -242,7 +207,6 @@ LogicalCollection::LogicalCollection(
_shardIds(new ShardMap()),
_keyOptions(nullptr),
_keyGenerator(),
_globallyUniqueId(Helper::getStringValue(info, "globallyUniqueId", "")),
_physical(
EngineSelectorFeature::ENGINE->createPhysicalCollection(this, info)),
_clusterEstimateTTL(0) {
@ -261,12 +225,7 @@ LogicalCollection::LogicalCollection(
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_FAILED, errorMsg);
}
if (_globallyUniqueId.empty()) {
// no id found. generate a new one
_globallyUniqueId = generateGloballyUniqueId();
}
TRI_ASSERT(!_globallyUniqueId.empty());
TRI_ASSERT(!guid().empty());
// add keyOptions from slice
VPackSlice keyOpts = info.get("keyOptions");
@ -462,14 +421,6 @@ uint32_t LogicalCollection::internalVersion() const { return _internalVersion; }
TRI_col_type_e LogicalCollection::type() const { return _type; }
std::string LogicalCollection::globallyUniqueId() const {
if (!_globallyUniqueId.empty()) {
return _globallyUniqueId;
}
return generateGloballyUniqueId();
}
std::string const LogicalCollection::distributeShardsLike() const {
return _distributeShardsLike;
}
@ -813,7 +764,7 @@ void LogicalCollection::toVelocyPack(VPackBuilder& result, bool translateCids,
result.add("deleted", VPackValue(deleted()));
result.add("isSystem", VPackValue(system()));
result.add("waitForSync", VPackValue(_waitForSync));
result.add("globallyUniqueId", VPackValue(_globallyUniqueId));
result.add("globallyUniqueId", VPackValue(guid()));
// TODO is this still releveant or redundant in keyGenerator?
result.add("allowUserKeys", VPackValue(_allowUserKeys));
@ -1339,43 +1290,6 @@ Result LogicalCollection::compareChecksums(VPackSlice checksumSlice, std::string
return Result();
}
std::string LogicalCollection::generateGloballyUniqueId() const {
if (_version < VERSION_33) {
return name(); // predictable UUID for legacy collections
}
ServerState::RoleEnum role = ServerState::instance()->getRole();
std::string result;
result.reserve(64);
if (ServerState::isCoordinator(role)) {
TRI_ASSERT(planId());
result.append("c");
result.append(std::to_string(planId()));
} else if (ServerState::isDBServer(role)) {
TRI_ASSERT(planId());
result.append("c");
// we add the shard name to the collection. If we ever
// replicate shards, we can identify them cluster-wide
result.append(std::to_string(planId()));
result.push_back('/');
result.append(name());
} else { // single server
if (system()) { // system collection can't be renamed
result.append(name());
} else {
TRI_ASSERT(id());
result.append("h");
char buff[sizeof(TRI_server_id_t) * 2 + 1];
size_t len = TRI_StringUInt64HexInPlace(ServerIdFeature::getId(), buff);
result.append(buff, len);
TRI_ASSERT(result.size() > 3);
result.push_back('/');
result.append(std::to_string(id()));
}
}
return result;
}
// -----------------------------------------------------------------------------
// --SECTION-- END-OF-FILE
// -----------------------------------------------------------------------------
// -----------------------------------------------------------------------------

View File

@ -354,17 +354,14 @@ class LogicalCollection: public LogicalDataSource {
void increaseInternalVersion();
std::string generateGloballyUniqueId() const;
protected:
virtual void includeVelocyPackEnterprise(velocypack::Builder& result) const;
protected:
// SECTION: Meta Information
//
// @brief Internal version used for caching
uint32_t _internalVersion;
bool const _isAStub;
// @brief Collection type
@ -412,10 +409,6 @@ class LogicalCollection: public LogicalDataSource {
_keyOptions; // options for key creation
std::unique_ptr<KeyGenerator> _keyGenerator;
/// @brief globally unique collection id. assigned by the
/// initial creator of the collection
std::string _globallyUniqueId;
std::unique_ptr<PhysicalCollection> _physical;
mutable basics::ReadWriteLock _lock; // lock protecting the status and name
@ -429,4 +422,4 @@ class LogicalCollection: public LogicalDataSource {
} // namespace arangodb
#endif
#endif

View File

@ -23,10 +23,79 @@
#include <mutex>
#include "Basics/conversions.h"
#include "Basics/VelocyPackHelper.h"
#include "Cluster/ClusterInfo.h"
#include "Cluster/ServerState.h"
#include "RestServer/ServerIdFeature.h"
#include "velocypack/StringRef.h"
#include "VocBase/LogicalCollection.h"
#include "VocBase/ticks.h"
#include "LogicalDataSource.h"
namespace {
std::string ensureGuid(
std::string&& guid,
TRI_voc_cid_t id,
TRI_voc_cid_t planId,
std::string const& name,
bool isSystem
) {
if (!guid.empty()) {
return std::move(guid);
}
guid.reserve(64);
if (arangodb::ServerState::instance()->isCoordinator()) {
TRI_ASSERT(planId);
guid.append("c");
guid.append(std::to_string(planId));
} else if (arangodb::ServerState::instance()->isDBServer()) {
TRI_ASSERT(planId);
guid.append("c");
// we add the shard name to the collection. If we ever
// replicate shards, we can identify them cluster-wide
guid.append(std::to_string(planId));
guid.push_back('/');
guid.append(name);
} else if (isSystem) {
guid.append(name);
} else {
char buf[sizeof(TRI_server_id_t) * 2 + 1];
auto len =
TRI_StringUInt64HexInPlace(arangodb::ServerIdFeature::getId(), buf);
TRI_ASSERT(id);
guid.append("h");
guid.append(buf, len);
TRI_ASSERT(guid.size() > 3);
guid.push_back('/');
guid.append(std::to_string(id));
}
return std::move(guid);
}
TRI_voc_cid_t ensureId(TRI_voc_cid_t id) {
if (id) {
return id;
}
if (arangodb::ServerState::instance()->isCoordinator()
|| arangodb::ServerState::instance()->isDBServer()) {
auto* ci = arangodb::ClusterInfo::instance();
return ci ? ci->uniqid(1) : 0;
}
return TRI_NewTickServer();
}
} // namespace
namespace arangodb {
/*static*/ LogicalDataSource::Type const& LogicalDataSource::Type::emplace(
@ -54,6 +123,31 @@ namespace arangodb {
return itr.first->second;
}
LogicalDataSource::LogicalDataSource(
Category const& category,
Type const& type,
TRI_vocbase_t& vocbase,
TRI_voc_cid_t id,
std::string&& guid,
TRI_voc_cid_t planId,
std::string&& name,
uint64_t planVersion,
bool system,
bool deleted
): _name(std::move(name)),
_category(category),
_type(type),
_vocbase(vocbase),
_id(ensureId(id)),
_planId(planId ? planId : _id),
_planVersion(planVersion),
_guid(ensureGuid(std::move(guid), _id, _planId, _name, system)),
_deleted(deleted),
_system(system) {
TRI_ASSERT(_id);
TRI_ASSERT(!_guid.empty());
}
} // arangodb
// -----------------------------------------------------------------------------

View File

@ -83,27 +83,24 @@ class LogicalDataSource {
Type& operator=(Type&&) noexcept = delete;
};
//////////////////////////////////////////////////////////////////////////////
/// @brief constructor for a logical data-source
/// @note 'id' autogenerated IFF 'id' == 0
/// @note 'planId' taken from evaluated value of 'id' IFF 'planId' == 0
/// @note 'guid' autogenerated IFF 'guid'.empty()
//////////////////////////////////////////////////////////////////////////////
LogicalDataSource(
Category const& category,
Type const& type,
TRI_vocbase_t& vocbase,
TRI_voc_cid_t id,
TRI_voc_cid_t planId,
std::string&& name,
uint64_t planVersion,
bool system,
bool deleted
) noexcept
: _name(std::move(name)),
_category(category),
_type(type),
_vocbase(vocbase),
_id(id),
_planId(planId ? planId : id),
_planVersion(planVersion),
_deleted(deleted),
_system(system) {
}
Category const& category,
Type const& type,
TRI_vocbase_t& vocbase,
TRI_voc_cid_t id,
std::string&& guid,
TRI_voc_cid_t planId,
std::string&& name,
uint64_t planVersion,
bool system,
bool deleted
);
LogicalDataSource(LogicalDataSource const& other)
: _name(other._name),
@ -113,6 +110,7 @@ class LogicalDataSource {
_id(other._id),
_planId(other._planId),
_planVersion(other._planVersion),
_guid(other._guid),
_deleted(other._deleted),
_system(other._system) {
}
@ -122,6 +120,7 @@ class LogicalDataSource {
Category const& category() const noexcept { return _category; }
bool deleted() const noexcept { return _deleted; }
virtual arangodb::Result drop() = 0;
std::string const& guid() const noexcept { return _guid; }
TRI_voc_cid_t const& id() const noexcept { return _id; } // reference required for ShardDistributionReporterTest
std::string const& name() const noexcept { return _name; }
TRI_voc_cid_t planId() const noexcept { return _planId; }
@ -136,7 +135,7 @@ class LogicalDataSource {
void name(std::string&& name) noexcept { _name = std::move(name); }
private:
// members ordered by sizeof(decltype(..))
// members ordered by sizeof(decltype(..)) except for '_guid'
std::string _name; // data-source name
Category const& _category; // the category of the logical data-source
Type const& _type; // the type of the underlying data-source implementation
@ -148,6 +147,7 @@ class LogicalDataSource {
// to represent a cluster wide collection. This is
// then the version in the agency Plan that underpins
// the information in this object. Otherwise 0.
std::string const _guid; // globally unique data-source id (cluster-wide) for proper initialization must be positioned after '_name' and '_planId' since they are autogenerated
bool _deleted; // data-source marked as deleted
bool const _system; // this instance represents a system data-source
};

View File

@ -24,15 +24,13 @@
#include "LogicalView.h"
#include "RestServer/ViewTypesFeature.h"
#include "Basics/ReadLocker.h"
#include "Basics/Result.h"
#include "Basics/VelocyPackHelper.h"
#include "Basics/Exceptions.h"
#include "Cluster/ClusterInfo.h"
#include "Cluster/ServerState.h"
#include "StorageEngine/EngineSelectorFeature.h"
#include "StorageEngine/StorageEngine.h"
#include "velocypack/Iterator.h"
#include "VocBase/ticks.h"
#include "VocBase/vocbase.h"
using Helper = arangodb::basics::VelocyPackHelper;
@ -56,55 +54,6 @@ bool ReadIsSystem(arangodb::velocypack::Slice info) {
);
}
TRI_voc_cid_t ReadPlanId(VPackSlice info, TRI_voc_cid_t vid) {
if (!info.isObject()) {
// ERROR CASE
return 0;
}
VPackSlice id = info.get("planId");
if (id.isNone()) {
return vid;
}
if (id.isString()) {
// string cid, e.g. "9988488"
return arangodb::basics::StringUtils::uint64(id.copyString());
} else if (id.isNumber()) {
// numeric cid, e.g. 9988488
return id.getNumericValue<uint64_t>();
}
// TODO Throw error for invalid type?
return vid;
}
/*static*/ TRI_voc_cid_t ReadViewId(VPackSlice info) {
if (!info.isObject()) {
// ERROR CASE
return 0;
}
// Somehow the id is now propagated to dbservers
TRI_voc_cid_t id = Helper::extractIdValue(info);
if (id) {
return id;
}
if (arangodb::ServerState::instance()->isDBServer()) {
auto* ci = arangodb::ClusterInfo::instance();
return ci ? ci->uniqid(1) : 0;
}
if (arangodb::ServerState::instance()->isCoordinator()) {
auto* ci = arangodb::ClusterInfo::instance();
return ci ? ci->uniqid(1) : 0;
}
return TRI_NewTickServer();
}
} // namespace
namespace arangodb {
@ -126,13 +75,22 @@ LogicalView::LogicalView(
arangodb::basics::VelocyPackHelper::getStringRef(definition, "type", "")
),
vocbase,
ReadViewId(definition),
ReadPlanId(definition, 0),
arangodb::basics::VelocyPackHelper::extractIdValue(definition),
arangodb::basics::VelocyPackHelper::getStringValue(definition, "globallyUniqueId", ""),
arangodb::basics::VelocyPackHelper::stringUInt64(definition.get("planId")),
arangodb::basics::VelocyPackHelper::getStringValue(definition, "name", ""),
planVersion,
ReadIsSystem(definition),
Helper::readBooleanValue(definition, "deleted", false)
) {
// ensure that the 'definition' was used as the configuration source
if (!definition.isObject()) {
THROW_ARANGO_EXCEPTION_MESSAGE(
TRI_ERROR_BAD_PARAMETER,
"got an invalid view definition while constructing LogicalView"
);
}
if (!TRI_vocbase_t::IsAllowedName(definition)) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_ARANGO_ILLEGAL_NAME);
}
@ -327,6 +285,7 @@ void DBServerLogicalView::toVelocyPack(
if (includeSystem) {
result.add("deleted", VPackValue(deleted()));
result.add("globallyUniqueId", VPackValue(guid()));
result.add("isSystem", VPackValue(system()));
// FIXME not sure if the following is relevant

View File

@ -229,7 +229,7 @@ void TRI_vocbase_t::signalCleanup() {
void TRI_vocbase_t::checkCollectionInvariants() const {
TRI_ASSERT(_dataSourceByName.size() == _dataSourceById.size());
TRI_ASSERT(_dataSourceByUuid.size() <= _dataSourceById.size()); // does not contain views
TRI_ASSERT(_dataSourceByUuid.size() == _dataSourceById.size());
}
/// @brief adds a new collection
@ -274,11 +274,11 @@ void TRI_vocbase_t::registerCollection(
}
try {
auto it2 = _dataSourceByUuid.emplace(collection->globallyUniqueId(), collection);
auto it2 = _dataSourceByUuid.emplace(collection->guid(), collection);
if (!it2.second) {
std::string msg;
msg.append(std::string("duplicate entry for collection uuid '") + collection->globallyUniqueId() + "'");
msg.append(std::string("duplicate entry for collection uuid '") + collection->guid() + "'");
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << msg;
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_ARANGO_DUPLICATE_IDENTIFIER, msg);
@ -294,7 +294,7 @@ void TRI_vocbase_t::registerCollection(
} catch (...) {
_dataSourceByName.erase(name);
_dataSourceById.erase(cid);
_dataSourceByUuid.erase(collection->globallyUniqueId());
_dataSourceByUuid.erase(collection->guid());
throw;
}
@ -327,7 +327,7 @@ bool TRI_vocbase_t::unregisterCollection(
// this is because someone else might have created a new collection with the
// same name, but with a different id
_dataSourceByName.erase(collection->name());
_dataSourceByUuid.erase(collection->globallyUniqueId());
_dataSourceByUuid.erase(collection->guid());
// post-condition
checkCollectionInvariants();
@ -369,7 +369,7 @@ void TRI_vocbase_t::registerView(
_dataSourceByName.erase(name);
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "duplicate view identifier " << view->id() << " for name '"
<< "duplicate view identifier '" << view->id() << "' for name '"
<< name << "'";
THROW_ARANGO_EXCEPTION(TRI_ERROR_ARANGO_DUPLICATE_IDENTIFIER);
@ -379,6 +379,21 @@ void TRI_vocbase_t::registerView(
throw;
}
try {
auto it2 = _dataSourceByUuid.emplace(view->guid(), view);
if (!it2.second) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "duplicate view globally-unique identifier '" << view->guid() << "' for name '" << name << "'";
THROW_ARANGO_EXCEPTION(TRI_ERROR_ARANGO_DUPLICATE_IDENTIFIER);
}
} catch (...) {
_dataSourceByName.erase(name);
_dataSourceById.erase(id);
throw;
}
checkCollectionInvariants();
}
}
@ -405,6 +420,7 @@ bool TRI_vocbase_t::unregisterView(arangodb::LogicalView const& view) {
// this is because someone else might have created a new view with the
// same name, but with a different id
_dataSourceByName.erase(view.name());
_dataSourceByUuid.erase(view.guid());
// post-condition
checkCollectionInvariants();

View File

@ -252,7 +252,8 @@ SECTION("test_defaults") {
arangodb::iresearch::IResearchViewMeta meta;
std::string error;
CHECK(7 == slice.length());
CHECK((8 == slice.length()));
CHECK((slice.hasKey("globallyUniqueId") && slice.get("globallyUniqueId").isString() && false == slice.get("globallyUniqueId").copyString().empty()));
CHECK(slice.get("name").copyString() == "testView");
CHECK(slice.get("type").copyString() == arangodb::iresearch::DATA_SOURCE_TYPE.name());
CHECK(false == slice.get("deleted").getBool());

View File

@ -511,8 +511,9 @@ SECTION("test_toVelocyPack") {
wiew->toVelocyPack(builder, false, true);
builder.close();
auto slice = builder.slice();
CHECK((6 == slice.length()));
CHECK((7 == slice.length()));
CHECK((slice.hasKey("deleted") && slice.get("deleted").isBoolean() && false == slice.get("deleted").getBoolean()));
CHECK((slice.hasKey("globallyUniqueId") && slice.get("globallyUniqueId").isString() && false == slice.get("globallyUniqueId").copyString().empty()));
CHECK((slice.hasKey("id") && slice.get("id").isString() && std::string("3") == slice.get("id").copyString()));
CHECK((slice.hasKey("isSystem") && slice.get("isSystem").isBoolean() && false == slice.get("isSystem").getBoolean()));
CHECK((slice.hasKey("name") && slice.get("name").isString() && std::string("testView") == slice.get("name").copyString()));

View File

@ -120,6 +120,81 @@ SECTION("test_category") {
}
}
SECTION("test_construct") {
// LogicalCollection
{
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, 1, "testVocbase");
auto json = arangodb::velocypack::Parser::fromJson("{ \"id\": 1, \"planId\": 2, \"globallyUniqueId\": \"abc\", \"name\": \"testCollection\" }");
arangodb::LogicalCollection instance(vocbase, json->slice(), true);
CHECK((1 == instance.id()));
CHECK((2 == instance.planId()));
CHECK((std::string("abc") == instance.guid()));
}
// LogicalView
{
class LogicalViewImpl: public arangodb::LogicalView {
public:
LogicalViewImpl(TRI_vocbase_t& vocbase, arangodb::velocypack::Slice const& definition)
: LogicalView(vocbase, definition, 0) {
}
virtual arangodb::Result drop() override { return arangodb::Result(); }
virtual void open() override {}
virtual arangodb::Result rename(std::string&& newName, bool doSync) override { return arangodb::Result(); }
virtual void toVelocyPack(arangodb::velocypack::Builder& result, bool includeProperties, bool includeSystem) const override {}
virtual arangodb::Result updateProperties(arangodb::velocypack::Slice const& properties, bool partialUpdate, bool doSync) override { return arangodb::Result(); }
virtual bool visitCollections(CollectionVisitor const& visitor) const override { return true; }
};
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, 1, "testVocbase");
auto json = arangodb::velocypack::Parser::fromJson("{ \"id\": 1, \"planId\": 2, \"globallyUniqueId\": \"abc\", \"name\": \"testView\" }");
LogicalViewImpl instance(vocbase, json->slice());
CHECK((1 == instance.id()));
CHECK((2 == instance.planId()));
CHECK((std::string("abc") == instance.guid()));
}
}
SECTION("test_defaults") {
// LogicalCollection
{
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, 1, "testVocbase");
auto json = arangodb::velocypack::Parser::fromJson("{ \"name\": \"testCollection\" }");
arangodb::LogicalCollection instance(vocbase, json->slice(), true);
CHECK((0 != instance.id()));
CHECK((0 != instance.planId()));
CHECK((false == instance.guid().empty()));
}
// LogicalView
{
class LogicalViewImpl: public arangodb::LogicalView {
public:
LogicalViewImpl(TRI_vocbase_t& vocbase, arangodb::velocypack::Slice const& definition)
: LogicalView(vocbase, definition, 0) {
}
virtual arangodb::Result drop() override { return arangodb::Result(); }
virtual void open() override {}
virtual arangodb::Result rename(std::string&& newName, bool doSync) override { return arangodb::Result(); }
virtual void toVelocyPack(arangodb::velocypack::Builder& result, bool includeProperties, bool includeSystem) const override {}
virtual arangodb::Result updateProperties(arangodb::velocypack::Slice const& properties, bool partialUpdate, bool doSync) override { return arangodb::Result(); }
virtual bool visitCollections(CollectionVisitor const& visitor) const override { return true; }
};
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, 1, "testVocbase");
auto json = arangodb::velocypack::Parser::fromJson("{ \"name\": \"testView\" }");
LogicalViewImpl instance(vocbase, json->slice());
CHECK((0 != instance.id()));
CHECK((0 != instance.planId()));
CHECK((instance.id() == instance.planId()));
CHECK((false == instance.guid().empty()));
}
}
////////////////////////////////////////////////////////////////////////////////
/// @brief generate tests
////////////////////////////////////////////////////////////////////////////////