1
0
Fork 0

Optimize memory usage in AQL by using constrained heap sort when reasonable (#7990)

This commit is contained in:
Dan Larkin-York 2019-01-31 16:46:57 -05:00 committed by Jan
parent bbfae0ceef
commit efb8364d99
19 changed files with 1586 additions and 389 deletions

View File

@ -23,13 +23,13 @@
/// @author Jan Steemann
////////////////////////////////////////////////////////////////////////////////
#include "ExecutionBlock.h"
#include "Aql/AqlItemBlock.h"
#include "Aql/Ast.h"
#include "Aql/BlockCollector.h"
#include "Aql/ExecutionEngine.h"
#include "Aql/Query.h"
#include "Basics/Exceptions.h"
#include "ExecutionBlock.h"
using namespace arangodb;
using namespace arangodb::aql;

View File

@ -23,7 +23,6 @@
/// @author Copyright 2014, triagens GmbH, Cologne, Germany
////////////////////////////////////////////////////////////////////////////////
#include "ExecutionNode.h"
#include "Aql/AqlItemBlock.h"
#include "Aql/Ast.h"
#include "Aql/BasicBlocks.h"
@ -45,6 +44,7 @@
#include "Aql/SubqueryBlock.h"
#include "Aql/TraversalNode.h"
#include "Aql/WalkerWorker.h"
#include "ExecutionNode.h"
#include "Transaction/Methods.h"
#include "Utils/OperationCursor.h"

View File

@ -183,6 +183,9 @@ struct OptimizerRule {
// handling.
substituteSingleDocumentOperations,
// make sort node aware of subsequent limit statements for internal optimizations
applySortLimitRule,
/// Pass 9: push down calculations beyond FILTERs and LIMITs
moveCalculationsDownRule,

View File

@ -22,7 +22,6 @@
/// @author Jan Steemann
////////////////////////////////////////////////////////////////////////////////
#include "OptimizerRules.h"
#include "Aql/AqlItemBlock.h"
#include "Aql/ClusterNodes.h"
#include "Aql/CollectNode.h"
@ -55,6 +54,7 @@
#include "GeoIndex/Index.h"
#include "Graph/TraverserOptions.h"
#include "Indexes/Index.h"
#include "OptimizerRules.h"
#include "StorageEngine/EngineSelectorFeature.h"
#include "StorageEngine/StorageEngine.h"
#include "Transaction/Methods.h"
@ -574,15 +574,20 @@ std::string getSingleShardId(arangodb::aql::ExecutionPlan const* plan,
arangodb::aql::Variable const* inputVariable = nullptr;
if (node->getType() == EN::INDEX) {
inputVariable = ExecutionNode::castTo<arangodb::aql::IndexNode const*>(node)->outVariable();
inputVariable =
ExecutionNode::castTo<arangodb::aql::IndexNode const*>(node)->outVariable();
} else if (node->getType() == EN::FILTER) {
inputVariable = ExecutionNode::castTo<arangodb::aql::FilterNode const*>(node)->inVariable();
inputVariable =
ExecutionNode::castTo<arangodb::aql::FilterNode const*>(node)->inVariable();
} else if (node->getType() == EN::INSERT) {
inputVariable = ExecutionNode::castTo<arangodb::aql::InsertNode const*>(node)->inVariable();
inputVariable =
ExecutionNode::castTo<arangodb::aql::InsertNode const*>(node)->inVariable();
} else if (node->getType() == EN::REMOVE) {
inputVariable = ExecutionNode::castTo<arangodb::aql::RemoveNode const*>(node)->inVariable();
inputVariable =
ExecutionNode::castTo<arangodb::aql::RemoveNode const*>(node)->inVariable();
} else if (node->getType() == EN::REPLACE || node->getType() == EN::UPDATE) {
auto updateReplaceNode = ExecutionNode::castTo<arangodb::aql::UpdateReplaceNode const*>(node);
auto updateReplaceNode =
ExecutionNode::castTo<arangodb::aql::UpdateReplaceNode const*>(node);
if (updateReplaceNode->inKeyVariable() != nullptr) {
inputVariable = updateReplaceNode->inKeyVariable();
} else {
@ -716,6 +721,31 @@ std::string getSingleShardId(arangodb::aql::ExecutionPlan const* plan,
return shardId;
}
bool shouldApplyHeapOptimization(arangodb::aql::ExecutionNode* node,
arangodb::aql::LimitNode* limit) {
TRI_ASSERT(node != nullptr);
TRI_ASSERT(limit != nullptr);
size_t input = node->getCost().estimatedNrItems;
size_t output = limit->limit() + limit->offset();
// first check an easy case
if (input < 100) { // TODO fine-tune this cut-off
// no reason to complicate things for such a small input
return false;
}
// now check something a little more sophisticated, comparing best estimate of
// cost of heap sort to cost of regular sort (ignoring some variables)
double N = static_cast<double>(input);
double M = static_cast<double>(output);
double lgN = std::log2(N);
double lgM = std::log2(M);
// the 0.25 here comes from some experiments, may need to be tweaked;
// should kick in if output is roughly at most 3/4 of input
return (0.25 * N * lgM + M * lgM) < (N * lgN);
}
} // namespace
using namespace arangodb;
@ -2198,8 +2228,7 @@ void arangodb::aql::simplifyConditionsRule(Optimizer* opt,
return node;
}
accessed =
ExecutionNode::castTo<CalculationNode*>(setter)->expression()->node();
accessed = ExecutionNode::castTo<CalculationNode*>(setter)->expression()->node();
if (accessed == nullptr) {
return node;
}
@ -2252,8 +2281,7 @@ void arangodb::aql::simplifyConditionsRule(Optimizer* opt,
return node;
}
accessed =
ExecutionNode::castTo<CalculationNode*>(setter)->expression()->node();
accessed = ExecutionNode::castTo<CalculationNode*>(setter)->expression()->node();
if (accessed == nullptr) {
return node;
}
@ -2273,8 +2301,8 @@ void arangodb::aql::simplifyConditionsRule(Optimizer* opt,
if (indexValue->isStringValue()) {
// string index, e.g. ['123']
attributeName = StringRef(indexValue->getStringValue(),
indexValue->getStringLength());
attributeName =
StringRef(indexValue->getStringValue(), indexValue->getStringLength());
} else {
// numeric index, e.g. [123]
TRI_ASSERT(indexValue->isNumericValue());
@ -4038,13 +4066,15 @@ void arangodb::aql::distributeInClusterRule(Optimizer* opt,
// was only UPDATE <doc> IN <collection>
inputVariable = updateReplaceNode->inDocVariable();
}
distNode = new DistributeNode(plan.get(), plan->nextId(), collection, inputVariable,
inputVariable, false, updateReplaceNode->inKeyVariable() != nullptr);
distNode = new DistributeNode(plan.get(), plan->nextId(), collection,
inputVariable, inputVariable, false,
updateReplaceNode->inKeyVariable() != nullptr);
} else if (nodeType == ExecutionNode::UPSERT) {
// an UPSERT node has two input variables!
auto upsertNode = ExecutionNode::castTo<UpsertNode const*>(node);
auto d = new DistributeNode(plan.get(), plan->nextId(), collection,
upsertNode->inDocVariable(), upsertNode->insertVariable(), true, true);
upsertNode->inDocVariable(),
upsertNode->insertVariable(), true, true);
d->setAllowSpecifiedKeys(true);
distNode = ExecutionNode::castTo<ExecutionNode*>(d);
} else {
@ -5083,7 +5113,8 @@ class RemoveToEnumCollFinder final : public WalkerWorker<ExecutionNode> {
arangodb::HashSet<Variable const*> varsUsedHere;
cn->getVariablesUsedHere(varsUsedHere);
if (varsUsedHere.size() != 1 || varsUsedHere.find(_variable) == varsUsedHere.end()) {
if (varsUsedHere.size() != 1 ||
varsUsedHere.find(_variable) == varsUsedHere.end()) {
break; // abort . . .
}
_lastNode = en;
@ -6918,6 +6949,45 @@ void arangodb::aql::geoIndexRule(Optimizer* opt, std::unique_ptr<ExecutionPlan>
opt->addPlan(std::move(plan), rule, mod);
}
void arangodb::aql::sortLimitRule(Optimizer* opt, std::unique_ptr<ExecutionPlan> plan,
OptimizerRule const* rule) {
SmallVector<ExecutionNode*>::allocator_type::arena_type a;
SmallVector<ExecutionNode*> nodes{a};
bool mod = false;
plan->findNodesOfType(nodes, EN::SORT, true);
for (ExecutionNode* node : nodes) {
ExecutionNode* current = node->getFirstParent();
LimitNode* limit = nullptr;
while (current) {
if (current->getType() == EN::LIMIT) {
limit = ExecutionNode::castTo<LimitNode*>(current);
break; // stop parsing after first LIMIT
} else if (current->getType() == EN::FILTER || current->getType() == EN::RETURN ||
current->getType() == EN::ENUMERATE_COLLECTION ||
current->getType() == EN::ENUMERATE_LIST ||
current->getType() == EN::ENUMERATE_IRESEARCH_VIEW ||
current->getType() == EN::TRAVERSAL || current->getType() == EN::SHORTEST_PATH ||
current->getType() == EN::INDEX || current->getType() == EN::COLLECT) {
// TODO check other end conditions
break; // stop parsing
}
current = current->getFirstParent(); // inspect next node
}
// if we found a limit and we meet the heuristic, make the sort node
// aware of the limit
if (limit != nullptr && shouldApplyHeapOptimization(node, limit)) {
auto sn = static_cast<SortNode*>(node);
sn->setLimit(limit->limit() + limit->offset());
mod = true;
}
}
opt->addPlan(std::move(plan), rule, mod);
}
void arangodb::aql::optimizeSubqueriesRule(Optimizer* opt,
std::unique_ptr<ExecutionPlan> plan,
OptimizerRule const* rule) {

View File

@ -249,6 +249,9 @@ void inlineSubqueriesRule(Optimizer*, std::unique_ptr<ExecutionPlan>, OptimizerR
/// @brief replace FILTER and SORT containing DISTANCE function
void geoIndexRule(Optimizer*, std::unique_ptr<aql::ExecutionPlan>, OptimizerRule const*);
/// @brief make sort node aware of limit to enable internal optimizations
void sortLimitRule(Optimizer*, std::unique_ptr<aql::ExecutionPlan>, OptimizerRule const*);
/// @brief push LIMIT into subqueries, and simplify them
void optimizeSubqueriesRule(Optimizer*, std::unique_ptr<ExecutionPlan>, OptimizerRule const*);

View File

@ -20,12 +20,12 @@
/// @author Jan Steemann
////////////////////////////////////////////////////////////////////////////////
#include "OptimizerRulesFeature.h"
#include "Aql/ExecutionPlan.h"
#include "Aql/OptimizerRules.h"
#include "Basics/Exceptions.h"
#include "Basics/StringRef.h"
#include "Cluster/ServerState.h"
#include "OptimizerRulesFeature.h"
#include "StorageEngine/EngineSelectorFeature.h"
#include "StorageEngine/StorageEngine.h"
@ -284,6 +284,11 @@ void OptimizerRulesFeature::addRules() {
OptimizerRule::applyGeoIndexRule,
DoesNotCreateAdditionalPlans, CanBeDisabled);
// make sort node aware of subsequent limit statements for internal optimizations
OptimizerRulesFeature::registerRule("sort-limit", sortLimitRule,
OptimizerRule::applySortLimitRule,
DoesNotCreateAdditionalPlans, CanBeDisabled);
if (arangodb::ServerState::instance()->isCoordinator()) {
registerRule("optimize-cluster-single-document-operations",
substituteClusterSingleDocumentOperations,

View File

@ -561,8 +561,8 @@ void arangodb::aql::replaceNearWithinFulltext(Optimizer* opt,
for (auto const& node : nodes) {
auto visitor = [&modified, &node, &plan](AstNode* astnode) {
auto* fun = getFunction(astnode); // if fun != nullptr -> astnode->type NODE_TYPE_FCALL
AstNode* replacement = nullptr;
if (fun) {
AstNode* replacement = nullptr;
if (fun->name == "NEAR") {
replacement = replaceNearOrWithin(astnode, node, plan.get(), true /*isNear*/);
TRI_ASSERT(replacement);
@ -576,11 +576,13 @@ void arangodb::aql::replaceNearWithinFulltext(Optimizer* opt,
replacement = replaceFullText(astnode, node, plan.get());
TRI_ASSERT(replacement);
}
}
if (replacement) {
modified = true;
return replacement;
}
}
return astnode;
};
@ -596,5 +598,4 @@ void arangodb::aql::replaceNearWithinFulltext(Optimizer* opt,
}
opt->addPlan(std::move(plan), rule, modified);
}; // replaceJSFunctions
} // replaceJSFunctions

View File

@ -21,21 +21,167 @@
/// @author Max Neunhoeffer
////////////////////////////////////////////////////////////////////////////////
#include "SortBlock.h"
#include "Aql/AqlItemBlock.h"
#include "Aql/ExecutionEngine.h"
#include "Basics/Exceptions.h"
#include "Basics/ScopeGuard.h"
#include "SortBlock.h"
#include "VocBase/vocbase.h"
using namespace arangodb::aql;
namespace {
void stealRow(std::unordered_map<arangodb::aql::AqlValue, arangodb::aql::AqlValue>& cache,
arangodb::aql::RegisterId const nrRegs, arangodb::aql::AqlItemBlock* src,
size_t sRow, arangodb::aql::AqlItemBlock* dst, size_t dRow) {
for (arangodb::aql::RegisterId reg = 0; reg < nrRegs; reg++) {
auto const& original = src->getValueReference(sRow, reg);
// If we have already dealt with this value for the next
// block, then we just put the same value again:
if (!original.isEmpty()) {
if (original.requiresDestruction()) {
// complex value, with ownership transfer
auto it = cache.find(original);
/// @brief OurLessThan
if (it != cache.end()) {
// If one of the following throws, all is well, because
// the new block already has either a copy or stolen
// the AqlValue:
src->eraseValue(sRow, reg);
dst->setValue(dRow, reg, (*it).second);
} else {
// We need to copy original, if it has already been stolen from
// its source buffer, which we know by looking at the
// valueCount there.
auto vCount = src->valueCount(original);
if (vCount == 0) {
// Was already stolen for another block
arangodb::aql::AqlValue copy = original.clone();
try {
TRI_IF_FAILURE("SortBlock::doSortingCache") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
cache.emplace(original, copy);
} catch (...) {
copy.destroy();
throw;
}
try {
TRI_IF_FAILURE("SortBlock::doSortingNext1") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
dst->setValue(dRow, reg, copy);
} catch (...) {
cache.erase(copy);
copy.destroy();
throw;
}
// It does not matter whether the following works or not,
// since the source block keeps its responsibility
// for original:
src->eraseValue(sRow, reg);
} else {
TRI_IF_FAILURE("SortBlock::doSortingNext2") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
// Here we are the first to want to inherit original, so we
// steal it:
dst->setValue(dRow, reg, original);
src->steal(original);
src->eraseValue(sRow, reg);
// If this has worked, responsibility is now with the
// new block or requestBlock, indeed with us!
// If the following does not work, we will create a
// few unnecessary copies, but this does not matter:
cache.emplace(original, original);
}
}
} else {
// simple value, which does not need ownership transfer
TRI_IF_FAILURE("SortBlock::doSortingCache") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
TRI_IF_FAILURE("SortBlock::doSortingNext1") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
TRI_IF_FAILURE("SortBlock::doSortingNext2") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
dst->setValue(dRow, reg, original);
src->eraseValue(sRow, reg);
}
}
}
}
void stealRowNoCache(arangodb::aql::RegisterId const nrRegs,
arangodb::aql::AqlItemBlock* src, size_t sRow,
arangodb::aql::AqlItemBlock* dst, size_t dRow) {
for (arangodb::aql::RegisterId reg = 0; reg < nrRegs; reg++) {
auto const& original = src->getValueReference(sRow, reg);
// If we have already dealt with this value for the next
// block, then we just put the same value again:
if (!original.isEmpty()) {
if (original.requiresDestruction()) {
// We need to copy original, if it has already been stolen from
// its source buffer, which we know by looking at the
// valueCount there.
auto vCount = src->valueCount(original);
if (vCount == 0) {
// Was already stolen for another block
arangodb::aql::AqlValue copy = original.clone();
try {
TRI_IF_FAILURE("SortBlock::doSortingNext1") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
dst->setValue(dRow, reg, copy);
} catch (...) {
copy.destroy();
throw;
}
// It does not matter whether the following works or not,
// since the source block keeps its responsibility
// for original:
src->eraseValue(sRow, reg);
} else {
TRI_IF_FAILURE("SortBlock::doSortingNext2") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
// Here we are the first to want to inherit original, so we
// steal it:
dst->setValue(dRow, reg, original);
src->steal(original);
src->eraseValue(sRow, reg);
}
} else {
// simple value, which does not need ownership transfer
TRI_IF_FAILURE("SortBlock::doSortingCache") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
TRI_IF_FAILURE("SortBlock::doSortingNext1") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
TRI_IF_FAILURE("SortBlock::doSortingNext2") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
dst->setValue(dRow, reg, original);
src->eraseValue(sRow, reg);
}
}
}
}
class StandardSorter : public arangodb::aql::SortBlock::Sorter {
private:
class OurLessThan {
public:
OurLessThan(arangodb::transaction::Methods* trx, std::deque<AqlItemBlock*>& buffer,
std::vector<SortRegister>& sortRegisters) noexcept
OurLessThan(arangodb::transaction::Methods* trx,
std::deque<arangodb::aql::AqlItemBlock*>& buffer,
std::vector<arangodb::aql::SortRegister>& sortRegisters) noexcept
: _trx(trx), _buffer(buffer), _sortRegisters(sortRegisters) {}
bool operator()(std::pair<uint32_t, uint32_t> const& a,
@ -44,7 +190,7 @@ class OurLessThan {
auto const& lhs = _buffer[a.first]->getValueReference(a.second, reg.reg);
auto const& rhs = _buffer[b.first]->getValueReference(b.second, reg.reg);
int const cmp = AqlValue::Compare(_trx, lhs, rhs, true);
int const cmp = arangodb::aql::AqlValue::Compare(_trx, lhs, rhs, true);
if (cmp < 0) {
return reg.asc;
@ -58,68 +204,48 @@ class OurLessThan {
private:
arangodb::transaction::Methods* _trx;
std::deque<AqlItemBlock*>& _buffer;
std::vector<SortRegister>& _sortRegisters;
std::deque<arangodb::aql::AqlItemBlock*>& _buffer;
std::vector<arangodb::aql::SortRegister>& _sortRegisters;
}; // OurLessThan
} // namespace
public:
StandardSorter(arangodb::aql::SortBlock& block, arangodb::transaction::Methods* trx,
std::deque<arangodb::aql::AqlItemBlock*>& buffer,
std::vector<arangodb::aql::SortRegister>& sortRegisters,
Fetcher&& fetch, Allocator&& allocate)
: arangodb::aql::SortBlock::Sorter(block, trx, buffer, sortRegisters,
std::move(fetch), std::move(allocate)) {}
SortBlock::SortBlock(ExecutionEngine* engine, SortNode const* en)
: ExecutionBlock(engine, en), _stable(en->_stable), _mustFetchAll(true) {
TRI_ASSERT(en && en->plan() && en->getRegisterPlan());
SortRegister::fill(*en->plan(), *en->getRegisterPlan(), en->elements(), _sortRegisters);
}
std::pair<arangodb::aql::ExecutionState, arangodb::Result> fetch() override {
using arangodb::aql::ExecutionBlock;
using arangodb::aql::ExecutionState;
SortBlock::~SortBlock() {}
std::pair<ExecutionState, arangodb::Result> SortBlock::initializeCursor(AqlItemBlock* items,
size_t pos) {
auto res = ExecutionBlock::initializeCursor(items, pos);
if (res.first == ExecutionState::WAITING || !res.second.ok()) {
// If we need to wait or get an error we return as is.
return res;
}
_mustFetchAll = !_done;
_pos = 0;
return res;
}
std::pair<ExecutionState, arangodb::Result> SortBlock::getOrSkipSome(
size_t atMost, bool skipping, AqlItemBlock*& result, size_t& skipped) {
TRI_ASSERT(result == nullptr && skipped == 0);
if (_mustFetchAll) {
ExecutionState res = ExecutionState::HASMORE;
// suck all blocks into _buffer
while (res != ExecutionState::DONE) {
res = getBlock(DefaultBatchSize()).first;
res = _fetch(ExecutionBlock::DefaultBatchSize()).first;
if (res == ExecutionState::WAITING) {
return {res, TRI_ERROR_NO_ERROR};
}
}
_mustFetchAll = false;
if (!_buffer.empty()) {
doSorting();
}
return {ExecutionState::DONE, TRI_ERROR_NO_ERROR};
}
return ExecutionBlock::getOrSkipSome(atMost, skipping, result, skipped);
}
void SortBlock::doSorting() {
size_t sum = 0;
for (auto const& block : _buffer) {
sum += block->size();
}
arangodb::Result sort() override {
using arangodb::aql::AqlItemBlock;
using arangodb::aql::AqlValue;
using arangodb::aql::ExecutionBlock;
using arangodb::aql::RegisterId;
TRI_IF_FAILURE("SortBlock::doSorting") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
size_t sum = 0;
for (auto const& block : _buffer) {
sum += block->size();
}
// coords[i][j] is the <j>th row of the <i>th block
std::vector<std::pair<uint32_t, uint32_t>> coords;
coords.reserve(sum);
@ -142,17 +268,18 @@ void SortBlock::doSorting() {
OurLessThan ourLessThan(_trx, _buffer, _sortRegisters);
// sort coords
if (_stable) {
if (_block.stable()) {
std::stable_sort(coords.begin(), coords.end(), ourLessThan);
} else {
std::sort(coords.begin(), coords.end(), ourLessThan);
}
// here we collect the new blocks (later swapped into _buffer):
std::deque<AqlItemBlock*> newbuffer;
std::deque<AqlItemBlock*> newBuffer;
try { // If we throw from here, the catch will delete the new
// blocks in newbuffer
try {
// If we throw from here, the cleanup will delete the new
// blocks in newBuffer
count = 0;
RegisterId const nrRegs = _buffer.front()->getNrRegs();
@ -162,14 +289,14 @@ void SortBlock::doSorting() {
// install the rearranged values from _buffer into newbuffer
while (count < sum) {
size_t sizeNext = (std::min)(sum - count, DefaultBatchSize());
AqlItemBlock* next = requestBlock(sizeNext, nrRegs);
size_t sizeNext = (std::min)(sum - count, ExecutionBlock::DefaultBatchSize());
AqlItemBlock* next = _allocate(sizeNext, nrRegs);
try {
TRI_IF_FAILURE("SortBlock::doSortingInner") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
newbuffer.emplace_back(next);
newBuffer.emplace_back(next);
} catch (...) {
delete next;
throw;
@ -177,100 +304,328 @@ void SortBlock::doSorting() {
// only copy as much as needed!
for (size_t i = 0; i < sizeNext; i++) {
for (RegisterId j = 0; j < nrRegs; j++) {
auto const& a =
_buffer[coords[count].first]->getValueReference(coords[count].second, j);
// If we have already dealt with this value for the next
// block, then we just put the same value again:
if (!a.isEmpty()) {
if (a.requiresDestruction()) {
// complex value, with ownership transfer
auto it = cache.find(a);
if (it != cache.end()) {
// If one of the following throws, all is well, because
// the new block already has either a copy or stolen
// the AqlValue:
_buffer[coords[count].first]->eraseValue(coords[count].second, j);
next->setValue(i, j, (*it).second);
} else {
// We need to copy a, if it has already been stolen from
// its original buffer, which we know by looking at the
// valueCount there.
auto vCount = _buffer[coords[count].first]->valueCount(a);
if (vCount == 0) {
// Was already stolen for another block
AqlValue b = a.clone();
try {
TRI_IF_FAILURE("SortBlock::doSortingCache") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
cache.emplace(a, b);
} catch (...) {
b.destroy();
throw;
}
try {
TRI_IF_FAILURE("SortBlock::doSortingNext1") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
next->setValue(i, j, b);
} catch (...) {
cache.erase(b);
b.destroy();
throw;
}
// It does not matter whether the following works or not,
// since the original block keeps its responsibility
// for a:
_buffer[coords[count].first]->eraseValue(coords[count].second, j);
} else {
TRI_IF_FAILURE("SortBlock::doSortingNext2") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
// Here we are the first to want to inherit a, so we
// steal it:
next->setValue(i, j, a);
_buffer[coords[count].first]->steal(a);
_buffer[coords[count].first]->eraseValue(coords[count].second, j);
// If this has worked, responsibility is now with the
// new block or indeed with us!
// If the following does not work, we will create a
// few unnecessary copies, but this does not matter:
cache.emplace(a, a);
}
}
} else {
// simple value, which does not need ownership transfer
TRI_IF_FAILURE("SortBlock::doSortingCache") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
TRI_IF_FAILURE("SortBlock::doSortingNext1") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
TRI_IF_FAILURE("SortBlock::doSortingNext2") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
next->setValue(i, j, a);
_buffer[coords[count].first]->eraseValue(coords[count].second, j);
}
}
}
::stealRow(cache, nrRegs, _buffer[coords[count].first],
coords[count].second, next, i);
count++;
}
cache.clear();
}
} catch (...) {
for (auto& x : newbuffer) {
for (auto& x : newBuffer) {
delete x;
}
throw;
}
_buffer.swap(newbuffer); // does not throw since allocators
// are the same
for (auto& x : newbuffer) {
_buffer.swap(newBuffer); // does not throw since allocators are the same
for (auto& x : newBuffer) {
delete x;
}
return TRI_ERROR_NO_ERROR;
}
bool empty() const override { return _buffer.empty(); }
};
class ConstrainedHeapSorter : public arangodb::aql::SortBlock::Sorter {
private:
class OurLessThan {
public:
OurLessThan(arangodb::transaction::Methods* trx,
std::vector<arangodb::aql::SortRegister>& sortRegisters) noexcept
: _trx(trx), _lhsBuffer(nullptr), _rhsBuffer(nullptr), _sortRegisters(sortRegisters) {}
void setBuffers(arangodb::aql::AqlItemBlock* lhsBuffer,
arangodb::aql::AqlItemBlock* rhsBuffer) {
_lhsBuffer = lhsBuffer;
_rhsBuffer = rhsBuffer;
}
bool operator()(uint32_t const& a, uint32_t const& b) const {
TRI_ASSERT(_lhsBuffer);
TRI_ASSERT(_rhsBuffer);
for (auto const& reg : _sortRegisters) {
auto const& lhs = _lhsBuffer->getValueReference(a, reg.reg);
auto const& rhs = _rhsBuffer->getValueReference(b, reg.reg);
int const cmp = arangodb::aql::AqlValue::Compare(_trx, lhs, rhs, true);
if (cmp < 0) {
return reg.asc;
} else if (cmp > 0) {
return !reg.asc;
}
}
return false;
}
private:
arangodb::transaction::Methods* _trx;
arangodb::aql::AqlItemBlock* _lhsBuffer;
arangodb::aql::AqlItemBlock* _rhsBuffer;
std::vector<arangodb::aql::SortRegister>& _sortRegisters;
}; // OurLessThan
public:
ConstrainedHeapSorter(arangodb::aql::SortBlock& block, arangodb::transaction::Methods* trx,
std::deque<arangodb::aql::AqlItemBlock*>& buffer,
std::vector<arangodb::aql::SortRegister>& sortRegisters,
Fetcher&& fetch, Allocator&& allocate, size_t limit)
: arangodb::aql::SortBlock::Sorter(block, trx, buffer, sortRegisters,
std::move(fetch), std::move(allocate)),
_limit{limit},
_cmpHeap(_trx, _sortRegisters),
_cmpInput(_trx, _sortRegisters) {
TRI_ASSERT(_limit > 0);
_rows.reserve(_limit);
}
~ConstrainedHeapSorter() { releaseHeapBuffer(); }
std::pair<arangodb::aql::ExecutionState, arangodb::Result> fetch() override {
using arangodb::aql::AqlItemBlock;
using arangodb::aql::ExecutionBlock;
using arangodb::aql::ExecutionState;
ExecutionState res = ExecutionState::HASMORE;
// suck all blocks through _buffer into heap
while (res != ExecutionState::DONE) {
res = _fetch(ExecutionBlock::DefaultBatchSize()).first;
if (res == ExecutionState::WAITING) {
return {res, TRI_ERROR_NO_ERROR};
}
if (!_buffer.empty()) {
ensureHeapBuffer(_buffer.front()); // make sure we have a dst
}
// handle batch
while (!_buffer.empty()) {
std::unique_ptr<AqlItemBlock> src{_buffer.front()};
_cmpInput.setBuffers(_heapBuffer.get(), src.get());
_buffer.pop_front();
for (size_t row = 0; row < src->size(); row++) {
pushRow(src.get(), row);
}
}
}
return {ExecutionState::DONE, TRI_ERROR_NO_ERROR};
}
arangodb::Result sort() override {
using arangodb::aql::AqlItemBlock;
using arangodb::aql::AqlValue;
using arangodb::aql::ExecutionBlock;
using arangodb::aql::RegisterId;
TRI_IF_FAILURE("SortBlock::doSorting") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
// make sure we don't have less than limit
size_t total = std::min(_limit, _rowsPushed);
uint32_t count = 0;
// sort rows
std::sort(_rows.begin(), _rows.end(), _cmpHeap);
// here we collect the new blocks:
TRI_ASSERT(_buffer.empty());
count = 0;
RegisterId const nrRegs = _heapBuffer->getNrRegs();
std::unordered_map<AqlValue, AqlValue> cache;
// install the rearranged values from _buffer into newbuffer
while (count < total) {
size_t sizeNext = (std::min)(total - count, ExecutionBlock::DefaultBatchSize());
std::unique_ptr<AqlItemBlock> next(_allocate(sizeNext, nrRegs));
TRI_IF_FAILURE("SortBlock::doSortingInner") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
// hand over ownership for the item block
AqlItemBlock* p = next.get();
_buffer.emplace_back(p);
next.release();
// only copy as much as needed!
for (size_t i = 0; i < sizeNext; i++) {
::stealRow(cache, nrRegs, _heapBuffer.get(), _rows[count], p, i);
count++;
}
cache.clear();
}
// pre-emptively cleanup memory from heap buffer (also done in destructor)
releaseHeapBuffer();
return TRI_ERROR_NO_ERROR;
}
bool empty() const override {
return _buffer.empty() && _heapBuffer == nullptr;
}
private:
arangodb::Result pushRow(arangodb::aql::AqlItemBlock* srcBlock, size_t sRow) {
using arangodb::aql::AqlItemBlock;
using arangodb::aql::AqlValue;
using arangodb::aql::RegisterId;
if (_rowsPushed >= _limit && _cmpInput(_rows.front(), sRow)) {
// skip row, already too low in sort order to make it past limit
return TRI_ERROR_NO_ERROR;
}
TRI_ASSERT(srcBlock != nullptr);
AqlItemBlock* dstBlock = _heapBuffer.get();
TRI_ASSERT(dstBlock != nullptr);
size_t dRow = _rowsPushed;
if (_rowsPushed >= _limit) {
// pop an entry first
std::pop_heap(_rows.begin(), _rows.end(), _cmpHeap);
dRow = _rows.back();
eraseRow(dRow);
_rows.pop_back();
}
TRI_ASSERT(dRow < _limit);
TRI_IF_FAILURE("SortBlock::doSortingInner") {
THROW_ARANGO_EXCEPTION(TRI_ERROR_DEBUG);
}
RegisterId nrRegs = srcBlock->getNrRegs();
::stealRowNoCache(nrRegs, srcBlock, sRow, dstBlock, dRow);
_rows.emplace_back(dRow);
++_rowsPushed;
// now insert copy into heap
std::push_heap(_rows.begin(), _rows.end(), _cmpHeap);
return TRI_ERROR_NO_ERROR;
}
void eraseRow(size_t row) {
arangodb::aql::RegisterId const nrRegs = _heapBuffer->getNrRegs();
for (size_t i = 0; i < nrRegs; i++) {
_heapBuffer->destroyValue(row, i);
}
}
void ensureHeapBuffer(arangodb::aql::AqlItemBlock* src) {
TRI_ASSERT(src != nullptr);
if (_heapBuffer == nullptr) {
arangodb::aql::RegisterId const nrRegs = src->getNrRegs();
_heapBuffer.reset(_allocate(_limit, nrRegs));
_cmpHeap.setBuffers(_heapBuffer.get(), _heapBuffer.get());
}
}
void releaseHeapBuffer() { _heapBuffer.reset(); }
private:
size_t _limit;
size_t _rowsPushed = 0;
std::unique_ptr<arangodb::aql::AqlItemBlock> _heapBuffer;
std::vector<uint32_t> _rows;
OurLessThan _cmpHeap;
OurLessThan _cmpInput;
};
} // namespace
namespace arangodb {
namespace aql {
SortBlock::Sorter::Sorter(arangodb::aql::SortBlock& block, transaction::Methods* trx,
std::deque<AqlItemBlock*>& buffer,
std::vector<SortRegister>& sortRegisters,
Fetcher&& fetch, Allocator&& allocate)
: _block{block}, _trx{trx}, _buffer{buffer}, _sortRegisters{sortRegisters}, _fetch{std::move(fetch)}, _allocate{std::move(allocate)} {}
SortBlock::Sorter::~Sorter() {}
SortBlock::SortBlock(ExecutionEngine* engine, SortNode const* en,
SortNode::SorterType type, size_t limit)
: ExecutionBlock(engine, en), _stable(en->_stable), _type{type}, _limit{limit} {
TRI_ASSERT(en && en->plan() && en->getRegisterPlan());
SortRegister::fill(*en->plan(), *en->getRegisterPlan(), en->elements(), _sortRegisters);
initializeSorter();
}
SortBlock::~SortBlock() {}
std::pair<ExecutionState, arangodb::Result> SortBlock::initializeCursor(AqlItemBlock* items,
size_t pos) {
auto res = ExecutionBlock::initializeCursor(items, pos);
if (res.first == ExecutionState::WAITING || res.second.fail()) {
// If we need to wait or get an error we return as is.
return res;
}
_mustFetchAll = !_done;
_pos = 0;
return res;
}
std::pair<ExecutionState, arangodb::Result> SortBlock::getOrSkipSome(
size_t atMost, bool skipping, AqlItemBlock*& result, size_t& skipped) {
TRI_ASSERT(_sorter != nullptr && result == nullptr && skipped == 0);
if (_mustFetchAll) {
// sorter handles all the dirty work
auto res = _sorter->fetch();
if (res.first == ExecutionState::WAITING || res.second.fail()) {
// If we need to wait or get an error we return as is.
return res;
}
_mustFetchAll = false;
if (!_sorter->empty()) {
auto result = _sorter->sort();
if (result.fail()) {
return {ExecutionState::DONE, result};
}
}
}
return ExecutionBlock::getOrSkipSome(atMost, skipping, result, skipped);
}
bool SortBlock::stable() const { return _stable; }
void SortBlock::initializeSorter() {
if (_sorter == nullptr) {
auto fetch = [this](size_t atMost) -> std::pair<ExecutionState, bool> {
return getBlock(atMost);
};
auto allocate = [this](size_t nrItems, RegisterId nrRegs) -> AqlItemBlock* {
return requestBlock(nrItems, nrRegs);
};
switch (_type) {
case SortNode::SorterType::Standard: {
_sorter = std::make_unique<::StandardSorter>(*this, _trx, _buffer,
_sortRegisters, std::move(fetch),
std::move(allocate));
break;
}
case SortNode::SorterType::ConstrainedHeap: {
TRI_ASSERT(!_stable && _limit > 0);
_sorter = std::make_unique<::ConstrainedHeapSorter>(*this, _trx, _buffer, _sortRegisters,
std::move(fetch),
std::move(allocate), _limit);
break;
}
}
}
}
} // namespace aql
} // namespace arangodb

View File

@ -33,12 +33,34 @@ namespace arangodb {
namespace aql {
class AqlItemBlock;
class ExecutionEngine;
class SortBlock final : public ExecutionBlock {
public:
SortBlock(ExecutionEngine*, SortNode const*);
class Sorter {
public:
using Fetcher = std::function<std::pair<ExecutionState, bool>(size_t)>;
using Allocator = std::function<AqlItemBlock*(size_t, RegisterId)>;
public:
Sorter(arangodb::aql::SortBlock&, transaction::Methods*, std::deque<AqlItemBlock*>&,
std::vector<SortRegister>&, Fetcher&&, Allocator&&);
virtual ~Sorter();
virtual std::pair<ExecutionState, arangodb::Result> fetch() = 0;
virtual arangodb::Result sort() = 0;
virtual bool empty() const = 0;
protected:
SortBlock& _block;
transaction::Methods* _trx;
std::deque<AqlItemBlock*>& _buffer;
std::vector<SortRegister>& _sortRegisters;
Fetcher _fetch;
Allocator _allocate;
};
public:
SortBlock(ExecutionEngine*, SortNode const*, SortNode::SorterType type, size_t limit);
~SortBlock();
@ -49,9 +71,10 @@ class SortBlock final : public ExecutionBlock {
AqlItemBlock*&,
size_t& skipped) override final;
/// @brief dosorting
bool stable() const;
private:
void doSorting();
void initializeSorter();
/// @brief pairs, consisting of variable and sort direction
/// (true = ascending | false = descending)
@ -60,7 +83,17 @@ class SortBlock final : public ExecutionBlock {
/// @brief whether or not the sort should be stable
bool _stable;
/// @brief whether or not results must still be fetched from dependencies
bool _mustFetchAll;
/// @brief the type of sorter to use
SortNode::SorterType _type;
/// @brief the maximum number of items to return; unlimited if zero
size_t _limit;
/// @brief the object which actually handles the sorting
std::unique_ptr<Sorter> _sorter = nullptr;
};
} // namespace aql

View File

@ -21,22 +21,40 @@
/// @author Max Neunhoeffer
////////////////////////////////////////////////////////////////////////////////
#include "SortNode.h"
#include "Aql/Ast.h"
#include "Aql/ExecutionPlan.h"
#include "Aql/SortBlock.h"
#include "Aql/WalkerWorker.h"
#include "Basics/StringBuffer.h"
#include "Basics/VelocyPackHelper.h"
#include "SortNode.h"
namespace {
std::string const ConstrainedHeap = "constrained-heap";
std::string const Standard = "standard";
} // namespace
using namespace arangodb::basics;
using namespace arangodb::aql;
std::string const& SortNode::sorterTypeName(SorterType type) {
switch (type) {
case SorterType::Standard:
return ::Standard;
case SorterType::ConstrainedHeap:
return ::ConstrainedHeap;
default:
return ::Standard;
}
}
SortNode::SortNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base,
SortElementVector const& elements, bool stable)
: ExecutionNode(plan, base),
_reinsertInCluster(true),
_elements(elements),
_stable(stable) {}
_stable(stable),
_limit(VelocyPackHelper::getNumericValue<size_t>(base, "limit", 0)) {}
/// @brief toVelocyPack, for SortNode
void SortNode::toVelocyPackHelper(VPackBuilder& nodes, unsigned flags) const {
@ -61,6 +79,8 @@ void SortNode::toVelocyPackHelper(VPackBuilder& nodes, unsigned flags) const {
}
}
nodes.add("stable", VPackValue(_stable));
nodes.add("limit", VPackValue(_limit));
nodes.add("strategy", VPackValue(sorterTypeName(sorterType())));
// And close it:
nodes.close();
@ -195,7 +215,7 @@ SortInformation SortNode::getSortInformation(ExecutionPlan* plan,
/// @brief creates corresponding ExecutionBlock
std::unique_ptr<ExecutionBlock> SortNode::createBlock(
ExecutionEngine& engine, std::unordered_map<ExecutionNode*, ExecutionBlock*> const&) const {
return std::make_unique<SortBlock>(&engine, this);
return std::make_unique<SortBlock>(&engine, this, sorterType(), _limit);
}
/// @brief estimateCost
@ -209,3 +229,7 @@ CostEstimate SortNode::estimateCost() const {
}
return estimate;
}
SortNode::SorterType SortNode::sorterType() const {
return (!isStable() && _limit > 0) ? SorterType::ConstrainedHeap : SorterType::Standard;
}

View File

@ -50,6 +50,10 @@ class SortNode : public ExecutionNode {
friend class SortBlock;
friend class RedundantCalculationsReplacer;
public:
enum SorterType { Standard, ConstrainedHeap };
static std::string const& sorterTypeName(SorterType);
public:
SortNode(ExecutionPlan* plan, size_t id, SortElementVector const& elements, bool stable)
: ExecutionNode(plan, id),
@ -60,6 +64,9 @@ class SortNode : public ExecutionNode {
SortNode(ExecutionPlan* plan, arangodb::velocypack::Slice const& base,
SortElementVector const& elements, bool stable);
/// @brief if non-zero, limits the number of elements that the node will return
void setLimit(size_t limit) { _limit = limit; }
/// @brief return the type of the node
NodeType getType() const override final { return SORT; }
@ -110,6 +117,8 @@ class SortNode : public ExecutionNode {
/// values (e.g. when a FILTER condition exists that guarantees this)
void removeConditions(size_t count);
SorterType sorterType() const;
// reinsert node when building gather node - this is used e.g for the
// geo-index
bool _reinsertInCluster;
@ -121,6 +130,9 @@ class SortNode : public ExecutionNode {
/// whether or not the sort is stable
bool _stable;
/// the maximum number of items to return if non-zero; if zero, unlimited
size_t _limit = 0;
};
} // namespace aql

View File

@ -21,11 +21,11 @@
/// @author Vasiliy Nabatchikov
////////////////////////////////////////////////////////////////////////////////
#include "IResearchDocument.h"
#include "Basics/Endian.h"
#include "Basics/StaticStrings.h"
#include "Basics/VelocyPackHelper.h"
#include "IResearchCommon.h"
#include "IResearchDocument.h"
#include "IResearchKludge.h"
#include "IResearchPrimaryKeyFilter.h"
#include "IResearchViewMeta.h"
@ -47,7 +47,6 @@ namespace {
// --SECTION-- Primary key endianness
// ----------------------------------------------------------------------------
constexpr bool const LittleEndian = true;
constexpr bool const BigEndian = false;
template <bool IsLittleEndian>

View File

@ -63,10 +63,8 @@ class FlushFeature final : public application_features::ApplicationFeature {
/// @param callback the callback to invoke
/// @return success, false == handler for the specified type already registered
/// @note not thread-safe on the assumption of static factory registration
static bool registerFlushRecoveryCallback(
std::string const& type,
FlushRecoveryCallback const& callback
);
static bool registerFlushRecoveryCallback(std::string const& type,
FlushRecoveryCallback const& callback);
/// @brief register a flush subscription that will ensure replay of all WAL
/// entries after the latter of registration or the last successful
@ -76,10 +74,8 @@ class FlushFeature final : public application_features::ApplicationFeature {
/// @return a token used for marking flush synchronization
/// release of the token will unregister the subscription
/// nullptr == error
std::shared_ptr<FlushSubscription> registerFlushSubscription(
std::string const& type,
TRI_vocbase_t const& vocbase
);
std::shared_ptr<FlushSubscription> registerFlushSubscription(std::string const& type,
TRI_vocbase_t const& vocbase);
/// @brief release all ticks not used by the flush subscriptions
arangodb::Result releaseUnusedTicks();

View File

@ -191,8 +191,6 @@ char const* arangodb::rocksDBLogTypeName(arangodb::RocksDBLogType type) {
return "TrackedDocumentRemove";
case arangodb::RocksDBLogType::Invalid:
return "Invalid";
default:
return "Invalid";
}
return "Invalid";
}

View File

@ -1265,7 +1265,7 @@ function processQuery (query, explain, planIndex) {
case 'SortNode':
return keyword('SORT') + ' ' + node.elements.map(function (node) {
return variableName(node.inVariable) + ' ' + keyword(node.ascending ? 'ASC' : 'DESC');
}).join(', ');
}).join(', ') + annotation(` /* sorting strategy: ${node.strategy.split("-").join(" ")} */`);
case 'LimitNode':
return keyword('LIMIT') + ' ' + value(JSON.stringify(node.offset)) + ', ' + value(JSON.stringify(node.limit)) + (node.fullCount ? ' ' + annotation('/* fullCount */') : '');
case 'ReturnNode':

View File

@ -0,0 +1,380 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2017-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 Dan Larkin-York
////////////////////////////////////////////////////////////////////////////////
#include "catch.hpp"
#include <velocypack/Iterator.h>
#include <velocypack/velocypack-aliases.h>
// test setup
#include "IResearch/StorageEngineMock.h"
#include "IResearch/common.h"
#include "Aql/AqlFunctionFeature.h"
#include "Aql/ExecutionPlan.h"
#include "Aql/OptimizerRulesFeature.h"
#include "Aql/Query.h"
#include "Basics/VelocyPackHelper.h"
#include "ClusterEngine/ClusterEngine.h"
#include "Logger/LogTopic.h"
#include "Logger/Logger.h"
#include "Random/RandomGenerator.h"
#include "RestServer/AqlFeature.h"
#include "RestServer/DatabaseFeature.h"
#include "RestServer/DatabasePathFeature.h"
#include "RestServer/QueryRegistryFeature.h"
#include "RestServer/SystemDatabaseFeature.h"
#include "RestServer/TraverserEngineRegistryFeature.h"
#include "StorageEngine/EngineSelectorFeature.h"
#include "Transaction/Methods.h"
#include "Transaction/StandaloneContext.h"
#include "Utils/SingleCollectionTransaction.h"
#include "VocBase/LogicalCollection.h"
extern const char* ARGV0; // defined in main.cpp
namespace {
// -----------------------------------------------------------------------------
// --SECTION-- setup / tear-down
// -----------------------------------------------------------------------------
struct SortLimitSetup {
StorageEngineMock engine;
arangodb::application_features::ApplicationServer server;
std::unique_ptr<TRI_vocbase_t> system;
std::vector<std::pair<arangodb::application_features::ApplicationFeature*, bool>> features;
SortLimitSetup() : engine(server), server(nullptr, nullptr) {
arangodb::EngineSelectorFeature::ENGINE = &engine;
arangodb::transaction::Methods::clearDataSourceRegistrationCallbacks();
arangodb::ClusterEngine::Mocking = true;
arangodb::RandomGenerator::initialize(arangodb::RandomGenerator::RandomType::MERSENNE);
// suppress log messages since tests check error conditions
arangodb::LogTopic::setLogLevel(arangodb::Logger::FIXME.name(), arangodb::LogLevel::ERR); // suppress WARNING DefaultCustomTypeHandler called
// setup required application features
features.emplace_back(new arangodb::DatabasePathFeature(server), false);
features.emplace_back(new arangodb::DatabaseFeature(server), false);
features.emplace_back(new arangodb::QueryRegistryFeature(server), false); // must be first
arangodb::application_features::ApplicationServer::server->addFeature(
features.back().first); // need QueryRegistryFeature feature to be added now in order to create the system database
system = std::make_unique<TRI_vocbase_t>(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL,
0, TRI_VOC_SYSTEM_DATABASE);
features.emplace_back(new arangodb::SystemDatabaseFeature(server, system.get()),
false); // required for IResearchAnalyzerFeature
features.emplace_back(new arangodb::TraverserEngineRegistryFeature(server), false); // must be before AqlFeature
features.emplace_back(new arangodb::AqlFeature(server), true);
features.emplace_back(new arangodb::aql::OptimizerRulesFeature(server), true);
features.emplace_back(new arangodb::aql::AqlFunctionFeature(server), true); // required for IResearchAnalyzerFeature
for (auto& f : features) {
arangodb::application_features::ApplicationServer::server->addFeature(f.first);
}
for (auto& f : features) {
f.first->prepare();
}
for (auto& f : features) {
if (f.second) {
f.first->start();
}
}
auto* dbPathFeature =
arangodb::application_features::ApplicationServer::getFeature<arangodb::DatabasePathFeature>(
"DatabasePath");
arangodb::tests::setDatabasePath(*dbPathFeature); // ensure test data is stored in a unique directory
}
~SortLimitSetup() {
system.reset(); // destroy before reseting the 'ENGINE'
arangodb::AqlFeature(server).stop(); // unset singleton instance
arangodb::LogTopic::setLogLevel(arangodb::Logger::FIXME.name(),
arangodb::LogLevel::DEFAULT);
arangodb::application_features::ApplicationServer::server = nullptr;
arangodb::EngineSelectorFeature::ENGINE = nullptr;
// destroy application features
for (auto& f : features) {
if (f.second) {
f.first->stop();
}
}
for (auto& f : features) {
f.first->unprepare();
}
}
}; // SortLimitSetup
} // namespace
// -----------------------------------------------------------------------------
// --SECTION-- test suite
// -----------------------------------------------------------------------------
////////////////////////////////////////////////////////////////////////////////
/// @brief setup
////////////////////////////////////////////////////////////////////////////////
TEST_CASE("SortLimit", "[aql][sort-limit]") {
SortLimitSetup s;
UNUSED(s);
TRI_vocbase_t vocbase(TRI_vocbase_type_e::TRI_VOCBASE_TYPE_NORMAL, 1,
"testVocbase");
std::vector<arangodb::velocypack::Builder> insertedDocs;
auto sorterType = [](TRI_vocbase_t& vocbase, std::string const& queryString,
std::string rules = "") -> std::string {
auto options = arangodb::velocypack::Parser::fromJson(
"{\"optimizer\": {\"rules\": [" + rules + "]}}");
arangodb::aql::Query query(false, vocbase, arangodb::aql::QueryString(queryString),
nullptr, options, arangodb::aql::PART_MAIN);
auto result = query.explain();
VPackSlice nodes = result.result->slice().get("nodes");
CHECK(nodes.isArray());
std::string strategy;
for (auto const& it : VPackArrayIterator(nodes)) {
if (!it.get("type").isEqualString("SortNode")) {
continue;
}
CHECK(strategy.empty());
strategy = it.get("strategy").copyString();
}
CHECK(!strategy.empty());
return strategy;
};
auto verifyExpectedResults = [&insertedDocs](TRI_vocbase_t& vocbase,
std::string const& queryString,
std::vector<size_t> const& expected,
std::string rules = "") -> bool {
auto options = arangodb::velocypack::Parser::fromJson(
"{\"optimizer\": {\"rules\": [" + rules + "]}}");
arangodb::aql::Query query(false, vocbase, arangodb::aql::QueryString(queryString),
nullptr, options, 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->waitForAsyncResponse();
} else {
break;
}
}
REQUIRE((TRI_ERROR_NO_ERROR == result.code));
auto slice = result.result->slice();
REQUIRE(slice.isArray());
if (slice.length() != expected.size()) {
return false;
}
size_t i = 0;
for (arangodb::velocypack::ArrayIterator itr(slice); itr.valid(); ++itr) {
auto const resolved = itr.value().resolveExternals();
if (0 != arangodb::basics::VelocyPackHelper::compare(
insertedDocs[expected[i++]].slice(), resolved, true)) {
return false;
};
}
return true;
};
// create collection0, insertedDocs[0, 9999]
{
auto createJson = arangodb::velocypack::Parser::fromJson(
"{ \"name\": \"testCollection0\" }");
auto collection = vocbase.createCollection(createJson->slice());
REQUIRE((nullptr != collection));
std::vector<std::shared_ptr<arangodb::velocypack::Builder>> docs;
size_t total = 10000;
for (size_t i = 0; i < total; i++) {
docs.emplace_back(arangodb::velocypack::Parser::fromJson(
"{ \"valAsc\": " + std::to_string(i) +
", \"valDsc\": " + std::to_string(total - 1 - i) +
", \"mod\": " + std::to_string(i % 100) + "}"));
};
arangodb::OperationOptions options;
options.returnNew = true;
arangodb::SingleCollectionTransaction trx(arangodb::transaction::StandaloneContext::Create(vocbase),
*collection,
arangodb::AccessMode::Type::WRITE);
CHECK((trx.begin().ok()));
for (auto& entry : docs) {
auto res = trx.insert(collection->name(), entry->slice(), options);
CHECK((res.ok()));
insertedDocs.emplace_back(res.slice().get("new"));
}
CHECK((trx.commit().ok()));
CHECK(insertedDocs.size() == total);
}
// check simple limit sorted asc in insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc LIMIT 0, 10 RETURN d";
std::vector<size_t> expected = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check limit with offset sorted asc in insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc LIMIT 10, 10 RETURN d";
std::vector<size_t> expected = {10, 11, 12, 13, 14, 15, 16, 17, 18, 19};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check simple limit sorted asc in reverse insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valDsc LIMIT 0, 10 RETURN d";
std::vector<size_t> expected = {9999, 9998, 9997, 9996, 9995,
9994, 9993, 9992, 9991, 9990};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check limit with offset sorted asc in reverse insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valDsc LIMIT 10, 10 RETURN d";
std::vector<size_t> expected = {9989, 9988, 9987, 9986, 9985,
9984, 9983, 9982, 9981, 9980};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check simple limit sorted dsc in insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc DESC LIMIT 0, 10 RETURN d";
std::vector<size_t> expected = {9999, 9998, 9997, 9996, 9995,
9994, 9993, 9992, 9991, 9990};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check limit with offset sorted dsc in insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc DESC LIMIT 10, 10 RETURN d";
std::vector<size_t> expected = {9989, 9988, 9987, 9986, 9985,
9984, 9983, 9982, 9981, 9980};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check simple limit sorted dsc in reverse insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valDsc DESC LIMIT 0, 10 RETURN d";
std::vector<size_t> expected = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check limit with offset sorted dsc in reverse insertion order
{
std::string query =
"FOR d IN testCollection0 SORT d.valDsc DESC LIMIT 10, 10 RETURN d";
std::vector<size_t> expected = {10, 11, 12, 13, 14, 15, 16, 17, 18, 19};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check limit with offset compound sort
{
std::string query =
"FOR d IN testCollection0 SORT d.mod, d.valAsc LIMIT 10, 10 RETURN d";
std::vector<size_t> expected = {1000, 1100, 1200, 1300, 1400,
1500, 1600, 1700, 1800, 1900};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check limit with offset compound sort again
{
std::string query =
"FOR d IN testCollection0 SORT d.mod, d.valAsc LIMIT 100, 10 RETURN d";
std::vector<size_t> expected = {1, 101, 201, 301, 401,
501, 601, 701, 801, 901};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check interloper, filter moved up
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc FILTER d.mod == 0 LIMIT 0, 10 "
"RETURN d";
std::vector<size_t> expected = {0, 100, 200, 300, 400,
500, 600, 700, 800, 900};
CHECK(sorterType(vocbase, query) == "constrained-heap");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
// check interloper, filter not moved
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc FILTER d.mod == 0 LIMIT 0, 10 "
"RETURN d";
std::string rules = "\"-move-filters-up\", \"-move-filters-up-2\"";
std::vector<size_t> expected = {0, 100, 200, 300, 400,
500, 600, 700, 800, 900};
CHECK(sorterType(vocbase, query, rules) == "standard");
CHECK(verifyExpectedResults(vocbase, query, expected, rules));
}
// check interloper, enumerate list
{
std::string query =
"FOR d IN testCollection0 SORT d.valAsc FOR e IN 1..10 FILTER e == 1 "
"LIMIT 0, 10 RETURN d";
std::vector<size_t> expected = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
CHECK(sorterType(vocbase, query) == "standard");
CHECK(verifyExpectedResults(vocbase, query, expected));
}
}
// -----------------------------------------------------------------------------
// --SECTION-- END-OF-FILE
// -----------------------------------------------------------------------------

View File

@ -1,3 +1,5 @@
include_directories(.)
foreach (LINK_DIR ${V8_LINK_DIRECTORIES})
link_directories("${LINK_DIR}")
endforeach()
@ -83,6 +85,7 @@ set(ARANGODB_TESTS_SOURCES
Aql/DateFunctionsTest.cpp
Aql/EngineInfoContainerCoordinatorTest.cpp
Aql/RestAqlHandlerTest.cpp
Aql/SortLimit-test.cpp
Aql/WaitingExecutionBlockMock.cpp
Auth/UserManagerTest.cpp
Basics/icu-helper.cpp

View File

@ -388,7 +388,7 @@ function ahuacatlFailureSuite () {
testExecutionBlock3 : function () {
internal.debugSetFailAt("ExecutionBlock::getOrSkipSome1");
assertFailingQuery("FOR u in " + c.name() + " SORT u.id DESC LIMIT 0,4 RETURN u");
assertFailingQuery("FOR u in " + c.name() + " SORT u.id DESC LIMIT 0,4 RETURN u", ['-sort-limit']);
},
////////////////////////////////////////////////////////////////////////////////
@ -397,7 +397,7 @@ function ahuacatlFailureSuite () {
testExecutionBlock4 : function () {
internal.debugSetFailAt("ExecutionBlock::getOrSkipSome2");
assertFailingQuery("FOR u in " + c.name() + " SORT u.id DESC LIMIT " + (count - 1) + ",100 RETURN u");
assertFailingQuery("FOR u in " + c.name() + " SORT u.id DESC LIMIT " + (count - 1) + ",100 RETURN u", ['-sort-limit']);
},
////////////////////////////////////////////////////////////////////////////////

View File

@ -0,0 +1,315 @@
/*jshint globalstrict:false, strict:false, maxlen: 500 */
/*global assertEqual, AQL_EXPLAIN */
////////////////////////////////////////////////////////////////////////////////
/// @brief tests for query language, limit optimizations
///
/// @file
///
/// DISCLAIMER
///
/// Copyright 2010-2012 triagens GmbH, Cologne, Germany
///
/// Licensed under the Apache License, Version 2.0 (the "License");
/// you may not use this file except in compliance with the License.
/// You may obtain a copy of the License at
///
/// http://www.apache.org/licenses/LICENSE-2.0
///
/// Unless required by applicable law or agreed to in writing, software
/// distributed under the License is distributed on an "AS IS" BASIS,
/// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
/// See the License for the specific language governing permissions and
/// limitations under the License.
///
/// Copyright holder is triAGENS GmbH, Cologne, Germany
///
/// @author Jan Steemann
/// @author Copyright 2012, triAGENS GmbH, Cologne, Germany
////////////////////////////////////////////////////////////////////////////////
var jsunity = require("jsunity");
var internal = require("internal");
var helper = require("@arangodb/aql-helper");
var getQueryResults = helper.getQueryResults;
var db = internal.db;
////////////////////////////////////////////////////////////////////////////////
/// @brief test suite
////////////////////////////////////////////////////////////////////////////////
function ahuacatlQueryOptimizerLimitTestSuite () {
var collection = null;
var docCount = 1000;
var cn = "UnitTestsAhuacatlOptimizerLimit";
var getSorts = function (query, params) {
return AQL_EXPLAIN(query, params, { optimizer: { rules: [ "-all", "+use-indexes", "+use-index-for-sort", "+sort-limit" ] } }).plan.nodes.filter(node => node.type === "SortNode");
};
return {
////////////////////////////////////////////////////////////////////////////////
/// @brief set up
////////////////////////////////////////////////////////////////////////////////
setUp : function () {
internal.db._drop(cn);
collection = internal.db._create(cn);
for (var i = 0; i < docCount; ++i) {
collection.save({ _key: "test" + i, value : i });
}
},
////////////////////////////////////////////////////////////////////////////////
/// @brief tear down
////////////////////////////////////////////////////////////////////////////////
tearDown : function () {
internal.db._drop(cn);
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with simple data, too short
////////////////////////////////////////////////////////////////////////////////
testLimitSimple : function () {
var query = "FOR c IN [1,3,5,2,4] SORT c LIMIT 3 RETURN c";
var actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([1, 2, 3], actual);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
query = "FOR c IN [1,3,5,2,4] SORT c DESC LIMIT 3 RETURN c";
actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([5, 4, 3], actual);
sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with simple data, filter, too short
////////////////////////////////////////////////////////////////////////////////
testLimitSimpleFilter : function () {
var query = "FOR c IN [1,3,5,2,4] SORT c FILTER c >= 3 LIMIT 2 RETURN c";
var actual = getQueryResults(query);
assertEqual(2, actual.length);
assertEqual([3, 4], actual);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
query = "FOR c IN [1,3,5,2,4] SORT c DESC FILTER c >= 3 LIMIT 2 RETURN c";
actual = getQueryResults(query);
assertEqual(2, actual.length);
assertEqual([5, 4], actual);
sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with simple data
////////////////////////////////////////////////////////////////////////////////
testLimitSimpleLong : function () {
var query = "FOR c IN 1..1000 SORT c LIMIT 3 RETURN c";
var actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([1, 2, 3], actual);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 3);
assertEqual(sorts[0].strategy, "constrained-heap");
query = "FOR c IN 1..1000 SORT c LIMIT 100, 3 RETURN c";
actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([101, 102, 103], actual);
sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 103);
assertEqual(sorts[0].strategy, "constrained-heap");
query = "FOR c IN 1..1000 SORT c DESC LIMIT 3 RETURN c";
actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([1000, 999, 998], actual);
sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 3);
assertEqual(sorts[0].strategy, "constrained-heap");
query = "FOR c IN 1..1000 SORT c DESC LIMIT 100, 3 RETURN c";
actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([900, 899, 898], actual);
sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 103);
assertEqual(sorts[0].strategy, "constrained-heap");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with simple data, filter
////////////////////////////////////////////////////////////////////////////////
testLimitSimpleFilterLong : function () {
var query = "FOR c IN 1..1000 SORT c FILTER c > 3 LIMIT 3 RETURN c";
var actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([4, 5, 6], actual);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
query = "FOR c IN 1..1000 SORT c DESC FILTER c < 900 LIMIT 3 RETURN c";
actual = getQueryResults(query);
assertEqual(3, actual.length);
assertEqual([899, 898, 897], actual);
sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with 2 limits
////////////////////////////////////////////////////////////////////////////////
testLimitFullCollectionLimitLimit : function () {
var tests = [
{ offset: 0, limit: 500, offset2: 0, limit2: 1, expectedLength: 1 },
{ offset: 10, limit: 5, offset2: 0, limit2: 1, expectedLength: 1 },
{ offset: 10, limit: 5, offset2: 0, limit2: 20, expectedLength: 5 },
{ offset: 10, limit: 50, offset2: 1, limit2: 20, expectedLength: 20 },
{ offset: 10, limit: 90, offset2: 10, limit2: 20, expectedLength: 20 },
{ offset: 90, limit: 10, offset2: 9, limit2: 20, expectedLength: 1 },
{ offset: 50, limit: 50, offset2: 0, limit2: 50, expectedLength: 50 },
{ offset: 50, limit: 50, offset2: 10, limit2: 50, expectedLength: 40 },
{ offset: 50, limit: 50, offset2: 50, limit2: 50, expectedLength: 0 }
];
for (var i = 0; i < tests.length; ++i) {
var test = tests[i];
var query = "FOR c IN " + cn + " SORT c.value LIMIT " + test.offset + ", " + test.limit + " LIMIT " + test.offset2 + ", " + test.limit2 + " RETURN c";
var actual = getQueryResults(query);
assertEqual(test.expectedLength, actual.length);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, test.offset + test.limit);
assertEqual(sorts[0].strategy, "constrained-heap");
}
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with sort
////////////////////////////////////////////////////////////////////////////////
testLimitFullCollectionSort1 : function () {
var query = "FOR c IN " + cn + " FILTER c.value >= 20 && c.value < 30 LIMIT 0, 10 SORT c.value RETURN c";
var actual = getQueryResults(query);
assertEqual(10, actual.length);
assertEqual(20, actual[0].value);
assertEqual(21, actual[1].value);
assertEqual(29, actual[9].value);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with sort
////////////////////////////////////////////////////////////////////////////////
testLimitFullCollectionSort2 : function () {
var query = "FOR c IN " + cn + " FILTER c.value >= 20 && c.value < 30 LIMIT 0, 10 SORT c.value RETURN c";
var actual = getQueryResults(query);
assertEqual(10, actual.length);
assertEqual(20, actual[0].value);
assertEqual(21, actual[1].value);
assertEqual(22, actual[2].value);
assertEqual(29, actual[9].value);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 0);
assertEqual(sorts[0].strategy, "standard");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with sort
////////////////////////////////////////////////////////////////////////////////
testLimitFullCollectionSort3 : function () {
var query = "FOR c IN " + cn + " SORT c.value LIMIT 0, 10 FILTER c.value >= 20 && c.value < 30 RETURN c";
var actual = getQueryResults(query);
assertEqual(0, actual.length);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 10);
assertEqual(sorts[0].strategy, "constrained-heap");
},
////////////////////////////////////////////////////////////////////////////////
/// @brief check limit optimization with sort
////////////////////////////////////////////////////////////////////////////////
testLimitFullCollectionSort4 : function () {
var query = "FOR c IN " + cn + " FILTER c.value >= 20 && c.value < 30 SORT c.value LIMIT 0, 10 RETURN c";
var actual = getQueryResults(query);
assertEqual(10, actual.length);
assertEqual(20, actual[0].value);
assertEqual(21, actual[1].value);
assertEqual(22, actual[2].value);
assertEqual(29, actual[9].value);
var sorts = getSorts(query);
assertEqual(sorts.length, 1);
assertEqual(sorts[0].limit, 10);
assertEqual(sorts[0].strategy, "constrained-heap");
},
};
}
////////////////////////////////////////////////////////////////////////////////
/// @brief executes the test suite
////////////////////////////////////////////////////////////////////////////////
jsunity.run(ahuacatlQueryOptimizerLimitTestSuite);
return jsunity.done();