1
0
Fork 0
arangodb/arangod/Aql/ExecutionBlock.h

2494 lines
98 KiB
C++

////////////////////////////////////////////////////////////////////////////////
/// @brief Infrastructure for ExecutionBlocks (the execution engine)
///
/// @file arangod/Aql/ExecutionBlock.h
///
/// DISCLAIMER
///
/// Copyright 2010-2014 triagens 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 triAGENS GmbH, Cologne, Germany
///
/// @author Max Neunhoeffer
/// @author Copyright 2014, triagens GmbH, Cologne, Germany
////////////////////////////////////////////////////////////////////////////////
#ifndef ARANGODB_AQL_EXECUTION_BLOCK_H
#define ARANGODB_AQL_EXECUTION_BLOCK_H 1
#include "Basics/JsonHelper.h"
#include "Aql/AqlItemBlock.h"
#include "Aql/Collection.h"
#include "Aql/CollectionScanner.h"
#include "Aql/ExecutionNode.h"
#include "Aql/Range.h"
#include "Aql/WalkerWorker.h"
#include "Aql/ExecutionStats.h"
#include "Basics/StringBuffer.h"
#include "Cluster/ClusterComm.h"
#include "Utils/AqlTransaction.h"
#include "Utils/transactions.h"
#include "Utils/V8TransactionContext.h"
#include "VocBase/shaped-json.h"
struct TRI_df_marker_s;
struct TRI_doc_mptr_copy_t;
struct TRI_edge_index_iterator_t;
struct TRI_hash_index_element_multi_s;
struct TRI_json_t;
namespace triagens {
namespace aql {
struct CollectionScanner;
class ExecutionEngine;
// -----------------------------------------------------------------------------
// --SECTION-- AggregatorGroup
// -----------------------------------------------------------------------------
////////////////////////////////////////////////////////////////////////////////
/// @brief details about the current group
///////////////////////////////////////////////////////////////////////////////
struct AggregatorGroup {
std::vector<AqlValue> groupValues;
std::vector<TRI_document_collection_t const*> collections;
std::vector<AqlItemBlock*> groupBlocks;
size_t firstRow;
size_t lastRow;
size_t groupLength;
bool rowsAreValid;
bool const count;
explicit AggregatorGroup (bool);
~AggregatorGroup ();
void initialize (size_t capacity);
void reset ();
void setFirstRow (size_t value) {
firstRow = value;
rowsAreValid = true;
}
void setLastRow (size_t value) {
lastRow = value;
rowsAreValid = true;
}
void addValues (AqlItemBlock const* src,
RegisterId groupRegister);
};
// -----------------------------------------------------------------------------
// --SECTION-- ExecutionBlock
// -----------------------------------------------------------------------------
class ExecutionBlock {
// -----------------------------------------------------------------------------
// --SECTION-- constructors / destructors
// -----------------------------------------------------------------------------
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
ExecutionBlock (ExecutionEngine*,
ExecutionNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
virtual ~ExecutionBlock ();
// -----------------------------------------------------------------------------
// --SECTION-- public methods
// -----------------------------------------------------------------------------
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief determine the number of rows in a vector of blocks
////////////////////////////////////////////////////////////////////////////////
size_t countBlocksRows (std::vector<AqlItemBlock*> const&) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the query was killed
////////////////////////////////////////////////////////////////////////////////
bool isKilled () const;
////////////////////////////////////////////////////////////////////////////////
/// @brief throw an exception if query was killed
////////////////////////////////////////////////////////////////////////////////
void throwIfKilled ();
////////////////////////////////////////////////////////////////////////////////
/// @brief add a dependency
////////////////////////////////////////////////////////////////////////////////
void addDependency (ExecutionBlock* ep) {
_dependencies.emplace_back(ep);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief get all dependencies
////////////////////////////////////////////////////////////////////////////////
std::vector<ExecutionBlock*> getDependencies () const {
return _dependencies;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief remove a dependency, returns true if the pointer was found and
/// removed, please note that this does not delete ep!
////////////////////////////////////////////////////////////////////////////////
bool removeDependency (ExecutionBlock* ep);
////////////////////////////////////////////////////////////////////////////////
/// @brief access the pos-th dependency
////////////////////////////////////////////////////////////////////////////////
ExecutionBlock* operator[] (size_t pos) {
if (pos >= _dependencies.size()) {
return nullptr;
}
return _dependencies.at(pos);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief Methods for execution
/// Lifecycle is:
/// CONSTRUCTOR
/// then the ExecutionEngine automatically calls
/// initialize() once, including subqueries
/// possibly repeat many times:
/// initializeCursor(...) (optionally with bind parameters)
/// // use cursor functionality
/// then the ExecutionEngine automatically calls
/// shutdown()
/// DESTRUCTOR
////////////////////////////////////////////////////////////////////////////////
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize
////////////////////////////////////////////////////////////////////////////////
virtual int initialize ();
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor, could be called multiple times
////////////////////////////////////////////////////////////////////////////////
virtual int initializeCursor (AqlItemBlock* items, size_t pos);
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown, will be called exactly once for the whole query
////////////////////////////////////////////////////////////////////////////////
virtual int shutdown (int);
////////////////////////////////////////////////////////////////////////////////
/// @brief getOne, gets one more item
////////////////////////////////////////////////////////////////////////////////
virtual AqlItemBlock* getOne () {
return getSome(1, 1);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome, gets some more items, semantic is as follows: not
/// more than atMost items may be delivered. The method tries to
/// return a block of at least atLeast items, however, it may return
/// less (for example if there are not enough items to come). However,
/// if it returns an actual block, it must contain at least one item.
////////////////////////////////////////////////////////////////////////////////
virtual AqlItemBlock* getSome (size_t atLeast, size_t atMost);
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief request an AqlItemBlock from the memory manager
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* requestBlock (size_t,
RegisterId);
////////////////////////////////////////////////////////////////////////////////
/// @brief return an AqlItemBlock to the memory manager
////////////////////////////////////////////////////////////////////////////////
void returnBlock (AqlItemBlock*&);
////////////////////////////////////////////////////////////////////////////////
/// @brief resolve a collection name and return cid and document key
/// this is used for parsing _from, _to and _id values
////////////////////////////////////////////////////////////////////////////////
int resolve (char const*,
TRI_voc_cid_t&,
std::string&) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief copy register data from one block (src) into another (dst)
/// register values are cloned
////////////////////////////////////////////////////////////////////////////////
void inheritRegisters (AqlItemBlock const* src,
AqlItemBlock* dst,
size_t row);
void inheritRegisters (AqlItemBlock const* src,
AqlItemBlock* dst,
size_t,
size_t);
////////////////////////////////////////////////////////////////////////////////
/// @brief the following is internal to pull one more block and append it to
/// our _buffer deque. Returns true if a new block was appended and false if
/// the dependent node is exhausted.
////////////////////////////////////////////////////////////////////////////////
bool getBlock (size_t atLeast, size_t atMost);
////////////////////////////////////////////////////////////////////////////////
/// @brief getSomeWithoutRegisterClearout, same as above, however, this
/// is the actual worker which does not clear out registers at the end
/// the idea is that somebody who wants to call the generic functionality
/// in a derived class but wants to modify the results before the register
/// cleanup can use this method, internal use only
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSomeWithoutRegisterClearout (size_t atLeast, size_t atMost);
////////////////////////////////////////////////////////////////////////////////
/// @brief clearRegisters, clears out registers holding values that are no
/// longer needed by later nodes
////////////////////////////////////////////////////////////////////////////////
void clearRegisters (AqlItemBlock* result);
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome, skips some more items, semantic is as follows: not
/// more than atMost items may be skipped. The method tries to
/// skip a block of at least atLeast items, however, it may skip
/// less (for example if there are not enough items to come). The number of
/// elements skipped is returned.
////////////////////////////////////////////////////////////////////////////////
virtual size_t skipSome (size_t atLeast, size_t atMost);
// skip exactly <number> outputs, returns <true> if _done after
// skipping, and <false> otherwise . . .
bool skip (size_t number);
virtual bool hasMore ();
virtual int64_t count () const {
return _dependencies[0]->count();
}
virtual int64_t remaining ();
ExecutionNode const* getPlanNode () const {
return _exeNode;
}
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief generic method to get or skip some
////////////////////////////////////////////////////////////////////////////////
virtual int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped);
////////////////////////////////////////////////////////////////////////////////
/// @brief the execution engine
////////////////////////////////////////////////////////////////////////////////
ExecutionEngine* _engine;
////////////////////////////////////////////////////////////////////////////////
/// @brief the transaction for this query
////////////////////////////////////////////////////////////////////////////////
triagens::arango::AqlTransaction* _trx;
////////////////////////////////////////////////////////////////////////////////
/// @brief our corresponding ExecutionNode node
////////////////////////////////////////////////////////////////////////////////
ExecutionNode const* _exeNode;
////////////////////////////////////////////////////////////////////////////////
/// @brief our dependent nodes
////////////////////////////////////////////////////////////////////////////////
std::vector<ExecutionBlock*> _dependencies;
////////////////////////////////////////////////////////////////////////////////
/// @brief this is our buffer for the items, it is a deque of AqlItemBlocks.
/// We keep the following invariant between this and the other two variables
/// _pos and _done: If _buffer.size() != 0, then 0 <= _pos < _buffer[0]->size()
/// and _buffer[0][_pos] is the next item to be handed on. If _done is true,
/// then no more documents will ever be returned. _done will be set to
/// true if and only if we have no more data ourselves (i.e. _buffer.size()==0)
/// and we have unsuccessfully tried to get another block from our dependency.
////////////////////////////////////////////////////////////////////////////////
std::deque<AqlItemBlock*> _buffer;
////////////////////////////////////////////////////////////////////////////////
/// @brief current working position in the first entry of _buffer
////////////////////////////////////////////////////////////////////////////////
size_t _pos;
////////////////////////////////////////////////////////////////////////////////
/// @brief if this is set, we are done, this is reset to false by execute()
////////////////////////////////////////////////////////////////////////////////
bool _done;
// -----------------------------------------------------------------------------
// --SECTION-- public variables
// -----------------------------------------------------------------------------
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief batch size value
////////////////////////////////////////////////////////////////////////////////
static size_t const DefaultBatchSize;
};
// -----------------------------------------------------------------------------
// --SECTION-- SingletonBlock
// -----------------------------------------------------------------------------
class SingletonBlock : public ExecutionBlock {
void deleteInputVariables() {
delete _inputRegisterValues;
_inputRegisterValues = nullptr;
}
public:
SingletonBlock (ExecutionEngine* engine,
SingletonNode const* ep)
: ExecutionBlock(engine, ep),
_inputRegisterValues(nullptr) {
}
~SingletonBlock () {
deleteInputVariables();
}
int initialize () override {
_inputRegisterValues = nullptr; // just in case
return ExecutionBlock::initialize();
}
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor, store a copy of the register values coming from above
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
int shutdown (int) override final;
bool hasMore () override final {
return ! _done;
}
int64_t count () const override final {
return 1;
}
int64_t remaining () override final {
return _done ? 0 : 1;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief the bind data coming from outside
////////////////////////////////////////////////////////////////////////////////
private:
int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief _inputRegisterValues
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* _inputRegisterValues;
};
// -----------------------------------------------------------------------------
// --SECTION-- EnumerateCollectionBlock
// -----------------------------------------------------------------------------
class EnumerateCollectionBlock : public ExecutionBlock {
public:
EnumerateCollectionBlock (ExecutionEngine* engine,
EnumerateCollectionNode const* ep);
~EnumerateCollectionBlock ();
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize fetching of documents
////////////////////////////////////////////////////////////////////////////////
void initializeDocuments () {
_scanner->reset();
_documents.clear();
_posInDocuments = 0;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief continue fetching of documents
////////////////////////////////////////////////////////////////////////////////
bool moreDocuments (size_t hint);
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize, here we fetch all docs from the database
////////////////////////////////////////////////////////////////////////////////
int initialize () override;
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast, size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
// skip between atLeast and atMost, returns the number actually skipped . . .
// will only return less than atLeast if there aren't atLeast many
// things to skip overall.
////////////////////////////////////////////////////////////////////////////////
size_t skipSome (size_t atLeast, size_t atMost) override final;
// -----------------------------------------------------------------------------
// --SECTION-- private variables
// -----------------------------------------------------------------------------
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief collection
////////////////////////////////////////////////////////////////////////////////
Collection* _collection;
////////////////////////////////////////////////////////////////////////////////
/// @brief collection scanner
////////////////////////////////////////////////////////////////////////////////
CollectionScanner* _scanner;
////////////////////////////////////////////////////////////////////////////////
/// @brief document buffer
////////////////////////////////////////////////////////////////////////////////
std::vector<TRI_doc_mptr_copy_t> _documents;
////////////////////////////////////////////////////////////////////////////////
/// @brief current position in _documents
////////////////////////////////////////////////////////////////////////////////
size_t _posInDocuments;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not we're doing random iteration
////////////////////////////////////////////////////////////////////////////////
bool const _random;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the enumerated documents need to be stored
////////////////////////////////////////////////////////////////////////////////
bool _mustStoreResult;
};
// -----------------------------------------------------------------------------
// --SECTION-- IndexRangeBlock
// -----------------------------------------------------------------------------
class IndexRangeBlock : public ExecutionBlock {
public:
IndexRangeBlock (ExecutionEngine* engine,
IndexRangeNode const* ep);
~IndexRangeBlock ();
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize, here we fetch all docs from the database
////////////////////////////////////////////////////////////////////////////////
int initialize () override;
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor, here we release our docs from this collection
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
AqlItemBlock* getSome (size_t atLeast, size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
// skip between atLeast and atMost, returns the number actually skipped . . .
// will only return less than atLeast if there aren't atLeast many
// things to skip overall.
////////////////////////////////////////////////////////////////////////////////
size_t skipSome (size_t atLeast, size_t atMost) override final;
// -----------------------------------------------------------------------------
// --SECTION-- private methods
// -----------------------------------------------------------------------------
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the high bound values should be taken into account
/// they can be ignored for indexes that only support equality conditions,
/// i.e. primary index, edge index and hash index
////////////////////////////////////////////////////////////////////////////////
bool useHighBounds () const;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not one of the bounds expressions requires V8
////////////////////////////////////////////////////////////////////////////////
bool hasV8Expression () const;
////////////////////////////////////////////////////////////////////////////////
/// @brief build the bounds expressions
////////////////////////////////////////////////////////////////////////////////
void buildExpressions ();
////////////////////////////////////////////////////////////////////////////////
/// @brief free _condition if it belongs to us
////////////////////////////////////////////////////////////////////////////////
void freeCondition ();
////////////////////////////////////////////////////////////////////////////////
/// @brief continue fetching of documents
////////////////////////////////////////////////////////////////////////////////
bool readIndex (size_t atMost);
////////////////////////////////////////////////////////////////////////////////
/// @brief set up the index for reading. This should be called once per incoming
/// block.
////////////////////////////////////////////////////////////////////////////////
bool initRanges ();
////////////////////////////////////////////////////////////////////////////////
/// @brief read using the primary index
////////////////////////////////////////////////////////////////////////////////
void readPrimaryIndex (IndexOrCondition const&);
////////////////////////////////////////////////////////////////////////////////
/// @brief destroy the hash index search value
////////////////////////////////////////////////////////////////////////////////
void destroyHashIndexSearchValues ();
////////////////////////////////////////////////////////////////////////////////
/// @brief set up a hash index search value
////////////////////////////////////////////////////////////////////////////////
bool setupHashIndexSearchValue (IndexAndCondition const&);
////////////////////////////////////////////////////////////////////////////////
/// @brief produce a reentrant hash index iterator
////////////////////////////////////////////////////////////////////////////////
void getHashIndexIterator (IndexAndCondition const&);
////////////////////////////////////////////////////////////////////////////////
/// @brief read using a hash index
////////////////////////////////////////////////////////////////////////////////
void readHashIndex (size_t);
////////////////////////////////////////////////////////////////////////////////
/// @brief this tries to create an edge iterator to read from the index.
////////////////////////////////////////////////////////////////////////////////
void getEdgeIndexIterator (IndexAndCondition const&);
////////////////////////////////////////////////////////////////////////////////
/// @brief read using an edge index
////////////////////////////////////////////////////////////////////////////////
void readEdgeIndex (size_t atMost);
////////////////////////////////////////////////////////////////////////////////
/// @brief this tries to create a skiplistIterator to read from the index.
////////////////////////////////////////////////////////////////////////////////
void getSkiplistIterator (IndexAndCondition const&);
////////////////////////////////////////////////////////////////////////////////
/// @brief read using a skiplist index
////////////////////////////////////////////////////////////////////////////////
void readSkiplistIndex (size_t atMost);
////////////////////////////////////////////////////////////////////////////////
// @brief: sorts the index range conditions and resets _posInRanges to 0
////////////////////////////////////////////////////////////////////////////////
void sortConditions ();
////////////////////////////////////////////////////////////////////////////////
/// @brief andCombineRangeInfoVecs: combine the arguments into a single vector,
/// by intersecting every pair of range infos and inserting them in the returned
/// value if the intersection is valid.
////////////////////////////////////////////////////////////////////////////////
std::vector<RangeInfo> andCombineRangeInfoVecs (std::vector<RangeInfo> const&,
std::vector<RangeInfo> const&) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief cartesian: form the cartesian product of the inner vectors. This is
/// required in case a dynamic bound evaluates to a list, then we have an
/// "and" condition containing an "or" condition, which we must then distribute.
////////////////////////////////////////////////////////////////////////////////
IndexOrCondition* cartesian (std::vector<std::vector<RangeInfo>> const&) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief: subclass for comparing IndexAndConditions in _condition. Similar to
/// OurLessThan in the SortBlock
////////////////////////////////////////////////////////////////////////////////
class SortFunc {
public:
SortFunc (std::vector<std::vector<size_t>> const& prefix,
IndexOrCondition* condition,
bool reverse)
: _prefix(prefix),
_condition(condition),
_reverse(reverse) {
}
bool operator() (size_t const&,
size_t const&) const;
private:
std::vector<std::vector<size_t>> const& _prefix;
IndexOrCondition* _condition;
bool const _reverse;
};
// -----------------------------------------------------------------------------
// --SECTION-- private variables
// -----------------------------------------------------------------------------
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief collection
////////////////////////////////////////////////////////////////////////////////
Collection const* _collection;
////////////////////////////////////////////////////////////////////////////////
/// @brief document buffer
////////////////////////////////////////////////////////////////////////////////
std::vector<TRI_doc_mptr_copy_t> _documents;
////////////////////////////////////////////////////////////////////////////////
/// @brief current position in _allDocs
////////////////////////////////////////////////////////////////////////////////
size_t _posInDocs;
////////////////////////////////////////////////////////////////////////////////
/// @brief _allBoundsConstant, this indicates whether all given bounds
/// are constant
////////////////////////////////////////////////////////////////////////////////
std::vector<bool> _allBoundsConstant;
bool _anyBoundVariable;
////////////////////////////////////////////////////////////////////////////////
/// @brief _allBoundsConstant, this indicates whether all given bounds
/// are constant
////////////////////////////////////////////////////////////////////////////////
std::vector<Expression*> _allVariableBoundExpressions;
////////////////////////////////////////////////////////////////////////////////
/// @brief _inVars, a vector containing for each expression above
/// a vector of Variable*, used to execute the expression
/////////////////////////////////////////////////////////////////////////////////
std::vector<std::vector<Variable const*>> _inVars;
////////////////////////////////////////////////////////////////////////////////
/// @brief _inRegs, a vector containing for each expression above
/// a vector of RegisterId, used to execute the expression
////////////////////////////////////////////////////////////////////////////////
std::vector<std::vector<RegisterId>> _inRegs;
////////////////////////////////////////////////////////////////////////////////
/// @brief _skiplistIterator: holds the skiplist iterator found using
/// getSkiplistIterator (if any) so that it can be read in chunks and not
/// necessarily all at once.
////////////////////////////////////////////////////////////////////////////////
TRI_skiplist_iterator_t* _skiplistIterator;
////////////////////////////////////////////////////////////////////////////////
/// @brief _edgeIterator: holds the edge iterator found using
/// getEdgeIndexIterator (if any) so that it can be read in chunks and not
/// necessarily all at once.
////////////////////////////////////////////////////////////////////////////////
struct TRI_edge_index_iterator_t* _edgeIndexIterator;
////////////////////////////////////////////////////////////////////////////////
/// @brief current search value for hash index lookup
////////////////////////////////////////////////////////////////////////////////
TRI_index_search_value_t _hashIndexSearchValue;
////////////////////////////////////////////////////////////////////////////////
/// @brief reentrant hash index iterator state
////////////////////////////////////////////////////////////////////////////////
TRI_index_element_t* _hashNextElement;
////////////////////////////////////////////////////////////////////////////////
/// @brief reentrant edge index iterator state
////////////////////////////////////////////////////////////////////////////////
void* _edgeNextElement;
////////////////////////////////////////////////////////////////////////////////
/// @brief _condition: holds the IndexAndCondition for the current incoming block,
/// this is just the _ranges[_rangesPos] member of the plan node if _allBoundsConstant
/// otherwise it is reevaluated every time initIndex is called, i.e. once per
/// incoming block.
////////////////////////////////////////////////////////////////////////////////
IndexOrCondition* _condition;
////////////////////////////////////////////////////////////////////////////////
/// @brief _flag: since readIndex for primary, hash, edges indexes reads the
/// whole index, this is <true> if initIndex has been called but readIndex has
/// not been called, otherwise it is <false> to avoid rereading the entire index
/// with successive calls to readIndex.
//////////////////////////////////////////////////////////////////////////////////
bool _flag;
size_t _posInRanges;
std::vector<size_t> _sortCoords;
////////////////////////////////////////////////////////////////////////////////
/// @brief _freeCondition: whether or not the _condition is owned by the
/// IndexRangeBlock and must be freed
////////////////////////////////////////////////////////////////////////////////
bool _freeCondition;
bool _hasV8Expression;
};
// -----------------------------------------------------------------------------
// --SECTION-- EnumerateListBlock
// -----------------------------------------------------------------------------
class EnumerateListBlock : public ExecutionBlock {
public:
EnumerateListBlock (ExecutionEngine*,
EnumerateListNode const*);
~EnumerateListBlock ();
int initialize () override;
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor, here we release our docs from this collection
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
AqlItemBlock* getSome (size_t atLeast, size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
// skip between atLeast and atMost returns the number actually skipped . . .
// will only return less than atLeast if there aren't atLeast many
// things to skip overall.
////////////////////////////////////////////////////////////////////////////////
size_t skipSome (size_t atLeast, size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief create an AqlValue from the inVariable using the current _index
////////////////////////////////////////////////////////////////////////////////
// -----------------------------------------------------------------------------
// --SECTION-- private functions
// -----------------------------------------------------------------------------
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief create an AqlValue from the inVariable using the current _index
////////////////////////////////////////////////////////////////////////////////
AqlValue getAqlValue (AqlValue const&);
////////////////////////////////////////////////////////////////////////////////
/// @brief throws an "array expected" exception
////////////////////////////////////////////////////////////////////////////////
void throwArrayExpectedException ();
// -----------------------------------------------------------------------------
// --SECTION-- private variables
// -----------------------------------------------------------------------------
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief current position in the _inVariable
////////////////////////////////////////////////////////////////////////////////
size_t _index;
////////////////////////////////////////////////////////////////////////////////
/// @brief current block in DOCVEC
////////////////////////////////////////////////////////////////////////////////
size_t _thisblock;
////////////////////////////////////////////////////////////////////////////////
/// @brief number of elements in DOCVEC before the current block
////////////////////////////////////////////////////////////////////////////////
size_t _seen;
////////////////////////////////////////////////////////////////////////////////
/// @brief total number of elements in DOCVEC
////////////////////////////////////////////////////////////////////////////////
size_t _DOCVECsize;
////////////////////////////////////////////////////////////////////////////////
/// @brief document collection from DOCVEC
////////////////////////////////////////////////////////////////////////////////
TRI_document_collection_t const* _collection;
////////////////////////////////////////////////////////////////////////////////
/// @brief the register index containing the inVariable of the EnumerateListNode
////////////////////////////////////////////////////////////////////////////////
RegisterId _inVarRegId;
};
// -----------------------------------------------------------------------------
// --SECTION-- CalculationBlock
// -----------------------------------------------------------------------------
class CalculationBlock : public ExecutionBlock {
public:
CalculationBlock (ExecutionEngine*,
CalculationNode const*);
~CalculationBlock ();
int initialize () override;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief fill the target register in the item block with a reference to
/// another variable
////////////////////////////////////////////////////////////////////////////////
void fillBlockWithReference (AqlItemBlock*);
////////////////////////////////////////////////////////////////////////////////
/// @brief shared code for executing a simple or a V8 expression
////////////////////////////////////////////////////////////////////////////////
void executeExpression (AqlItemBlock*);
////////////////////////////////////////////////////////////////////////////////
/// @brief doEvaluation, private helper to do the work
////////////////////////////////////////////////////////////////////////////////
void doEvaluation (AqlItemBlock*);
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast,
size_t atMost) override final;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief we hold a pointer to the expression in the plan
////////////////////////////////////////////////////////////////////////////////
Expression* _expression;
////////////////////////////////////////////////////////////////////////////////
/// @brief info about input variables
////////////////////////////////////////////////////////////////////////////////
std::vector<Variable const*> _inVars;
////////////////////////////////////////////////////////////////////////////////
/// @brief info about input registers
////////////////////////////////////////////////////////////////////////////////
std::vector<RegisterId> _inRegs;
////////////////////////////////////////////////////////////////////////////////
/// @brief output register
////////////////////////////////////////////////////////////////////////////////
RegisterId _outReg;
////////////////////////////////////////////////////////////////////////////////
/// @brief condition variable register
////////////////////////////////////////////////////////////////////////////////
RegisterId _conditionReg;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the expression is a simple variable reference
////////////////////////////////////////////////////////////////////////////////
bool _isReference;
};
// -----------------------------------------------------------------------------
// --SECTION-- SubqueryBlock
// -----------------------------------------------------------------------------
class SubqueryBlock : public ExecutionBlock {
public:
SubqueryBlock (ExecutionEngine*,
SubqueryNode const*,
ExecutionBlock*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~SubqueryBlock ();
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize, tell dependency and the subquery
////////////////////////////////////////////////////////////////////////////////
int initialize () override;
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast,
size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown, tell dependency and the subquery
////////////////////////////////////////////////////////////////////////////////
int shutdown (int errorCode) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief getter for the pointer to the subquery
////////////////////////////////////////////////////////////////////////////////
ExecutionBlock* getSubquery() {
return _subquery;
}
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief execute the subquery and return its results
////////////////////////////////////////////////////////////////////////////////
std::vector<AqlItemBlock*>* executeSubquery ();
////////////////////////////////////////////////////////////////////////////////
/// @brief destroy the results of a subquery
////////////////////////////////////////////////////////////////////////////////
void destroySubqueryResults (std::vector<AqlItemBlock*>*);
////////////////////////////////////////////////////////////////////////////////
/// @brief output register
////////////////////////////////////////////////////////////////////////////////
RegisterId _outReg;
////////////////////////////////////////////////////////////////////////////////
/// @brief we need to have an executionblock and where to write the result
////////////////////////////////////////////////////////////////////////////////
ExecutionBlock* _subquery;
};
// -----------------------------------------------------------------------------
// --SECTION-- FilterBlock
// -----------------------------------------------------------------------------
class FilterBlock : public ExecutionBlock {
public:
FilterBlock (ExecutionEngine*,
FilterNode const*);
~FilterBlock ();
int initialize () override final;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief internal function to actually decide if the document should be used
////////////////////////////////////////////////////////////////////////////////
inline bool takeItem (AqlItemBlock* items, size_t index) const {
return items->getValueReference(index, _inReg).isTrue();
}
////////////////////////////////////////////////////////////////////////////////
/// @brief internal function to get another block
////////////////////////////////////////////////////////////////////////////////
bool getBlock (size_t atLeast, size_t atMost);
int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped) override;
bool hasMore () override final;
int64_t count () const override final {
return -1; // refuse to work
}
int64_t remaining () override final {
return -1; // refuse to work
}
////////////////////////////////////////////////////////////////////////////////
/// @brief input register
////////////////////////////////////////////////////////////////////////////////
private:
RegisterId _inReg;
////////////////////////////////////////////////////////////////////////////////
/// @brief vector of indices of those documents in the current block
/// that are chosen
////////////////////////////////////////////////////////////////////////////////
std::vector<size_t> _chosen;
};
// -----------------------------------------------------------------------------
// --SECTION-- SortedAggregateBlock
// -----------------------------------------------------------------------------
class SortedAggregateBlock : public ExecutionBlock {
public:
SortedAggregateBlock (ExecutionEngine*,
AggregateNode const*);
~SortedAggregateBlock ();
int initialize () override;
private:
int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief writes the current group data into the result
////////////////////////////////////////////////////////////////////////////////
void emitGroup (AqlItemBlock const* cur,
AqlItemBlock* res,
size_t row);
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief pairs, consisting of out register and in register
////////////////////////////////////////////////////////////////////////////////
std::vector<std::pair<RegisterId, RegisterId>> _aggregateRegisters;
////////////////////////////////////////////////////////////////////////////////
/// @brief details about the current group
////////////////////////////////////////////////////////////////////////////////
AggregatorGroup _currentGroup;
////////////////////////////////////////////////////////////////////////////////
/// @brief the optional register that contains the input expression values for
/// each group
////////////////////////////////////////////////////////////////////////////////
RegisterId _expressionRegister;
////////////////////////////////////////////////////////////////////////////////
/// @brief the optional register that contains the values for each group
/// if no values should be returned, then this has a value of MaxRegisterId
/// this register is also used for counting in case WITH COUNT INTO var is used
////////////////////////////////////////////////////////////////////////////////
RegisterId _groupRegister;
////////////////////////////////////////////////////////////////////////////////
/// @brief list of variables names for the registers
////////////////////////////////////////////////////////////////////////////////
std::vector<std::string> _variableNames;
};
// -----------------------------------------------------------------------------
// --SECTION-- HashedAggregateBlock
// -----------------------------------------------------------------------------
class HashedAggregateBlock : public ExecutionBlock {
public:
HashedAggregateBlock (ExecutionEngine*,
AggregateNode const*);
~HashedAggregateBlock ();
int initialize () override;
private:
int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped) override;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief pairs, consisting of out register and in register
////////////////////////////////////////////////////////////////////////////////
std::vector<std::pair<RegisterId, RegisterId>> _aggregateRegisters;
////////////////////////////////////////////////////////////////////////////////
/// @brief the optional register that contains the values for each group
/// if no values should be returned, then this has a value of MaxRegisterId
/// this register is also used for counting in case WITH COUNT INTO var is used
////////////////////////////////////////////////////////////////////////////////
RegisterId _groupRegister;
////////////////////////////////////////////////////////////////////////////////
/// @brief hasher for a vector of AQL values
////////////////////////////////////////////////////////////////////////////////
struct GroupKeyHash {
GroupKeyHash (triagens::arango::AqlTransaction* trx,
std::vector<TRI_document_collection_t const*>& colls)
: _trx(trx),
_colls(colls),
_num(colls.size()) {
}
size_t operator() (std::vector<AqlValue> const& value) const;
triagens::arango::AqlTransaction* _trx;
std::vector<TRI_document_collection_t const*>& _colls;
size_t const _num;
};
////////////////////////////////////////////////////////////////////////////////
/// @brief comparator for a vector of AQL values
////////////////////////////////////////////////////////////////////////////////
struct GroupKeyEqual {
GroupKeyEqual (triagens::arango::AqlTransaction* trx,
std::vector<TRI_document_collection_t const*>& colls)
: _trx(trx),
_colls(colls) {
}
bool operator() (std::vector<AqlValue> const&,
std::vector<AqlValue> const&) const;
triagens::arango::AqlTransaction* _trx;
std::vector<TRI_document_collection_t const*>& _colls;
};
};
// -----------------------------------------------------------------------------
// --SECTION-- SortBlock
// -----------------------------------------------------------------------------
class SortBlock : public ExecutionBlock {
public:
SortBlock (ExecutionEngine*,
SortNode const*);
~SortBlock ();
int initialize () override;
int initializeCursor (AqlItemBlock* items, size_t pos) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief dosorting
////////////////////////////////////////////////////////////////////////////////
private:
void doSorting ();
////////////////////////////////////////////////////////////////////////////////
/// @brief OurLessThan
////////////////////////////////////////////////////////////////////////////////
class OurLessThan {
public:
OurLessThan (triagens::arango::AqlTransaction* trx,
std::deque<AqlItemBlock*>& buffer,
std::vector<std::pair<RegisterId, bool>>& sortRegisters,
std::vector<TRI_document_collection_t const*>& colls)
: _trx(trx),
_buffer(buffer),
_sortRegisters(sortRegisters),
_colls(colls) {
}
bool operator() (std::pair<size_t, size_t> const& a,
std::pair<size_t, size_t> const& b);
private:
triagens::arango::AqlTransaction* _trx;
std::deque<AqlItemBlock*>& _buffer;
std::vector<std::pair<RegisterId, bool>>& _sortRegisters;
std::vector<TRI_document_collection_t const*>& _colls;
};
////////////////////////////////////////////////////////////////////////////////
/// @brief pairs, consisting of variable and sort direction
/// (true = ascending | false = descending)
////////////////////////////////////////////////////////////////////////////////
std::vector<std::pair<RegisterId, bool>> _sortRegisters;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the sort should be stable
////////////////////////////////////////////////////////////////////////////////
bool _stable;
};
// -----------------------------------------------------------------------------
// --SECTION-- LimitBlock
// -----------------------------------------------------------------------------
class LimitBlock : public ExecutionBlock {
public:
LimitBlock (ExecutionEngine* engine,
LimitNode const* ep)
: ExecutionBlock(engine, ep),
_offset(ep->_offset),
_limit(ep->_limit),
_state(0), // start in the beginning
_fullCount(ep->_fullCount) {
}
~LimitBlock () {
}
int initialize () override;
int initializeCursor (AqlItemBlock* items, size_t pos) override final;
virtual int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief _offset
////////////////////////////////////////////////////////////////////////////////
size_t _offset;
////////////////////////////////////////////////////////////////////////////////
/// @brief _limit
////////////////////////////////////////////////////////////////////////////////
size_t _limit;
////////////////////////////////////////////////////////////////////////////////
/// @brief _count, number of items already handed on
////////////////////////////////////////////////////////////////////////////////
size_t _count;
////////////////////////////////////////////////////////////////////////////////
/// @brief _state, 0 is beginning, 1 is after offset, 2 is done
////////////////////////////////////////////////////////////////////////////////
int _state;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the block should count what it limits
////////////////////////////////////////////////////////////////////////////////
bool const _fullCount;
};
// -----------------------------------------------------------------------------
// --SECTION-- ReturnBlock
// -----------------------------------------------------------------------------
class ReturnBlock : public ExecutionBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
ReturnBlock (ExecutionEngine* engine,
ReturnNode const* ep)
: ExecutionBlock(engine, ep),
_returnInheritedResults(false) {
}
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~ReturnBlock () {
}
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast,
size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief make the return block return the results inherited from above,
/// without creating new blocks
/// returns the id of the register the final result can be found in
////////////////////////////////////////////////////////////////////////////////
RegisterId returnInheritedResults ();
// -----------------------------------------------------------------------------
// --SECTION-- private variables
// -----------------------------------------------------------------------------
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief if set to true, the return block will return the AqlItemBlocks it
/// gets from above directly. if set to false, the return block will create a
/// new AqlItemBlock with one output register and copy the data from its input
/// block into it
////////////////////////////////////////////////////////////////////////////////
bool _returnInheritedResults;
};
// -----------------------------------------------------------------------------
// --SECTION-- ModificationBlock
// -----------------------------------------------------------------------------
class ModificationBlock : public ExecutionBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
ModificationBlock (ExecutionEngine*,
ModificationNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
virtual ~ModificationBlock ();
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast,
size_t atMost) override final;
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief the actual work horse
////////////////////////////////////////////////////////////////////////////////
virtual AqlItemBlock* work (std::vector<AqlItemBlock*>&) = 0;
////////////////////////////////////////////////////////////////////////////////
/// @brief extract a key from the AqlValue passed
////////////////////////////////////////////////////////////////////////////////
int extractKey (AqlValue const&,
TRI_document_collection_t const*,
std::string&);
////////////////////////////////////////////////////////////////////////////////
/// @brief constructs a master pointer from the marker passed
////////////////////////////////////////////////////////////////////////////////
void constructMptr (TRI_doc_mptr_copy_t*,
TRI_df_marker_s const*) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief check whether a shard key value has changed
////////////////////////////////////////////////////////////////////////////////
bool isShardKeyChange (struct TRI_json_t const*,
struct TRI_json_t const*,
bool) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief check whether a shard key was set when it must not be set
////////////////////////////////////////////////////////////////////////////////
bool isShardKeyError (struct TRI_json_t const*) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief process the result of a data-modification operation
////////////////////////////////////////////////////////////////////////////////
void handleResult (int,
bool,
std::string const *errorMessage = nullptr);
// -----------------------------------------------------------------------------
// --SECTION-- protected variables
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief output register ($OLD)
////////////////////////////////////////////////////////////////////////////////
RegisterId _outRegOld;
////////////////////////////////////////////////////////////////////////////////
/// @brief output register ($NEW)
////////////////////////////////////////////////////////////////////////////////
RegisterId _outRegNew;
////////////////////////////////////////////////////////////////////////////////
/// @brief collection
////////////////////////////////////////////////////////////////////////////////
Collection const* _collection;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not we're a DB server in a cluster
////////////////////////////////////////////////////////////////////////////////
bool _isDBServer;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the collection uses the default sharding attributes
////////////////////////////////////////////////////////////////////////////////
bool _usesDefaultSharding;
////////////////////////////////////////////////////////////////////////////////
/// @brief temporary string buffer for extracting system attributes
////////////////////////////////////////////////////////////////////////////////
triagens::basics::StringBuffer _buffer;
};
// -----------------------------------------------------------------------------
// --SECTION-- RemoveBlock
// -----------------------------------------------------------------------------
class RemoveBlock : public ModificationBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
RemoveBlock (ExecutionEngine*,
RemoveNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~RemoveBlock ();
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief the actual work horse for removing data
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* work (std::vector<AqlItemBlock*>&) override final;
};
// -----------------------------------------------------------------------------
// --SECTION-- InsertBlock
// -----------------------------------------------------------------------------
class InsertBlock : public ModificationBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
InsertBlock (ExecutionEngine*,
InsertNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~InsertBlock ();
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief the actual work horse for inserting data
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* work (std::vector<AqlItemBlock*>&) override final;
};
// -----------------------------------------------------------------------------
// --SECTION-- UpdateBlock
// -----------------------------------------------------------------------------
class UpdateBlock : public ModificationBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
UpdateBlock (ExecutionEngine*,
UpdateNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~UpdateBlock ();
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief the actual work horse for updating data
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* work (std::vector<AqlItemBlock*>&) override final;
};
// -----------------------------------------------------------------------------
// --SECTION-- ReplaceBlock
// -----------------------------------------------------------------------------
class ReplaceBlock : public ModificationBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
ReplaceBlock (ExecutionEngine*,
ReplaceNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~ReplaceBlock ();
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief the actual work horse for replacing data
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* work (std::vector<AqlItemBlock*>&) override final;
};
// -----------------------------------------------------------------------------
// --SECTION-- UpsertBlock
// -----------------------------------------------------------------------------
class UpsertBlock : public ModificationBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
UpsertBlock (ExecutionEngine*,
UpsertNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~UpsertBlock ();
// -----------------------------------------------------------------------------
// --SECTION-- protected methods
// -----------------------------------------------------------------------------
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief the actual work horse for updating data
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* work (std::vector<AqlItemBlock*>&) override final;
};
// -----------------------------------------------------------------------------
// --SECTION-- NoResultsBlock
// -----------------------------------------------------------------------------
class NoResultsBlock : public ExecutionBlock {
public:
NoResultsBlock (ExecutionEngine* engine,
NoResultsNode const* ep)
: ExecutionBlock(engine, ep) {
}
~NoResultsBlock () {
}
int initialize () override {
return ExecutionBlock::initialize();
}
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor, store a copy of the register values coming from above
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override final;
bool hasMore () override final {
return false;
}
int64_t count () const override final {
return 0;
}
int64_t remaining () override final {
return 0;
}
private:
int getOrSkipSome (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped) override;
};
// -----------------------------------------------------------------------------
// --SECTION-- GatherBlock
// -----------------------------------------------------------------------------
class GatherBlock : public ExecutionBlock {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
GatherBlock (ExecutionEngine*,
GatherNode const*);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~GatherBlock ();
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize
////////////////////////////////////////////////////////////////////////////////
int initialize () override;
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown: need our own method since our _buffer is different
////////////////////////////////////////////////////////////////////////////////
int shutdown (int) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief count: the sum of the count() of the dependencies or -1 (if any
/// dependency has count -1
////////////////////////////////////////////////////////////////////////////////
int64_t count () const override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief remaining: the sum of the remaining() of the dependencies or -1 (if
/// any dependency has remaining -1
////////////////////////////////////////////////////////////////////////////////
int64_t remaining () override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief hasMore: true if any position of _buffer hasMore and false
/// otherwise.
////////////////////////////////////////////////////////////////////////////////
bool hasMore () override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t, size_t) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief skipSome
////////////////////////////////////////////////////////////////////////////////
size_t skipSome (size_t, size_t) override final;
protected:
////////////////////////////////////////////////////////////////////////////////
/// @brief getBlock: from dependency i into _gatherBlockBuffer.at(i),
/// non-simple case only
////////////////////////////////////////////////////////////////////////////////
bool getBlock (size_t i, size_t atLeast, size_t atMost);
////////////////////////////////////////////////////////////////////////////////
/// @brief _gatherBlockBuffer: buffer the incoming block from each dependency
/// separately
////////////////////////////////////////////////////////////////////////////////
std::vector<std::deque<AqlItemBlock*>> _gatherBlockBuffer;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief _gatherBlockPos: pairs (i, _pos in _buffer.at(i)), i.e. the same as
/// the usual _pos but one pair per dependency
////////////////////////////////////////////////////////////////////////////////
std::vector<std::pair<size_t, size_t>> _gatherBlockPos;
////////////////////////////////////////////////////////////////////////////////
/// @brief _atDep: currently pulling blocks from _dependencies.at(_atDep),
/// simple case only
////////////////////////////////////////////////////////////////////////////////
size_t _atDep = 0;
////////////////////////////////////////////////////////////////////////////////
/// @brief pairs, consisting of variable and sort direction
/// (true = ascending | false = descending)
////////////////////////////////////////////////////////////////////////////////
std::vector<std::pair<RegisterId, bool>> _sortRegisters;
////////////////////////////////////////////////////////////////////////////////
/// @brief isSimple: the block is simple if we do not do merge sort . . .
////////////////////////////////////////////////////////////////////////////////
bool const _isSimple;
////////////////////////////////////////////////////////////////////////////////
/// @brief OurLessThan: comparison method for elements of _gatherBlockPos
////////////////////////////////////////////////////////////////////////////////
class OurLessThan {
public:
OurLessThan (triagens::arango::AqlTransaction* trx,
std::vector<std::deque<AqlItemBlock*>>& gatherBlockBuffer,
std::vector<std::pair<RegisterId, bool>>& sortRegisters,
std::vector<TRI_document_collection_t const*>& colls)
: _trx(trx),
_gatherBlockBuffer(gatherBlockBuffer),
_sortRegisters(sortRegisters),
_colls(colls) {
}
bool operator() (std::pair<size_t, size_t> const& a,
std::pair<size_t, size_t> const& b);
private:
triagens::arango::AqlTransaction* _trx;
std::vector<std::deque<AqlItemBlock*>>& _gatherBlockBuffer;
std::vector<std::pair<RegisterId, bool>>& _sortRegisters;
std::vector<TRI_document_collection_t const*>& _colls;
};
};
// -----------------------------------------------------------------------------
// --SECTION-- BlockWithClients
// -----------------------------------------------------------------------------
class BlockWithClients : public ExecutionBlock {
public:
BlockWithClients (ExecutionEngine* engine,
ExecutionNode const* ep,
std::vector<std::string> const& shardIds);
virtual ~BlockWithClients () {}
// -----------------------------------------------------------------------------
// --SECTION-- BlockWithClients public methods
// -----------------------------------------------------------------------------
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown
////////////////////////////////////////////////////////////////////////////////
int shutdown (int) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome: shouldn't be used, use skipSomeForShard
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast, size_t atMost) override final {
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION(TRI_ERROR_NOT_IMPLEMENTED);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief skipSome: shouldn't be used, use skipSomeForShard
////////////////////////////////////////////////////////////////////////////////
size_t skipSome (size_t atLeast, size_t atMost) override final {
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION(TRI_ERROR_NOT_IMPLEMENTED);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief remaining
////////////////////////////////////////////////////////////////////////////////
int64_t remaining () override final {
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION(TRI_ERROR_NOT_IMPLEMENTED);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief hasMore
////////////////////////////////////////////////////////////////////////////////
bool hasMore () override final {
TRI_ASSERT(false);
THROW_ARANGO_EXCEPTION(TRI_ERROR_NOT_IMPLEMENTED);
}
////////////////////////////////////////////////////////////////////////////////
/// @brief getSomeForShard
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSomeForShard (size_t atLeast, size_t atMost,
std::string const& shardId);
////////////////////////////////////////////////////////////////////////////////
/// @brief skipSomeForShard
////////////////////////////////////////////////////////////////////////////////
size_t skipSomeForShard (size_t atLeast, size_t atMost, std::string
const& shardId);
////////////////////////////////////////////////////////////////////////////////
/// @brief skipForShard
////////////////////////////////////////////////////////////////////////////////
bool skipForShard (size_t number, std::string const& shardId);
////////////////////////////////////////////////////////////////////////////////
/// @brief hasMoreForShard: any more for shard <shardId>?
////////////////////////////////////////////////////////////////////////////////
virtual bool hasMoreForShard (std::string const& shardId) = 0;
////////////////////////////////////////////////////////////////////////////////
/// @brief remainingForShard: remaining for shard <shardId>?
////////////////////////////////////////////////////////////////////////////////
virtual int64_t remainingForShard (std::string const& shardId) = 0;
protected:
// -----------------------------------------------------------------------------
// --SECTION-- BlockWithClients protected methods
// -----------------------------------------------------------------------------
////////////////////////////////////////////////////////////////////////////////
/// @brief getOrSkipSomeForShard
////////////////////////////////////////////////////////////////////////////////
virtual int getOrSkipSomeForShard (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped,
std::string const& shardId) = 0;
////////////////////////////////////////////////////////////////////////////////
/// @brief getClientId: get the number <clientId> (used internally)
/// corresponding to <shardId>
////////////////////////////////////////////////////////////////////////////////
size_t getClientId (std::string const& shardId);
// -----------------------------------------------------------------------------
// --SECTION-- BlockWithClients protected data
// -----------------------------------------------------------------------------
////////////////////////////////////////////////////////////////////////////////
/// @brief _shardIdMap: map from shardIds to clientNrs
////////////////////////////////////////////////////////////////////////////////
std::unordered_map<std::string, size_t> _shardIdMap;
////////////////////////////////////////////////////////////////////////////////
/// @brief _nrClients: total number of clients
////////////////////////////////////////////////////////////////////////////////
size_t _nrClients;
////////////////////////////////////////////////////////////////////////////////
/// @brief _doneForClient: the analogue of _done: _doneForClient.at(i) = true
/// if we are done for the shard with clientId = i
////////////////////////////////////////////////////////////////////////////////
std::vector<bool> _doneForClient;
////////////////////////////////////////////////////////////////////////////////
/// @brief _ignoreInitCursor: should we really initialiseCursor?
////////////////////////////////////////////////////////////////////////////////
bool _ignoreInitCursor;
////////////////////////////////////////////////////////////////////////////////
/// @brief _shutdown: should we really shutdown?
////////////////////////////////////////////////////////////////////////////////
bool _ignoreShutdown;
};
// -----------------------------------------------------------------------------
// --SECTION-- ScatterBlock
// -----------------------------------------------------------------------------
class ScatterBlock : public BlockWithClients {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
ScatterBlock (ExecutionEngine* engine,
ScatterNode const* ep,
std::vector<std::string> const& shardIds)
: BlockWithClients(engine, ep, shardIds) {
}
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~ScatterBlock () {
}
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown
////////////////////////////////////////////////////////////////////////////////
int shutdown (int) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief hasMoreForShard: any more for shard <shardId>?
////////////////////////////////////////////////////////////////////////////////
bool hasMoreForShard (std::string const& shardId) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief remainingForShard: remaining for shard <shardId>?
////////////////////////////////////////////////////////////////////////////////
int64_t remainingForShard (std::string const& shardId) override;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief getOrSkipSomeForShard
////////////////////////////////////////////////////////////////////////////////
int getOrSkipSomeForShard (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped,
std::string const& shardId) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief _posForClient:
////////////////////////////////////////////////////////////////////////////////
std::vector<std::pair<size_t, size_t>> _posForClient;
};
// -----------------------------------------------------------------------------
// --SECTION-- DistributeBlock
// -----------------------------------------------------------------------------
class DistributeBlock : public BlockWithClients {
public:
////////////////////////////////////////////////////////////////////////////////
/// @brief constructor
////////////////////////////////////////////////////////////////////////////////
DistributeBlock (ExecutionEngine* engine,
DistributeNode const* ep,
std::vector<std::string> const& shardIds,
Collection const* collection);
////////////////////////////////////////////////////////////////////////////////
/// @brief destructor
////////////////////////////////////////////////////////////////////////////////
~DistributeBlock () {
}
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown
////////////////////////////////////////////////////////////////////////////////
int shutdown (int) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief remainingForShard: remaining for shard <shardId>?
////////////////////////////////////////////////////////////////////////////////
int64_t remainingForShard (std::string const& shardId) override {
return -1;
}
////////////////////////////////////////////////////////////////////////////////
/// @brief hasMoreForShard: any more for shard <shardId>?
////////////////////////////////////////////////////////////////////////////////
bool hasMoreForShard (std::string const& shardId) override;
private:
////////////////////////////////////////////////////////////////////////////////
/// @brief getOrSkipSomeForShard
////////////////////////////////////////////////////////////////////////////////
int getOrSkipSomeForShard (size_t atLeast,
size_t atMost,
bool skipping,
AqlItemBlock*& result,
size_t& skipped,
std::string const& shardId) override;
////////////////////////////////////////////////////////////////////////////////
/// @brief getBlockForClient: try to get at atLeast/atMost pairs into
/// _distBuffer.at(clientId).
////////////////////////////////////////////////////////////////////////////////
bool getBlockForClient (size_t atLeast,
size_t atMost,
size_t clientId);
////////////////////////////////////////////////////////////////////////////////
/// @brief return the JSON that is used to determine the initial shard
////////////////////////////////////////////////////////////////////////////////
struct TRI_json_t const* getInputJson (AqlItemBlock const*) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief sendToClient: for each row of the incoming AqlItemBlock use the
/// attributes <shardKeys> of the register <id> to determine to which shard the
/// row should be sent.
////////////////////////////////////////////////////////////////////////////////
size_t sendToClient (AqlItemBlock*);
////////////////////////////////////////////////////////////////////////////////
/// @brief create a new document key
////////////////////////////////////////////////////////////////////////////////
std::string createKey () const;
////////////////////////////////////////////////////////////////////////////////
/// @brief _distBuffer.at(i) is a deque containing pairs (j,k) such that
// _buffer.at(j) row k should be sent to the client with id = i.
////////////////////////////////////////////////////////////////////////////////
std::vector<std::deque<std::pair<size_t, size_t>>> _distBuffer;
////////////////////////////////////////////////////////////////////////////////
/// @brief _colectionName: the name of the sharded collection
////////////////////////////////////////////////////////////////////////////////
Collection const* _collection;
////////////////////////////////////////////////////////////////////////////////
/// @brief _index: the block in _buffer we are currently considering
////////////////////////////////////////////////////////////////////////////////
size_t _index;
////////////////////////////////////////////////////////////////////////////////
/// @brief _regId: the register to inspect
////////////////////////////////////////////////////////////////////////////////
RegisterId _regId;
////////////////////////////////////////////////////////////////////////////////
/// @brief a second register to inspect (used only for UPSERT nodes at the
/// moment to distinguish between search and insert)
////////////////////////////////////////////////////////////////////////////////
RegisterId _alternativeRegId;
////////////////////////////////////////////////////////////////////////////////
/// @brief whether or not the collection uses the default sharding
////////////////////////////////////////////////////////////////////////////////
bool _usesDefaultSharding;
};
// -----------------------------------------------------------------------------
// --SECTION-- RemoteBlock
// -----------------------------------------------------------------------------
class RemoteBlock : public ExecutionBlock {
////////////////////////////////////////////////////////////////////////////////
/// @brief constructors/destructors
////////////////////////////////////////////////////////////////////////////////
public:
RemoteBlock (ExecutionEngine* engine,
RemoteNode const* en,
std::string const& server,
std::string const& ownName,
std::string const& queryId);
~RemoteBlock ();
////////////////////////////////////////////////////////////////////////////////
/// @brief timeout
////////////////////////////////////////////////////////////////////////////////
static double const defaultTimeOut;
////////////////////////////////////////////////////////////////////////////////
/// @brief initialize
////////////////////////////////////////////////////////////////////////////////
int initialize () override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief initializeCursor, could be called multiple times
////////////////////////////////////////////////////////////////////////////////
int initializeCursor (AqlItemBlock* items, size_t pos) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief shutdown, will be called exactly once for the whole query
////////////////////////////////////////////////////////////////////////////////
int shutdown (int) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief getSome
////////////////////////////////////////////////////////////////////////////////
AqlItemBlock* getSome (size_t atLeast,
size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief skipSome
////////////////////////////////////////////////////////////////////////////////
size_t skipSome (size_t atLeast, size_t atMost) override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief hasMore
////////////////////////////////////////////////////////////////////////////////
bool hasMore () override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief count
////////////////////////////////////////////////////////////////////////////////
int64_t count () const override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief remaining
////////////////////////////////////////////////////////////////////////////////
int64_t remaining () override final;
////////////////////////////////////////////////////////////////////////////////
/// @brief internal method to send a request
////////////////////////////////////////////////////////////////////////////////
private:
triagens::arango::ClusterCommResult* sendRequest (
rest::HttpRequest::HttpRequestType type,
std::string const& urlPart,
std::string const& body) const;
////////////////////////////////////////////////////////////////////////////////
/// @brief our server, can be like "shard:S1000" or like "server:Claus"
////////////////////////////////////////////////////////////////////////////////
std::string _server;
////////////////////////////////////////////////////////////////////////////////
/// @brief our own identity, in case of the coordinator this is empty,
/// in case of the DBservers, this is the shard ID as a string
////////////////////////////////////////////////////////////////////////////////
std::string _ownName;
////////////////////////////////////////////////////////////////////////////////
/// @brief the ID of the query on the server as a string
////////////////////////////////////////////////////////////////////////////////
std::string _queryId;
////////////////////////////////////////////////////////////////////////////////
/// @brief the ID of the query on the server as a string
////////////////////////////////////////////////////////////////////////////////
ExecutionStats _deltaStats;
};
} // namespace triagens::aql
} // namespace triagens
#endif
// Local Variables:
// mode: outline-minor
// outline-regexp: "^\\(/// @brief\\|/// {@inheritDoc}\\|/// @addtogroup\\|// --SECTION--\\|/// @\\}\\)"
// End: