1
0
Fork 0

RocksDB replication hardening & cleanup (#4639)

This commit is contained in:
Simon 2018-02-23 18:15:04 +01:00 committed by Jan
parent 1b64f03bf8
commit cde96d97aa
43 changed files with 1369 additions and 1472 deletions

View File

@ -321,7 +321,7 @@ void IResearchRocksDBRecoveryHelper::PutCF(uint32_t column_family_id,
return;
}
auto rev = RocksDBKey::revisionId(RocksDBEntryType::Document, key);
auto docId = RocksDBKey::documentId(RocksDBEntryType::Document, key);
auto doc = RocksDBValue::data(value);
SingleCollectionTransaction trx(
@ -334,7 +334,7 @@ void IResearchRocksDBRecoveryHelper::PutCF(uint32_t column_family_id,
for (auto link : links) {
link->insert(
&trx,
LocalDocumentId(rev),
docId,
doc,
Index::OperationMode::internal
);
@ -364,7 +364,7 @@ void IResearchRocksDBRecoveryHelper::DeleteCF(uint32_t column_family_id,
return;
}
auto rev = RocksDBKey::revisionId(RocksDBEntryType::Document, key);
auto docId = RocksDBKey::documentId(RocksDBEntryType::Document, key);
SingleCollectionTransaction trx(
transaction::StandaloneContext::Create(vocbase), coll->cid(),
@ -376,12 +376,12 @@ void IResearchRocksDBRecoveryHelper::DeleteCF(uint32_t column_family_id,
for (auto link : links) {
link->remove(
&trx,
LocalDocumentId(rev),
docId,
arangodb::velocypack::Slice::emptyObjectSlice(),
Index::OperationMode::internal
);
// LOG_TOPIC(TRACE, IResearchFeature::IRESEARCH) << "recovery helper
// removed: " << rev;
// removed: " << docId.id();
}
trx.commit();

View File

@ -1936,7 +1936,7 @@ int MMFilesCollection::iterateMarkersOnLoad(transaction::Methods* trx) {
}
LocalDocumentId MMFilesCollection::lookupKey(transaction::Methods *trx,
VPackSlice const& key) {
VPackSlice const& key) const {
MMFilesPrimaryIndex *index = primaryIndex();
MMFilesSimpleIndexElement element = index->lookupKey(trx, key);
return element ? LocalDocumentId(element.localDocumentId()) : LocalDocumentId();
@ -2761,7 +2761,7 @@ void MMFilesCollection::truncate(transaction::Methods* trx,
LocalDocumentId const documentId = LocalDocumentId::create();
TRI_voc_rid_t revisionId;
newObjectForRemove(trx, oldDoc, documentId, *builder.get(), options.isRestore, revisionId);
newObjectForRemove(trx, oldDoc, *builder.get(), options.isRestore, revisionId);
Result res = removeFastPath(trx, revisionId, oldDocumentId, VPackSlice(vpack),
options, documentId, builder->slice());
@ -2846,8 +2846,8 @@ Result MMFilesCollection::insert(transaction::Methods* trx,
VPackSlice newSlice;
Result res(TRI_ERROR_NO_ERROR);
if (options.recoveryData == nullptr) {
res = newObjectForInsert(trx, slice, fromSlice, toSlice, documentId,
isEdgeCollection, *builder.get(), options.isRestore, revisionId);
res = newObjectForInsert(trx, slice, fromSlice, toSlice, isEdgeCollection,
*builder.get(), options.isRestore, revisionId);
if (res.fail()) {
return res;
}
@ -3323,8 +3323,8 @@ Result MMFilesCollection::update(
transaction::BuilderLeaser builder(trx);
if (options.recoveryData == nullptr) {
mergeObjectsForUpdate(trx, oldDoc, newSlice, isEdgeCollection,
documentId, options.mergeObjects,
options.keepNull, *builder.get(), options.isRestore, revisionId);
options.mergeObjects, options.keepNull,
*builder.get(), options.isRestore, revisionId);
if (_isDBServer) {
// Need to check that no sharding keys have changed:
@ -3456,8 +3456,8 @@ Result MMFilesCollection::replace(
TRI_voc_rid_t revisionId;
transaction::BuilderLeaser builder(trx);
newObjectForReplace(trx, oldDoc, newSlice, fromSlice, toSlice,
isEdgeCollection, documentId,
*builder.get(), options.isRestore, revisionId);
isEdgeCollection, *builder.get(),
options.isRestore, revisionId);
if (_isDBServer) {
// Need to check that no sharding keys have changed:
@ -3532,7 +3532,7 @@ Result MMFilesCollection::remove(arangodb::transaction::Methods* trx,
LocalDocumentId const documentId = LocalDocumentId::create();
transaction::BuilderLeaser builder(trx);
newObjectForRemove(trx, slice, documentId, *builder.get(), options.isRestore, revisionId);
newObjectForRemove(trx, slice, *builder.get(), options.isRestore, revisionId);
TRI_IF_FAILURE("RemoveDocumentNoMarker") {
// test what happens when no marker can be created

View File

@ -329,7 +329,7 @@ class MMFilesCollection final : public PhysicalCollection {
void truncate(transaction::Methods* trx, OperationOptions& options) override;
LocalDocumentId lookupKey(transaction::Methods* trx,
velocypack::Slice const& key) override;
velocypack::Slice const& key) const override;
Result read(transaction::Methods*, arangodb::StringRef const& key,
ManagedDocumentResult& result, bool) override;

View File

@ -94,8 +94,9 @@ Result DatabaseInitialSyncer::runWithInventory(bool incremental,
try {
setProgress("fetching master state");
LOG_TOPIC(DEBUG, Logger::REPLICATION) << "client: getting master state";
LOG_TOPIC(DEBUG, Logger::REPLICATION) << "client: getting master state to dump "
<< vocbase()->name();
Result r;
if (!_isChildSyncer) {
r = getMasterState();

View File

@ -24,6 +24,7 @@
#include "Syncer.h"
#include "Basics/Exceptions.h"
#include "Basics/MutexLocker.h"
#include "Basics/RocksDBUtils.h"
#include "Basics/StaticStrings.h"
#include "Basics/VelocyPackHelper.h"
#include "GeneralServer/AuthenticationFeature.h"
@ -595,7 +596,6 @@ Result Syncer::createCollection(TRI_vocbase_t* vocbase,
VPackBuilder s;
s.openObject();
s.add("isSystem", VPackValue(true));
s.add("objectId", VPackSlice::nullSlice());
if ((uuid.isString() && !simulate32Client()) || forceRemoveCid) { // need to use cid for 3.2 master
// if we received a globallyUniqueId from the remote, then we will always use this id
// so we can discard the "cid" and "id" values for the collection
@ -603,12 +603,13 @@ Result Syncer::createCollection(TRI_vocbase_t* vocbase,
s.add("cid", VPackSlice::nullSlice());
}
s.close();
VPackBuilder merged = VPackCollection::merge(slice, s.slice(),
/*mergeValues*/true, /*nullMeansRemove*/true);
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(merged.slice());
col = vocbase->createCollection(stripped.first);
} catch (basics::Exception const& ex) {
return Result(ex.code(), ex.what());
} catch (std::exception const& ex) {

View File

@ -71,7 +71,6 @@
#include <velocypack/velocypack-aliases.h>
using namespace arangodb;
using namespace arangodb::rocksutils;
RocksDBCollection::RocksDBCollection(LogicalCollection* collection,
VPackSlice const& info)
@ -93,8 +92,9 @@ RocksDBCollection::RocksDBCollection(LogicalCollection* collection,
"volatile collections are unsupported in the RocksDB engine");
}
addCollectionMapping(_objectId, _logicalCollection->vocbase()->id(),
_logicalCollection->cid());
rocksutils::globalRocksEngine()->addCollectionMapping(
_objectId, _logicalCollection->vocbase()->id(),
_logicalCollection->cid());
if (_cacheEnabled) {
createCache();
}
@ -110,10 +110,11 @@ RocksDBCollection::RocksDBCollection(LogicalCollection* collection,
_primaryIndex(nullptr),
_cache(nullptr),
_cachePresent(false),
_cacheEnabled(static_cast<RocksDBCollection const*>(physical)->_cacheEnabled) {
addCollectionMapping(_objectId, _logicalCollection->vocbase()->id(),
_logicalCollection->cid());
_cacheEnabled(
static_cast<RocksDBCollection const*>(physical)->_cacheEnabled) {
rocksutils::globalRocksEngine()->addCollectionMapping(
_objectId, _logicalCollection->vocbase()->id(),
_logicalCollection->cid());
if (_cacheEnabled) {
createCache();
}
@ -136,10 +137,11 @@ void RocksDBCollection::setPath(std::string const&) {
// we do not have any path
}
Result RocksDBCollection::updateProperties(VPackSlice const& slice, bool doSync) {
Result RocksDBCollection::updateProperties(VPackSlice const& slice,
bool doSync) {
bool isSys = _logicalCollection != nullptr && _logicalCollection->isSystem();
_cacheEnabled = !isSys && basics::VelocyPackHelper::readBooleanValue(slice,
"cacheEnabled", _cacheEnabled);
_cacheEnabled = !isSys && basics::VelocyPackHelper::readBooleanValue(
slice, "cacheEnabled", _cacheEnabled);
primaryIndex()->setCacheEnabled(_cacheEnabled);
if (_cacheEnabled) {
createCache();
@ -613,7 +615,7 @@ bool RocksDBCollection::dropIndex(TRI_idx_iid_t iid) {
size_t i = 0;
WRITE_LOCKER(guard, _indexesLock);
for (auto index : _indexes) {
for (std::shared_ptr<Index> index : _indexes) {
RocksDBIndex* cindex = static_cast<RocksDBIndex*>(index.get());
TRI_ASSERT(cindex != nullptr);
@ -630,17 +632,18 @@ bool RocksDBCollection::dropIndex(TRI_idx_iid_t iid) {
// lock anymore, this branch always returns
guard.unlock();
auto engine = static_cast<RocksDBEngine*>(EngineSelectorFeature::ENGINE);
engine->removeIndexMapping(cindex->objectId());
VPackBuilder builder = _logicalCollection->toVelocyPackIgnore(
{"path", "statusString"}, true, true);
StorageEngine* engine = EngineSelectorFeature::ENGINE;
// log this event in the WAL and in the collection meta-data
int res =
static_cast<RocksDBEngine*>(engine)->writeCreateCollectionMarker(
_logicalCollection->vocbase()->id(), _logicalCollection->cid(),
builder.slice(),
RocksDBLogValue::IndexDrop(_logicalCollection->vocbase()->id(),
_logicalCollection->cid(), iid));
int res = engine->writeCreateCollectionMarker(
_logicalCollection->vocbase()->id(), _logicalCollection->cid(),
builder.slice(),
RocksDBLogValue::IndexDrop(_logicalCollection->vocbase()->id(),
_logicalCollection->cid(), iid));
return res == TRI_ERROR_NO_ERROR;
}
@ -660,21 +663,22 @@ std::unique_ptr<IndexIterator> RocksDBCollection::getAllIterator(
_logicalCollection, trx, primaryIndex(), reverse));
}
std::unique_ptr<IndexIterator> RocksDBCollection::getAnyIterator(transaction::Methods* trx) const {
return std::unique_ptr<IndexIterator>(new RocksDBAnyIndexIterator(
_logicalCollection, trx, primaryIndex()));
std::unique_ptr<IndexIterator> RocksDBCollection::getAnyIterator(
transaction::Methods* trx) const {
return std::unique_ptr<IndexIterator>(
new RocksDBAnyIndexIterator(_logicalCollection, trx, primaryIndex()));
}
std::unique_ptr<IndexIterator> RocksDBCollection::getSortedAllIterator(transaction::Methods* trx) const {
return std::unique_ptr<RocksDBSortedAllIterator>(new RocksDBSortedAllIterator(
_logicalCollection, trx, primaryIndex()));
std::unique_ptr<IndexIterator> RocksDBCollection::getSortedAllIterator(
transaction::Methods* trx) const {
return std::unique_ptr<RocksDBSortedAllIterator>(
new RocksDBSortedAllIterator(_logicalCollection, trx, primaryIndex()));
}
void RocksDBCollection::invokeOnAllElements(
transaction::Methods* trx,
std::function<bool(LocalDocumentId const&)> callback) {
std::unique_ptr<IndexIterator> cursor(
this->getAllIterator(trx, false));
std::unique_ptr<IndexIterator> cursor(this->getAllIterator(trx, false));
bool cnt = true;
auto cb = [&](LocalDocumentId token) {
if (cnt) {
@ -702,49 +706,53 @@ void RocksDBCollection::truncate(transaction::Methods* trx,
rocksdb::ReadOptions ro = mthd->readOptions();
rocksdb::Slice const end = documentBounds.end();
ro.iterate_upper_bound = &end;
// avoid OOM error for truncate by committing earlier
uint64_t const prvICC = state->options().intermediateCommitCount;
state->options().intermediateCommitCount = std::min<uint64_t>(prvICC, 10000);
std::unique_ptr<rocksdb::Iterator> iter =
mthd->NewIterator(ro, documentBounds.columnFamily());
iter->Seek(documentBounds.start());
uint64_t found = 0;
while (iter->Valid() && cmp->Compare(iter->key(), end) < 0) {
++found;
TRI_ASSERT(_objectId == RocksDBKey::objectId(iter->key()));
LocalDocumentId docId(RocksDBKey::revisionId(RocksDBEntryType::Document, iter->key()));
VPackSlice doc = VPackSlice(iter->value().data());
TRI_ASSERT(doc.isObject());
VPackSlice key = doc.get(StaticStrings::KeyString);
// To print the WAL we need key and RID
VPackSlice key = transaction::helpers::extractKeyFromDocument(doc);
TRI_ASSERT(key.isString());
TRI_voc_rid_t rid = transaction::helpers::extractRevFromDocument(doc);
TRI_ASSERT(rid != 0);
blackListKey(iter->key().data(), static_cast<uint32_t>(iter->key().size()));
state->prepareOperation(_logicalCollection->cid(),
rid, // actual revision ID!!
TRI_VOC_DOCUMENT_OPERATION_REMOVE);
state->prepareOperation(_logicalCollection->cid(), docId.id(),
StringRef(key),TRI_VOC_DOCUMENT_OPERATION_REMOVE);
LocalDocumentId const docId =
RocksDBKey::documentId(RocksDBEntryType::Document, iter->key());
auto res = removeDocument(trx, docId, doc, options);
if (res.fail()) {
// Failed to remove document in truncate.
// Throw
THROW_ARANGO_EXCEPTION_MESSAGE(res.errorNumber(), res.errorMessage());
// Failed to remove document in truncate. Throw
THROW_ARANGO_EXCEPTION(res);
}
res = state->addOperation(_logicalCollection->cid(), docId.id(),
TRI_VOC_DOCUMENT_OPERATION_REMOVE, 0,
res.keySize());
TRI_VOC_DOCUMENT_OPERATION_REMOVE);
// transaction size limit reached
if (res.fail()) {
// This should never happen...
THROW_ARANGO_EXCEPTION_MESSAGE(res.errorNumber(), res.errorMessage());
}
trackWaitForSync(trx, options);
if (found % 10000 == 0) {
state->triggerIntermediateCommit();
}
iter->Next();
}
// reset to previous value after truncate is finished
state->options().intermediateCommitCount = prvICC;
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
if (state->numCommits() == 0) {
@ -773,7 +781,7 @@ void RocksDBCollection::truncate(transaction::Methods* trx,
}
LocalDocumentId RocksDBCollection::lookupKey(transaction::Methods* trx,
VPackSlice const& key) {
VPackSlice const& key) const {
TRI_ASSERT(key.isString());
return primaryIndex()->lookupKey(trx, StringRef(key));
}
@ -832,31 +840,30 @@ Result RocksDBCollection::insert(arangodb::transaction::Methods* trx,
// _from:
fromSlice = slice.get(StaticStrings::FromString);
if (!fromSlice.isString()) {
return RocksDBOperationResult(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
return Result(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
}
VPackValueLength len;
char const* docId = fromSlice.getString(len);
size_t split;
if (!TRI_ValidateDocumentIdKeyGenerator(docId, static_cast<size_t>(len),
&split)) {
return RocksDBOperationResult(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
return Result(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
}
// _to:
toSlice = slice.get(StaticStrings::ToString);
if (!toSlice.isString()) {
return RocksDBOperationResult(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
return Result(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
}
docId = toSlice.getString(len);
if (!TRI_ValidateDocumentIdKeyGenerator(docId, static_cast<size_t>(len),
&split)) {
return RocksDBOperationResult(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
return Result(TRI_ERROR_ARANGO_INVALID_EDGE_ATTRIBUTE);
}
}
transaction::BuilderLeaser builder(trx);
RocksDBOperationResult res(
newObjectForInsert(trx, slice, fromSlice, toSlice, documentId,
isEdgeCollection, *builder.get(), options.isRestore, revisionId));
Result res(newObjectForInsert(trx, slice, fromSlice, toSlice, isEdgeCollection,
*builder.get(), options.isRestore, revisionId));
if (res.fail()) {
return res;
}
@ -864,10 +871,9 @@ Result RocksDBCollection::insert(arangodb::transaction::Methods* trx,
auto state = RocksDBTransactionState::toState(trx);
auto mthds = RocksDBTransactionState::toMethods(trx);
RocksDBSavePoint guard(mthds, trx->isSingleOperationTransaction(),
[&state]() { state->resetLogState(); });
RocksDBSavePoint guard(mthds, trx->isSingleOperationTransaction());
state->prepareOperation(_logicalCollection->cid(), revisionId, StringRef(),
state->prepareOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_INSERT);
res = insertDocument(trx, documentId, newSlice, options);
@ -877,9 +883,8 @@ Result RocksDBCollection::insert(arangodb::transaction::Methods* trx,
mdr.setManaged(newSlice.begin(), documentId);
// report document and key size
RocksDBOperationResult result = state->addOperation(
_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_INSERT, newSlice.byteSize(), res.keySize());
Result result = state->addOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_INSERT);
// transaction size limit reached -- fail
if (result.fail()) {
@ -905,8 +910,7 @@ Result RocksDBCollection::update(arangodb::transaction::Methods* trx,
bool const isEdgeCollection =
(_logicalCollection->type() == TRI_COL_TYPE_EDGE);
RocksDBOperationResult res = lookupDocument(trx, key, previous);
Result res = lookupDocument(trx, key, previous);
if (res.fail()) {
return res;
}
@ -915,7 +919,7 @@ Result RocksDBCollection::update(arangodb::transaction::Methods* trx,
LocalDocumentId const oldDocumentId = previous.localDocumentId();
VPackSlice oldDoc(previous.vpack());
TRI_voc_rid_t oldRevisionId =
TRI_voc_rid_t const oldRevisionId =
transaction::helpers::extractRevFromDocument(oldDoc);
prevRev = oldRevisionId;
@ -947,8 +951,8 @@ Result RocksDBCollection::update(arangodb::transaction::Methods* trx,
TRI_voc_rid_t revisionId;
transaction::BuilderLeaser builder(trx);
mergeObjectsForUpdate(trx, oldDoc, newSlice, isEdgeCollection,
documentId, options.mergeObjects,
options.keepNull, *builder.get(), options.isRestore, revisionId);
options.mergeObjects, options.keepNull, *builder.get(),
options.isRestore, revisionId);
if (_isDBServer) {
// Need to check that no sharding keys have changed:
if (arangodb::shardKeysChanged(_logicalCollection->dbName(),
@ -963,11 +967,10 @@ Result RocksDBCollection::update(arangodb::transaction::Methods* trx,
auto state = RocksDBTransactionState::toState(trx);
RocksDBSavePoint guard(RocksDBTransactionState::toMethods(trx),
trx->isSingleOperationTransaction(),
[&state]() { state->resetLogState(); });
trx->isSingleOperationTransaction());
// add possible log statement under guard
state->prepareOperation(_logicalCollection->cid(), revisionId, StringRef(),
state->prepareOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_UPDATE);
res = updateDocument(trx, oldDocumentId, oldDoc, documentId, newDoc, options);
@ -978,11 +981,10 @@ Result RocksDBCollection::update(arangodb::transaction::Methods* trx,
TRI_ASSERT(!mdr.empty());
// report document and key size
RocksDBOperationResult result = state->addOperation(
_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_UPDATE, newDoc.byteSize(), res.keySize());
Result result = state->addOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_UPDATE);
// transaction size limit reached -- fail
// transaction size limit reached -- fail hard
if (result.fail()) {
THROW_ARANGO_EXCEPTION(result);
}
@ -993,12 +995,15 @@ Result RocksDBCollection::update(arangodb::transaction::Methods* trx,
return res;
}
Result RocksDBCollection::replace(
transaction::Methods* trx, arangodb::velocypack::Slice const newSlice,
ManagedDocumentResult& mdr, OperationOptions& options,
TRI_voc_tick_t& resultMarkerTick, bool /*lock*/, TRI_voc_rid_t& prevRev,
ManagedDocumentResult& previous, arangodb::velocypack::Slice const fromSlice,
arangodb::velocypack::Slice const toSlice) {
Result RocksDBCollection::replace(transaction::Methods* trx,
arangodb::velocypack::Slice const newSlice,
ManagedDocumentResult& mdr,
OperationOptions& options,
TRI_voc_tick_t& resultMarkerTick,
bool /*lock*/, TRI_voc_rid_t& prevRev,
ManagedDocumentResult& previous,
arangodb::velocypack::Slice const fromSlice,
arangodb::velocypack::Slice const toSlice) {
resultMarkerTick = 0;
LocalDocumentId const documentId = LocalDocumentId::create();
@ -1043,8 +1048,8 @@ Result RocksDBCollection::replace(
TRI_voc_rid_t revisionId;
transaction::BuilderLeaser builder(trx);
newObjectForReplace(trx, oldDoc, newSlice, fromSlice, toSlice,
isEdgeCollection, documentId,
*builder.get(), options.isRestore, revisionId);
isEdgeCollection, *builder.get(), options.isRestore,
revisionId);
if (_isDBServer) {
// Need to check that no sharding keys have changed:
@ -1060,15 +1065,13 @@ Result RocksDBCollection::replace(
auto state = RocksDBTransactionState::toState(trx);
RocksDBSavePoint guard(RocksDBTransactionState::toMethods(trx),
trx->isSingleOperationTransaction(),
[&state]() { state->resetLogState(); });
trx->isSingleOperationTransaction());
// add possible log statement under guard
state->prepareOperation(_logicalCollection->cid(), revisionId, StringRef(),
state->prepareOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_REPLACE);
RocksDBOperationResult opResult = updateDocument(
trx, oldDocumentId, oldDoc, documentId, newDoc, options);
Result opResult = updateDocument(trx, oldDocumentId, oldDoc, documentId, newDoc, options);
if (opResult.ok()) {
trackWaitForSync(trx, options);
@ -1077,10 +1080,8 @@ Result RocksDBCollection::replace(
TRI_ASSERT(!mdr.empty());
// report document and key size
RocksDBOperationResult result =
state->addOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_REPLACE,
newDoc.byteSize(), opResult.keySize());
Result result = state->addOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_REPLACE);
// transaction size limit reached -- fail
if (result.fail()) {
@ -1098,15 +1099,12 @@ Result RocksDBCollection::remove(arangodb::transaction::Methods* trx,
arangodb::ManagedDocumentResult& previous,
OperationOptions& options,
TRI_voc_tick_t& resultMarkerTick,
bool /*lock*/,
TRI_voc_rid_t& prevRev,
bool /*lock*/, TRI_voc_rid_t& prevRev,
TRI_voc_rid_t& revisionId) {
// store the tick that was used for writing the document
// note that we don't need it for this engine
resultMarkerTick = 0;
LocalDocumentId const documentId = LocalDocumentId::create();
prevRev = 0;
revisionId = newRevisionId();
VPackSlice key;
@ -1118,8 +1116,7 @@ Result RocksDBCollection::remove(arangodb::transaction::Methods* trx,
TRI_ASSERT(!key.isNone());
// get the previous revision
RocksDBOperationResult res = lookupDocument(trx, key, previous);
Result res = lookupDocument(trx, key, previous);
if (res.fail()) {
return res;
}
@ -1144,21 +1141,20 @@ Result RocksDBCollection::remove(arangodb::transaction::Methods* trx,
auto state = RocksDBTransactionState::toState(trx);
RocksDBSavePoint guard(RocksDBTransactionState::toMethods(trx),
trx->isSingleOperationTransaction(),
[&state]() { state->resetLogState(); });
trx->isSingleOperationTransaction());
// add possible log statement under guard
state->prepareOperation(_logicalCollection->cid(), documentId.id(),
StringRef(key),TRI_VOC_DOCUMENT_OPERATION_REMOVE);
state->prepareOperation(_logicalCollection->cid(), oldRevisionId,
TRI_VOC_DOCUMENT_OPERATION_REMOVE);
res = removeDocument(trx, oldDocumentId, oldDoc, options);
if (res.ok()) {
trackWaitForSync(trx, options);
// report key size
res = state->addOperation(_logicalCollection->cid(), documentId.id(),
TRI_VOC_DOCUMENT_OPERATION_REMOVE, 0,
res.keySize());
res = state->addOperation(_logicalCollection->cid(), revisionId,
TRI_VOC_DOCUMENT_OPERATION_REMOVE);
// transaction size limit reached -- fail
if (res.fail()) {
THROW_ARANGO_EXCEPTION(res);
@ -1299,8 +1295,8 @@ arangodb::Result RocksDBCollection::fillIndexes(
std::unique_ptr<IndexIterator> it(new RocksDBAllIndexIterator(
_logicalCollection, trx, primaryIndex(), false));
// fillindex can be non transactional
rocksdb::DB* db = globalRocksDB()->GetBaseDB();
// fillindex can be non transactional, we just need to clean up
rocksdb::DB* db = rocksutils::globalRocksDB()->GetBaseDB();
TRI_ASSERT(db != nullptr);
uint64_t numDocsWritten = 0;
@ -1371,26 +1367,23 @@ arangodb::Result RocksDBCollection::fillIndexes(
return res;
}
RocksDBOperationResult RocksDBCollection::insertDocument(
Result RocksDBCollection::insertDocument(
arangodb::transaction::Methods* trx, LocalDocumentId const& documentId,
VPackSlice const& doc, OperationOptions& options) const {
RocksDBOperationResult res;
// Coordinator doesn't know index internals
TRI_ASSERT(!ServerState::instance()->isCoordinator());
TRI_ASSERT(trx->state()->isRunning());
RocksDBKeyLeaser key(trx);
key->constructDocument(_objectId, documentId.id());
key->constructDocument(_objectId, documentId);
blackListKey(key->string().data(), static_cast<uint32_t>(key->string().size()));
RocksDBMethods* mthd = RocksDBTransactionState::toMethods(trx);
res = mthd->Put(RocksDBColumnFamily::documents(), key.ref(),
rocksdb::Slice(reinterpret_cast<char const*>(doc.begin()),
static_cast<size_t>(doc.byteSize())));
Result res = mthd->Put(RocksDBColumnFamily::documents(), key.ref(),
rocksdb::Slice(reinterpret_cast<char const*>(doc.begin()),
static_cast<size_t>(doc.byteSize())));
if (!res.ok()) {
// set keysize that is passed up to the crud operations
res.keySize(key->string().size());
return res;
}
@ -1414,7 +1407,7 @@ RocksDBOperationResult RocksDBCollection::insertDocument(
return res;
}
RocksDBOperationResult RocksDBCollection::removeDocument(
Result RocksDBCollection::removeDocument(
arangodb::transaction::Methods* trx, LocalDocumentId const& documentId,
VPackSlice const& doc, OperationOptions& options) const {
// Coordinator doesn't know index internals
@ -1423,13 +1416,12 @@ RocksDBOperationResult RocksDBCollection::removeDocument(
TRI_ASSERT(_objectId != 0);
RocksDBKeyLeaser key(trx);
key->constructDocument(_objectId, documentId.id());
key->constructDocument(_objectId, documentId);
blackListKey(key->string().data(), static_cast<uint32_t>(key->string().size()));
RocksDBMethods* mthd = RocksDBTransactionState::toMethods(trx);
RocksDBOperationResult res =
mthd->Delete(RocksDBColumnFamily::documents(), key.ref());
Result res = mthd->Delete(RocksDBColumnFamily::documents(), key.ref());
if (!res.ok()) {
return res;
}
@ -1439,7 +1431,7 @@ RocksDBOperationResult RocksDBCollection::removeDocument(
<< " seq: " << mthd->readOptions().snapshot->GetSequenceNumber()
<< " objectID " << _objectId << " name: " << _logicalCollection->name();*/
RocksDBOperationResult resInner;
Result resInner;
READ_LOCKER(guard, _indexesLock);
for (std::shared_ptr<Index> const& idx : _indexes) {
Result tmpres = idx->remove(trx, documentId, doc, options.indexOperationMode);
@ -1458,22 +1450,21 @@ RocksDBOperationResult RocksDBCollection::removeDocument(
/// @brief looks up a document by key, low level worker
/// the key must be a string slice, no revision check is performed
RocksDBOperationResult RocksDBCollection::lookupDocument(
Result RocksDBCollection::lookupDocument(
transaction::Methods* trx, VPackSlice const& key,
ManagedDocumentResult& mdr) const {
if (!key.isString()) {
return RocksDBOperationResult(TRI_ERROR_ARANGO_DOCUMENT_KEY_BAD);
return Result(TRI_ERROR_ARANGO_DOCUMENT_KEY_BAD);
}
LocalDocumentId documentId = primaryIndex()->lookupKey(trx, StringRef(key));
if (documentId.isSet()) {
return lookupDocumentVPack(documentId, trx, mdr, true);
}
return RocksDBOperationResult(TRI_ERROR_ARANGO_DOCUMENT_NOT_FOUND);
return Result(TRI_ERROR_ARANGO_DOCUMENT_NOT_FOUND);
}
RocksDBOperationResult RocksDBCollection::updateDocument(
Result RocksDBCollection::updateDocument(
transaction::Methods* trx, LocalDocumentId const& oldDocumentId,
VPackSlice const& oldDoc, LocalDocumentId const& newDocumentId,
VPackSlice const& newDoc, OperationOptions& options) const {
@ -1488,23 +1479,20 @@ RocksDBOperationResult RocksDBCollection::updateDocument(
// We NEED to do the PUT first, otherwise WAL tailing breaks
RocksDBKeyLeaser newKey(trx);
newKey->constructDocument(_objectId, newDocumentId.id());
newKey->constructDocument(_objectId, newDocumentId);
// TODO: given that this should have a unique revision ID, do
// we really need to blacklist the new key?
blackListKey(newKey->string().data(),
static_cast<uint32_t>(newKey->string().size()));
rocksdb::Slice docSlice(reinterpret_cast<char const*>(newDoc.begin()),
static_cast<size_t>(newDoc.byteSize()));
RocksDBOperationResult res = mthd->Put(RocksDBColumnFamily::documents(),
newKey.ref(), docSlice);
Result res = mthd->Put(RocksDBColumnFamily::documents(), newKey.ref(), docSlice);
if (!res.ok()) {
// set keysize that is passed up to the crud operations
res.keySize(newKey->size());
return res;
}
RocksDBKeyLeaser oldKey(trx);
oldKey->constructDocument(_objectId, oldDocumentId.id());
oldKey->constructDocument(_objectId, oldDocumentId);
blackListKey(oldKey->string().data(),
static_cast<uint32_t>(oldKey->string().size()));
@ -1516,9 +1504,8 @@ RocksDBOperationResult RocksDBCollection::updateDocument(
READ_LOCKER(guard, _indexesLock);
for (std::shared_ptr<Index> const& idx : _indexes) {
RocksDBIndex* rIdx = static_cast<RocksDBIndex*>(idx.get());
Result tmpres = rIdx->updateInternal(trx, mthd, oldDocumentId, oldDoc,
newDocumentId, newDoc,
options.indexOperationMode);
Result tmpres = rIdx->updateInternal(trx, mthd, oldDocumentId, oldDoc, newDocumentId,
newDoc, options.indexOperationMode);
if (!tmpres.ok()) {
if (tmpres.is(TRI_ERROR_OUT_OF_MEMORY)) {
// in case of OOM return immediately
@ -1538,7 +1525,7 @@ arangodb::Result RocksDBCollection::lookupDocumentVPack(
TRI_ASSERT(_objectId != 0);
RocksDBKeyLeaser key(trx);
key->constructDocument(_objectId, documentId.id());
key->constructDocument(_objectId, documentId);
bool lockTimeout = false;
if (withCache && useCache()) {
@ -1589,6 +1576,7 @@ arangodb::Result RocksDBCollection::lookupDocumentVPack(
<< " seq: " << mthd->readOptions().snapshot->GetSequenceNumber()
<< " objectID " << _objectId << " name: " << _logicalCollection->name();
mdr.reset();
TRI_ASSERT(false);
}
return res;
}
@ -1600,7 +1588,7 @@ arangodb::Result RocksDBCollection::lookupDocumentVPack(
TRI_ASSERT(_objectId != 0);
RocksDBKeyLeaser key(trx);
key->constructDocument(_objectId, documentId.id());
key->constructDocument(_objectId, documentId);
bool lockTimeout = false;
if (withCache && useCache()) {
@ -1650,6 +1638,7 @@ arangodb::Result RocksDBCollection::lookupDocumentVPack(
<< "NOT FOUND rev: " << documentId.id() << " trx: " << trx->state()->id()
<< " seq: " << mthd->readOptions().snapshot->GetSequenceNumber()
<< " objectID " << _objectId << " name: " << _logicalCollection->name();
TRI_ASSERT(false);
}
return res;
}
@ -1778,18 +1767,19 @@ uint64_t RocksDBCollection::recalculateCounts() {
THROW_ARANGO_EXCEPTION(res);
}
RocksDBEngine* engine = rocksutils::globalRocksEngine();
// count documents
auto documentBounds = RocksDBKeyBounds::CollectionDocuments(_objectId);
_numberDocuments =
rocksutils::countKeyRange(globalRocksDB(), documentBounds, true);
rocksutils::countKeyRange(engine->db(), documentBounds, true);
// update counter manager value
res = globalRocksEngine()->settingsManager()->setAbsoluteCounter(
_objectId, _numberDocuments);
res = engine->settingsManager()->setAbsoluteCounter(_objectId,
_numberDocuments);
if (res.ok()) {
// in case of fail the counter has never been written and hence does not
// need correction. The value is not changed and does not need to be synced
globalRocksEngine()->settingsManager()->sync(true);
engine->settingsManager()->sync(true);
}
trx.commit();
@ -1948,8 +1938,7 @@ void RocksDBCollection::deserializeKeyGenerator(RocksDBSettingsManager* mgr) {
}
void RocksDBCollection::createCache() const {
if (!_cacheEnabled || _cachePresent ||
_logicalCollection->isAStub() ||
if (!_cacheEnabled || _cachePresent || _logicalCollection->isAStub() ||
ServerState::instance()->isCoordinator()) {
// we leave this if we do not need the cache
// or if cache already created

View File

@ -124,9 +124,8 @@ class RocksDBCollection final : public PhysicalCollection {
void truncate(transaction::Methods* trx, OperationOptions& options) override;
LocalDocumentId lookupKey(
transaction::Methods* trx,
arangodb::velocypack::Slice const& key) override;
LocalDocumentId lookupKey(transaction::Methods* trx,
velocypack::Slice const& key) const override;
Result read(transaction::Methods*, arangodb::StringRef const& key,
ManagedDocumentResult& result, bool) override;
@ -230,19 +229,19 @@ class RocksDBCollection final : public PhysicalCollection {
return _primaryIndex;
}
arangodb::RocksDBOperationResult insertDocument(
arangodb::Result insertDocument(
arangodb::transaction::Methods* trx, LocalDocumentId const& documentId,
arangodb::velocypack::Slice const& doc, OperationOptions& options) const;
arangodb::RocksDBOperationResult removeDocument(
arangodb::Result removeDocument(
arangodb::transaction::Methods* trx, LocalDocumentId const& documentId,
arangodb::velocypack::Slice const& doc, OperationOptions& options) const;
arangodb::RocksDBOperationResult lookupDocument(
arangodb::Result lookupDocument(
transaction::Methods* trx, arangodb::velocypack::Slice const& key,
ManagedDocumentResult& result) const;
arangodb::RocksDBOperationResult updateDocument(
arangodb::Result updateDocument(
transaction::Methods* trx, LocalDocumentId const& oldDocumentId,
arangodb::velocypack::Slice const& oldDoc,
LocalDocumentId const& newDocumentId,

View File

@ -86,15 +86,6 @@ uint64_t latestSequenceNumber() {
return static_cast<uint64_t>(seq);
}
void addCollectionMapping(uint64_t objectId, TRI_voc_tick_t did,
TRI_voc_cid_t cid) {
StorageEngine* engine = EngineSelectorFeature::ENGINE;
TRI_ASSERT(engine != nullptr);
RocksDBEngine* rocks = static_cast<RocksDBEngine*>(engine);
TRI_ASSERT(rocks->db() != nullptr);
return rocks->addCollectionMapping(objectId, did, cid);
}
std::pair<TRI_voc_tick_t, TRI_voc_cid_t> mapObjectToCollection(
uint64_t objectId) {
StorageEngine* engine = EngineSelectorFeature::ENGINE;
@ -103,6 +94,15 @@ std::pair<TRI_voc_tick_t, TRI_voc_cid_t> mapObjectToCollection(
TRI_ASSERT(rocks->db() != nullptr);
return rocks->mapObjectToCollection(objectId);
}
std::tuple<TRI_voc_tick_t, TRI_voc_cid_t, TRI_idx_iid_t> mapObjectToIndex(
uint64_t objectId) {
StorageEngine* engine = EngineSelectorFeature::ENGINE;
TRI_ASSERT(engine != nullptr);
RocksDBEngine* rocks = static_cast<RocksDBEngine*>(engine);
TRI_ASSERT(rocks->db() != nullptr);
return rocks->mapObjectToIndex(objectId);
}
/// @brief count all keys in the given column family
std::size_t countKeys(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf) {

View File

@ -58,24 +58,6 @@ class ColumnFamilyHandle;
namespace arangodb {
class RocksDBOperationResult : public Result {
public:
RocksDBOperationResult()
: Result(), _keySize(0) {}
RocksDBOperationResult(Result const& other)
: Result(other), _keySize(0) {}
RocksDBOperationResult(Result&& other) noexcept
: Result(std::move(other)), _keySize(0) {}
uint64_t keySize() const { return _keySize; }
void keySize(uint64_t s) { _keySize = s; }
protected:
uint64_t _keySize;
};
class RocksDBMethods;
class RocksDBKeyBounds;
class RocksDBEngine;
@ -232,8 +214,8 @@ arangodb::Result globalRocksDBRemove(
uint64_t latestSequenceNumber();
void addCollectionMapping(uint64_t, TRI_voc_tick_t, TRI_voc_cid_t);
std::pair<TRI_voc_tick_t, TRI_voc_cid_t> mapObjectToCollection(uint64_t);
std::tuple<TRI_voc_tick_t, TRI_voc_cid_t, TRI_idx_iid_t> mapObjectToIndex(uint64_t);
/// @brief count all keys in the given column family
std::size_t countKeys(rocksdb::DB*, rocksdb::ColumnFamilyHandle* cf);

View File

@ -344,8 +344,8 @@ void RocksDBEdgeIndexIterator::lookupInRocksDB(StringRef fromTo) {
_builder.openArray(true);
auto end = _bounds.end();
while (_iterator->Valid() && (cmp->Compare(_iterator->key(), end) < 0)) {
LocalDocumentId const documentId = LocalDocumentId(RocksDBKey::revisionId(
RocksDBEntryType::EdgeIndexValue, _iterator->key()));
LocalDocumentId const documentId = RocksDBKey::documentId(
RocksDBEntryType::EdgeIndexValue, _iterator->key());
// adding revision ID and _from or _to value
_builder.add(VPackValue(documentId.id()));
@ -453,7 +453,7 @@ Result RocksDBEdgeIndex::insertInternal(transaction::Methods* trx,
TRI_ASSERT(fromTo.isString());
auto fromToRef = StringRef(fromTo);
RocksDBKeyLeaser key(trx);
key->constructEdgeIndexValue(_objectId, fromToRef, documentId.id());
key->constructEdgeIndexValue(_objectId, fromToRef, documentId);
VPackSlice toFrom = _isFromIndex
? transaction::helpers::extractToFromDocument(doc)
: transaction::helpers::extractFromFromDocument(doc);
@ -486,7 +486,7 @@ Result RocksDBEdgeIndex::removeInternal(transaction::Methods* trx,
auto fromToRef = StringRef(fromTo);
TRI_ASSERT(fromTo.isString());
RocksDBKeyLeaser key(trx);
key->constructEdgeIndexValue(_objectId, fromToRef, documentId.id());
key->constructEdgeIndexValue(_objectId, fromToRef, documentId);
VPackSlice toFrom = _isFromIndex
? transaction::helpers::extractToFromDocument(doc)
: transaction::helpers::extractFromFromDocument(doc);
@ -525,7 +525,7 @@ void RocksDBEdgeIndex::batchInsert(
TRI_ASSERT(fromTo.isString());
auto fromToRef = StringRef(fromTo);
RocksDBKeyLeaser key(trx);
key->constructEdgeIndexValue(_objectId, fromToRef, doc.first.id());
key->constructEdgeIndexValue(_objectId, fromToRef, doc.first);
blackListKey(fromToRef);
Result r = mthds->Put(_cf, key.ref(),
@ -841,7 +841,7 @@ void RocksDBEdgeIndex::warmupInternal(transaction::Methods* trx,
}
}
if (needsInsert) {
LocalDocumentId const documentId(RocksDBKey::revisionId(RocksDBEntryType::EdgeIndexValue, key));
LocalDocumentId const documentId = RocksDBKey::documentId(RocksDBEntryType::EdgeIndexValue, key);
if (rocksColl->readDocument(trx, documentId, mmdr)) {
builder.add(VPackValue(documentId.id()));

View File

@ -71,8 +71,8 @@
#include "RocksDBEngine/RocksDBTypes.h"
#include "RocksDBEngine/RocksDBV8Functions.h"
#include "RocksDBEngine/RocksDBValue.h"
#include "RocksDBEngine/RocksDBWalAccess.h"
#include "RocksDBEngine/RocksDBView.h"
#include "RocksDBEngine/RocksDBWalAccess.h"
#include "Transaction/Context.h"
#include "Transaction/Options.h"
#include "VocBase/ticks.h"
@ -87,8 +87,8 @@
#include <rocksdb/statistics.h>
#include <rocksdb/table.h>
#include <rocksdb/transaction_log.h>
#include <rocksdb/write_batch.h>
#include <rocksdb/utilities/transaction_db.h>
#include <rocksdb/write_batch.h>
#include <velocypack/Iterator.h>
#include <velocypack/velocypack-aliases.h>
@ -139,9 +139,7 @@ RocksDBEngine::RocksDBEngine(application_features::ApplicationServer* server)
server->addFeature(new RocksDBRecoveryManager(server));
}
RocksDBEngine::~RocksDBEngine() {
shutdownRocksDBInstance();
}
RocksDBEngine::~RocksDBEngine() { shutdownRocksDBInstance(); }
/// shuts down the RocksDB instance. this is called from unprepare
/// and the dtor
@ -150,7 +148,7 @@ void RocksDBEngine::shutdownRocksDBInstance() noexcept {
// turn off RocksDBThrottle, and release our pointers to it
if (nullptr != _listener.get()) {
_listener->StopThread();
} // if
} // if
for (rocksdb::ColumnFamilyHandle* h : RocksDBColumnFamily::_allHandles) {
_db->DestroyColumnFamilyHandle(h);
@ -288,7 +286,8 @@ void RocksDBEngine::start() {
static_cast<int>(opts->_maxBytesForLevelMultiplier);
_options.optimize_filters_for_hits = opts->_optimizeFiltersForHits;
_options.use_direct_reads = opts->_useDirectReads;
_options.use_direct_io_for_flush_and_compaction = opts->_useDirectIoForFlushAndCompaction;
_options.use_direct_io_for_flush_and_compaction =
opts->_useDirectIoForFlushAndCompaction;
// limit the total size of WAL files. This forces the flush of memtables of
// column families still backed by WAL files. If we would not do this, WAL
// files may linger around forever and will not get removed
@ -301,8 +300,8 @@ void RocksDBEngine::start() {
}
LOG_TOPIC(TRACE, arangodb::Logger::ROCKSDB) << "initializing RocksDB, path: '"
<< _path << "', WAL directory '" << _options.wal_dir << "'";
<< _path << "', WAL directory '"
<< _options.wal_dir << "'";
if (opts->_skipCorrupted) {
_options.wal_recovery_mode =
@ -368,9 +367,10 @@ void RocksDBEngine::start() {
rocksdb::BlockBasedTableOptions tableOptions;
if (opts->_blockCacheSize > 0) {
tableOptions.block_cache = rocksdb::NewLRUCache(opts->_blockCacheSize,
static_cast<int>(opts->_blockCacheShardBits));
//tableOptions.cache_index_and_filter_blocks = opts->_compactionReadaheadSize > 0;
tableOptions.block_cache = rocksdb::NewLRUCache(
opts->_blockCacheSize, static_cast<int>(opts->_blockCacheShardBits));
// tableOptions.cache_index_and_filter_blocks =
// opts->_compactionReadaheadSize > 0;
} else {
tableOptions.no_block_cache = true;
}
@ -408,7 +408,8 @@ void RocksDBEngine::start() {
// cf options with fixed 8 byte object id prefix for documents
rocksdb::ColumnFamilyOptions fixedPrefCF(_options);
fixedPrefCF.prefix_extractor = std::shared_ptr<rocksdb::SliceTransform const>(rocksdb::NewFixedPrefixTransform(RocksDBKey::objectIdSize()));
fixedPrefCF.prefix_extractor = std::shared_ptr<rocksdb::SliceTransform const>(
rocksdb::NewFixedPrefixTransform(RocksDBKey::objectIdSize()));
// construct column family options with prefix containing indexed value
rocksdb::ColumnFamilyOptions dynamicPrefCF(_options);
@ -422,7 +423,7 @@ void RocksDBEngine::start() {
// velocypack based index variants with custom comparator
rocksdb::ColumnFamilyOptions vpackFixedPrefCF(fixedPrefCF);
rocksdb::BlockBasedTableOptions tblo2(tableOptions);
tblo2.filter_policy.reset(); // intentionally no bloom filter here
tblo2.filter_policy.reset(); // intentionally no bloom filter here
vpackFixedPrefCF.table_factory = std::shared_ptr<rocksdb::TableFactory>(
rocksdb::NewBlockBasedTableFactory(tblo2));
vpackFixedPrefCF.comparator = _vpackCmp.get();
@ -431,13 +432,13 @@ void RocksDBEngine::start() {
std::vector<rocksdb::ColumnFamilyDescriptor> cfFamilies;
// no prefix families for default column family (Has to be there)
cfFamilies.emplace_back(rocksdb::kDefaultColumnFamilyName,
definitionsCF); // 0
cfFamilies.emplace_back("Documents", fixedPrefCF); // 1
cfFamilies.emplace_back("PrimaryIndex", fixedPrefCF); // 2
cfFamilies.emplace_back("EdgeIndex", dynamicPrefCF); // 3
cfFamilies.emplace_back("VPackIndex", vpackFixedPrefCF); // 4
cfFamilies.emplace_back("GeoIndex", fixedPrefCF); // 5
cfFamilies.emplace_back("FulltextIndex", fixedPrefCF); // 6
definitionsCF); // 0
cfFamilies.emplace_back("Documents", fixedPrefCF); // 1
cfFamilies.emplace_back("PrimaryIndex", fixedPrefCF); // 2
cfFamilies.emplace_back("EdgeIndex", dynamicPrefCF); // 3
cfFamilies.emplace_back("VPackIndex", vpackFixedPrefCF); // 4
cfFamilies.emplace_back("GeoIndex", fixedPrefCF); // 5
cfFamilies.emplace_back("FulltextIndex", fixedPrefCF); // 6
// DO NOT FORGET TO DESTROY THE CFs ON CLOSE
// Update max_write_buffer_number above if you change number of families used
@ -457,7 +458,6 @@ void RocksDBEngine::start() {
// check if we have found the database directory or not
Result res = rocksutils::convertStatus(status);
if (res.errorNumber() != TRI_ERROR_ARANGO_IO_ERROR) {
// not an I/O error. so we better report the error and abort here
LOG_TOPIC(FATAL, arangodb::Logger::STARTUP)
<< "unable to initialize RocksDB engine: " << status.ToString();
@ -554,23 +554,26 @@ void RocksDBEngine::start() {
RocksDBKey key;
key.constructSettingsValue(RocksDBSettingsType::Version);
rocksdb::PinnableSlice oldVersion;
rocksdb::Status s = _db->Get(rocksdb::ReadOptions(), cfHandles[0],
key.string(), &oldVersion);
rocksdb::Status s =
_db->Get(rocksdb::ReadOptions(), cfHandles[0], key.string(), &oldVersion);
if (dbExisted) {
if (s.IsNotFound() || oldVersion.data()[0] < version) {
LOG_TOPIC(FATAL, Logger::ENGINES)
<< "Your db directory is in an old format. Please delete the directory.";
// Should only ever happen if we forgot to provide an upgrade routine
// and CheckVersionFeature did not prevent the start for some reason
LOG_TOPIC(FATAL, Logger::ENGINES) << "Your db directory is in an old "
<< "format. Please downgrade the server, "
<< "export & re-import your data.";
FATAL_ERROR_EXIT();
} else if (oldVersion.data()[0] > version) {
LOG_TOPIC(FATAL, Logger::ENGINES)
<< "You are using an old version of ArangoDB, please update "
<< "before opening this dir.";
<< "You are using an old version of ArangoDB, please update "
<< "before opening this dir.";
FATAL_ERROR_EXIT();
}
}
// store current version
s = _db->Put(rocksdb::WriteOptions(), RocksDBColumnFamily::definitions(), key.string(),
rocksdb::Slice(&version, sizeof(char)));
s = _db->Put(rocksdb::WriteOptions(), RocksDBColumnFamily::definitions(),
key.string(), rocksdb::Slice(&version, sizeof(char)));
TRI_ASSERT(s.ok());
// only enable logger after RocksDB start
@ -612,7 +615,6 @@ void RocksDBEngine::stop() {
return;
}
// in case we missed the beginShutdown somehow, call it again
replicationManager()->beginShutdown();
@ -650,8 +652,8 @@ transaction::ContextData* RocksDBEngine::createTransactionContextData() {
return new RocksDBTransactionContextData();
}
TransactionState* RocksDBEngine::createTransactionState(TRI_vocbase_t* vocbase,
transaction::Options const& options) {
TransactionState* RocksDBEngine::createTransactionState(
TRI_vocbase_t* vocbase, transaction::Options const& options) {
return new RocksDBTransactionState(vocbase, options);
}
@ -826,8 +828,7 @@ int RocksDBEngine::getViews(TRI_vocbase_t* vocbase,
result.openArray();
auto bounds = RocksDBKeyBounds::DatabaseViews(vocbase->id());
for (iter->Seek(bounds.start());
iter->Valid() && iter->key().compare(bounds.end()) < 0;
iter->Next()) {
iter->Valid() && iter->key().compare(bounds.end()) < 0; iter->Next()) {
auto slice = VPackSlice(iter->value().data());
LOG_TOPIC(TRACE, Logger::FIXME) << "got view slice: " << slice.toJson();
@ -848,8 +849,8 @@ std::string RocksDBEngine::versionFilename(TRI_voc_tick_t id) const {
return _basePath + TRI_DIR_SEPARATOR_CHAR + "VERSION-" + std::to_string(id);
}
VPackBuilder RocksDBEngine::getReplicationApplierConfiguration(TRI_vocbase_t* vocbase,
int& status) {
VPackBuilder RocksDBEngine::getReplicationApplierConfiguration(
TRI_vocbase_t* vocbase, int& status) {
RocksDBKey key;
key.constructReplicationApplierConfig(vocbase->id());
return getReplicationApplierConfiguration(key, status);
@ -861,12 +862,14 @@ VPackBuilder RocksDBEngine::getReplicationApplierConfiguration(int& status) {
return getReplicationApplierConfiguration(key, status);
}
VPackBuilder RocksDBEngine::getReplicationApplierConfiguration(RocksDBKey const& key, int& status) {
VPackBuilder RocksDBEngine::getReplicationApplierConfiguration(
RocksDBKey const& key, int& status) {
rocksdb::PinnableSlice value;
auto db = rocksutils::globalRocksDB();
auto opts = rocksdb::ReadOptions();
auto s = db->Get(opts, RocksDBColumnFamily::definitions(), key.string(), &value);
auto s =
db->Get(opts, RocksDBColumnFamily::definitions(), key.string(), &value);
if (!s.ok()) {
status = TRI_ERROR_FILE_NOT_FOUND;
return arangodb::velocypack::Builder();
@ -878,7 +881,8 @@ VPackBuilder RocksDBEngine::getReplicationApplierConfiguration(RocksDBKey const&
return builder;
}
int RocksDBEngine::removeReplicationApplierConfiguration(TRI_vocbase_t* vocbase) {
int RocksDBEngine::removeReplicationApplierConfiguration(
TRI_vocbase_t* vocbase) {
RocksDBKey key;
key.constructReplicationApplierConfig(vocbase->id());
return removeReplicationApplierConfiguration(key);
@ -890,9 +894,10 @@ int RocksDBEngine::removeReplicationApplierConfiguration() {
return removeReplicationApplierConfiguration(key);
}
int RocksDBEngine::removeReplicationApplierConfiguration(RocksDBKey const& key) {
auto status = rocksutils::globalRocksDBRemove(RocksDBColumnFamily::definitions(),
key.string());
int RocksDBEngine::removeReplicationApplierConfiguration(
RocksDBKey const& key) {
auto status = rocksutils::globalRocksDBRemove(
RocksDBColumnFamily::definitions(), key.string());
if (!status.ok()) {
return status.errorNumber();
}
@ -907,13 +912,15 @@ int RocksDBEngine::saveReplicationApplierConfiguration(
return saveReplicationApplierConfiguration(key, slice, doSync);
}
int RocksDBEngine::saveReplicationApplierConfiguration(arangodb::velocypack::Slice slice, bool doSync) {
int RocksDBEngine::saveReplicationApplierConfiguration(
arangodb::velocypack::Slice slice, bool doSync) {
RocksDBKey key;
key.constructReplicationApplierConfig(databaseIdForGlobalApplier);
return saveReplicationApplierConfiguration(key, slice, doSync);
}
int RocksDBEngine::saveReplicationApplierConfiguration(RocksDBKey const& key, arangodb::velocypack::Slice slice, bool doSync) {
int RocksDBEngine::saveReplicationApplierConfiguration(
RocksDBKey const& key, arangodb::velocypack::Slice slice, bool doSync) {
auto value = RocksDBValue::ReplicationApplierConfig(slice);
auto status = rocksutils::globalRocksDBPut(RocksDBColumnFamily::definitions(),
@ -950,8 +957,8 @@ TRI_vocbase_t* RocksDBEngine::createDatabase(
int RocksDBEngine::writeCreateDatabaseMarker(TRI_voc_tick_t id,
VPackSlice const& slice) {
Result res = writeDatabaseMarker(id, slice,
RocksDBLogValue::DatabaseCreate(id));
Result res =
writeDatabaseMarker(id, slice, RocksDBLogValue::DatabaseCreate(id));
return res.errorNumber();
}
@ -961,13 +968,12 @@ Result RocksDBEngine::writeDatabaseMarker(TRI_voc_tick_t id,
RocksDBKey key;
key.constructDatabase(id);
auto value = RocksDBValue::Database(slice);
rocksdb::WriteOptions wo; // TODO: check which options would make sense
rocksdb::WriteOptions wo;
// Write marker + key into RocksDB inside one batch
rocksdb::WriteBatch batch;
batch.PutLogData(logValue.slice());
batch.Put(RocksDBColumnFamily::definitions(),
key.string(), value.string());
batch.Put(RocksDBColumnFamily::definitions(), key.string(), value.string());
rocksdb::Status res = _db->Write(wo, &batch);
return rocksutils::convertStatus(res);
}
@ -979,7 +985,7 @@ int RocksDBEngine::writeCreateCollectionMarker(TRI_voc_tick_t databaseId,
RocksDBKey key;
key.constructCollection(databaseId, cid);
auto value = RocksDBValue::Collection(slice);
rocksdb::WriteOptions wo; // TODO: check which options would make sense
rocksdb::WriteOptions wo;
// Write marker + key into RocksDB inside one batch
rocksdb::WriteBatch batch;
@ -1001,7 +1007,8 @@ void RocksDBEngine::prepareDropDatabase(TRI_vocbase_t* vocbase,
builder.close();
auto log = RocksDBLogValue::DatabaseDrop(vocbase->id());
Result res = writeDatabaseMarker(vocbase->id(), builder.slice(), std::move(log));
Result res =
writeDatabaseMarker(vocbase->id(), builder.slice(), std::move(log));
status = res.errorNumber();
}
@ -1061,7 +1068,7 @@ arangodb::Result RocksDBEngine::dropCollection(
RocksDBCollection* coll = toRocksDBCollection(collection->getPhysical());
uint64_t const numberDocuments = coll->numberDocuments();
rocksdb::WriteOptions options; // TODO: check which options would make sense
rocksdb::WriteOptions wo;
// If we get here the collection is safe to drop.
//
@ -1083,15 +1090,15 @@ arangodb::Result RocksDBEngine::dropCollection(
TRI_ASSERT(collection->status() == TRI_VOC_COL_STATUS_DELETED);
// Prepare collection remove batch
RocksDBLogValue logValue =
RocksDBLogValue::CollectionDrop(vocbase->id(), collection->cid(),
StringRef(collection->globallyUniqueId()));
RocksDBLogValue logValue = RocksDBLogValue::CollectionDrop(
vocbase->id(), collection->cid(),
StringRef(collection->globallyUniqueId()));
rocksdb::WriteBatch batch;
batch.PutLogData(logValue.slice());
RocksDBKey key;
key.constructCollection(vocbase->id(), collection->cid());
batch.Delete(RocksDBColumnFamily::definitions(), key.string());
rocksdb::Status res = _db->Write(options, &batch);
rocksdb::Status res = _db->Write(wo, &batch);
// TODO FAILURE Simulate !res.ok()
if (!res.ok()) {
@ -1124,11 +1131,14 @@ arangodb::Result RocksDBEngine::dropCollection(
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
//check if documents have been deleted
size_t numDocs = rocksutils::countKeyRange(rocksutils::globalRocksDB(), bounds, true);
// check if documents have been deleted
size_t numDocs =
rocksutils::countKeyRange(rocksutils::globalRocksDB(), bounds, true);
if (numDocs > 0) {
std::string errorMsg("deletion check in collection drop failed - not all documents in the index have been deleted. remaining: ");
std::string errorMsg(
"deletion check in collection drop failed - not all documents in the "
"index have been deleted. remaining: ");
errorMsg.append(std::to_string(numDocs));
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, errorMsg);
}
@ -1144,7 +1154,8 @@ arangodb::Result RocksDBEngine::dropCollection(
// We try to remove all indexed values.
// If it does not work they cannot be accessed any more and leaked.
// User View remains consistent.
LOG_TOPIC(ERR, Logger::FIXME) << "unable to drop index: " << TRI_errno_string(dropRes);
LOG_TOPIC(ERR, Logger::FIXME) << "unable to drop index: "
<< TRI_errno_string(dropRes);
return TRI_ERROR_NO_ERROR;
}
}
@ -1202,16 +1213,15 @@ void RocksDBEngine::createIndex(TRI_vocbase_t* vocbase,
void RocksDBEngine::unloadCollection(TRI_vocbase_t* vocbase,
arangodb::LogicalCollection* collection) {
// TODO: does anything else have to happen?
collection->setStatus(TRI_VOC_COL_STATUS_UNLOADED);
}
void RocksDBEngine::createView(TRI_vocbase_t* vocbase, TRI_voc_cid_t id,
arangodb::LogicalView const*) {
rocksdb::WriteBatch batch;
rocksdb::WriteOptions wo; // TODO: check which options would make sense
RocksDBLogValue logValue = RocksDBLogValue::ViewCreate(vocbase->id(), id);
rocksdb::WriteOptions wo;
RocksDBLogValue logValue = RocksDBLogValue::ViewCreate(vocbase->id(), id);
RocksDBKey key;
key.constructView(vocbase->id(), id);
auto value = RocksDBValue::View(VPackSlice::emptyObjectSlice());
@ -1229,9 +1239,9 @@ void RocksDBEngine::createView(TRI_vocbase_t* vocbase, TRI_voc_cid_t id,
// asks the storage engine to persist renaming of a view
// This will write a renameMarker if not in recovery
Result RocksDBEngine::renameView(
TRI_vocbase_t* vocbase, std::shared_ptr<arangodb::LogicalView> view,
std::string const& oldName) {
Result RocksDBEngine::renameView(TRI_vocbase_t* vocbase,
std::shared_ptr<arangodb::LogicalView> view,
std::string const& oldName) {
return persistView(vocbase, view.get());
}
@ -1287,24 +1297,43 @@ void RocksDBEngine::addRestHandlers(rest::RestHandlerFactory* handlerFactory) {
void RocksDBEngine::addCollectionMapping(uint64_t objectId, TRI_voc_tick_t did,
TRI_voc_cid_t cid) {
if (objectId == 0) {
return;
if (objectId != 0) {
WRITE_LOCKER(guard, _mapLock);
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
auto it = _collectionMap.find(objectId);
if (it != _collectionMap.end()) {
TRI_ASSERT(it->second.first == did);
TRI_ASSERT(it->second.second == cid);
}
#endif
_collectionMap[objectId] = std::make_pair(did, cid);
}
WRITE_LOCKER(guard, _mapLock);
_collectionMap[objectId] = std::make_pair(did, cid);
}
void RocksDBEngine::addIndexMapping(uint64_t objectId, Index* index) {
if (objectId == 0) {
return;
void RocksDBEngine::addIndexMapping(uint64_t objectId, TRI_voc_tick_t did,
TRI_voc_cid_t cid, TRI_idx_iid_t iid) {
if (objectId != 0) {
WRITE_LOCKER(guard, _mapLock);
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
auto it = _indexMap.find(objectId);
if (it != _indexMap.end()) {
TRI_ASSERT(std::get<0>(it->second) == did);
TRI_ASSERT(std::get<1>(it->second) == cid);
TRI_ASSERT(std::get<2>(it->second) == iid);
}
#endif
_indexMap[objectId] = std::make_tuple(did, cid, iid);
}
WRITE_LOCKER(guard, _mapLock);
_indexMap[objectId] = index;
}
std::pair<TRI_voc_tick_t, TRI_voc_cid_t> RocksDBEngine::mapObjectToCollection(
void RocksDBEngine::removeIndexMapping(uint64_t objectId) {
if (objectId != 0) {
WRITE_LOCKER(guard, _mapLock);
_indexMap.erase(objectId);
}
}
RocksDBEngine::CollectionPair RocksDBEngine::mapObjectToCollection(
uint64_t objectId) const {
READ_LOCKER(guard, _mapLock);
auto it = _collectionMap.find(objectId);
@ -1314,17 +1343,18 @@ std::pair<TRI_voc_tick_t, TRI_voc_cid_t> RocksDBEngine::mapObjectToCollection(
return it->second;
}
Index* RocksDBEngine::mapObjectToIndex(uint64_t objectId) const {
RocksDBEngine::IndexTriple RocksDBEngine::mapObjectToIndex(
uint64_t objectId) const {
READ_LOCKER(guard, _mapLock);
auto it = _indexMap.find(objectId);
if (it == _indexMap.end()) {
return nullptr;
return RocksDBEngine::IndexTriple(0, 0, 0);
}
return it->second;
}
Result RocksDBEngine::flushWal(bool waitForSync, bool waitForCollector,
bool /*writeShutdownFile*/) {
bool /*writeShutdownFile*/) {
rocksdb::Status status;
#ifndef _WIN32
// SyncWAL always reports "not implemented" on Windows
@ -1347,7 +1377,8 @@ Result RocksDBEngine::flushWal(bool waitForSync, bool waitForCollector,
return TRI_ERROR_NO_ERROR;
}
void RocksDBEngine::waitForEstimatorSync(std::chrono::milliseconds maxWaitTime) {
void RocksDBEngine::waitForEstimatorSync(
std::chrono::milliseconds maxWaitTime) {
auto start = std::chrono::high_resolution_clock::now();
auto beginSeq = _db->GetLatestSequenceNumber();
while (std::chrono::high_resolution_clock::now() - start < maxWaitTime) {
@ -1370,7 +1401,8 @@ Result RocksDBEngine::registerRecoveryHelper(
return {TRI_ERROR_NO_ERROR};
}
std::vector<std::shared_ptr<RocksDBRecoveryHelper>> const& RocksDBEngine::recoveryHelpers() {
std::vector<std::shared_ptr<RocksDBRecoveryHelper>> const&
RocksDBEngine::recoveryHelpers() {
return _recoveryHelpers;
}
@ -1395,7 +1427,6 @@ std::vector<std::string> RocksDBEngine::currentWalFiles() {
return names;
}
void RocksDBEngine::determinePrunableWalFiles(TRI_voc_tick_t minTickExternal) {
WRITE_LOCKER(lock, _walFileLock);
rocksdb::VectorLogPtr files;
@ -1423,7 +1454,10 @@ void RocksDBEngine::determinePrunableWalFiles(TRI_voc_tick_t minTickExternal) {
auto const& f = files[current].get();
if (f->Type() == rocksdb::WalFileType::kArchivedLogFile) {
if (_prunableWalFiles.find(f->PathName()) == _prunableWalFiles.end()) {
LOG_TOPIC(DEBUG, Logger::ROCKSDB) << "RocksDB WAL file '" << f->PathName() << "' with start sequence " << f->StartSequence() << " added to prunable list";
LOG_TOPIC(DEBUG, Logger::ROCKSDB)
<< "RocksDB WAL file '" << f->PathName()
<< "' with start sequence " << f->StartSequence()
<< " added to prunable list";
_prunableWalFiles.emplace(f->PathName(),
TRI_microtime() + _pruneWaitTime);
}
@ -1441,7 +1475,8 @@ void RocksDBEngine::pruneWalFiles() {
/* no hoisting */) {
// check if WAL file is expired
if ((*it).second < TRI_microtime()) {
LOG_TOPIC(DEBUG, Logger::ROCKSDB) << "deleting RocksDB WAL file '" << (*it).first << "'";
LOG_TOPIC(DEBUG, Logger::ROCKSDB) << "deleting RocksDB WAL file '"
<< (*it).first << "'";
auto s = _db->DeleteFile((*it).first);
// apparently there is a case where a file was already deleted
// but is still in _prunableWalFiles. In this case we get an invalid
@ -1460,12 +1495,12 @@ void RocksDBEngine::pruneWalFiles() {
Result RocksDBEngine::dropDatabase(TRI_voc_tick_t id) {
using namespace rocksutils;
Result res;
rocksdb::WriteOptions options; // TODO: check which options would make sense
rocksdb::WriteOptions wo;
// remove views
for (auto const& val : viewKVPairs(id)) {
res = globalRocksDBRemove(RocksDBColumnFamily::definitions(),
val.first.string(), options);
val.first.string(), wo);
if (res.fail()) {
return res;
}
@ -1501,8 +1536,9 @@ Result RocksDBEngine::dropDatabase(TRI_voc_tick_t id) {
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
//check if documents have been deleted
numDocsLeft += rocksutils::countKeyRange(rocksutils::globalRocksDB(), bounds, prefix_same_as_start);
// check if documents have been deleted
numDocsLeft += rocksutils::countKeyRange(rocksutils::globalRocksDB(),
bounds, prefix_same_as_start);
#endif
}
}
@ -1518,21 +1554,22 @@ Result RocksDBEngine::dropDatabase(TRI_voc_tick_t id) {
// delete collection meta-data
_settingsManager->removeCounter(objectId);
res = globalRocksDBRemove(RocksDBColumnFamily::definitions(),
val.first.string(), options);
val.first.string(), wo);
if (res.fail()) {
return res;
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
//check if documents have been deleted
numDocsLeft += rocksutils::countKeyRange(rocksutils::globalRocksDB(), bounds, true);
// check if documents have been deleted
numDocsLeft +=
rocksutils::countKeyRange(rocksutils::globalRocksDB(), bounds, true);
#endif
}
RocksDBKey key;
key.constructDatabase(id);
res = rocksutils::globalRocksDBRemove(RocksDBColumnFamily::definitions(),
key.string(), options);
key.string(), wo);
// remove VERSION file for database. it's not a problem when this fails
// because it will simply remain there and be ignored on subsequent starts
@ -1540,7 +1577,9 @@ Result RocksDBEngine::dropDatabase(TRI_voc_tick_t id) {
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
if (numDocsLeft > 0) {
std::string errorMsg("deletion check in drop database failed - not all documents have been deleted. remaining: ");
std::string errorMsg(
"deletion check in drop database failed - not all documents have been "
"deleted. remaining: ");
errorMsg.append(std::to_string(numDocsLeft));
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, errorMsg);
}
@ -1650,8 +1689,8 @@ TRI_vocbase_t* RocksDBEngine::openExistingDatabase(TRI_voc_tick_t id,
for (auto const& it : VPackArrayIterator(slice)) {
// we found a collection that is still active
TRI_ASSERT(!it.get("id").isNone() || !it.get("cid").isNone());
auto uniqCol =
std::make_unique<arangodb::LogicalCollection>(vocbase.get(), it, false);
auto uniqCol = std::make_unique<arangodb::LogicalCollection>(
vocbase.get(), it, false);
auto collection = uniqCol.get();
TRI_ASSERT(collection != nullptr);
StorageEngine::registerCollection(vocbase.get(), uniqCol.get());
@ -1711,12 +1750,17 @@ void RocksDBEngine::getStatistics(VPackBuilder& builder) const {
// estimate size on disk and in memtables
uint64_t out = 0;
rocksdb::Range r(rocksdb::Slice("\x00\x00\x00\x00\x00\x00\x00\x00", 8), rocksdb::Slice("\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff", 16));
rocksdb::Range r(
rocksdb::Slice("\x00\x00\x00\x00\x00\x00\x00\x00", 8),
rocksdb::Slice(
"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff",
16));
_db->GetApproximateSizes(c, &r, 1, &out,
static_cast<uint8_t>(
rocksdb::DB::SizeApproximationFlags::INCLUDE_MEMTABLES |
rocksdb::DB::SizeApproximationFlags::INCLUDE_FILES));
_db->GetApproximateSizes(
c, &r, 1, &out,
static_cast<uint8_t>(
rocksdb::DB::SizeApproximationFlags::INCLUDE_MEMTABLES |
rocksdb::DB::SizeApproximationFlags::INCLUDE_FILES));
builder.add("memory", VPackValue(out));
builder.close();
@ -1753,7 +1797,8 @@ void RocksDBEngine::getStatistics(VPackBuilder& builder) const {
if (_options.statistics) {
for (auto const& stat : rocksdb::TickersNameMap) {
builder.add(stat.second, VPackValue(_options.statistics->getTickerCount(stat.first)));
builder.add(stat.second,
VPackValue(_options.statistics->getTickerCount(stat.first)));
}
}
if (_options.table_factory) {
@ -1763,14 +1808,11 @@ void RocksDBEngine::getStatistics(VPackBuilder& builder) const {
if (bto != nullptr && bto->block_cache != nullptr) {
// block cache is present
builder.add(
"rocksdb.block-cache-used",
VPackValue(bto->block_cache->GetUsage()));
builder.add("rocksdb.block-cache-used",
VPackValue(bto->block_cache->GetUsage()));
} else {
// no block cache present
builder.add(
"rocksdb.block-cache-used",
VPackValue(0));
builder.add("rocksdb.block-cache-used", VPackValue(0));
}
}
}
@ -1925,8 +1967,8 @@ Result RocksDBEngine::lastLogger(
}
WalAccess const* RocksDBEngine::walAccess() const {
TRI_ASSERT(_walAccess);
return _walAccess.get();
TRI_ASSERT(_walAccess);
return _walAccess.get();
}
/// @brief get compression supported by RocksDB

View File

@ -72,6 +72,8 @@ class RocksDBEngine final : public StorageEngine {
// create the storage engine
explicit RocksDBEngine(application_features::ApplicationServer*);
~RocksDBEngine();
public:
// inherited from ApplicationFeature
// ---------------------------------
@ -253,11 +255,15 @@ class RocksDBEngine final : public StorageEngine {
RocksDBLogValue&& logValue);
void addCollectionMapping(uint64_t, TRI_voc_tick_t, TRI_voc_cid_t);
void addIndexMapping(uint64_t, Index*);
void addIndexMapping(uint64_t objectId, TRI_voc_tick_t,
TRI_voc_cid_t, TRI_idx_iid_t);
void removeIndexMapping(uint64_t);
std::pair<TRI_voc_tick_t, TRI_voc_cid_t> mapObjectToCollection(
uint64_t) const;
Index* mapObjectToIndex(uint64_t) const;
// Identifies a collection
typedef std::pair<TRI_voc_tick_t, TRI_voc_cid_t> CollectionPair;
typedef std::tuple<TRI_voc_tick_t, TRI_voc_cid_t, TRI_idx_iid_t> IndexTriple;
CollectionPair mapObjectToCollection(uint64_t) const;
IndexTriple mapObjectToIndex(uint64_t) const;
std::vector<std::string> currentWalFiles();
void determinePrunableWalFiles(TRI_voc_tick_t minTickToKeep);
@ -346,12 +352,10 @@ class RocksDBEngine final : public StorageEngine {
static std::vector<std::shared_ptr<RocksDBRecoveryHelper>> _recoveryHelpers;
mutable basics::ReadWriteLock _mapLock;
std::unordered_map<uint64_t, std::pair<TRI_voc_tick_t, TRI_voc_cid_t>>
_collectionMap;
std::unordered_map<uint64_t, Index*> _indexMap;
std::unordered_map<uint64_t, CollectionPair> _collectionMap;
std::unordered_map<uint64_t, IndexTriple> _indexMap;
mutable basics::ReadWriteLock _walFileLock;
// which WAL files can be pruned when
std::unordered_map<std::string, double> _prunableWalFiles;

View File

@ -186,7 +186,7 @@ Result RocksDBFulltextIndex::insertInternal(transaction::Methods* trx,
// size_t const count = words.size();
for (std::string const& word : words) {
RocksDBKeyLeaser key(trx);
key->constructFulltextIndexValue(_objectId, StringRef(word), documentId.id());
key->constructFulltextIndexValue(_objectId, StringRef(word), documentId);
Result r = mthd->Put(_cf, key.ref(), value.string(), rocksutils::index);
if (!r.ok()) {
@ -212,7 +212,7 @@ Result RocksDBFulltextIndex::removeInternal(transaction::Methods* trx,
int res = TRI_ERROR_NO_ERROR;
for (std::string const& word : words) {
RocksDBKeyLeaser key(trx);
key->constructFulltextIndexValue(_objectId, StringRef(word), documentId.id());
key->constructFulltextIndexValue(_objectId, StringRef(word), documentId);
Result r = mthd->Delete(_cf, key.ref());
if (!r.ok()) {
@ -428,8 +428,8 @@ Result RocksDBFulltextIndex::applyQueryToken(
return rocksutils::convertStatus(s);
}
LocalDocumentId documentId(RocksDBKey::revisionId(
RocksDBEntryType::FulltextIndexValue, iter->key()));
LocalDocumentId documentId = RocksDBKey::documentId(
RocksDBEntryType::FulltextIndexValue, iter->key());
if (token.operation == FulltextQueryToken::AND) {
intersect.insert(documentId);
} else if (token.operation == FulltextQueryToken::OR) {

View File

@ -64,7 +64,7 @@ RocksDBIndex::RocksDBIndex(
createCache();
}
RocksDBEngine* engine = static_cast<RocksDBEngine*>(EngineSelectorFeature::ENGINE);
engine->addIndexMapping(_objectId, this);
engine->addIndexMapping(_objectId, collection->vocbase()->id(), collection->cid(), _iid);
}
RocksDBIndex::RocksDBIndex(TRI_idx_iid_t id, LogicalCollection* collection,
@ -85,7 +85,7 @@ RocksDBIndex::RocksDBIndex(TRI_idx_iid_t id, LogicalCollection* collection,
createCache();
}
RocksDBEngine* engine = static_cast<RocksDBEngine*>(EngineSelectorFeature::ENGINE);
engine->addIndexMapping(_objectId, this);
engine->addIndexMapping(_objectId, collection->vocbase()->id(), collection->cid(), _iid);
}
RocksDBIndex::~RocksDBIndex() {
@ -248,6 +248,9 @@ Result RocksDBIndex::updateInternal(transaction::Methods* trx, RocksDBMethods* m
LocalDocumentId const& newDocumentId,
arangodb::velocypack::Slice const& newDoc,
OperationMode mode) {
// It is illegal to call this method on the primary index
// RocksDBPrimaryIndex must override this method accordingly
TRI_ASSERT(type() != TRI_IDX_TYPE_PRIMARY_INDEX);
Result res = removeInternal(trx, mthd, oldDocumentId, oldDoc, mode);
if (!res.ok()) {
return res;

View File

@ -97,10 +97,7 @@ bool RocksDBAllIndexIterator::next(LocalDocumentIdCallback const& cb, size_t lim
TRI_ASSERT(_bounds.objectId() == RocksDBKey::objectId(_iterator->key()));
#endif
TRI_voc_rid_t revisionId =
RocksDBKey::revisionId(RocksDBEntryType::Document, _iterator->key());
cb(LocalDocumentId(revisionId));
cb(RocksDBKey::documentId(RocksDBEntryType::Document, _iterator->key()));
--limit;
if (_reverse) {
_iterator->Prev();
@ -128,8 +125,8 @@ bool RocksDBAllIndexIterator::nextDocument(
}
while (limit > 0) {
TRI_voc_rid_t documentId = RocksDBKey::revisionId(RocksDBEntryType::Document, _iterator->key());
cb(LocalDocumentId(documentId), VPackSlice(_iterator->value().data()));
cb(RocksDBKey::documentId(RocksDBEntryType::Document, _iterator->key()),
VPackSlice(_iterator->value().data()));
--limit;
if (_reverse) {
@ -200,7 +197,7 @@ RocksDBAnyIndexIterator::RocksDBAnyIndexIterator(
auto initialKey = RocksDBKey();
initialKey.constructDocument(
static_cast<RocksDBCollection*>(col->getPhysical())->objectId(),
RandomGenerator::interval(UINT64_MAX)
LocalDocumentId(RandomGenerator::interval(UINT64_MAX))
);
_iterator->Seek(initialKey.string());
@ -250,9 +247,7 @@ bool RocksDBAnyIndexIterator::next(LocalDocumentIdCallback const& cb, size_t lim
}
while (limit > 0) {
TRI_voc_rid_t revisionId =
RocksDBKey::revisionId(RocksDBEntryType::Document, _iterator->key());
cb(LocalDocumentId(revisionId));
cb(RocksDBKey::documentId(RocksDBEntryType::Document, _iterator->key()));
--limit;
_returned++;
_iterator->Next();
@ -279,8 +274,8 @@ bool RocksDBAnyIndexIterator::nextDocument(
}
while (limit > 0) {
TRI_voc_rid_t documentId = RocksDBKey::revisionId(RocksDBEntryType::Document, _iterator->key());
cb(LocalDocumentId(documentId), VPackSlice(_iterator->value().data()));
cb(RocksDBKey::documentId(RocksDBEntryType::Document, _iterator->key()),
VPackSlice(_iterator->value().data()));
--limit;
_returned++;
_iterator->Next();
@ -340,8 +335,7 @@ bool RocksDBSortedAllIterator::next(LocalDocumentIdCallback const& cb, size_t li
}
while (limit > 0) {
LocalDocumentId documentId(RocksDBValue::revisionId(_iterator->value()));
cb(documentId);
cb(RocksDBValue::documentId(_iterator->value()));
--limit;

View File

@ -59,13 +59,13 @@ void RocksDBKey::constructCollection(TRI_voc_tick_t databaseId,
void RocksDBKey::constructDocument(uint64_t collectionId,
TRI_voc_rid_t revisionId) {
LocalDocumentId documentId) {
_type = RocksDBEntryType::Document;
size_t keyLength = 2 * sizeof(uint64_t);
_buffer.clear();
_buffer.reserve(keyLength);
uint64ToPersistent(_buffer, collectionId);
uint64ToPersistent(_buffer, revisionId);
uint64ToPersistent(_buffer, documentId.id());
TRI_ASSERT(_buffer.size() == keyLength);
_slice = rocksdb::Slice(_buffer.data(), keyLength);
}
@ -90,7 +90,7 @@ void RocksDBKey::constructPrimaryIndexValue(uint64_t indexId,
void RocksDBKey::constructEdgeIndexValue(uint64_t indexId,
arangodb::StringRef const& vertexId,
TRI_voc_rid_t revisionId) {
LocalDocumentId documentId) {
_type = RocksDBEntryType::EdgeIndexValue;
size_t keyLength = (sizeof(uint64_t) + sizeof(char)) * 2 + vertexId.size();
_buffer.clear();
@ -98,7 +98,7 @@ void RocksDBKey::constructEdgeIndexValue(uint64_t indexId,
uint64ToPersistent(_buffer, indexId);
_buffer.append(vertexId.data(), vertexId.length());
_buffer.push_back(_stringSeparator);
uint64ToPersistent(_buffer, revisionId);
uint64ToPersistent(_buffer, documentId.id());
_buffer.push_back(0xFFU);
TRI_ASSERT(_buffer.size() == keyLength);
_slice = rocksdb::Slice(_buffer.data(), keyLength);
@ -106,7 +106,7 @@ void RocksDBKey::constructEdgeIndexValue(uint64_t indexId,
void RocksDBKey::constructVPackIndexValue(uint64_t indexId,
VPackSlice const& indexValues,
TRI_voc_rid_t revisionId) {
LocalDocumentId documentId) {
_type = RocksDBEntryType::VPackIndexValue;
size_t const byteSize = static_cast<size_t>(indexValues.byteSize());
size_t keyLength = 2 * sizeof(uint64_t) + byteSize;
@ -114,7 +114,7 @@ void RocksDBKey::constructVPackIndexValue(uint64_t indexId,
_buffer.reserve(keyLength);
uint64ToPersistent(_buffer, indexId);
_buffer.append(reinterpret_cast<char const*>(indexValues.begin()), byteSize);
uint64ToPersistent(_buffer, revisionId);
uint64ToPersistent(_buffer, documentId.id());
TRI_ASSERT(_buffer.size() == keyLength);
_slice = rocksdb::Slice(_buffer.data(), keyLength);
}
@ -134,7 +134,7 @@ void RocksDBKey::constructUniqueVPackIndexValue(uint64_t indexId,
void RocksDBKey::constructFulltextIndexValue(uint64_t indexId,
arangodb::StringRef const& word,
TRI_voc_rid_t revisionId) {
LocalDocumentId documentId) {
_type = RocksDBEntryType::FulltextIndexValue;
size_t keyLength = sizeof(uint64_t) * 2 + word.size() + sizeof(char);
_buffer.clear();
@ -142,7 +142,7 @@ void RocksDBKey::constructFulltextIndexValue(uint64_t indexId,
uint64ToPersistent(_buffer, indexId);
_buffer.append(word.data(), word.length());
_buffer.push_back(_stringSeparator);
uint64ToPersistent(_buffer, revisionId);
uint64ToPersistent(_buffer, documentId.id());
TRI_ASSERT(_buffer.size() == keyLength);
_slice = rocksdb::Slice(_buffer.data(), keyLength);
}
@ -271,13 +271,13 @@ TRI_voc_cid_t RocksDBKey::viewId(rocksdb::Slice const& slice) {
return viewId(slice.data(), slice.size());
}
TRI_voc_rid_t RocksDBKey::revisionId(RocksDBKey const& key) {
return revisionId(key._type, key._buffer.data(), key._buffer.size());
LocalDocumentId RocksDBKey::documentId(RocksDBKey const& key) {
return documentId(key._type, key._buffer.data(), key._buffer.size());
}
TRI_voc_rid_t RocksDBKey::revisionId(RocksDBEntryType type,
rocksdb::Slice const& slice) {
return revisionId(type, slice.data(), slice.size());
LocalDocumentId RocksDBKey::documentId(RocksDBEntryType type,
rocksdb::Slice const& slice) {
return documentId(type, slice.data(), slice.size());
}
arangodb::StringRef RocksDBKey::primaryKey(RocksDBKey const& key) {
@ -368,7 +368,7 @@ TRI_voc_cid_t RocksDBKey::objectId(char const* data, size_t size) {
return uint64FromPersistent(data);
}
TRI_voc_rid_t RocksDBKey::revisionId(RocksDBEntryType type, char const* data,
LocalDocumentId RocksDBKey::documentId(RocksDBEntryType type, char const* data,
size_t size) {
TRI_ASSERT(data != nullptr);
TRI_ASSERT(size >= sizeof(char));
@ -378,14 +378,14 @@ TRI_voc_rid_t RocksDBKey::revisionId(RocksDBEntryType type, char const* data,
case RocksDBEntryType::FulltextIndexValue: {
TRI_ASSERT(size >= (2 * sizeof(uint64_t)));
// last 8 bytes should be the revision
return uint64FromPersistent(data + size - sizeof(uint64_t));
return LocalDocumentId(uint64FromPersistent(data + size - sizeof(uint64_t)));
}
case RocksDBEntryType::EdgeIndexValue: {
TRI_ASSERT(size >= (sizeof(char) * 3 + 2 * sizeof(uint64_t)));
// 1 byte prefix + 8 byte objectID + _from/_to + 1 byte \0
// + 8 byte revision ID + 1-byte 0xff
return uint64FromPersistent(data + size - sizeof(uint64_t) -
sizeof(char));
return LocalDocumentId(uint64FromPersistent(data + size - sizeof(uint64_t) -
sizeof(char)));
}
default:

View File

@ -28,6 +28,7 @@
#include "Basics/Common.h"
#include "Basics/StringRef.h"
#include "RocksDBEngine/RocksDBTypes.h"
#include "VocBase/LocalDocumentId.h"
#include "VocBase/vocbase.h"
#include <rocksdb/slice.h>
@ -71,7 +72,7 @@ class RocksDBKey {
//////////////////////////////////////////////////////////////////////////////
/// @brief Create a fully-specified document key
//////////////////////////////////////////////////////////////////////////////
void constructDocument(uint64_t objectId, TRI_voc_rid_t revisionId);
void constructDocument(uint64_t objectId, LocalDocumentId docId);
//////////////////////////////////////////////////////////////////////////////
/// @brief Create a fully-specified key for an entry in a primary index
@ -92,7 +93,7 @@ class RocksDBKey {
//////////////////////////////////////////////////////////////////////////////
void constructEdgeIndexValue(uint64_t indexId,
arangodb::StringRef const& vertexId,
TRI_voc_rid_t revisionId);
LocalDocumentId docId);
//////////////////////////////////////////////////////////////////////////////
/// @brief Create a fully-specified key for an entry in a user-defined,
@ -103,7 +104,7 @@ class RocksDBKey {
//////////////////////////////////////////////////////////////////////////////
void constructVPackIndexValue(uint64_t indexId,
VPackSlice const& indexValues,
TRI_voc_rid_t revisionId);
LocalDocumentId docId);
//////////////////////////////////////////////////////////////////////////////
/// @brief Create a fully-specified key for an entry in a unique user-defined
@ -120,7 +121,7 @@ class RocksDBKey {
//////////////////////////////////////////////////////////////////////////////
void constructFulltextIndexValue(uint64_t indexId,
arangodb::StringRef const& word,
TRI_voc_rid_t revisionId);
LocalDocumentId docId);
//////////////////////////////////////////////////////////////////////////////
/// @brief Create a fully-specified key for a geoIndexValue
@ -213,12 +214,12 @@ class RocksDBKey {
static uint64_t objectId(rocksdb::Slice const&);
//////////////////////////////////////////////////////////////////////////////
/// @brief Extracts the revisionId from a key
/// @brief Extracts the LocalDocumentId from a key
///
/// May be called only on Document keys. Other types will throw.
//////////////////////////////////////////////////////////////////////////////
static TRI_voc_rid_t revisionId(RocksDBKey const&);
static TRI_voc_rid_t revisionId(RocksDBEntryType type, rocksdb::Slice const&);
static LocalDocumentId documentId(RocksDBKey const&);
static LocalDocumentId documentId(RocksDBEntryType type, rocksdb::Slice const&);
//////////////////////////////////////////////////////////////////////////////
/// @brief Extracts the primary key (`_key`) from a key
@ -299,7 +300,7 @@ class RocksDBKey {
// valid on data entries like document, edge, vpack
static TRI_voc_cid_t objectId(char const* data, size_t size);
static TRI_voc_rid_t revisionId(RocksDBEntryType, char const*, size_t);
static LocalDocumentId documentId(RocksDBEntryType, char const*, size_t);
static StringRef primaryKey(char const* data, size_t size);
static StringRef vertexId(char const* data, size_t size);
static VPackSlice indexedVPack(char const* data, size_t size);

View File

@ -108,28 +108,23 @@ RocksDBLogValue RocksDBLogValue::BeginTransaction(TRI_voc_tick_t dbid,
return RocksDBLogValue(RocksDBLogType::BeginTransaction, dbid, tid);
}
RocksDBLogValue RocksDBLogValue::DocumentOpsPrologue(TRI_voc_cid_t cid) {
return RocksDBLogValue(RocksDBLogType::DocumentOperationsPrologue, cid);
RocksDBLogValue RocksDBLogValue::CommitTransaction(TRI_voc_tick_t dbid,
TRI_voc_tid_t tid) {
return RocksDBLogValue(RocksDBLogType::CommitTransaction, dbid, tid);
}
RocksDBLogValue RocksDBLogValue::DocumentRemove(
arangodb::StringRef const& key) {
return RocksDBLogValue(RocksDBLogType::DocumentRemove, key);
}
RocksDBLogValue RocksDBLogValue::DocumentRemoveAsPartOfUpdate(
arangodb::StringRef const& key) {
return RocksDBLogValue(RocksDBLogType::DocumentRemoveAsPartOfUpdate, key);
RocksDBLogValue RocksDBLogValue::DocumentRemoveV2(TRI_voc_rid_t rid) {
return RocksDBLogValue(RocksDBLogType::DocumentRemoveV2, rid);
}
RocksDBLogValue RocksDBLogValue::SinglePut(TRI_voc_tick_t vocbaseId,
TRI_voc_cid_t cid) {
return RocksDBLogValue(RocksDBLogType::SinglePut, vocbaseId, cid);
}
RocksDBLogValue RocksDBLogValue::SingleRemove(TRI_voc_tick_t vocbaseId,
TRI_voc_cid_t cid,
arangodb::StringRef const& key) {
return RocksDBLogValue(RocksDBLogType::SingleRemove, vocbaseId, cid, key);
RocksDBLogValue RocksDBLogValue::SingleRemoveV2(TRI_voc_tick_t vocbaseId,
TRI_voc_cid_t cid,
TRI_voc_rid_t rid) {
return RocksDBLogValue(RocksDBLogType::SingleRemoveV2, vocbaseId, cid, rid);
}
RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t val)
@ -137,7 +132,7 @@ RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t val)
switch (type) {
case RocksDBLogType::DatabaseCreate:
case RocksDBLogType::DatabaseDrop:
case RocksDBLogType::DocumentOperationsPrologue: {
case RocksDBLogType::DocumentRemoveV2: {
_buffer.reserve(sizeof(RocksDBLogType) + sizeof(uint64_t));
_buffer.push_back(static_cast<char>(type));
uint64ToPersistent(_buffer, val); // database or collection ID
@ -145,6 +140,7 @@ RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t val)
}
default:
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_BAD_PARAMETER,
"invalid type for log value");
}
@ -159,7 +155,8 @@ RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t dbId,
case RocksDBLogType::CollectionDrop:
case RocksDBLogType::ViewCreate:
case RocksDBLogType::BeginTransaction:
case RocksDBLogType::SinglePut: {
case RocksDBLogType::SinglePut:
case RocksDBLogType::CommitTransaction: {
_buffer.reserve(sizeof(RocksDBLogType) + sizeof(uint64_t) * 2);
_buffer.push_back(static_cast<char>(type));
uint64ToPersistent(_buffer, dbId);
@ -168,21 +165,23 @@ RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t dbId,
}
default:
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_BAD_PARAMETER,
"invalid type for log value");
}
}
RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t dbId,
uint64_t cid, uint64_t iid)
uint64_t cid, uint64_t third)
: _buffer() {
switch (type) {
case RocksDBLogType::IndexDrop: {
case RocksDBLogType::IndexDrop:
case RocksDBLogType::SingleRemoveV2: {
_buffer.reserve(sizeof(RocksDBLogType) + sizeof(uint64_t) * 3);
_buffer.push_back(static_cast<char>(type));
uint64ToPersistent(_buffer, dbId);
uint64ToPersistent(_buffer, cid);
uint64ToPersistent(_buffer, iid);
uint64ToPersistent(_buffer, third);
break;
}
default:
@ -237,7 +236,6 @@ RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t dbId,
uint64_t cid, StringRef const& data)
: _buffer() {
switch (type) {
case RocksDBLogType::SingleRemove:
case RocksDBLogType::CollectionDrop:
case RocksDBLogType::CollectionRename: {
_buffer.reserve(sizeof(RocksDBLogType) + sizeof(uint64_t) * 2 +
@ -257,23 +255,6 @@ RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, uint64_t dbId,
}
}
RocksDBLogValue::RocksDBLogValue(RocksDBLogType type, StringRef const& data)
: _buffer() {
switch (type) {
case RocksDBLogType::DocumentRemove:
case RocksDBLogType::DocumentRemoveAsPartOfUpdate: {
_buffer.reserve(data.length() + sizeof(RocksDBLogType));
_buffer.push_back(static_cast<char>(type));
_buffer.append(data.data(), data.length()); // primary key
break;
}
default:
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_BAD_PARAMETER,
"invalid type for log value");
}
}
// ================= Instance Methods ===================
RocksDBLogType RocksDBLogValue::type(rocksdb::Slice const& slice) {
@ -293,13 +274,9 @@ TRI_voc_cid_t RocksDBLogValue::collectionId(rocksdb::Slice const& slice) {
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + sizeof(uint64_t));
RocksDBLogType type = static_cast<RocksDBLogType>(slice.data()[0]);
TRI_ASSERT(RocksDBLogValue::containsCollectionId(type));
if (type == RocksDBLogType::DocumentOperationsPrologue) { // only exception
return uint64FromPersistent(slice.data() + sizeof(RocksDBLogType));
} else {
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + sizeof(uint64_t) * 2);
return uint64FromPersistent(slice.data() + sizeof(RocksDBLogType) +
sizeof(uint64_t));
}
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + sizeof(uint64_t) * 2);
return uint64FromPersistent(slice.data() + sizeof(RocksDBLogType) +
sizeof(uint64_t));
}
TRI_voc_cid_t RocksDBLogValue::viewId(rocksdb::Slice const& slice) {
@ -323,7 +300,8 @@ TRI_voc_cid_t RocksDBLogValue::viewId(rocksdb::Slice const& slice) {
TRI_voc_tid_t RocksDBLogValue::transactionId(rocksdb::Slice const& slice) {
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + sizeof(uint64_t));
RocksDBLogType type = static_cast<RocksDBLogType>(slice.data()[0]);
TRI_ASSERT(type == RocksDBLogType::BeginTransaction);
TRI_ASSERT(type == RocksDBLogType::BeginTransaction ||
type == RocksDBLogType::CommitTransaction);
// <type> + 8-byte <dbId> + 8-byte <trxId>
return uint64FromPersistent(slice.data() + sizeof(RocksDBLogType) +
sizeof(TRI_voc_tick_t));
@ -346,6 +324,21 @@ TRI_idx_iid_t RocksDBLogValue::indexId(rocksdb::Slice const& slice) {
(2 * sizeof(uint64_t)));
}
/// For DocumentRemoveV2 and SingleRemoveV2
TRI_voc_rid_t RocksDBLogValue::revisionId(rocksdb::Slice const& slice) {
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + (sizeof(uint64_t)));
RocksDBLogType type = static_cast<RocksDBLogType>(slice.data()[0]);
if (type == RocksDBLogType::DocumentRemoveV2) {
return uint64FromPersistent(slice.data() + sizeof(RocksDBLogType));
} else if (type == RocksDBLogType::SingleRemoveV2) {
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + (3 * sizeof(uint64_t)));
return uint64FromPersistent(slice.data() + sizeof(RocksDBLogType) +
2 * sizeof(uint64_t));
}
TRI_ASSERT(false);// invalid type
return 0;
}
VPackSlice RocksDBLogValue::indexSlice(rocksdb::Slice const& slice) {
TRI_ASSERT(slice.size() >= sizeof(RocksDBLogType) + sizeof(uint64_t) * 2);
RocksDBLogType type = static_cast<RocksDBLogType>(slice.data()[0]);
@ -385,20 +378,6 @@ StringRef RocksDBLogValue::oldCollectionName(
return StringRef(slice.data() + off, slice.size() - off);
}
StringRef RocksDBLogValue::documentKey(rocksdb::Slice const& slice) {
RocksDBLogType type = static_cast<RocksDBLogType>(slice.data()[0]);
TRI_ASSERT(type == RocksDBLogType::SingleRemove ||
type == RocksDBLogType::DocumentRemove ||
type == RocksDBLogType::DocumentRemoveAsPartOfUpdate);
size_t off = sizeof(RocksDBLogType);
// only single remove contains vocbase id and cid
if (type == RocksDBLogType::SingleRemove) {
off += sizeof(uint64_t) * 2;
}
TRI_ASSERT(slice.size() >= off);
return StringRef(slice.data() + off, slice.size() - off);
}
bool RocksDBLogValue::containsDatabaseId(RocksDBLogType type) {
return type == RocksDBLogType::DatabaseCreate ||
type == RocksDBLogType::DatabaseDrop ||
@ -416,8 +395,9 @@ bool RocksDBLogValue::containsDatabaseId(RocksDBLogType type) {
type == RocksDBLogType::IndexCreate ||
type == RocksDBLogType::IndexDrop ||
type == RocksDBLogType::BeginTransaction ||
type == RocksDBLogType::CommitTransaction ||
type == RocksDBLogType::SinglePut ||
type == RocksDBLogType::SingleRemove;
type == RocksDBLogType::SingleRemoveV2;
}
bool RocksDBLogValue::containsCollectionId(RocksDBLogType type) {
@ -430,9 +410,8 @@ bool RocksDBLogValue::containsCollectionId(RocksDBLogType type) {
#endif
type == RocksDBLogType::IndexCreate ||
type == RocksDBLogType::IndexDrop ||
type == RocksDBLogType::DocumentOperationsPrologue ||
type == RocksDBLogType::SinglePut ||
type == RocksDBLogType::SingleRemove;
type == RocksDBLogType::SingleRemoveV2;
}
bool RocksDBLogValue::containsViewId(RocksDBLogType type) {

View File

@ -74,21 +74,16 @@ class RocksDBLogValue {
#endif
static RocksDBLogValue BeginTransaction(TRI_voc_tick_t vocbaseId,
TRI_voc_tid_t trxId);
static RocksDBLogValue DocumentOpsPrologue(TRI_voc_cid_t cid);
static RocksDBLogValue DocumentRemove(arangodb::StringRef const&);
static RocksDBLogValue DocumentRemoveAsPartOfUpdate(arangodb::StringRef const&);
TRI_voc_tid_t tid);
static RocksDBLogValue CommitTransaction(TRI_voc_tick_t vocbaseId,
TRI_voc_tid_t tid);
static RocksDBLogValue DocumentRemoveV2(TRI_voc_rid_t rid);
static RocksDBLogValue SinglePut(TRI_voc_tick_t vocbaseId, TRI_voc_cid_t cid);
static RocksDBLogValue SingleRemove(TRI_voc_tick_t vocbaseId,
TRI_voc_cid_t cid, StringRef const&);
static RocksDBLogValue SingleRemoveV2(TRI_voc_tick_t vocbaseId, TRI_voc_cid_t cid,
TRI_voc_rid_t rid);
public:
//////////////////////////////////////////////////////////////////////////////
/// @brief Extracts the revisionId from a value
///
/// May be called only on PrimaryIndexValue values. Other types will throw.
//////////////////////////////////////////////////////////////////////////////
static RocksDBLogType type(rocksdb::Slice const&);
static TRI_voc_tick_t databaseId(rocksdb::Slice const&);
@ -96,11 +91,12 @@ class RocksDBLogValue {
static TRI_voc_cid_t collectionId(rocksdb::Slice const&);
static TRI_voc_cid_t viewId(rocksdb::Slice const&);
static TRI_idx_iid_t indexId(rocksdb::Slice const&);
/// For DocumentRemoveV2 and SingleRemoveV2
static TRI_voc_rid_t revisionId(rocksdb::Slice const&);
static velocypack::Slice indexSlice(rocksdb::Slice const&);
static velocypack::Slice viewSlice(rocksdb::Slice const&);
static arangodb::StringRef collectionUUID(rocksdb::Slice const&);
static arangodb::StringRef oldCollectionName(rocksdb::Slice const&);
static arangodb::StringRef documentKey(rocksdb::Slice const&);
static bool containsDatabaseId(RocksDBLogType type);
static bool containsCollectionId(RocksDBLogType type);
@ -129,7 +125,6 @@ class RocksDBLogValue {
#endif
RocksDBLogValue(RocksDBLogType, uint64_t, uint64_t, VPackSlice const&);
RocksDBLogValue(RocksDBLogType, uint64_t, uint64_t, StringRef const& data);
RocksDBLogValue(RocksDBLogType, StringRef const& data);
private:
std::string _buffer;

View File

@ -37,9 +37,8 @@ using namespace arangodb;
// ================= RocksDBSavePoint ==================
RocksDBSavePoint::RocksDBSavePoint(
RocksDBMethods* trx, bool handled,
std::function<void()> const& rollbackCallback)
: _trx(trx), _rollbackCallback(rollbackCallback), _handled(handled) {
RocksDBMethods* trx, bool handled)
: _trx(trx), _handled(handled) {
TRI_ASSERT(trx != nullptr);
if (!_handled) {
_trx->SetSavePoint();
@ -61,7 +60,6 @@ void RocksDBSavePoint::rollback() {
TRI_ASSERT(!_handled);
_trx->RollbackToSavePoint();
_handled = true; // in order to not roll back again by accident
_rollbackCallback();
}
// =================== RocksDBMethods ===================

View File

@ -45,8 +45,7 @@ class RocksDBTransactionState;
class RocksDBSavePoint {
public:
RocksDBSavePoint(RocksDBMethods* trx, bool handled,
std::function<void()> const& rollbackCallback);
RocksDBSavePoint(RocksDBMethods* trx, bool handled);
~RocksDBSavePoint();
void commit();
@ -56,7 +55,6 @@ class RocksDBSavePoint {
private:
RocksDBMethods* _trx;
std::function<void()> const _rollbackCallback;
bool _handled;
};

View File

@ -174,7 +174,7 @@ LocalDocumentId RocksDBPrimaryIndex::lookupKey(transaction::Methods* trx,
if (f.found()) {
rocksdb::Slice s(reinterpret_cast<char const*>(f.value()->value()),
f.value()->valueSize());
return LocalDocumentId(RocksDBValue::revisionId(s));
return RocksDBValue::documentId(s);
} else if (f.result().errorNumber() == TRI_ERROR_LOCK_TIMEOUT) {
// assuming someone is currently holding a write lock, which
// is why we cannot access the TransactionalBucket.
@ -213,7 +213,7 @@ LocalDocumentId RocksDBPrimaryIndex::lookupKey(transaction::Methods* trx,
}
}
return LocalDocumentId(RocksDBValue::revisionId(value));
return RocksDBValue::documentId(value);
}
Result RocksDBPrimaryIndex::insertInternal(transaction::Methods* trx,
@ -224,7 +224,7 @@ Result RocksDBPrimaryIndex::insertInternal(transaction::Methods* trx,
VPackSlice keySlice = transaction::helpers::extractKeyFromDocument(slice);
RocksDBKeyLeaser key(trx);
key->constructPrimaryIndexValue(_objectId, StringRef(keySlice));
auto value = RocksDBValue::PrimaryIndexValue(documentId.id());
auto value = RocksDBValue::PrimaryIndexValue(documentId);
if (mthd->Exists(_cf, key.ref())) {
std::string existingId(slice.get(StaticStrings::KeyString).copyString());
@ -254,7 +254,7 @@ Result RocksDBPrimaryIndex::updateInternal(transaction::Methods* trx,
TRI_ASSERT(keySlice == oldDoc.get(StaticStrings::KeyString));
RocksDBKeyLeaser key(trx);
key->constructPrimaryIndexValue(_objectId, StringRef(keySlice));
auto value = RocksDBValue::PrimaryIndexValue(newDocumentId.id());
auto value = RocksDBValue::PrimaryIndexValue(newDocumentId);
TRI_ASSERT(mthd->Exists(_cf, key.ref()));
blackListKey(key->string().data(),

View File

@ -211,15 +211,31 @@ class WBReader final : public rocksdb::WriteBatch::Handler {
}
}
RocksDBCuckooIndexEstimator<uint64_t>* findEstimator(
uint64_t objectId) {
RocksDBCuckooIndexEstimator<uint64_t>* findEstimator(uint64_t objectId) {
RocksDBEngine* engine =
static_cast<RocksDBEngine*>(EngineSelectorFeature::ENGINE);
Index* index = engine->mapObjectToIndex(objectId);
RocksDBEngine::IndexTriple triple = engine->mapObjectToIndex(objectId);
if (std::get<0>(triple) == 0 && std::get<1>(triple) == 0) {
return nullptr;
}
DatabaseFeature* df = DatabaseFeature::DATABASE;
TRI_vocbase_t* vb = df->useDatabase(std::get<0>(triple));
if (vb == nullptr) {
return nullptr;
}
TRI_DEFER(vb->release());
LogicalCollection* coll = vb->lookupCollection(std::get<1>(triple));
if (coll == nullptr) {
return nullptr;
}
std::shared_ptr<Index> index = coll->lookupIndex(std::get<2>(triple));
if (index == nullptr) {
return nullptr;
}
return static_cast<RocksDBIndex*>(index)->estimator();
return static_cast<RocksDBIndex*>(index.get())->estimator();
}
void updateMaxTick(uint32_t column_family_id, const rocksdb::Slice& key,
@ -235,7 +251,7 @@ class WBReader final : public rocksdb::WriteBatch::Handler {
// - databases
if (column_family_id == RocksDBColumnFamily::documents()->GetID()) {
storeMaxHLC(RocksDBKey::revisionId(RocksDBEntryType::Document, key));
storeMaxHLC(RocksDBKey::documentId(RocksDBEntryType::Document, key).id());
storeLastKeyValue(RocksDBKey::objectId(key),
RocksDBValue::keyValue(value));
} else if (column_family_id == RocksDBColumnFamily::primary()->GetID()) {
@ -285,14 +301,13 @@ class WBReader final : public rocksdb::WriteBatch::Handler {
updateMaxTick(column_family_id, key, value);
if (shouldHandleDocument(column_family_id, key)) {
uint64_t objectId = RocksDBKey::objectId(key);
uint64_t revisionId =
RocksDBKey::revisionId(RocksDBEntryType::Document, key);
LocalDocumentId docId = RocksDBKey::documentId(RocksDBEntryType::Document, key);
auto const& it = deltas.find(objectId);
if (it != deltas.end()) {
it->second._sequenceNum = currentSeqNum;
it->second._added++;
it->second._revisionId = revisionId;
it->second._revisionId = docId.id();
}
} else {
// We have to adjust the estimate with an insert
@ -326,14 +341,13 @@ class WBReader final : public rocksdb::WriteBatch::Handler {
const rocksdb::Slice& key) override {
if (shouldHandleDocument(column_family_id, key)) {
uint64_t objectId = RocksDBKey::objectId(key);
uint64_t revisionId =
RocksDBKey::revisionId(RocksDBEntryType::Document, key);
LocalDocumentId docId = RocksDBKey::documentId(RocksDBEntryType::Document, key);
auto const& it = deltas.find(objectId);
if (it != deltas.end()) {
it->second._sequenceNum = currentSeqNum;
it->second._removed++;
it->second._revisionId = revisionId;
it->second._revisionId = docId.id();
}
} else {
// We have to adjust the estimate with an insert

View File

@ -29,6 +29,8 @@
#include "RocksDBEngine/RocksDBColumnFamily.h"
#include "RocksDBEngine/RocksDBCommon.h"
#include "RocksDBEngine/RocksDBLogValue.h"
#include "Utils/CollectionGuard.h"
#include "VocBase/LogicalCollection.h"
#include "VocBase/ticks.h"
#include <velocypack/Builder.h>
@ -40,8 +42,8 @@
#include <rocksdb/write_batch.h>
using namespace arangodb;
using namespace arangodb::basics;
using namespace arangodb::rocksutils;
using namespace arangodb::velocypack;
// define to INFO to see the WAL output
#define _LOG TRACE
@ -77,6 +79,8 @@ TRI_replication_operation_e rocksutils::convertLogType(RocksDBLogType t) {
return REPLICATION_VIEW_CHANGE;
case RocksDBLogType::BeginTransaction:
return REPLICATION_TRANSACTION_START;
case RocksDBLogType::CommitTransaction:
return REPLICATION_TRANSACTION_COMMIT;
default:
TRI_ASSERT(false);
@ -86,11 +90,31 @@ TRI_replication_operation_e rocksutils::convertLogType(RocksDBLogType t) {
/// WAL parser
class WALParser : public rocksdb::WriteBatch::Handler {
// internal WAL parser states
enum State : char {
INVALID = 0,
COLLECTION_CREATE,
COLLECTION_DROP,
COLLECTION_RENAME,
COLLECTION_CHANGE,
INDEX_CREATE,
INDEX_DROP,
VIEW_CREATE,
VIEW_DROP,
VIEW_CHANGE,
VIEW_RENAME,
TRANSACTION,
SINGLE_PUT,
SINGLE_REMOVE
};
public:
WALParser(TRI_vocbase_t* vocbase, bool includeSystem,
TRI_voc_cid_t collectionId, VPackBuilder& builder)
: _documentsCF(RocksDBColumnFamily::documents()->GetID()),
_definitionsCF(RocksDBColumnFamily::definitions()->GetID()),
: _definitionsCF(RocksDBColumnFamily::definitions()->GetID()),
_documentsCF(RocksDBColumnFamily::documents()->GetID()),
_primaryCF(RocksDBColumnFamily::primary()->GetID()),
_vocbase(vocbase),
_includeSystem(includeSystem),
_onlyCollectionId(collectionId),
@ -101,94 +125,85 @@ class WALParser : public rocksdb::WriteBatch::Handler {
void LogData(rocksdb::Slice const& blob) override {
RocksDBLogType type = RocksDBLogValue::type(blob);
TRI_DEFER(_lastLogType = type);
// skip ignored databases and collections
if (RocksDBLogValue::containsDatabaseId(type)) {
TRI_voc_tick_t dbId = RocksDBLogValue::databaseId(blob);
_currentDbId = dbId;
if (!shouldHandleDB(dbId)) {
resetTransientState();
return;
}
if (RocksDBLogValue::containsCollectionId(type)) {
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
_currentCid = cid;
if (!shouldHandleCollection(dbId, cid)) {
if (type == RocksDBLogType::SingleRemove || type == RocksDBLogType::SinglePut) {
resetTransientState();
} else {
_currentCid = 0;
}
return;
}
}
}
LOG_TOPIC(_LOG, Logger::ROCKSDB) << "[LOG] " << rocksDBLogTypeName(type);
switch (type) {
case RocksDBLogType::DatabaseCreate:
case RocksDBLogType::DatabaseCreate: // not handled here
case RocksDBLogType::DatabaseDrop: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
break;
}
case RocksDBLogType::CollectionRename:
case RocksDBLogType::CollectionCreate:
resetTransientState(); // finish ongoing trx
if (shouldHandleCollection(RocksDBLogValue::databaseId(blob),
RocksDBLogValue::collectionId(blob))) {
_state = COLLECTION_CREATE;
}
break;
case RocksDBLogType::CollectionRename:
resetTransientState(); // finish ongoing trx
if (shouldHandleCollection(RocksDBLogValue::databaseId(blob),
RocksDBLogValue::collectionId(blob))) {
_state = COLLECTION_RENAME;
_oldCollectionName = RocksDBLogValue::oldCollectionName(blob).toString();
}
break;
case RocksDBLogType::CollectionChange:
resetTransientState(); // finish ongoing trx
if (shouldHandleCollection(RocksDBLogValue::databaseId(blob),
RocksDBLogValue::collectionId(blob))) {
_state = COLLECTION_CHANGE;
}
break;
case RocksDBLogType::CollectionDrop: {
resetTransientState(); // finish ongoing trx
if (_lastLogType == RocksDBLogType::IndexCreate) {
TRI_ASSERT(_currentDbId == RocksDBLogValue::databaseId(blob));
TRI_ASSERT(_currentCid == RocksDBLogValue::collectionId(blob));
}
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
if (type == RocksDBLogType::CollectionRename) {
_oldCollectionName = RocksDBLogValue::oldCollectionName(blob).toString();
} else if (type == RocksDBLogType::CollectionDrop) {
std::string UUID = RocksDBLogValue::collectionUUID(blob).toString();
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
LOG_TOPIC(_LOG, Logger::ROCKSDB) << "CID: " << _currentCid;
// reset name in collection name cache
_collectionNames.erase(_currentCid);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type", VPackValue(REPLICATION_COLLECTION_DROP));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
if (!UUID.empty()) {
_builder.add("cuid", VPackValue(UUID));
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
// always print drop collection marker, shouldHandleCollection will
// always return false for dropped collections
if (shouldHandleDB(dbid)) {
{ // tick number
StringRef uuid = RocksDBLogValue::collectionUUID(blob);
TRI_ASSERT(!uuid.empty());
uint64_t tick = _currentSequence + (_startOfBatch ? 0 : 1);
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(tick)));
marker->add("type", VPackValue(REPLICATION_COLLECTION_DROP));
marker->add("database", VPackValue(std::to_string(dbid)));
if (!uuid.empty()) {
marker->add("cuid", VPackValuePair(uuid.data(), uuid.size(),
VPackValueType::String));
}
marker->add("cid", VPackValue(std::to_string(cid)));
VPackObjectBuilder data(&_builder, "data", true);
data->add("id", VPackValue(std::to_string(cid)));
data->add("name", VPackValue("")); // not used at all
}
_builder.add("cid", VPackValue(std::to_string(_currentCid)));
_builder.add("data", VPackValue(VPackValueType::Object));
_builder.add("id", VPackValue(std::to_string(_currentCid)));
_builder.add("name", VPackValue("")); // not used at all
_builder.close();
_builder.close();
updateLastEmittedTick(_currentSequence);
}
break;
}
case RocksDBLogType::IndexCreate: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
// only print markers from this collection if it is set
if (_onlyCollectionId == 0 || _currentCid == _onlyCollectionId) {
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);
VPackSlice indexSlice = RocksDBLogValue::indexSlice(blob);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(tick)));
_builder.add("type", VPackValue(convertLogType(type)));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("cid", VPackValue(std::to_string(_currentCid)));
std::string const& cname = nameFromCid(_currentCid);
if (!cname.empty()) {
_builder.add("cname", VPackValue(cname));
}
_builder.add("data", indexSlice);
_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("cname", VPackValue(coll->name()));
_builder.add("data", stripped.first);
_builder.close();
updateLastEmittedTick(tick);
}
@ -196,17 +211,21 @@ class WALParser : public rocksdb::WriteBatch::Handler {
}
case RocksDBLogType::IndexDrop: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
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 (_onlyCollectionId == 0 || _currentCid == _onlyCollectionId) {
if (shouldHandleCollection(dbid, cid)) {
TRI_ASSERT(_vocbase->id() == dbid);
LogicalCollection* coll = loadCollection(cid);
TRI_ASSERT(coll != nullptr);
uint64_t tick = _currentSequence + (_startOfBatch ? 0 : 1);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(tick)));
_builder.add("type", VPackValue(convertLogType(type)));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("cid", VPackValue(std::to_string(_currentCid)));
_builder.add("type", VPackValue(REPLICATION_INDEX_DROP));
_builder.add("database", VPackValue(std::to_string(dbid)));
_builder.add("cid", VPackValue(std::to_string(cid)));
_builder.add("cname", VPackValue(coll->name()));
_builder.add("data", VPackValue(VPackValueType::Object));
_builder.add("id", VPackValue(std::to_string(iid)));
_builder.close();
@ -224,60 +243,80 @@ class WALParser : public rocksdb::WriteBatch::Handler {
break;
}
case RocksDBLogType::BeginTransaction: {
TRI_ASSERT(!_singleOp);
resetTransientState(); // finish ongoing trx
_seenBeginTransaction = true;
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentTrxId = RocksDBLogValue::transactionId(blob);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type", VPackValue(convertLogType(type)));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("tid", VPackValue(std::to_string(_currentTrxId)));
_builder.close();
updateLastEmittedTick(_currentSequence);
break;
}
case RocksDBLogType::DocumentOperationsPrologue: {
// part of an ongoing transaction
if (_currentDbId != 0 && _currentTrxId != 0) {
// database (and therefore transaction) may be ignored
TRI_ASSERT(_seenBeginTransaction && !_singleOp);
// document ops can ignore this collection later
_currentCid = RocksDBLogValue::collectionId(blob);
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_tid_t tid = RocksDBLogValue::transactionId(blob);
if (shouldHandleDB(dbid)) {
_state = TRANSACTION;
_currentTrxId = tid;
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type", VPackValue(convertLogType(type)));
_builder.add("database", VPackValue(std::to_string(dbid)));
_builder.add("tid", VPackValue(std::to_string(tid)));
_builder.close();
updateLastEmittedTick(_currentSequence);
}
break;
}
case RocksDBLogType::DocumentRemove:
case RocksDBLogType::DocumentRemoveAsPartOfUpdate: {
// part of an ongoing transaction
if (_currentDbId != 0 && _currentTrxId != 0 && _currentCid != 0) {
// collection may be ignored
TRI_ASSERT(_seenBeginTransaction && !_singleOp);
if (shouldHandleCollection(_currentDbId, _currentCid)) {
_removeDocumentKey = RocksDBLogValue::documentKey(blob).toString();
case RocksDBLogType::CommitTransaction: { // ideally optional
if (_state == TRANSACTION) {
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_tid_t tid = RocksDBLogValue::transactionId(blob);
TRI_ASSERT(_currentTrxId == tid && _vocbase->id() == dbid);
if (shouldHandleDB(dbid) && _currentTrxId == tid) {
writeCommitMarker();
}
}
break;
}
case RocksDBLogType::SingleRemove: {
// we can only get here if we can handle this collection
resetTransientState(); // finish ongoing trx
_removeDocumentKey = RocksDBLogValue::documentKey(blob).toString();
_singleOp = true;
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
resetTransientState();
break;
}
case RocksDBLogType::SinglePut: {
resetTransientState(); // finish ongoing trx
_singleOp = true;
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
_state = SINGLE_PUT;
}
break;
}
case RocksDBLogType::SingleRemove: { // deprecated
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
_state = SINGLE_REMOVE; // revisionId is unknown
}
break;
}
case RocksDBLogType::DocumentRemoveV2: { // remove within a trx
if (_state == TRANSACTION) {
TRI_ASSERT(_removedDocRid == 0);
_removedDocRid = RocksDBLogValue::revisionId(blob);
} else {
resetTransientState();
}
break;
}
case RocksDBLogType::SingleRemoveV2: {
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
_state = SINGLE_REMOVE;
_removedDocRid = RocksDBLogValue::revisionId(blob);
}
break;
}
case RocksDBLogType::DocumentOperationsPrologue:
case RocksDBLogType::DocumentRemove:
case RocksDBLogType::DocumentRemoveAsPartOfUpdate:
break; // ignore deprecated && unused markers
default:
LOG_TOPIC(WARN, Logger::REPLICATION) << "Unhandled wal log entry "
<< rocksDBLogTypeName(type);
break;
}
}
@ -285,84 +324,85 @@ class WALParser : public rocksdb::WriteBatch::Handler {
rocksdb::Status PutCF(uint32_t column_family_id, rocksdb::Slice const& key,
rocksdb::Slice const& value) override {
tick();
if (!shouldHandleKey(column_family_id, true, key)) {
return rocksdb::Status();
}
LOG_TOPIC(_LOG, Logger::ROCKSDB) << "PUT: key:" << key.ToString()
<< " value: " << value.ToString();
if (column_family_id == _definitionsCF) {
if (RocksDBKey::type(key) == RocksDBEntryType::Database) {
TRI_ASSERT(_lastLogType == RocksDBLogType::DatabaseCreate ||
_lastLogType == RocksDBLogType::DatabaseDrop);
// this version of the protocol will always ignore database markers
} else if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
if (_lastLogType == RocksDBLogType::IndexCreate ||
_lastLogType == RocksDBLogType::IndexDrop) {
_lastLogType = RocksDBLogType::Invalid;
return rocksdb::Status();
}
TRI_ASSERT(_lastLogType == RocksDBLogType::CollectionCreate ||
_lastLogType == RocksDBLogType::CollectionChange ||
_lastLogType == RocksDBLogType::CollectionRename);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
LOG_TOPIC(_LOG, Logger::ROCKSDB) << "CID: " << _currentCid;
if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
VPackSlice data = RocksDBValue::data(value);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type", VPackValue(convertLogType(_lastLogType)));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("cid", VPackValue(std::to_string(_currentCid)));
updateLastEmittedTick(_currentSequence);
VPackSlice cname = data.get("name");
if (cname.isString()) {
_builder.add("cname", cname);
// set name in cache
_collectionNames[_currentCid] = cname.copyString();
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_ASSERT(_vocbase->id() == dbid);
LogicalCollection* coll = loadCollection(cid);
TRI_ASSERT(coll != nullptr);
VPackSlice collectionDef = RocksDBValue::data(value);
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("database", VPackValue(std::to_string(dbid)));
marker->add("cid", VPackValue(std::to_string(cid)));
marker->add("cname", VPackValue(coll->name()));
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(coll->name()));
data->add("oldName", VPackValue(_oldCollectionName));
data->add("id", VPackValue(std::to_string(cid)));
} else if (_state == COLLECTION_CHANGE) {
auto stripped = rocksutils::stripObjectIds(collectionDef);
marker->add("type", VPackValue(REPLICATION_COLLECTION_CHANGE));
marker->add("data", stripped.first);
}
updateLastEmittedTick(_currentSequence);
}
if (_lastLogType == RocksDBLogType::CollectionRename) {
_builder.add("data", VPackValue(VPackValueType::Object));
_builder.add("id", VPackValue(std::to_string(_currentCid)));
_builder.add("oldName", VPackValue(_oldCollectionName));
_builder.add("name", cname);
_builder.close();
} else { // change and create need full data
_builder.add("data", data);
}
_builder.close();
// log type is only ever relevant, immediately after it appeared
// we want double occurences create / drop / change collection to fail
_lastLogType = RocksDBLogType::Invalid;
_currentDbId = 0;
_currentCid = 0;
} // if (RocksDBKey::type(key) == RocksDBEntryType::Collection)
// reset everything immediately after DDL operations
resetTransientState();
} else if (column_family_id == _documentsCF) {
TRI_ASSERT((_seenBeginTransaction && !_singleOp) ||
(!_seenBeginTransaction && _singleOp));
// if real transaction, we need the trx id
TRI_ASSERT(!_seenBeginTransaction || _currentTrxId != 0);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type", VPackValue(REPLICATION_MARKER_DOCUMENT));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("cid", VPackValue(std::to_string(_currentCid)));
updateLastEmittedTick(_currentSequence);
// collection name
std::string const& cname = nameFromCid(_currentCid);
if (!cname.empty()) {
_builder.add("cname", VPackValue(cname));
}
_builder.add("tid", VPackValue(std::to_string(_currentTrxId)));
_builder.add("data", RocksDBValue::data(value));
_builder.close();
if (_singleOp) { // reset state immediately
if (_state != TRANSACTION && _state != SINGLE_PUT) {
resetTransientState();
return rocksdb::Status();
}
TRI_ASSERT(_state != SINGLE_PUT || _currentTrxId == 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(_vocbase->id() == dbid);
LogicalCollection* col = loadCollection(cid);
TRI_ASSERT(col != nullptr);
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type", VPackValue(REPLICATION_MARKER_DOCUMENT));
marker->add("database", VPackValue(std::to_string(dbid)));
marker->add("tid", VPackValue(std::to_string(_currentTrxId)));
marker->add("cid", VPackValue(cid));
marker->add("cname", VPackValue(col->name()));
marker->add("data", RocksDBValue::data(value));
}
updateLastEmittedTick(_currentSequence);
if (_state == SINGLE_PUT) {
resetTransientState(); // always reset after single op
}
}
@ -371,74 +411,48 @@ class WALParser : public rocksdb::WriteBatch::Handler {
rocksdb::Status DeleteCF(uint32_t column_family_id,
rocksdb::Slice const& key) override {
return handleDeletion(column_family_id, key);
}
rocksdb::Status SingleDeleteCF(uint32_t column_family_id,
rocksdb::Slice const& key) override {
return handleDeletion(column_family_id, key);
}
rocksdb::Status handleDeletion(uint32_t column_family_id,
rocksdb::Slice const& key) {
tick();
if (!shouldHandleKey(column_family_id, false, key) ||
column_family_id != _documentsCF) {
if (column_family_id == _documentsCF) {
if (_lastLogType == RocksDBLogType::SingleRemove) {
TRI_ASSERT(!_seenBeginTransaction);
resetTransientState(); // ignoring the entire op
} else {
TRI_ASSERT(!_singleOp);
_removeDocumentKey.clear(); // just ignoring this key
}
}
return rocksdb::Status();
}
if (_lastLogType == RocksDBLogType::DocumentRemoveAsPartOfUpdate) {
_removeDocumentKey.clear();
return rocksdb::Status();
}
// document removes, because of a collection drop is not transactional and
// should not appear in the WAL.
if (!(_seenBeginTransaction || _singleOp)) {
return rocksdb::Status();
} else if (_lastLogType != RocksDBLogType::DocumentRemove &&
_lastLogType != RocksDBLogType::SingleRemove) {
// collection drops etc may be batched directly after a transaction
// single operation updates could come in a weird sequence pre 3.3:
// [..., LogType::SinglePut, DELETE old, PUT new, ...]
if (_lastLogType != RocksDBLogType::SinglePut) {
resetTransientState(); // finish ongoing trx
}
if (column_family_id != _primaryCF) {
return rocksdb::Status(); // ignore all document operations
} else if (_state != TRANSACTION && _state != SINGLE_REMOVE) {
resetTransientState();
return rocksdb::Status();
}
TRI_ASSERT(!_seenBeginTransaction || _currentTrxId != 0);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
TRI_ASSERT(!_removeDocumentKey.empty());
_builder.openObject();
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type", VPackValue(static_cast<uint64_t>(REPLICATION_MARKER_REMOVE)));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("cid", VPackValue(std::to_string(_currentCid)));
TRI_ASSERT(_state != SINGLE_REMOVE || _currentTrxId == 0);
uint64_t objectId = RocksDBKey::objectId(key);
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
}
TRI_ASSERT(_vocbase->id() == dbid);
StringRef docKey = RocksDBKey::primaryKey(key);
LogicalCollection* coll = loadCollection(cid);
TRI_ASSERT(coll != nullptr);
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type", VPackValue(REPLICATION_MARKER_REMOVE));
marker->add("database", VPackValue(std::to_string(dbid)));
marker->add("cid", VPackValue(std::to_string(cid)));
marker->add("cname", VPackValue(coll->name()));
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)));
}
updateLastEmittedTick(_currentSequence);
std::string const& cname = nameFromCid(_currentCid);
if (!cname.empty()) {
_builder.add("cname", VPackValue(cname));
}
_builder.add("tid", VPackValue(std::to_string(_currentTrxId)));
_builder.add("data", VPackValue(VPackValueType::Object));
// only pass on _key, but no _rev
_builder.add(StaticStrings::KeyString, VPackValue(_removeDocumentKey));
_builder.close();
_builder.close();
_removeDocumentKey.clear();
if (_singleOp) { // reset state immediately
_removedDocRid = 0; // always reset
if (_state == SINGLE_REMOVE) {
resetTransientState();
}
return rocksdb::Status();
}
@ -446,12 +460,16 @@ class WALParser : public rocksdb::WriteBatch::Handler {
// starting new write batch
_startSequence = startSequence;
_currentSequence = startSequence;
_lastLogType = RocksDBLogType::Invalid;
_startOfBatch = true;
// reset all states
_state = INVALID;
_currentTrxId = 0;
_removedDocRid = 0;
_oldCollectionName.clear();
}
void writeCommitMarker() {
TRI_ASSERT(_seenBeginTransaction && !_singleOp);
TRI_ASSERT(_state == TRANSACTION);
LOG_TOPIC(_LOG, Logger::ROCKSDB) << "tick: " << _currentSequence
<< " commit transaction";
@ -459,32 +477,30 @@ class WALParser : public rocksdb::WriteBatch::Handler {
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
_builder.add("type",
VPackValue(static_cast<uint64_t>(REPLICATION_TRANSACTION_COMMIT)));
_builder.add("database", VPackValue(std::to_string(_currentDbId)));
_builder.add("database", VPackValue(std::to_string(_vocbase->id())));
_builder.add("tid", VPackValue(std::to_string(_currentTrxId)));
_builder.close();
updateLastEmittedTick(_currentSequence);
_state = INVALID; // for safety
}
// should reset state flags which are only valid between
// observing a specific log entry and a sequence of immediately
// following PUT / DELETE / Log entries
void resetTransientState() {
if (_seenBeginTransaction) {
if (_state == TRANSACTION) {
writeCommitMarker();
}
// reset all states
_lastLogType = RocksDBLogType::Invalid;
_seenBeginTransaction = false;
_singleOp = false;
_state = INVALID;
_currentTrxId = 0;
_currentDbId = 0;
_currentCid = 0;
_removeDocumentKey.clear();
_removedDocRid = 0;
_oldCollectionName.clear();
}
uint64_t endBatch() {
TRI_ASSERT(_removeDocumentKey.empty());
TRI_ASSERT(_removedDocRid == 0);
TRI_ASSERT(_oldCollectionName.empty());
resetTransientState();
return _currentSequence;
}
@ -517,118 +533,64 @@ class WALParser : public rocksdb::WriteBatch::Handler {
return _vocbase->id() == dbid;
}
/// @brief Check if collection is in filter
bool shouldHandleCollection(TRI_voc_tick_t dbid,
TRI_voc_cid_t cid) const {
return shouldHandleDB(dbid) &&
(_onlyCollectionId == 0 || _onlyCollectionId == cid);
}
bool shouldHandleKey(uint32_t column_family_id,
bool isPut, rocksdb::Slice const& key) const {
TRI_voc_tick_t dbId = 0;
TRI_voc_cid_t cid = 0;
if (column_family_id == _definitionsCF) {
if (RocksDBKey::type(key) == RocksDBEntryType::Database) {
return false;// ignore in this protocol version
} else if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
dbId = RocksDBKey::databaseId(key);
cid = RocksDBKey::collectionId(key);
if (!isPut || dbId == 0 || cid == 0) {
// FIXME: seems broken to get a key with zero entries here
return false;
}
} else {
/// @brief Check if collection is in filter, will load collection
bool shouldHandleCollection(TRI_voc_tick_t dbid, TRI_voc_cid_t cid) {
if (dbid == 0 || cid == 0 || !shouldHandleDB(dbid)) {
return false;
}
if (_onlyCollectionId == 0 || _onlyCollectionId == cid) {
LogicalCollection* collection = loadCollection(cid);
if (collection == nullptr) {
return false;
}
} else if (column_family_id == _documentsCF) {
dbId = _currentDbId;
cid = _currentCid;
// happens when dropping a collection or log markers
// are ignored for dbs and collections
if (!(_seenBeginTransaction || _singleOp)) {
TRI_ASSERT(dbId == 0 && cid == 0);
return false;
}
} else {
return false;
return !TRI_ExcludeCollectionReplication(collection->name(), _includeSystem);
}
// only return results for one collection
if (!shouldHandleCollection(dbId, cid)) {
return false;
}
// allow document removes of dropped collections
std::string const collectionName = _vocbase->collectionName(cid);
if (collectionName.empty()) {
return true;
}
if (!_includeSystem && collectionName[0] == '_') {
return false;
}
if (TRI_ExcludeCollectionReplication(collectionName, _includeSystem)) {
return false;
}
return true;
return false;
}
/// @brief translate a (local) collection id into a collection name
std::string const& nameFromCid(TRI_voc_cid_t cid) {
auto it = _collectionNames.find(cid);
if (it != _collectionNames.end()) {
// collection name is in cache already
return (*it).second;
}
// collection name not in cache yet
std::string name(_vocbase->collectionName(cid));
if (!name.empty()) {
// insert into cache
try {
_collectionNames.emplace(cid, std::move(name));
} catch (...) {
return emptyString;
LogicalCollection* loadCollection(TRI_voc_cid_t cid) {
TRI_ASSERT(cid != 0);
if (_vocbase != nullptr) {
auto const& it = _collectionCache.find(cid);
if (it != _collectionCache.end()) {
return it->second.collection();
}
LogicalCollection* collection = _vocbase->lookupCollection(cid);
if (collection != nullptr) {
_collectionCache.emplace(cid, CollectionGuard(_vocbase, collection));
return collection;
}
// and look it up again
return nameFromCid(cid);
}
return emptyString;
return nullptr;
}
private:
uint32_t const _documentsCF;
uint32_t const _definitionsCF;
uint32_t const _documentsCF;
uint32_t const _primaryCF;
// these parameters are relevant to determine if we can print
// a specific marker from the WAL
TRI_vocbase_t* const _vocbase;
// @brief collection replication UUID cache
std::map<TRI_voc_cid_t, CollectionGuard> _collectionCache;
bool const _includeSystem;
TRI_voc_cid_t const _onlyCollectionId;
/// result builder
VPackBuilder& _builder;
// collection name cache
std::unordered_map<TRI_voc_cid_t, std::string> _collectionNames;
// Various state machine flags
rocksdb::SequenceNumber _startSequence;
rocksdb::SequenceNumber _currentSequence;
rocksdb::SequenceNumber _lastEmittedTick; // just used for validation
RocksDBLogType _lastLogType = RocksDBLogType::Invalid;
bool _seenBeginTransaction = false;
bool _singleOp = false;
bool _startOfBatch = false;
// Various state machine flags
State _state = INVALID;
TRI_voc_tick_t _currentTrxId = 0;
TRI_voc_tick_t _currentDbId = 0;
TRI_voc_cid_t _currentCid = 0;
TRI_voc_rid_t _removedDocRid = 0;
std::string _oldCollectionName;
std::string _removeDocumentKey;
};
// iterates over WAL starting at 'from' and returns up to 'limit' documents

View File

@ -27,6 +27,7 @@
#include "Logger/Logger.h"
#include "RocksDBEngine/RocksDBCollection.h"
#include "RocksDBEngine/RocksDBIndex.h"
#include "RocksDBEngine/RocksDBSettingsManager.h"
#include "StorageEngine/TransactionState.h"
#include "Transaction/Hints.h"
#include "Transaction/Methods.h"
@ -42,7 +43,6 @@ RocksDBTransactionCollection::RocksDBTransactionCollection(
_nestingLevel(nestingLevel),
_initialNumberDocuments(0),
_revision(0),
_operationSize(0),
_numInserts(0),
_numUpdates(0),
_numRemoves(0),
@ -263,7 +263,7 @@ void RocksDBTransactionCollection::release() {
/// @brief add an operation for a transaction collection
void RocksDBTransactionCollection::addOperation(
TRI_voc_document_operation_e operationType, uint64_t operationSize,
TRI_voc_document_operation_e operationType,
TRI_voc_rid_t revisionId) {
switch (operationType) {
case TRI_VOC_DOCUMENT_OPERATION_UNKNOWN:
@ -282,7 +282,6 @@ void RocksDBTransactionCollection::addOperation(
_revision = revisionId;
break;
}
_operationSize += operationSize;
}
void RocksDBTransactionCollection::prepareCommit(uint64_t trxId,
@ -320,8 +319,24 @@ void RocksDBTransactionCollection::abortCommit(uint64_t trxId) {
void RocksDBTransactionCollection::commitCounts(uint64_t trxId,
uint64_t commitSeq) {
TRI_ASSERT(_collection != nullptr);
// Update the collection count
int64_t const adjustment = _numInserts - _numRemoves;
if (commitSeq != 0) {
if (_numInserts != 0 || _numRemoves != 0 || _revision != 0) {
RocksDBCollection* coll = static_cast<RocksDBCollection*>(_collection->getPhysical());
coll->adjustNumberDocuments(adjustment);
coll->setRevision(_revision);
RocksDBEngine* engine = rocksutils::globalRocksEngine();
RocksDBSettingsManager::CounterAdjustment update(commitSeq, _numInserts, _numRemoves,
_revision);
engine->settingsManager()->updateCounter(coll->objectId(), update);
}
}
// Update the index estimates.
TRI_ASSERT(_collection != nullptr);
for (auto& pair : _trackedIndexOperations) {
auto idx = _collection->lookupIndex(pair.first);
if (idx == nullptr) {
@ -337,8 +352,7 @@ void RocksDBTransactionCollection::commitCounts(uint64_t trxId,
}
}
_initialNumberDocuments += _numInserts - _numRemoves;
_operationSize = 0;
_initialNumberDocuments += adjustment;
_numInserts = 0;
_numUpdates = 0;
_numRemoves = 0;

View File

@ -87,7 +87,7 @@ class RocksDBTransactionCollection final : public TransactionCollection {
/// @brief add an operation for a transaction collection
void addOperation(TRI_voc_document_operation_e operationType,
uint64_t operationSize, TRI_voc_rid_t revisionId);
TRI_voc_rid_t revisionId);
/**
* @brief Prepare collection for commit by placing index blockers
@ -132,7 +132,6 @@ class RocksDBTransactionCollection final : public TransactionCollection {
int _nestingLevel; // the transaction level that added this collection
uint64_t _initialNumberDocuments;
TRI_voc_rid_t _revision;
uint64_t _operationSize;
uint64_t _numInserts;
uint64_t _numUpdates;
uint64_t _numRemoves;

View File

@ -34,7 +34,6 @@
#include "RocksDBEngine/RocksDBEngine.h"
#include "RocksDBEngine/RocksDBLogValue.h"
#include "RocksDBEngine/RocksDBMethods.h"
#include "RocksDBEngine/RocksDBSettingsManager.h"
#include "RocksDBEngine/RocksDBTransactionCollection.h"
#include "StorageEngine/StorageEngine.h"
#include "StorageEngine/TransactionCollection.h"
@ -66,11 +65,9 @@ RocksDBTransactionState::RocksDBTransactionState(
_rocksReadOptions(),
_cacheTx(nullptr),
_numCommits(0),
_numInternal(0),
_numInserts(0),
_numUpdates(0),
_numRemoves(0),
_lastUsedCollection(0),
_keys{_arena},
_parallel(false) {}
@ -187,7 +184,8 @@ void RocksDBTransactionState::createTransaction() {
TRI_ASSERT(_rocksTransaction == nullptr ||
_rocksTransaction->GetState() == rocksdb::Transaction::COMMITED ||
_rocksTransaction->GetState() == rocksdb::Transaction::ROLLEDBACK);
(_rocksTransaction->GetState() == rocksdb::Transaction::STARTED &&
_rocksTransaction->GetNumKeys() == 0));
_rocksTransaction =
db->BeginTransaction(_rocksWriteOptions, trxOpts, _rocksTransaction);
@ -201,7 +199,6 @@ void RocksDBTransactionState::createTransaction() {
++_numLogdata;
#endif
}
TRI_ASSERT(_lastUsedCollection == 0);
}
void RocksDBTransactionState::cleanupTransaction() noexcept {
@ -221,21 +218,6 @@ void RocksDBTransactionState::cleanupTransaction() noexcept {
arangodb::Result RocksDBTransactionState::internalCommit() {
TRI_ASSERT(_rocksTransaction != nullptr);
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
uint64_t x = _numInserts + _numRemoves + _numUpdates;
if (hasHint(transaction::Hints::Hint::SINGLE_OPERATION)) {
TRI_ASSERT(x <= 1 && _numLogdata == x);
} else {
if (_numLogdata < 1 + (x > 0 ? 1 : 0) + _numRemoves) {
LOG_TOPIC(DEBUG, Logger::FIXME) << "_numInserts " << _numInserts << " "
<< "_numRemoves " << _numRemoves << " "
<< "_numUpdates " << _numUpdates << " "
<< "_numLogdata " << _numLogdata;
}
// begin transaction + n DocumentOpsPrologue + m doc removes
TRI_ASSERT(_numLogdata >= 1 + (x > 0 ? 1 : 0) + _numRemoves);
}
#endif
ExecContext const* exe = ExecContext::CURRENT;
if (!isReadOnlyTransaction() && exe != nullptr) {
@ -246,7 +228,36 @@ arangodb::Result RocksDBTransactionState::internalCommit() {
}
Result result;
if (_rocksTransaction->GetNumKeys() > 0) {
if (hasOperations()) {
// we are actually going to attempt a commit
if (!hasHint(transaction::Hints::Hint::SINGLE_OPERATION)) {
// add custom commit marker to increase WAL tailing reliability
RocksDBLogValue logValue = RocksDBLogValue::CommitTransaction(_vocbase->id(), id());
_rocksTransaction->PutLogData(logValue.slice());
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
_numLogdata++;
#endif
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
uint64_t x = _numInserts + _numRemoves + _numUpdates;
if (hasHint(transaction::Hints::Hint::SINGLE_OPERATION)) {
TRI_ASSERT(x <= 1 && _numLogdata == x);
} else {
if (_numLogdata != (2 + _numRemoves)) {
LOG_TOPIC(ERR, Logger::FIXME) << "_numInserts " << _numInserts << " "
<< "_numRemoves " << _numRemoves << " "
<< "_numUpdates " << _numUpdates << " "
<< "_numLogdata " << _numLogdata;
sleep(1); // let it print
}
// begin transaction + commit transaction + n doc removes
TRI_ASSERT(_numLogdata == (2 + _numRemoves));
}
#endif
// set wait for sync flag if required
if (waitForSync()) {
_rocksWriteOptions.sync = true;
@ -283,22 +294,6 @@ arangodb::Result RocksDBTransactionState::internalCommit() {
for (auto& trxCollection : _collections) {
RocksDBTransactionCollection* collection =
static_cast<RocksDBTransactionCollection*>(trxCollection);
int64_t adjustment =
collection->numInserts() - collection->numRemoves();
if (collection->numInserts() != 0 || collection->numRemoves() != 0 ||
collection->revision() != 0) {
RocksDBCollection* coll = static_cast<RocksDBCollection*>(
trxCollection->collection()->getPhysical());
coll->adjustNumberDocuments(adjustment);
coll->setRevision(collection->revision());
RocksDBEngine* engine = rocksutils::globalRocksEngine();
RocksDBSettingsManager::CounterAdjustment update(
latestSeq, collection->numInserts(), collection->numRemoves(),
collection->revision());
engine->settingsManager()->updateCounter(coll->objectId(), update);
}
// we need this in case of an intermediate commit. The number of
// initial documents is adjusted and numInserts / removes is set to 0
// index estimator updates are buffered
@ -307,6 +302,10 @@ arangodb::Result RocksDBTransactionState::internalCommit() {
}
}
} else {
TRI_ASSERT(_rocksTransaction->GetNumKeys() == 0 &&
_rocksTransaction->GetNumPuts() == 0 &&
_rocksTransaction->GetNumDeletes() == 0);
for (auto& trxCollection : _collections) {
RocksDBTransactionCollection* collection =
static_cast<RocksDBTransactionCollection*>(trxCollection);
@ -378,95 +377,61 @@ Result RocksDBTransactionState::abortTransaction(
return result;
}
void RocksDBTransactionState::prepareOperation(
TRI_voc_cid_t collectionId, TRI_voc_rid_t revisionId, StringRef const& key,
void RocksDBTransactionState::prepareOperation(TRI_voc_cid_t cid, TRI_voc_rid_t rid,
TRI_voc_document_operation_e operationType) {
TRI_ASSERT(!isReadOnlyTransaction());
bool singleOp = hasHint(transaction::Hints::Hint::SINGLE_OPERATION);
if (collectionId != _lastUsedCollection) {
// single operations should never call this method twice
// singleOp => lastUsedColl == 0
TRI_ASSERT(!singleOp || _lastUsedCollection == 0);
_lastUsedCollection = collectionId;
if (!singleOp) {
if (operationType != TRI_VOC_DOCUMENT_OPERATION_UNKNOWN) {
RocksDBLogValue logValue =
RocksDBLogValue::DocumentOpsPrologue(collectionId);
if (singleOp) {
// singleOp => no modifications yet
TRI_ASSERT(_rocksTransaction->GetNumPuts() == 0 &&
_rocksTransaction->GetNumDeletes() == 0);
switch (operationType) {
case TRI_VOC_DOCUMENT_OPERATION_INSERT:
case TRI_VOC_DOCUMENT_OPERATION_UPDATE:
case TRI_VOC_DOCUMENT_OPERATION_REPLACE: {
RocksDBLogValue logValue = RocksDBLogValue::SinglePut(_vocbase->id(), cid);
_rocksTransaction->PutLogData(logValue.slice());
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
++_numLogdata;
TRI_ASSERT(_numLogdata++ == 0);
#endif
break;
}
} else {
// singleOp => no modifications yet
TRI_ASSERT(!singleOp || (_rocksTransaction->GetNumPuts() == 0 &&
_rocksTransaction->GetNumDeletes() == 0));
case TRI_VOC_DOCUMENT_OPERATION_REMOVE: {
TRI_ASSERT(rid != 0);
RocksDBLogValue logValue = RocksDBLogValue::SingleRemoveV2(_vocbase->id(),
cid, rid);
_rocksTransaction->PutLogData(logValue.slice());
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
TRI_ASSERT(_numLogdata == 0);
TRI_ASSERT(_numLogdata++ == 0);
#endif
switch (operationType) {
case TRI_VOC_DOCUMENT_OPERATION_INSERT:
case TRI_VOC_DOCUMENT_OPERATION_UPDATE:
case TRI_VOC_DOCUMENT_OPERATION_REPLACE: {
RocksDBLogValue logValue =
RocksDBLogValue::SinglePut(_vocbase->id(), collectionId);
_rocksTransaction->PutLogData(logValue.slice());
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
++_numLogdata;
#endif
break;
}
case TRI_VOC_DOCUMENT_OPERATION_REMOVE: {
TRI_ASSERT(!key.empty());
RocksDBLogValue logValue =
RocksDBLogValue::SingleRemove(_vocbase->id(), collectionId, key);
_rocksTransaction->PutLogData(logValue.slice());
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
++_numLogdata;
#endif
} break;
case TRI_VOC_DOCUMENT_OPERATION_UNKNOWN:
break;
}
} break;
case TRI_VOC_DOCUMENT_OPERATION_UNKNOWN:
break;
}
}
// we need to log the remove log entry, if we don't have the single
// optimization
if (!singleOp && (operationType == TRI_VOC_DOCUMENT_OPERATION_UPDATE ||
operationType == TRI_VOC_DOCUMENT_OPERATION_REPLACE ||
operationType == TRI_VOC_DOCUMENT_OPERATION_REMOVE)) {
} else {
if (operationType == TRI_VOC_DOCUMENT_OPERATION_REMOVE) {
RocksDBLogValue logValue = RocksDBLogValue::DocumentRemove(key);
RocksDBLogValue logValue = RocksDBLogValue::DocumentRemoveV2(rid);
_rocksTransaction->PutLogData(logValue.slice());
} else {
RocksDBLogValue logValue =
RocksDBLogValue::DocumentRemoveAsPartOfUpdate(key);
_rocksTransaction->PutLogData(logValue.slice());
}
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
++_numLogdata;
++_numLogdata;
#endif
}
}
}
/// @brief add an operation for a transaction collection
RocksDBOperationResult RocksDBTransactionState::addOperation(
Result RocksDBTransactionState::addOperation(
TRI_voc_cid_t cid, TRI_voc_rid_t revisionId,
TRI_voc_document_operation_e operationType, uint64_t operationSize,
uint64_t keySize) {
size_t currentSize =
_rocksTransaction->GetWriteBatch()->GetWriteBatch()->GetDataSize();
uint64_t newSize = currentSize + operationSize + keySize;
if (newSize > _options.maxTransactionSize) {
TRI_voc_document_operation_e operationType) {
size_t currentSize = _rocksTransaction->GetWriteBatch()->GetWriteBatch()->GetDataSize();
if (currentSize > _options.maxTransactionSize) {
// we hit the transaction size limit
std::string message =
"aborting transaction because maximal transaction size limit of " +
std::to_string(_options.maxTransactionSize) + " bytes is reached";
return RocksDBOperationResult(Result(TRI_ERROR_RESOURCE_LIMIT, message));
return Result(Result(TRI_ERROR_RESOURCE_LIMIT, message));
}
auto collection =
@ -479,7 +444,7 @@ RocksDBOperationResult RocksDBTransactionState::addOperation(
}
// should not fail or fail with exception
collection->addOperation(operationType, operationSize, revisionId);
collection->addOperation(operationType, revisionId);
// clear the query cache for this collection
if (arangodb::aql::QueryCache::instance()->mayBeActive()) {
@ -503,31 +468,7 @@ RocksDBOperationResult RocksDBTransactionState::addOperation(
}
// perform an intermediate commit if necessary
checkIntermediateCommit(newSize);
return RocksDBOperationResult();
}
/// @brief add an internal operation for a transaction
RocksDBOperationResult RocksDBTransactionState::addInternalOperation(
uint64_t operationSize, uint64_t keySize) {
size_t currentSize =
_rocksTransaction->GetWriteBatch()->GetWriteBatch()->GetDataSize();
uint64_t newSize = currentSize + operationSize + keySize;
if (newSize > _options.maxTransactionSize) {
// we hit the transaction size limit
std::string message =
"aborting transaction because maximal transaction size limit of " +
std::to_string(_options.maxTransactionSize) + " bytes is reached";
return RocksDBOperationResult(Result(TRI_ERROR_RESOURCE_LIMIT, message));
}
++_numInternal;
// perform an intermediate commit if necessary
checkIntermediateCommit(newSize);
return RocksDBOperationResult();
return checkIntermediateCommit(currentSize);
}
RocksDBMethods* RocksDBTransactionState::rocksdbMethods() {
@ -562,7 +503,7 @@ uint64_t RocksDBTransactionState::sequenceNumber() const {
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, "No snapshot set");
}
void RocksDBTransactionState::triggerIntermediateCommit() {
Result RocksDBTransactionState::triggerIntermediateCommit() {
TRI_IF_FAILURE("FailBeforeIntermediateCommit") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
@ -575,7 +516,11 @@ void RocksDBTransactionState::triggerIntermediateCommit() {
LOG_TOPIC(DEBUG, Logger::ROCKSDB) << "INTERMEDIATE COMMIT!";
#endif
internalCommit();
Result res = internalCommit();
if (res.fail()) {
// FIXME: do we abort the transaction ?
return res;
}
TRI_IF_FAILURE("FailAfterIntermediateCommit") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
@ -584,8 +529,6 @@ void RocksDBTransactionState::triggerIntermediateCommit() {
TRI_SegfaultDebugging("SegfaultAfterIntermediateCommit");
}
_lastUsedCollection = 0;
_numInternal = 0;
_numInserts = 0;
_numUpdates = 0;
_numRemoves = 0;
@ -593,17 +536,19 @@ void RocksDBTransactionState::triggerIntermediateCommit() {
_numLogdata = 0;
#endif
createTransaction();
return TRI_ERROR_NO_ERROR;
}
void RocksDBTransactionState::checkIntermediateCommit(uint64_t newSize) {
auto numOperations = _numInserts + _numUpdates + _numRemoves + _numInternal;
Result RocksDBTransactionState::checkIntermediateCommit(uint64_t newSize) {
auto numOperations = _numInserts + _numUpdates + _numRemoves;
// perform an intermediate commit
// this will be done if either the "number of operations" or the
// "transaction size" counters have reached their limit
if (_options.intermediateCommitCount <= numOperations ||
_options.intermediateCommitSize <= newSize) {
triggerIntermediateCommit();
return triggerIntermediateCommit();
}
return TRI_ERROR_NO_ERROR;
}
/// @brief temporarily lease a Builder object

View File

@ -79,35 +79,24 @@ class RocksDBTransactionState final : public TransactionState {
Result abortTransaction(transaction::Methods* trx) override;
uint64_t numCommits() const { return _numCommits; }
uint64_t numInternal() const { return _numInternal; }
uint64_t numInserts() const { return _numInserts; }
uint64_t numUpdates() const { return _numUpdates; }
uint64_t numRemoves() const { return _numRemoves; }
/// @brief reset previous log state after a rollback to safepoint
void resetLogState() { _lastUsedCollection = 0; }
inline bool hasOperations() const {
return (_numInserts > 0 || _numRemoves > 0 || _numUpdates > 0 || _numInternal > 0);
return (_numInserts > 0 || _numRemoves > 0 || _numUpdates > 0);
}
bool hasFailedOperations() const override {
return (_status == transaction::Status::ABORTED) && hasOperations();
}
void prepareOperation(TRI_voc_cid_t collectionId, TRI_voc_rid_t revisionId,
StringRef const& key,
void prepareOperation(TRI_voc_cid_t cid, TRI_voc_rid_t rid,
TRI_voc_document_operation_e operationType);
/// @brief add an operation for a transaction collection
RocksDBOperationResult addOperation(
TRI_voc_cid_t collectionId, TRI_voc_rid_t revisionId,
TRI_voc_document_operation_e operationType, uint64_t operationSize,
uint64_t keySize);
/// @brief add an internal operation for a transaction
RocksDBOperationResult addInternalOperation(
uint64_t operationSize, uint64_t keySize);
Result addOperation(TRI_voc_cid_t collectionId,
TRI_voc_rid_t revisionId, TRI_voc_document_operation_e opType);
RocksDBMethods* rocksdbMethods();
@ -145,11 +134,6 @@ class RocksDBTransactionState final : public TransactionState {
RocksDBKey* leaseRocksDBKey();
/// @brief return a temporary RocksDBKey object. Not thread safe
void returnRocksDBKey(RocksDBKey* key);
/// @brief Trigger an intermediate commit.
/// Handle with care if failing after this commit it will only
/// be rolled back until this point of time.
/// Not thread safe
void triggerIntermediateCommit();
/// @brief Every index can track hashes inserted into this index
/// Used to update the estimate after the trx commited
@ -166,8 +150,15 @@ class RocksDBTransactionState final : public TransactionState {
void cleanupTransaction() noexcept;
/// @brief internally commit a transaction
arangodb::Result internalCommit();
/// @brief Trigger an intermediate commit.
/// Handle with care if failing after this commit it will only
/// be rolled back until this point of time.
/// Not thread safe
Result triggerIntermediateCommit();
/// @brief check sizes and call internalCommit if too big
void checkIntermediateCommit(uint64_t newSize);
Result checkIntermediateCommit(uint64_t newSize);
private:
/// @brief rocksdb transaction may be null for read only transactions
@ -184,15 +175,12 @@ class RocksDBTransactionState final : public TransactionState {
std::unique_ptr<RocksDBMethods> _rocksMethods;
uint64_t _numCommits;
uint64_t _numInternal;
// if a transaction gets bigger than these values then an automatic
// intermediate commit will be done
uint64_t _numInserts;
uint64_t _numUpdates;
uint64_t _numRemoves;
/// @brief Last collection used for transaction. Used for WAL
TRI_voc_cid_t _lastUsedCollection;
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
/// store the number of log entries in WAL
uint64_t _numLogdata = 0;

View File

@ -29,7 +29,8 @@ namespace {
static RocksDBEntryType placeholder = arangodb::RocksDBEntryType::Placeholder;
static rocksdb::Slice Placeholder(
reinterpret_cast<std::underlying_type<RocksDBEntryType>::type*>(&placeholder),
reinterpret_cast<std::underlying_type<RocksDBEntryType>::type*>(
&placeholder),
1);
static RocksDBEntryType database = arangodb::RocksDBEntryType::Database;
@ -192,16 +193,22 @@ char const* arangodb::rocksDBLogTypeName(arangodb::RocksDBLogType type) {
#endif
case arangodb::RocksDBLogType::BeginTransaction:
return "BeginTransaction";
case arangodb::RocksDBLogType::CommitTransaction:
return "CommitTransaction";
case arangodb::RocksDBLogType::DocumentOperationsPrologue:
return "DocumentOperationsPrologue";
case arangodb::RocksDBLogType::DocumentRemove:
return "DocumentRemove";
case arangodb::RocksDBLogType::DocumentRemoveV2:
return "DocumentRemoveV2";
case arangodb::RocksDBLogType::DocumentRemoveAsPartOfUpdate:
return "IgnoreRemoveAsPartOfUpdate";
case arangodb::RocksDBLogType::SinglePut:
return "SinglePut";
case arangodb::RocksDBLogType::SingleRemove:
return "SingleRemove";
case arangodb::RocksDBLogType::SingleRemoveV2:
return "SingleRemoveV2";
case arangodb::RocksDBLogType::Invalid:
return "Invalid";
}
@ -247,7 +254,4 @@ rocksdb::Slice const& arangodb::rocksDBSlice(RocksDBEntryType const& type) {
return Placeholder; // avoids warning - errorslice instead ?!
}
char arangodb::rocksDBFormatVersion() {
return '0';
}
char arangodb::rocksDBFormatVersion() { return '0'; }

View File

@ -70,15 +70,18 @@ enum class RocksDBLogType : char {
ViewDrop = ':',
ViewChange = ';',
BeginTransaction = '<',
DocumentOperationsPrologue = '=',
DocumentRemove = '>',
DocumentOperationsPrologue = '=', // <- deprecated
DocumentRemove = '>', // <- deprecated
SinglePut = '?',
SingleRemove = '@',
DocumentRemoveAsPartOfUpdate = 'A',
SingleRemove = '@', // <- deprecated
DocumentRemoveAsPartOfUpdate = 'A', // <- deprecated
ViewRename = 'B',
#ifdef USE_IRESEARCH
IResearchLinkDrop = 'C'
IResearchLinkDrop = 'C',
#endif
CommitTransaction = 'D',
DocumentRemoveV2 = 'E',
SingleRemoveV2 = 'F'
};
enum class RocksDBSettingsType : char {

View File

@ -98,7 +98,8 @@ void RocksDBVPackUniqueIndexIterator::reset() {
_done = false;
}
bool RocksDBVPackUniqueIndexIterator::next(LocalDocumentIdCallback const& cb, size_t limit) {
bool RocksDBVPackUniqueIndexIterator::next(LocalDocumentIdCallback const& cb,
size_t limit) {
TRI_ASSERT(_trx->state()->isRunning());
if (limit == 0 || _done) {
@ -110,10 +111,11 @@ bool RocksDBVPackUniqueIndexIterator::next(LocalDocumentIdCallback const& cb, si
auto value = RocksDBValue::Empty(RocksDBEntryType::PrimaryIndexValue);
RocksDBMethods* mthds = RocksDBTransactionState::toMethods(_trx);
arangodb::Result r = mthds->Get(_index->columnFamily(), _key.ref(), value.buffer());
arangodb::Result r =
mthds->Get(_index->columnFamily(), _key.ref(), value.buffer());
if (r.ok()) {
cb(LocalDocumentId(RocksDBValue::revisionId(*value.buffer())));
cb(RocksDBValue::documentId(*value.buffer()));
}
// there is at most one element, so we are done now
@ -169,7 +171,8 @@ bool RocksDBVPackIndexIterator::outOfRange() const {
}
}
bool RocksDBVPackIndexIterator::next(LocalDocumentIdCallback const& cb, size_t limit) {
bool RocksDBVPackIndexIterator::next(LocalDocumentIdCallback const& cb,
size_t limit) {
TRI_ASSERT(_trx->state()->isRunning());
if (limit == 0 || !_iterator->Valid() || outOfRange()) {
@ -182,11 +185,9 @@ bool RocksDBVPackIndexIterator::next(LocalDocumentIdCallback const& cb, size_t l
while (limit > 0) {
TRI_ASSERT(_index->objectId() == RocksDBKey::objectId(_iterator->key()));
LocalDocumentId const documentId(
_index->_unique
? RocksDBValue::revisionId(_iterator->value())
: RocksDBKey::revisionId(_bounds.type(), _iterator->key()));
cb(documentId);
cb(_index->_unique
? RocksDBValue::documentId(_iterator->value())
: RocksDBKey::documentId(_bounds.type(), _iterator->key()));
--limit;
if (_reverse) {
@ -351,7 +352,7 @@ int RocksDBVPackIndex::fillElement(VPackBuilder& leased,
// value(s) + revisionID
// - Value: empty
RocksDBKey key;
key.constructVPackIndexValue(_objectId, leased.slice(), documentId.id());
key.constructVPackIndexValue(_objectId, leased.slice(), documentId);
elements.emplace_back(std::move(key));
hashes.push_back(leased.slice().normalizedHash());
}
@ -360,7 +361,8 @@ int RocksDBVPackIndex::fillElement(VPackBuilder& leased,
std::vector<VPackSlice> sliceStack;
try {
buildIndexValues(leased, documentId, doc, 0, elements, sliceStack, hashes);
buildIndexValues(leased, documentId, doc, 0, elements, sliceStack,
hashes);
} catch (arangodb::basics::Exception const& ex) {
return ex.code();
} catch (std::bad_alloc const&) {
@ -400,7 +402,7 @@ void RocksDBVPackIndex::addIndexValue(VPackBuilder& leased,
// + primary key
// - Value: empty
RocksDBKey key;
key.constructVPackIndexValue(_objectId, leased.slice(), documentId.id());
key.constructVPackIndexValue(_objectId, leased.slice(), documentId);
elements.emplace_back(std::move(key));
hashes.push_back(leased.slice().normalizedHash());
}
@ -569,12 +571,12 @@ Result RocksDBVPackIndex::insertInternal(transaction::Methods* trx,
// now we are going to construct the value to insert into rocksdb
// unique indexes have a different key structure
RocksDBValue value = _unique ? RocksDBValue::UniqueVPackIndexValue(documentId.id())
RocksDBValue value = _unique ? RocksDBValue::UniqueVPackIndexValue(documentId)
: RocksDBValue::VPackIndexValue();
size_t const count = elements.size();
RocksDBValue existing =
RocksDBValue::Empty(RocksDBEntryType::UniqueVPackIndexValue);
RocksDBValue::Empty(RocksDBEntryType::UniqueVPackIndexValue);
for (size_t i = 0; i < count; ++i) {
RocksDBKey& key = elements[i];
if (_unique) {
@ -617,12 +619,12 @@ Result RocksDBVPackIndex::insertInternal(transaction::Methods* trx,
}
if (res == TRI_ERROR_ARANGO_UNIQUE_CONSTRAINT_VIOLATED) {
LocalDocumentId rev(RocksDBValue::revisionId(existing));
LocalDocumentId rev = RocksDBValue::documentId(existing);
ManagedDocumentResult mmdr;
bool success = _collection->getPhysical()->readDocument(trx, rev, mmdr);
TRI_ASSERT(success);
std::string existingKey(
VPackSlice(mmdr.vpack()).get(StaticStrings::KeyString).copyString());
VPackSlice(mmdr.vpack()).get(StaticStrings::KeyString).copyString());
if (mode == OperationMode::internal) {
return IndexResult(res, std::move(existingKey));
}
@ -632,21 +634,18 @@ Result RocksDBVPackIndex::insertInternal(transaction::Methods* trx,
return IndexResult(res, this);
}
Result RocksDBVPackIndex::updateInternal(transaction::Methods* trx,
RocksDBMethods* mthds,
LocalDocumentId const& oldDocumentId,
arangodb::velocypack::Slice const& oldDoc,
LocalDocumentId const& newDocumentId,
velocypack::Slice const& newDoc,
OperationMode mode) {
Result RocksDBVPackIndex::updateInternal(
transaction::Methods* trx, RocksDBMethods* mthds,
LocalDocumentId const& oldDocumentId,
arangodb::velocypack::Slice const& oldDoc,
LocalDocumentId const& newDocumentId, velocypack::Slice const& newDoc,
OperationMode mode) {
if (!_unique || _useExpansion) {
// only unique index supports in-place updates
// lets also not handle the complex case of expanded arrays
return RocksDBIndex::updateInternal(trx, mthds, oldDocumentId, oldDoc,
newDocumentId, newDoc, mode);
} else {
bool equal = true;
for (size_t i = 0; i < _paths.size(); ++i) {
TRI_ASSERT(!_paths[i].empty());
@ -678,19 +677,20 @@ Result RocksDBVPackIndex::updateInternal(transaction::Methods* trx,
{
// rethrow all types of exceptions from here...
transaction::BuilderLeaser leased(trx);
res = fillElement(*(leased.get()), newDocumentId, newDoc, elements, hashes);
res =
fillElement(*(leased.get()), newDocumentId, newDoc, elements, hashes);
}
if (res != TRI_ERROR_NO_ERROR) {
return IndexResult(res, this);
}
RocksDBValue value = RocksDBValue::UniqueVPackIndexValue(newDocumentId.id());
RocksDBValue value = RocksDBValue::UniqueVPackIndexValue(newDocumentId);
size_t const count = elements.size();
for (size_t i = 0; i < count; ++i) {
RocksDBKey& key = elements[i];
if (res == TRI_ERROR_NO_ERROR) {
arangodb::Result r =
mthds->Put(_cf, key, value.string(), rocksutils::index);
mthds->Put(_cf, key, value.string(), rocksutils::index);
if (!r.ok()) {
res = r.errorNumber();
}
@ -726,7 +726,6 @@ Result RocksDBVPackIndex::removeInternal(transaction::Methods* trx,
return IndexResult(res, this);
}
size_t const count = elements.size();
for (size_t i = 0; i < count; ++i) {
arangodb::Result r = mthds->Delete(_cf, elements[i]);
@ -750,9 +749,10 @@ Result RocksDBVPackIndex::removeInternal(transaction::Methods* trx,
/// @brief attempts to locate an entry in the index
/// Warning: who ever calls this function is responsible for destroying
/// the RocksDBVPackIndexIterator* results
IndexIterator* RocksDBVPackIndex::lookup(
transaction::Methods* trx, ManagedDocumentResult* mmdr,
VPackSlice const searchValues, bool reverse) const {
IndexIterator* RocksDBVPackIndex::lookup(transaction::Methods* trx,
ManagedDocumentResult* mmdr,
VPackSlice const searchValues,
bool reverse) const {
TRI_ASSERT(searchValues.isArray());
TRI_ASSERT(searchValues.length() <= _fields.size());
@ -770,10 +770,12 @@ IndexIterator* RocksDBVPackIndex::lookup(
leftSearch.add(eq);
}
if (lastNonEq.isNone() && _unique && searchValues.length() == _fields.size()) {
if (lastNonEq.isNone() && _unique &&
searchValues.length() == _fields.size()) {
leftSearch.close();
return new RocksDBVPackUniqueIndexIterator(_collection, trx, mmdr, this, leftSearch.slice());
return new RocksDBVPackUniqueIndexIterator(_collection, trx, mmdr, this,
leftSearch.slice());
}
VPackSlice leftBorder;
@ -855,7 +857,8 @@ IndexIterator* RocksDBVPackIndex::lookup(
: RocksDBKeyBounds::VPackIndex(
_objectId, leftBorder, rightBorder);
return new RocksDBVPackIndexIterator(_collection, trx, mmdr, this, reverse, std::move(bounds));
return new RocksDBVPackIndexIterator(_collection, trx, mmdr, this, reverse,
std::move(bounds));
}
bool RocksDBVPackIndex::accessFitsIndex(
@ -990,7 +993,9 @@ void RocksDBVPackIndex::matchAttributes(
case arangodb::aql::NODE_TYPE_OPERATOR_BINARY_IN:
if (accessFitsIndex(op->getMember(0), op->getMember(1), op, reference,
found, nonNullAttributes, isExecution)) {
size_t av = SimpleAttributeEqualityMatcher::estimateNumberOfArrayMembers(op->getMember(1));
size_t av =
SimpleAttributeEqualityMatcher::estimateNumberOfArrayMembers(
op->getMember(1));
if (av > 1) {
// attr IN [ a, b, c ] => this will produce multiple items, so
// count them!
@ -1532,7 +1537,8 @@ bool RocksDBVPackIndex::isDuplicateOperator(
return duplicate;
}
rocksdb::SequenceNumber RocksDBVPackIndex::serializeEstimate(std::string& output, rocksdb::SequenceNumber seq) const {
rocksdb::SequenceNumber RocksDBVPackIndex::serializeEstimate(
std::string& output, rocksdb::SequenceNumber seq) const {
TRI_ASSERT(!ServerState::instance()->isCoordinator());
if (!_unique) {
TRI_ASSERT(_estimator != nullptr);

View File

@ -100,15 +100,12 @@ class RocksDBVPackIndexIterator final : public IndexIterator {
transaction::Methods* trx,
ManagedDocumentResult* mmdr,
arangodb::RocksDBVPackIndex const* index,
bool reverse,
RocksDBKeyBounds&& bounds);
bool reverse, RocksDBKeyBounds&& bounds);
~RocksDBVPackIndexIterator() = default;
public:
char const* typeName() const override {
return "rocksdb-index-iterator";
}
char const* typeName() const override { return "rocksdb-index-iterator"; }
/// @brief Get the next limit many element in the index
bool next(LocalDocumentIdCallback const& cb, size_t limit) override;
@ -169,10 +166,8 @@ class RocksDBVPackIndex : public RocksDBIndex {
///
/// Warning: who ever calls this function is responsible for destroying
/// the velocypack::Slice and the RocksDBVPackIndexIterator* results
IndexIterator* lookup(transaction::Methods*,
ManagedDocumentResult* mmdr,
arangodb::velocypack::Slice const,
bool reverse) const;
IndexIterator* lookup(transaction::Methods*, ManagedDocumentResult* mmdr,
arangodb::velocypack::Slice const, bool reverse) const;
bool supportsFilterCondition(arangodb::aql::AstNode const*,
arangodb::aql::Variable const*, size_t, size_t&,
@ -241,14 +236,16 @@ class RocksDBVPackIndex : public RocksDBIndex {
std::vector<int>& expanding);
/// @brief helper function to insert a document into any index type
int fillElement(velocypack::Builder& leased, LocalDocumentId const& documentId,
VPackSlice const& doc, std::vector<RocksDBKey>& elements,
int fillElement(velocypack::Builder& leased,
LocalDocumentId const& documentId, VPackSlice const& doc,
std::vector<RocksDBKey>& elements,
std::vector<uint64_t>& hashes);
/// @brief helper function to build the key and value for rocksdb from the
/// vector of slices
/// @param hashes list of VPackSlice hashes for the estimator.
void addIndexValue(velocypack::Builder& leased, LocalDocumentId const& documentId,
void addIndexValue(velocypack::Builder& leased,
LocalDocumentId const& documentId,
VPackSlice const& document,
std::vector<RocksDBKey>& elements,
std::vector<VPackSlice>& sliceStack,

View File

@ -40,8 +40,8 @@ RocksDBValue RocksDBValue::Collection(VPackSlice const& data) {
return RocksDBValue(RocksDBEntryType::Collection, data);
}
RocksDBValue RocksDBValue::PrimaryIndexValue(TRI_voc_rid_t revisionId) {
return RocksDBValue(RocksDBEntryType::PrimaryIndexValue, revisionId);
RocksDBValue RocksDBValue::PrimaryIndexValue(LocalDocumentId const& docId) {
return RocksDBValue(RocksDBEntryType::PrimaryIndexValue, docId);
}
RocksDBValue RocksDBValue::EdgeIndexValue(arangodb::StringRef const& vertexId) {
@ -52,8 +52,8 @@ RocksDBValue RocksDBValue::VPackIndexValue() {
return RocksDBValue(RocksDBEntryType::VPackIndexValue);
}
RocksDBValue RocksDBValue::UniqueVPackIndexValue(TRI_voc_rid_t revisionId) {
return RocksDBValue(RocksDBEntryType::UniqueVPackIndexValue, revisionId);
RocksDBValue RocksDBValue::UniqueVPackIndexValue(LocalDocumentId const& docId) {
return RocksDBValue(RocksDBEntryType::UniqueVPackIndexValue, docId);
}
RocksDBValue RocksDBValue::View(VPackSlice const& data) {
@ -72,16 +72,16 @@ RocksDBValue RocksDBValue::Empty(RocksDBEntryType type) {
return RocksDBValue(type);
}
TRI_voc_rid_t RocksDBValue::revisionId(RocksDBValue const& value) {
return revisionId(value._buffer.data(), value._buffer.size());
LocalDocumentId RocksDBValue::documentId(RocksDBValue const& value) {
return documentId(value._buffer.data(), value._buffer.size());
}
TRI_voc_rid_t RocksDBValue::revisionId(rocksdb::Slice const& slice) {
return revisionId(slice.data(), slice.size());
LocalDocumentId RocksDBValue::documentId(rocksdb::Slice const& slice) {
return documentId(slice.data(), slice.size());
}
TRI_voc_rid_t RocksDBValue::revisionId(std::string const& s) {
return revisionId(s.data(), s.size());
LocalDocumentId RocksDBValue::documentId(std::string const& s) {
return documentId(s.data(), s.size());
}
StringRef RocksDBValue::vertexId(rocksdb::Slice const& s) {
@ -114,13 +114,13 @@ uint64_t RocksDBValue::keyValue(std::string const& s) {
RocksDBValue::RocksDBValue(RocksDBEntryType type) : _type(type), _buffer() {}
RocksDBValue::RocksDBValue(RocksDBEntryType type, uint64_t data)
RocksDBValue::RocksDBValue(RocksDBEntryType type, LocalDocumentId const& docId)
: _type(type), _buffer() {
switch (_type) {
case RocksDBEntryType::UniqueVPackIndexValue:
case RocksDBEntryType::PrimaryIndexValue: {
_buffer.reserve(sizeof(uint64_t));
uint64ToPersistent(_buffer, data); // revision id
uint64ToPersistent(_buffer, docId.id()); // LocalDocumentId
break;
}
@ -166,9 +166,9 @@ RocksDBValue::RocksDBValue(RocksDBEntryType type, StringRef const& data)
}
}
TRI_voc_rid_t RocksDBValue::revisionId(char const* data, uint64_t size) {
TRI_ASSERT(data != nullptr && size >= sizeof(uint64_t));
return uint64FromPersistent(data);
LocalDocumentId RocksDBValue::documentId(char const* data, uint64_t size) {
TRI_ASSERT(data != nullptr && size >= sizeof(LocalDocumentId::BaseType));
return LocalDocumentId(uint64FromPersistent(data));
}
StringRef RocksDBValue::vertexId(char const* data, size_t size) {

View File

@ -28,7 +28,7 @@
#include "Basics/Common.h"
#include "Basics/StringRef.h"
#include "RocksDBEngine/RocksDBTypes.h"
#include "VocBase/vocbase.h"
#include "VocBase/LocalDocumentId.h"
#include <rocksdb/slice.h>
@ -47,10 +47,10 @@ class RocksDBValue {
static RocksDBValue Database(VPackSlice const& data);
static RocksDBValue Collection(VPackSlice const& data);
static RocksDBValue PrimaryIndexValue(TRI_voc_rid_t revisionId);
static RocksDBValue PrimaryIndexValue(LocalDocumentId const& docId);
static RocksDBValue EdgeIndexValue(arangodb::StringRef const& vertexId);
static RocksDBValue VPackIndexValue();
static RocksDBValue UniqueVPackIndexValue(TRI_voc_rid_t revisionId);
static RocksDBValue UniqueVPackIndexValue(LocalDocumentId const& docId);
static RocksDBValue View(VPackSlice const& data);
static RocksDBValue ReplicationApplierConfig(VPackSlice const& data);
static RocksDBValue KeyGeneratorValue(VPackSlice const& data);
@ -62,14 +62,14 @@ class RocksDBValue {
public:
//////////////////////////////////////////////////////////////////////////////
/// @brief Extracts the revisionId from a value
/// @brief Extracts the LocalDocumentId from a value
///
/// May be called only on PrimaryIndexValue values. Other types will throw.
//////////////////////////////////////////////////////////////////////////////
static TRI_voc_rid_t revisionId(RocksDBValue const&);
static TRI_voc_rid_t revisionId(rocksdb::Slice const&);
static TRI_voc_rid_t revisionId(std::string const&);
static LocalDocumentId documentId(RocksDBValue const&);
static LocalDocumentId documentId(rocksdb::Slice const&);
static LocalDocumentId documentId(std::string const&);
//////////////////////////////////////////////////////////////////////////////
/// @brief Extracts the vertex _to or _from ID (`_key`) from a value
@ -125,13 +125,13 @@ class RocksDBValue {
private:
RocksDBValue();
explicit RocksDBValue(RocksDBEntryType type);
RocksDBValue(RocksDBEntryType type, uint64_t data);
RocksDBValue(RocksDBEntryType type, LocalDocumentId const& docId);
RocksDBValue(RocksDBEntryType type, VPackSlice const& data);
RocksDBValue(RocksDBEntryType type, arangodb::StringRef const& data);
private:
static RocksDBEntryType type(char const* data, size_t size);
static TRI_voc_rid_t revisionId(char const* data, uint64_t size);
static LocalDocumentId documentId(char const* data, uint64_t size);
static StringRef vertexId(char const* data, size_t size);
static VPackSlice data(char const* data, size_t size);
static uint64_t keyValue(char const* data, size_t size);

View File

@ -80,83 +80,127 @@ WalAccessResult RocksDBWalAccess::openTransactions(
/// can potentially be batched into the same rocksdb write batch
/// but transactions can never be interleaved with operations
/// outside of the transaction
class MyWALParser : public rocksdb::WriteBatch::Handler,
public WalAccessContext {
class MyWALParser : public rocksdb::WriteBatch::Handler, public WalAccessContext {
// internal WAL parser states
enum State : char {
INVALID,
DB_CREATE,
DB_DROP,
COLLECTION_CREATE,
COLLECTION_DROP,
COLLECTION_RENAME,
COLLECTION_CHANGE,
INDEX_CREATE,
INDEX_DROP,
VIEW_CREATE,
VIEW_DROP,
VIEW_CHANGE,
VIEW_RENAME,
TRANSACTION,
SINGLE_PUT,
SINGLE_REMOVE
};
public:
MyWALParser(WalAccess::Filter const& filter,
WalAccess::MarkerCallback const& f)
: WalAccessContext(filter, f),
_documentsCF(RocksDBColumnFamily::documents()->GetID()),
_definitionsCF(RocksDBColumnFamily::definitions()->GetID()),
_documentsCF(RocksDBColumnFamily::documents()->GetID()),
_primaryCF(RocksDBColumnFamily::primary()->GetID()),
_startSequence(0),
_currentSequence(0) {}
void LogData(rocksdb::Slice const& blob) override {
// rocksdb does not count LogData towards sequence-number
RocksDBLogType type = RocksDBLogValue::type(blob);
TRI_DEFER(_lastLogType = type);
// skip ignored databases and collections
if (RocksDBLogValue::containsDatabaseId(type)) {
TRI_voc_tick_t dbId = RocksDBLogValue::databaseId(blob);
_currentDbId = dbId;
if (!shouldHandleDB(dbId)) {
resetTransientState();
return;
}
if (RocksDBLogValue::containsCollectionId(type)) {
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
_currentCid = cid;
if (!shouldHandleCollection(dbId, cid)) {
if (type == RocksDBLogType::SingleRemove || type == RocksDBLogType::SinglePut) {
resetTransientState();
} else {
_currentCid = 0;
}
return;
}
}
}
//LOG_TOPIC(ERR, Logger::FIXME) << "[LOG] " << _currentSequence
// << " " << rocksDBLogTypeName(type);
switch (type) {
case RocksDBLogType::DatabaseCreate:
case RocksDBLogType::DatabaseDrop: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
if (shouldHandleDB(RocksDBLogValue::databaseId(blob))) {
_state = DB_CREATE;
}
// wait for marker data in Put entry
break;
}
case RocksDBLogType::CollectionRename:
case RocksDBLogType::CollectionCreate:
case RocksDBLogType::CollectionChange: {
case RocksDBLogType::DatabaseDrop:
resetTransientState(); // finish ongoing trx
if (_lastLogType == RocksDBLogType::IndexCreate) {
TRI_ASSERT(_currentDbId == RocksDBLogValue::databaseId(blob));
TRI_ASSERT(_currentCid == RocksDBLogValue::collectionId(blob));
if (shouldHandleDB(RocksDBLogValue::databaseId(blob))) {
_state = DB_DROP;
}
// wait for marker data in Put entry
break;
case RocksDBLogType::CollectionCreate:
resetTransientState(); // finish ongoing trx
if (shouldHandleCollection(RocksDBLogValue::databaseId(blob),
RocksDBLogValue::collectionId(blob))) {
_state = COLLECTION_CREATE;
}
break;
case RocksDBLogType::CollectionRename:
resetTransientState(); // finish ongoing trx
if (shouldHandleCollection(RocksDBLogValue::databaseId(blob),
RocksDBLogValue::collectionId(blob))) {
_state = COLLECTION_RENAME; // collection name is not needed
// LOG_TOPIC(DEBUG, Logger::REPLICATION) << "renaming "
// << RocksDBLogValue::oldCollectionName(blob).toString();
}
break;
case RocksDBLogType::CollectionChange:
resetTransientState(); // finish ongoing trx
if (shouldHandleCollection(RocksDBLogValue::databaseId(blob),
RocksDBLogValue::collectionId(blob))) {
_state = COLLECTION_CHANGE;
}
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
break;
}
case RocksDBLogType::CollectionDrop: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
StringRef uuid = RocksDBLogValue::collectionUUID(blob);
TRI_ASSERT(!uuid.empty());
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
if (vocbase != nullptr) {
{ // tick number
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
// always print drop collection marker, shouldHandleCollection will
// always return false for dropped collections
if (shouldHandleDB(dbid)) {
TRI_vocbase_t* vocbase = loadVocbase(dbid);
if (vocbase != nullptr) {
{ // tick number
StringRef uuid = RocksDBLogValue::collectionUUID(blob);
TRI_ASSERT(!uuid.empty());
uint64_t tick = _currentSequence + (_startOfBatch ? 0 : 1);
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(tick)));
marker->add("type", VPackValue(REPLICATION_COLLECTION_DROP));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValuePair(uuid.data(), uuid.size(),
VPackValueType::String));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
}
break;
}
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);
LogicalCollection* coll = loadCollection(dbid, cid);
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", VPackValuePair(uuid.data(), uuid.size(),
VPackValueType::String));
marker->add("cuid", VPackValue(coll->globallyUniqueId()));
marker->add("data", stripped.first);
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
@ -164,55 +208,29 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
}
break;
}
case RocksDBLogType::IndexCreate: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
// only print markers from this collection if it is set
if (shouldHandleCollection(_currentDbId, _currentCid)) {
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
LogicalCollection* col = loadCollection(_currentDbId, _currentCid);
if (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("data", RocksDBLogValue::indexSlice(blob));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
}
break;
}
case RocksDBLogType::IndexDrop: {
resetTransientState(); // finish ongoing trx
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
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(_currentDbId, _currentCid)) {
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
LogicalCollection* col = loadCollection(_currentDbId, _currentCid);
if (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()));
VPackObjectBuilder data(&_builder, "data", true);
data->add("id", VPackValue(std::to_string(iid)));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
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()));
VPackObjectBuilder data(&_builder, "data", true);
data->add("id", VPackValue(std::to_string(iid)));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
break;
}
@ -225,68 +243,83 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
break;
}
case RocksDBLogType::BeginTransaction: {
TRI_ASSERT(!_singleOp);
resetTransientState(); // finish ongoing trx
_seenBeginTransaction = true;
_currentTrxId = RocksDBLogValue::transactionId(blob);
_currentDbId = RocksDBLogValue::databaseId(blob);
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
if (vocbase != nullptr) {
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type", VPackValue(rocksutils::convertLogType(type)));
marker->add("db", VPackValue(vocbase->name()));
marker->add("tid", VPackValue(std::to_string(_currentTrxId)));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
break;
}
case RocksDBLogType::DocumentOperationsPrologue: {
// part of an ongoing transaction
if (_currentDbId != 0 && _currentTrxId != 0) {
// database (and therefore transaction) may be ignored
TRI_ASSERT(_seenBeginTransaction && !_singleOp);
// document ops can ignore this collection later
_currentCid = RocksDBLogValue::collectionId(blob);
}
break;
}
case RocksDBLogType::DocumentRemove:
case RocksDBLogType::DocumentRemoveAsPartOfUpdate: {
// part of an ongoing transaction
if (_currentDbId != 0 && _currentTrxId != 0) {
// collection may be ignored
TRI_ASSERT(_seenBeginTransaction && !_singleOp);
if (shouldHandleCollection(_currentDbId, _currentCid)) {
_removeDocumentKey = RocksDBLogValue::documentKey(blob).toString();
TRI_voc_tid_t tid = RocksDBLogValue::transactionId(blob);
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
if (shouldHandleDB(dbid)) {
TRI_vocbase_t* vocbase = loadVocbase(dbid);
if (vocbase != nullptr) {
_state = TRANSACTION;
_currentTrxId = tid;
_trxDbId = dbid;
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type", VPackValue(rocksutils::convertLogType(type)));
marker->add("db", VPackValue(vocbase->name()));
marker->add("tid", VPackValue(std::to_string(_currentTrxId)));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
}
break;
}
case RocksDBLogType::SingleRemove: {
// we can only get here if we can handle this collection
TRI_ASSERT(!_singleOp);
resetTransientState(); // finish ongoing trx
_removeDocumentKey = RocksDBLogValue::documentKey(blob).toString();
_singleOp = true;
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
case RocksDBLogType::CommitTransaction: {
if (_state == TRANSACTION) {
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_tid_t tid = RocksDBLogValue::transactionId(blob);
TRI_ASSERT(_currentTrxId == tid && _trxDbId == dbid);
if (shouldHandleDB(dbid) && _currentTrxId == tid) {
writeCommitMarker(dbid);
}
}
resetTransientState();
break;
}
case RocksDBLogType::SinglePut: {
TRI_ASSERT(!_singleOp);
resetTransientState(); // finish ongoing trx
_singleOp = true;
_currentDbId = RocksDBLogValue::databaseId(blob);
_currentCid = RocksDBLogValue::collectionId(blob);
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
_state = SINGLE_PUT;
}
break;
}
case RocksDBLogType::SingleRemove: { // deprecated
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
_state = SINGLE_REMOVE; // revisionId is unknown
}
break;
}
case RocksDBLogType::DocumentRemoveV2: { // remove within a trx
if (_state == TRANSACTION) {
TRI_ASSERT(_removedDocRid == 0);
_removedDocRid = RocksDBLogValue::revisionId(blob);
} else {
resetTransientState();
}
break;
}
case RocksDBLogType::SingleRemoveV2: {
resetTransientState(); // finish ongoing trx
TRI_voc_tick_t dbid = RocksDBLogValue::databaseId(blob);
TRI_voc_cid_t cid = RocksDBLogValue::collectionId(blob);
if (shouldHandleCollection(dbid, cid)) {
_state = SINGLE_REMOVE;
_removedDocRid = RocksDBLogValue::revisionId(blob);
}
break;
}
case RocksDBLogType::DocumentOperationsPrologue:
case RocksDBLogType::DocumentRemove:
case RocksDBLogType::DocumentRemoveAsPartOfUpdate:
break; // ignore deprecated markers
default:
LOG_TOPIC(WARN, Logger::REPLICATION) << "Unhandled wal log entry "
@ -298,20 +331,11 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
rocksdb::Status PutCF(uint32_t column_family_id, rocksdb::Slice const& key,
rocksdb::Slice const& value) override {
tick();
if (!shouldHandleMarker(column_family_id, true, key)) {
if (column_family_id == _documentsCF && // ignoring collection
_lastLogType == RocksDBLogType::SinglePut) {
TRI_ASSERT(!_seenBeginTransaction);
resetTransientState(); // ignoring the put
}
return rocksdb::Status();
}
//LOG_TOPIC(ERR, Logger::ROCKSDB) << "[PUT] cf: " << column_family_id
// << ", key:" << key.ToString() << " value: " << value.ToString();
if (column_family_id == _definitionsCF) {
TRI_ASSERT(!_seenBeginTransaction && !_singleOp);
// LogData should have triggered a commit on ongoing transactions
if (RocksDBKey::type(key) == RocksDBEntryType::Database) {
// database slice should contain "id" and "name"
@ -319,22 +343,22 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
VPackSlice const name = data.get("name");
TRI_ASSERT(name.isString() && name.getStringLength() > 0);
if (_lastLogType == RocksDBLogType::DatabaseCreate) {
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
TRI_voc_tick_t dbid = RocksDBKey::databaseId(key);
if (_state == DB_CREATE) {
TRI_vocbase_t* vocbase = loadVocbase(dbid);
if (vocbase != nullptr) { // db is already deleted
{
VPackObjectBuilder marker(&_builder, true);
marker->add("tick", VPackValue(std::to_string(_currentSequence)));
marker->add("type",
VPackValue(rocksutils::convertLogType(_lastLogType)));
marker->add("type", VPackValue(REPLICATION_DATABASE_CREATE));
marker->add("db", name);
marker->add("data", data);
}
_callback(loadVocbase(_currentDbId), _builder.slice());
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
} else if (_lastLogType == RocksDBLogType::DatabaseDrop) {
} else if (_state == DB_DROP) {
// prepareDropDatabase should always write entry
VPackSlice const del = data.get("deleted");
TRI_ASSERT(del.isBool() && del.getBool());
@ -344,83 +368,87 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
marker->add("type", VPackValue(REPLICATION_DATABASE_DROP));
marker->add("db", name);
}
_callback(loadVocbase(_currentDbId), _builder.slice());
_callback(loadVocbase(dbid), _builder.slice());
_responseSize += _builder.size();
_builder.clear();
} else {
TRI_ASSERT(false); // unexpected
}
} // ignore Put in any other case
} else if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
// creating dropping indexes will change the collection definition
// in rocksdb. We do not need or want to print that again
if (_lastLogType == RocksDBLogType::IndexCreate ||
_lastLogType == RocksDBLogType::IndexDrop) {
_lastLogType = RocksDBLogType::Invalid;
return rocksdb::Status();
}
TRI_ASSERT(_lastLogType == RocksDBLogType::CollectionCreate ||
_lastLogType == RocksDBLogType::CollectionChange ||
_lastLogType == RocksDBLogType::CollectionRename);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
TRI_ASSERT(!_seenBeginTransaction && !_singleOp);
// LogData should have triggered a commit on ongoing transactions
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
LogicalCollection* col = loadCollection(_currentDbId, _currentCid);
if (vocbase != nullptr && col != nullptr) {
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("type",
VPackValue(rocksutils::convertLogType(_lastLogType)));
marker->add("db", VPackValue(loadVocbase(_currentDbId)->name()));
marker->add("db", VPackValue(vocbase->name()));
marker->add("cuid", VPackValue(col->globallyUniqueId()));
if (_lastLogType == RocksDBLogType::CollectionRename) {
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 { // change and create need full data
marker->add("data", RocksDBValue::data(value));
} else if (_state == COLLECTION_CHANGE) {
auto stripped = rocksutils::stripObjectIds(collectionDef);
marker->add("type", VPackValue(REPLICATION_COLLECTION_CHANGE));
marker->add("data", stripped.first);
}
}
_callback(loadVocbase(_currentDbId), _builder.slice());
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
// log type is only ever relevant, immediately after it appeared
// we want double occurences create / drop / change collection to fail
resetTransientState();
} // if (RocksDBKey::type(key) == RocksDBEntryType::Collection)
// reset everything immediately after DDL operations
resetTransientState();
} else if (column_family_id == _documentsCF) {
TRI_ASSERT((_seenBeginTransaction && !_singleOp) ||
(!_seenBeginTransaction && _singleOp));
// transaction needs the trx id
TRI_ASSERT(!_seenBeginTransaction || _currentTrxId != 0);
TRI_ASSERT(!_singleOp || _currentTrxId == 0);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
LogicalCollection* col = loadCollection(_currentDbId, _currentCid);
// db or collection may be deleted already
if (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("tid", VPackValue(std::to_string(_currentTrxId)));
marker->add("data", RocksDBValue::data(value));
}
_callback(loadVocbase(_currentDbId), _builder.slice());
_responseSize += _builder.size();
_builder.clear();
}
// reset whether or not marker was printed
if (_singleOp) {
if (_state != TRANSACTION && _state != SINGLE_PUT) {
resetTransientState();
return rocksdb::Status();
}
TRI_ASSERT(_state != SINGLE_PUT || _currentTrxId == 0);
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("tid", VPackValue(std::to_string(_currentTrxId)));
marker->add("data", RocksDBValue::data(value));
}
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
if (_state == SINGLE_PUT) {
resetTransientState(); // always reset after single op
}
}
return rocksdb::Status();
@ -428,99 +456,48 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
rocksdb::Status DeleteCF(uint32_t column_family_id,
rocksdb::Slice const& key) override {
return handleDeletion(column_family_id, key);
}
rocksdb::Status SingleDeleteCF(uint32_t column_family_id,
rocksdb::Slice const& key) override {
return handleDeletion(column_family_id, key);
}
rocksdb::Status handleDeletion(uint32_t column_family_id,
rocksdb::Slice const& key) {
tick();
#ifdef ARANGODB_ENABLE_MAINTAINER_MODE
if (column_family_id == _definitionsCF &&
shouldHandleMarker(column_family_id, false, key)) {
if (RocksDBKey::type(key) == RocksDBEntryType::Database) {
// databases are deleted when the last reference to it disappears
// the definitions entry deleted later in a cleanup thread
TRI_ASSERT(false);
} else if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
TRI_ASSERT(_lastLogType == RocksDBLogType::CollectionDrop);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
// we already printed this marker upon reading the log value
}
}
#endif
if (column_family_id != _documentsCF ||
!shouldHandleMarker(column_family_id, false, key)) {
if (column_family_id == _documentsCF) {
if (_lastLogType == RocksDBLogType::SingleRemove) {
TRI_ASSERT(!_seenBeginTransaction);
resetTransientState(); // ignoring the entire op
} else {
TRI_ASSERT(!_singleOp);
_removeDocumentKey.clear(); // just ignoring this key
}
}
if (column_family_id != _primaryCF) {
return rocksdb::Status(); // ignore all document operations
} else if (_state != TRANSACTION && _state != SINGLE_REMOVE) {
resetTransientState();
return rocksdb::Status();
}
TRI_ASSERT(_state != SINGLE_REMOVE || _currentTrxId == 0);
TRI_ASSERT(_state != TRANSACTION || _trxDbId != 0);
if (_lastLogType == RocksDBLogType::DocumentRemoveAsPartOfUpdate) {
_removeDocumentKey.clear();
return rocksdb::Status();
uint64_t objectId = RocksDBKey::objectId(key);
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
}
//LOG_TOPIC(ERR, Logger::ROCKSDB) << "[Delete] cf: " << column_family_id
//<< " key:" << key.ToString();
// document removes, because of a db / collection drop is not transactional
// and should not appear in the WAL.
if (!(_seenBeginTransaction || _singleOp) ||
(_lastLogType != RocksDBLogType::DocumentRemove &&
_lastLogType != RocksDBLogType::SingleRemove)) {
TRI_ASSERT(_removeDocumentKey.empty());
// collection drops etc may be batched directly after a transaction
// single operation updates in the WAL are wrongly ordered pre 3.3:
// [..., LogType::SinglePut, DELETE old, PUT new, ...]
if (_lastLogType != RocksDBLogType::SinglePut) {
resetTransientState(); // finish ongoing trx
}
return rocksdb::Status();
}
TRI_ASSERT(!_seenBeginTransaction || _currentTrxId != 0);
TRI_ASSERT(!_singleOp || _currentTrxId == 0);
TRI_ASSERT(_currentDbId != 0 && _currentCid != 0);
TRI_ASSERT(!_removeDocumentKey.empty());
StringRef docKey = RocksDBKey::primaryKey(key);
TRI_ASSERT(_state != TRANSACTION || _trxDbId == dbid);
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
LogicalCollection* col = loadCollection(_currentDbId, _currentCid);
// db or collection may be deleted already
if (vocbase != nullptr && col != nullptr) {
// FIXME: this revision is entirely meaningless
uint64_t rid = RocksDBKey::revisionId(RocksDBEntryType::Document, key);
{
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("tid", VPackValue(std::to_string(_currentTrxId)));
VPackObjectBuilder data(&_builder, "data", true);
data->add(StaticStrings::KeyString, VPackValue(_removeDocumentKey));
data->add(StaticStrings::RevString, VPackValue(std::to_string(rid)));
}
_callback(loadVocbase(_currentDbId), _builder.slice());
_responseSize += _builder.size();
_builder.clear();
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("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)));
}
// reset whether or not marker was printed
_removeDocumentKey.clear();
if (_singleOp) {
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
_removedDocRid = 0; // always reset
if (_state == SINGLE_REMOVE) {
resetTransientState();
}
@ -531,14 +508,16 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
// starting new write batch
_startSequence = startSequence;
_currentSequence = startSequence;
_lastLogType = RocksDBLogType::Invalid;
_startOfBatch = true;
TRI_ASSERT(!_singleOp);
_state = INVALID;
_currentTrxId = 0;
_trxDbId = 0;
_removedDocRid = 0;
}
void writeCommitMarker() {
TRI_ASSERT(_seenBeginTransaction && !_singleOp);
TRI_vocbase_t* vocbase = loadVocbase(_currentDbId);
void writeCommitMarker(TRI_voc_tick_t dbid) {
TRI_ASSERT(_state == TRANSACTION);
TRI_vocbase_t* vocbase = loadVocbase(dbid);
if (vocbase != nullptr) { // we be in shutdown
_builder.openObject(true);
_builder.add("tick", VPackValue(std::to_string(_currentSequence)));
@ -549,29 +528,26 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
_callback(vocbase, _builder.slice());
_responseSize += _builder.size();
_builder.clear();
_seenBeginTransaction = false;
}
_state = INVALID;
}
// should reset state flags which are only valid between
// observing a specific log entry and a sequence of immediately
// following PUT / DELETE / Log entries
void resetTransientState() {
if (_seenBeginTransaction) {
writeCommitMarker();
if (_state == TRANSACTION) {
writeCommitMarker(_trxDbId);
}
// reset all states
_lastLogType = RocksDBLogType::Invalid;
_seenBeginTransaction = false;
_singleOp = false;
_state = INVALID;
_currentTrxId = 0;
_currentDbId = 0;
_currentCid = 0;
_removeDocumentKey.clear();
_trxDbId = 0;
_removedDocRid = 0;
}
uint64_t endBatch() {
TRI_ASSERT(_removeDocumentKey.empty());
TRI_ASSERT(_removedDocRid == 0);
resetTransientState();
return _currentSequence;
}
@ -591,77 +567,20 @@ class MyWALParser : public rocksdb::WriteBatch::Handler,
}
}
bool shouldHandleMarker(uint32_t column_family_id,
bool isPut, rocksdb::Slice const& key) {
TRI_voc_tick_t dbId = 0;
TRI_voc_cid_t cid = 0;
if (column_family_id == _definitionsCF) {
// only a PUT should be handled here anyway
if (RocksDBKey::type(key) == RocksDBEntryType::Database) {
return isPut && shouldHandleDB(RocksDBKey::databaseId(key));
} else if (RocksDBKey::type(key) == RocksDBEntryType::Collection) {
// || RocksDBKey::type(key) == RocksDBEntryType::View
dbId = RocksDBKey::databaseId(key);
cid = RocksDBKey::collectionId(key);
if (!isPut || dbId == 0 || cid == 0) {
// FIXME: seems broken to get a key with zero entries here
return false;
}
} else {
return false;
}
} else if (column_family_id == _documentsCF) {
dbId = _currentDbId;
cid = _currentCid;
// happens when dropping a collection or log markers
// are ignored for dbs and collections
if (!(_seenBeginTransaction || _singleOp)) {
return false;
}
} else {
return false;
}
if (!shouldHandleCollection(dbId, cid)) {
return false;
}
if (_lastLogType != RocksDBLogType::CollectionDrop) {
// no document removes of dropped collections / vocbases
TRI_vocbase_t* vocbase = loadVocbase(dbId);
if (vocbase == nullptr) {
return false;
}
std::string const collectionName = vocbase->collectionName(cid);
if (collectionName.empty()) {
return false;
}
if (!_filter.includeSystem && collectionName[0] == '_') {
return false;
}
if (TRI_ExcludeCollectionReplication(collectionName,
_filter.includeSystem)) {
return false;
}
}
return true;
}
private:
uint32_t const _documentsCF;
uint32_t const _definitionsCF;
uint32_t const _documentsCF;
uint32_t const _primaryCF;
rocksdb::SequenceNumber _startSequence;
rocksdb::SequenceNumber _currentSequence;
RocksDBLogType _lastLogType = RocksDBLogType::Invalid;
bool _startOfBatch = false;
// Various state machine flags
bool _seenBeginTransaction = false;
bool _singleOp = false;
bool _startOfBatch = false;
State _state = INVALID;
TRI_voc_tick_t _currentTrxId = 0;
TRI_voc_tick_t _currentDbId = 0;
TRI_voc_cid_t _currentCid = 0;
std::string _removeDocumentKey;
TRI_voc_tick_t _trxDbId = 0; // remove eventually
TRI_voc_rid_t _removedDocRid = 0;
};
// iterates over WAL starting at 'from' and returns up to 'chunkSize' documents

View File

@ -98,7 +98,7 @@ TRI_voc_rid_t PhysicalCollection::newRevisionId() const {
/// _key and _id attributes
void PhysicalCollection::mergeObjectsForUpdate(
transaction::Methods* trx, VPackSlice const& oldValue,
VPackSlice const& newValue, bool isEdgeCollection, LocalDocumentId const& documentId,
VPackSlice const& newValue, bool isEdgeCollection,
bool mergeObjects, bool keepNull, VPackBuilder& b, bool isRestore, TRI_voc_rid_t& revisionId) const {
b.openObject();
@ -243,8 +243,8 @@ void PhysicalCollection::mergeObjectsForUpdate(
int PhysicalCollection::newObjectForInsert(
transaction::Methods* trx, VPackSlice const& value,
VPackSlice const& fromSlice, VPackSlice const& toSlice,
LocalDocumentId const& documentId, bool isEdgeCollection,
VPackBuilder& builder, bool isRestore, TRI_voc_rid_t& revisionId) const {
bool isEdgeCollection, VPackBuilder& builder, bool isRestore,
TRI_voc_rid_t& revisionId) const {
builder.openObject();
// add system attributes first, in this order:
@ -327,7 +327,6 @@ int PhysicalCollection::newObjectForInsert(
/// @brief new object for remove, must have _key set
void PhysicalCollection::newObjectForRemove(transaction::Methods* trx,
VPackSlice const& oldValue,
LocalDocumentId const& documentId,
VPackBuilder& builder,
bool isRestore, TRI_voc_rid_t& revisionId) const {
// create an object consisting of _key and _rev (in this order)
@ -349,7 +348,7 @@ void PhysicalCollection::newObjectForRemove(transaction::Methods* trx,
void PhysicalCollection::newObjectForReplace(
transaction::Methods* trx, VPackSlice const& oldValue,
VPackSlice const& newValue, VPackSlice const& fromSlice,
VPackSlice const& toSlice, bool isEdgeCollection, LocalDocumentId const& documentId,
VPackSlice const& toSlice, bool isEdgeCollection,
VPackBuilder& builder, bool isRestore, TRI_voc_rid_t& revisionId) const {
builder.openObject();

View File

@ -140,7 +140,7 @@ class PhysicalCollection {
OperationOptions& options) = 0;
virtual LocalDocumentId lookupKey(
transaction::Methods*, arangodb::velocypack::Slice const&) = 0;
transaction::Methods*, arangodb::velocypack::Slice const&) const = 0;
virtual Result read(transaction::Methods*,
arangodb::StringRef const& key,
@ -221,7 +221,6 @@ class PhysicalCollection {
velocypack::Slice const& value,
velocypack::Slice const& fromSlice,
velocypack::Slice const& toSlice,
LocalDocumentId const& documentId,
bool isEdgeCollection, velocypack::Builder& builder,
bool isRestore,
TRI_voc_rid_t& revisionId) const;
@ -229,7 +228,6 @@ class PhysicalCollection {
/// @brief new object for remove, must have _key set
void newObjectForRemove(transaction::Methods* trx,
velocypack::Slice const& oldValue,
LocalDocumentId const& documentId,
velocypack::Builder& builder,
bool isRestore, TRI_voc_rid_t& revisionId) const;
@ -237,7 +235,7 @@ class PhysicalCollection {
void mergeObjectsForUpdate(transaction::Methods* trx,
velocypack::Slice const& oldValue,
velocypack::Slice const& newValue,
bool isEdgeCollection, LocalDocumentId const& documentId,
bool isEdgeCollection,
bool mergeObjects, bool keepNull,
velocypack::Builder& builder,
bool isRestore, TRI_voc_rid_t& revisionId) const;
@ -248,7 +246,7 @@ class PhysicalCollection {
velocypack::Slice const& newValue,
velocypack::Slice const& fromSlice,
velocypack::Slice const& toSlice,
bool isEdgeCollection, LocalDocumentId const& documentId,
bool isEdgeCollection,
velocypack::Builder& builder,
bool isRestore, TRI_voc_rid_t& revisionId) const;

View File

@ -22,20 +22,33 @@
////////////////////////////////////////////////////////////////////////////////
#include "WalAccess.h"
#include "Replication/common-defines.h"
#include "RestServer/DatabaseFeature.h"
#include "VocBase/LogicalCollection.h"
using namespace arangodb;
/// @brief check if db should be handled, might already be deleted
bool WalAccessContext::shouldHandleDB(TRI_voc_tick_t dbid) const {
return _filter.vocbase == 0 || _filter.vocbase == dbid;
}
/// @brief Check if collection is in filter
/// @brief Check if collection is in filter, will load collection
bool WalAccessContext::shouldHandleCollection(TRI_voc_tick_t dbid,
TRI_voc_cid_t cid) const {
return _filter.vocbase == 0 || (_filter.vocbase == dbid &&
(_filter.collection == 0 || _filter.collection == cid));
TRI_voc_cid_t cid) {
if (dbid == 0 || cid == 0 || !shouldHandleDB(dbid)) {
return false;
}
if (_filter.vocbase == 0 || (_filter.vocbase == dbid &&
(_filter.collection == 0 || _filter.collection == cid))) {
LogicalCollection* collection = loadCollection(dbid, cid);
if (collection == nullptr) {
return false;
}
return !TRI_ExcludeCollectionReplication(collection->name(),
_filter.includeSystem);
}
return false;
}
/// @brief try to get collection, may return null

View File

@ -141,10 +141,12 @@ struct WalAccessContext {
~WalAccessContext() {}
/// @brief check if db should be handled, might already be deleted
bool shouldHandleDB(TRI_voc_tick_t dbid) const;
/// @brief Check if collection is in filter
bool shouldHandleCollection(TRI_voc_tick_t dbid, TRI_voc_cid_t cid) const;
/// @brief Check if collection is in filter, will load collection
/// and prevent deletion
bool shouldHandleCollection(TRI_voc_tick_t dbid, TRI_voc_cid_t cid);
/// @brief try to get collection, may return null
TRI_vocbase_t* loadVocbase(TRI_voc_tick_t dbid);

View File

@ -1,6 +1,51 @@
#!/bin/bash
params=("$@")
. `dirname $0`/cluster-run-common.sh
rm -rf cluster
if [ -d cluster-init ];then
echo "== creating cluster directory from existing cluster-init directory"
cp -a cluster-init cluster
else
echo "== creating fresh directory"
mkdir -p cluster || { echo "failed to create cluster directory"; exit 1; }
#if we want to restart we should probably store the parameters line wise
fi
case $OSTYPE in
darwin*)
lib="$PWD/scripts/cluster-run-common.sh"
;;
*)
lib="$(dirname $(readlink -f ${BASH_SOURCE[0]}))/cluster-run-common.sh"
;;
esac
if [[ -f "$lib" ]]; then
. "$lib"
else
echo "could not source $lib"
exit 1
fi
if [[ -f cluster/startup_parameters ]];then
string="$(< cluster/startup_parameters)"
if [[ -z "${params[@]}" ]]; then
params=( $string )
else
if ! [[ "$*" == "$string" ]]; then
echo "stored and given params do not match:"
echo "given: ${params[@]}"
echo "stored: $string"
fi
fi
else
#store parmeters
if [[ -n "${params[@]}" ]]; then
echo "${params[@]}" > cluster/startup_parameters
fi
fi
parse_args "${params[@]}"
if [ "$POOLSZ" == "" ] ; then
POOLSZ=$NRAGENTS
@ -13,6 +58,10 @@ else
fi
DEFAULT_REPLICATION=""
if [[ $NRAGENTS -le 0 ]]; then
echo "you need as least one agent currently you have $NRAGENTS"
exit 1
fi
printf "Starting agency ... \n"
printf " # agents: %s," "$NRAGENTS"

View File

@ -646,7 +646,6 @@ arangodb::Result PhysicalCollectionMock::insert(arangodb::transaction::Methods*
newSlice,
fromSlice,
toSlice,
arangodb::LocalDocumentId(),
isEdgeCollection,
builder,
options.isRestore,
@ -690,7 +689,7 @@ std::shared_ptr<arangodb::Index> PhysicalCollectionMock::lookupIndex(arangodb::v
return nullptr;
}
arangodb::LocalDocumentId PhysicalCollectionMock::lookupKey(arangodb::transaction::Methods*, arangodb::velocypack::Slice const&) {
arangodb::LocalDocumentId PhysicalCollectionMock::lookupKey(arangodb::transaction::Methods*, arangodb::velocypack::Slice const&) const {
before();
TRI_ASSERT(false);
return arangodb::LocalDocumentId();
@ -1379,4 +1378,4 @@ bool TransactionStateMock::hasFailedOperations() const {
// -----------------------------------------------------------------------------
// --SECTION-- END-OF-FILE
// -----------------------------------------------------------------------------
// -----------------------------------------------------------------------------

View File

@ -71,7 +71,7 @@ class PhysicalCollectionMock: public arangodb::PhysicalCollection {
virtual arangodb::Result insert(arangodb::transaction::Methods* trx, arangodb::velocypack::Slice const newSlice, arangodb::ManagedDocumentResult& result, arangodb::OperationOptions& options, TRI_voc_tick_t& resultMarkerTick, bool lock, TRI_voc_tick_t& revisionId) override;
virtual void invokeOnAllElements(arangodb::transaction::Methods* trx, std::function<bool(arangodb::LocalDocumentId const&)> callback) override;
virtual std::shared_ptr<arangodb::Index> lookupIndex(arangodb::velocypack::Slice const&) const override;
virtual arangodb::LocalDocumentId lookupKey(arangodb::transaction::Methods*, arangodb::velocypack::Slice const&) override;
virtual arangodb::LocalDocumentId lookupKey(arangodb::transaction::Methods*, arangodb::velocypack::Slice const&) const override;
virtual size_t memory() const override;
virtual uint64_t numberDocuments(arangodb::transaction::Methods* trx) const override;
virtual void open(bool ignoreErrors) override;

View File

@ -145,44 +145,44 @@ TEST_CASE("RocksDBKeyTest", "[rocksdbkeytest]") {
/// @brief test document
SECTION("test_document") {
RocksDBKey key;
key.constructDocument(0, 0);
key.constructDocument(0, LocalDocumentId(0));
auto const& s1 = key.string();
CHECK(s1.size() == +sizeof(uint64_t) + sizeof(uint64_t));
CHECK(s1 == std::string("\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0", 16));
key.constructDocument(23, 42);
key.constructDocument(23, LocalDocumentId(42));
auto const& s2 = key.string();
CHECK(s2.size() == +sizeof(uint64_t) + sizeof(uint64_t));
CHECK(s2 == std::string("\x17\0\0\0\0\0\0\0\x2a\0\0\0\0\0\0\0", 16));
key.constructDocument(255, 255);
key.constructDocument(255, LocalDocumentId(255));
auto const& s3 = key.string();
CHECK(s3.size() == sizeof(uint64_t) + sizeof(uint64_t));
CHECK(s3 == std::string("\xff\0\0\0\0\0\0\0\xff\0\0\0\0\0\0\0", 16));
key.constructDocument(256, 257);
key.constructDocument(256, LocalDocumentId(257));
auto const& s4 = key.string();
CHECK(s4.size() == sizeof(uint64_t) + sizeof(uint64_t));
CHECK(s4 == std::string("\0\x01\0\0\0\0\0\0\x01\x01\0\0\0\0\0\0", 16));
key.constructDocument(49152, 16384);
key.constructDocument(49152, LocalDocumentId(16384));
auto const& s5 = key.string();
CHECK(s5.size() == sizeof(uint64_t) + sizeof(uint64_t));
CHECK(s5 == std::string("\0\xc0\0\0\0\0\0\0\0\x40\0\0\0\0\0\0", 16));
key.constructDocument(12345678901, 987654321);
key.constructDocument(12345678901, LocalDocumentId(987654321));
auto const& s6 = key.string();
CHECK(s6.size() == sizeof(uint64_t) + sizeof(uint64_t));
CHECK(s6 == std::string(
"\x35\x1c\xdc\xdf\x02\0\0\0\xb1\x68\xde\x3a\0\0\0\0", 16));
key.constructDocument(0xf0f1f2f3f4f5f6f7ULL, 0xf0f1f2f3f4f5f6f7ULL);
key.constructDocument(0xf0f1f2f3f4f5f6f7ULL, LocalDocumentId(0xf0f1f2f3f4f5f6f7ULL));
auto const& s7 = key.string();
CHECK(s7.size() == sizeof(uint64_t) + sizeof(uint64_t));
@ -242,9 +242,9 @@ TEST_CASE("RocksDBKeyTest", "[rocksdbkeytest]") {
/// @brief test edge index
SECTION("test_edge_index") {
RocksDBKey key1;
key1.constructEdgeIndexValue(0, StringRef("a/1"), 33);
key1.constructEdgeIndexValue(0, StringRef("a/1"), LocalDocumentId(33));
RocksDBKey key2;
key2.constructEdgeIndexValue(0, StringRef("b/1"), 33);
key2.constructEdgeIndexValue(0, StringRef("b/1"), LocalDocumentId(33));
auto const& s1 = key1.string();
CHECK(s1.size() ==
@ -295,7 +295,7 @@ TEST_CASE("RocksDBKeyBoundsTest", "[rocksdbkeybounds]") {
/// @brief test edge index with dynamic prefix extractor
SECTION("test_edge_index") {
RocksDBKey key1;
key1.constructEdgeIndexValue(0, StringRef("a/1"), 33);
key1.constructEdgeIndexValue(0, StringRef("a/1"), LocalDocumentId(33));
// check the variable length edge prefix
auto pe = std::make_unique<RocksDBPrefixExtractor>();
REQUIRE(pe->InDomain(key1.string()));
@ -320,13 +320,13 @@ TEST_CASE("RocksDBKeyBoundsTest", "[rocksdbkeybounds]") {
CHECK(cmp->Compare(prefixEnd, key1.string()) > 0);
RocksDBKey key2;
key2.constructEdgeIndexValue(0, StringRef("c/1000"), 33);
key2.constructEdgeIndexValue(0, StringRef("c/1000"), LocalDocumentId(33));
CHECK(cmp->Compare(prefixBegin, key2.string()) < 0);
CHECK(cmp->Compare(prefixEnd, key2.string()) > 0);
// test higher prefix
RocksDBKey key3;
key3.constructEdgeIndexValue(1, StringRef("c/1000"), 33);
key3.constructEdgeIndexValue(1, StringRef("c/1000"), LocalDocumentId(33));
CHECK(cmp->Compare(prefixBegin, key3.string()) < 0);
CHECK(cmp->Compare(prefixEnd, key3.string()) < 0);
}
@ -341,9 +341,9 @@ TEST_CASE("RocksDBKeyBoundsTest", "[rocksdbkeybounds]") {
higher(VPackValue(VPackValueType::Array))(VPackValue("b"))();
RocksDBKey key1, key2, key3;
key1.constructVPackIndexValue(1, lower.slice(), 33);
key2.constructVPackIndexValue(1, higher.slice(), 33);
key3.constructVPackIndexValue(2, lower.slice(), 16);
key1.constructVPackIndexValue(1, lower.slice(), LocalDocumentId(33));
key2.constructVPackIndexValue(1, higher.slice(), LocalDocumentId(33));
key3.constructVPackIndexValue(2, lower.slice(), LocalDocumentId(16));
// check the variable length edge prefix
std::unique_ptr<rocksdb::SliceTransform const> pe(rocksdb::NewFixedPrefixTransform(RocksDBKey::objectIdSize()));
@ -385,10 +385,10 @@ TEST_CASE("RocksDBKeyBoundsTest", "[rocksdbkeybounds]") {
c(VPackValue(VPackValueType::Array))(VPackValue(5))();
RocksDBKey key4, key5, key6, key7;
key4.constructVPackIndexValue(1, a.slice(), 18);
key5.constructVPackIndexValue(1, b.slice(), 60);
key6.constructVPackIndexValue(1, b.slice(), 90);
key7.constructVPackIndexValue(1, c.slice(), 12);
key4.constructVPackIndexValue(1, a.slice(), LocalDocumentId(18));
key5.constructVPackIndexValue(1, b.slice(), LocalDocumentId(60));
key6.constructVPackIndexValue(1, b.slice(), LocalDocumentId(90));
key7.constructVPackIndexValue(1, c.slice(), LocalDocumentId(12));
bounds = RocksDBKeyBounds::VPackIndex(1, a.slice(), c.slice());
CHECK(cmp->Compare(bounds.start(), key4.string()) < 0);