1
0
Fork 0

Feature/issue #642 (#10352)

* Index late materialization draft

* Index late materialization. Refactoring.

* Index late materialization. Expansion draft.

* HashIndexMock

* Refactoring.

* Hash index mock expansion fixes

* Fix compare of attributes.

* Hash index mock. Last expansion support.

* Hash index mock. Take value for array from document, not equal key.

* Optimizations.

* Query tests

* lateMaterialized improvements

* velocypack fixes and tests for index node

* Fix after merge.

* Const expression in a condition test

* Clone tests

* More tests

* Fix for materialized view (nullptr var)

* Fixed index vars registers count planning

* javascript tests

* Fixed cluster tests

* Cluster support

* MaterializeExecutor and MaterializerExecutorInfos refactoring

* More tests

* js tests rocksdb only

* js test. Removed print()

* js testd added skiplist

* C++ unit tests for invalid json

* js test var i

* js tests == -> ===

* iFix js tests ===

* jslint ;

* int unsigned int fix

* double deletion fix

* refactoring

* Added const for collection source

* override final, added materialize namespace, fixed femove in hash index mock

* Removed expansion support. Index can contain null value in case of several object with same fields

* isNumber<>

* Optimization for single collection case

* Renaming

* refactoring

* Unit tests. Added checking documents

* !sliceIt.valid() instead of !(!=)

* Fix primary index (object instead of array)

* Edge collection test

* Windows compilation fix

* IndexExecutor getCallback optimization

* ADB_UNLIKELY

* explainer.js

* removed unsused variable in js test

* fix explainer.js

* do not use several indexes with js test

* Removed several indexes implementation

* space

* fix explainer.js

* fix explainer.js

* code alignment
This commit is contained in:
Iurii Popov 2019-11-07 19:16:34 +03:00 committed by Andrey Abramov
parent b6a400aec1
commit 26309c5c96
26 changed files with 2792 additions and 126 deletions

View File

@ -374,7 +374,8 @@ static SkipVariants constexpr skipType() {
std::is_same<Executor, ConstrainedSortExecutor>::value ||
std::is_same<Executor, SortingGatherExecutor>::value ||
std::is_same<Executor, UnsortedGatherExecutor>::value ||
std::is_same<Executor, MaterializeExecutor>::value),
std::is_same<Executor, MaterializeExecutor<RegisterId>>::value ||
std::is_same<Executor, MaterializeExecutor<std::string const&>>::value),
"Unexpected executor for SkipVariants::EXECUTOR");
// The LimitExecutor will not work correctly with SkipVariants::FETCHER!
@ -896,7 +897,9 @@ template class ::arangodb::aql::ExecutionBlockImpl<SubqueryStartExecutor>;
template class ::arangodb::aql::ExecutionBlockImpl<TraversalExecutor>;
template class ::arangodb::aql::ExecutionBlockImpl<SortingGatherExecutor>;
template class ::arangodb::aql::ExecutionBlockImpl<UnsortedGatherExecutor>;
template class ::arangodb::aql::ExecutionBlockImpl<MaterializeExecutor>;
template class ::arangodb::aql::ExecutionBlockImpl<MaterializeExecutor<RegisterId>>;
template class ::arangodb::aql::ExecutionBlockImpl<MaterializeExecutor<std::string const&>>;
template class ::arangodb::aql::ExecutionBlockImpl<ModificationExecutor<AllRowsFetcher, InsertModifier>>;
template class ::arangodb::aql::ExecutionBlockImpl<ModificationExecutor<SingleRowFetcher<BlockPassthrough::Disable>, InsertModifier>>;

View File

@ -76,6 +76,7 @@
using namespace arangodb;
using namespace arangodb::aql;
using namespace arangodb::basics;
using namespace materialize;
namespace {
@ -341,7 +342,7 @@ ExecutionNode* ExecutionNode::fromVPackFactory(ExecutionPlan* plan, VPackSlice c
case DISTRIBUTE_CONSUMER:
return new DistributeConsumerNode(plan, slice);
case MATERIALIZE:
return new MaterializeNode(plan, slice);
return createMaterializeNode(plan, slice);
default: {
// should not reach this point
TRI_ASSERT(false);
@ -2334,47 +2335,90 @@ SortInformation::Match SortInformation::isCoveredBy(SortInformation const& other
return allEqual;
}
namespace {
const char* MATERIALIZE_NODE_IN_NM_COL_PARAM = "inNmColPtr";
const char* MATERIALIZE_NODE_IN_NM_DOC_PARAM = "inNmDocId";
const char* MATERIALIZE_NODE_OUT_VARIABLE_PARAM = "outVariable";
}
MaterializeNode::MaterializeNode(ExecutionPlan * plan, size_t id,
aql::Variable const & inColPtr,
aql::Variable const & inDocId,
aql::Variable const & outVariable)
: ExecutionNode(plan, id), _inNonMaterializedColPtr(&inColPtr),
_inNonMaterializedDocId(&inDocId), _outVariable(&outVariable) {}
MaterializeNode* materialize::createMaterializeNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base) {
if (base.hasKey(MATERIALIZE_NODE_IN_NM_COL_PARAM)) {
return new MaterializeMultiNode(plan, base);
}
return new MaterializeSingleNode(plan, base);
}
MaterializeNode::MaterializeNode(ExecutionPlan * plan, arangodb::velocypack::Slice const & base)
MaterializeNode::MaterializeNode(ExecutionPlan* plan, size_t id,
aql::Variable const& inDocId,
aql::Variable const& outVariable)
: ExecutionNode(plan, id), _inNonMaterializedDocId(&inDocId), _outVariable(&outVariable) {}
MaterializeNode::MaterializeNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base)
: ExecutionNode(plan, base),
_inNonMaterializedColPtr(aql::Variable::varFromVPack(plan->getAst(), base, MATERIALIZE_NODE_IN_NM_COL_PARAM, true)),
_inNonMaterializedDocId(aql::Variable::varFromVPack(plan->getAst(), base, MATERIALIZE_NODE_IN_NM_DOC_PARAM, true)),
_outVariable(aql::Variable::varFromVPack(plan->getAst(), base, MATERIALIZE_NODE_OUT_VARIABLE_PARAM)) {}
_inNonMaterializedDocId(aql::Variable::varFromVPack(plan->getAst(), base, MATERIALIZE_NODE_IN_NM_DOC_PARAM, true)),
_outVariable(aql::Variable::varFromVPack(plan->getAst(), base, MATERIALIZE_NODE_OUT_VARIABLE_PARAM)) {}
void MaterializeNode::toVelocyPackHelper(arangodb::velocypack::Builder & nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const {
void MaterializeNode::toVelocyPackHelper(arangodb::velocypack::Builder& nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const {
// call base class method
aql::ExecutionNode::toVelocyPackHelperGeneric(nodes, flags, seen);
nodes.add(VPackValue(MATERIALIZE_NODE_IN_NM_COL_PARAM));
_inNonMaterializedColPtr->toVelocyPack(nodes);
nodes.add(VPackValue(MATERIALIZE_NODE_IN_NM_DOC_PARAM));
_inNonMaterializedDocId->toVelocyPack(nodes);
nodes.add(VPackValue(MATERIALIZE_NODE_OUT_VARIABLE_PARAM));
_outVariable->toVelocyPack(nodes);
}
CostEstimate MaterializeNode::estimateCost() const {
if (_dependencies.empty()) {
// we should always have dependency as we need input for materializing
TRI_ASSERT(false);
return aql::CostEstimate::empty();
}
aql::CostEstimate estimate = _dependencies[0]->getCost();
// we will materialize all output of our dependency
estimate.estimatedCost += estimate.estimatedNrItems;
return estimate;
}
void MaterializeNode::getVariablesUsedHere(::arangodb::containers::HashSet<Variable const*>& vars) const {
vars.emplace(_inNonMaterializedDocId);
}
std::vector<Variable const*> MaterializeNode::getVariablesSetHere() const {
return std::vector<Variable const*>{_outVariable};
}
MaterializeMultiNode::MaterializeMultiNode(ExecutionPlan* plan, size_t id,
aql::Variable const& inColPtr,
aql::Variable const& inDocId,
aql::Variable const& outVariable)
: MaterializeNode(plan, id, inDocId, outVariable),
_inNonMaterializedColPtr(&inColPtr) {}
MaterializeMultiNode::MaterializeMultiNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base)
: MaterializeNode(plan, base),
_inNonMaterializedColPtr(aql::Variable::varFromVPack(plan->getAst(), base, MATERIALIZE_NODE_IN_NM_COL_PARAM, true)) {}
void MaterializeMultiNode::toVelocyPackHelper(arangodb::velocypack::Builder& nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const {
// call base class method
MaterializeNode::toVelocyPackHelper(nodes, flags, seen);
nodes.add(VPackValue(MATERIALIZE_NODE_IN_NM_COL_PARAM));
_inNonMaterializedColPtr->toVelocyPack(nodes);
nodes.close();
}
std::unique_ptr<ExecutionBlock> MaterializeNode::createBlock(
ExecutionEngine & engine, std::unordered_map<ExecutionNode*, ExecutionBlock*> const &) const {
std::unique_ptr<ExecutionBlock> MaterializeMultiNode::createBlock(
ExecutionEngine& engine, std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const {
ExecutionNode const* previousNode = getFirstDependency();
TRI_ASSERT(previousNode != nullptr);
RegisterId inNmColPtrRegId;
{
auto it = getRegisterPlan()->varInfo.find(_inNonMaterializedColPtr->id);
@ -2394,14 +2438,15 @@ std::unique_ptr<ExecutionBlock> MaterializeNode::createBlock(
outDocumentRegId = it->second.registerId;
}
TRI_ASSERT(engine.getQuery());
MaterializerExecutorInfos infos(getRegisterPlan()->nrRegs[previousNode->getDepth()],
getRegisterPlan()->nrRegs[getDepth()], getRegsToClear(),
calcRegsToKeep(), inNmColPtrRegId, inNmDocIdRegId, outDocumentRegId, engine.getQuery()->trx());
return std::make_unique<ExecutionBlockImpl<MaterializeExecutor>>(&engine, this,
std::move(infos));
return std::make_unique<ExecutionBlockImpl<MaterializeExecutor<decltype(inNmColPtrRegId)>>>(&engine, this,
MaterializerExecutorInfos(getRegisterPlan()->nrRegs[previousNode->getDepth()],
getRegisterPlan()->nrRegs[getDepth()], getRegsToClear(),
calcRegsToKeep(), inNmColPtrRegId, inNmDocIdRegId,
outDocumentRegId, engine.getQuery()->trx()));
}
ExecutionNode * MaterializeNode::clone(ExecutionPlan * plan, bool withDependencies, bool withProperties) const {
ExecutionNode* MaterializeMultiNode::clone(ExecutionPlan* plan, bool withDependencies, bool withProperties) const {
TRI_ASSERT(plan);
auto* outVariable = _outVariable;
@ -2414,28 +2459,74 @@ ExecutionNode * MaterializeNode::clone(ExecutionPlan * plan, bool withDependenci
inNonMaterializedColId = plan->getAst()->variables()->createVariable(inNonMaterializedColId);
}
auto node =
std::make_unique<MaterializeNode>(plan, _id, *inNonMaterializedColId, *inNonMaterializedDocId, *outVariable);
return cloneHelper(std::move(node), withDependencies, withProperties);
auto c = std::make_unique<MaterializeMultiNode>(plan, _id, *inNonMaterializedColId, *inNonMaterializedDocId, *outVariable);
return cloneHelper(std::move(c), withDependencies, withProperties);
}
CostEstimate MaterializeNode::estimateCost() const {
if (_dependencies.empty()) {
// we should always have dependency as we need input for materializing
TRI_ASSERT(false);
return aql::CostEstimate::empty();
}
aql::CostEstimate estimate = _dependencies[0]->getCost();
// we will materialize all output of our dependency
estimate.estimatedCost += estimate.estimatedNrItems;
return estimate;
}
void MaterializeMultiNode::getVariablesUsedHere(::arangodb::containers::HashSet<Variable const*>& vars) const {
// call base class method
MaterializeNode::getVariablesUsedHere(vars);
void MaterializeNode::getVariablesUsedHere(::arangodb::containers::HashSet<Variable const*>& vars) const {
vars.emplace(_inNonMaterializedColPtr);
vars.emplace(_inNonMaterializedDocId);
}
std::vector<Variable const*> MaterializeNode::getVariablesSetHere() const {
return std::vector<Variable const*>{_outVariable};
MaterializeSingleNode::MaterializeSingleNode(ExecutionPlan* plan, size_t id, aql::Collection const* collection,
aql::Variable const& inDocId, aql::Variable const& outVariable)
: MaterializeNode(plan, id, inDocId, outVariable), CollectionAccessingNode(collection) {}
MaterializeSingleNode::MaterializeSingleNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base)
: MaterializeNode(plan, base), CollectionAccessingNode(plan, base) {}
void MaterializeSingleNode::toVelocyPackHelper(arangodb::velocypack::Builder& nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const {
// call base class method
MaterializeNode::toVelocyPackHelper(nodes, flags, seen);
// add collection information
CollectionAccessingNode::toVelocyPack(nodes, flags);
nodes.close();
}
std::unique_ptr<ExecutionBlock> MaterializeSingleNode::createBlock(
ExecutionEngine& engine, std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const {
ExecutionNode const* previousNode = getFirstDependency();
TRI_ASSERT(previousNode != nullptr);
RegisterId inNmDocIdRegId;
{
auto it = getRegisterPlan()->varInfo.find(_inNonMaterializedDocId->id);
TRI_ASSERT(it != getRegisterPlan()->varInfo.end());
inNmDocIdRegId = it->second.registerId;
}
RegisterId outDocumentRegId;
{
auto it = getRegisterPlan()->varInfo.find(_outVariable->id);
TRI_ASSERT(it != getRegisterPlan()->varInfo.end());
outDocumentRegId = it->second.registerId;
}
TRI_ASSERT(engine.getQuery());
auto const& name = _collection->name();
return std::make_unique<ExecutionBlockImpl<MaterializeExecutor<decltype(name)>>>(&engine, this,
MaterializerExecutorInfos<decltype(name)>(getRegisterPlan()->nrRegs[previousNode->getDepth()],
getRegisterPlan()->nrRegs[getDepth()], getRegsToClear(),
calcRegsToKeep(), _collection->name(), inNmDocIdRegId,
outDocumentRegId, engine.getQuery()->trx()));
}
ExecutionNode* MaterializeSingleNode::clone(ExecutionPlan * plan, bool withDependencies, bool withProperties) const {
TRI_ASSERT(plan);
auto* outVariable = _outVariable;
auto* inNonMaterializedDocId = _inNonMaterializedDocId;
if (withProperties) {
outVariable = plan->getAst()->variables()->createVariable(outVariable);
inNonMaterializedDocId = plan->getAst()->variables()->createVariable(inNonMaterializedDocId);
}
auto c = std::make_unique<MaterializeSingleNode>(plan, _id, _collection, *inNonMaterializedDocId, *outVariable);
CollectionAccessingNode::cloneInto(*c);
return cloneHelper(std::move(c), withDependencies, withProperties);
}

View File

@ -938,36 +938,34 @@ class NoResultsNode : public ExecutionNode {
CostEstimate estimateCost() const override final;
};
class MaterializeNode : public ExecutionNode {
friend class ExecutionNode;
friend class ExecutionBlock;
public:
MaterializeNode(ExecutionPlan* plan, size_t id, aql::Variable const& inColPtr,
aql::Variable const& inDocId, aql::Variable const& outVariable);
namespace materialize {
class MaterializeNode : public ExecutionNode {
protected:
MaterializeNode(ExecutionPlan* plan, size_t id, aql::Variable const& inDocId, aql::Variable const& outVariable);
MaterializeNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base);
public:
/// @brief return the type of the node
NodeType getType() const override final { return ExecutionNode::MATERIALIZE; }
/// @brief export to VelocyPack
void toVelocyPackHelper(arangodb::velocypack::Builder& nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const override final;
std::unordered_set<ExecutionNode const*>& seen) const override;
/// @brief creates corresponding ExecutionBlock
std::unique_ptr<ExecutionBlock> createBlock(
ExecutionEngine& engine,
std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const override;
std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const override = 0;
/// @brief clone ExecutionNode recursively
ExecutionNode* clone(ExecutionPlan* plan, bool withDependencies,
bool withProperties) const override final;
bool withProperties) const override = 0;
CostEstimate estimateCost() const override final;
/// @brief getVariablesUsedHere, modifying the set in-place
void getVariablesUsedHere(::arangodb::containers::HashSet<Variable const*>& vars) const override final;
void getVariablesUsedHere(::arangodb::containers::HashSet<Variable const*>& vars) const override;
/// @brief getVariablesSetHere
std::vector<Variable const*> getVariablesSetHere() const override final;
@ -977,10 +975,7 @@ class MaterializeNode : public ExecutionNode {
return *_outVariable;
}
private:
/// @brief input variable non-materialized collection ids
aql::Variable const* _inNonMaterializedColPtr;
protected:
/// @brief input variable non-materialized document ids
aql::Variable const* _inNonMaterializedDocId;
@ -988,6 +983,58 @@ class MaterializeNode : public ExecutionNode {
Variable const* _outVariable;
};
class MaterializeMultiNode : public MaterializeNode {
public:
MaterializeMultiNode(ExecutionPlan* plan, size_t id, aql::Variable const& inColPtr,
aql::Variable const& inDocId, aql::Variable const& outVariable);
MaterializeMultiNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base);
/// @brief export to VelocyPack
void toVelocyPackHelper(arangodb::velocypack::Builder& nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const override final;
/// @brief creates corresponding ExecutionBlock
std::unique_ptr<ExecutionBlock> createBlock(
ExecutionEngine& engine,
std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const override final;
/// @brief clone ExecutionNode recursively
ExecutionNode* clone(ExecutionPlan* plan, bool withDependencies,
bool withProperties) const override final;
/// @brief getVariablesUsedHere, modifying the set in-place
void getVariablesUsedHere(::arangodb::containers::HashSet<Variable const*>& vars) const override final;
private:
/// @brief input variable non-materialized collection ids
aql::Variable const* _inNonMaterializedColPtr;
};
class MaterializeSingleNode : public MaterializeNode, public CollectionAccessingNode {
public:
MaterializeSingleNode(ExecutionPlan* plan, size_t id, aql::Collection const* collection,
aql::Variable const& inDocId, aql::Variable const& outVariable);
MaterializeSingleNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base);
/// @brief export to VelocyPack
void toVelocyPackHelper(arangodb::velocypack::Builder& nodes, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const override final;
/// @brief creates corresponding ExecutionBlock
std::unique_ptr<ExecutionBlock> createBlock(
ExecutionEngine& engine,
std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const override final;
/// @brief clone ExecutionNode recursively
ExecutionNode* clone(ExecutionPlan* plan, bool withDependencies,
bool withProperties) const override final;
};
MaterializeNode* createMaterializeNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base);
} // namespace materialize
} // namespace aql
} // namespace arangodb

View File

@ -861,7 +861,7 @@ void IResearchViewNode::planNodeRegisters(
nrRegsHere.emplace_back(0);
// create a copy of the last value here
// this is requried because back returns a reference and emplace/push_back
// this is required because back returns a reference and emplace/push_back
// may invalidate all references
aql::RegisterCount const prevRegistersCount = nrRegs.back();
nrRegs.emplace_back(prevRegistersCount);

View File

@ -269,14 +269,14 @@ namespace arangodb {
namespace iresearch {
void lateDocumentMaterializationRule(arangodb::aql::Optimizer* opt,
void lateDocumentMaterializationArangoSearchRule(arangodb::aql::Optimizer* opt,
std::unique_ptr<arangodb::aql::ExecutionPlan> plan,
arangodb::aql::OptimizerRule const& rule) {
bool modified = false;
auto addPlan = arangodb::scopeGuard([opt, &plan, &rule, &modified]() {
opt->addPlan(std::move(plan), rule, modified);
});
// currently only arangosearch view node supports late materialization
// arangosearch view node supports late materialization
if (!plan->contains(EN::ENUMERATE_IRESEARCH_VIEW) ||
// we need sort node to be present (without sort it will be just skip, nothing to optimize)
!plan->contains(EN::SORT) ||
@ -297,7 +297,7 @@ void lateDocumentMaterializationRule(arangodb::aql::Optimizer* opt,
}
ExecutionNode* current = limitNode->getFirstDependency();
ExecutionNode* sortNode = nullptr;
// examinig plan. We are looking for SortNode closest to lowerest LimitNode
// examining plan. We are looking for SortNode closest to lowest LimitNode
// without document body usage before that node.
// this node could be appended with materializer
bool stopSearch = false;
@ -347,7 +347,7 @@ void lateDocumentMaterializationRule(arangodb::aql::Optimizer* opt,
viewNode.setLateMaterialized(localColPtrTmp, localDocIdTmp);
// insert a materialize node
auto materializeNode =
plan->registerNode(std::make_unique<MaterializeNode>(
plan->registerNode(std::make_unique<materialize::MaterializeMultiNode>(
plan.get(), plan->nextId(), *localColPtrTmp, *localDocIdTmp, viewNode.outVariable()));
// on cluster we need to materialize node stay close to sort node on db server (to avoid network hop for materialization calls)

View File

@ -37,7 +37,7 @@ class ExecutionPlan;
namespace iresearch {
/// @brief moves document materialization from view nodes to sort nodes
void lateDocumentMaterializationRule(arangodb::aql::Optimizer* opt,
void lateDocumentMaterializationArangoSearchRule(arangodb::aql::Optimizer* opt,
std::unique_ptr<arangodb::aql::ExecutionPlan> plan,
arangodb::aql::OptimizerRule const& rule);

View File

@ -41,10 +41,13 @@
#include "Basics/ScopeGuard.h"
#include "Cluster/ServerState.h"
#include "ExecutorExpressionContext.h"
#include "Transaction/Helpers.h"
#include "Transaction/Methods.h"
#include "Utils/OperationCursor.h"
#include "V8/v8-globals.h"
#include <velocypack/Iterator.h>
#include <memory>
#include <utility>
@ -70,8 +73,77 @@ static void resolveFCallConstAttributes(AstNode* fcall) {
}
}
template <bool checkUniqueness>
IndexIterator::DocumentCallback getCallback(DocumentProducingFunctionContext& context,
transaction::Methods::IndexHandle const& index,
IndexNode::IndexValuesRegisters const& outNonMaterializedIndRegs) {
return [&context, &index, &outNonMaterializedIndRegs](LocalDocumentId const& token, VPackSlice slice) {
if constexpr (checkUniqueness) {
if (!context.checkUniqueness(token)) {
// Document already found, skip it
return false;
}
}
context.incrScanned();
if (context.hasFilter()) {
if (!context.checkFilter(slice)) {
context.incrFiltered();
return false;
}
}
InputAqlItemRow const& input = context.getInputRow();
OutputAqlItemRow& output = context.getOutputRow();
RegisterId registerId = context.getOutputRegister();
// move a document id
AqlValue v(AqlValueHintUInt(token.id()));
AqlValueGuard guard{v, true};
TRI_ASSERT(!output.isFull());
output.moveValueInto(registerId, input, guard);
auto indexId = index.getIndex()->id();
TRI_ASSERT(indexId == outNonMaterializedIndRegs.first);
if (ADB_UNLIKELY(indexId != outNonMaterializedIndRegs.first)) {
return false;
}
// hash/skiplist/edge
if (slice.isArray()) {
for (auto const& indReg : outNonMaterializedIndRegs.second) {
TRI_ASSERT(indReg.first < slice.length());
if (ADB_UNLIKELY(indReg.first >= slice.length())) {
return false;
}
auto s = slice.at(indReg.first);
AqlValue v(s);
AqlValueGuard guard{v, true};
TRI_ASSERT(!output.isFull());
output.moveValueInto(indReg.second, input, guard);
}
} else { // primary
auto indReg = outNonMaterializedIndRegs.second.cbegin();
TRI_ASSERT(indReg != outNonMaterializedIndRegs.second.cend());
if (ADB_UNLIKELY(indReg == outNonMaterializedIndRegs.second.cend())) {
return false;
}
AqlValue v(slice);
AqlValueGuard guard{v, true};
TRI_ASSERT(!output.isFull());
output.moveValueInto(indReg->second, input, guard);
}
TRI_ASSERT(output.produced());
output.advanceRow();
context.incrScanned();
return true;
};
}
static inline DocumentProducingFunctionContext createContext(InputAqlItemRow const& inputRow,
IndexExecutorInfos& infos) {
IndexExecutorInfos const& infos) {
return DocumentProducingFunctionContext(
inputRow, nullptr, infos.getOutputRegisterId(), infos.getProduceResult(),
infos.getQuery(), infos.getFilter(),
@ -82,7 +154,10 @@ static inline DocumentProducingFunctionContext createContext(InputAqlItemRow con
} // namespace
IndexExecutorInfos::IndexExecutorInfos(
RegisterId outputRegister, RegisterId nrInputRegisters, RegisterId nrOutputRegisters,
std::shared_ptr<std::unordered_set<aql::RegisterId>>&& writableOutputRegisters,
RegisterId nrInputRegisters,
RegisterId outputRegister,
RegisterId nrOutputRegisters,
// cppcheck-suppress passedByValue
std::unordered_set<RegisterId> registersToClear,
// cppcheck-suppress passedByValue
@ -95,9 +170,10 @@ IndexExecutorInfos::IndexExecutorInfos(
std::vector<Variable const*>&& expInVars, std::vector<RegisterId>&& expInRegs,
bool hasV8Expression, AstNode const* condition,
std::vector<transaction::Methods::IndexHandle> indexes, Ast* ast,
IndexIteratorOptions options)
IndexIteratorOptions options,
IndexNode::IndexValuesRegisters&& outNonMaterializedIndRegs)
: ExecutorInfos(make_shared_unordered_set(),
make_shared_unordered_set({outputRegister}),
writableOutputRegisters,
nrInputRegisters, nrOutputRegisters,
std::move(registersToClear), std::move(registersToKeep)),
_indexes(std::move(indexes)),
@ -114,6 +190,7 @@ IndexExecutorInfos::IndexExecutorInfos(
_expInRegs(std::move(expInRegs)),
_nonConstExpression(std::move(nonConstExpression)),
_outputRegisterId(outputRegister),
_outNonMaterializedIndRegs(std::move(outNonMaterializedIndRegs)),
_hasMultipleExpansions(false),
_useRawDocumentPointers(useRawDocumentPointers),
_produceResult(produceResult),
@ -270,16 +347,26 @@ IndexExecutor::CursorReader::CursorReader(IndexExecutorInfos const& infos,
_cursor(std::make_unique<OperationCursor>(infos.getTrxPtr()->indexScanForCondition(
index, condition, infos.getOutVariable(), infos.getOptions()))),
_context(context),
_type(!infos.getProduceResult()
_type(infos.isLateMaterialized()
? Type::LateMaterialized
: !infos.getProduceResult()
? Type::NoResult
: _cursor->hasCovering() &&
!infos.getCoveringIndexAttributePositions().empty()
? Type::Covering
: Type::Document) {
if (_type == Type::NoResult) {
switch (_type) {
case Type::NoResult: {
_documentNonProducer = checkUniqueness ? getNullCallback<true>(context) : getNullCallback<false>(context);
} else {
break;
}
case Type::LateMaterialized:
_documentProducer = checkUniqueness ? ::getCallback<true>(context, _index, _infos.getOutNonMaterializedIndRegs()) :
::getCallback<false>(context, _index, _infos.getOutNonMaterializedIndRegs());
break;
default:
_documentProducer = checkUniqueness ? buildDocumentCallback<true, false>(context) : buildDocumentCallback<false, false>(context);
break;
}
_documentSkipper = checkUniqueness ? buildDocumentCallback<true, true>(context) : buildDocumentCallback<false, true>(context);
}
@ -323,6 +410,9 @@ bool IndexExecutor::CursorReader::readIndex(OutputAqlItemRow& output) {
case Type::Document:
TRI_ASSERT(_documentProducer != nullptr);
return _cursor->nextDocument(_documentProducer, output.numRowsLeft());
case Type::LateMaterialized:
TRI_ASSERT(_documentProducer != nullptr);
return _cursor->nextCovering(_documentProducer, output.numRowsLeft());
}
// The switch above is covering all values and this code
// cannot be reached

View File

@ -29,6 +29,7 @@
#include "Aql/DocumentProducingHelper.h"
#include "Aql/ExecutionState.h"
#include "Aql/ExecutorInfos.h"
#include "Aql/IndexNode.h"
#include "Aql/InputAqlItemRow.h"
#include "Aql/Stats.h"
#include "Indexes/IndexIterator.h"
@ -56,8 +57,8 @@ struct NonConstExpression;
class IndexExecutorInfos : public ExecutorInfos {
public:
IndexExecutorInfos(
RegisterId outputRegister, RegisterId nrInputRegisters,
RegisterId nrOutputRegisters, std::unordered_set<RegisterId> registersToClear,
std::shared_ptr<std::unordered_set<aql::RegisterId>>&& writableOutputRegisters, RegisterId nrInputRegisters,
RegisterId firstOutputRegister, RegisterId nrOutputRegisters, std::unordered_set<RegisterId> registersToClear,
std::unordered_set<RegisterId> registersToKeep, ExecutionEngine* engine,
Collection const* collection, Variable const* outVariable, bool produceResult,
Expression* filter,
@ -67,7 +68,8 @@ class IndexExecutorInfos : public ExecutorInfos {
std::vector<Variable const*>&& expInVars, std::vector<RegisterId>&& expInRegs,
bool hasV8Expression, AstNode const* condition,
std::vector<transaction::Methods::IndexHandle> indexes, Ast* ast,
IndexIteratorOptions options);
IndexIteratorOptions options,
IndexNode::IndexValuesRegisters&& outNonMaterializedIndRegs);
IndexExecutorInfos() = delete;
IndexExecutorInfos(IndexExecutorInfos&&) = default;
@ -105,6 +107,14 @@ class IndexExecutorInfos : public ExecutorInfos {
bool hasNonConstParts() const;
bool isLateMaterialized() const noexcept {
return !_outNonMaterializedIndRegs.second.empty();
}
IndexNode::IndexValuesRegisters const& getOutNonMaterializedIndRegs() const noexcept {
return _outNonMaterializedIndRegs;
}
private:
/// @brief _indexes holds all Indexes used in this block
std::vector<transaction::Methods::IndexHandle> _indexes;
@ -138,6 +148,9 @@ class IndexExecutorInfos : public ExecutorInfos {
std::vector<std::unique_ptr<NonConstExpression>> _nonConstExpression;
RegisterId _outputRegisterId;
IndexNode::IndexValuesRegisters _outNonMaterializedIndRegs;
/// @brief true if one of the indexes uses more than one expanded attribute,
/// e.g. the index is on values[*].name and values[*].type
bool _hasMultipleExpansions;
@ -173,7 +186,7 @@ class IndexExecutor {
CursorReader(CursorReader&& other) noexcept;
private:
enum Type { NoResult, Covering, Document };
enum Type { NoResult, Covering, Document, LateMaterialized };
IndexExecutorInfos const& _infos;
AstNode const* _condition;

View File

@ -59,7 +59,8 @@ IndexNode::IndexNode(ExecutionPlan* plan, size_t id,
_indexes(indexes),
_condition(std::move(condition)),
_needsGatherNodeSort(false),
_options(opts) {
_options(opts),
_outNonMaterializedDocId(nullptr) {
TRI_ASSERT(_condition != nullptr);
initIndexCoversProjections();
@ -73,7 +74,9 @@ IndexNode::IndexNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& bas
_indexes(),
_needsGatherNodeSort(
basics::VelocyPackHelper::readBooleanValue(base, "needsGatherNodeSort", false)),
_options() {
_options(),
_outNonMaterializedDocId(
aql::Variable::varFromVPack(plan->getAst(), base, "outNmDocId", true)) {
_options.sorted = basics::VelocyPackHelper::readBooleanValue(base, "sorted", true);
_options.ascending =
basics::VelocyPackHelper::readBooleanValue(base, "ascending", false);
@ -113,6 +116,56 @@ IndexNode::IndexNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& bas
TRI_ASSERT(_condition != nullptr);
initIndexCoversProjections();
if (_outNonMaterializedDocId != nullptr) {
auto const* vars = plan->getAst()->variables();
TRI_ASSERT(vars);
auto const indexIdSlice = base.get("indexIdOfVars");
if (!indexIdSlice.isNumber<TRI_idx_iid_t>()) {
THROW_ARANGO_EXCEPTION_FORMAT(
TRI_ERROR_BAD_PARAMETER, "\"indexIdOfVars\" %s should be a number",
indexIdSlice.toString().c_str());
}
auto const indexId = indexIdSlice.getNumber<TRI_idx_iid_t>();
auto const indexValuesVarsSlice = base.get("IndexValuesVars");
if (!indexValuesVarsSlice.isArray()) {
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_BAD_PARAMETER,
"\"IndexValuesVars\" attribute should be an array");
}
std::unordered_map<size_t, Variable const*> indexValuesVars;
indexValuesVars.reserve(indexValuesVarsSlice.length());
for (auto const indVar : velocypack::ArrayIterator(indexValuesVarsSlice)) {
auto const fieldNumberSlice = indVar.get("fieldNumber");
if (!fieldNumberSlice.isNumber<size_t>()) {
THROW_ARANGO_EXCEPTION_FORMAT(
TRI_ERROR_BAD_PARAMETER, "\"IndexValuesVars[*].fieldNumber\" %s should be a number",
fieldNumberSlice.toString().c_str());
}
auto const fieldNumber = fieldNumberSlice.getNumber<size_t>();
auto const varIdSlice = indVar.get("id");
if (!varIdSlice.isNumber<aql::VariableId>()) {
THROW_ARANGO_EXCEPTION_FORMAT(
TRI_ERROR_BAD_PARAMETER, "\"IndexValuesVars[*].id\" variable id %s should be a number",
varIdSlice.toString().c_str());
}
auto const varId = varIdSlice.getNumber<aql::VariableId>();
auto const* var = vars->getVariable(varId);
if (!var) {
THROW_ARANGO_EXCEPTION_FORMAT(
TRI_ERROR_BAD_PARAMETER, "\"IndexValuesVars[*].id\" unable to find variable by id %d",
varId);
}
indexValuesVars.emplace(fieldNumber, var);
}
_outNonMaterializedIndVars.first = indexId;
_outNonMaterializedIndVars.second = std::move(indexValuesVars);
}
}
/// @brief called to build up the matching positions of the index values for
@ -173,6 +226,30 @@ void IndexNode::initIndexCoversProjections() {
_options.forceProjection = true;
}
void IndexNode::planNodeRegisters(
std::vector<aql::RegisterId>& nrRegsHere, std::vector<aql::RegisterId>& nrRegs,
std::unordered_map<aql::VariableId, aql::VarInfo>& varInfo,
unsigned int& totalNrRegs, unsigned int depth) const {
// create a copy of the last value here
// this is required because back returns a reference and emplace/push_back
// may invalidate all references
auto regsCount = nrRegs.back();
nrRegs.emplace_back(regsCount + 1);
nrRegsHere.emplace_back(1);
if (isLateMaterialized()) {
varInfo.emplace(_outNonMaterializedDocId->id, aql::VarInfo(depth, totalNrRegs++));
// plan registers for index references
for (auto const& var : _outNonMaterializedIndVars.second) {
++nrRegsHere[depth];
++nrRegs[depth];
varInfo.emplace(var.second->id, aql::VarInfo(depth, totalNrRegs++));
}
} else {
varInfo.emplace(_outVariable->id, aql::VarInfo(depth, totalNrRegs++));
}
}
/// @brief toVelocyPack, for IndexNode
void IndexNode::toVelocyPackHelper(VPackBuilder& builder, unsigned flags,
std::unordered_set<ExecutionNode const*>& seen) const {
@ -206,6 +283,30 @@ void IndexNode::toVelocyPackHelper(VPackBuilder& builder, unsigned flags,
builder.add("evalFCalls", VPackValue(_options.evaluateFCalls));
builder.add("limit", VPackValue(_options.limit));
if (isLateMaterialized()) {
builder.add(VPackValue("outNmDocId"));
_outNonMaterializedDocId->toVelocyPack(builder);
builder.add("indexIdOfVars", VPackValue(_outNonMaterializedIndVars.first));
// container _indexes contains a few items
auto indIt = std::find_if(_indexes.cbegin(), _indexes.cend(), [this](auto const& index) {
return index.getIndex()->id() == _outNonMaterializedIndVars.first;
});
TRI_ASSERT(indIt != _indexes.cend());
auto const& fields = indIt->getIndex()->fields();
VPackArrayBuilder arrayScope(&builder, "IndexValuesVars");
for (auto const& indVar : _outNonMaterializedIndVars.second) {
VPackObjectBuilder objectScope(&builder);
builder.add("fieldNumber", VPackValue(indVar.first));
builder.add("id", VPackValue(indVar.second->id));
builder.add("name", VPackValue(indVar.second->name)); // for explainer.js
std::string fieldName;
TRI_ASSERT(indVar.first < fields.size());
basics::TRI_AttributeNamesToString(fields[indVar.first], fieldName, true);
builder.add("field", VPackValue(fieldName)); // for explainer.js
}
}
// And close it:
builder.close();
}
@ -262,7 +363,7 @@ void IndexNode::initializeOnce(bool hasV8Expression, std::vector<Variable const*
for (auto const& v : innerVars) {
inVars.emplace_back(v);
auto it = getRegisterPlan()->varInfo.find(v->id);
TRI_ASSERT(it != getRegisterPlan()->varInfo.end());
TRI_ASSERT(it != getRegisterPlan()->varInfo.cend());
TRI_ASSERT(it->second.registerId < RegisterPlan::MaxRegisterId);
inRegs.emplace_back(it->second.registerId);
}
@ -358,9 +459,6 @@ std::unique_ptr<ExecutionBlock> IndexNode::createBlock(
ExecutionEngine& engine, std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const {
ExecutionNode const* previousNode = getFirstDependency();
TRI_ASSERT(previousNode != nullptr);
auto it = getRegisterPlan()->varInfo.find(_outVariable->id);
TRI_ASSERT(it != getRegisterPlan()->varInfo.end());
RegisterId outputRegister = it->second.registerId;
transaction::Methods* trxPtr = _plan->getAst()->query()->trx();
@ -381,8 +479,47 @@ std::unique_ptr<ExecutionBlock> IndexNode::createBlock(
initializeOnce(hasV8Expression, inVars, inRegs, nonConstExpressions, trxPtr);
IndexExecutorInfos infos(outputRegister,
auto const firstOutputRegister = getNrInputRegisters();
auto numIndVarsRegisters = static_cast<aql::RegisterCount>(_outNonMaterializedIndVars.second.size());
TRI_ASSERT(0 == numIndVarsRegisters || isLateMaterialized());
// We could be asked to produce only document id for later materialization or full document body at once
aql::RegisterCount numDocumentRegs = 1;
// if late materialized
// We have one additional output register for each index variable which is used later, before
// the output register for document id
// These must of course fit in the available registers.
// There may be unused registers reserved for later blocks.
std::shared_ptr<std::unordered_set<aql::RegisterId>> writableOutputRegisters =
aql::make_shared_unordered_set();
writableOutputRegisters->reserve(numDocumentRegs + numIndVarsRegisters);
for (aql::RegisterId reg = firstOutputRegister;
reg < firstOutputRegister + numIndVarsRegisters + numDocumentRegs; ++reg) {
writableOutputRegisters->emplace(reg);
}
TRI_ASSERT(writableOutputRegisters->size() == numDocumentRegs + numIndVarsRegisters);
TRI_ASSERT(writableOutputRegisters->begin() != writableOutputRegisters->end());
TRI_ASSERT(firstOutputRegister == *std::min_element(writableOutputRegisters->cbegin(),
writableOutputRegisters->cend()));
auto const& varInfos = getRegisterPlan()->varInfo;
IndexValuesRegisters outNonMaterializedIndRegs;
outNonMaterializedIndRegs.first = _outNonMaterializedIndVars.first;
outNonMaterializedIndRegs.second.reserve(_outNonMaterializedIndVars.second.size());
std::transform(_outNonMaterializedIndVars.second.cbegin(), _outNonMaterializedIndVars.second.cend(),
std::inserter(outNonMaterializedIndRegs.second, outNonMaterializedIndRegs.second.end()),
[&varInfos](auto const& indVar) {
auto it = varInfos.find(indVar.second->id);
TRI_ASSERT(it != varInfos.cend());
return std::make_pair(indVar.first, it->second.registerId);
});
IndexExecutorInfos infos(std::move(writableOutputRegisters),
getRegisterPlan()->nrRegs[previousNode->getDepth()],
firstOutputRegister,
getRegisterPlan()->nrRegs[getDepth()], getRegsToClear(),
calcRegsToKeep(), &engine, this->_collection, _outVariable,
(this->isVarUsedLater(_outVariable) || this->_filter != nullptr),
@ -391,18 +528,26 @@ std::unique_ptr<ExecutionBlock> IndexNode::createBlock(
EngineSelectorFeature::ENGINE->useRawDocumentPointers(),
std::move(nonConstExpressions), std::move(inVars),
std::move(inRegs), hasV8Expression, _condition->root(),
this->getIndexes(), _plan->getAst(), this->options());
this->getIndexes(), _plan->getAst(), this->options(),
std::move(outNonMaterializedIndRegs));
return std::make_unique<ExecutionBlockImpl<IndexExecutor>>(&engine, this,
std::move(infos));
return std::make_unique<ExecutionBlockImpl<IndexExecutor>>(&engine, this, std::move(infos));
}
ExecutionNode* IndexNode::clone(ExecutionPlan* plan, bool withDependencies,
bool withProperties) const {
auto outVariable = _outVariable;
auto outNonMaterializedDocId = _outNonMaterializedDocId;
auto outNonMaterializedIndVars = _outNonMaterializedIndVars;
if (withProperties) {
outVariable = plan->getAst()->variables()->createVariable(outVariable);
if (outNonMaterializedDocId != nullptr) {
outNonMaterializedDocId = plan->getAst()->variables()->createVariable(outNonMaterializedDocId);
}
for (auto& indVar : outNonMaterializedIndVars.second) {
indVar.second = plan->getAst()->variables()->createVariable(indVar.second);
}
}
auto c = std::make_unique<IndexNode>(plan, _id, _collection, outVariable, _indexes,
@ -412,6 +557,8 @@ ExecutionNode* IndexNode::clone(ExecutionPlan* plan, bool withDependencies,
c->projections(_projections);
c->needsGatherNodeSort(_needsGatherNodeSort);
c->initIndexCoversProjections();
c->_outNonMaterializedDocId = outNonMaterializedDocId;
c->_outNonMaterializedIndVars = std::move(outNonMaterializedIndVars);
CollectionAccessingNode::cloneInto(*c);
DocumentProducingNode::cloneInto(plan, *c);
return cloneHelper(std::move(c), withDependencies, withProperties);
@ -473,13 +620,38 @@ void IndexNode::needsGatherNodeSort(bool value) {
}
std::vector<Variable const*> IndexNode::getVariablesSetHere() const {
return std::vector<Variable const*>{_outVariable};
if (!isLateMaterialized()) {
return std::vector<Variable const*>{_outVariable};
}
std::vector<arangodb::aql::Variable const*> vars;
vars.reserve(1 + _outNonMaterializedIndVars.second.size());
vars.emplace_back(_outNonMaterializedDocId);
std::transform(_outNonMaterializedIndVars.second.cbegin(),
_outNonMaterializedIndVars.second.cend(),
std::back_inserter(vars),
[](auto const& indVar) {
return indVar.second;
});
return vars;
}
std::vector<transaction::Methods::IndexHandle> const& IndexNode::getIndexes() const {
return _indexes;
}
void IndexNode::setLateMaterialized(aql::Variable const* docIdVariable,
TRI_idx_iid_t commonIndexId,
IndexVarsInfo const& indexVariables) {
_outNonMaterializedIndVars.second.clear();
_outNonMaterializedIndVars.first = commonIndexId;
_outNonMaterializedDocId = docIdVariable;
for (auto& indVars : indexVariables) {
_outNonMaterializedIndVars.second[indVars.second.indexFieldNum] = indVars.second.var;
}
}
NonConstExpression::NonConstExpression(std::unique_ptr<Expression> exp,
std::vector<size_t>&& idxPath)
: expression(std::move(exp)), indexPath(std::move(idxPath)) {}

View File

@ -30,6 +30,7 @@
#include "Aql/CollectionAccessingNode.h"
#include "Aql/DocumentProducingNode.h"
#include "Aql/ExecutionNode.h"
#include "Aql/RegisterPlan.h"
#include "Aql/types.h"
#include "Containers/HashSet.h"
#include "Indexes/IndexIterator.h"
@ -115,6 +116,30 @@ class IndexNode : public ExecutionNode, public DocumentProducingNode, public Col
/// the projection attributes (if any)
void initIndexCoversProjections();
void planNodeRegisters(std::vector<aql::RegisterId>& nrRegsHere,
std::vector<aql::RegisterId>& nrRegs,
std::unordered_map<aql::VariableId, aql::VarInfo>& varInfo,
unsigned int& totalNrRegs, unsigned int depth) const;
bool isLateMaterialized() const noexcept {
TRI_ASSERT((_outNonMaterializedDocId == nullptr && _outNonMaterializedIndVars.second.empty()) ||
!(_outNonMaterializedDocId == nullptr || _outNonMaterializedIndVars.second.empty()));
return !_outNonMaterializedIndVars.second.empty();
}
struct IndexVariable {
size_t indexFieldNum;
Variable const* var;
};
using IndexValuesVars = std::pair<TRI_idx_iid_t, std::unordered_map<size_t, Variable const*>>;
using IndexValuesRegisters = std::pair<TRI_idx_iid_t, std::unordered_map<size_t, RegisterId>>;
using IndexVarsInfo = std::unordered_map<std::vector<arangodb::basics::AttributeName> const*, IndexNode::IndexVariable>;
void setLateMaterialized(aql::Variable const* docIdVariable, TRI_idx_iid_t commonIndexId, IndexVarsInfo const& indexVariables);
private:
void initializeOnce(bool hasV8Expression, std::vector<Variable const*>& inVars,
std::vector<RegisterId>& inRegs,
@ -136,6 +161,12 @@ class IndexNode : public ExecutionNode, public DocumentProducingNode, public Col
/// @brief the index iterator options - same for all indexes
IndexIteratorOptions _options;
/// @brief output variable to write only non-materialized document ids
aql::Variable const* _outNonMaterializedDocId;
/// @brief output variables to non-materialized document index references
IndexValuesVars _outNonMaterializedIndVars;
};
} // namespace aql

View File

@ -0,0 +1,314 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2019 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 Yuriy Popov
////////////////////////////////////////////////////////////////////////////////
#include "Aql/Ast.h"
#include "Aql/Collection.h"
#include "Aql/Condition.h"
#include "Aql/Expression.h"
#include "Aql/IndexNode.h"
#include "Aql/Optimizer.h"
#include "IndexNodeOptimizerRules.h"
#include "Basics/AttributeNameParser.h"
#include "Cluster/ServerState.h"
using EN = arangodb::aql::ExecutionNode;
namespace {
struct NodeWithAttrs {
struct AttributeAndField {
std::vector<arangodb::basics::AttributeName> attr;
arangodb::aql::AstNode* astNode;
size_t astNodeChildNum;
size_t indexFieldNum;
std::vector<arangodb::basics::AttributeName> const* indexField;
};
std::vector<AttributeAndField> attrs;
arangodb::aql::CalculationNode* node;
};
bool attributesMatch(TRI_idx_iid_t& commonIndexId, arangodb::aql::IndexNode const* indexNode, NodeWithAttrs& node) {
// check all node attributes to be in index
for (auto& nodeAttr : node.attrs) {
for (auto& index : indexNode->getIndexes()) {
auto const& ind = index.getIndex();
if (!ind->hasCoveringIterator()) {
continue;
}
auto indexId = ind->id();
// use one index only
if (commonIndexId != 0 && commonIndexId != indexId) {
continue;
}
size_t indexFieldNum = 0;
for (auto const& field : ind->fields()) {
if (arangodb::basics::AttributeName::isIdentical(nodeAttr.attr, field, false)) {
if (commonIndexId == 0) {
commonIndexId = indexId;
}
nodeAttr.indexFieldNum = indexFieldNum;
nodeAttr.indexField = &field;
break;
}
++indexFieldNum;
}
if (nodeAttr.indexField != nullptr) {
break;
}
}
// not found
if (nodeAttr.indexField == nullptr) {
return false;
}
}
return true;
}
// traverse the AST, using previsitor
void traverseReadOnly(arangodb::aql::AstNode* node, arangodb::aql::AstNode* parentNode, size_t childNumber,
std::function<bool(arangodb::aql::AstNode const*, arangodb::aql::AstNode*, size_t)> const& preVisitor) {
if (node == nullptr) {
return;
}
if (!preVisitor(node, parentNode, childNumber)) {
return;
}
size_t const n = node->numMembers();
for (size_t i = 0; i < n; ++i) {
auto member = node->getMemberUnchecked(i);
if (member != nullptr) {
traverseReadOnly(member, node, i, preVisitor);
}
}
}
// traversal state
struct TraversalState {
arangodb::aql::Variable const* variable;
NodeWithAttrs& nodeAttrs;
bool optimize;
bool wasAccess;
};
// determines attributes referenced in an expression for the specified out variable
bool getReferencedAttributes(arangodb::aql::AstNode* node,
arangodb::aql::Variable const* variable,
NodeWithAttrs& nodeAttrs) {
TraversalState state{variable, nodeAttrs, true, false};
auto preVisitor = [&state](arangodb::aql::AstNode const* node,
arangodb::aql::AstNode* parentNode, size_t childNumber) {
if (node == nullptr) {
return false;
}
switch (node->type) {
case arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS:
if (!state.wasAccess) {
state.nodeAttrs.attrs.emplace_back(
NodeWithAttrs::AttributeAndField{std::vector<arangodb::basics::AttributeName>{
{std::string(node->getStringValue(), node->getStringLength()), false}}, parentNode, childNumber, 0, nullptr});
state.wasAccess = true;
} else {
state.nodeAttrs.attrs.back().attr.emplace_back(std::string(node->getStringValue(), node->getStringLength()), false);
}
return true;
case arangodb::aql::NODE_TYPE_REFERENCE: {
// reference to a variable
auto v = static_cast<arangodb::aql::Variable const*>(node->getData());
if (v == state.variable) {
if (!state.wasAccess) {
// we haven't seen an attribute access directly before
state.optimize = false;
return false;
}
std::reverse(state.nodeAttrs.attrs.back().attr.begin(), state.nodeAttrs.attrs.back().attr.end());
} else {
if (state.wasAccess) {
state.nodeAttrs.attrs.pop_back();
}
}
// finish an attribute path
state.wasAccess = false;
return true;
}
default:
break;
}
if (state.wasAccess) {
// not appropriate node type
state.wasAccess = false;
state.optimize = false;
return false;
}
return true;
};
traverseReadOnly(node, nullptr, 0, preVisitor);
return state.optimize;
}
}
void arangodb::aql::lateDocumentMaterializationRule(arangodb::aql::Optimizer* opt,
std::unique_ptr<arangodb::aql::ExecutionPlan> plan,
arangodb::aql::OptimizerRule const& rule) {
auto modified = false;
auto addPlan = arangodb::scopeGuard([opt, &plan, &rule, &modified]() {
opt->addPlan(std::move(plan), rule, modified);
});
// index node supports late materialization
if (!plan->contains(EN::INDEX) ||
// we need sort node to be present (without sort it will be just skip, nothing to optimize)
!plan->contains(EN::SORT) ||
// limit node is needed as without limit all documents will be returned anyway, nothing to optimize
!plan->contains(EN::LIMIT)) {
return;
}
::arangodb::containers::SmallVector<ExecutionNode*>::allocator_type::arena_type a;
::arangodb::containers::SmallVector<ExecutionNode*> nodes{a};
plan->findNodesOfType(nodes, EN::LIMIT, true);
for (auto limitNode : nodes) {
auto loop = const_cast<ExecutionNode*>(limitNode->getLoop());
if (arangodb::aql::ExecutionNode::INDEX == loop->getType()) {
auto indexNode = EN::castTo<IndexNode*>(loop);
if (indexNode->isLateMaterialized()) {
continue; // loop is aleady optimized
}
auto current = limitNode->getFirstDependency();
ExecutionNode* sortNode = nullptr;
// examining plan. We are looking for SortNode closest to lowest LimitNode
// without document body usage before that node.
// this node could be appended with materializer
bool stopSearch = false;
std::vector<NodeWithAttrs> nodesToChange;
TRI_idx_iid_t commonIndexId = 0; // use one index only
while (current != loop) {
switch (current->getType()) {
case arangodb::aql::ExecutionNode::SORT:
if (sortNode == nullptr) { // we need nearest to limit sort node, so keep selected if any
sortNode = current;
}
break;
case arangodb::aql::ExecutionNode::CALCULATION: {
auto calculationNode = EN::castTo<CalculationNode*>(current);
auto astNode = calculationNode->expression()->nodeForModification();
NodeWithAttrs node;
node.node = calculationNode;
// find attributes referenced to index node out variable
if (!getReferencedAttributes(astNode, indexNode->outVariable(), node)) {
// is not safe for optimization
stopSearch = true;
} else if (!node.attrs.empty()) {
if (!attributesMatch(commonIndexId, indexNode, node)) {
// the node uses attributes which is not in index
stopSearch = true;
} else {
nodesToChange.emplace_back(node);
}
}
break;
}
case arangodb::aql::ExecutionNode::REMOTE:
// REMOTE node is a blocker - we do not want to make materialization calls across cluster!
if (sortNode != nullptr) {
stopSearch = true;
}
break;
default: // make clang happy
break;
}
if (sortNode != nullptr && current->getType() != arangodb::aql::ExecutionNode::CALCULATION) {
::arangodb::containers::HashSet<Variable const*> currentUsedVars;
current->getVariablesUsedHere(currentUsedVars);
if (currentUsedVars.find(indexNode->outVariable()) != currentUsedVars.end()) {
// this limit node affects only closest sort, if this sort is invalid
// we need to check other limit node
stopSearch = true;
}
}
if (stopSearch) {
// we have a doc body used before selected SortNode. Forget it, let`s look for better sort to use
sortNode = nullptr;
nodesToChange.clear();
break;
}
current = current->getFirstDependency(); // inspect next node
}
if (sortNode && !nodesToChange.empty()) {
auto ast = plan->getAst();
IndexNode::IndexVarsInfo uniqueVariables;
for (auto& node : nodesToChange) {
std::transform(node.attrs.cbegin(), node.attrs.cend(), std::inserter(uniqueVariables, uniqueVariables.end()),
[&ast](auto const& attrAndField) {
return std::make_pair(attrAndField.indexField, IndexNode::IndexVariable{attrAndField.indexFieldNum,
ast->variables()->createTemporaryVariable()});
});
}
auto localDocIdTmp = ast->variables()->createTemporaryVariable();
for (auto& node : nodesToChange) {
for (auto& attr : node.attrs) {
auto it = uniqueVariables.find(attr.indexField);
TRI_ASSERT(it != uniqueVariables.cend());
auto newNode = ast->createNodeReference(it->second.var);
if (attr.astNode != nullptr) {
TEMPORARILY_UNLOCK_NODE(attr.astNode);
attr.astNode->changeMember(attr.astNodeChildNum, newNode);
} else {
TRI_ASSERT(node.attrs.size() == 1);
node.node->expression()->replaceNode(newNode);
}
}
}
// we could apply late materialization
// 1. We need to notify index node - it should not materialize documents, but produce only localDocIds
indexNode->setLateMaterialized(localDocIdTmp, commonIndexId, uniqueVariables);
// 2. We need to add materializer after limit node to do materialization
// insert a materialize node
auto materializeNode =
plan->registerNode(std::make_unique<materialize::MaterializeSingleNode>(
plan.get(), plan->nextId(), indexNode->collection(),
*localDocIdTmp, *indexNode->outVariable()));
// on cluster we need to materialize node stay close to sort node on db server (to avoid network hop for materialization calls)
// however on single server we move it to limit node to make materialization as lazy as possible
auto materializeDependency = ServerState::instance()->isCoordinator() ? sortNode : limitNode;
auto dependencyParent = materializeDependency->getFirstParent();
TRI_ASSERT(dependencyParent != nullptr);
dependencyParent->replaceDependency(materializeDependency, materializeNode);
materializeDependency->addParent(materializeNode);
modified = true;
}
}
}
}

View File

@ -0,0 +1,43 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2019 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 Yuriy Popov
////////////////////////////////////////////////////////////////////////////////
#ifndef ARANGOD_AQL_INDEX_NODE_OPTIMIZER_RULES_H
#define ARANGOD_AQL_INDEX_NODE_OPTIMIZER_RULES_H 1
#include <memory>
namespace arangodb {
namespace aql {
class Optimizer;
struct OptimizerRule;
class ExecutionPlan;
/// @brief moves document materialization from index nodes to sort nodes
void lateDocumentMaterializationRule(arangodb::aql::Optimizer* opt,
std::unique_ptr<arangodb::aql::ExecutionPlan> plan,
arangodb::aql::OptimizerRule const& rule);
} // namespace aql
} // namespace arangodb
#endif // ARANGOD_AQL_INDEX_NODE_OPTIMIZER_RULES_H

View File

@ -24,14 +24,15 @@
#include "StorageEngine/EngineSelectorFeature.h"
#include "StorageEngine/StorageEngine.h"
#include "VocBase/LogicalCollection.h"
#include "Aql/SingleRowFetcher.h"
#include "Aql/Stats.h"
#include "Transaction/Methods.h"
using namespace arangodb;
using namespace arangodb::aql;
arangodb::IndexIterator::DocumentCallback MaterializeExecutor::ReadContext::copyDocumentCallback(ReadContext & ctx) {
template<typename T>
arangodb::IndexIterator::DocumentCallback MaterializeExecutor<T>::ReadContext::copyDocumentCallback(ReadContext & ctx) {
auto* engine = EngineSelectorFeature::ENGINE;
TRI_ASSERT(engine);
typedef std::function<arangodb::IndexIterator::DocumentCallback(ReadContext&)> CallbackFactory;
@ -69,30 +70,34 @@ arangodb::IndexIterator::DocumentCallback MaterializeExecutor::ReadContext::copy
return callbackFactories[size_t(engine->useRawDocumentPointers())](ctx);
}
arangodb::aql::MaterializerExecutorInfos::MaterializerExecutorInfos(
template<typename T>
arangodb::aql::MaterializerExecutorInfos<T>::MaterializerExecutorInfos(
RegisterId nrInputRegisters, RegisterId nrOutputRegisters,
// cppcheck-suppress passedByValue
std::unordered_set<RegisterId> registersToClear,
// cppcheck-suppress passedByValue
std::unordered_set<RegisterId> registersToKeep,
RegisterId inNmColPtr, RegisterId inNmDocId, RegisterId outDocRegId, transaction::Methods* trx )
T const collectionSource, RegisterId inNmDocId,
RegisterId outDocRegId, transaction::Methods* trx)
: ExecutorInfos(
make_shared_unordered_set(std::initializer_list<RegisterId>({inNmColPtr, inNmDocId})),
getReadableInputRegisters(collectionSource, inNmDocId),
make_shared_unordered_set(std::initializer_list<RegisterId>({outDocRegId})),
nrInputRegisters, nrOutputRegisters,
std::move(registersToClear), std::move(registersToKeep)),
_inNonMaterializedColRegId(inNmColPtr), _inNonMaterializedDocRegId(inNmDocId),
_collectionSource(collectionSource),
_inNonMaterializedDocRegId(inNmDocId),
_outMaterializedDocumentRegId(outDocRegId), _trx(trx) {
}
std::pair<ExecutionState, NoStats> arangodb::aql::MaterializeExecutor::produceRows(OutputAqlItemRow & output) {
template<typename T>
std::pair<ExecutionState, NoStats> arangodb::aql::MaterializeExecutor<T>::produceRows(OutputAqlItemRow & output) {
InputAqlItemRow input{CreateInvalidInputRowHint{}};
ExecutionState state;
bool written = false;
// some micro-optimization
auto& callback = _readDocumentContext._callback;
auto docRegId = _readDocumentContext._infos->inputNonMaterializedDocRegId();
auto colRegId = _readDocumentContext._infos->inputNonMaterializedColRegId();
T collectionSource = _readDocumentContext._infos->collectionSource();
auto* trx = _readDocumentContext._infos->trx();
do {
std::tie(state, input) = _fetcher.fetchRow();
@ -104,9 +109,17 @@ std::pair<ExecutionState, NoStats> arangodb::aql::MaterializeExecutor::produceRo
TRI_ASSERT(state == ExecutionState::DONE);
return {state, NoStats{}};
}
auto collection =
reinterpret_cast<arangodb::LogicalCollection const*>(
input.getValue(colRegId).slice().getUInt());
arangodb::LogicalCollection const* collection = nullptr;
if constexpr (std::is_same<T, std::string const&>::value) {
if (_collection == nullptr) {
_collection = trx->documentCollection(collectionSource);
}
collection = _collection;
} else {
collection =
reinterpret_cast<arangodb::LogicalCollection const*>(
input.getValue(collectionSource).slice().getUInt());
}
TRI_ASSERT(collection != nullptr);
_readDocumentContext._inputRow = &input;
_readDocumentContext._outputRow = &output;
@ -117,9 +130,16 @@ std::pair<ExecutionState, NoStats> arangodb::aql::MaterializeExecutor::produceRo
return {state, NoStats{}};
}
std::tuple<ExecutionState, NoStats, size_t> arangodb::aql::MaterializeExecutor::skipRows(size_t toSkipRequested) {
template<typename T>
std::tuple<ExecutionState, NoStats, size_t> arangodb::aql::MaterializeExecutor<T>::skipRows(size_t toSkipRequested) {
ExecutionState state;
size_t skipped;
std::tie(state, skipped) = _fetcher.skipRows(toSkipRequested);
return std::make_tuple(state, NoStats{}, skipped);
}
template class ::arangodb::aql::MaterializeExecutor<RegisterId>;
template class ::arangodb::aql::MaterializeExecutor<std::string const&>;
template class ::arangodb::aql::MaterializerExecutorInfos<RegisterId>;
template class ::arangodb::aql::MaterializerExecutorInfos<std::string const&>;

View File

@ -31,6 +31,7 @@
#include "Aql/types.h"
#include "Indexes/IndexIterator.h"
#include "VocBase/LocalDocumentId.h"
#include "VocBase/LogicalCollection.h"
#include <iosfwd>
#include <memory>
@ -44,38 +45,47 @@ template <BlockPassthrough>
class SingleRowFetcher;
class NoStats;
template<typename T>
class MaterializerExecutorInfos : public ExecutorInfos {
public:
MaterializerExecutorInfos(RegisterId nrInputRegisters, RegisterId nrOutputRegisters,
std::unordered_set<RegisterId> registersToClear,
std::unordered_set<RegisterId> registersToKeep,
RegisterId inNmColPtr, RegisterId inNmDocId,
RegisterId outDocRegId, transaction::Methods* trx);
std::unordered_set<RegisterId> registersToClear,
std::unordered_set<RegisterId> registersToKeep,
T collectionSource, RegisterId inNmDocId,
RegisterId outDocRegId, transaction::Methods* trx);
MaterializerExecutorInfos() = delete;
MaterializerExecutorInfos(MaterializerExecutorInfos&&) = default;
MaterializerExecutorInfos(MaterializerExecutorInfos const&) = delete;
~MaterializerExecutorInfos() = default;
inline RegisterId outputMaterializedDocumentRegId() const {
RegisterId outputMaterializedDocumentRegId() const {
return _outMaterializedDocumentRegId;
}
inline RegisterId inputNonMaterializedColRegId() const {
return _inNonMaterializedColRegId;
}
inline RegisterId inputNonMaterializedDocRegId() const {
RegisterId inputNonMaterializedDocRegId() const {
return _inNonMaterializedDocRegId;
}
inline transaction::Methods* trx() const {
transaction::Methods* trx() const {
return _trx;
}
T collectionSource() const {
return _collectionSource;
}
private:
/// @brief register to store raw collection pointer
RegisterId const _inNonMaterializedColRegId;
std::shared_ptr<std::unordered_set<RegisterId>> getReadableInputRegisters(T const collectionSource, RegisterId inNmDocId) {
if constexpr (std::is_same<T, RegisterId>::value) {
return make_shared_unordered_set(std::initializer_list<RegisterId>({collectionSource, inNmDocId}));
} else {
return make_shared_unordered_set(std::initializer_list<RegisterId>({inNmDocId}));
}
}
/// @brief register to store raw collection pointer or collection name
T const _collectionSource;
/// @brief register to store local document id
RegisterId const _inNonMaterializedDocRegId;
/// @brief register to store materialized document
@ -84,6 +94,7 @@ class MaterializerExecutorInfos : public ExecutorInfos {
transaction::Methods* _trx;
};
template<typename T>
class MaterializeExecutor {
public:
struct Properties {
@ -92,7 +103,7 @@ class MaterializeExecutor {
static constexpr bool inputSizeRestrictsOutputSize = false;
};
using Fetcher = SingleRowFetcher<Properties::allowsBlockPassthrough>;
using Infos = MaterializerExecutorInfos;
using Infos = MaterializerExecutorInfos<T>;
using Stats = NoStats;
MaterializeExecutor(MaterializeExecutor&&) = default;
@ -124,6 +135,9 @@ class MaterializeExecutor {
ReadContext _readDocumentContext;
Infos const& _infos;
Fetcher& _fetcher;
// for single collection case
LogicalCollection const* _collection = nullptr;
};
} // namespace aql

View File

@ -302,6 +302,13 @@ struct OptimizerRule {
// move document materialization after SORT and LIMIT
// this must be run AFTER all cluster rules as this rule
// needs to take into account query distribution across cluster nodes
// for arango search view
lateDocumentMaterializationArangoSearchRule,
// move document materialization after SORT and LIMIT
// this must be run AFTER all cluster rules as this rule
// needs to take into account query distribution across cluster nodes
// for index
lateDocumentMaterializationRule,
// splice subquery into the place of a subquery node

View File

@ -22,6 +22,7 @@
#include "OptimizerRulesFeature.h"
#include "Aql/ExecutionPlan.h"
#include "Aql/IndexNodeOptimizerRules.h"
#include "Aql/IResearchViewOptimizerRules.h"
#include "Aql/OptimizerRules.h"
#include "Basics/Exceptions.h"
@ -385,11 +386,16 @@ void OptimizerRulesFeature::addRules() {
OptimizerRule::makeFlags(OptimizerRule::Flags::CanBeDisabled,
OptimizerRule::Flags::ClusterOnly));
// apply late materialization for view queries
registerRule("late-document-materialization", arangodb::iresearch::lateDocumentMaterializationRule,
// apply late materialization for index queries
registerRule("late-document-materialization", lateDocumentMaterializationRule,
OptimizerRule::lateDocumentMaterializationRule,
OptimizerRule::makeFlags(OptimizerRule::Flags::CanBeDisabled));
// apply late materialization for view queries
registerRule("late-document-materialization-arangosearch", arangodb::iresearch::lateDocumentMaterializationArangoSearchRule,
OptimizerRule::lateDocumentMaterializationArangoSearchRule,
OptimizerRule::makeFlags(OptimizerRule::Flags::CanBeDisabled));
// add the storage-engine specific rules
addStorageEngineRules();
@ -406,7 +412,6 @@ void OptimizerRulesFeature::addRules() {
registerRule("splice-subqueries", spliceSubqueriesRule, OptimizerRule::spliceSubqueriesRule,
OptimizerRule::makeFlags(OptimizerRule::Flags::CanBeDisabled));
// finally sort all rules by their level
std::sort(_rules.begin(), _rules.end(),
[](OptimizerRule const& lhs, OptimizerRule const& rhs) noexcept {

View File

@ -76,6 +76,15 @@ void QuerySnippet::addNode(ExecutionNode* node) {
_expansions.emplace_back(node, false, false);
break;
}
case ExecutionNode::MATERIALIZE: {
auto collectionAccessingNode = dynamic_cast<CollectionAccessingNode*>(node);
// Materialize index node - true
// Materialize view node - false
if (collectionAccessingNode != nullptr) {
_expansions.emplace_back(node, true, false);
}
break;
}
default:
// do nothing
break;

View File

@ -28,6 +28,7 @@
#include "Aql/CollectNode.h"
#include "Aql/ExecutionNode.h"
#include "Aql/GraphNode.h"
#include "Aql/IndexNode.h"
#include "Aql/IResearchViewNode.h"
#include "Aql/ModificationNodes.h"
#include "Aql/SubqueryEndExecutionNode.h"
@ -92,8 +93,7 @@ RegisterPlan* RegisterPlan::clone(ExecutionPlan* otherPlan, ExecutionPlan* plan)
void RegisterPlan::after(ExecutionNode* en) {
switch (en->getType()) {
case ExecutionNode::ENUMERATE_COLLECTION:
case ExecutionNode::INDEX: {
case ExecutionNode::ENUMERATE_COLLECTION: {
depth++;
nrRegsHere.emplace_back(1);
// create a copy of the last value here
@ -113,6 +113,13 @@ void RegisterPlan::after(ExecutionNode* en) {
totalNrRegs++;
break;
}
case ExecutionNode::INDEX: {
auto ep = ExecutionNode::castTo<IndexNode const*>(en);
TRI_ASSERT(ep);
ep->planNodeRegisters(nrRegsHere, nrRegs, varInfo, totalNrRegs, ++depth);
break;
}
case ExecutionNode::ENUMERATE_LIST: {
depth++;
@ -322,7 +329,7 @@ void RegisterPlan::after(ExecutionNode* en) {
// may invalidate all references
RegisterId registerId = nrRegs.back() + 1;
nrRegs.emplace_back(registerId);
auto ep = ExecutionNode::castTo<MaterializeNode const*>(en);
auto ep = ExecutionNode::castTo<materialize::MaterializeNode const*>(en);
TRI_ASSERT(ep != nullptr);
varInfo.emplace(ep->outVariable().id, VarInfo(depth, totalNrRegs));
totalNrRegs++;

View File

@ -282,6 +282,7 @@ set(LIB_ARANGO_AQL_SOURCES
Aql/IndexExecutor.cpp
Aql/IndexHint.cpp
Aql/IndexNode.cpp
Aql/IndexNodeOptimizerRules.cpp
Aql/InputAqlItemRow.cpp
Aql/InsertModifier.cpp
Aql/KShortestPathsExecutor.cpp

View File

@ -1143,9 +1143,20 @@ function processQuery(query, explain, planIndex) {
if (node.filter) {
filter = ' ' + keyword('FILTER') + ' ' + buildExpression(node.filter) + ' ' + annotation('/* early pruning */');
}
let indexAnnotation = '';
let indexVariables = '';
if (node.hasOwnProperty('outNmDocId')) {
indexAnnotation += '/* with late materialization */';
if (node.hasOwnProperty('IndexValuesVars') && node.IndexValuesVars.length > 0) {
indexVariables = node.IndexValuesVars.map(function (IndexValuesVar) {
return keyword(' LET ') + variableName(IndexValuesVar) + ' = ' + variableName(node.outVariable) + '.' + attribute(IndexValuesVar.field);
}).join('');
}
}
node.indexes.forEach(function (idx, i) { iterateIndexes(idx, i, node, types, false); });
return `${keyword('FOR')} ${variableName(node.outVariable)} ${keyword('IN')} ${collection(node.collection)} ${annotation(`/* ${types.join(', ')}${projection(node)}${node.satellite ? ', satellite' : ''}${restriction(node)}`)} */` + filter;
return `${keyword('FOR')} ${variableName(node.outVariable)} ${keyword('IN')} ${collection(node.collection)}` + indexVariables +
` ${annotation(`/* ${types.join(', ')}${projection(node)}${node.satellite ? ', satellite' : ''}${restriction(node)} */`)} ` + filter +
' ' + annotation(indexAnnotation);
case 'TraversalNode':
if (node.hasOwnProperty("options")) {
node.minMaxDepth = node.options.minDepth + '..' + node.options.maxDepth;

898
tests/Aql/IndexNodeTest.cpp Normal file
View File

@ -0,0 +1,898 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2019 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 Yuriy Popov
////////////////////////////////////////////////////////////////////////////////
#include "gtest/gtest.h"
#include "Aql/Ast.h"
#include "Aql/AqlItemBlockSerializationFormat.h"
#include "Aql/IndexNode.h"
#include "Aql/Query.h"
#include "Mocks/Servers.h"
#include "RestServer/QueryRegistryFeature.h"
#include "Transaction/StandaloneContext.h"
#include "velocypack/Iterator.h"
#include "VocBase/LogicalCollection.h"
#include "VocBase/ManagedDocumentResult.h"
namespace {
class IndexNodeTest
: public ::testing::Test,
public arangodb::tests::LogSuppressor<arangodb::Logger::AUTHENTICATION, arangodb::LogLevel::ERR> {
protected:
arangodb::tests::mocks::MockAqlServer server;
IndexNodeTest() : server(false) {
server.startFeatures();
}
}; // IndexNodeTest
arangodb::CreateDatabaseInfo createInfo(arangodb::application_features::ApplicationServer& server) {
arangodb::CreateDatabaseInfo info(server);
info.allowSystemDB(false);
auto rv = info.load("testVocbase", 2);
if (rv.fail()) {
throw std::runtime_error(rv.errorMessage());
}
return info;
}
arangodb::aql::QueryResult executeQuery(TRI_vocbase_t& vocbase, std::string const& queryString,
std::shared_ptr<arangodb::velocypack::Builder> bindVars = nullptr,
std::string const& optionsString = "{}"
) {
arangodb::aql::Query query(false, vocbase, arangodb::aql::QueryString(queryString), bindVars,
arangodb::velocypack::Parser::fromJson(optionsString),
arangodb::aql::PART_MAIN);
std::shared_ptr<arangodb::aql::SharedQueryState> ss = query.sharedState();
arangodb::aql::QueryResult result;
while (true) {
auto state = query.execute(arangodb::QueryRegistryFeature::registry(), result);
if (state == arangodb::aql::ExecutionState::WAITING) {
ss->waitForAsyncWakeup();
} else {
break;
}
}
return result;
}
TEST_F(IndexNodeTest, objectQuery) {
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, createInfo(server.server()));
// create a collection
auto collectionJson = arangodb::velocypack::Parser::fromJson(
"{\"name\": \"testCollection\", \"id\": 42}");
auto collection = vocbase.createCollection(collectionJson->slice());
ASSERT_FALSE(!collection);
auto indexJson = arangodb::velocypack::Parser::fromJson("{\"type\": \"hash\", \"fields\": [\"obj.a\", \"obj.b\", \"obj.c\"]}");
auto createdIndex = false;
auto index = collection->createIndex(indexJson->slice(), createdIndex);
ASSERT_TRUE(createdIndex);
ASSERT_FALSE(!index);
std::vector<std::string> const EMPTY;
arangodb::transaction::Methods trx(arangodb::transaction::StandaloneContext::Create(vocbase),
EMPTY, EMPTY, EMPTY,
arangodb::transaction::Options());
EXPECT_TRUE(trx.begin().ok());
arangodb::OperationOptions opt;
arangodb::ManagedDocumentResult mmdoc;
auto jsonDocument = arangodb::velocypack::Parser::fromJson("{\"_key\": \"doc\", \"obj\": {\"a\": \"a_val\", \"b\": \"b_val\", \"c\": \"c_val\"}}");
auto const res = collection->insert(&trx, jsonDocument->slice(), mmdoc, opt, false);
EXPECT_TRUE(res.ok());
EXPECT_TRUE(trx.commit().ok());
{
auto queryString = "FOR d IN testCollection FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 10 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(1, resultIt.size());
ASSERT_EQ(jsonDocument->slice().get("_key").toJson(), resultIt.value().get("_key").toJson());
}
// const object in condition
{
auto queryString = "FOR d IN testCollection FILTER d.obj.a == {sub_a: \"a_val\"}.sub_a SORT d.obj.c LIMIT 10 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(1, resultIt.size());
ASSERT_EQ(jsonDocument->slice().get("_key").toJson(), resultIt.value().get("_key").toJson());
}
// two index variables for registers
{
auto queryString = "FOR d IN testCollection FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 2 SORT d.obj.b DESC LIMIT 1 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(1, resultIt.size());
ASSERT_EQ(jsonDocument->slice().get("_key").toJson(), resultIt.value().get("_key").toJson());
}
}
TEST_F(IndexNodeTest, expansionQuery) {
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, createInfo(server.server()));
// create a collection
auto collectionJson = arangodb::velocypack::Parser::fromJson(
"{\"name\": \"testCollection\", \"id\": 42}");
auto collection = vocbase.createCollection(collectionJson->slice());
ASSERT_FALSE(!collection);
auto indexJson = arangodb::velocypack::Parser::fromJson("{\"type\": \"hash\", \"fields\": [\"tags.hop[*].foo.fo\", \"tags.hop[*].bar.br\", \"tags.hop[*].baz.bz\"]}");
auto createdIndex = false;
auto index = collection->createIndex(indexJson->slice(), createdIndex);
ASSERT_TRUE(createdIndex);
ASSERT_FALSE(!index);
std::vector<std::string> const EMPTY;
arangodb::transaction::Methods trx(arangodb::transaction::StandaloneContext::Create(vocbase),
EMPTY, EMPTY, EMPTY,
arangodb::transaction::Options());
EXPECT_TRUE(trx.begin().ok());
arangodb::OperationOptions opt;
arangodb::ManagedDocumentResult mmdoc;
auto jsonDocument0 = arangodb::velocypack::Parser::fromJson("{\"_key\": \"doc_0\", \"tags\": {\"hop\": [{\"foo\": {\"fo\": \"foo_val\"}, \"bar\": {\"br\": \"bar_val\"}, \"baz\": {\"bz\": \"baz_val_0\"}}]}}");
auto jsonDocument1 = arangodb::velocypack::Parser::fromJson("{\"_key\": \"doc_1\", \"tags\": {\"hop\": [{\"foo\": {\"fo\": \"foo_val\"}}, {\"bar\": {\"br\": \"bar_val\"}}, {\"baz\": {\"bz\": \"baz_val_1\"}}]}}");
auto const res0 = collection->insert(&trx, jsonDocument0->slice(), mmdoc, opt, false);
EXPECT_TRUE(res0.ok());
auto const res1 = collection->insert(&trx, jsonDocument1->slice(), mmdoc, opt, false);
EXPECT_TRUE(res1.ok());
EXPECT_TRUE(trx.commit().ok());
auto queryString = "FOR d IN testCollection FILTER 'foo_val' IN d.tags.hop[*].foo.fo SORT d.tags.hop[*].baz.bz LIMIT 2 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(2, resultIt.size());
ASSERT_EQ(jsonDocument1->slice().get("_key").toJson(), resultIt.value().get("_key").toJson());
ASSERT_EQ(jsonDocument0->slice().get("_key").toJson(), (++resultIt).value().get("_key").toJson());
}
TEST_F(IndexNodeTest, expansionIndexAndNotExpansionDocumentQuery) {
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, createInfo(server.server()));
// create a collection
auto collectionJson = arangodb::velocypack::Parser::fromJson(
"{\"name\": \"testCollection\", \"id\": 42}");
auto collection = vocbase.createCollection(collectionJson->slice());
ASSERT_FALSE(!collection);
auto indexJson = arangodb::velocypack::Parser::fromJson("{\"type\": \"hash\", \"fields\": [\"tags.hop[*].foo.fo\", \"tags.hop[*].bar.br\", \"tags.hop[*].baz.bz\"]}");
auto createdIndex = false;
auto index = collection->createIndex(indexJson->slice(), createdIndex);
ASSERT_TRUE(createdIndex);
ASSERT_FALSE(!index);
std::vector<std::string> const EMPTY;
arangodb::transaction::Methods trx(arangodb::transaction::StandaloneContext::Create(vocbase),
EMPTY, EMPTY, EMPTY,
arangodb::transaction::Options());
EXPECT_TRUE(trx.begin().ok());
arangodb::OperationOptions opt;
arangodb::ManagedDocumentResult mmdoc;
auto jsonDocument = arangodb::velocypack::Parser::fromJson("{\"tags\": {\"hop\": {\"foo\": {\"fo\": \"foo_val\"}, \"bar\": {\"br\": \"bar_val\"}, \"baz\": {\"bz\": \"baz_val\"}}}}");
auto const res = collection->insert(&trx, jsonDocument->slice(), mmdoc, opt, false);
EXPECT_TRUE(res.ok());
EXPECT_TRUE(trx.commit().ok());
auto queryString = "FOR d IN testCollection FILTER 'foo_val' IN d.tags.hop[*].foo.fo SORT d.tags.hop[*].baz.bz LIMIT 10 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(0, resultIt.size());
}
TEST_F(IndexNodeTest, lastExpansionQuery) {
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, createInfo(server.server()));
// create a collection
auto collectionJson = arangodb::velocypack::Parser::fromJson(
"{\"name\": \"testCollection\", \"id\": 42}");
auto collection = vocbase.createCollection(collectionJson->slice());
ASSERT_FALSE(!collection);
auto indexJson = arangodb::velocypack::Parser::fromJson("{\"type\": \"hash\", \"fields\": [\"tags[*]\"]}");
auto createdIndex = false;
auto index = collection->createIndex(indexJson->slice(), createdIndex);
ASSERT_TRUE(createdIndex);
ASSERT_FALSE(!index);
std::vector<std::string> const EMPTY;
arangodb::transaction::Methods trx(arangodb::transaction::StandaloneContext::Create(vocbase),
EMPTY, EMPTY, EMPTY,
arangodb::transaction::Options());
EXPECT_TRUE(trx.begin().ok());
arangodb::OperationOptions opt;
arangodb::ManagedDocumentResult mmdoc;
auto jsonDocument = arangodb::velocypack::Parser::fromJson("{\"_key\": \"doc\", \"tags\": [\"foo_val\", \"bar_val\", \"baz_val\"]}");
auto const res = collection->insert(&trx, jsonDocument->slice(), mmdoc, opt, false);
EXPECT_TRUE(res.ok());
EXPECT_TRUE(trx.commit().ok());
{
auto queryString = "FOR d IN testCollection FILTER 'foo_val' IN d.tags[*] SORT d.tags LIMIT 10 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(1, resultIt.size());
ASSERT_EQ(jsonDocument->slice().get("_key").toJson(), resultIt.value().get("_key").toJson());
}
{
auto queryString = "FOR d IN testCollection FILTER 'foo_val' IN d.tags SORT d.tags LIMIT 10 RETURN d";
auto queryResult = ::executeQuery(vocbase, queryString);
EXPECT_TRUE(queryResult.result.ok()); // commit
auto result = queryResult.data->slice();
EXPECT_TRUE(result.isArray());
arangodb::velocypack::ArrayIterator resultIt(result);
ASSERT_EQ(1, resultIt.size());
ASSERT_EQ(jsonDocument->slice().get("_key").toJson(), resultIt.value().get("_key").toJson());
}
}
TEST_F(IndexNodeTest, constructIndexNode) {
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, createInfo(server.server()));
// create a collection
auto collectionJson = arangodb::velocypack::Parser::fromJson(
"{\"name\": \"testCollection\", \"id\": 42}");
auto collection = vocbase.createCollection(collectionJson->slice());
ASSERT_FALSE(!collection);
// create an index node
auto indexJson = arangodb::velocypack::Parser::fromJson("{\"type\": \"hash\", \"id\": 2086177, \"fields\": [\"obj.a\", \"obj.b\", \"obj.c\"]}");
auto createdIndex = false;
auto index = collection->createIndex(indexJson->slice(), createdIndex);
ASSERT_TRUE(createdIndex);
ASSERT_FALSE(!index);
// auto jsonDocument = arangodb::velocypack::Parser::fromJson("{\"obj\": {\"a\": \"a_val\", \"b\": \"b_val\", \"c\": \"c_val\"}}");
// correct json
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : 2,"
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ],"
" \"indexIdOfVars\" : 2086177,"
" \"ascending\" : true,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" \"subNodes\" : ["
" {"
" \"subNodes\" : ["
" {"
" \"excludesNull\" : false,"
" \"subNodes\" : ["
" {"
" \"name\" : \"a\","
" \"subNodes\" : ["
" {"
" \"name\" : \"obj\","
" \"subNodes\" : ["
" {"
" \"id\" : 0,"
" \"name\" : \"d\","
" \"type\" : \"reference\","
" \"typeID\" : 45"
" }"
" ],"
" \"type\" : \"attribute access\","
" \"typeID\" : 35"
" }"
" ],"
" \"type\" : \"attribute access\","
" \"typeID\" : 35"
" },"
" {"
" \"type\" : \"value\","
" \"typeID\" : 40,"
" \"vType\" : \"string\","
" \"vTypeID\" : 4,"
" \"value\" : \"a_val\""
" }"
" ],"
" \"type\" : \"compare ==\","
" \"typeID\" : 25"
" }"
" ],"
" \"type\" : \"n-ary and\","
" \"typeID\" : 62"
" }"
" ],"
" \"type\" : \"n-ary or\","
" \"typeID\" : 63"
" },"
" \"database\" : \"testVocbase\","
" \"dependencies\" : ["
" 1"
" ],"
" \"depth\" : 1,"
" \"evalFCalls\" : true,"
" \"id\" : 9,"
" \"indexCoversProjections\" : false,"
" \"indexes\" : ["
" {"
" \"deduplicate\" : true,"
" \"fields\" : ["
" \"obj.a\","
" \"obj.b\","
" \"obj.c\""
" ],"
" \"id\" : \"2086177\","
" \"name\" : \"idx_1648634948960124928\","
" \"selectivityEstimate\" : 1,"
" \"sparse\" : false,"
" \"type\" : \"hash\","
" \"unique\" : false"
" }"
" ],"
" \"isSatellite\" : false,"
" \"limit\" : 0,"
" \"needsGatherNodeSort\" : false,"
" \"nrRegs\" : ["
" 0,"
" 3,"
" 4"
" ],"
" \"nrRegsHere\" : ["
" 0,"
" 3,"
" 1"
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" },"
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"producesResult\" : true,"
" \"projections\" : ["
" ],"
" \"regsToClear\" : ["
" ],"
" \"reverse\" : false,"
" \"satellite\" : false,"
" \"sorted\" : true,"
" \"totalNrRegs\" : 4,"
" \"type\" : \"IndexNode\","
" \"typeID\" : 23,"
" \"varInfoList\" : ["
" {"
" \"RegisterId\" : 3,"
" \"VariableId\" : 0,"
" \"depth\" : 2"
" },"
" {"
" \"RegisterId\" : 2,"
" \"VariableId\" : 4,"
" \"depth\" : 1"
" },"
" {"
" \"RegisterId\" : 0,"
" \"VariableId\" : 8,"
" \"depth\" : 1"
" },"
" {"
" \"RegisterId\" : 1,"
" \"VariableId\" : 6,"
" \"depth\" : 1"
" }"
" ],"
" \"varsUsedLater\" : ["
" {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" {"
" \"id\" : 8,"
" \"name\" : \"7\""
" },"
" {"
" \"id\" : 4,"
" \"name\" : \"3\""
" },"
" {"
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ],"
" \"varsValid\" : ["
" {"
" \"id\" : 8,"
" \"name\" : \"7\""
" },"
" {"
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ]"
"}"
);
arangodb::aql::Query query(false, vocbase, arangodb::aql::QueryString(
"FOR d IN testCollection FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 10 RETURN d"),
nullptr, arangodb::velocypack::Parser::fromJson("{}"),
arangodb::aql::PART_MAIN);
query.prepare(arangodb::QueryRegistryFeature::registry(),
arangodb::aql::SerializationFormat::SHADOWROWS);
{
// short path for a test
{
auto vars = query.plan()->getAst()->variables();
for (auto const& v : {std::make_unique<arangodb::aql::Variable>("d", 0),
std::make_unique<arangodb::aql::Variable>("3", 4),
std::make_unique<arangodb::aql::Variable>("5", 6),
std::make_unique<arangodb::aql::Variable>("7", 8)}) {
if (vars->getVariable(v->id) == nullptr) {
vars->createVariable(v.get());
}
}
}
// deserialization
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
ASSERT_TRUE(indNode.isLateMaterialized());
// serialization and deserialization
{
VPackBuilder builder;
std::unordered_set<arangodb::aql::ExecutionNode const*> seen;
{
VPackArrayBuilder guard(&builder);
indNode.toVelocyPackHelper(builder, arangodb::aql::ExecutionNode::SERIALIZE_DETAILS, seen);
}
arangodb::aql::IndexNode indNodeDeserialized(query.plan(), createJson->slice());
ASSERT_TRUE(indNodeDeserialized.isLateMaterialized());
}
// clone
{
// without properties
{
auto indNodeClone = dynamic_cast<arangodb::aql::IndexNode*>(indNode.clone(query.plan(), true, false));
EXPECT_EQ(indNode.getType(), indNodeClone->getType());
EXPECT_EQ(indNode.outVariable(), indNodeClone->outVariable());
EXPECT_EQ(indNode.plan(), indNodeClone->plan());
EXPECT_EQ(indNode.vocbase(), indNodeClone->vocbase());
EXPECT_EQ(indNode.isLateMaterialized(), indNodeClone->isLateMaterialized());
ASSERT_TRUE(indNodeClone->isLateMaterialized());
}
// with properties
{
arangodb::aql::Query queryClone(false, vocbase, arangodb::aql::QueryString(
"RETURN 1"),
nullptr, arangodb::velocypack::Parser::fromJson("{}"),
arangodb::aql::PART_MAIN);
queryClone.prepare(arangodb::QueryRegistryFeature::registry(),
arangodb::aql::SerializationFormat::SHADOWROWS);
indNode.invalidateVarUsage();
auto indNodeClone = dynamic_cast<arangodb::aql::IndexNode*>(indNode.clone(queryClone.plan(), true, true));
EXPECT_EQ(indNode.getType(), indNodeClone->getType());
EXPECT_NE(indNode.outVariable(), indNodeClone->outVariable());
EXPECT_NE(indNode.plan(), indNodeClone->plan());
EXPECT_EQ(indNode.vocbase(), indNodeClone->vocbase());
EXPECT_EQ(indNode.isLateMaterialized(), indNodeClone->isLateMaterialized());
ASSERT_TRUE(indNodeClone->isLateMaterialized());
}
}
// not materialized
{
indNode.setLateMaterialized(nullptr, 0, arangodb::aql::IndexNode::IndexVarsInfo());
ASSERT_FALSE(indNode.isLateMaterialized());
}
}
}
TEST_F(IndexNodeTest, invalidLateMaterializedJSON) {
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, createInfo(server.server()));
// create a collection
auto collectionJson = arangodb::velocypack::Parser::fromJson(
"{\"name\": \"testCollection\", \"id\": 42}");
auto collection = vocbase.createCollection(collectionJson->slice());
ASSERT_FALSE(!collection);
arangodb::aql::Query query(false, vocbase, arangodb::aql::QueryString(
"FOR d IN testCollection FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 10 RETURN d"),
nullptr, arangodb::velocypack::Parser::fromJson("{}"),
arangodb::aql::PART_MAIN);
query.prepare(arangodb::QueryRegistryFeature::registry(),
arangodb::aql::SerializationFormat::SHADOWROWS);
auto vars = query.plan()->getAst()->variables();
auto const& v = std::make_unique<arangodb::aql::Variable>("5", 6);
if (vars->getVariable(v->id) == nullptr) {
vars->createVariable(v.get());
}
// correct json
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : 2,"
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ],"
" \"indexIdOfVars\" : 2086177,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" }, "
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
// deserialization
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
ASSERT_TRUE(indNode.isLateMaterialized());
}
// incorrect IndexValuesVars
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : {"
" \"fieldNumber\" : 2,"
" \"id\" : 6,"
" \"name\" : \"5\""
" },"
" \"indexIdOfVars\" : 2086177,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" }, "
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
// deserialization
try {
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
EXPECT_TRUE(false);
} catch (arangodb::basics::Exception const& e) {
EXPECT_EQ(TRI_ERROR_BAD_PARAMETER, e.code());
} catch (...) {
EXPECT_TRUE(false);
}
}
// incorrect fieldNumber
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : \"two\","
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ],"
" \"indexIdOfVars\" : 2086177,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" }, "
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
// deserialization
try {
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
EXPECT_TRUE(false);
} catch (arangodb::basics::Exception const& e) {
EXPECT_EQ(TRI_ERROR_BAD_PARAMETER, e.code());
} catch (...) {
EXPECT_TRUE(false);
}
}
// incorrect id
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : 2,"
" \"id\" : \"six\","
" \"name\" : \"5\""
" }"
" ],"
" \"indexIdOfVars\" : 2086177,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" }, "
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
// deserialization
try {
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
EXPECT_TRUE(false);
} catch (arangodb::basics::Exception const& e) {
EXPECT_EQ(TRI_ERROR_BAD_PARAMETER, e.code());
} catch (...) {
EXPECT_TRUE(false);
}
}
// incorrect name
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : 2,"
" \"id\" : 6,"
" \"name\" : 5"
" }"
" ],"
" \"indexIdOfVars\" : 2086177,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" }, "
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
ASSERT_TRUE(indNode.isLateMaterialized()); // do not read the name
}
// incorrect indexIdOfVars
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : 2,"
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ],"
" \"indexIdOfVars\" : \"2086177\","
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outNmDocId\" : {"
" \"id\" : 8,"
" \"name\" : \"7\""
" }, "
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
// deserialization
try {
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
EXPECT_TRUE(false);
} catch (arangodb::basics::Exception const& e) {
EXPECT_EQ(TRI_ERROR_BAD_PARAMETER, e.code());
} catch (...) {
EXPECT_TRUE(false);
}
}
// no outNmDocId
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{"
" \"IndexValuesVars\" : ["
" {"
" \"fieldNumber\" : 2,"
" \"id\" : 6,"
" \"name\" : \"5\""
" }"
" ],"
" \"indexIdOfVars\" : 2086177,"
" \"collection\" : \"testCollection\","
" \"condition\" : {"
" },"
" \"depth\" : 1,"
" \"id\" : 9,"
" \"indexes\" : ["
" ],"
" \"nrRegs\" : ["
" ],"
" \"nrRegsHere\" : ["
" ],"
" \"outVariable\" : {"
" \"id\" : 0,"
" \"name\" : \"d\""
" },"
" \"regsToClear\" : ["
" ],"
" \"totalNrRegs\" : 0,"
" \"varInfoList\" : ["
" ],"
" \"varsUsedLater\" : ["
" ],"
" \"varsValid\" : ["
" ]"
"}"
);
// deserialization
arangodb::aql::IndexNode indNode(query.plan(), createJson->slice());
ASSERT_FALSE(indNode.isLateMaterialized());
}
}
}

View File

@ -44,6 +44,7 @@ set(ARANGODB_TESTS_SOURCES
Aql/FilterExecutorTest.cpp
Aql/HashedCollectExecutorTest.cpp
Aql/IdExecutorTest.cpp
Aql/IndexNodeTest.cpp
Aql/InsertExecutorTest.cpp
Aql/LimitExecutorTest.cpp
Aql/MultiDepFetcherHelper.cpp

View File

@ -21,6 +21,9 @@
/// @author Vasiliy Nabatchikov
////////////////////////////////////////////////////////////////////////////////
#ifndef ARANGOD_AQL_IRESEARCH_QUERY_COMMON_H
#define ARANGOD_AQL_IRESEARCH_QUERY_COMMON_H
#include "gtest/gtest.h"
#include "3rdParty/iresearch/tests/tests_config.hpp"
@ -149,3 +152,5 @@ class IResearchQueryTest
return *_vocbase;
}
}; // IResearchQueryTest
#endif // ARANGOD_AQL_IRESEARCH_QUERY_COMMON_H

View File

@ -36,6 +36,7 @@
#include "IResearch/VelocyPackHelper.h"
#include "Indexes/IndexIterator.h"
#include "Indexes/SimpleAttributeEqualityMatcher.h"
#include "Indexes/SortedIndexAttributeMatcher.h"
#include "RestServer/FlushFeature.h"
#include "StorageEngine/EngineSelectorFeature.h"
#include "Transaction/Helpers.h"
@ -432,6 +433,456 @@ struct IndexFactoryMock : arangodb::IndexFactory {
}
};
class HashIndexMap {
struct VPackBuilderHasher {
std::size_t operator()(VPackBuilder const& builder) const {
return std::hash<VPackSlice>()(builder.slice());
}
};
struct VPackBuilderComparator {
bool operator()(VPackBuilder const& builder1, VPackBuilder const& builder2) const {
return ::arangodb::basics::VelocyPackHelper::compare(builder1.slice(), builder2.slice(), true) == 0;
}
};
using ValueMap = std::unordered_multimap<VPackBuilder, arangodb::LocalDocumentId, VPackBuilderHasher, VPackBuilderComparator>;
using DocumentsIndexMap = std::unordered_map<arangodb::LocalDocumentId, VPackBuilder>;
arangodb::velocypack::Slice getSliceByField(arangodb::velocypack::Slice const& doc, size_t i) {
TRI_ASSERT(i < _fields.size());
TRI_ASSERT(!doc.isNone());
auto slice = doc;
for (auto const& f : _fields[i]) {
slice = slice.get(f.name);
if (slice.isNone() || slice.isNull()) {
break;
}
}
return slice;
}
void insertSlice(arangodb::LocalDocumentId const& documentId, arangodb::velocypack::Slice const& slice, size_t i) {
VPackBuilder builder;
if (slice.isNone() || slice.isNull()) {
builder.add(VPackSlice::nullSlice());
} else {
builder.add(slice);
}
_valueMaps[i].emplace(std::move(builder), documentId);
}
public:
HashIndexMap(std::vector<std::vector<arangodb::basics::AttributeName>> const& fields) : _fields(fields), _valueMaps(fields.size()) {
TRI_ASSERT(!_fields.empty());
}
void insert(arangodb::LocalDocumentId const& documentId, arangodb::velocypack::Slice const& doc) {
VPackBuilder builder;
builder.openArray();
auto toClose = true;
// find fields for the index
for (size_t i = 0; i < _fields.size(); ++i) {
auto slice = doc;
auto isExpansion = false;
for (auto fieldIt = _fields[i].begin(); fieldIt != _fields[i].end(); ++fieldIt) {
TRI_ASSERT(slice.isObject() || slice.isArray());
if (slice.isObject()) {
slice = slice.get(fieldIt->name);
if ((fieldIt->shouldExpand && slice.isObject()) ||
(!fieldIt->shouldExpand && slice.isArray())) {
slice = VPackSlice::nullSlice();
break;
}
if (slice.isNone() || slice.isNull()) {
break;
}
} else { // expansion
isExpansion = slice.isArray();
TRI_ASSERT(isExpansion);
auto found = false;
for (auto sliceIt = arangodb::velocypack::ArrayIterator(slice); sliceIt != sliceIt.end(); ++sliceIt) {
auto subSlice = sliceIt.value();
if (!(subSlice.isNone() || subSlice.isNull())) {
for (auto fieldItForArray = fieldIt; fieldItForArray != _fields[i].end(); ++fieldItForArray) {
TRI_ASSERT(subSlice.isObject());
subSlice = subSlice.get(fieldItForArray->name);
if (subSlice.isNone() || subSlice.isNull()) {
break;
}
}
if (!(subSlice.isNone() || subSlice.isNull())) {
insertSlice(documentId, subSlice, i);
builder.add(subSlice);
found = true;
break;
}
}
}
if (!found) {
insertSlice(documentId, VPackSlice::nullSlice(), i);
builder.add(VPackSlice::nullSlice());
}
break;
}
}
if (!isExpansion) {
// if the last expansion (at the end) leave the array open
if (slice.isArray() && i == _fields.size() - 1) {
auto found = false;
auto wasNull = false;
for (auto sliceIt = arangodb::velocypack::ArrayIterator(slice); sliceIt != sliceIt.end(); ++sliceIt) {
auto subSlice = sliceIt.value();
if (!(subSlice.isNone() || subSlice.isNull())) {
insertSlice(documentId, subSlice, i);
found = true;
} else {
wasNull = true;
}
}
if (!found || wasNull) {
insertSlice(documentId, VPackSlice::nullSlice(), i);
}
toClose = false;
} else { // object
insertSlice(documentId, slice, i);
builder.add(slice);
}
}
}
if (toClose) {
builder.close();
}
_docIndexMap.try_emplace(documentId, std::move(builder));
}
bool remove(arangodb::LocalDocumentId const& documentId, arangodb::velocypack::Slice const& doc) {
size_t i = 0;
auto documentRemoved = false;
for (auto& map : _valueMaps) {
auto slice = getSliceByField(doc, i++);
auto [begin, end] = map.equal_range(VPackBuilder(slice));
for (; begin != end; ++begin) {
if (begin->second == documentId) {
map.erase(begin);
documentRemoved = true;
// not break because of expansions
}
}
}
_docIndexMap.erase(documentId);
return documentRemoved;
}
void clear() {
_valueMaps.clear();
_docIndexMap.clear();
}
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder> find(std::unique_ptr<VPackBuilder>&& keys) const {
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder const*> found;
TRI_ASSERT(keys->slice().isArray());
auto sliceIt = arangodb::velocypack::ArrayIterator(keys->slice());
if (!sliceIt.valid()) {
return std::unordered_map<arangodb::LocalDocumentId, VPackBuilder>();
}
for (auto const& map : _valueMaps) {
auto [begin, end] = map.equal_range(VPackBuilder(sliceIt.value()));
if (begin == end) {
return std::unordered_map<arangodb::LocalDocumentId, VPackBuilder>();
}
if (found.empty()) {
std::transform(begin, end, std::inserter(found, found.end()), [] (auto const& item) {
return std::make_pair(item.second, &item.first);
});
} else {
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder const*> tmpFound;
for (; begin != end; ++begin) {
if (found.find(begin->second) != found.cend()) {
tmpFound.try_emplace(begin->second, &begin->first);
}
}
if (tmpFound.empty()) {
return std::unordered_map<arangodb::LocalDocumentId, VPackBuilder>();
}
found.swap(tmpFound);
}
if (!(++sliceIt).valid()) {
break;
}
}
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder> foundWithCovering;
for (auto const& d : found) {
auto doc = _docIndexMap.find(d.first);
TRI_ASSERT(doc != _docIndexMap.cend());
auto builder = doc->second;
// the array was left open for the last expansion (at the end)
if (doc->second.isOpenArray()) {
builder.add(d.second->slice());
builder.close();
}
foundWithCovering.try_emplace(doc->first, std::move(builder));
}
return foundWithCovering;
}
private:
std::vector<std::vector<arangodb::basics::AttributeName>> const& _fields;
std::vector<ValueMap> _valueMaps;
DocumentsIndexMap _docIndexMap;
};
class HashIndexIteratorMock final : public arangodb::IndexIterator {
public:
HashIndexIteratorMock(arangodb::LogicalCollection* collection,
arangodb::transaction::Methods* trx, arangodb::Index const* index,
HashIndexMap const& map, std::unique_ptr<VPackBuilder>&& keys)
: IndexIterator(collection, trx), _map(map) {
_documents = _map.find(std::move(keys));
_begin = _documents.begin();
_end = _documents.end();
}
char const* typeName() const override { return "hash-index-iterator-mock"; }
bool nextCovering(DocumentCallback const& cb, size_t limit) override {
while (limit && _begin != _end) {
cb(_begin->first, _begin->second.slice());
++_begin;
--limit;
}
return _begin != _end;
}
bool next(LocalDocumentIdCallback const& cb, size_t limit) override {
while (limit && _begin != _end) {
cb(_begin->first);
++_begin;
--limit;
}
return _begin != _end;
}
void reset() override {
_documents.clear();
_begin = _documents.begin();
_end = _documents.end();
}
bool hasCovering() const override {
return true;
}
private:
HashIndexMap const& _map;
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder> _documents;
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder>::const_iterator _begin;
std::unordered_map<arangodb::LocalDocumentId, VPackBuilder>::const_iterator _end;
}; // HashIndexIteratorMock
class HashIndexMock final : public arangodb::Index {
public:
static std::shared_ptr<arangodb::Index> make(TRI_idx_iid_t iid,
arangodb::LogicalCollection& collection,
arangodb::velocypack::Slice const& definition) {
auto const typeSlice = definition.get("type");
if (typeSlice.isNone()) {
return nullptr;
}
auto const type = arangodb::basics::VelocyPackHelper::getStringRef(typeSlice,
arangodb::velocypack::StringRef());
if (type.compare("hash") != 0) {
return nullptr;
}
return std::make_shared<HashIndexMock>(iid, collection, definition);
}
IndexType type() const override { return Index::TRI_IDX_TYPE_HASH_INDEX; }
char const* typeName() const override { return "hash"; }
bool isPersistent() const override { return false; }
bool canBeDropped() const override { return false; }
bool hasCoveringIterator() const override { return true; }
bool isHidden() const override { return false; }
bool isSorted() const override { return false; }
bool hasSelectivityEstimate() const override { return false; }
size_t memory() const override { return sizeof(HashIndexMock); }
void load() override {}
void unload() override {}
void afterTruncate(TRI_voc_tick_t) override {
_hashData.clear();
}
void toVelocyPack(VPackBuilder& builder,
std::underlying_type<arangodb::Index::Serialize>::type flags) const override {
builder.openObject();
Index::toVelocyPack(builder, flags);
builder.add("sparse", VPackValue(sparse()));
builder.add("unique", VPackValue(unique()));
builder.close();
}
void toVelocyPackFigures(VPackBuilder& builder) const override {
Index::toVelocyPackFigures(builder);
}
arangodb::Result insert(arangodb::transaction::Methods&,
arangodb::LocalDocumentId const& documentId,
arangodb::velocypack::Slice const& doc, OperationMode) {
if (!doc.isObject()) {
return {TRI_ERROR_INTERNAL};
}
_hashData.insert(documentId, doc);
return {}; // ok
}
arangodb::Result remove(arangodb::transaction::Methods&,
arangodb::LocalDocumentId const& documentId,
arangodb::velocypack::Slice const& doc, OperationMode) {
if (!doc.isObject()) {
return {TRI_ERROR_INTERNAL};
}
_hashData.remove(documentId, doc);
return {}; // ok
}
Index::FilterCosts supportsFilterCondition(
std::vector<std::shared_ptr<arangodb::Index>> const& allIndexes,
arangodb::aql::AstNode const* node, arangodb::aql::Variable const* reference,
size_t itemsInIndex) const override {
return arangodb::SortedIndexAttributeMatcher::supportsFilterCondition(allIndexes, this, node, reference, itemsInIndex);
}
Index::SortCosts supportsSortCondition(arangodb::aql::SortCondition const* sortCondition,
arangodb::aql::Variable const* reference,
size_t itemsInIndex) const override {
return arangodb::SortedIndexAttributeMatcher::supportsSortCondition(this, sortCondition, reference, itemsInIndex);
}
arangodb::aql::AstNode* specializeCondition(
arangodb::aql::AstNode* node, arangodb::aql::Variable const* reference) const override {
return arangodb::SortedIndexAttributeMatcher::specializeCondition(this, node, reference);
}
std::unique_ptr<arangodb::IndexIterator> iteratorForCondition(
arangodb::transaction::Methods* trx, arangodb::aql::AstNode const* node,
arangodb::aql::Variable const*, arangodb::IndexIteratorOptions const&) override {
arangodb::transaction::BuilderLeaser builder(trx);
std::unique_ptr<VPackBuilder> keys(builder.steal());
keys->openArray();
if (nullptr == node) {
keys->close();
return std::make_unique<HashIndexIteratorMock>(&_collection, trx, this,
_hashData,
std::move(keys));
}
TRI_ASSERT(node->type == arangodb::aql::NODE_TYPE_OPERATOR_NARY_AND);
std::vector<std::pair<std::vector<arangodb::basics::AttributeName>, arangodb::aql::AstNode*>> allAttributes;
for (size_t i = 0; i < node->numMembers(); ++i) {
auto comp = node->getMember(i);
// a.b == value
if (!(comp->type == arangodb::aql::NODE_TYPE_OPERATOR_BINARY_EQ ||
comp->type == arangodb::aql::NODE_TYPE_OPERATOR_BINARY_IN)) {
// operator type unsupported
return std::make_unique<arangodb::EmptyIndexIterator>(&_collection, trx);
}
// assume a.b == value
auto attrNode = comp->getMember(0);
auto valNode = comp->getMember(1);
if (!(attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS ||
attrNode->type == arangodb::aql::NODE_TYPE_EXPANSION)) {
// got value == a.b -> flip sides
std::swap(attrNode, valNode);
}
TRI_ASSERT(attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS ||
attrNode->type == arangodb::aql::NODE_TYPE_EXPANSION);
std::vector<arangodb::basics::AttributeName> attributes;
if (attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS) {
do {
attributes.emplace_back(std::string(attrNode->getStringValue(), attrNode->getStringLength()), false);
attrNode = attrNode->getMember(0);
} while (attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS);
std::reverse(attributes.begin(), attributes.end());
} else { // expansion
TRI_ASSERT(attrNode->type == arangodb::aql::NODE_TYPE_EXPANSION);
auto expNode = attrNode;
TRI_ASSERT(expNode->numMembers() >= 2);
auto left = expNode->getMember(0);
TRI_ASSERT(left->type == arangodb::aql::NODE_TYPE_ITERATOR);
attrNode = left->getMember(1);
TRI_ASSERT(attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS);
do {
attributes.emplace_back(std::string(attrNode->getStringValue(), attrNode->getStringLength()), false);
attrNode = attrNode->getMember(0);
} while (attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS);
attributes.front().shouldExpand = true;
std::reverse(attributes.begin(), attributes.end());
std::vector<arangodb::basics::AttributeName> attributesRight;
attrNode = expNode->getMember(1);
TRI_ASSERT(attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS ||
attrNode->type == arangodb::aql::NODE_TYPE_REFERENCE);
while (attrNode->type == arangodb::aql::NODE_TYPE_ATTRIBUTE_ACCESS) {
attributesRight.emplace_back(std::string(attrNode->getStringValue(), attrNode->getStringLength()), false);
attrNode = attrNode->getMember(0);
}
attributes.insert(attributes.end(), attributesRight.crbegin(), attributesRight.crend());
}
allAttributes.emplace_back(std::move(attributes), valNode);
}
size_t nullsCount = 0;
for (auto const& f : _fields) {
auto it = std::find_if(allAttributes.cbegin(), allAttributes.cend(), [&f] (auto const& attrs) {
return arangodb::basics::AttributeName::isIdentical(attrs.first, f, true);
});
if (it != allAttributes.cend()) {
while (nullsCount > 0) {
keys->add(VPackSlice::nullSlice());
--nullsCount;
}
it->second->toVelocyPackValue(*keys);
} else {
++nullsCount;
}
}
keys->close();
return std::make_unique<HashIndexIteratorMock>(&_collection, trx, this,
_hashData,
std::move(keys));
}
HashIndexMock(TRI_idx_iid_t iid, arangodb::LogicalCollection& collection, VPackSlice const& slice)
: arangodb::Index(iid, collection, slice), _hashData(_fields) {}
/// @brief the hash table for data
HashIndexMap _hashData;
}; // HashIndexMock
} // namespace
PhysicalCollectionMock::DocElement::DocElement(
@ -502,6 +953,8 @@ std::shared_ptr<arangodb::Index> PhysicalCollectionMock::createIndex(
if (0 == type.compare("edge")) {
index = EdgeIndexMock::make(id, _logicalCollection, info);
} else if (0 == type.compare("hash")) {
index = HashIndexMock::make(id, _logicalCollection, info);
} else if (0 == type.compare(arangodb::iresearch::DATA_SOURCE_TYPE.name())) {
try {
if (arangodb::ServerState::instance()->isCoordinator()) {
@ -543,6 +996,12 @@ std::shared_ptr<arangodb::Index> PhysicalCollectionMock::createIndex(
for (auto const& pair : docs) {
l->insert(trx, pair.first, pair.second, arangodb::Index::OperationMode::internal);
}
} else if (index->type() == arangodb::Index::TRI_IDX_TYPE_HASH_INDEX) {
auto* l = dynamic_cast<HashIndexMock*>(index.get());
TRI_ASSERT(l != nullptr);
for (auto const& pair : docs) {
l->insert(trx, pair.first, pair.second, arangodb::Index::OperationMode::internal);
}
} else if (index->type() == arangodb::Index::TRI_IDX_TYPE_IRESEARCH_LINK) {
auto* l = dynamic_cast<arangodb::iresearch::IResearchLink*>(index.get());
TRI_ASSERT(l != nullptr);
@ -655,6 +1114,15 @@ arangodb::Result PhysicalCollectionMock::insert(
return arangodb::Result(TRI_ERROR_BAD_PARAMETER);
}
continue;
} else if (index->type() == arangodb::Index::TRI_IDX_TYPE_HASH_INDEX) {
auto* l = static_cast<HashIndexMock*>(index.get());
if (!l->insert(*trx, ref->second.docId(),
arangodb::velocypack::Slice(result.vpack()),
arangodb::Index::OperationMode::normal)
.ok()) {
return arangodb::Result(TRI_ERROR_BAD_PARAMETER);
}
continue;
} else if (index->type() == arangodb::Index::TRI_IDX_TYPE_IRESEARCH_LINK) {
if (arangodb::ServerState::instance()->isCoordinator()) {
auto* l =

View File

@ -2,7 +2,7 @@
/*global assertTrue, assertFalse, assertEqual, assertNotEqual, AQL_EXECUTE, AQL_EXPLAIN */
////////////////////////////////////////////////////////////////////////////////
/// @brief tests for late document materialization rule
/// @brief tests for late document materialization arangosearch rule
///
/// @file
///
@ -32,8 +32,8 @@ let jsunity = require("jsunity");
let db = require("@arangodb").db;
let isCluster = require("internal").isCluster();
function lateDocumentMaterializationRuleTestSuite () {
const ruleName = "late-document-materialization";
function lateDocumentMaterializationArangoSearchRuleTestSuite () {
const ruleName = "late-document-materialization-arangosearch";
const cn = "UnitTestsCollection";
const cn1 = "UnitTestsCollection1";
const vn = "UnitTestsView";
@ -305,6 +305,6 @@ function lateDocumentMaterializationRuleTestSuite () {
};
}
jsunity.run(lateDocumentMaterializationRuleTestSuite);
jsunity.run(lateDocumentMaterializationArangoSearchRuleTestSuite);
return jsunity.done();
return jsunity.done();

View File

@ -0,0 +1,416 @@
/*jshint globalstrict:false, strict:false, maxlen: 500 */
/*global assertTrue, assertFalse, assertEqual, assertNotEqual, AQL_EXECUTE, AQL_EXPLAIN */
////////////////////////////////////////////////////////////////////////////////
/// @brief tests for late document materialization rule
///
/// @file
///
/// DISCLAIMER
///
/// Copyright 2019 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 Yuriy Popov
/// @author Copyright 2019, ArangoDB GmbH, Cologne, Germany
////////////////////////////////////////////////////////////////////////////////
let jsunity = require("jsunity");
let db = require("@arangodb").db;
let isCluster = require("internal").isCluster();
function lateDocumentMaterializationRuleTestSuite () {
const ruleName = "late-document-materialization";
const numOfCollectionIndexes = 2;
const numOfExpCollections = 2;
let collectionNames = [];
let expCollectionNames = [];
let primaryIndexCollectionName = "UnitTestsPrimCollection";
let edgeIndexCollectionName = "UnitTestsEdgeCollection";
let severalIndexesCollectionName = "UnitTestsSeveralIndexesCollection";
var i;
var j;
for (i = 0; i < numOfCollectionIndexes; ++i) {
collectionNames.push("UnitTestsCollection" + i);
}
for (i = 0; i < numOfExpCollections * numOfCollectionIndexes; ++i) {
expCollectionNames.push("UnitTestsExpCollection" + i);
}
return {
setUpAll : function () {
for (i = 0; i < numOfCollectionIndexes; ++i) {
db._drop(collectionNames[i]);
for (j = 0; j < numOfExpCollections; ++j) {
db._drop(expCollectionNames[i * numOfCollectionIndexes + j]);
}
}
db._drop(primaryIndexCollectionName);
db._drop(edgeIndexCollectionName);
db._drop(severalIndexesCollectionName);
var collections = [];
var expCollections = [];
for (i = 0; i < numOfCollectionIndexes; ++i) {
collections.push(db._create(collectionNames[i], { numberOfShards: 3 }));
for (j = 0; j < numOfExpCollections; ++j) {
expCollections.push(db._create(expCollectionNames[i * numOfCollectionIndexes + j], { numberOfShards: 3 }));
}
}
var primCollection = db._create(primaryIndexCollectionName, { numberOfShards: 3 });
var edgeCollection = db._createEdgeCollection(edgeIndexCollectionName, { numberOfShards: 3 });
var severalIndexesCollection = db._create(severalIndexesCollectionName, { numberOfShards: 3 });
for (i = 0; i < numOfCollectionIndexes; ++i) {
let type;
if (i < numOfCollectionIndexes / 2) {
type = "hash";
} else {
type = "skiplist";
}
collections[i].ensureIndex({type: type, fields: ["obj.a", "obj.b", "obj.c"]});
for (j = 0; j < numOfExpCollections; ++j) {
let fields;
if (i === 0) {
fields = ["tags.hop[*].foo.fo", "tags.hop[*].bar.br", "tags.hop[*].baz.bz"];
} else {
fields = ["tags.hop[*]"]; // last expansion
}
expCollections[i * numOfCollectionIndexes + j].ensureIndex({type: type, fields: fields});
}
}
severalIndexesCollection.ensureIndex({type: "hash", fields: ["a"]});
severalIndexesCollection.ensureIndex({type: "hash", fields: ["b"]});
for (i = 0; i < numOfCollectionIndexes; ++i) {
collections[i].save({_key: 'c0', "obj": {"a": "a_val", "b": "b_val", "c": "c_val", "d": "d_val"}});
collections[i].save({_key: 'c1', "obj": {"a": "a_val_1", "b": "b_val_1", "c": "c_val_1", "d": "d_val_1"}});
collections[i].save({_key: 'c2', "obj": {"a": "a_val", "b": "b_val_2", "c": "c_val_2", "d": "d_val_2"}});
collections[i].save({_key: 'c3', "obj": {"a": "a_val_3", "b": "b_val_3", "c": "c_val_3", "d": "d_val_3"}});
for (j = 0; j < numOfExpCollections; ++j) {
let doc;
if (i === 0) {
doc = {_key: 'c1', "tags": {"hop": [{"foo": {"fo": "a_val_1"}}, {"bar": {"br": "bar_val"}}, {"baz": {"bz": "baz_val"}}]}};
} else {
doc = {"tags": {"hop": ["hop_array_val"]}}; // for last expansion
}
expCollections[i * numOfCollectionIndexes + j].save(doc);
}
}
primCollection.save({_key: "c0", foo: "a_val"});
primCollection.save({_key: "c1", foo: "b_val"});
primCollection.save({_key: "c2", foo: "c_val"});
edgeCollection.save({_key: "c0", _from: "testVertices/c0", _to: "testVertices/c1"});
edgeCollection.save({_key: "c1", _from: "testVertices/c0", _to: "testVertices/c0"});
severalIndexesCollection.save({_key: "c0", a: "a_val", b: "b_val"});
},
tearDownAll : function () {
for (i = 0; i < numOfCollectionIndexes; ++i) {
try { db._drop(collectionNames[i]); } catch(e) {}
for (j = 0; j < numOfExpCollections; ++j) {
try { db._drop(expCollectionNames[i * numOfCollectionIndexes + j]); } catch(e) {}
}
}
try { db._drop(primaryIndexCollectionName); } catch(e) {}
try { db._drop(edgeIndexCollectionName); } catch(e) {}
try { db._drop(severalIndexesCollectionName); } catch(e) {}
},
testNotAppliedDueToNoFilter() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " SORT d.obj.c LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToSort() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.b LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToNoSort() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToUsedInInnerSort() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c ASC SORT d.obj.b LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToNoLimit() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToLimitOnWrongNode() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' " +
"SORT d.obj.c LET c = CHAR_LENGTH(d.obj.d) * 2 SORT CONCAT(d.obj.c, c) LIMIT 10 RETURN { doc: d, sc: c}";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToNoReferences() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT RAND() LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
},
testNotAppliedDueToExpansion() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
for (j = 0; j < numOfExpCollections; ++j) {
let filterSort;
if (i === 0) {
filterSort = "'a_val' IN d.tags.hop[*].foo.fo SORT d.tags.hop[*].baz.bz";
} else {
filterSort = "'hop_array_val' IN d.tags.hop[*] SORT NOOPT(d.tags.hop[*])";
}
let query = "FOR d IN " + expCollectionNames[i * numOfCollectionIndexes + j] + " FILTER " + filterSort + " LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
}
}
},
testNotAppliedDueToSeveralIndexes() {
let query = "FOR d IN " + severalIndexesCollectionName + " FILTER d.a == 'a_val' OR d.b == 'b_val' SORT NOOPT(d.b) DESC LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertEqual(-1, plan.rules.indexOf(ruleName));
},
testQueryResultsWithCalculation() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' LET c = CONCAT(d.obj.b, RAND()) SORT c LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(2, result.json.length);
let expectedKeys = new Set(['c0', 'c2']);
result.json.forEach(function(doc) {
assertTrue(expectedKeys.has(doc._key));
expectedKeys.delete(doc._key);
});
assertEqual(0, expectedKeys.size);
}
},
testQueryResultsWithAfterSort() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 10 SORT NOOPT(d.obj.a) ASC RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(2, result.json.length);
let expectedKeys = new Set(['c0', 'c2']);
result.json.forEach(function(doc) {
assertTrue(expectedKeys.has(doc._key));
expectedKeys.delete(doc._key);
});
assertEqual(0, expectedKeys.size);
}
},
testQueryResultsWithMultipleSort() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' " +
"SORT d.obj.c LIMIT 2 SORT d.obj.b DESC LIMIT 1 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let materializeNodeFound = false;
let nodeDependency = null;
plan.nodes.forEach(function(node) {
if (node.type === "MaterializeNode") {
// there should be no materializer before (e.g. double materialization)
assertFalse(materializeNodeFound);
materializeNodeFound = true;
// the other sort node should be sorted but not have a materializer
// d.obj.c node on single and d.obj.b on cluster as for cluster
// only first sort will be on DBServers (identified by sort ASC)
isCluster ? assertTrue(nodeDependency.elements[0].ascending) : assertEqual(nodeDependency.limit, 1);
}
nodeDependency = node; // as we walk the plan this will be next node dependency
});
// materilizer should be there
assertTrue(materializeNodeFound);
let result = AQL_EXECUTE(query);
assertEqual(1, result.json.length);
let expectedKeys = new Set(['c2']);
result.json.forEach(function(doc) {
assertTrue(expectedKeys.has(doc._key));
expectedKeys.delete(doc._key);
});
assertEqual(0, expectedKeys.size);
}
},
testQueryResultsWithAfterCalc() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 10 LET c = CONCAT(NOOPT(d._key), '-C') RETURN c";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(2, result.json.length);
let expected = new Set(['c0-C', 'c2-C']);
result.json.forEach(function(doc) {
assertTrue(expected.has(doc));
expected.delete(doc);
});
assertEqual(0, expected.size);
}
},
testQueryResultsSkipSome() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c DESC LIMIT 1, 1 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(1, result.json.length);
assertEqual(result.json[0]._key, 'c0');
}
},
testQueryResultsSkipAll() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c LIMIT 5, 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(0, result.json.length);
}
},
testQueryResultsInSubquery() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR c IN " + collectionNames[i % numOfCollectionIndexes] + " FILTER c.obj.a == 'a_val_1' " +
"FOR d IN " + collectionNames[(i + 1) % numOfCollectionIndexes] + " FILTER c.obj.a == d.obj.a SORT d.obj.c LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(1, result.json.length);
let expected = new Set(['c1']);
result.json.forEach(function(doc) {
assertTrue(expected.has(doc._key));
expected.delete(doc._key);
});
assertEqual(0, expected.size);
}
},
testQueryResultsInOuterSubquery() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR c IN " + collectionNames[i % numOfCollectionIndexes] + " FILTER c.obj.a == 'a_val_1' SORT c.obj.c LIMIT 10 " +
"FOR d IN " + collectionNames[(i + 1) % numOfCollectionIndexes] + " FILTER c.obj.a == d.obj.a RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(1, result.json.length);
let expected = new Set(['c1']);
result.json.forEach(function(doc) {
assertTrue(expected.has(doc._key));
expected.delete(doc._key);
});
assertEqual(0, expected.size);
}
},
testQueryResultsMultipleLimits() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c " +
"LIMIT 1, 5 SORT d.obj.b LIMIT 1, 3 SORT NOOPT(d.obj.d) DESC " +
"LIMIT 1, 1 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let materializeNodeFound = false;
let nodeDependency = null;
// sort by d.obj.b node`s limit must be appended with materializer (identified by sort ASC)
// as last SORT needs materialized document
// and SORT by d.obj.d is not lowest possible variant
// However in cluster only first sort suitable, as later sorts depend
// on all db servers results and performed on coordinator
plan.nodes.forEach(function(node) {
if (node.type === "MaterializeNode") {
assertFalse(materializeNodeFound); // no double materialization
isCluster ? assertTrue(nodeDependency.elements[0].ascending) : assertEqual(nodeDependency.limit, 3);
materializeNodeFound = true;
}
nodeDependency = node;
});
assertTrue(materializeNodeFound);
}
},
testQueryResultsMultipleLimits2() {
for (i = 0; i < numOfCollectionIndexes; ++i) {
// almost the same as testQueryResultsMultipleLimits but without last sort - this
// will not create addition variable for sort
// value but it should not affect results especially on cluster!
let query = "FOR d IN " + collectionNames[i] + " FILTER d.obj.a == 'a_val' SORT d.obj.c " +
"LIMIT 1, 5 SORT d.obj.b DESC LIMIT 1, 3 " +
"RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let materializeNodeFound = false;
// sort by d.obj.b node`s limit must be appended with materializer (identified by SORT ASC)
// as SORT by d.obj.c is not lowest possible variant
// However in cluster only first sort suitable, as later sorts depend
// on all db servers results and performed on coordinator
let nodeDependency = null;
plan.nodes.forEach(function(node) {
if (node.type === "MaterializeNode") {
assertFalse(materializeNodeFound);
isCluster ? assertTrue(nodeDependency.elements[0].ascending) : assertEqual(nodeDependency.limit, 3);
materializeNodeFound = true;
}
nodeDependency = node;
});
assertTrue(materializeNodeFound);
}
},
testQueryResultsPrimaryIndex() {
let query = "FOR d IN " + primaryIndexCollectionName + " FILTER d._key IN ['c0', 'c1'] SORT NOOPT(d._key) DESC LIMIT 1 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(1, result.json.length);
let expectedKeys = new Set(['c1']);
result.json.forEach(function(doc) {
assertTrue(expectedKeys.has(doc._key));
expectedKeys.delete(doc._key);
});
assertEqual(0, expectedKeys.size);
},
testQueryResultsEdgeIndex() {
let query = "FOR d IN " + edgeIndexCollectionName + " FILTER d._from == 'testVertices/c0' SORT NOOPT(d._from) DESC LIMIT 10 RETURN d";
let plan = AQL_EXPLAIN(query).plan;
assertNotEqual(-1, plan.rules.indexOf(ruleName));
let result = AQL_EXECUTE(query);
assertEqual(2, result.json.length);
let expectedKeys = new Set(['c0', 'c1']);
result.json.forEach(function(doc) {
assertTrue(expectedKeys.has(doc._key));
expectedKeys.delete(doc._key);
});
assertEqual(0, expectedKeys.size);
}
};
}
jsunity.run(lateDocumentMaterializationRuleTestSuite);
return jsunity.done();