1
0
Fork 0

Moved indexes down into the physical collections

This commit is contained in:
Michael Hackstein 2017-03-08 11:03:45 +01:00
parent e6ab0afb50
commit 580d2a4336
6 changed files with 366 additions and 308 deletions

View File

@ -22,6 +22,7 @@
////////////////////////////////////////////////////////////////////////////////
#include "MMFilesCollection.h"
#include "Aql/PlanCache.h"
#include "Aql/QueryCache.h"
#include "ApplicationFeatures/ApplicationServer.h"
#include "Basics/FileUtils.h"
@ -446,6 +447,8 @@ MMFilesCollection::MMFilesCollection(LogicalCollection* collection,
TRI_JOURNAL_DEFAULT_SIZE))),
_isVolatile(arangodb::basics::VelocyPackHelper::readBooleanValue(
info, "isVolatile", false)),
_cleanupIndexes(0),
_persistentIndexes(0),
_indexBuckets(Helper::readNumericValue<uint32_t>(
info, "indexBuckets", DatabaseFeature::defaultIndexBuckets())),
_useSecondaryIndexes(true),
@ -489,8 +492,14 @@ MMFilesCollection::MMFilesCollection(LogicalCollection* logical,
_path = mmfiles._path;
_doCompact = mmfiles._doCompact;
_maxTick = mmfiles._maxTick;
setCompactionStatus("compaction not yet started");
// Copy over index definitions
_indexes.reserve(mmfiles._indexes.size());
for (auto const& idx : mmfiles._indexes) {
_indexes.emplace_back(idx);
}
setCompactionStatus("compaction not yet started");
// not copied
// _datafiles; // all datafiles
// _journals; // all journals
@ -498,7 +507,6 @@ MMFilesCollection::MMFilesCollection(LogicalCollection* logical,
// _uncollectedLogfileEntries = mmfiles.uncollectedLogfileEntries; //TODO FIXME
// _datafileStatistics;
// _revisionsCache;
}
MMFilesCollection::~MMFilesCollection() {
@ -549,7 +557,7 @@ int MMFilesCollection::close() {
}
// We also have to unload the indexes.
for (auto& idx : *(_logicalCollection->indexList())) {
for (auto& idx : _indexes) {
idx->unload();
}
@ -1445,7 +1453,7 @@ arangodb::MMFilesPrimaryIndex* MMFilesCollection::primaryIndex() const {
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "got invalid indexes for collection '" << _logicalCollection->name()
<< "'";
for (auto const& it : *(_logicalCollection->indexList())) {
for (auto const& it : _indexes) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << "- " << it.get();
}
}
@ -1456,7 +1464,7 @@ arangodb::MMFilesPrimaryIndex* MMFilesCollection::primaryIndex() const {
}
int MMFilesCollection::fillAllIndexes(transaction::Methods* trx) {
return fillIndexes(trx, *(_logicalCollection->indexList()));
return fillIndexes(trx, _indexes);
}
/// @brief Fill the given list of Indexes
@ -1708,7 +1716,7 @@ void MMFilesCollection::open(bool ignoreErrors) {
if (!engine->inRecovery()) {
// build the index structures, and fill the indexes
fillIndexes(&trx, *(_logicalCollection->indexList()));
fillIndexes(&trx, _indexes);
}
// successfully opened collection. now adjust version number
@ -1829,6 +1837,146 @@ bool MMFilesCollection::readDocumentConditional(
return false;
}
void MMFilesCollection::prepareIndexes(VPackSlice indexesSlice) {
createInitialIndexes();
if (indexesSlice.isArray()) {
StorageEngine* engine = EngineSelectorFeature::ENGINE;
IndexFactory const* idxFactory = engine->indexFactory();
TRI_ASSERT(idxFactory != nullptr);
for (auto const& v : VPackArrayIterator(indexesSlice)) {
if (arangodb::basics::VelocyPackHelper::getBooleanValue(v, "error",
false)) {
// We have an error here.
// Do not add index.
// TODO Handle Properly
continue;
}
auto idx =
idxFactory->prepareIndexFromSlice(v, false, _logicalCollection, true);
if (idx->type() == Index::TRI_IDX_TYPE_PRIMARY_INDEX ||
idx->type() == Index::TRI_IDX_TYPE_EDGE_INDEX) {
continue;
}
if (ServerState::instance()->isRunningInCluster()) {
addIndexCoordinator(idx, false);
} else {
addIndex(idx);
}
}
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
if (_indexes[0]->type() != Index::IndexType::TRI_IDX_TYPE_PRIMARY_INDEX) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << "got invalid indexes for collection '" << _logicalCollection->name() << "'";
for (auto const& it : _indexes) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << "- " << it.get();
}
}
#endif
}
std::shared_ptr<Index> MMFilesCollection::lookupIndex(
VPackSlice const& info) const {
TRI_ASSERT(info.isObject());
// extract type
VPackSlice value = info.get("type");
if (!value.isString()) {
// Compatibility with old v8-vocindex.
THROW_ARANGO_EXCEPTION(TRI_ERROR_OUT_OF_MEMORY);
}
std::string tmp = value.copyString();
arangodb::Index::IndexType const type = arangodb::Index::type(tmp.c_str());
for (auto const& idx : _indexes) {
if (idx->type() == type) {
// Only check relevant indices
if (idx->matchesDefinition(info)) {
// We found an index for this definition.
return idx;
}
}
}
return nullptr;
}
/// @brief creates the initial indexes for the collection
void MMFilesCollection::createInitialIndexes() {
if (!_indexes.empty()) {
return;
}
std::vector<std::shared_ptr<arangodb::Index>> systemIndexes;
StorageEngine* engine = EngineSelectorFeature::ENGINE;
IndexFactory const* idxFactory = engine->indexFactory();
TRI_ASSERT(idxFactory != nullptr);
idxFactory->fillSystemIndexes(_logicalCollection, systemIndexes);
for (auto const& it : systemIndexes) {
addIndex(it);
}
}
std::shared_ptr<Index> MMFilesCollection::createIndex(transaction::Methods* trx,
VPackSlice const& info,
bool& created) {
// TODO Get LOCK for the vocbase
auto idx = lookupIndex(info);
if (idx != nullptr) {
created = false;
// We already have this index.
return idx;
}
StorageEngine* engine = EngineSelectorFeature::ENGINE;
IndexFactory const* idxFactory = engine->indexFactory();
TRI_ASSERT(idxFactory != nullptr);
// We are sure that we do not have an index of this type.
// We also hold the lock.
// Create it
idx =
idxFactory->prepareIndexFromSlice(info, true, _logicalCollection, false);
TRI_ASSERT(idx != nullptr);
if (ServerState::instance()->isCoordinator()) {
// In the coordinator case we do not fill the index
// We only inform the others.
addIndexCoordinator(idx, true);
created = true;
return idx;
}
int res = saveIndex(trx, idx);
if (res != TRI_ERROR_NO_ERROR) {
THROW_ARANGO_EXCEPTION(res);
}
arangodb::aql::PlanCache::instance()->invalidate(_logicalCollection->vocbase());
// Until here no harm is done if sth fails. The shared ptr will clean up. if
// left before
addIndex(idx);
{
bool const doSync =
application_features::ApplicationServer::getFeature<DatabaseFeature>(
"Database")
->forceSyncProperties();
VPackBuilder builder = _logicalCollection->toVelocyPackIgnore({"path", "statusString"}, true);
_logicalCollection->updateProperties(builder.slice(), doSync);
}
created = true;
return idx;
}
/// @brief Persist an index information to file
int MMFilesCollection::saveIndex(transaction::Methods* trx, std::shared_ptr<arangodb::Index> idx) {
TRI_ASSERT(!ServerState::instance()->isCoordinator());
@ -1881,6 +2029,48 @@ int MMFilesCollection::saveIndex(transaction::Methods* trx, std::shared_ptr<aran
return res;
}
void MMFilesCollection::addIndex(std::shared_ptr<arangodb::Index> idx) {
// primary index must be added at position 0
TRI_ASSERT(idx->type() != arangodb::Index::TRI_IDX_TYPE_PRIMARY_INDEX ||
_indexes.empty());
auto const id = idx->id();
for (auto const& it : _indexes) {
if (it->id() == id) {
// already have this particular index. do not add it again
return;
}
}
TRI_UpdateTickServer(static_cast<TRI_voc_tick_t>(id));
_indexes.emplace_back(idx);
// update statistics
if (idx->type() == arangodb::Index::TRI_IDX_TYPE_FULLTEXT_INDEX) {
++_cleanupIndexes;
}
if (idx->isPersistent()) {
++_persistentIndexes;
}
}
void MMFilesCollection::addIndexCoordinator(
std::shared_ptr<arangodb::Index> idx, bool distribute) {
auto const id = idx->id();
for (auto const& it : _indexes) {
if (it->id() == id) {
// already have this particular index. do not add it again
return;
}
}
_indexes.emplace_back(idx);
if (distribute) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_NOT_IMPLEMENTED);
}
}
int MMFilesCollection::restoreIndex(transaction::Methods* trx,
VPackSlice const& info,
std::shared_ptr<arangodb::Index>& idx) {
@ -1920,7 +2110,7 @@ int MMFilesCollection::restoreIndex(transaction::Methods* trx,
return res;
}
_logicalCollection->addIndex(newIdx);
addIndex(newIdx);
idx = newIdx;
return TRI_ERROR_NO_ERROR;
}
@ -1933,7 +2123,7 @@ bool MMFilesCollection::dropIndex(TRI_idx_iid_t iid) {
}
auto vocbase = _logicalCollection->vocbase();
if (!_logicalCollection->removeIndex(iid)) {
if (!removeIndex(iid)) {
// We tried to remove an index that does not exist
events::DropIndex("", std::to_string(iid),
TRI_ERROR_ARANGO_INDEX_NOT_FOUND);
@ -1975,6 +2165,40 @@ bool MMFilesCollection::dropIndex(TRI_idx_iid_t iid) {
return true;
}
/// @brief removes an index by id
bool MMFilesCollection::removeIndex(TRI_idx_iid_t iid) {
size_t const n = _indexes.size();
for (size_t i = 0; i < n; ++i) {
auto idx = _indexes[i];
if (!idx->canBeDropped()) {
continue;
}
if (idx->id() == iid) {
// found!
idx->drop();
_indexes.erase(_indexes.begin() + i);
// update statistics
if (idx->type() == arangodb::Index::TRI_IDX_TYPE_FULLTEXT_INDEX) {
--_cleanupIndexes;
}
if (idx->isPersistent()) {
--_persistentIndexes;
}
return true;
}
}
// not found
return false;
}
/// @brief garbage-collect a collection's indexes
int MMFilesCollection::cleanupIndexes() {
@ -1982,11 +2206,9 @@ int MMFilesCollection::cleanupIndexes() {
// cleaning indexes is expensive, so only do it if the flag is set for the
// collection
// TODO FIXME
if (_logicalCollection->_cleanupIndexes > 0) {
if (_cleanupIndexes > 0) {
WRITE_LOCKER(writeLocker, _idxLock);
auto indexes = _logicalCollection->getIndexes();
for (auto& idx : indexes) {
for (auto& idx : _indexes) {
if (idx->type() == arangodb::Index::TRI_IDX_TYPE_FULLTEXT_INDEX) {
res = idx->cleanup();
@ -1996,7 +2218,6 @@ int MMFilesCollection::cleanupIndexes() {
}
}
}
return res;
}
@ -2565,14 +2786,13 @@ int MMFilesCollection::insertSecondaryIndexes(arangodb::transaction::Methods* tr
TRI_IF_FAILURE("InsertSecondaryIndexes") { return TRI_ERROR_DEBUG; }
bool const useSecondary = useSecondaryIndexes();
if (!useSecondary && _logicalCollection->_persistentIndexes == 0) {
if (!useSecondary && _persistentIndexes == 0) {
return TRI_ERROR_NO_ERROR;
}
int result = TRI_ERROR_NO_ERROR;
// TODO FIXME
auto indexes = _logicalCollection->getIndexes();
auto indexes = _indexes;
size_t const n = indexes.size();
for (size_t i = 1; i < n; ++i) {
@ -2610,7 +2830,7 @@ int MMFilesCollection::deleteSecondaryIndexes(arangodb::transaction::Methods* tr
TRI_ASSERT(!ServerState::instance()->isCoordinator());
bool const useSecondary = useSecondaryIndexes();
if (!useSecondary && _logicalCollection->_persistentIndexes == 0) {
if (!useSecondary && _persistentIndexes == 0) {
return TRI_ERROR_NO_ERROR;
}

View File

@ -263,12 +263,19 @@ class MMFilesCollection final : public PhysicalCollection {
int fillAllIndexes(transaction::Methods*);
int saveIndex(transaction::Methods* trx, std::shared_ptr<arangodb::Index> idx) override;
std::unique_ptr<IndexIterator> getAllIterator(transaction::Methods* trx, ManagedDocumentResult* mdr, bool reverse) override;
void prepareIndexes(arangodb::velocypack::Slice indexesSlice) override;
/// @brief Find index by definition
std::shared_ptr<Index> lookupIndex(velocypack::Slice const&) const override;
std::unique_ptr<IndexIterator> getAllIterator(transaction::Methods* trx, ManagedDocumentResult* mdr, bool reverse) override;
std::unique_ptr<IndexIterator> getAnyIterator(transaction::Methods* trx, ManagedDocumentResult* mdr) override;
void invokeOnAllElements(std::function<bool(DocumentIdentifierToken const&)> callback) override;
std::shared_ptr<Index> createIndex(transaction::Methods* trx,
arangodb::velocypack::Slice const& info,
bool& created) override;
/// @brief Restores an index from VelocyPack.
int restoreIndex(transaction::Methods*, velocypack::Slice const&,
std::shared_ptr<Index>&) override;
@ -359,6 +366,9 @@ class MMFilesCollection final : public PhysicalCollection {
private:
/// @brief creates the initial indexes for the collection
void createInitialIndexes();
bool openIndex(VPackSlice const& description, transaction::Methods* trx);
/// @brief initializes an index with all existing documents
@ -435,15 +445,25 @@ class MMFilesCollection final : public PhysicalCollection {
private:
void addIndex(std::shared_ptr<arangodb::Index> idx);
void addIndexCoordinator(std::shared_ptr<arangodb::Index> idx,
bool distribute);
bool removeIndex(TRI_idx_iid_t iid);
/// @brief return engine-specific figures
void figuresSpecific(std::shared_ptr<arangodb::velocypack::Builder>&) override;
// SECTION: Index storage
int saveIndex(transaction::Methods* trx,
std::shared_ptr<arangodb::Index> idx);
/// @brief Detect all indexes form file
int detectIndexes(transaction::Methods* trx);
int insertIndexes(transaction::Methods * trx, TRI_voc_rid_t revisionId,
int insertIndexes(transaction::Methods* trx, TRI_voc_rid_t revisionId,
velocypack::Slice const& doc);
int insertPrimaryIndex(transaction::Methods*, TRI_voc_rid_t revisionId,
@ -504,9 +524,13 @@ class MMFilesCollection final : public PhysicalCollection {
// SECTION: Indexes
size_t _cleanupIndexes;
size_t _persistentIndexes;
uint32_t _indexBuckets;
// whether or not secondary indexes should be filled
bool _useSecondaryIndexes;
bool _doCompact;
TRI_voc_tick_t _maxTick;
};

View File

@ -158,27 +158,17 @@ LogicalCollection::LogicalCollection(LogicalCollection const& other)
_isSystem(other.isSystem()),
_version(other._version),
_waitForSync(other.waitForSync()),
_indexes(),
_replicationFactor(other.replicationFactor()),
_numberOfShards(other.numberOfShards()),
_allowUserKeys(other.allowUserKeys()),
_shardIds(new ShardMap()), // Not needed
_vocbase(other.vocbase()),
_cleanupIndexes(0),
_persistentIndexes(0),
_physical(other.getPhysical()->clone(this, other.getPhysical())) {
TRI_ASSERT(_physical != nullptr);
if (ServerState::instance()->isDBServer() ||
!ServerState::instance()->isRunningInCluster()) {
_followers.reset(new FollowerInfo(this));
}
// Copy over index definitions
_indexes.reserve(other._indexes.size());
for (auto const& idx : other._indexes) {
_indexes.emplace_back(idx);
}
}
// @brief Constructor used in coordinator case.
@ -209,8 +199,6 @@ LogicalCollection::LogicalCollection(TRI_vocbase_t* vocbase,
_allowUserKeys(Helper::readBooleanValue(info, "allowUserKeys", true)),
_shardIds(new ShardMap()),
_vocbase(vocbase),
_cleanupIndexes(0),
_persistentIndexes(0),
_physical(
EngineSelectorFeature::ENGINE->createPhysicalCollection(this, info)) {
TRI_ASSERT(_physical != nullptr);
@ -218,6 +206,9 @@ LogicalCollection::LogicalCollection(TRI_vocbase_t* vocbase,
THROW_ARANGO_EXCEPTION(TRI_ERROR_ARANGO_ILLEGAL_NAME);
}
// This has to be called AFTER _phyiscal and _logical are properly linked together.
_physical->prepareIndexes(info.get("indexes"));
if (_version < minimumVersion()) {
// collection is too "old"
std::string errorMsg(std::string("collection '") + _name +
@ -353,46 +344,6 @@ LogicalCollection::LogicalCollection(TRI_vocbase_t* vocbase,
}
}
createInitialIndexes();
auto indexesSlice = info.get("indexes");
if (indexesSlice.isArray()) {
StorageEngine* engine = EngineSelectorFeature::ENGINE;
IndexFactory const* idxFactory = engine->indexFactory();
TRI_ASSERT(idxFactory != nullptr);
for (auto const& v : VPackArrayIterator(indexesSlice)) {
if (arangodb::basics::VelocyPackHelper::getBooleanValue(v, "error",
false)) {
// We have an error here.
// Do not add index.
// TODO Handle Properly
continue;
}
auto idx = idxFactory->prepareIndexFromSlice(v, false, this, true);
// TODO Move IndexTypeCheck out
if (idx->type() == Index::TRI_IDX_TYPE_PRIMARY_INDEX ||
idx->type() == Index::TRI_IDX_TYPE_EDGE_INDEX) {
continue;
}
if (isCluster) {
addIndexCoordinator(idx, false);
} else {
addIndex(idx);
}
}
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
if (_indexes[0]->type() != Index::IndexType::TRI_IDX_TYPE_PRIMARY_INDEX) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << "got invalid indexes for collection '" << _name << "'";
for (auto const& it : _indexes) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << "- " << it.get();
}
}
#endif
if (ServerState::instance()->isDBServer() ||
!ServerState::instance()->isRunningInCluster()) {
_followers.reset(new FollowerInfo(this));
@ -611,18 +562,12 @@ void LogicalCollection::setDeleted(bool newValue) { _isDeleted = newValue; }
// SECTION: Indexes
std::vector<std::shared_ptr<arangodb::Index>> const&
LogicalCollection::getIndexes() const {
return _indexes;
return getPhysical()->getIndexes();
}
void LogicalCollection::getIndexesVPack(VPackBuilder& result,
bool withFigures) const {
result.openArray();
for (auto const& idx : _indexes) {
result.openObject();
idx->toVelocyPack(result, withFigures);
result.close();
}
result.close();
getPhysical()->getIndexesVPack(result, withFigures);
}
// SECTION: Replication
@ -740,14 +685,7 @@ void LogicalCollection::drop() {
engine->dropCollection(_vocbase, this);
_isDeleted = true;
// save some memory by freeing the indexes
_indexes.clear();
try {
// close collection. this will also invalidate the revisions cache
_physical->close();
} catch (...) {
// don't throw from here... dropping should succeed
}
_physical->drop();
}
void LogicalCollection::setStatus(TRI_vocbase_col_status_e status) {
@ -909,9 +847,8 @@ CollectionResult LogicalCollection::updateProperties(VPackSlice const& slice,
/// @brief return the figures for a collection
std::shared_ptr<arangodb::velocypack::Builder> LogicalCollection::figures() {
auto builder = std::make_shared<VPackBuilder>();
if (ServerState::instance()->isCoordinator()) {
auto builder = std::make_shared<VPackBuilder>();
builder->openObject();
builder->close();
int res = figuresOnCoordinator(dbName(), cid_as_string(), builder);
@ -919,28 +856,9 @@ std::shared_ptr<arangodb::velocypack::Builder> LogicalCollection::figures() {
if (res != TRI_ERROR_NO_ERROR) {
THROW_ARANGO_EXCEPTION(res);
}
} else {
builder->openObject();
// add index information
size_t sizeIndexes = getPhysical()->memory();
size_t numIndexes = 0;
for (auto const& idx : _indexes) {
sizeIndexes += static_cast<size_t>(idx->memory());
++numIndexes;
}
builder->add("indexes", VPackValue(VPackValueType::Object));
builder->add("count", VPackValue(numIndexes));
builder->add("size", VPackValue(sizeIndexes));
builder->close(); // indexes
// add engine-specific figures
getPhysical()->figures(builder);
builder->close();
return builder;
}
return builder;
return getPhysical()->figures();
}
/// @brief opens an existing collection
@ -953,12 +871,7 @@ void LogicalCollection::open(bool ignoreErrors) {
std::shared_ptr<Index> LogicalCollection::lookupIndex(
TRI_idx_iid_t idxId) const {
for (auto const& idx : _indexes) {
if (idx->id() == idxId) {
return idx;
}
}
return nullptr;
return getPhysical()->lookupIndex(idxId);
}
std::shared_ptr<Index> LogicalCollection::lookupIndex(
@ -967,114 +880,13 @@ std::shared_ptr<Index> LogicalCollection::lookupIndex(
// Compatibility with old v8-vocindex.
THROW_ARANGO_EXCEPTION(TRI_ERROR_OUT_OF_MEMORY);
}
// extract type
VPackSlice value = info.get("type");
if (!value.isString()) {
// Compatibility with old v8-vocindex.
THROW_ARANGO_EXCEPTION(TRI_ERROR_OUT_OF_MEMORY);
}
std::string tmp = value.copyString();
arangodb::Index::IndexType const type = arangodb::Index::type(tmp.c_str());
for (auto const& idx : _indexes) {
if (idx->type() == type) {
// Only check relevant indices
if (idx->matchesDefinition(info)) {
// We found an index for this definition.
return idx;
}
}
}
return nullptr;
return getPhysical()->lookupIndex(info);
}
std::shared_ptr<Index> LogicalCollection::createIndex(transaction::Methods* trx,
VPackSlice const& info,
bool& created) {
// TODO Get LOCK for the vocbase
auto idx = lookupIndex(info);
if (idx != nullptr) {
created = false;
// We already have this index.
// Should we throw instead?
return idx;
}
StorageEngine* engine = EngineSelectorFeature::ENGINE;
IndexFactory const* idxFactory = engine->indexFactory();
TRI_ASSERT(idxFactory != nullptr);
// We are sure that we do not have an index of this type.
// We also hold the lock.
// Create it
idx = idxFactory->prepareIndexFromSlice(info, true, this, false);
TRI_ASSERT(idx != nullptr);
if (ServerState::instance()->isCoordinator()) {
// In the coordinator case we do not fill the index
// We only inform the others.
addIndexCoordinator(idx, true);
created = true;
return idx;
}
int res = getPhysical()->saveIndex(trx, idx);
if (res != TRI_ERROR_NO_ERROR) {
THROW_ARANGO_EXCEPTION(res);
}
arangodb::aql::PlanCache::instance()->invalidate(_vocbase);
// Until here no harm is done if sth fails. The shared ptr will clean up. if
// left before
addIndex(idx);
{
bool const doSync =
application_features::ApplicationServer::getFeature<DatabaseFeature>(
"Database")
->forceSyncProperties();
VPackBuilder builder = toVelocyPackIgnore({"path", "statusString"}, true);
updateProperties(builder.slice(), doSync);
}
created = true;
return idx;
}
/// @brief removes an index by id
bool LogicalCollection::removeIndex(TRI_idx_iid_t iid) {
size_t const n = _indexes.size();
for (size_t i = 0; i < n; ++i) {
auto idx = _indexes[i];
if (!idx->canBeDropped()) {
continue;
}
if (idx->id() == iid) {
// found!
idx->drop();
_indexes.erase(_indexes.begin() + i);
// update statistics
if (idx->type() == arangodb::Index::TRI_IDX_TYPE_FULLTEXT_INDEX) {
--_cleanupIndexes;
}
if (idx->isPersistent()) {
--_persistentIndexes;
}
return true;
}
}
// not found
return false;
return _physical->createIndex(trx, info, created);
}
/// @brief drops an index, including index file removal and replication
@ -1100,70 +912,6 @@ void LogicalCollection::persistPhysicalCollection() {
getPhysical()->setPath(path);
}
/// @brief creates the initial indexes for the collection
void LogicalCollection::createInitialIndexes() {
if (!_indexes.empty()) {
return;
}
std::vector<std::shared_ptr<arangodb::Index>> systemIndexes;
StorageEngine* engine = EngineSelectorFeature::ENGINE;
IndexFactory const* idxFactory = engine->indexFactory();
TRI_ASSERT(idxFactory != nullptr);
idxFactory->fillSystemIndexes(this, systemIndexes);
for (auto const& it : systemIndexes) {
addIndex(it);
}
}
std::vector<std::shared_ptr<arangodb::Index>> const*
LogicalCollection::indexList() const {
return &_indexes;
}
void LogicalCollection::addIndex(std::shared_ptr<arangodb::Index> idx) {
// primary index must be added at position 0
TRI_ASSERT(idx->type() != arangodb::Index::TRI_IDX_TYPE_PRIMARY_INDEX ||
_indexes.empty());
auto const id = idx->id();
for (auto const& it : _indexes) {
if (it->id() == id) {
// already have this particular index. do not add it again
return;
}
}
TRI_UpdateTickServer(static_cast<TRI_voc_tick_t>(id));
_indexes.emplace_back(idx);
// update statistics
if (idx->type() == arangodb::Index::TRI_IDX_TYPE_FULLTEXT_INDEX) {
++_cleanupIndexes;
}
if (idx->isPersistent()) {
++_persistentIndexes;
}
}
void LogicalCollection::addIndexCoordinator(
std::shared_ptr<arangodb::Index> idx, bool distribute) {
auto const id = idx->id();
for (auto const& it : _indexes) {
if (it->id() == id) {
// already have this particular index. do not add it again
return;
}
}
_indexes.emplace_back(idx);
if (distribute) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_NOT_IMPLEMENTED);
}
}
/// @brief reads an element from the document collection
int LogicalCollection::read(transaction::Methods* trx, std::string const& key,
ManagedDocumentResult& result, bool lock) {

View File

@ -273,11 +273,6 @@ class LogicalCollection {
/// @brief Find index by iid
std::shared_ptr<Index> lookupIndex(TRI_idx_iid_t) const;
// SECTION: Indexes (local only)
/// @brief Exposes a pointer to index list
std::vector<std::shared_ptr<Index>> const* indexList() const;
bool dropIndex(TRI_idx_iid_t iid);
// SECTION: Index access (local only)
@ -323,23 +318,12 @@ class LogicalCollection {
private:
// SECTION: Index creation
/// @brief creates the initial indexes for the collection
void createInitialIndexes();
public:
// TODO Fix Visibility
bool removeIndex(TRI_idx_iid_t iid);
void addIndex(std::shared_ptr<Index>);
private:
void addIndexCoordinator(std::shared_ptr<Index>, bool);
// SECTION: Indexes (local only)
// @brief create index with the given definition.
bool openIndex(velocypack::Slice const&, transaction::Methods*);
private:
void increaseInternalVersion();
protected:
@ -389,8 +373,6 @@ private:
uint32_t _version;
bool _waitForSync;
std::vector<std::shared_ptr<Index>> _indexes;
// SECTION: Replication
size_t _replicationFactor;
@ -408,8 +390,6 @@ private:
// SECTION: Local Only has to be moved to PhysicalCollection
public:
// TODO MOVE ME
size_t _cleanupIndexes;
size_t _persistentIndexes;
protected:
std::unique_ptr<PhysicalCollection> _physical;

View File

@ -26,6 +26,7 @@
#include "Basics/encoding.h"
#include "Basics/StaticStrings.h"
#include "Basics/VelocyPackHelper.h"
#include "Indexes/Index.h"
#include "StorageEngine/TransactionState.h"
#include "Transaction/Methods.h"
#include "VocBase/KeyGenerator.h"
@ -43,7 +44,10 @@ using namespace arangodb;
PhysicalCollection::PhysicalCollection(LogicalCollection* collection,
VPackSlice const& info)
: _logicalCollection(collection), _keyOptions(nullptr), _keyGenerator() {
: _logicalCollection(collection),
_keyOptions(nullptr),
_indexes(),
_keyGenerator() {
TRI_ASSERT(info.isObject());
auto keyOpts = info.get("keyOptions");
@ -57,6 +61,26 @@ void PhysicalCollection::figures(std::shared_ptr<arangodb::velocypack::Builder>&
this->figuresSpecific(builder);
};
void PhysicalCollection::drop() {
_indexes.clear();
try {
// close collection. this will also invalidate the revisions cache
close();
} catch (...) {
// don't throw from here... dropping should succeed
}
}
std::shared_ptr<Index> PhysicalCollection::lookupIndex(
TRI_idx_iid_t idxId) const {
for (auto const& idx : _indexes) {
if (idx->id() == idxId) {
return idx;
}
}
return nullptr;
}
/// @brief merge two objects for update, oldValue must have correctly set
/// _key and _id attributes
@ -343,6 +367,48 @@ int PhysicalCollection::checkRevision(transaction::Methods* trx,
return TRI_ERROR_NO_ERROR;
}
/// @brief hands out a list of indexes
std::vector<std::shared_ptr<arangodb::Index>> const&
PhysicalCollection::getIndexes() const {
return _indexes;
}
void PhysicalCollection::getIndexesVPack(VPackBuilder& result,
bool withFigures) const {
result.openArray();
for (auto const& idx : _indexes) {
result.openObject();
idx->toVelocyPack(result, withFigures);
result.close();
}
result.close();
}
/// @brief return the figures for a collection
std::shared_ptr<arangodb::velocypack::Builder> PhysicalCollection::figures() {
auto builder = std::make_shared<VPackBuilder>();
builder->openObject();
// add index information
size_t sizeIndexes = memory();
size_t numIndexes = 0;
for (auto const& idx : _indexes) {
sizeIndexes += static_cast<size_t>(idx->memory());
++numIndexes;
}
builder->add("indexes", VPackValue(VPackValueType::Object));
builder->add("count", VPackValue(numIndexes));
builder->add("size", VPackValue(sizeIndexes));
builder->close(); // indexes
// add engine-specific figures
figures(builder);
builder->close();
return builder;
}
// SECTION: Key Options
VPackSlice PhysicalCollection::keyOptions() const {
if (_keyOptions == nullptr) {

View File

@ -73,6 +73,9 @@ class PhysicalCollection {
/// @brief export properties
virtual void getPropertiesVPack(velocypack::Builder&) const = 0;
/// @brief return the figures for a collection
std::shared_ptr<velocypack::Builder> figures();
void figures(std::shared_ptr<arangodb::velocypack::Builder>& builder);
virtual int close() = 0;
@ -104,12 +107,27 @@ class PhysicalCollection {
virtual bool isFullyCollected() const = 0;
void drop();
////////////////////////////////////
// -- SECTION Indexes --
///////////////////////////////////
virtual int saveIndex(transaction::Methods* trx,
std::shared_ptr<arangodb::Index> idx) = 0;
virtual void prepareIndexes(arangodb::velocypack::Slice indexesSlice) = 0;
/// @brief Find index by definition
virtual std::shared_ptr<Index> lookupIndex(velocypack::Slice const&) const = 0;
/// @brief Find index by iid
std::shared_ptr<Index> lookupIndex(TRI_idx_iid_t) const;
std::vector<std::shared_ptr<Index>> const& getIndexes() const;
void getIndexesVPack(velocypack::Builder&, bool) const;
virtual std::shared_ptr<Index> createIndex(
transaction::Methods* trx, arangodb::velocypack::Slice const& info,
bool& created) = 0;
/// @brief Restores an index from VelocyPack.
virtual int restoreIndex(transaction::Methods*, velocypack::Slice const&,
@ -120,6 +138,7 @@ class PhysicalCollection {
virtual std::unique_ptr<IndexIterator> getAllIterator(transaction::Methods* trx, ManagedDocumentResult* mdr, bool reverse) = 0;
virtual std::unique_ptr<IndexIterator> getAnyIterator(transaction::Methods* trx, ManagedDocumentResult* mdr) = 0;
virtual void invokeOnAllElements(std::function<bool(DocumentIdentifierToken const&)> callback) = 0;
////////////////////////////////////
// -- SECTION DML Operations --
///////////////////////////////////
@ -232,6 +251,7 @@ class PhysicalCollection {
std::shared_ptr<velocypack::Buffer<uint8_t> const>
_keyOptions; // options for key creation
std::vector<std::shared_ptr<Index>> _indexes;
std::unique_ptr<KeyGenerator> _keyGenerator;