1
0
Fork 0

Pregel segmented buffers (#9109)

This commit is contained in:
Simon 2019-05-28 18:21:56 +02:00 committed by Jan
parent 7a240e1dc4
commit 87977d1228
32 changed files with 872 additions and 1153 deletions

View File

@ -1,6 +1,9 @@
v3.4.7 (2019-XX-XX)
-------------------
* Pregel algorithms can be run with the option "useMemoryMaps: true" to be able to run algorithms
on data that is bigger than the available RAM.
* Bugfix for smart graph traversals with uniqueVertices: path, which could
sometimes lead to erroneous traversal results

View File

@ -339,7 +339,6 @@ SET(ARANGOD_SOURCES
Graph/BreadthFirstEnumerator.cpp
Graph/ConstantWeightShortestPathFinder.cpp
Graph/ClusterTraverserCache.cpp
Graph/EdgeCollectionInfo.cpp
Graph/Graph.cpp
Graph/GraphManager.cpp
Graph/GraphOperations.cpp
@ -380,6 +379,7 @@ SET(ARANGOD_SOURCES
Pregel/Conductor.cpp
Pregel/GraphStore.cpp
Pregel/IncomingCache.cpp
Pregel/IndexHelpers.cpp
Pregel/OutgoingCache.cpp
Pregel/PregelFeature.cpp
Pregel/Recovery.cpp

View File

@ -1,172 +0,0 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2016 ArangoDB GmbH, Cologne, Germany
///
/// Licensed under the Apache License, Version 2.0 (the "License");
/// you may not use this file except in compliance with the License.
/// You may obtain a copy of the License at
///
/// http://www.apache.org/licenses/LICENSE-2.0
///
/// Unless required by applicable law or agreed to in writing, software
/// distributed under the License is distributed on an "AS IS" BASIS,
/// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
/// See the License for the specific language governing permissions and
/// limitations under the License.
///
/// Copyright holder is ArangoDB GmbH, Cologne, Germany
///
/// @author Michael Hackstein
////////////////////////////////////////////////////////////////////////////////
#include "EdgeCollectionInfo.h"
#include "Cluster/ClusterMethods.h"
#include "Transaction/Methods.h"
#include "Utils/OperationCursor.h"
using namespace arangodb;
using namespace arangodb::traverser;
EdgeCollectionInfo::EdgeCollectionInfo(transaction::Methods* trx,
std::string const& collectionName,
TRI_edge_direction_e const direction,
std::string const& weightAttribute, double defaultWeight)
: _trx(trx),
_collectionName(collectionName),
_searchBuilder(),
_weightAttribute(weightAttribute),
_defaultWeight(defaultWeight),
_dir(direction) {
TRI_ASSERT(_dir == TRI_EDGE_OUT || _dir == TRI_EDGE_IN);
if (!trx->isEdgeCollection(collectionName)) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_ARANGO_COLLECTION_TYPE_INVALID);
}
auto var = _searchBuilder.getVariable();
if (_dir == TRI_EDGE_OUT) {
auto cond = _searchBuilder.getOutboundCondition();
bool worked = _trx->getBestIndexHandleForFilterCondition(_collectionName, cond,
var, 1000, _forwardIndexId);
TRI_ASSERT(worked); // We always have an edge Index
cond = _searchBuilder.getInboundCondition();
worked = _trx->getBestIndexHandleForFilterCondition(_collectionName, cond, var,
1000, _backwardIndexId);
TRI_ASSERT(worked); // We always have an edge Index
} else {
auto cond = _searchBuilder.getInboundCondition();
bool worked = _trx->getBestIndexHandleForFilterCondition(_collectionName, cond,
var, 1000, _forwardIndexId);
TRI_ASSERT(worked); // We always have an edge Index
cond = _searchBuilder.getOutboundCondition();
worked = _trx->getBestIndexHandleForFilterCondition(_collectionName, cond, var,
1000, _backwardIndexId);
TRI_ASSERT(worked); // We always have an edge Index
}
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex.
////////////////////////////////////////////////////////////////////////////////
std::unique_ptr<arangodb::OperationCursor> EdgeCollectionInfo::getEdges(
std::string const& vertexId, arangodb::ManagedDocumentResult* mmdr) {
_searchBuilder.setVertexId(vertexId);
std::unique_ptr<arangodb::OperationCursor> res;
IndexIteratorOptions opts;
opts.enableCache = false;
if (_dir == TRI_EDGE_OUT) {
res.reset(_trx->indexScanForCondition(_forwardIndexId,
_searchBuilder.getOutboundCondition(),
_searchBuilder.getVariable(), mmdr, opts));
} else {
res.reset(_trx->indexScanForCondition(_forwardIndexId, _searchBuilder.getInboundCondition(),
_searchBuilder.getVariable(), mmdr, opts));
}
return res;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex. On Coordinator.
////////////////////////////////////////////////////////////////////////////////
int EdgeCollectionInfo::getEdgesCoordinator(VPackSlice const& vertexId, VPackBuilder& result) {
TRI_ASSERT(result.isEmpty());
arangodb::rest::ResponseCode responseCode;
result.openObject();
int res = getFilteredEdgesOnCoordinator(_trx->vocbase().name(), _collectionName,
*_trx, vertexId.copyString(), _dir,
responseCode, result);
result.close();
return res;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex. Reverse version
////////////////////////////////////////////////////////////////////////////////
std::unique_ptr<arangodb::OperationCursor> EdgeCollectionInfo::getReverseEdges(
std::string const& vertexId, arangodb::ManagedDocumentResult* mmdr) {
_searchBuilder.setVertexId(vertexId);
std::unique_ptr<arangodb::OperationCursor> res;
IndexIteratorOptions opts;
if (_dir == TRI_EDGE_OUT) {
res.reset(_trx->indexScanForCondition(_backwardIndexId,
_searchBuilder.getInboundCondition(),
_searchBuilder.getVariable(), mmdr, opts));
} else {
res.reset(_trx->indexScanForCondition(_backwardIndexId,
_searchBuilder.getOutboundCondition(),
_searchBuilder.getVariable(), mmdr, opts));
}
return res;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex. Reverse version on Coordinator.
////////////////////////////////////////////////////////////////////////////////
int EdgeCollectionInfo::getReverseEdgesCoordinator(VPackSlice const& vertexId,
VPackBuilder& result) {
TRI_ASSERT(result.isEmpty());
arangodb::rest::ResponseCode responseCode;
result.openObject();
TRI_edge_direction_e dir = TRI_EDGE_OUT;
if (_dir == TRI_EDGE_OUT) {
dir = TRI_EDGE_IN;
}
int res = getFilteredEdgesOnCoordinator(_trx->vocbase().name(), _collectionName,
*_trx, vertexId.copyString(), dir,
responseCode, result);
result.close();
return res;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Compute the weight of an edge
////////////////////////////////////////////////////////////////////////////////
double EdgeCollectionInfo::weightEdge(VPackSlice const edge) {
TRI_ASSERT(!_weightAttribute.empty());
return arangodb::basics::VelocyPackHelper::getNumericValue<double>(
edge, _weightAttribute.c_str(), _defaultWeight);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Return name of the wrapped collection
////////////////////////////////////////////////////////////////////////////////
std::string const& EdgeCollectionInfo::getName() const {
return _collectionName;
}

View File

@ -46,7 +46,7 @@ enum SCCPhase {
BACKWARD_TRAVERSAL_REST = 4
};
struct ASCCComputation
struct ASCCComputation final
: public VertexComputation<SCCValue, int8_t, SenderMessage<uint64_t>> {
ASCCComputation() {}
@ -166,17 +166,12 @@ struct SCCGraphFormat : public GraphFormat<SCCValue, int8_t> {
size_t estimatedEdgeSize() const override { return 0; };
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
SCCValue* targetPtr, size_t maxSize) override {
SCCValue* senders = (SCCValue*)targetPtr;
senders->vertexID = vertexIdRange++;
return sizeof(SCCValue);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
SCCValue& targetPtr) override {
targetPtr.vertexID = vertexIdRange++;
}
size_t copyEdgeData(arangodb::velocypack::Slice document, int8_t* targetPtr,
size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, int8_t& targetPtr) override {}
bool buildVertexDocument(arangodb::velocypack::Builder& b,
const SCCValue* ptr, size_t size) const override {

View File

@ -58,7 +58,7 @@ VertexComputation<int64_t, int64_t, int64_t>* ConnectedComponents::createComputa
return new MyComputation();
}
struct MyGraphFormat : public VertexGraphFormat<int64_t, int64_t> {
struct MyGraphFormat final : public VertexGraphFormat<int64_t, int64_t> {
uint64_t vertexIdRange = 0;
explicit MyGraphFormat(std::string const& result)
@ -74,10 +74,9 @@ struct MyGraphFormat : public VertexGraphFormat<int64_t, int64_t> {
}
}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
int64_t* targetPtr, size_t maxSize) override {
*targetPtr = vertexIdRange++;
return sizeof(int64_t);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
int64_t& targetPtr) override {
targetPtr = vertexIdRange++;
}
};

View File

@ -157,8 +157,9 @@ struct DMIDComputation : public VertexComputation<DMIDValue, float, DMIDMessage>
void superstep0(MessageIterator<DMIDMessage> const& messages) {
DMIDMessage message(pregelId(), 0);
RangeIterator<Edge<float>> edges = getEdges();
for (Edge<float>* edge : edges) {
message.weight = *edge->data(); // edge weight
for(; edges.hasMore(); ++edges) {
Edge<float>* edge = *edges;
message.weight = edge->data(); // edge weight
sendMessage(edge, message);
}
}
@ -230,7 +231,7 @@ struct DMIDComputation : public VertexComputation<DMIDValue, float, DMIDMessage>
* */
VertexSumAggregator* agg = (VertexSumAggregator*)getWriteAggregator(DA_AGG);
agg->aggregate(this->shard(), this->key(), 1.0 / context()->vertexCount());
agg->aggregate(this->shard(), this->key().toString(), 1.0 / context()->vertexCount());
// DoubleDenseVector init = new DoubleDenseVector(
// (int)
// getTotalNumVertices());
@ -264,7 +265,7 @@ struct DMIDComputation : public VertexComputation<DMIDValue, float, DMIDMessage>
}
});
VertexSumAggregator* newDA = (VertexSumAggregator*)getWriteAggregator(DA_AGG);
newDA->aggregate(this->shard(), this->key(), newEntryDA);
newDA->aggregate(this->shard(), this->key().toString(), newEntryDA);
}
/**
@ -278,10 +279,10 @@ struct DMIDComputation : public VertexComputation<DMIDValue, float, DMIDMessage>
// DoubleDenseVector finalDA = getAggregatedValue(DA_AGG);
// vertex.getValue().getWeightedInDegree();
double weightedInDegree = vertexState->weightedInDegree;
double lsAggValue = finalDA->getAggregatedValue(shard(), key()) * weightedInDegree;
double lsAggValue = finalDA->getAggregatedValue(shard(), key().toString()) * weightedInDegree;
VertexSumAggregator* tmpLS = (VertexSumAggregator*)getWriteAggregator(LS_AGG);
tmpLS->aggregate(this->shard(), this->key(), lsAggValue);
tmpLS->aggregate(this->shard(), this->key().toString(), lsAggValue);
// finalDA->aggregateValue(shard(), key(), );
// int vertexID = (int) vertex.getId().get();
@ -308,14 +309,17 @@ struct DMIDComputation : public VertexComputation<DMIDValue, float, DMIDMessage>
float senderWeight = message->weight;
float myInfluence = (float)vecLS->getAggregatedValue(this->shard(), this->key());
float myInfluence = (float)vecLS->getAggregatedValue(this->shard(), this->key().toString());
myInfluence *= senderWeight;
/**
* hasEdgeToSender determines if sender has influence on this vertex
*/
bool hasEdgeToSender = false;
for (Edge<float>* edge : getEdges()) {
for (auto edges = getEdges(); edges.hasMore(); ++edges) {
Edge<float>* edge = *edges;
if (edge->targetShard() == senderID.shard && edge->toKey() == senderID.key) {
hasEdgeToSender = true;
/**
@ -324,7 +328,7 @@ struct DMIDComputation : public VertexComputation<DMIDValue, float, DMIDMessage>
*/
float senderInfluence =
(float)vecLS->getAggregatedValue(senderID.shard, senderID.key);
senderInfluence *= *(edge->data());
senderInfluence *= edge->data();
if (myInfluence > senderInfluence) {
/** send new message */
@ -588,17 +592,14 @@ struct DMIDGraphFormat : public GraphFormat<DMIDValue, float> {
}
}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
DMIDValue* value, size_t maxSize) override {
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
DMIDValue& value) override {
// SCCValue* senders = (SCCValue*)targetPtr;
// senders->vertexID = vertexIdRange++;
return sizeof(SCCValue);
}
size_t copyEdgeData(arangodb::velocypack::Slice document, float* targetPtr,
size_t maxSize) override {
*targetPtr = 1.0f;
return sizeof(float);
void copyEdgeData(arangodb::velocypack::Slice document, float& targetPtr) override {
targetPtr = 1.0f;
}
bool buildVertexDocument(arangodb::velocypack::Builder& b,

View File

@ -35,8 +35,8 @@ namespace arangodb {
namespace pregel {
struct VertexSumAggregator : public IAggregator {
typedef std::map<PregelShard, std::unordered_map<PregelKey, double>> VertexMap;
typedef std::pair<PregelShard, std::unordered_map<PregelKey, double>> MyPair;
typedef std::map<PregelShard, std::unordered_map<std::string, double>> VertexMap;
typedef std::pair<PregelShard, std::unordered_map<std::string, double>> MyPair;
VertexSumAggregator(bool perm = false) : _permanent(perm) {}
@ -53,7 +53,7 @@ struct VertexSumAggregator : public IAggregator {
void parseAggregate(VPackSlice const& slice) override {
for (auto const& pair : VPackObjectIterator(slice)) {
PregelShard shard = std::stoi(pair.key.copyString());
PregelKey key;
std::string key;
VPackValueLength i = 0;
for (VPackSlice const& val : VPackArrayIterator(pair.value)) {
if (i % 2 == 0) {
@ -71,7 +71,7 @@ struct VertexSumAggregator : public IAggregator {
void setAggregatedValue(VPackSlice const& slice) override {
for (auto const& pair : VPackObjectIterator(slice)) {
PregelShard shard = std::stoi(pair.key.copyString());
PregelKey key;
std::string key;
VPackValueLength i = 0;
for (VPackSlice const& val : VPackArrayIterator(pair.value)) {
if (i % 2 == 0) {
@ -104,7 +104,7 @@ struct VertexSumAggregator : public IAggregator {
}
}
double getAggregatedValue(PregelShard shard, PregelKey const& key) const {
double getAggregatedValue(PregelShard shard, std::string const& key) const {
auto const& it1 = _entries.find(shard);
if (it1 != _entries.end()) {
auto const& it2 = it1->second.find(key);
@ -119,7 +119,7 @@ struct VertexSumAggregator : public IAggregator {
// _entries[shard][key] = val;
//}
void aggregate(PregelShard shard, PregelKey const& key, double val) {
void aggregate(PregelShard shard, std::string const& key, double val) {
_entries[shard][key] += val;
}
@ -128,8 +128,8 @@ struct VertexSumAggregator : public IAggregator {
void forEach(std::function<void(PregelID const& _id, double value)> func) const {
for (auto const& pair : _entries) {
PregelShard shard = pair.first;
std::unordered_map<PregelKey, double> const& vertexMap = pair.second;
for (std::pair<PregelKey, double> const& vertexMessage : vertexMap) {
std::unordered_map<std::string, double> const& vertexMap = pair.second;
for (std::pair<std::string, double> const& vertexMessage : vertexMap) {
func(PregelID(shard, vertexMessage.first), vertexMessage.second);
}
}

View File

@ -94,15 +94,10 @@ struct ECGraphFormat : public GraphFormat<ECValue, int8_t> {
size_t estimatedEdgeSize() const override { return 0; };
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
ECValue* targetPtr, size_t maxSize) override {
return sizeof(ECValue);
}
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
ECValue& targetPtr) override {}
size_t copyEdgeData(arangodb::velocypack::Slice document, int8_t* targetPtr,
size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, int8_t& targetPtr) override {}
bool buildVertexDocument(arangodb::velocypack::Builder& b, const ECValue* ptr,
size_t size) const override {

View File

@ -106,15 +106,10 @@ struct HITSGraphFormat : public GraphFormat<HITSValue, int8_t> {
size_t estimatedEdgeSize() const override { return 0; };
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
HITSValue* targetPtr, size_t maxSize) override {
return sizeof(HITSValue);
}
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
HITSValue& targetPtr) override {}
size_t copyEdgeData(arangodb::velocypack::Slice document, int8_t* targetPtr,
size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, int8_t& targetPtr) override {}
bool buildVertexDocument(arangodb::velocypack::Builder& b,
const HITSValue* value, size_t size) const override {

View File

@ -129,16 +129,12 @@ struct LPGraphFormat : public GraphFormat<LPValue, int8_t> {
}
}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
LPValue* value, size_t maxSize) override {
value->currentCommunity = vertexIdRange++;
return sizeof(LPValue);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
LPValue& value) override {
value.currentCommunity = vertexIdRange++;
}
size_t copyEdgeData(arangodb::velocypack::Slice document, int8_t* targetPtr,
size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, int8_t& targetPtr) override {}
bool buildVertexDocument(arangodb::velocypack::Builder& b, const LPValue* ptr,
size_t size) const override {

View File

@ -56,9 +56,7 @@ struct SeededPRGraphFormat final : public NumberGraphFormat<float, float> {
SeededPRGraphFormat(std::string const& source, std::string const& result, float vertexNull)
: NumberGraphFormat(source, result, vertexNull, 0.0f) {}
size_t copyEdgeData(arangodb::velocypack::Slice document, float*, size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, float&) override {}
bool buildEdgeDocument(arangodb::velocypack::Builder& b, float const*,
size_t size) const override {
return false;

View File

@ -162,17 +162,12 @@ struct SCCGraphFormat : public GraphFormat<SCCValue, int8_t> {
size_t estimatedEdgeSize() const override { return 0; };
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
SCCValue* targetPtr, size_t maxSize) override {
SCCValue* senders = (SCCValue*)targetPtr;
senders->vertexID = vertexIdRange++;
return sizeof(SCCValue);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
SCCValue& senders) override {
senders.vertexID = vertexIdRange++;
}
size_t copyEdgeData(arangodb::velocypack::Slice document, int8_t* targetPtr,
size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, int8_t& targetPtr) override {}
bool buildVertexDocument(arangodb::velocypack::Builder& b,
const SCCValue* ptr, size_t size) const override {

View File

@ -152,15 +152,12 @@ struct SLPAGraphFormat : public GraphFormat<SLPAValue, int8_t> {
}
}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
SLPAValue* value, size_t maxSize) override {
value->nodeId = (uint32_t)vertexIdRange++;
return sizeof(SLPAValue);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
SLPAValue& value) override {
value.nodeId = (uint32_t)vertexIdRange++;
}
size_t copyEdgeData(arangodb::velocypack::Slice document, int8_t* targetPtr,
size_t maxSize) override {
return 0;
void copyEdgeData(arangodb::velocypack::Slice document, int8_t& targetPtr) override {
}
bool buildVertexDocument(arangodb::velocypack::Builder& b,

View File

@ -43,8 +43,9 @@ struct SSSPComputation : public VertexComputation<int64_t, int64_t, int64_t> {
*state = tmp; // update state
RangeIterator<Edge<int64_t>> edges = getEdges();
for (Edge<int64_t>* edge : edges) {
int64_t val = *edge->data() + tmp;
for (; edges.hasMore(); ++edges) {
Edge<int64_t>* edge = *edges;
int64_t val = edge->data() + tmp;
sendMessage(edge, val);
}
}
@ -75,10 +76,9 @@ struct SSSPGraphFormat : public InitGraphFormat<int64_t, int64_t> {
SSSPGraphFormat(std::string const& source, std::string const& result)
: InitGraphFormat<int64_t, int64_t>(result, 0, 1), _sourceDocId(source) {}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
int64_t* targetPtr, size_t maxSize) override {
*targetPtr = documentId == _sourceDocId ? 0 : INT64_MAX;
return sizeof(int64_t);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
int64_t& targetPtr) override {
targetPtr = (documentId == _sourceDocId) ? 0 : INT64_MAX;
}
bool buildEdgeDocument(arangodb::velocypack::Builder& b,

View File

@ -63,8 +63,9 @@ struct SPComputation : public VertexComputation<int64_t, int64_t, int64_t> {
}
RangeIterator<Edge<int64_t>> edges = getEdges();
for (Edge<int64_t>* edge : edges) {
int64_t val = *edge->data() + current;
for (; edges.hasMore(); ++edges) {
Edge<int64_t>* edge = *edges;
int64_t val = edge->data() + current;
if (val < *max) {
sendMessage(edge, val);
}
@ -84,10 +85,9 @@ struct arangodb::pregel::algos::SPGraphFormat : public InitGraphFormat<int64_t,
_sourceDocId(source),
_targetDocId(target) {}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
int64_t* targetPtr, size_t maxSize) override {
*targetPtr = documentId == _sourceDocId ? 0 : INT64_MAX;
return sizeof(int64_t);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
int64_t& targetPtr) override {
targetPtr = (documentId == _sourceDocId) ? 0 : INT64_MAX;
}
bool buildEdgeDocument(arangodb::velocypack::Builder& b,

View File

@ -32,29 +32,28 @@
namespace arangodb {
namespace pregel {
typedef std::string PregelKey;
// typedef uint64_t PregelKey;
typedef uint16_t PregelShard;
const PregelShard InvalidPregelShard = -1;
struct PregelID {
PregelShard shard;
PregelKey key;
std::string key; // std::string 24
PregelShard shard; // uint16_t
PregelID() : shard(InvalidPregelShard), key("") {}
PregelID(PregelShard s, PregelKey const& k) : shard(s), key(k) {}
PregelID() : key(""), shard(InvalidPregelShard) {}
PregelID(PregelShard s, std::string const& k) : key(k), shard(s) {}
// PregelID(PregelShard s, std::string const& k) : shard(s),
// key(std::stoull(k)) {}
inline bool operator==(const PregelID& rhs) const {
bool operator==(const PregelID& rhs) const {
return shard == rhs.shard && key == rhs.key;
}
inline bool operator<(const PregelID& rhs) const {
bool operator<(const PregelID& rhs) const {
return shard < rhs.shard || (shard == rhs.shard && key < rhs.key);
}
bool inline isValid() const {
bool isValid() const {
return shard != InvalidPregelShard && !key.empty();
}
};
@ -68,109 +67,58 @@ class Edge {
template <typename V, typename E2>
friend class GraphStore;
// PregelShard _sourceShard;
PregelShard _targetShard;
PregelKey _toKey;
static_assert(sizeof(std::string) > 2, "");
char* _toKey; // uint64_t
uint16_t _toKeyLength; // uint16_t
PregelShard _targetShard; // uint16_t
E _data;
public:
// EdgeEntry() : _nextEntryOffset(0), _dataSize(0), _vertexIDSize(0) {}
Edge() : _targetShard(InvalidPregelShard), _data(0) {}
Edge(PregelShard target, PregelKey const& key)
: _targetShard(target), _toKey(key), _data(0) {}
// size_t getSize() { return sizeof(EdgeEntry) + _vertexIDSize + _dataSize; }
PregelKey const& toKey() const { return _toKey; }
StringRef toKey() const { return StringRef(_toKey, _toKeyLength); }
// size_t getDataSize() { return _dataSize; }
inline E* data() {
return &_data; // static_cast<E>(this + sizeof(EdgeEntry) + _vertexIDSize);
E& data() {
return _data; // static_cast<E>(this + sizeof(EdgeEntry) + _vertexIDSize);
}
// inline PregelShard sourceShard() const { return _sourceShard; }
inline PregelShard targetShard() const { return _targetShard; }
// PregelShard sourceShard() const { return _sourceShard; }
PregelShard targetShard() const { return _targetShard; }
};
class VertexEntry {
template <typename V, typename E>
friend class GraphStore;
template <typename V, typename E>
class Vertex {
friend class GraphStore<V,E>;
const char* _key; // uint64_t
Edge<E>* _edges; // uint64_t
size_t _edgeCount; // uint64_t
uint16_t _keyLength; // uint16_t
PregelShard _shard; // uint16_t
bool _active = true; // bool8_t
PregelShard _shard;
PregelKey _key;
size_t _vertexDataOffset = 0;
size_t _edgeDataOffset = 0;
size_t _edgeCount = 0;
bool _active = true;
V _data; // variable byte size
public:
VertexEntry() : _shard(InvalidPregelShard) {}
VertexEntry(PregelShard shard, PregelKey const& key)
: _shard(shard), _key(key) {}
Edge<E>* getEdges() const { return _edges; }
size_t getEdgeCount() const { return _edgeCount; }
bool active() const { return _active; }
void setActive(bool bb) { _active = bb; }
inline size_t getVertexDataOffset() const { return _vertexDataOffset; }
inline size_t getEdgeDataOffset() const { return _edgeDataOffset; }
inline size_t getEdgeCount() const { return _edgeCount; }
// inline size_t getSize() { return sizeof(VertexEntry) + _vertexIDSize; }
inline size_t getSize() { return sizeof(VertexEntry); }
inline bool active() const { return _active; }
inline void setActive(bool bb) { _active = bb; }
inline PregelShard shard() const { return _shard; }
inline PregelKey const& key() const { return _key; };
PregelID pregelId() const { return PregelID(_shard, _key); }
PregelShard shard() const { return _shard; }
StringRef key() const { return StringRef(_key, _keyLength); };
V const& data() const& { return _data; }
V& data() & { return _data; }
PregelID pregelId() const { return PregelID(_shard, std::string(_key, _keyLength)); }
/*std::string const& key() const {
return std::string(_key, _keySize);
};*/
};
// unused right now
/*class LinkedListIterator {
private:
intptr_t _begin, _end, _current;
VertexIterator(const VertexIterator&) = delete;
VertexIterator& operator=(const FileInfo&) = delete;
public:
typedef VertexIterator iterator;
typedef const VertexIterator const_iterator;
VertexIterator(intptr_t beginPtr, intptr_t endPtr)
: _begin(beginPtr), _end(endPtr), _current(beginPtr) {}
iterator begin() { return VertexIterator(_begin, _end); }
const_iterator begin() const { return VertexIterator(_begin, _end); }
iterator end() {
auto it = VertexIterator(_begin, _end);
it._current = it._end;
return it;
}
const_iterator end() const {
auto it = VertexIterator(_begin, _end);
it._current = it._end;
return it;
}
// prefix ++
VertexIterator& operator++() {
VertexEntry* entry = (VertexEntry*)_current;
_current += entry->getSize();
return *this;
}
// postfix ++
VertexIterator& operator++(int) {
VertexEntry* entry = (VertexEntry*)_current;
_current += entry->getSize();
return *this;
}
VertexEntry* operator*() const {
return _current != _end ? (VertexEntry*)_current : nullptr;
}
bool operator!=(VertexIterator const& other) const {
return _current != other._current;
}
};*/
} // namespace pregel
} // namespace arangodb
@ -185,9 +133,9 @@ struct hash<arangodb::pregel::PregelID> {
// Compute individual hash values for first,
// second and third and combine them using XOR
// and bit shifting:
size_t h1 = std::hash<arangodb::pregel::PregelKey>()(k.key);
size_t h1 = std::hash<std::string>()(k.key);
size_t h2 = std::hash<size_t>()(k.shard);
return h1 ^ (h2 << 1);
return h2 ^ (h1 << 1);
}
};
} // namespace std

View File

@ -46,11 +46,11 @@ struct GraphFormat {
/// This must not be called again before not all docs were loaded
virtual void willLoadVertices(uint64_t count) {}
virtual size_t copyVertexData(std::string const& documentId,
arangodb::velocypack::Slice document,
V* targetPtr, size_t maxSize) = 0;
virtual size_t copyEdgeData(arangodb::velocypack::Slice edgeDocument,
E* targetPtr, size_t maxSize) = 0;
virtual void copyVertexData(std::string const& documentId,
arangodb::velocypack::Slice document,
V& targetPtr) = 0;
virtual void copyEdgeData(arangodb::velocypack::Slice edgeDocument,
E& targetPtr) = 0;
virtual bool buildVertexDocument(arangodb::velocypack::Builder& b,
const V* targetPtr, size_t size) const = 0;
@ -76,33 +76,31 @@ class NumberGraphFormat : public GraphFormat<V, E> {
_vDefault(vertexNull),
_eDefault(edgeNull) {}
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
V* targetPtr, size_t maxSize) override {
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
V& targetPtr) override {
arangodb::velocypack::Slice val = document.get(_sourceField);
if (std::is_integral<V>::value) {
if (std::is_signed<V>::value) {
*targetPtr = val.isInteger() ? val.getInt() : _vDefault;
targetPtr = val.isInteger() ? val.getInt() : _vDefault;
} else {
*targetPtr = val.isInteger() ? val.getUInt() : _vDefault;
targetPtr = val.isInteger() ? val.getUInt() : _vDefault;
}
} else {
*targetPtr = val.isNumber() ? val.getNumber<V>() : _vDefault;
targetPtr = val.isNumber() ? val.getNumber<V>() : _vDefault;
}
return sizeof(V);
}
size_t copyEdgeData(arangodb::velocypack::Slice document, E* targetPtr, size_t maxSize) override {
void copyEdgeData(arangodb::velocypack::Slice document, E& targetPtr) override {
arangodb::velocypack::Slice val = document.get(_sourceField);
if (std::is_integral<E>::value) {
if (std::is_signed<E>::value) { // getNumber does range checks
*((E*)targetPtr) = val.isInteger() ? val.getInt() : _eDefault;
targetPtr = val.isInteger() ? val.getInt() : _eDefault;
} else {
*targetPtr = val.isInteger() ? val.getUInt() : _eDefault;
targetPtr = val.isInteger() ? val.getUInt() : _eDefault;
}
} else {
*targetPtr = val.isNumber() ? val.getNumber<E>() : _eDefault;
targetPtr = val.isNumber() ? val.getNumber<E>() : _eDefault;
}
return sizeof(E);
}
bool buildVertexDocument(arangodb::velocypack::Builder& b, const V* ptr,
@ -129,17 +127,15 @@ class InitGraphFormat : public GraphFormat<V, E> {
InitGraphFormat(std::string const& result, V vertexNull, E edgeNull)
: _resultField(result), _vDefault(vertexNull), _eDefault(edgeNull) {}
virtual size_t copyVertexData(std::string const& documentId,
arangodb::velocypack::Slice document,
V* targetPtr, size_t maxSize) override {
*(targetPtr) = _vDefault;
return sizeof(V);
virtual void copyVertexData(std::string const& documentId,
arangodb::velocypack::Slice document,
V& targetPtr) override {
targetPtr = _vDefault;
}
virtual size_t copyEdgeData(arangodb::velocypack::Slice document,
E* targetPtr, size_t maxSize) override {
*targetPtr = _eDefault;
return sizeof(E);
virtual void copyEdgeData(arangodb::velocypack::Slice document,
E& targetPtr) override {
targetPtr = _eDefault;
}
virtual bool buildVertexDocument(arangodb::velocypack::Builder& b,
@ -168,15 +164,12 @@ class VertexGraphFormat : public GraphFormat<V, E> {
size_t estimatedVertexSize() const override { return sizeof(V); };
size_t estimatedEdgeSize() const override { return 0; };
size_t copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
V* targetPtr, size_t maxSize) override {
*targetPtr = _vDefault;
return sizeof(V);
void copyVertexData(std::string const& documentId, arangodb::velocypack::Slice document,
V& targetPtr) override {
targetPtr = _vDefault;
}
size_t copyEdgeData(arangodb::velocypack::Slice document, V* targetPtr, size_t maxSize) override {
return 0;
}
void copyEdgeData(arangodb::velocypack::Slice document, E& targetPtr) override {}
bool buildVertexDocument(arangodb::velocypack::Builder& b, const V* ptr,
size_t size) const override {

View File

@ -24,8 +24,8 @@
#include "Basics/Common.h"
#include "Basics/MutexLocker.h"
#include "Graph/EdgeCollectionInfo.h"
#include "Pregel/CommonFormats.h"
#include "Pregel/IndexHelpers.h"
#include "Pregel/PregelFeature.h"
#include "Pregel/TypedBuffer.h"
#include "Pregel/Utils.h"
@ -33,6 +33,7 @@
#include "Scheduler/Scheduler.h"
#include "Scheduler/SchedulerFeature.h"
#include "Transaction/Context.h"
#include "Transaction/Helpers.h"
#include "Transaction/Methods.h"
#include "Transaction/StandaloneContext.h"
#include "Utils/CollectionNameResolver.h"
@ -55,22 +56,6 @@
using namespace arangodb;
using namespace arangodb::pregel;
#if 0
static uint64_t TRI_totalSystemMemory() {
#ifdef _WIN32
MEMORYSTATUSEX status;
status.dwLength = sizeof(status);
GlobalMemoryStatusEx(&status);
return status.ullTotalPhys;
#else
long pages = sysconf(_SC_PHYS_PAGES);
long page_size = sysconf(_SC_PAGE_SIZE);
long mem = pages * page_size;
return mem > 0 ? (uint64_t)mem : 0;
#endif
}
#endif
template <typename V, typename E>
GraphStore<V, E>::GraphStore(TRI_vocbase_t& vb, GraphFormat<V, E>* graphFormat)
: _vocbaseGuard(vb),
@ -82,9 +67,6 @@ GraphStore<V, E>::GraphStore(TRI_vocbase_t& vb, GraphFormat<V, E>* graphFormat)
template <typename V, typename E>
GraphStore<V, E>::~GraphStore() {
_destroyed = true;
std::this_thread::sleep_for(std::chrono::microseconds(25 * 1000));
delete _vertexData;
delete _edges;
}
static const char* shardError =
@ -92,31 +74,28 @@ static const char* shardError =
" use distributeShardsLike";
template <typename V, typename E>
std::map<CollectionID, std::vector<VertexShardInfo>> GraphStore<V, E>::_allocateSpace() {
if (_vertexData || _edges) {
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL,
"Only allocate messages once");
//_edges->resize(count);
}
std::map<CollectionID, std::vector<VertexShardInfo>> result;
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Allocating memory";
//uint64_t totalMemory = TRI_totalSystemMemory();
void GraphStore<V, E>::loadShards(WorkerConfig* config,
std::function<void(bool)> const& cb) {
_config = config;
TRI_ASSERT(_runningThreads == 0);
LOG_TOPIC(DEBUG, Logger::PREGEL)
<< "Using " << config->localVertexShardIDs().size() << " threads to load data";
// hold the current position where the ith vertex shard can
// start to write its data. At the end the offset should equal the
// sum of the counts of all ith edge shards
// Contains the shards located on this db server in the right order
// assuming edges are sharded after _from, vertices after _key
// then every ith vertex shard has the corresponding edges in
// the ith edge shard
std::map<CollectionID, std::vector<ShardID>> const& vertexCollMap =
_config->vertexCollectionShards();
_config->vertexCollectionShards();
std::map<CollectionID, std::vector<ShardID>> const& edgeCollMap =
_config->edgeCollectionShards();
_config->edgeCollectionShards();
size_t numShards = SIZE_MAX;
// Allocating some memory
uint64_t vCount = 0;
uint64_t eCount = 0;
for (auto const& pair : vertexCollMap) {
std::vector<ShardID> const& vertexShards = pair.second;
if (numShards == SIZE_MAX) {
@ -124,164 +103,65 @@ std::map<CollectionID, std::vector<VertexShardInfo>> GraphStore<V, E>::_allocate
} else if (numShards != vertexShards.size()) {
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_BAD_PARAMETER, shardError);
}
for (size_t i = 0; i < vertexShards.size(); i++) {
VertexShardInfo info;
info.vertexShard = vertexShards[i];
info.trx = _createTransaction();
TRI_voc_cid_t cid = info.trx->addCollectionAtRuntime(info.vertexShard);
info.trx->pinData(cid); // will throw when it fails
OperationResult opResult =
info.trx->count(info.vertexShard, transaction::CountType::Normal);
if (opResult.fail() || _destroyed) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_BAD_PARAMETER);
}
info.numVertices = opResult.slice().getUInt();
vCount += info.numVertices;
if (info.numVertices == 0) {
continue;
}
ShardID const& vertexShard = vertexShards[i];
// distributeshardslike should cause the edges for a vertex to be
// in the same shard index. x in vertexShard2 => E(x) in edgeShard2
std::vector<ShardID> edges;
for (auto const& pair2 : edgeCollMap) {
std::vector<ShardID> const& edgeShards = pair2.second;
if (vertexShards.size() != edgeShards.size()) {
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_BAD_PARAMETER, shardError);
}
ShardID const& eShard = edgeShards[i];
info.edgeShards.push_back(eShard);
cid = info.trx->addCollectionAtRuntime(eShard);
info.trx->pinData(cid); // will throw when it fails
opResult = info.trx->count(eShard, transaction::CountType::Normal);
if (opResult.fail() || _destroyed) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_BAD_PARAMETER);
}
info.numEdges += opResult.slice().getUInt();
edges.emplace_back(edgeShards[i]);
}
eCount += info.numEdges;
result[pair.first].push_back(std::move(info));
try {
// we might have already loaded these shards
if (_loadedShards.find(vertexShard) != _loadedShards.end()) {
continue;
}
_loadedShards.insert(vertexShard);
_runningThreads++;
rest::Scheduler* scheduler = SchedulerFeature::SCHEDULER;
TRI_ASSERT(scheduler);
scheduler->queue(RequestPriority::LOW, [this, vertexShard,
edges](bool isDirect) {
TRI_DEFER(_runningThreads--); // exception safe
_loadVertices(vertexShard, edges);
});
} catch (...) {
LOG_TOPIC(WARN, Logger::PREGEL) << "unhandled exception while "
<< "loading pregel graph";
}
}
// we can only load one vertex collection at a time
while (_runningThreads > 0) {
std::this_thread::sleep_for(std::chrono::microseconds(5000));
}
}
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Estimating #numVertices: " << vCount;
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Estimating #numEdges: " << eCount;
_index.resize(vCount);
// size_t requiredMem = vCount * _graphFormat->estimatedVertexSize() +
// eCount * _graphFormat->estimatedEdgeSize();
// if (!_config->lazyLoading() &&
// (_config->useMemoryMaps() || requiredMem > totalMemory / 2)) {
// LOG_TOPIC(DEBUG, Logger::PREGEL) << "Using memory mapped storage";
// if (_graphFormat->estimatedVertexSize() > 0) {
// _vertexData = new MappedFileBuffer<V>(vCount);
// }
// _edges = new MappedFileBuffer<Edge<E>>(eCount);
// } else {
if (_graphFormat->estimatedVertexSize() > 0) {
_vertexData = new VectorTypedBuffer<V>(vCount);
}
_edges = new VectorTypedBuffer<Edge<E>>(eCount);
// }
return result;
}
template <typename V, typename E>
void GraphStore<V, E>::loadShards(WorkerConfig* config,
std::function<void(bool)> const& callback) {
_config = config;
TRI_ASSERT(_runningThreads == 0);
LOG_TOPIC(DEBUG, Logger::PREGEL)
<< "Using " << config->localVertexShardIDs().size() << " threads to load data";
TRI_ASSERT(SchedulerFeature::SCHEDULER != nullptr);
rest::Scheduler* scheduler = SchedulerFeature::SCHEDULER;
scheduler->queue(RequestPriority::LOW, [this, scheduler, callback](bool) {
// hold the current position where the ith vertex shard can
// start to write its data. At the end the offset should equal the
// sum of the counts of all ith edge shards
auto collectionShards = _allocateSpace();
uint64_t vertexOff = 0;
std::vector<size_t> edgeDataOffsets; // will contain # off edges in ith shard
for (auto& collection : collectionShards) {
if (edgeDataOffsets.size() == 0) {
edgeDataOffsets.resize(collection.second.size() + 1);
std::fill(edgeDataOffsets.begin(), edgeDataOffsets.end(), 0);
}
TRI_ASSERT(collection.second.size() < edgeDataOffsets.size());
size_t shardIdx = 0;
for (VertexShardInfo& info : collection.second) {
edgeDataOffsets[++shardIdx] += info.numEdges;
}
}
for (auto& collection : collectionShards) {
size_t shardIdx = 0;
for (VertexShardInfo& info : collection.second) {
try {
// we might have already loaded these shards
if (_loadedShards.find(info.vertexShard) != _loadedShards.end()) {
continue;
}
_loadedShards.insert(info.vertexShard);
_runningThreads++;
TRI_ASSERT(info.numVertices > 0);
TRI_ASSERT(vertexOff < _index.size());
TRI_ASSERT(info.numEdges == 0 || edgeDataOffsets[shardIdx] < _edges->size());
scheduler->queue(RequestPriority::LOW, [this, &info, &edgeDataOffsets,
vertexOff, shardIdx](bool) {
TRI_DEFER(_runningThreads--); // exception safe
_loadVertices(*info.trx, info.vertexShard, info.edgeShards,
vertexOff, edgeDataOffsets[shardIdx]);
});
// update to next offset
vertexOff += info.numVertices;
} catch (...) {
LOG_TOPIC(WARN, Logger::PREGEL) << "unhandled exception while "
<< "loading pregel graph";
}
shardIdx++;
}
// we can only load one vertex collection at a time
while (_runningThreads > 0) {
std::this_thread::sleep_for(std::chrono::microseconds(5000));
}
}
scheduler->queue(RequestPriority::LOW, callback);
});
scheduler->queue(RequestPriority::LOW, cb);
}
template <typename V, typename E>
void GraphStore<V, E>::loadDocument(WorkerConfig* config, std::string const& documentID) {
if (!_vertexData) {
_vertexData = new VectorTypedBuffer<V>(100);
}
if (!_edges) {
_edges = new VectorTypedBuffer<Edge<E>>(100);
}
// figure out if we got this vertex locally
PregelID _id = config->documentIdToPregel(documentID);
if (config->isLocalVertexShard(_id.shard)) {
loadDocument(config, _id.shard, _id.key);
loadDocument(config, _id.shard, StringRef(_id.key));
}
}
template <typename V, typename E>
void GraphStore<V, E>::loadDocument(WorkerConfig* config, PregelShard sourceShard,
PregelKey const& _key) {
StringRef const& _key) {
TRI_ASSERT(false);
#if 0
_config = config;
std::unique_ptr<transaction::Methods> trx(_createTransaction());
TRI_ASSERT(_config->lazyLoading());
@ -334,132 +214,202 @@ void GraphStore<V, E>::loadDocument(WorkerConfig* config, PregelShard sourceShar
LOG_TOPIC(WARN, Logger::PREGEL)
<< "Pregel worker: Failed to commit on a read transaction";
}
#endif
}
template <typename V, typename E>
RangeIterator<VertexEntry> GraphStore<V, E>::vertexIterator() {
return vertexIterator(0, _index.size());
RangeIterator<Vertex<V,E>> GraphStore<V, E>::vertexIterator() {
if (_vertices.empty()) {
return RangeIterator<Vertex<V,E>>(_vertices, 0, nullptr, 0);
}
TypedBuffer<Vertex<V,E>>* front = _vertices.front().get();
return RangeIterator<Vertex<V,E>>(_vertices, 0, front->begin(),
_localVerticeCount);
}
template <typename V, typename E>
RangeIterator<VertexEntry> GraphStore<V, E>::vertexIterator(size_t start, size_t end) {
return RangeIterator<VertexEntry>(_index.data() + start, end - start);
RangeIterator<Vertex<V,E>> GraphStore<V, E>::vertexIterator(size_t i, size_t j) {
if (_vertices.size() <= i) {
return RangeIterator<Vertex<V,E>>(_vertices, 0, nullptr, 0);
}
size_t numVertices = 0;
for (size_t x = i; x < j && x < _vertices.size(); x++) {
numVertices += _vertices[x]->size();
}
return RangeIterator<Vertex<V,E>>(_vertices, i,
_vertices[i]->begin(),
numVertices);
}
template <typename V, typename E>
V* GraphStore<V, E>::mutableVertexData(VertexEntry const* entry) {
return _vertexData->data() + entry->_vertexDataOffset;
RangeIterator<Edge<E>> GraphStore<V, E>::edgeIterator(Vertex<V,E> const* entry) {
if (entry->getEdgeCount() == 0) {
return RangeIterator<Edge<E>>(_edges, 0, nullptr, 0);
}
size_t i = 0;
for (; i < _edges.size(); i++) {
if (_edges[i]->begin() <= entry->getEdges() &&
entry->getEdges() <= _edges[i]->end()) {
break;
}
}
TRI_ASSERT(i < _edges.size());
TRI_ASSERT(i != _edges.size() - 1 ||
_edges[i]->size() >= entry->getEdgeCount());
return RangeIterator<Edge<E>>(_edges, i,
static_cast<Edge<E>*>(entry->getEdges()),
entry->getEdgeCount());
}
template <typename V, typename E>
void GraphStore<V, E>::replaceVertexData(VertexEntry const* entry, void* data, size_t size) {
// if (size <= entry->_vertexDataOffset)
void* ptr = _vertexData->data() + entry->_vertexDataOffset;
memcpy(ptr, data, size);
LOG_TOPIC(WARN, Logger::PREGEL)
<< "Don't use this function with varying sizes";
namespace {
template <typename X>
void moveAppend(std::vector<X>& src, std::vector<X>& dst) {
if (dst.empty()) {
dst = std::move(src);
} else {
dst.reserve(dst.size() + src.size());
std::move(std::begin(src), std::end(src), std::back_inserter(dst));
src.clear();
}
}
template<typename M>
std::unique_ptr<TypedBuffer<M>> createBuffer(WorkerConfig const& config, size_t cap) {
if (config.useMemoryMaps()) {
auto ptr = std::make_unique<MappedFileBuffer<M>>(cap);
ptr->sequentialAccess();
return ptr;
} else {
return std::make_unique<VectorTypedBuffer<M>>(cap);
}
}
}
template <typename V, typename E>
RangeIterator<Edge<E>> GraphStore<V, E>::edgeIterator(VertexEntry const* entry) {
return RangeIterator<Edge<E>>(_edges->data() + entry->_edgeDataOffset, entry->_edgeCount);
}
static constexpr size_t stringChunkSize = 32 * 1024 * 1024 * sizeof(char);
template <typename V, typename E>
std::unique_ptr<transaction::Methods> GraphStore<V, E>::_createTransaction() {
void GraphStore<V, E>::_loadVertices(ShardID const& vertexShard,
std::vector<ShardID> const& edgeShards) {
LOG_TOPIC(DEBUG, Logger::PREGEL)
<< "Pregel worker: loading from vertex shard '" << vertexShard << "'";
transaction::Options trxOpts;
trxOpts.waitForSync = false;
trxOpts.allowImplicitCollections = true;
auto ctx = transaction::StandaloneContext::Create(_vocbaseGuard.database());
auto trx = std::unique_ptr<transaction::Methods>(
new transaction::Methods(ctx, {}, {}, {}, trxOpts));
Result res = trx->begin();
transaction::Methods trx(ctx, {}, {}, {}, trxOpts);
Result res = trx.begin();
if (!res.ok()) {
THROW_ARANGO_EXCEPTION(res);
}
return trx;
}
template <typename V, typename E>
void GraphStore<V, E>::_loadVertices(transaction::Methods& trx, ShardID const& vertexShard,
std::vector<ShardID> const& edgeShards,
size_t vertexOffset, size_t& edgeOffset) {
LOG_TOPIC(DEBUG, Logger::PREGEL)
<< "Pregel worker: loading from vertex shard " << vertexShard;
TRI_ASSERT(vertexOffset < _index.size());
uint64_t originalVertexOffset = vertexOffset;
PregelShard sourceShard = (PregelShard)_config->shardId(vertexShard);
std::unique_ptr<OperationCursor> cursor =
trx.indexScan(vertexShard, transaction::Methods::CursorType::ALL);
auto cursor = trx.indexScan(vertexShard, transaction::Methods::CursorType::ALL);
if (cursor->fail()) {
THROW_ARANGO_EXCEPTION_FORMAT(cursor->code, "while looking up shard '%s'",
vertexShard.c_str());
}
// tell the formatter the number of docs we are about to load
LogicalCollection* collection = cursor->collection();
uint64_t number = collection->numberDocuments(&trx, transaction::CountType::Normal);
_graphFormat->willLoadVertices(number);
LogicalCollection* coll = cursor->collection();
uint64_t numVertices = coll->numberDocuments(&trx, transaction::CountType::Normal);
_graphFormat->willLoadVertices(numVertices);
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Shard '" << vertexShard << "' has "
<< numVertices << " vertices";
std::vector<std::unique_ptr<TypedBuffer<Vertex<V,E>>>> vertices;
std::vector<std::unique_ptr<TypedBuffer<char>>> vKeys;
std::vector<std::unique_ptr<TypedBuffer<Edge<E>>>> edges;
std::vector<std::unique_ptr<TypedBuffer<char>>> eKeys;
TypedBuffer<Vertex<V,E>>* vertexBuff = nullptr;
TypedBuffer<char>* keyBuff = nullptr;
size_t segmentSize = std::min<size_t>(numVertices, vertexSegmentSize());
std::string documentId; // temp buffer for _id of vertex
auto cb = [&](LocalDocumentId const& token, VPackSlice slice) {
if (slice.isExternal()) {
slice = slice.resolveExternal();
}
VertexEntry& ventry = _index[vertexOffset];
ventry._shard = sourceShard;
ventry._key = transaction::helpers::extractKeyFromDocument(slice).copyString();
ventry._edgeDataOffset = edgeOffset;
// load vertex data
std::string documentId = trx.extractIdString(slice);
if (_graphFormat->estimatedVertexSize() > 0) {
TRI_ASSERT(vertexOffset < _vertexData->size());
ventry._vertexDataOffset = vertexOffset;
V* ptr = _vertexData->data() + vertexOffset;
_graphFormat->copyVertexData(documentId, slice, ptr, sizeof(V));
if (vertexBuff == nullptr || vertexBuff->remainingCapacity() == 0) {
vertices.push_back(createBuffer<Vertex<V,E>>(*_config, segmentSize));
vertexBuff = vertices.back().get();
}
Vertex<V,E>* ventry = vertexBuff->appendElement();
VPackValueLength keyLen;
VPackSlice keySlice = transaction::helpers::extractKeyFromDocument(slice);
char const* key = keySlice.getString(keyLen);
if (keyBuff == nullptr || keyLen > keyBuff->remainingCapacity()) {
TRI_ASSERT(keyLen < stringChunkSize);
vKeys.push_back(createBuffer<char>(*_config, stringChunkSize));
keyBuff = vKeys.back().get();
}
ventry->_shard = sourceShard;
ventry->_key = keyBuff->end();
ventry->_keyLength = keyLen;
// actually copy in the key
memcpy(keyBuff->end(), key, keyLen);
keyBuff->advance(keyLen);
// load vertex data
documentId = trx.extractIdString(slice);
if (_graphFormat->estimatedVertexSize() > 0) {
_graphFormat->copyVertexData(documentId, slice, ventry->_data);
}
ventry->_edges = nullptr;
ventry->_edgeCount = 0;
// load edges
for (ShardID const& edgeShard : edgeShards) {
_loadEdges(trx, edgeShard, ventry, documentId);
_loadEdges(trx, *ventry, edgeShard, documentId, edges, eKeys);
}
vertexOffset++;
edgeOffset += ventry._edgeCount;
};
while (cursor->nextDocument(cb, 1000)) {
_localVerticeCount += numVertices;
bool hasMore = true;
while(hasMore && numVertices > 0) {
TRI_ASSERT(segmentSize > 0);
hasMore = cursor->nextDocument(cb, segmentSize);
if (_destroyed) {
LOG_TOPIC(WARN, Logger::PREGEL) << "Aborted loading graph";
break;
}
TRI_ASSERT(numVertices >= segmentSize);
numVertices -= segmentSize;
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Shard '" << vertexShard << "', "
<< numVertices << " remaining vertices";
segmentSize = std::min<size_t>(numVertices, vertexSegmentSize());
}
std::lock_guard<std::mutex> guard(_bufferMutex);
::moveAppend(vertices, _vertices);
::moveAppend(vKeys, _vertexKeys);
::moveAppend(edges, _edges);
::moveAppend(eKeys, _edgeKeys);
// Add all new vertices
_localVerticeCount += (vertexOffset - originalVertexOffset);
if (!trx.commit().ok()) {
LOG_TOPIC(WARN, Logger::PREGEL)
<< "Pregel worker: Failed to commit on a read transaction";
}
LOG_TOPIC(DEBUG, Logger::PREGEL)
<< "Pregel worker: done loading from vertex shard " << vertexShard;
<< "Pregel worker: done loading from vertex shard '" << vertexShard << "'";
}
template <typename V, typename E>
void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, ShardID const& edgeShard,
VertexEntry& vertexEntry, std::string const& documentID) {
size_t added = 0;
size_t offset = vertexEntry._edgeDataOffset + vertexEntry._edgeCount;
// moving pointer to edge
void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, Vertex<V,E>& vertex,
ShardID const& edgeShard, std::string const& documentID,
std::vector<std::unique_ptr<TypedBuffer<Edge<E>>>>& edges,
std::vector<std::unique_ptr<TypedBuffer<char>>>& edgeKeys) {
traverser::EdgeCollectionInfo info(&trx, edgeShard, TRI_EDGE_OUT,
StaticStrings::FromString, 0);
traverser::EdgeCollectionInfo info(&trx, edgeShard);
ManagedDocumentResult mmdr;
std::unique_ptr<OperationCursor> cursor = info.getEdges(documentID, &mmdr);
if (cursor->fail()) {
@ -468,32 +418,43 @@ void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, ShardID const& edge
documentID.c_str(), edgeShard.c_str());
}
auto allocateSpace = [&] {
// If this is called from loadDocument we didn't preallocate the vector
if (_edges->size() <= offset) {
if (!_config->lazyLoading()) {
LOG_TOPIC(ERR, Logger::PREGEL) << "Pregel did not preallocate enough "
<< "space for all edges. This hints "
<< "at a bug with collection count()";
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION(TRI_ERROR_INTERNAL);
}
// lazy loading always uses vector backed storage
((VectorTypedBuffer<Edge<E>>*)_edges)->appendEmptyElement();
TypedBuffer<Edge<E>>* edgeBuff = edges.empty() ? nullptr : edges.back().get();
TypedBuffer<char>* keyBuff = edgeKeys.empty() ? nullptr : edgeKeys.back().get();
auto allocateSpace = [&](size_t keyLen) {
if (edgeBuff == nullptr || edgeBuff->remainingCapacity() == 0) {
edges.push_back(createBuffer<Edge<E>>(*_config, edgeSegmentSize()));
edgeBuff = edges.back().get();
}
if (keyBuff == nullptr || keyLen > keyBuff->remainingCapacity()) {
TRI_ASSERT(keyLen < stringChunkSize);
edgeKeys.push_back(createBuffer<char>(*_config, stringChunkSize));
keyBuff = edgeKeys.back().get();
}
};
size_t addedEdges = 0;
auto buildEdge = [&](Edge<E>* edge, StringRef toValue) {
++addedEdges;
if (++(vertex._edgeCount) == 1) {
vertex._edges = edge;
}
std::size_t pos = toValue.find('/');
StringRef collectionName = toValue.substr(0, pos);
StringRef toVal = toValue.substr(pos + 1);
edge->_toKey = toVal.toString();
StringRef key = toValue.substr(pos + 1);
edge->_toKey = keyBuff->end();
edge->_toKeyLength = key.size();
keyBuff->advance(key.size());
// actually copy in the key
memcpy(edge->_toKey, key.data(), key.size());
// resolve the shard of the target vertex.
ShardID responsibleShard;
int res = Utils::resolveShard(_config, collectionName.toString(),
StaticStrings::KeyString,
toVal, responsibleShard);
key, responsibleShard);
if (res != TRI_ERROR_NO_ERROR) {
LOG_TOPIC(ERR, Logger::PREGEL)
<< "Could not resolve target shard of edge";
@ -507,8 +468,6 @@ void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, ShardID const& edge
<< "Could not resolve target shard of edge";
return TRI_ERROR_CLUSTER_BACKEND_UNAVAILABLE;
}
added++;
offset++;
return TRI_ERROR_NO_ERROR;
};
@ -517,11 +476,11 @@ void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, ShardID const& edge
_graphFormat->estimatedEdgeSize() == 0) {
auto cb = [&](LocalDocumentId const& token, VPackSlice edgeSlice) {
allocateSpace();
TRI_ASSERT(edgeSlice.isString());
StringRef toValue(edgeSlice);
Edge<E>* edge = _edges->data() + offset;
allocateSpace(toValue.size());
Edge<E>* edge = edgeBuff->appendElement();
buildEdge(edge, toValue);
};
while (cursor->nextWithExtra(cb, 1000)) {
@ -536,13 +495,13 @@ void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, ShardID const& edge
if (slice.isExternal()) {
slice = slice.resolveExternal();
}
allocateSpace();
StringRef toValue(transaction::helpers::extractToFromDocument(slice));
Edge<E>* edge = _edges->data() + offset;
allocateSpace(toValue.size());
Edge<E>* edge = edgeBuff->appendElement();
int res = buildEdge(edge, toValue);
if (res == TRI_ERROR_NO_ERROR) {
_graphFormat->copyEdgeData(slice, edge->data(), sizeof(E));
_graphFormat->copyEdgeData(slice, edge->data());
}
};
while (cursor->nextDocument(cb, 1000)) {
@ -554,27 +513,24 @@ void GraphStore<V, E>::_loadEdges(transaction::Methods& trx, ShardID const& edge
}
// Add up all added elements
vertexEntry._edgeCount += added;
_localEdgeCount += added;
_localEdgeCount += addedEdges;
}
/// Loops over the array starting a new transaction for different shards
/// Should not dead-lock unless we have to wait really long for other threads
template <typename V, typename E>
void GraphStore<V, E>::_storeVertices(std::vector<ShardID> const& globalShards,
RangeIterator<VertexEntry>& it) {
RangeIterator<Vertex<V,E>>& it) {
// transaction on one shard
std::unique_ptr<arangodb::SingleCollectionTransaction> trx;
PregelShard currentShard = (PregelShard)-1;
Result res = TRI_ERROR_NO_ERROR;
V* vData = _vertexData->data();
VPackBuilder builder;
size_t numDocs = 0;
// loop over vertices
while (it != it.end()) {
for (; it.hasMore(); ++it) {
if (it->shard() != currentShard || numDocs >= 1000) {
if (trx) {
res = trx->finish(res);
@ -597,20 +553,21 @@ void GraphStore<V, E>::_storeVertices(std::vector<ShardID> const& globalShards,
numDocs = 0;
}
StringRef const key = it->key();
V const& data = it->data();
builder.clear();
// This loop will fill a buffer of vertices until we run into a new
// collection
// or there are no more vertices for to store (or the buffer is full)
V* data = vData + it->_vertexDataOffset;
builder.openObject();
builder.add(StaticStrings::KeyString, VPackValue(it->key()));
builder.add(StaticStrings::KeyString, VPackValuePair(key.data(), key.size(),
VPackValueType::String));
/// bool store =
_graphFormat->buildVertexDocument(builder, data, sizeof(V));
_graphFormat->buildVertexDocument(builder, &data, sizeof(V));
builder.close();
++it;
++numDocs;
if (_destroyed) {
LOG_TOPIC(WARN, Logger::PREGEL)
<< "Storing data was canceled prematurely";
@ -618,13 +575,18 @@ void GraphStore<V, E>::_storeVertices(std::vector<ShardID> const& globalShards,
trx.reset();
break;
}
ShardID const& shard = globalShards[currentShard];
OperationOptions options;
OperationResult result = trx->update(shard, builder.slice(), options);
if (result.fail() && result.isNot(TRI_ERROR_ARANGO_DOCUMENT_NOT_FOUND)) {
if (result.fail() &&
result.isNot(TRI_ERROR_ARANGO_DOCUMENT_NOT_FOUND) &&
result.isNot(TRI_ERROR_ARANGO_CONFLICT)) {
THROW_ARANGO_EXCEPTION(result.result);
}
if (result.is(TRI_ERROR_ARANGO_CONFLICT)) {
LOG_TOPIC(WARN, Logger::PREGEL) << "conflict while storing " << builder.toJson();
}
}
if (trx) {
@ -638,29 +600,29 @@ void GraphStore<V, E>::_storeVertices(std::vector<ShardID> const& globalShards,
template <typename V, typename E>
void GraphStore<V, E>::storeResults(WorkerConfig* config,
std::function<void()> cb) {
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Storing vertex data";
_config = config;
double now = TRI_microtime();
size_t total = _index.size();
size_t delta = _index.size() / _config->localVertexShardIDs().size();
if (delta < 1000) {
delta = _index.size();
}
size_t start = 0, end = delta;
TRI_ASSERT(SchedulerFeature::SCHEDULER != nullptr);
do {
_runningThreads++;
SchedulerFeature::SCHEDULER->queue(RequestPriority::LOW, [this, start, end, now,
cb](bool cancelled) {
if (cancelled) {
cb();
return;
}
size_t numSegments = _vertices.size();
if (_localVerticeCount > 100000) {
_runningThreads = std::min<size_t>(_config->parallelism(), numSegments);
} else {
_runningThreads = 1;
}
size_t numT = _runningThreads;
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Storing vertex data using " <<
numT << " threads";
for (size_t i = 0; i < numT; i++) {
SchedulerFeature::SCHEDULER->queue(RequestPriority::LOW, [=](bool isDirect) {
size_t startI = i * (numSegments / numT);
size_t endI = (i+1) * (numSegments / numT);
TRI_ASSERT(endI <= numSegments);
try {
RangeIterator<VertexEntry> it = vertexIterator(start, end);
RangeIterator<Vertex<V,E>> it = vertexIterator(startI, endI);
_storeVertices(_config->globalShardIDs(), it);
// TODO can't just write edges with smart graphs
} catch(basics::Exception const& e) {
@ -671,16 +633,11 @@ void GraphStore<V, E>::storeResults(WorkerConfig* config,
_runningThreads--;
if (_runningThreads == 0) {
LOG_TOPIC(DEBUG, Logger::PREGEL)
<< "Storing data took " << (TRI_microtime() - now) << "s";
<< "Storing data took " << (TRI_microtime() - now) << "s";
cb();
}
});
start = end;
end = end + delta;
if (total < end + delta) { // swallow the rest
end = total;
}
} while (start != end);
}
}
template class arangodb::pregel::GraphStore<int64_t, int64_t>;

View File

@ -33,6 +33,7 @@
#include <atomic>
#include <cstdint>
#include <memory>
#include <set>
struct TRI_vocbase_t;
@ -52,26 +53,19 @@ struct TypedBuffer;
class WorkerConfig;
template <typename V, typename E>
struct GraphFormat;
// private struct to store some internal information
struct VertexShardInfo {
ShardID vertexShard;
std::vector<ShardID> edgeShards;
std::unique_ptr<transaction::Methods> trx;
/// number of vertices / edges
size_t numVertices = 0;
size_t numEdges = 0;
};
////////////////////////////////////////////////////////////////////////////////
/// @brief carry graph data for a worker job. NOT THREAD SAFE ON DOCUMENT LOADS
////////////////////////////////////////////////////////////////////////////////
template <typename V, typename E>
class GraphStore {
class GraphStore final {
public:
GraphStore(TRI_vocbase_t& vocbase, GraphFormat<V, E>* graphFormat);
~GraphStore();
uint64_t numberVertexSegments() const {
return _vertices.size();
}
uint64_t localVertexCount() const { return _localVerticeCount; }
uint64_t localEdgeCount() const { return _localEdgeCount; }
GraphFormat<V, E> const* graphFormat() { return _graphFormat.get(); }
@ -79,50 +73,57 @@ class GraphStore {
// ====================== NOT THREAD SAFE ===========================
void loadShards(WorkerConfig* state, std::function<void(bool)> const&);
void loadDocument(WorkerConfig* config, std::string const& documentID);
void loadDocument(WorkerConfig* config, PregelShard sourceShard, PregelKey const& _key);
void loadDocument(WorkerConfig* config, PregelShard sourceShard, StringRef const& _key);
// ======================================================================
// only thread safe if your threads coordinate access to memory locations
RangeIterator<VertexEntry> vertexIterator();
RangeIterator<VertexEntry> vertexIterator(size_t start, size_t count);
RangeIterator<Edge<E>> edgeIterator(VertexEntry const* entry);
/// get the pointer to the vertex
V* mutableVertexData(VertexEntry const* entry);
/// does nothing currently
void replaceVertexData(VertexEntry const* entry, void* data, size_t size);
RangeIterator<Vertex<V,E>> vertexIterator();
/// j and j are the first and last index of vertex segments
RangeIterator<Vertex<V,E>> vertexIterator(size_t i, size_t j);
RangeIterator<Edge<E>> edgeIterator(Vertex<V,E> const* entry);
/// Write results to database
void storeResults(WorkerConfig* config, std::function<void()>);
private:
std::map<CollectionID, std::vector<VertexShardInfo>> _allocateSpace();
void _loadVertices(transaction::Methods&, ShardID const& vertexShard,
std::vector<ShardID> const& edgeShards,
size_t vertexOffset, size_t& edgeOffset);
void _loadEdges(transaction::Methods& trx, ShardID const& shard,
VertexEntry& vertexEntry, std::string const& documentID);
void _loadVertices(ShardID const& vertexShard,
std::vector<ShardID> const& edgeShards);
void _loadEdges(transaction::Methods& trx, Vertex<V,E>& vertexEntry,
ShardID const& edgeShard,
std::string const& documentID,
std::vector<std::unique_ptr<TypedBuffer<Edge<E>>>>&,
std::vector<std::unique_ptr<TypedBuffer<char>>>&);
void _storeVertices(std::vector<ShardID> const& globalShards,
RangeIterator<VertexEntry>& it);
RangeIterator<Vertex<V,E>>& it);
std::unique_ptr<transaction::Methods> _createTransaction();
size_t vertexSegmentSize () const {
return std::ceil<size_t>( 64 * 1024 * 1024 / sizeof(Vertex<V,E>));
}
size_t edgeSegmentSize() const {
return std::ceil<size_t>( 64 * 1024 * 1024 / sizeof(Edge<E>));
}
private:
DatabaseGuard _vocbaseGuard;
const std::unique_ptr<GraphFormat<V, E>> _graphFormat;
WorkerConfig* _config = nullptr;
/// Holds vertex keys and pointers to vertex data and edges
std::vector<VertexEntry> _index;
/// Holds vertex keys, data and pointers to edges
std::mutex _bufferMutex;
std::vector<std::unique_ptr<TypedBuffer<Vertex<V,E>>>> _vertices;
std::vector<std::unique_ptr<TypedBuffer<char>>> _vertexKeys;
std::vector<std::unique_ptr<TypedBuffer<Edge<E>>>> _edges;
std::vector<TypedBuffer<Edge<E>>*> _nextEdgeBuffer;
std::vector<std::unique_ptr<TypedBuffer<char>>> _edgeKeys;
/// Vertex data
TypedBuffer<V>* _vertexData = nullptr;
/// Edges (and data)
TypedBuffer<Edge<E>>* _edges = nullptr;
// cache the amount of vertices
std::set<ShardID> _loadedShards;
// actual count of loaded vertices / edges
std::atomic<size_t> _localVerticeCount;
std::atomic<size_t> _localEdgeCount;

View File

@ -47,13 +47,13 @@ void InCache<M>::parseMessages(VPackSlice const& incomingData) {
// temporary variables
VPackValueLength i = 0;
PregelKey key;
StringRef key;
PregelShard shard = (PregelShard)shardSlice.getUInt();
std::lock_guard<std::mutex> guard(this->_bucketLocker[shard]);
for (VPackSlice current : VPackArrayIterator(messages)) {
if (i % 2 == 0) { // TODO support multiple recipients
key = current.copyString();
key = current;
} else {
TRI_ASSERT(!key.empty());
if (current.isArray()) {
@ -84,13 +84,13 @@ void InCache<M>::parseMessages(VPackSlice const& incomingData) {
template <typename M>
void InCache<M>::storeMessageNoLock(PregelShard shard,
PregelKey const& vertexId, M const& data) {
StringRef const& vertexId, M const& data) {
this->_set(shard, vertexId, data);
this->_containedMessageCount++;
}
template <typename M>
void InCache<M>::storeMessage(PregelShard shard, PregelKey const& vertexId, M const& data) {
void InCache<M>::storeMessage(PregelShard shard, StringRef const& vertexId, M const& data) {
std::lock_guard<std::mutex> guard(this->_bucketLocker[shard]);
this->_set(shard, vertexId, data);
this->_containedMessageCount++;
@ -112,9 +112,9 @@ ArrayInCache<M>::ArrayInCache(WorkerConfig const* config, MessageFormat<M> const
}
template <typename M>
void ArrayInCache<M>::_set(PregelShard shard, PregelKey const& key, M const& newValue) {
void ArrayInCache<M>::_set(PregelShard shard, StringRef const& key, M const& newValue) {
HMap& vertexMap(_shardMap[shard]);
vertexMap[key].push_back(newValue);
vertexMap[key.toString()].push_back(newValue);
}
template <typename M>
@ -157,9 +157,10 @@ void ArrayInCache<M>::mergeCache(WorkerConfig const& config, InCache<M> const* o
}
template <typename M>
MessageIterator<M> ArrayInCache<M>::getMessages(PregelShard shard, PregelKey const& key) {
MessageIterator<M> ArrayInCache<M>::getMessages(PregelShard shard, StringRef const& key) {
std::string keyS = key.toString();
HMap const& vertexMap = _shardMap[shard];
auto vmsg = vertexMap.find(key);
auto vmsg = vertexMap.find(keyS);
if (vmsg != vertexMap.end()) {
M const* ptr = vmsg->second.data();
return MessageIterator<M>(ptr, vmsg->second.size());
@ -179,9 +180,10 @@ void ArrayInCache<M>::clear() {
/// Deletes one entry. DOES NOT LOCK
template <typename M>
void ArrayInCache<M>::erase(PregelShard shard, PregelKey const& key) {
void ArrayInCache<M>::erase(PregelShard shard, StringRef const& key) {
std::string keyS = key.toString();
HMap& vertexMap = _shardMap[shard];
auto const& it = vertexMap.find(key);
auto const& it = vertexMap.find(keyS);
if (it != vertexMap.end()) {
vertexMap.erase(it);
this->_containedMessageCount--;
@ -189,13 +191,13 @@ void ArrayInCache<M>::erase(PregelShard shard, PregelKey const& key) {
}
template <typename M>
void ArrayInCache<M>::forEach(std::function<void(PregelShard, PregelKey const&, M const&)> func) {
void ArrayInCache<M>::forEach(std::function<void(PregelShard, StringRef const&, M const&)> func) {
for (auto const& pair : _shardMap) {
PregelShard shard = pair.first;
HMap const& vertexMap = pair.second;
for (auto& vertexMsgs : vertexMap) {
for (M const& val : vertexMsgs.second) {
func(shard, vertexMsgs.first, val);
func(shard, StringRef(vertexMsgs.first), val);
}
}
}
@ -219,13 +221,14 @@ CombiningInCache<M>::CombiningInCache(WorkerConfig const* config,
}
template <typename M>
void CombiningInCache<M>::_set(PregelShard shard, PregelKey const& key, M const& newValue) {
void CombiningInCache<M>::_set(PregelShard shard, StringRef const& key, M const& newValue) {
std::string keyS = key.toString();
HMap& vertexMap = _shardMap[shard];
auto vmsg = vertexMap.find(key);
auto vmsg = vertexMap.find(keyS);
if (vmsg != vertexMap.end()) { // got a message for the same vertex
_combiner->combine(vmsg->second, newValue);
} else {
vertexMap.insert(std::make_pair(key, newValue));
vertexMap.insert(std::make_pair(std::move(keyS), newValue));
}
}
@ -272,9 +275,10 @@ void CombiningInCache<M>::mergeCache(WorkerConfig const& config, InCache<M> cons
}
template <typename M>
MessageIterator<M> CombiningInCache<M>::getMessages(PregelShard shard, PregelKey const& key) {
MessageIterator<M> CombiningInCache<M>::getMessages(PregelShard shard, StringRef const& key) {
std::string keyS = key.toString();
HMap const& vertexMap = _shardMap[shard];
auto vmsg = vertexMap.find(key);
auto vmsg = vertexMap.find(keyS);
if (vmsg != vertexMap.end()) {
return MessageIterator<M>(&vmsg->second);
} else {
@ -292,9 +296,10 @@ void CombiningInCache<M>::clear() {
/// Deletes one entry. DOES NOT LOCK
template <typename M>
void CombiningInCache<M>::erase(PregelShard shard, PregelKey const& key) {
void CombiningInCache<M>::erase(PregelShard shard, StringRef const& key) {
std::string keyS = key.toString();
HMap& vertexMap = _shardMap[shard];
auto const& it = vertexMap.find(key);
auto const& it = vertexMap.find(keyS);
if (it != vertexMap.end()) {
vertexMap.erase(it);
this->_containedMessageCount--;
@ -304,12 +309,12 @@ void CombiningInCache<M>::erase(PregelShard shard, PregelKey const& key) {
/// Calls function for each entry. DOES NOT LOCK
template <typename M>
void CombiningInCache<M>::forEach(
std::function<void(PregelShard shard, PregelKey const& key, M const&)> func) {
std::function<void(PregelShard shard, StringRef const& key, M const&)> func) {
for (auto const& pair : _shardMap) {
PregelShard shard = pair.first;
HMap const& vertexMap = pair.second;
for (auto& vertexMessage : vertexMap) {
func(shard, vertexMessage.first, vertexMessage.second);
func(shard, StringRef(vertexMessage.first), vertexMessage.second);
}
}
}

View File

@ -54,7 +54,7 @@ class InCache {
/// Initialize format and mutex map.
/// @param config can be null if you don't want locks
explicit InCache(MessageFormat<M> const* format);
virtual void _set(PregelShard shard, PregelKey const& vertexId, M const& data) = 0;
virtual void _set(PregelShard shard, StringRef const& vertexId, M const& data) = 0;
public:
virtual ~InCache() {}
@ -66,54 +66,54 @@ class InCache {
/// @brief Store a single message.
/// Only ever call when you are sure this is a thread local store
void storeMessageNoLock(PregelShard shard, PregelKey const& vertexId, M const& data);
void storeMessageNoLock(PregelShard shard, StringRef const& vertexId, M const& data);
/// @brief Store a single message
void storeMessage(PregelShard shard, PregelKey const& vertexId, M const& data);
void storeMessage(PregelShard shard, StringRef const& vertexId, M const& data);
virtual void mergeCache(WorkerConfig const& config, InCache<M> const* otherCache) = 0;
/// @brief get messages for vertex id. (Don't use keys from _from or _to
/// directly, they contain the collection name)
virtual MessageIterator<M> getMessages(PregelShard shard, PregelKey const& key) = 0;
virtual MessageIterator<M> getMessages(PregelShard shard, StringRef const& key) = 0;
/// clear cache
virtual void clear() = 0;
/// Deletes one entry. DOES NOT LOCK
virtual void erase(PregelShard shard, PregelKey const& key) = 0;
virtual void erase(PregelShard shard, StringRef const& key) = 0;
/// Calls function for each entry. DOES NOT LOCK
virtual void forEach(std::function<void(PregelShard, PregelKey const&, M const&)> func) = 0;
virtual void forEach(std::function<void(PregelShard, StringRef const&, M const&)> func) = 0;
};
/// Cache version which stores a std::vector<M> for each pregel id
/// containing all messages for this vertex
template <typename M>
class ArrayInCache : public InCache<M> {
typedef std::unordered_map<PregelKey, std::vector<M>> HMap;
typedef std::unordered_map<std::string, std::vector<M>> HMap;
std::map<PregelShard, HMap> _shardMap;
protected:
void _set(PregelShard shard, PregelKey const& vertexId, M const& data) override;
void _set(PregelShard shard, StringRef const& vertexId, M const& data) override;
public:
ArrayInCache(WorkerConfig const* config, MessageFormat<M> const* format);
void mergeCache(WorkerConfig const& config, InCache<M> const* otherCache) override;
MessageIterator<M> getMessages(PregelShard shard, PregelKey const& key) override;
MessageIterator<M> getMessages(PregelShard shard, StringRef const& key) override;
void clear() override;
void erase(PregelShard shard, PregelKey const& key) override;
void forEach(std::function<void(PregelShard shard, PregelKey const& key, M const& val)> func) override;
void erase(PregelShard shard, StringRef const& key) override;
void forEach(std::function<void(PregelShard shard, StringRef const& key, M const& val)> func) override;
};
/// Cache which stores one value per vertex id
template <typename M>
class CombiningInCache : public InCache<M> {
typedef std::unordered_map<PregelKey, M> HMap;
typedef std::unordered_map<std::string, M> HMap;
MessageCombiner<M> const* _combiner;
std::map<PregelShard, HMap> _shardMap;
protected:
void _set(PregelShard shard, PregelKey const& vertexId,
void _set(PregelShard shard, StringRef const& vertexId,
M const& data) override;
public:
@ -123,10 +123,10 @@ class CombiningInCache : public InCache<M> {
MessageCombiner<M> const* combiner() const { return _combiner; }
void mergeCache(WorkerConfig const& config, InCache<M> const* otherCache) override;
MessageIterator<M> getMessages(PregelShard shard, PregelKey const& key) override;
MessageIterator<M> getMessages(PregelShard shard, StringRef const& key) override;
void clear() override;
void erase(PregelShard shard, PregelKey const& key) override;
void forEach(std::function<void(PregelShard, PregelKey const&, M const&)> func) override;
void erase(PregelShard shard, StringRef const& key) override;
void forEach(std::function<void(PregelShard, StringRef const&, M const&)> func) override;
};
} // namespace pregel
} // namespace arangodb

View File

@ -0,0 +1,75 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2016 ArangoDB GmbH, Cologne, Germany
///
/// Licensed under the Apache License, Version 2.0 (the "License");
/// you may not use this file except in compliance with the License.
/// You may obtain a copy of the License at
///
/// http://www.apache.org/licenses/LICENSE-2.0
///
/// Unless required by applicable law or agreed to in writing, software
/// distributed under the License is distributed on an "AS IS" BASIS,
/// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
/// See the License for the specific language governing permissions and
/// limitations under the License.
///
/// Copyright holder is ArangoDB GmbH, Cologne, Germany
///
/// @author Michael Hackstein
////////////////////////////////////////////////////////////////////////////////
#include "IndexHelpers.h"
#include "Cluster/ClusterMethods.h"
#include "Transaction/Methods.h"
#include "Utils/OperationCursor.h"
using namespace arangodb;
using namespace arangodb::traverser;
EdgeCollectionInfo::EdgeCollectionInfo(transaction::Methods* trx,
std::string const& collectionName)
: _trx(trx),
_collectionName(collectionName),
_searchBuilder() {
if (!trx->isEdgeCollection(collectionName)) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_ARANGO_COLLECTION_TYPE_INVALID);
}
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex.
////////////////////////////////////////////////////////////////////////////////
std::unique_ptr<arangodb::OperationCursor> EdgeCollectionInfo::getEdges(
std::string const& vertexId, arangodb::ManagedDocumentResult* mmdr) {
/// @brief index used for iteration
transaction::Methods::IndexHandle indexId;
auto var = _searchBuilder.getVariable();
auto cond = _searchBuilder.getOutboundCondition();
bool worked = _trx->getBestIndexHandleForFilterCondition(_collectionName, cond,
var, 1000, indexId);
TRI_ASSERT(worked); // We always have an edge Index
_searchBuilder.setVertexId(vertexId);
std::unique_ptr<arangodb::OperationCursor> res;
IndexIteratorOptions opts;
opts.enableCache = false;
res.reset(_trx->indexScanForCondition(indexId,
_searchBuilder.getOutboundCondition(),
_searchBuilder.getVariable(), mmdr, opts));
return res;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Return name of the wrapped collection
////////////////////////////////////////////////////////////////////////////////
std::string const& EdgeCollectionInfo::getName() const {
return _collectionName;
}

View File

@ -1,7 +1,7 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2014-2016 ArangoDB GmbH, Cologne, Germany
/// Copyright 2014-2019 ArangoDB GmbH, Cologne, Germany
/// Copyright 2004-2014 triAGENS GmbH, Cologne, Germany
///
/// Licensed under the Apache License, Version 2.0 (the "License");
@ -21,14 +21,18 @@
/// @author Michael Hackstein
////////////////////////////////////////////////////////////////////////////////
#ifndef ARANGOD_EDGE_COLLECTION_INFO_H
#define ARANGOD_EDGE_COLLECTION_INFO_H 1
#ifndef ARANGOD_PREGEL_INDEX_HELPERS_H
#define ARANGOD_PREGEL_INDEX_HELPERS_H 1
#include "Aql/Graphs.h"
#include "Graph/Traverser.h"
#include "Transaction/Methods.h"
namespace arangodb {
namespace transaction{
class Methods;
}
struct OperationCursor;
struct ManagedDocumentResult;
namespace traverser {
////////////////////////////////////////////////////////////////////////////////
@ -51,11 +55,6 @@ class EdgeCollectionInfo {
std::string _collectionName;
/// @brief index used for forward iteration
transaction::Methods::IndexHandle _forwardIndexId;
/// @brief index used for backward iteration
transaction::Methods::IndexHandle _backwardIndexId;
//////////////////////////////////////////////////////////////////////////////
/// @brief Temporary builder for index search values
@ -64,16 +63,8 @@ class EdgeCollectionInfo {
aql::EdgeConditionBuilderContainer _searchBuilder;
std::string _weightAttribute;
double _defaultWeight;
TRI_edge_direction_e _dir;
public:
EdgeCollectionInfo(transaction::Methods* trx, std::string const& collectionName,
TRI_edge_direction_e const direction,
std::string const& weightAttribute, double defaultWeight);
EdgeCollectionInfo(transaction::Methods* trx, std::string const& cname);
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex.
@ -82,29 +73,6 @@ class EdgeCollectionInfo {
std::unique_ptr<arangodb::OperationCursor> getEdges(std::string const&,
ManagedDocumentResult*);
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex. On Coordinator.
////////////////////////////////////////////////////////////////////////////////
int getEdgesCoordinator(arangodb::velocypack::Slice const&, arangodb::velocypack::Builder&);
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex. Reverse version
////////////////////////////////////////////////////////////////////////////////
std::unique_ptr<arangodb::OperationCursor> getReverseEdges(std::string const&,
ManagedDocumentResult*);
////////////////////////////////////////////////////////////////////////////////
/// @brief Get edges for the given direction and start vertex. Reverse version
/// on Coordinator.
////////////////////////////////////////////////////////////////////////////////
int getReverseEdgesCoordinator(arangodb::velocypack::Slice const&,
arangodb::velocypack::Builder&);
double weightEdge(arangodb::velocypack::Slice const);
transaction::Methods* trx() const { return _trx; }
////////////////////////////////////////////////////////////////////////////////

View File

@ -23,23 +23,27 @@
#ifndef ARANGODB_PREGEL_ITERATOR_H
#define ARANGODB_PREGEL_ITERATOR_H 1
#include "Pregel/TypedBuffer.h"
namespace arangodb {
namespace pregel {
template <typename M>
class MessageIterator {
M const* _data;
size_t _current = 0;
const size_t _size = 1;
const size_t _size;
size_t _current;
public:
MessageIterator() : _data(nullptr), _current(0), _size(0) {}
MessageIterator() : _data(nullptr), _size(0), _current(0) {}
typedef MessageIterator<M> iterator;
typedef const MessageIterator<M> const_iterator;
explicit MessageIterator(M const* data) : _data(data), _size(data ? 1 : 0) {}
explicit MessageIterator(M const* data, size_t s) : _data(data), _size(s) {}
explicit MessageIterator(M const* data)
: _data(data), _size(data ? 1 : 0), _current(0) {}
explicit MessageIterator(M const* data, size_t s)
: _data(data), _size(s), _current(0) {}
iterator begin() { return MessageIterator(_data, _size); }
const_iterator begin() const { return MessageIterator(_data, _size); }
@ -54,8 +58,6 @@ class MessageIterator {
return it;
}
// M const& operator*() const { return *(_data + _current); }
M const* operator*() const { return _data + _current; }
M const* operator->() const { return _data + _current; }
@ -83,77 +85,73 @@ class MessageIterator {
template <typename T>
class RangeIterator {
private:
// void *_begin, *_end, *_current;
T* _data;
size_t _current, _size;
std::vector<std::unique_ptr<TypedBuffer<T>>>& _buffers;
size_t _beginBuffer;
T* _beginPtr;
T* _currentBufferEnd;
size_t _size;
public:
typedef RangeIterator<T> iterator;
typedef const RangeIterator<T> const_iterator;
RangeIterator(T* v, size_t size) : _data(v), _current(0), _size(size) {}
iterator begin() { return RangeIterator(_data, _size); }
const_iterator begin() const { return RangeIterator(_data, _size); }
iterator end() {
auto it = RangeIterator(_data, _size);
it._current = it._size;
return it;
RangeIterator(std::vector<std::unique_ptr<TypedBuffer<T>>>& bufs,
size_t beginBuffer, T* beginPtr,
size_t size)
: _buffers(bufs),
_beginBuffer(beginBuffer),
_beginPtr(beginPtr),
_currentBufferEnd(bufs[_beginBuffer]->end()),
_size(size) {}
RangeIterator(RangeIterator const&) = delete;
RangeIterator& operator=(RangeIterator const&) = delete;
RangeIterator(RangeIterator&& other)
: _buffers(other._buffers),
_beginBuffer(other._beginBuffer),
_beginPtr(other._beginPtr),
_size(other._size) {
other._beginBuffer = 0;
other._beginPtr = nullptr;
other._currentBufferEnd = nullptr;
other._size = 0;
}
const_iterator end() const {
auto it = RangeIterator(_data, _size);
it._current = it._size;
return it;
RangeIterator& operator=(RangeIterator&& other) {
this->_beginBuffer = other._beginBuffer ;
this->_beginPtr = other._beginPtr ;
this->_currentBufferEnd = other._currentBufferEnd;
this->_size = other._size;
other._beginBuffer = 0;
other._beginPtr = nullptr;
other._currentBufferEnd = nullptr;
other._size = 0;
}
bool hasMore() const {
return _size > 0;
}
// prefix ++
RangeIterator& operator++() {
_current++;
TRI_ASSERT(_beginPtr != _currentBufferEnd);
TRI_ASSERT(_size > 0);
++_beginPtr;
--_size;
if (_beginPtr == _currentBufferEnd && _size > 0) {
++_beginBuffer;
TRI_ASSERT(_beginBuffer < _buffers.size());
TypedBuffer<T>* tb = _buffers[_beginBuffer].get();
_beginPtr = tb->begin();
_currentBufferEnd = tb->end();
}
return *this;
}
// postfix ++
RangeIterator<T>& operator++(int) {
RangeIterator<T> result(*this);
++(*this);
return result;
}
T* operator*() const { return _beginPtr; }
T* operator*() const { return _data + _current; }
T* operator->() const { return _data + _current; }
bool operator!=(RangeIterator<T> const& other) const {
return _current != other._current;
}
size_t size() const { return _size; }
/*EdgeIterator(void* beginPtr, void* endPtr)
: _begin(beginPtr), _end(endPtr), _current(_begin) {}
iterator begin() { return EdgeIterator(_begin, _end); }
const_iterator begin() const { return EdgeIterator(_begin, _end); }
iterator end() {
auto it = EdgeIterator(_begin, _end);
it._current = it._end;
return it;
}
const_iterator end() const {
auto it = EdgeIterator(_begin, _end);
it._current = it._end;
return it;
}
// prefix ++
EdgeIterator<E>& operator++() {
EdgeEntry<E>* entry = static_cast<EdgeEntry<E>>(_current);
_current += entry->getSize();
return *this;
}
EdgeEntry<E>* operator*() const {
return _current != _end ? static_cast<EdgeEntry<E>>(_current) : nullptr;
}*/
T* operator->() const { return _beginPtr; }
};
} // namespace pregel
} // namespace arangodb

View File

@ -58,7 +58,7 @@ void ArrayOutCache<M>::_removeContainedMessages() {
}
template <typename M>
void ArrayOutCache<M>::appendMessage(PregelShard shard, PregelKey const& key, M const& data) {
void ArrayOutCache<M>::appendMessage(PregelShard shard, StringRef const& key, M const& data) {
if (this->_config->isLocalVertexShard(shard)) {
if (this->_sendToNextGSS) { // I use the global cache, we need locking
this->_localCacheNextGSS->storeMessage(shard, key, data);
@ -94,7 +94,7 @@ void ArrayOutCache<M>::flushMessages() {
std::vector<ClusterCommRequest> requests;
for (auto const& it : _shardMap) {
PregelShard shard = it.first;
std::unordered_map<PregelKey, std::vector<M>> const& vertexMessageMap = it.second;
std::unordered_map<StringRef, std::vector<M>> const& vertexMessageMap = it.second;
if (vertexMessageMap.size() == 0) {
continue;
}
@ -156,7 +156,7 @@ void CombiningOutCache<M>::_removeContainedMessages() {
template <typename M>
void CombiningOutCache<M>::appendMessage(PregelShard shard,
PregelKey const& key, M const& data) {
StringRef const& key, M const& data) {
if (this->_config->isLocalVertexShard(shard)) {
if (this->_sendToNextGSS) {
this->_localCacheNextGSS->storeMessage(shard, key, data);
@ -166,7 +166,7 @@ void CombiningOutCache<M>::appendMessage(PregelShard shard,
this->_sendCount++;
}
} else {
std::unordered_map<PregelKey, M>& vertexMap = _shardMap[shard];
std::unordered_map<StringRef, M>& vertexMap = _shardMap[shard];
auto it = vertexMap.find(key);
if (it != vertexMap.end()) { // more than one message
_combiner->combine(vertexMap[key], data);
@ -198,7 +198,7 @@ void CombiningOutCache<M>::flushMessages() {
std::vector<ClusterCommRequest> requests;
for (auto const& it : _shardMap) {
PregelShard shard = it.first;
std::unordered_map<PregelKey, M> const& vertexMessageMap = it.second;
std::unordered_map<StringRef, M> const& vertexMessageMap = it.second;
if (vertexMessageMap.size() == 0) {
continue;
}

View File

@ -92,14 +92,14 @@ class OutCache {
_sendCountNextGSS = 0;
_removeContainedMessages();
};
virtual void appendMessage(PregelShard shard, PregelKey const& key, M const& data) = 0;
virtual void appendMessage(PregelShard shard, StringRef const& key, M const& data) = 0;
virtual void flushMessages() = 0;
};
template <typename M>
class ArrayOutCache : public OutCache<M> {
/// @brief two stage map: shard -> vertice -> message
std::unordered_map<PregelShard, std::unordered_map<PregelKey, std::vector<M>>> _shardMap;
std::unordered_map<PregelShard, std::unordered_map<StringRef, std::vector<M>>> _shardMap;
void _removeContainedMessages() override;
@ -108,7 +108,7 @@ class ArrayOutCache : public OutCache<M> {
: OutCache<M>(state, format) {}
~ArrayOutCache();
void appendMessage(PregelShard shard, PregelKey const& key, M const& data) override;
void appendMessage(PregelShard shard, StringRef const& key, M const& data) override;
void flushMessages() override;
};
@ -117,7 +117,7 @@ class CombiningOutCache : public OutCache<M> {
MessageCombiner<M> const* _combiner;
/// @brief two stage map: shard -> vertice -> message
std::unordered_map<PregelShard, std::unordered_map<PregelKey, M>> _shardMap;
std::unordered_map<PregelShard, std::unordered_map<StringRef, M>> _shardMap;
void _removeContainedMessages() override;
public:
@ -125,7 +125,7 @@ class CombiningOutCache : public OutCache<M> {
MessageCombiner<M> const* combiner);
~CombiningOutCache();
void appendMessage(PregelShard shard, PregelKey const& key, M const& data) override;
void appendMessage(PregelShard shard, StringRef const& key, M const& data) override;
void flushMessages() override;
};
} // namespace pregel

View File

@ -39,33 +39,51 @@
namespace arangodb {
namespace pregel {
/// continuous memory buffer with a fixed capacity
template <typename T>
struct TypedBuffer {
static_assert(std::is_default_constructible<T>::value, "");
/// close file (see close() )
virtual ~TypedBuffer() {}
TypedBuffer() : _ptr(nullptr) {}
TypedBuffer() : _begin(nullptr), _end(nullptr), _capacity(nullptr) {}
/// @brief return whether the datafile is a physical file (true) or an
/// anonymous mapped region (false)
// inline bool isPhysical() const { return !_filename.empty(); }
/// close datafile
/// end usage of the structure
virtual void close() = 0;
/// raw access
T* data() const { return _ptr; }
T* begin() const { return _begin; }
T* end() const { return _end; }
virtual T& back() = 0;
T& back() const {
return *(_end - 1);
}
/// get file size
// uint64_t size() const;
/// get size
size_t size() const {
return static_cast<size_t>(_end - _begin);
}
/// get number of actually mapped bytes
virtual size_t size() const = 0;
/// replace mapping by a new one of the same file, offset MUST be a multiple
/// of the page size
virtual void resize(size_t newSize) = 0;
size_t capacity() const {
return static_cast<size_t>(_capacity - _begin);
}
size_t remainingCapacity() const {
return static_cast<size_t>(_capacity - _end);
}
T* appendElement() {
TRI_ASSERT(_begin <= _end);
TRI_ASSERT(_end < _capacity);
return new (_end++) T();
}
template <typename U = T>
typename std::enable_if<std::is_trivially_constructible<U>::value>::type
advance(std::size_t value) {
TRI_ASSERT((_end + value) <= _capacity);
_end += value;
}
private:
/// don't copy object
TypedBuffer(const TypedBuffer&) = delete;
@ -73,36 +91,42 @@ struct TypedBuffer {
TypedBuffer& operator=(const TypedBuffer&) = delete;
protected:
T* _ptr;
T* _begin; // begin
T* _end; // current end
T* _capacity; // max capacity
};
template <typename T>
class VectorTypedBuffer : public TypedBuffer<T> {
std::vector<T> _vector;
public:
VectorTypedBuffer(size_t entries) : _vector(entries) {
this->_ptr = _vector.data();
VectorTypedBuffer(size_t capacity) : TypedBuffer<T>() {
this->_begin = static_cast<T*>(malloc(sizeof(T) * capacity));
this->_end = this->_begin;
this->_capacity = this->_begin + capacity;
if (this->_begin == nullptr) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_OUT_OF_MEMORY);
}
}
~VectorTypedBuffer() {
close();
}
void close() override {
//_data.clear();
}
if (this->_begin == nullptr) {
return;
}
T& back() override { return _vector.back(); }
// destroy all elements in the buffer
for (auto* p = this->_begin; p != this->_end; ++p) {
reinterpret_cast<T*>(p)->~T();
}
/// get file size
// uint64_t size() const;
/// get number of actually mapped bytes
size_t size() const override { return _vector.size(); }
/// replace mapping by a new one of the same file, offset MUST be a multiple
/// of the page size
virtual void resize(size_t newSize) override { _vector.resize(newSize); }
void appendEmptyElement() {
_vector.push_back(T());
this->_ptr = _vector.data(); // might change address
free(static_cast<void*>(this->_begin));
this->_begin = nullptr;
this->_end = nullptr;
this->_capacity = nullptr;
}
};
@ -110,61 +134,29 @@ class VectorTypedBuffer : public TypedBuffer<T> {
/** Filesize limited by size_t, usually 2^32 or 2^64 */
template <typename T>
class MappedFileBuffer : public TypedBuffer<T> {
std::string _filename; // underlying filename
int _fd = -1; // underlying file descriptor
void* _mmHandle; // underlying memory map object handle (windows only)
size_t _mappedSize; // actually mapped size
public:
#ifdef TRI_HAVE_ANONYMOUS_MMAP
explicit MappedFileBuffer(size_t entries) : _size(entries) {
#ifdef TRI_MMAP_ANONYMOUS
// fd -1 is required for "real" anonymous regions
_fd = -1;
int flags = TRI_MMAP_ANONYMOUS | MAP_SHARED;
#else
// ugly workaround if MAP_ANONYMOUS is not available
_fd = TRI_TRACKED_OPEN_FILE("/dev/zero", O_RDWR | TRI_O_CLOEXEC);
if (_fd == -1) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_INTERNAL);
}
int flags = MAP_PRIVATE;
#endif
// memory map the data
_mappedSize = sizeof(T) * entries;
void* ptr;
int res = TRI_MMFile(nullptr, _mappedSize, PROT_WRITE | PROT_READ, flags,
_fd, &_mmHandle, 0, &ptr);
#ifdef MAP_ANONYMOUS
// nothing to do
#else
// close auxilliary file
TRI_TRACKED_CLOSE_FILE(_fd);
_fd = -1;
#endif
if (res != TRI_ERROR_NO_ERROR) {
TRI_set_errno(res);
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "cannot memory map anonymous region: " << TRI_last_error();
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "The database directory might reside on a shared folder "
"(VirtualBox, VMWare) or an NFS "
"mounted volume which does not allow memory mapped files.";
THROW_ARANGO_EXCEPTION(TRI_ERROR_INTERNAL);
}
this->_ptr = (T*)ptr;
// return new TypedBuffer(StaticStrings::Empty, fd, mmHandle, initialSize,
// ptr);
}
#else
explicit MappedFileBuffer(size_t entries) : _size(entries) {
explicit MappedFileBuffer(size_t capacity) : TypedBuffer<T>() {
double tt = TRI_microtime();
std::string file = "pregel_" + std::to_string((uint64_t)tt) + ".mmap";
std::string filename = FileUtils::buildFilename(TRI_GetTempPath(), file);
long tt2 = random();
std::string file = "pregel_" + std::to_string((uint64_t)tt) + "_" + std::to_string(tt2) + ".mmap";
this->_filename = basics::FileUtils::buildFilename(TRI_GetTempPath(), file);
_mappedSize = sizeof(T) * _size;
_fd = TRI_CreateDatafile(filename, _mappedSize);
_mappedSize = sizeof(T) * capacity;
size_t pageSize = PageSizeFeature::getPageSize();
TRI_ASSERT(pageSize >= 256);
// use multiples of page-size
_mappedSize = (size_t)(((_mappedSize + pageSize - 1) / pageSize) * pageSize);
_fd = TRI_CreateDatafile(_filename, _mappedSize);
if (_fd < 0) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_INTERNAL);
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, "pregel cannot create mmap file");
}
// memory map the data
@ -179,142 +171,76 @@ class MappedFileBuffer : public TypedBuffer<T> {
if (res != TRI_ERROR_NO_ERROR) {
TRI_set_errno(res);
TRI_TRACKED_CLOSE_FILE(fd);
TRI_TRACKED_CLOSE_FILE(_fd);
_fd = -1;
// remove empty file
TRI_UnlinkFile(filename.c_str());
// remove file
TRI_UnlinkFile(_filename.c_str());
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "cannot memory map file '" << filename << "': '"
<< TRI_errno_string(res) << "'";
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "The database directory might reside on a shared folder "
"(VirtualBox, VMWare) or an NFS-mounted volume which does not "
"allow memory mapped files.";
THROW_ARANGO_EXCEPTION(TRI_ERROR_INTERNAL);
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, std::string("cannot memory map file '") + _filename + "': '" + TRI_errno_string(res) + "'");
}
this->_ptr = (T*)data;
this->_begin = static_cast<T*>(data);
this->_end = static_cast<T*>(data);
this->_capacity = this->_begin + capacity;
}
#endif
/// close file (see close() )
~MappedFileBuffer() { close(); }
/// @brief return whether the datafile is a physical file (true) or an
/// anonymous mapped region (false)
inline bool isPhysical() const { return !_filename.empty(); }
void sequentialAccess() {
TRI_MMFileAdvise(this->_ptr, _mappedSize, TRI_MADVISE_SEQUENTIAL);
TRI_MMFileAdvise(this->_begin, _mappedSize, TRI_MADVISE_SEQUENTIAL);
}
void randomAccess() {
TRI_MMFileAdvise(this->_ptr, _mappedSize, TRI_MADVISE_RANDOM);
TRI_MMFileAdvise(this->_begin, _mappedSize, TRI_MADVISE_RANDOM);
}
void willNeed() {
TRI_MMFileAdvise(this->_ptr, _mappedSize, TRI_MADVISE_WILLNEED);
TRI_MMFileAdvise(this->_begin, _mappedSize, TRI_MADVISE_WILLNEED);
}
void dontNeed() {
TRI_MMFileAdvise(this->_ptr, _mappedSize, TRI_MADVISE_DONTNEED);
TRI_MMFileAdvise(this->_begin, _mappedSize, TRI_MADVISE_DONTNEED);
}
/// close file
void close() override {
if (this->_ptr == nullptr) {
if (this->_begin == nullptr) {
// already closed or not opened
return;
}
int res = TRI_UNMMFile(this->_ptr, _mappedSize, _fd, &_mmHandle);
// destroy all elements in the buffer
for (auto* p = this->_begin; p != this->_end; ++p) {
reinterpret_cast<T*>(p)->~T();
}
int res = TRI_UNMMFile(this->_begin, _mappedSize, _fd, &_mmHandle);
if (res != TRI_ERROR_NO_ERROR) {
// leave file open here as it will still be memory-mapped
LOG_TOPIC(ERR, arangodb::Logger::FIXME) << "munmap failed with: " << res;
}
if (isPhysical()) {
if (_fd != -1) {
TRI_ASSERT(_fd >= 0);
int res = TRI_TRACKED_CLOSE_FILE(_fd);
if (res != TRI_ERROR_NO_ERROR) {
LOG_TOPIC(ERR, arangodb::Logger::FIXME)
<< "unable to close pregel mapped file '" << _filename << "': " << res;
}
// remove file
TRI_UnlinkFile(this->_filename.c_str());
}
this->_ptr = nullptr;
_fd = -1;
this->_begin = nullptr;
this->_end = nullptr;
this->_capacity = nullptr;
this->_fd = -1;
}
T& back() override { return *(this->_ptr + _size - 1); }
/// true, if file successfully opened
bool isValid() const { return this->_ptr != nullptr; }
/// get file size
// uint64_t size() const;
/// get number of actually mapped bytes
size_t size() const override { return _size; }
/// replace mapping by a new one of the same file, offset MUST be a multiple
/// of the page size
void resize(size_t newSize) override {
if (this->_ptr == nullptr) {
THROW_ARANGO_EXCEPTION(TRI_ERROR_INTERNAL);
} else if (newSize == _size) {
return;
}
#ifdef __linux__
size_t newMappedSize = sizeof(T) * newSize;
void* newPtr = mremap((void*)this->_ptr, _mappedSize, newMappedSize, MREMAP_MAYMOVE);
if (newPtr != MAP_FAILED) { // success
TRI_ASSERT(this->_ptr != nullptr);
this->_ptr = (T*)newPtr;
_mappedSize = newMappedSize;
_size = newSize;
return;
}
if (errno == ENOMEM) {
LOG_TOPIC(DEBUG, Logger::MMAP) << "out of memory in mmap";
THROW_ARANGO_EXCEPTION(TRI_ERROR_OUT_OF_MEMORY_MMAP);
} else {
// preserve errno value while we're logging
LOG_TOPIC(WARN, Logger::MMAP) << "memory-mapping failed";
THROW_ARANGO_EXCEPTION(TRI_ERROR_SYS_ERROR);
}
#else
// resizing mappings doesn't exist on other systems
if (_size < newSize || newSize * sizeof(T) <= _mappedSize) {
_size = newSize;
} else {
LOG_TOPIC(ERR, Logger::MMAP)
<< "Resizing mmap not supported on this platform";
THROW_ARANGO_EXCEPTION(TRI_ERROR_FAILED);
}
#endif
}
/// get OS page size (for remap)
/*int TypedBuffer::getpagesize() {
#ifdef _MSC_VER
SYSTEM_INFO sysInfo;
GetSystemInfo(&sysInfo);
return sysInfo.dwAllocationGranularity;
#else
return sysconf(_SC_PAGESIZE); //::getpagesize();
#endif
}*/
private:
/// get OS page size (for remap)
// static int getpagesize();
std::string _filename; // underlying filename
int _fd = -1; // underlying file descriptor
void* _mmHandle; // underlying memory map object handle (windows only)
size_t _size = 0;
size_t _mappedSize;
bool isValid() const { return this->_begin != nullptr; }
};
} // namespace pregel
} // namespace arangodb

View File

@ -49,7 +49,7 @@ class VertexContext {
GraphStore<V, E>* _graphStore = nullptr;
AggregatorHandler* _readAggregators = nullptr;
AggregatorHandler* _writeAggregators = nullptr;
VertexEntry* _vertexEntry = nullptr;
Vertex<V,E>* _vertexEntry = nullptr;
public:
virtual ~VertexContext() {}
@ -76,10 +76,10 @@ class VertexContext {
inline WorkerContext const* context() const { return _context; }
V* mutableVertexData() {
return (V*)_graphStore->mutableVertexData(_vertexEntry);
return &(_vertexEntry->data());
}
V vertexData() { return *((V*)_graphStore->mutableVertexData(_vertexEntry)); }
V vertexData() const { return _vertexEntry->data(); }
size_t getEdgeCount() const { return _vertexEntry->getEdgeCount(); }
@ -104,7 +104,7 @@ class VertexContext {
inline uint64_t localSuperstep() const { return _lss; }
PregelShard shard() const { return _vertexEntry->shard(); }
PregelKey const& key() const { return _vertexEntry->key(); }
StringRef key() const { return _vertexEntry->key(); }
PregelID pregelId() const { return _vertexEntry->pregelId(); }
};
@ -122,14 +122,15 @@ class VertexComputation : public VertexContext<V, E, M> {
}
void sendMessage(PregelID const& pid, M const& data) {
_cache->appendMessage(pid.shard, pid.key, data);
_cache->appendMessage(pid.shard, StringRef(pid.key), data);
}
/// Send message along outgoing edges to all reachable neighbours
/// TODO Multi-receiver messages
void sendMessageToAllNeighbours(M const& data) {
RangeIterator<Edge<E>> edges = this->getEdges();
for (Edge<E> const* edge : edges) {
for (; edges.hasMore(); ++edges) {
Edge<E> const* edge = *edges;
_cache->appendMessage(edge->targetShard(), edge->toKey(), data);
}
}

View File

@ -85,7 +85,6 @@ Worker<V, E, M>::Worker(TRI_vocbase_t& vocbase, Algorithm<V, E, M>* algo, VPackS
template <typename V, typename E, typename M>
Worker<V, E, M>::~Worker() {
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Called ~Worker()";
_state = WorkerState::DONE;
std::this_thread::sleep_for(std::chrono::microseconds(50000)); // 50ms wait for threads to die
delete _readCache;
@ -139,7 +138,7 @@ void Worker<V, E, M>::_initializeMessageCaches() {
// @brief load the initial worker data, call conductor eventually
template <typename V, typename E, typename M>
void Worker<V, E, M>::setupWorker() {
std::function<void(bool)> callback = [this](bool) {
std::function<void(bool)> cb = [this](bool) {
VPackBuilder package;
package.openObject();
package.add(Utils::senderKey, VPackValue(ServerState::instance()->getId()));
@ -160,14 +159,15 @@ void Worker<V, E, M>::setupWorker() {
for (std::string const& documentID : activeSet) {
_graphStore->loadDocument(&_config, documentID);
}
callback(false);
cb(false);
} else {
// initialization of the graphstore might take an undefined amount
// of time. Therefore this is performed asynchronous
TRI_ASSERT(SchedulerFeature::SCHEDULER != nullptr);
rest::Scheduler* scheduler = SchedulerFeature::SCHEDULER;
scheduler->queue(RequestPriority::LOW, [this, callback](bool) {
_graphStore->loadShards(&_config, callback);
auto self = shared_from_this();
scheduler->queue(RequestPriority::LOW, [self, this, cb](bool) {
_graphStore->loadShards(&_config, cb);
});
}
}
@ -315,36 +315,38 @@ void Worker<V, E, M>::_startProcessing() {
rest::Scheduler* scheduler = SchedulerFeature::SCHEDULER;
size_t total = _graphStore->localVertexCount();
size_t delta = total / _config.parallelism();
size_t start = 0, end = delta;
if (delta < 100 || total < 100) {
_runningThreads = 1;
end = total;
size_t numSegments = _graphStore->numberVertexSegments();
if (total > 100000) {
_runningThreads = std::min<size_t>(_config.parallelism(), numSegments);
} else {
_runningThreads = total / delta; // rounds-up unsigned integers
_runningThreads = 1;
}
size_t i = 0;
do {
scheduler->queue(RequestPriority::LOW, [this, start, end, i](bool) {
TRI_ASSERT(_runningThreads >= 1);
TRI_ASSERT(_runningThreads <= _config.parallelism());
size_t numT = _runningThreads;
auto self = shared_from_this();
for (size_t i = 0; i < numT; i++) {
scheduler->queue(RequestPriority::LOW, [self, this, i, numT, numSegments](bool) {
if (_state != WorkerState::COMPUTING) {
LOG_TOPIC(WARN, Logger::PREGEL) << "Execution aborted prematurely.";
return;
}
auto vertices = _graphStore->vertexIterator(start, end);
size_t startI = i * (numSegments / numT);
size_t endI = (i+1) * (numSegments / numT);
TRI_ASSERT(endI <= numSegments);
auto vertices = _graphStore->vertexIterator(startI, endI);
// should work like a join operation
if (_processVertices(i, vertices) && _state == WorkerState::COMPUTING) {
_finishedProcessing(); // last thread turns the lights out
}
});
start = end;
end = end + delta;
if (total < end + delta) { // swallow the rest
end = total;
}
i++;
} while (start != total);
}
// TRI_ASSERT(_runningThreads == i);
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Using " << i << " Threads";
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Using " << numT << " Threads";
}
template <typename V, typename E, typename M>
@ -359,7 +361,7 @@ void Worker<V, E, M>::_initializeVertexContext(VertexContext<V, E, M>* ctx) {
// internally called in a WORKER THREAD!!
template <typename V, typename E, typename M>
bool Worker<V, E, M>::_processVertices(size_t threadId,
RangeIterator<VertexEntry>& vertexIterator) {
RangeIterator<Vertex<V,E>>& vertexIterator) {
double start = TRI_microtime();
// thread local caches
@ -387,10 +389,11 @@ bool Worker<V, E, M>::_processVertices(size_t threadId,
}
size_t activeCount = 0;
for (VertexEntry* vertexEntry : vertexIterator) {
for (; vertexIterator.hasMore(); ++vertexIterator) {
Vertex<V,E>* vertexEntry = *vertexIterator;
MessageIterator<M> messages =
_readCache->getMessages(vertexEntry->shard(), vertexEntry->key());
if (messages.size() > 0 || vertexEntry->active()) {
vertexComputation->_vertexEntry = vertexEntry;
vertexComputation->compute(messages);
@ -428,17 +431,13 @@ bool Worker<V, E, M>::_processVertices(size_t threadId,
bool lastThread = false;
{ // only one thread at a time
MUTEX_LOCKER(guard, _threadMutex);
/*if (t > 0.005) {
LOG_TOPIC(DEBUG, Logger::PREGEL) << "Total " << stats.superstepRuntimeSecs
<< " s merge took " << t << " s";
}*/
// merge the thread local stats and aggregators
_workerAggregators->aggregateValues(workerAggregator);
_messageStats.accumulate(stats);
_activeCount += activeCount;
_runningThreads--;
lastThread = _runningThreads == 0; // should work like a join operation
lastThread = _runningThreads == 0; // should work like a join operation
}
return lastThread;
}
@ -470,13 +469,14 @@ void Worker<V, E, M>::_finishedProcessing() {
if (_config.lazyLoading()) { // TODO how to improve this?
// hack to determine newly added vertices
size_t currentAVCount = _graphStore->localVertexCount();
auto currentVertices = _graphStore->vertexIterator();
for (VertexEntry* vertexEntry : currentVertices) {
auto it = _graphStore->vertexIterator();
for (; it.hasMore(); ++it) {
Vertex<V,E>* vertexEntry = *it;
// reduces the containedMessageCount
_readCache->erase(vertexEntry->shard(), vertexEntry->key());
}
_readCache->forEach([this](PregelShard shard, PregelKey const& key, M const&) {
_readCache->forEach([this](PregelShard shard, StringRef const& key, M const&) {
_graphStore->loadDocument(&_config, shard, key);
});
@ -598,7 +598,8 @@ void Worker<V, E, M>::finalizeExecution(VPackSlice const& body,
return;
}
auto cleanup = [this, cb] {
auto self = shared_from_this();
auto cleanup = [self, this, cb] {
VPackBuilder body;
body.openObject();
body.add(Utils::senderKey, VPackValue(ServerState::instance()->getId()));
@ -630,7 +631,8 @@ void Worker<V, E, M>::aqlResult(VPackBuilder& b, bool withId) const {
b.openArray(/*unindexed*/true);
auto it = _graphStore->vertexIterator();
for (VertexEntry const* vertexEntry : it) {
for (; it.hasMore(); ++it) {
Vertex<V,E> const* vertexEntry = *it;
TRI_ASSERT(vertexEntry->shard() < _config.globalShardIDs().size());
ShardID const& shardId = _config.globalShardIDs()[vertexEntry->shard()];
@ -643,7 +645,7 @@ void Worker<V, E, M>::aqlResult(VPackBuilder& b, bool withId) const {
tmp.clear();
tmp.append(cname);
tmp.push_back('/');
tmp.append(vertexEntry->key());
tmp.append(vertexEntry->key().data(), vertexEntry->key().size());
b.add(StaticStrings::IdString, VPackValue(tmp));
}
}
@ -652,9 +654,9 @@ void Worker<V, E, M>::aqlResult(VPackBuilder& b, bool withId) const {
vertexEntry->key().size(),
VPackValueType::String));
V* data = _graphStore->mutableVertexData(vertexEntry);
V const& data = vertexEntry->data();
// bool store =
_graphStore->graphFormat()->buildVertexDocument(b, data, sizeof(V));
_graphStore->graphFormat()->buildVertexDocument(b, &data, sizeof(V));
b.close();
}
b.close();
@ -701,7 +703,8 @@ void Worker<V, E, M>::compensateStep(VPackSlice const& data) {
TRI_ASSERT(SchedulerFeature::SCHEDULER != nullptr);
rest::Scheduler* scheduler = SchedulerFeature::SCHEDULER;
scheduler->queue(RequestPriority::LOW, [this](bool) {
auto self = shared_from_this();
scheduler->queue(RequestPriority::LOW, [self, this](bool) {
if (_state != WorkerState::RECOVERING) {
LOG_TOPIC(WARN, Logger::PREGEL) << "Compensation aborted prematurely.";
return;
@ -719,7 +722,8 @@ void Worker<V, E, M>::compensateStep(VPackSlice const& data) {
vCompensate->_writeAggregators = _workerAggregators.get();
size_t i = 0;
for (VertexEntry* vertexEntry : vertexIterator) {
for (; vertexIterator.hasMore(); ++vertexIterator) {
Vertex<V,E>* vertexEntry = *vertexIterator;
vCompensate->_vertexEntry = vertexEntry;
vCompensate->compensate(i > _preRecoveryTotal);
i++;
@ -764,7 +768,8 @@ void Worker<V, E, M>::_callConductor(std::string const& path, VPackBuilder const
if (ServerState::instance()->isRunningInCluster() == false) {
TRI_ASSERT(SchedulerFeature::SCHEDULER != nullptr);
rest::Scheduler* scheduler = SchedulerFeature::SCHEDULER;
scheduler->queue(RequestPriority::LOW, [path, message](bool) {
auto self = shared_from_this();
scheduler->queue(RequestPriority::LOW, [self, path, message](bool) {
VPackBuilder response;
PregelFeature::handleConductorRequest(path, message.slice(), response);
});

View File

@ -44,7 +44,7 @@ class RestPregelHandler;
namespace pregel {
class IWorker {
class IWorker : public std::enable_shared_from_this<IWorker> {
public:
virtual ~IWorker() {}
virtual void setupWorker() = 0;
@ -71,7 +71,6 @@ class OutCache;
template <typename T>
class RangeIterator;
class VertexEntry;
template <typename V, typename E, typename M>
class VertexContext;
@ -138,7 +137,7 @@ class Worker : public IWorker {
void _initializeMessageCaches();
void _initializeVertexContext(VertexContext<V, E, M>* ctx);
void _startProcessing();
bool _processVertices(size_t threadId, RangeIterator<VertexEntry>& vertexIterator);
bool _processVertices(size_t threadId, RangeIterator<Vertex<V,E>>& vertexIterator);
void _finishedProcessing();
void _continueAsync();
void _callConductor(std::string const& path, VPackBuilder const& message);

View File

@ -35,42 +35,65 @@
/// @author Copyright 2017, ArangoDB GmbH, Cologne, Germany
////////////////////////////////////////////////////////////////////////////////
#include "Basics/Common.h"
#include "ApplicationFeatures/ApplicationServer.h"
#include "Pregel/TypedBuffer.h"
#include "catch.hpp"
#include "Pregel/TypedBuffer.h"
using namespace arangodb::pregel;
/***************************************/
TEST_CASE("tst_pregel1", "[pregel][mmap]") {
MappedFileBuffer<int> mapped(1024);
int *ptr = mapped.data();
for (int i = 0; i < 1024; i++) {
*(ptr+i) = i;
}
for (int i = 0; i < 1024; i++) {
REQUIRE(*(ptr+i) == i);
TEST_CASE("tst_pregel1", "[pregel]") {
arangodb::application_features::ApplicationServer server(nullptr, nullptr);
arangodb::PageSizeFeature feature(server);
feature.prepare();
SECTION("malloc") {
VectorTypedBuffer<int> mapped(1024);
REQUIRE(mapped.size() == 0);
REQUIRE(mapped.capacity() == 1024);
REQUIRE(mapped.remainingCapacity() == 1024);
mapped.advance(1024);
REQUIRE(mapped.size() == 1024);
REQUIRE(mapped.capacity() == 1024);
REQUIRE(mapped.remainingCapacity() == 0);
int *ptr = mapped.begin();
for (int i = 0; i < 1024; i++) {
*(ptr+i) = i;
}
for (int i = 0; i < 1024; i++) {
REQUIRE(*(ptr+i) == i);
}
}
#ifdef __linux__
mapped.resize(2048);
REQUIRE(mapped.size() == 2048);
ptr = mapped.data();
for (int i = 0; i < 1024; i++) {
REQUIRE(*(ptr+i) == i);
SECTION("mmap") {
MappedFileBuffer<int> mapped(1024);
REQUIRE(mapped.size() == 0);
REQUIRE(mapped.capacity() == 1024);
REQUIRE(mapped.remainingCapacity() == 1024);
mapped.advance(1024);
REQUIRE(mapped.size() == 1024);
REQUIRE(mapped.capacity() == 1024);
REQUIRE(mapped.remainingCapacity() == 0);
int *ptr = mapped.begin();
for (int i = 0; i < 1024; i++) {
*(ptr+i) = i;
}
for (int i = 0; i < 1024; i++) {
REQUIRE(*(ptr+i) == i);
}
mapped.close();
REQUIRE(mapped.begin() == nullptr);
}
#endif
mapped.resize(512);
ptr = mapped.data();
REQUIRE(mapped.size() == 512);
for (int i = 0; i < 512; i++) {
REQUIRE(*(ptr+i) == i);
}
mapped.close();
REQUIRE(mapped.data() == nullptr);
}

View File

@ -152,6 +152,11 @@ function basicTestSuite() {
testAlgo("pagerank", { threshold: EPS / 10, resultField: "result", store: true });
},
testPageRankMMap: function () {
// should test correct convergence behaviour, might fail if EPS is too low
testAlgo("pagerank", { threshold: EPS / 10, resultField: "result", store: true, useMemoryMaps: true });
},
testPageRankSeeded: function () {
// test that pagerank picks the seed value
testAlgo("pagerank", { maxGSS: 1, sourceField: "pagerank", resultField: "result", store: true });
@ -265,8 +270,8 @@ function exampleTestSuite() {
function randomTestSuite() {
'use strict';
const n = 10000; // vertices
const m = 150000; // edges
const n = 20000; // vertices
const m = 300000; // edges
return {
@ -276,6 +281,9 @@ function randomTestSuite() {
setUpAll: function () {
console.log("Beginning to insert test data with " + n +
" vertices, " + m + " edges");
var exists = graph_module._list().indexOf("random") !== -1;
if (exists || db.demo_v) {
return;
@ -301,9 +309,15 @@ function randomTestSuite() {
}
db[vColl].insert(vertices);
db[vColl].count();
if (x % 100000 === 0) {
console.log("Inserted " + x + " vertices");
}
}
assertEqual(db[vColl].count(), n);
console.log("Done inserting vertices, inserting edges");
x = 0;
while (x < m) {
let edges = [];
@ -317,6 +331,10 @@ function randomTestSuite() {
x++;
}
db[eColl].insert(edges);
if (x % 100000 === 0) {
console.log("Inserted " + x + " edges");
}
}
assertEqual(db[eColl].count(), m * 2);
},