From f1a67d4437025a28db921e2b613899773d4f2505 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Mon, 18 Nov 2024 19:16:43 +0100 Subject: [PATCH 01/31] Implement lazy join --- src/engine/AddCombinedRowToTable.h | 14 +- src/engine/Join.cpp | 455 ++++++++++++++------ src/engine/Join.h | 26 +- src/index/IndexImpl.cpp | 36 +- src/util/JoinAlgorithms/JoinColumnMapping.h | 10 +- test/JoinTest.cpp | 194 ++++++++- 6 files changed, 562 insertions(+), 173 deletions(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index 75c2bcd848..41d0d0d73e 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -26,6 +26,7 @@ class AddCombinedRowToIdTable { size_t numJoinColumns_; std::optional, 2>> inputLeftAndRight_; IdTable resultTable_; + LocalVocab mergedVocab_{}; // This struct stores the information, which row indices from the input are // combined into a given row index in the output, i.e. "To obtain the @@ -62,7 +63,7 @@ class AddCombinedRowToIdTable { // This callback is called with the result as an argument each time `flush()` // is called. It can be used to consume parts of the result early, before the // complete operation has finished. - using BlockwiseCallback = std::function; + using BlockwiseCallback = std::function; [[no_unique_address]] BlockwiseCallback blockwiseCallback_{ad_utility::noop}; using CancellationHandle = ad_utility::SharedCancellationHandle; @@ -138,10 +139,17 @@ class AddCombinedRowToIdTable { return table; } }; + auto mergeVocab = [this](const T& table) { + if constexpr (requires { table.getLocalVocab(); }) { + mergedVocab_.mergeWith(std::span{&table.getLocalVocab(), 1}); + } + }; if (nextIndex_ != 0) { AD_CORRECTNESS_CHECK(inputLeftAndRight_.has_value()); flush(); } + mergeVocab(inputLeft); + mergeVocab(inputRight); inputLeftAndRight_ = std::array{toView(inputLeft), toView(inputRight)}; checkNumColumns(); } @@ -188,6 +196,8 @@ class AddCombinedRowToIdTable { return std::move(resultTable_); } + LocalVocab& localVocab() { return mergedVocab_; } + // Disable copying and moving, it is currently not needed and makes it harder // to reason about AddCombinedRowToIdTable(const AddCombinedRowToIdTable&) = delete; @@ -320,7 +330,7 @@ class AddCombinedRowToIdTable { indexBuffer_.clear(); optionalIndexBuffer_.clear(); nextIndex_ = 0; - std::invoke(blockwiseCallback_, result); + std::invoke(blockwiseCallback_, result, mergedVocab_); } const IdTableView<0>& inputLeft() const { return inputLeftAndRight_.value()[0]; diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 184df75128..14ab366eee 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -20,10 +20,24 @@ #include "global/RuntimeParameters.h" #include "util/Exception.h" #include "util/HashMap.h" +#include "util/JoinAlgorithms/JoinAlgorithms.h" +#include "util/ThreadSafeQueue.h" using std::endl; using std::string; +namespace { +// Convert a `generator` to a `generator` for +// more efficient access in the join columns below. +cppcoro::generator> convertGenerator( + Result::Generator gen) { + for (auto& [table, localVocab] : gen) { + ad_utility::IdTableAndFirstCol t{std::move(table), std::move(localVocab)}; + co_yield t; + } +} +} // namespace + // _____________________________________________________________________________ Join::Join(QueryExecutionContext* qec, std::shared_ptr t1, std::shared_ptr t2, ColumnIndex t1JoinCol, @@ -90,44 +104,35 @@ string Join::getCacheKeyImpl() const { string Join::getDescriptor() const { return "Join on " + _joinVar.name(); } // _____________________________________________________________________________ -ProtoResult Join::computeResult([[maybe_unused]] bool requestLaziness) { +ProtoResult Join::computeResult(bool requestLaziness) { LOG(DEBUG) << "Getting sub-results for join result computation..." << endl; - IdTable idTable{getResultWidth(), getExecutionContext()->getAllocator()}; - if (_left->knownEmptyResult() || _right->knownEmptyResult()) { _left->getRootOperation()->updateRuntimeInformationWhenOptimizedOut(); _right->getRootOperation()->updateRuntimeInformationWhenOptimizedOut(); - return {std::move(idTable), resultSortedOn(), LocalVocab()}; + return {IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + resultSortedOn(), LocalVocab()}; } // Always materialize results that meet one of the following criteria: // * They are already present in the cache // * Their result is small - // * They might contain UNDEF values in the join column - // The first two conditions are for performance reasons, the last one is - // because we currently cannot perform the optimized lazy joins when UNDEF - // values are involved. - auto getCachedOrSmallResult = [](QueryExecutionTree& tree, - ColumnIndex joinCol) { + // This is purely for performance reasons. + auto getCachedOrSmallResult = [](const QueryExecutionTree& tree) { bool isSmall = tree.getRootOperation()->getSizeEstimate() < RuntimeParameters().get<"lazy-index-scan-max-size-materialization">(); - auto undefStatus = - tree.getVariableAndInfoByColumnIndex(joinCol).second.mightContainUndef_; - bool containsUndef = - undefStatus == ColumnIndexAndTypeInfo::UndefStatus::PossiblyUndefined; // The third argument means "only get the result if it can be read from the - // cache". So effectively, this returns the result if it is small, contains - // UNDEF values, or is contained in the cache, otherwise `nullptr`. + // cache". So effectively, this returns the result if it is small, or is + // contained in the cache, otherwise `nullptr`. // TODO Add a unit test that checks the correct conditions return tree.getRootOperation()->getResult( - false, (isSmall || containsUndef) ? ComputationMode::FULLY_MATERIALIZED - : ComputationMode::ONLY_IF_CACHED); + false, isSmall ? ComputationMode::FULLY_MATERIALIZED + : ComputationMode::ONLY_IF_CACHED); }; - auto leftResIfCached = getCachedOrSmallResult(*_left, _leftJoinCol); + auto leftResIfCached = getCachedOrSmallResult(*_left); checkCancellation(); - auto rightResIfCached = getCachedOrSmallResult(*_right, _rightJoinCol); + auto rightResIfCached = getCachedOrSmallResult(*_right); checkCancellation(); auto leftIndexScan = @@ -135,20 +140,13 @@ ProtoResult Join::computeResult([[maybe_unused]] bool requestLaziness) { if (leftIndexScan && std::dynamic_pointer_cast(_right->getRootOperation())) { if (rightResIfCached && !leftResIfCached) { - idTable = computeResultForIndexScanAndIdTable( - rightResIfCached->idTable(), _rightJoinCol, *leftIndexScan, - _leftJoinCol); - checkCancellation(); - return {std::move(idTable), resultSortedOn(), LocalVocab{}}; + AD_CORRECTNESS_CHECK(rightResIfCached->isFullyMaterialized()); + return computeResultForIndexScanAndIdTable( + requestLaziness, rightResIfCached->idTable(), _rightJoinCol, + leftIndexScan, _leftJoinCol); } else if (!leftResIfCached) { - idTable = computeResultForTwoIndexScans(); - checkCancellation(); - // TODO When we add triples to the - // index, the vocabularies of index scans will not necessarily be empty - // and we need a mechanism to still retrieve them when using the lazy - // scan. - return {std::move(idTable), resultSortedOn(), LocalVocab{}}; + return computeResultForTwoIndexScans(requestLaziness); } } @@ -157,46 +155,45 @@ ProtoResult Join::computeResult([[maybe_unused]] bool requestLaziness) { Service::precomputeSiblingResult(_left->getRootOperation(), _right->getRootOperation(), false, requestLaziness); - std::shared_ptr leftRes = - leftResIfCached ? leftResIfCached : _left->getResult(); + leftResIfCached ? leftResIfCached : _left->getResult(true); checkCancellation(); - if (leftRes->idTable().empty()) { + if (leftRes->isFullyMaterialized() && leftRes->idTable().empty()) { _right->getRootOperation()->updateRuntimeInformationWhenOptimizedOut(); // TODO When we add triples to the // index, the vocabularies of index scans will not necessarily be empty and // we need a mechanism to still retrieve them when using the lazy scan. - return {std::move(idTable), resultSortedOn(), LocalVocab()}; + return {IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + resultSortedOn(), LocalVocab()}; } // Note: If only one of the children is a scan, then we have made sure in the // constructor that it is the right child. - // We currently cannot use this optimized lazy scan if the result from `_left` - // contains UNDEF values. - const auto& leftIdTable = leftRes->idTable(); - auto leftHasUndef = - !leftIdTable.empty() && leftIdTable.at(0, _leftJoinCol).isUndefined(); auto rightIndexScan = std::dynamic_pointer_cast(_right->getRootOperation()); - if (rightIndexScan && !rightResIfCached && !leftHasUndef) { - idTable = computeResultForIndexScanAndIdTable( - leftRes->idTable(), _leftJoinCol, *rightIndexScan, _rightJoinCol); - checkCancellation(); - return {std::move(idTable), resultSortedOn(), - leftRes->getSharedLocalVocab()}; + if (rightIndexScan && !rightResIfCached && leftRes->isFullyMaterialized()) { + const auto& leftIdTable = leftRes->idTable(); + return computeResultForIndexScanAndIdTable( + requestLaziness, leftIdTable, _leftJoinCol, rightIndexScan, + _rightJoinCol, leftRes); } std::shared_ptr rightRes = - rightResIfCached ? rightResIfCached : _right->getResult(); - checkCancellation(); - join(leftRes->idTable(), _leftJoinCol, rightRes->idTable(), _rightJoinCol, - &idTable); + rightResIfCached ? rightResIfCached : _right->getResult(true); checkCancellation(); + if (leftRes->isFullyMaterialized() && rightRes->isFullyMaterialized()) { + IdTable idTable{getResultWidth(), getExecutionContext()->getAllocator()}; + join(leftRes->idTable(), _leftJoinCol, rightRes->idTable(), _rightJoinCol, + &idTable); + checkCancellation(); - // If only one of the two operands has a non-empty local vocabulary, share - // with that one (otherwise, throws an exception). - return {std::move(idTable), resultSortedOn(), - Result::getMergedLocalVocab(*leftRes, *rightRes)}; + // If only one of the two operands has a non-empty local vocabulary, share + // with that one (otherwise, throws an exception). + return {std::move(idTable), resultSortedOn(), + Result::getMergedLocalVocab(*leftRes, *rightRes)}; + } + return lazyJoin(std::move(leftRes), _leftJoinCol, std::move(rightRes), + _rightJoinCol, requestLaziness); } // _____________________________________________________________________________ @@ -408,6 +405,129 @@ void Join::join(const IdTable& a, ColumnIndex jc1, const IdTable& b, << ", size = " << result->size() << "\n"; } +// _____________________________________________________________________________ +ProtoResult Join::createResult(bool requestedLaziness, auto action) const { + if (requestedLaziness) { + auto queue = std::make_shared< + ad_utility::data_structures::ThreadSafeQueue>( + 1); + ad_utility::JThread{[queue, action = std::move(action)]() { + auto addValue = [&queue](Result::IdTableVocabPair value) { + if (value.idTable_.empty()) { + return; + } + queue->push(std::move(value)); + }; + try { + addValue(action(addValue)); + queue->finish(); + } catch (...) { + queue->pushException(std::current_exception()); + } + }}.detach(); + return {[](auto queue) -> Result::Generator { + while (true) { + auto val = queue->pop(); + if (!val.has_value()) { + break; + } + co_yield val.value(); + } + }(std::move(queue)), + resultSortedOn()}; + } else { + auto [idTable, localVocab] = action(ad_utility::noop); + return {std::move(idTable), resultSortedOn(), std::move(localVocab)}; + } +} + +// _____________________________________________________________________________ +bool Join::couldContainUndef(const auto& blocks, const auto& tree, + ColumnIndex joinColumn) { + if constexpr (std::ranges::random_access_range) { + AD_CORRECTNESS_CHECK(!std::ranges::empty(blocks)); + return !blocks[0].empty() && blocks[0][0].isUndefined(); + } else { + auto undefStatus = tree->getVariableAndInfoByColumnIndex(joinColumn) + .second.mightContainUndef_; + return undefStatus == + ColumnIndexAndTypeInfo::UndefStatus::PossiblyUndefined; + } +} + +// ______________________________________________________________________________ +ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, + std::shared_ptr b, ColumnIndex jc2, + bool requestLaziness) const { + return createResult( + requestLaziness, + [this, a = std::move(a), jc1, b = std::move(b), + jc2](std::invocable auto yieldTable) { + // If both inputs are fully materialized, we can join them more + // efficiently. + AD_CONTRACT_CHECK(!a->isFullyMaterialized() || + !b->isFullyMaterialized()); + auto joinColMap = ad_utility::JoinColumnMapping{ + {{jc1, jc2}}, + _left->getRootOperation()->getResultWidth(), + _right->getRootOperation()->getResultWidth()}; + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, + [&yieldTable, &joinColMap](IdTable& idTable, + LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; + } + idTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(idTable), + std::move(localVocab)}); + }}; + auto toBlockRange = []( + T& blockOrBlocks, + [[maybe_unused]] std::span + columnIndices = {}) + -> std::variant< + cppcoro::generator>, + std::array>, 1>> { + if constexpr (ad_utility::isSimilar) { + return std::array{ad_utility::IdTableAndFirstCol{ + blockOrBlocks.asColumnSubsetView(columnIndices)}}; + } else if constexpr (std::ranges::range) { + return convertGenerator(std::move(blockOrBlocks)); + } else { + static_assert(ad_utility::alwaysFalse, "Unexpected type"); + } + }; + auto leftRange = + a->isFullyMaterialized() + ? toBlockRange(a->idTable(), joinColMap.permutationLeft()) + : toBlockRange(a->idTables()); + auto rightRange = + b->isFullyMaterialized() + ? toBlockRange(b->idTable(), joinColMap.permutationRight()) + : toBlockRange(b->idTables()); + std::visit( + [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { + bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || + couldContainUndef(rightBlocks, _right, jc2); + if (containsUndef) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + leftBlocks, rightBlocks, std::less{}, rowAdder); + } else { + ad_utility::zipperJoinForBlocksWithoutUndef( + leftBlocks, rightBlocks, std::less{}, rowAdder); + } + }, + leftRange, rightRange); + auto localVocab = std::move(rowAdder.localVocab()); + auto result = std::move(rowAdder).resultTable(); + result.setColumnSubset(joinColMap.permutationResult()); + return Result::IdTableVocabPair{std::move(result), + std::move(localVocab)}; + }); +} + // ______________________________________________________________________________ template void Join::hashJoinImpl(const IdTable& dynA, ColumnIndex jc1, @@ -582,89 +702,154 @@ void updateRuntimeInfoForLazyScan( } // namespace // ______________________________________________________________________________________________________ -IdTable Join::computeResultForTwoIndexScans() { - auto leftScan = - std::dynamic_pointer_cast(_left->getRootOperation()); - auto rightScan = - std::dynamic_pointer_cast(_right->getRootOperation()); - AD_CORRECTNESS_CHECK(leftScan && rightScan); - // The join column already is the first column in both inputs, so we don't - // have to permute the inputs and results for the `AddCombinedRowToIdTable` - // class to work correctly. - AD_CORRECTNESS_CHECK(_leftJoinCol == 0 && _rightJoinCol == 0); - ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_}; - - ad_utility::Timer timer{ad_utility::timer::Timer::InitialStatus::Started}; - auto [leftBlocksInternal, rightBlocksInternal] = - IndexScan::lazyScanForJoinOfTwoScans(*leftScan, *rightScan); - runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); - - auto leftBlocks = convertGenerator(std::move(leftBlocksInternal)); - auto rightBlocks = convertGenerator(std::move(rightBlocksInternal)); - - ad_utility::zipperJoinForBlocksWithoutUndef(leftBlocks, rightBlocks, - std::less{}, rowAdder); - - updateRuntimeInfoForLazyScan(*leftScan, leftBlocks.details()); - updateRuntimeInfoForLazyScan(*rightScan, rightBlocks.details()); - - AD_CORRECTNESS_CHECK(leftBlocks.details().numBlocksRead_ <= - rightBlocks.details().numElementsRead_); - AD_CORRECTNESS_CHECK(rightBlocks.details().numBlocksRead_ <= - leftBlocks.details().numElementsRead_); - - return std::move(rowAdder).resultTable(); +ProtoResult Join::computeResultForTwoIndexScans(bool requestLaziness) const { + return createResult( + requestLaziness, + [this](std::invocable auto yieldTable) { + auto leftScan = + std::dynamic_pointer_cast(_left->getRootOperation()); + auto rightScan = + std::dynamic_pointer_cast(_right->getRootOperation()); + AD_CORRECTNESS_CHECK(leftScan && rightScan); + // The join column already is the first column in both inputs, so we + // don't have to permute the inputs and results for the + // `AddCombinedRowToIdTable` class to work correctly. + AD_CORRECTNESS_CHECK(_leftJoinCol == 0 && _rightJoinCol == 0); + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, + [&yieldTable](IdTable& idTable, LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; + } + yieldTable(Result::IdTableVocabPair{std::move(idTable), + std::move(localVocab)}); + }}; + + ad_utility::Timer timer{ + ad_utility::timer::Timer::InitialStatus::Started}; + auto [leftBlocksInternal, rightBlocksInternal] = + IndexScan::lazyScanForJoinOfTwoScans(*leftScan, *rightScan); + runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); + + auto leftBlocks = convertGenerator(std::move(leftBlocksInternal)); + auto rightBlocks = convertGenerator(std::move(rightBlocksInternal)); + + ad_utility::zipperJoinForBlocksWithoutUndef(leftBlocks, rightBlocks, + std::less{}, rowAdder); + + updateRuntimeInfoForLazyScan(*leftScan, leftBlocks.details()); + updateRuntimeInfoForLazyScan(*rightScan, rightBlocks.details()); + + AD_CORRECTNESS_CHECK(leftBlocks.details().numBlocksRead_ <= + rightBlocks.details().numElementsRead_); + AD_CORRECTNESS_CHECK(rightBlocks.details().numBlocksRead_ <= + leftBlocks.details().numElementsRead_); + auto localVocab = std::move(rowAdder.localVocab()); + return Result::IdTableVocabPair{std::move(rowAdder).resultTable(), + std::move(localVocab)}; + }); } // ______________________________________________________________________________________________________ template -IdTable Join::computeResultForIndexScanAndIdTable(const IdTable& idTable, - ColumnIndex joinColTable, - IndexScan& scan, - ColumnIndex joinColScan) { - // We first have to permute the columns. - auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { - return idTableIsRightInput - ? std::tuple{joinColScan, joinColTable, scan.getResultWidth(), - idTable.numColumns()} - : std::tuple{joinColTable, joinColScan, idTable.numColumns(), - scan.getResultWidth()}; - }(); - - auto joinColMap = ad_utility::JoinColumnMapping{ - {{jcLeft, jcRight}}, numColsLeft, numColsRight}; - ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_}; - - AD_CORRECTNESS_CHECK(joinColScan == 0); - auto permutationIdTable = - ad_utility::IdTableAndFirstCol{idTable.asColumnSubsetView( - idTableIsRightInput ? joinColMap.permutationRight() - : joinColMap.permutationLeft())}; - - ad_utility::Timer timer{ad_utility::timer::Timer::InitialStatus::Started}; - auto rightBlocksInternal = - scan.lazyScanForJoinOfColumnWithScan(permutationIdTable.col()); - auto rightBlocks = convertGenerator(std::move(rightBlocksInternal)); - - runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); - - auto doJoin = [&rowAdder](auto& left, auto& right) mutable { - ad_utility::zipperJoinForBlocksWithoutUndef(left, right, std::less{}, - rowAdder); - }; - auto blockForIdTable = std::span{&permutationIdTable, 1}; - if (idTableIsRightInput) { - doJoin(rightBlocks, blockForIdTable); - } else { - doJoin(blockForIdTable, rightBlocks); - } - auto result = std::move(rowAdder).resultTable(); - result.setColumnSubset(joinColMap.permutationResult()); +ProtoResult Join::computeResultForIndexScanAndIdTable( + bool requestLaziness, const IdTable& idTable, ColumnIndex joinColTable, + std::shared_ptr scan, ColumnIndex joinColScan, + const std::shared_ptr& subResult) const { + return createResult( + requestLaziness, + [this, &idTable, joinColTable, scan = std::move(scan), joinColScan, + subResult = std::move(subResult)]( + std::invocable auto yieldTable) { + // We first have to permute the columns. + auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { + return idTableIsRightInput + ? std::tuple{joinColScan, joinColTable, + scan->getResultWidth(), idTable.numColumns()} + : std::tuple{joinColTable, joinColScan, + idTable.numColumns(), scan->getResultWidth()}; + }(); + + auto joinColMap = ad_utility::JoinColumnMapping{ + {{jcLeft, jcRight}}, numColsLeft, numColsRight}; + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, + [&yieldTable, &joinColMap](IdTable& idTable, + LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; + } + idTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(idTable), + std::move(localVocab)}); + }}; + + AD_CORRECTNESS_CHECK(joinColScan == 0); + auto permutationIdTable = + ad_utility::IdTableAndFirstCol{idTable.asColumnSubsetView( + idTableIsRightInput ? joinColMap.permutationRight() + : joinColMap.permutationLeft())}; + + ad_utility::Timer timer{ + ad_utility::timer::Timer::InitialStatus::Started}; + bool idTableHasUndef = + !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); + std::optional> indexScanResult = + std::nullopt; + using FirstColView = ad_utility::IdTableAndFirstCol; + using GenWithDetails = + cppcoro::generator; + auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, + &indexScanResult]() + -> std::variant, GenWithDetails> { + if (idTableHasUndef) { + indexScanResult = + scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); + AD_CORRECTNESS_CHECK( + !indexScanResult.value()->isFullyMaterialized()); + return convertGenerator( + std::move(indexScanResult.value()->idTables())); + } else { + auto rightBlocksInternal = + scan->lazyScanForJoinOfColumnWithScan(permutationIdTable.col()); + return convertGenerator(std::move(rightBlocksInternal)); + } + }(); + + runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); + auto doJoin = [&rowAdder, idTableHasUndef](auto& left, + auto& right) mutable { + if (idTableHasUndef) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + left, right, std::less{}, rowAdder); + } else { + ad_utility::zipperJoinForBlocksWithoutUndef(left, right, + std::less{}, rowAdder); + } + }; + auto blockForIdTable = std::array{std::move(permutationIdTable)}; + std::visit( + [&doJoin, &blockForIdTable](auto& blocks) { + if constexpr (idTableIsRightInput) { + doJoin(blocks, blockForIdTable); + } else { + doJoin(blockForIdTable, blocks); + } + }, + rightBlocks); + + if (std::holds_alternative(rightBlocks)) { + updateRuntimeInfoForLazyScan( + *scan, std::get(rightBlocks).details()); + } - updateRuntimeInfoForLazyScan(scan, rightBlocks.details()); - return result; + auto localVocab = std::move(rowAdder.localVocab()); + auto result = std::move(rowAdder).resultTable(); + result.setColumnSubset(joinColMap.permutationResult()); + return Result::IdTableVocabPair{std::move(result), + std::move(localVocab)}; + }); } diff --git a/src/engine/Join.h b/src/engine/Join.h index d23a9e4f5d..d82bda849f 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -11,7 +11,7 @@ #include "engine/QueryExecutionTree.h" #include "util/HashMap.h" #include "util/HashSet.h" -#include "util/JoinAlgorithms/JoinAlgorithms.h" +#include "util/TypeTraits.h" class Join : public Operation { private: @@ -33,6 +33,7 @@ class Join : public Operation { std::shared_ptr t2, ColumnIndex t1JoinCol, ColumnIndex t2JoinCol); + static constexpr size_t CHUNK_SIZE = 100'000; // A very explicit constructor, which initializes an invalid join object (it // has no subtrees, which violates class invariants). These invalid Join // objects can be used for unit tests that only test member functions which @@ -93,6 +94,17 @@ class Join : public Operation { void join(const IdTable& a, ColumnIndex jc1, const IdTable& b, ColumnIndex jc2, IdTable* result) const; + // action is a lambda with signature + // Result::IdTableVocabPair(void(Result::IdTableVocabPair)) + ProtoResult createResult(bool requestedLaziness, auto action) const; + + static bool couldContainUndef(const auto& blocks, const auto& tree, + ColumnIndex joinColumn); + + ProtoResult lazyJoin(std::shared_ptr a, ColumnIndex jc1, + std::shared_ptr b, ColumnIndex jc2, + bool requestLaziness) const; + /** * @brief Joins IdTables dynA and dynB on join column jc2, returning * the result in dynRes. Creates a cross product for matching rows by putting @@ -113,24 +125,24 @@ class Join : public Operation { virtual string getCacheKeyImpl() const override; private: - ProtoResult computeResult([[maybe_unused]] bool requestLaziness) override; + ProtoResult computeResult(bool requestLaziness) override; VariableToColumnMap computeVariableToColumnMap() const override; // A special implementation that is called when both children are // `IndexScan`s. Uses the lazy scans to only retrieve the subset of the // `IndexScan`s that is actually needed without fully materializing them. - IdTable computeResultForTwoIndexScans(); + ProtoResult computeResultForTwoIndexScans(bool requestLaziness) const; // A special implementation that is called when one of the children is an // `IndexScan`. The argument `scanIsLeft` determines whether the `IndexScan` // is the left or the right child of this `Join`. This needs to be known to // determine the correct order of the columns in the result. template - IdTable computeResultForIndexScanAndIdTable(const IdTable& idTable, - ColumnIndex joinColTable, - IndexScan& scan, - ColumnIndex joinColScan); + ProtoResult computeResultForIndexScanAndIdTable( + bool requestLaziness, const IdTable& idTable, ColumnIndex joinColTable, + std::shared_ptr scan, ColumnIndex joinColScan, + const std::shared_ptr& subResult = nullptr) const; /* * @brief Combines 2 rows like in a join and inserts the result in the diff --git a/src/index/IndexImpl.cpp b/src/index/IndexImpl.cpp index 93b2fc09ea..6b0e071aa2 100644 --- a/src/index/IndexImpl.cpp +++ b/src/index/IndexImpl.cpp @@ -192,24 +192,24 @@ IndexImpl::buildOspWithPatterns( // them to the queue. IdTable outputBufferTable{NumColumnsIndexBuilding + 2, ad_utility::makeUnlimitedAllocator()}; - auto pushToQueue = - [&, bufferSize = - BUFFER_SIZE_JOIN_PATTERNS_WITH_OSP.load()](IdTable& table) { - if (table.numRows() >= bufferSize) { - if (!outputBufferTable.empty()) { - queue.push(std::move(outputBufferTable)); - outputBufferTable.clear(); - } - queue.push(std::move(table)); - } else { - outputBufferTable.insertAtEnd(table.begin(), table.end()); - if (outputBufferTable.size() >= bufferSize) { - queue.push(std::move(outputBufferTable)); - outputBufferTable.clear(); - } - } - table.clear(); - }; + auto pushToQueue = [&, bufferSize = + BUFFER_SIZE_JOIN_PATTERNS_WITH_OSP.load()]( + IdTable& table, LocalVocab&) { + if (table.numRows() >= bufferSize) { + if (!outputBufferTable.empty()) { + queue.push(std::move(outputBufferTable)); + outputBufferTable.clear(); + } + queue.push(std::move(table)); + } else { + outputBufferTable.insertAtEnd(table.begin(), table.end()); + if (outputBufferTable.size() >= bufferSize) { + queue.push(std::move(outputBufferTable)); + outputBufferTable.clear(); + } + } + table.clear(); + }; lazyOptionalJoinOnFirstColumn(ospAsBlocksTransformed, lazyPatternScan, pushToQueue); diff --git a/src/util/JoinAlgorithms/JoinColumnMapping.h b/src/util/JoinAlgorithms/JoinColumnMapping.h index bb01c24a9b..110256bc8a 100644 --- a/src/util/JoinAlgorithms/JoinColumnMapping.h +++ b/src/util/JoinAlgorithms/JoinColumnMapping.h @@ -8,6 +8,7 @@ #include #include +#include "engine/LocalVocab.h" #include "util/Algorithm.h" namespace ad_utility { @@ -108,6 +109,7 @@ template struct IdTableAndFirstCol { private: Table table_; + LocalVocab localVocab_; public: // Typedef needed for generic interfaces. @@ -116,7 +118,9 @@ struct IdTableAndFirstCol { std::decay_t; // Construct by taking ownership of the table. - explicit IdTableAndFirstCol(Table t) : table_{std::move(t)} {} + explicit IdTableAndFirstCol(Table t) : table_{std::move(t)}, localVocab_{} {} + IdTableAndFirstCol(Table t, LocalVocab localVocab) + : table_{std::move(t)}, localVocab_{std::move(localVocab)} {} // Get access to the first column. decltype(auto) col() { return table_.getColumn(0); } @@ -131,6 +135,8 @@ struct IdTableAndFirstCol { bool empty() const { return col().empty(); } const Id& operator[](size_t idx) const { return col()[idx]; } + const Id& front() const { return col().front(); } + const Id& back() const { return col().back(); } size_t size() const { return col().size(); } @@ -141,5 +147,7 @@ struct IdTableAndFirstCol { IdTableView asStaticView() const { return table_.template asStaticView(); } + + const LocalVocab& getLocalVocab() const { return localVocab_; } }; } // namespace ad_utility diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index c64c03d5d2..a9ec32b523 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -230,11 +230,27 @@ using ExpectedColumns = ad_utility::HashMap< std::pair, ColumnIndexAndTypeInfo::UndefStatus>>; // Test that the result of the `join` matches the `expected` outcome. -void testJoinOperation(Join& join, const ExpectedColumns& expected) { - auto res = join.getResult(); +void testJoinOperation(Join& join, const ExpectedColumns& expected, + bool requestLaziness = false, + bool expectLazinessParityWhenNonEmpty = false, + ad_utility::source_location location = + ad_utility::source_location::current()) { + auto lt = generateLocationTrace(location); + auto res = join.getResult(false, requestLaziness + ? ComputationMode::LAZY_IF_SUPPORTED + : ComputationMode::FULLY_MATERIALIZED); const auto& varToCols = join.getExternallyVisibleVariableColumns(); EXPECT_EQ(varToCols.size(), expected.size()); - const auto& table = res->idTable(); + if (expectLazinessParityWhenNonEmpty && + (!res->isFullyMaterialized() || !res->idTable().empty())) { + EXPECT_EQ(res->isFullyMaterialized(), !requestLaziness); + } + const auto& table = res->isFullyMaterialized() + ? res->idTable() + : static_cast( + aggregateTables(std::move(res->idTables()), + join.getResultWidth()) + .first); ASSERT_EQ(table.numColumns(), expected.size()); for (const auto& [var, columnAndStatus] : expected) { const auto& [colIndex, undefStatus] = varToCols.at(var); @@ -414,11 +430,20 @@ TEST(JoinTest, joinWithColumnAndScanUndefValues) { VariableToColumnMap expectedVariables{ {Variable{"?s"}, makeAlwaysDefinedColumn(0)}, {Variable{"?o"}, makeAlwaysDefinedColumn(1)}}; - testJoinOperation(join, makeExpectedColumns(expectedVariables, expected)); + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true, + materializationThreshold < 3); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); auto joinSwitched = Join{qec, valuesTree, fullScanPSO, 0, 0}; - testJoinOperation(joinSwitched, - makeExpectedColumns(expectedVariables, expected)); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, true, + materializationThreshold < 3); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, false); }; test(0); test(1); @@ -433,7 +458,6 @@ TEST(JoinTest, joinTwoScans) { "

1.

2. 3 . 4. 7. "); RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( materializationThreshold); - qec->getQueryTreeCache().clearAll(); auto scanP = ad_utility::makeExecutionTree( qec, PSO, SparqlTriple{Var{"?s"}, "

", Var{"?o"}}); auto scanP2 = ad_utility::makeExecutionTree( @@ -448,11 +472,20 @@ TEST(JoinTest, joinTwoScans) { {Variable{"?s"}, makeAlwaysDefinedColumn(0)}, {Variable{"?q"}, makeAlwaysDefinedColumn(1)}, {Variable{"?o"}, makeAlwaysDefinedColumn(2)}}; - testJoinOperation(join, makeExpectedColumns(expectedVariables, expected)); + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true, + materializationThreshold <= 3); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); auto joinSwitched = Join{qec, scanP2, scanP, 0, 0}; - testJoinOperation(joinSwitched, - makeExpectedColumns(expectedVariables, expected)); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, true, + materializationThreshold <= 3); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, false); }; test(0); test(1); @@ -469,3 +502,144 @@ TEST(JoinTest, invalidJoinVariable) { ASSERT_ANY_THROW(Join(qec, valuesTree2, valuesTree, 0, 0)); } + +// _____________________________________________________________________________ +TEST(JoinTest, joinTwoLazyOperationsWithAndWithoutUndefValues) { + auto performJoin = + [](std::vector leftTables, std::vector rightTables, + const IdTable& expected, bool expectPossiblyUndefinedResult, + ad_utility::source_location loc = + ad_utility::source_location::current()) { + auto l = generateLocationTrace(loc); + auto qec = ad_utility::testing::getQec(); + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); + auto leftTree = ad_utility::makeExecutionTree( + qec, std::move(leftTables), Vars{Variable{"?s"}}, false, + std::vector{0}); + auto rightTree = ad_utility::makeExecutionTree( + qec, std::move(rightTables), Vars{Variable{"?s"}}, false, + std::vector{0}); + VariableToColumnMap expectedVariables{ + {Variable{"?s"}, expectPossiblyUndefinedResult + ? makePossiblyUndefinedColumn(0) + : makeAlwaysDefinedColumn(0)}}; + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + auto join = Join{qec, leftTree, rightTree, 0, 0}; + EXPECT_EQ(join.getDescriptor(), "Join on ?s"); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); + + auto joinSwitched = Join{qec, rightTree, leftTree, 0, 0}; + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, true, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, false); + }; + auto U = Id::makeUndefined(); + std::vector leftTables; + std::vector rightTables; + IdTable expected1{1, ad_utility::makeUnlimitedAllocator()}; + performJoin(std::move(leftTables), std::move(rightTables), expected1, false); + + leftTables.push_back(makeIdTableFromVector({{U}})); + rightTables.push_back(makeIdTableFromVector({{U}})); + auto expected2 = makeIdTableFromVector({{U}}); + performJoin(std::move(leftTables), std::move(rightTables), expected2, true); + + leftTables.push_back(makeIdTableFromVector({{U}, {I(0)}})); + rightTables.push_back(makeIdTableFromVector({{U}})); + auto expected3 = makeIdTableFromVector({{U}, {I(0)}}); + performJoin(std::move(leftTables), std::move(rightTables), expected3, true); + + leftTables.push_back(makeIdTableFromVector({{U}, {I(0)}})); + leftTables.push_back(makeIdTableFromVector({{I(1)}})); + rightTables.push_back(makeIdTableFromVector({{I(0)}})); + auto expected4 = makeIdTableFromVector({{I(0)}, {I(0)}}); + performJoin(std::move(leftTables), std::move(rightTables), expected4, false); + + leftTables.push_back(makeIdTableFromVector({{U}, {I(0)}})); + leftTables.push_back(makeIdTableFromVector({{I(1)}})); + rightTables.push_back(IdTable{1, ad_utility::makeUnlimitedAllocator()}); + IdTable expected5{1, ad_utility::makeUnlimitedAllocator()}; + performJoin(std::move(leftTables), std::move(rightTables), expected5, false); + + leftTables.push_back(makeIdTableFromVector({{I(0)}})); + leftTables.push_back(makeIdTableFromVector({{I(1)}})); + rightTables.push_back(makeIdTableFromVector({{I(1)}})); + rightTables.push_back(makeIdTableFromVector({{I(2)}})); + auto expected6 = makeIdTableFromVector({{I(1)}}); + performJoin(std::move(leftTables), std::move(rightTables), expected6, false); +} + +// _____________________________________________________________________________ +TEST(JoinTest, joinLazyAndNonLazyOperationWithAndWithoutUndefValues) { + auto performJoin = [](IdTable leftTable, std::vector rightTables, + const IdTable& expected, + bool expectPossiblyUndefinedResult, + ad_utility::source_location loc = + ad_utility::source_location::current()) { + auto l = generateLocationTrace(loc); + auto qec = ad_utility::testing::getQec(); + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); + auto leftTree = + ad_utility::makeExecutionTree( + qec, std::move(leftTable), Vars{Variable{"?s"}}, false, + std::vector{0}, LocalVocab{}, std::nullopt, true); + auto rightTree = ad_utility::makeExecutionTree( + qec, std::move(rightTables), Vars{Variable{"?s"}}, false, + std::vector{0}); + VariableToColumnMap expectedVariables{ + {Variable{"?s"}, expectPossiblyUndefinedResult + ? makePossiblyUndefinedColumn(0) + : makeAlwaysDefinedColumn(0)}}; + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + auto join = Join{qec, leftTree, rightTree, 0, 0}; + EXPECT_EQ(join.getDescriptor(), "Join on ?s"); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); + + auto joinSwitched = Join{qec, rightTree, leftTree, 0, 0}; + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, false); + }; + auto U = Id::makeUndefined(); + std::vector rightTables; + rightTables.push_back(makeIdTableFromVector({{U}})); + auto expected1 = makeIdTableFromVector({{U}}); + performJoin(makeIdTableFromVector({{U}}), std::move(rightTables), expected1, + true); + + rightTables.push_back(makeIdTableFromVector({{U}})); + auto expected2 = makeIdTableFromVector({{U}, {I(0)}}); + performJoin(makeIdTableFromVector({{U}, {I(0)}}), std::move(rightTables), + expected2, true); + + rightTables.push_back(makeIdTableFromVector({{I(0)}})); + auto expected3 = makeIdTableFromVector({{I(0)}, {I(0)}}); + performJoin(makeIdTableFromVector({{U}, {I(0)}, {I(1)}}), + std::move(rightTables), expected3, false); + + rightTables.push_back(makeIdTableFromVector({{U}, {I(0)}})); + auto expected4 = makeIdTableFromVector({{I(0)}, {I(0)}, {I(1)}}); + performJoin(makeIdTableFromVector({{I(0)}, {I(1)}}), std::move(rightTables), + expected4, false); + + rightTables.push_back(IdTable{1, ad_utility::makeUnlimitedAllocator()}); + IdTable expected5{1, ad_utility::makeUnlimitedAllocator()}; + performJoin(makeIdTableFromVector({{U}, {I(0)}, {I(1)}}), + std::move(rightTables), expected5, false); + + rightTables.push_back(makeIdTableFromVector({{I(1)}})); + rightTables.push_back(makeIdTableFromVector({{I(2)}})); + auto expected6 = makeIdTableFromVector({{I(1)}}); + performJoin(makeIdTableFromVector({{I(0)}, {I(1)}}), std::move(rightTables), + expected6, false); +} From 711c3a79fcbbcbf88c459ca33e1f135fa889582c Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Mon, 18 Nov 2024 19:21:31 +0100 Subject: [PATCH 02/31] Properly reformat --- src/engine/Join.cpp | 299 +++++++++++++++++++++----------------------- 1 file changed, 146 insertions(+), 153 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 14ab366eee..7a1c65134b 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -459,73 +459,71 @@ bool Join::couldContainUndef(const auto& blocks, const auto& tree, ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, std::shared_ptr b, ColumnIndex jc2, bool requestLaziness) const { - return createResult( - requestLaziness, - [this, a = std::move(a), jc1, b = std::move(b), - jc2](std::invocable auto yieldTable) { - // If both inputs are fully materialized, we can join them more - // efficiently. - AD_CONTRACT_CHECK(!a->isFullyMaterialized() || - !b->isFullyMaterialized()); - auto joinColMap = ad_utility::JoinColumnMapping{ - {{jc1, jc2}}, - _left->getRootOperation()->getResultWidth(), - _right->getRootOperation()->getResultWidth()}; - ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, - [&yieldTable, &joinColMap](IdTable& idTable, - LocalVocab& localVocab) { - if (idTable.size() < CHUNK_SIZE) { - return; - } - idTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(idTable), - std::move(localVocab)}); - }}; - auto toBlockRange = []( - T& blockOrBlocks, - [[maybe_unused]] std::span - columnIndices = {}) - -> std::variant< - cppcoro::generator>, - std::array>, 1>> { - if constexpr (ad_utility::isSimilar) { - return std::array{ad_utility::IdTableAndFirstCol{ - blockOrBlocks.asColumnSubsetView(columnIndices)}}; - } else if constexpr (std::ranges::range) { - return convertGenerator(std::move(blockOrBlocks)); + return createResult(requestLaziness, [this, a = std::move(a), jc1, + b = std::move(b), + jc2](std::invocable< + Result::IdTableVocabPair> auto + yieldTable) { + // If both inputs are fully materialized, we can join them more + // efficiently. + AD_CONTRACT_CHECK(!a->isFullyMaterialized() || !b->isFullyMaterialized()); + auto joinColMap = ad_utility::JoinColumnMapping{ + {{jc1, jc2}}, + _left->getRootOperation()->getResultWidth(), + _right->getRootOperation()->getResultWidth()}; + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, + [&yieldTable, &joinColMap](IdTable& idTable, LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; + } + idTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(idTable), + std::move(localVocab)}); + }}; + auto toBlockRange = + []( + T& blockOrBlocks, + [[maybe_unused]] std::span columnIndices = {}) + -> std::variant< + cppcoro::generator>, + std::array>, 1>> { + if constexpr (ad_utility::isSimilar) { + return std::array{ad_utility::IdTableAndFirstCol{ + blockOrBlocks.asColumnSubsetView(columnIndices)}}; + } else if constexpr (std::ranges::range) { + return convertGenerator(std::move(blockOrBlocks)); + } else { + static_assert(ad_utility::alwaysFalse, "Unexpected type"); + } + }; + auto leftRange = + a->isFullyMaterialized() + ? toBlockRange(a->idTable(), joinColMap.permutationLeft()) + : toBlockRange(a->idTables()); + auto rightRange = + b->isFullyMaterialized() + ? toBlockRange(b->idTable(), joinColMap.permutationRight()) + : toBlockRange(b->idTables()); + std::visit( + [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { + bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || + couldContainUndef(rightBlocks, _right, jc2); + if (containsUndef) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + leftBlocks, rightBlocks, std::less{}, rowAdder); } else { - static_assert(ad_utility::alwaysFalse, "Unexpected type"); + ad_utility::zipperJoinForBlocksWithoutUndef(leftBlocks, rightBlocks, + std::less{}, rowAdder); } - }; - auto leftRange = - a->isFullyMaterialized() - ? toBlockRange(a->idTable(), joinColMap.permutationLeft()) - : toBlockRange(a->idTables()); - auto rightRange = - b->isFullyMaterialized() - ? toBlockRange(b->idTable(), joinColMap.permutationRight()) - : toBlockRange(b->idTables()); - std::visit( - [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { - bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || - couldContainUndef(rightBlocks, _right, jc2); - if (containsUndef) { - ad_utility::zipperJoinForBlocksWithPotentialUndef( - leftBlocks, rightBlocks, std::less{}, rowAdder); - } else { - ad_utility::zipperJoinForBlocksWithoutUndef( - leftBlocks, rightBlocks, std::less{}, rowAdder); - } - }, - leftRange, rightRange); - auto localVocab = std::move(rowAdder.localVocab()); - auto result = std::move(rowAdder).resultTable(); - result.setColumnSubset(joinColMap.permutationResult()); - return Result::IdTableVocabPair{std::move(result), - std::move(localVocab)}; - }); + }, + leftRange, rightRange); + auto localVocab = std::move(rowAdder.localVocab()); + auto result = std::move(rowAdder).resultTable(); + result.setColumnSubset(joinColMap.permutationResult()); + return Result::IdTableVocabPair{std::move(result), std::move(localVocab)}; + }); } // ______________________________________________________________________________ @@ -757,99 +755,94 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( bool requestLaziness, const IdTable& idTable, ColumnIndex joinColTable, std::shared_ptr scan, ColumnIndex joinColScan, const std::shared_ptr& subResult) const { - return createResult( - requestLaziness, - [this, &idTable, joinColTable, scan = std::move(scan), joinColScan, - subResult = std::move(subResult)]( - std::invocable auto yieldTable) { - // We first have to permute the columns. - auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { - return idTableIsRightInput - ? std::tuple{joinColScan, joinColTable, - scan->getResultWidth(), idTable.numColumns()} - : std::tuple{joinColTable, joinColScan, - idTable.numColumns(), scan->getResultWidth()}; - }(); - - auto joinColMap = ad_utility::JoinColumnMapping{ - {{jcLeft, jcRight}}, numColsLeft, numColsRight}; - ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, - [&yieldTable, &joinColMap](IdTable& idTable, - LocalVocab& localVocab) { - if (idTable.size() < CHUNK_SIZE) { - return; - } - idTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(idTable), - std::move(localVocab)}); - }}; - - AD_CORRECTNESS_CHECK(joinColScan == 0); - auto permutationIdTable = - ad_utility::IdTableAndFirstCol{idTable.asColumnSubsetView( - idTableIsRightInput ? joinColMap.permutationRight() - : joinColMap.permutationLeft())}; + return createResult(requestLaziness, [this, &idTable, joinColTable, + scan = std::move(scan), joinColScan, + subResult = std::move(subResult)]( + std::invocable< + Result::IdTableVocabPair> auto + yieldTable) { + // We first have to permute the columns. + auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { + return idTableIsRightInput + ? std::tuple{joinColScan, joinColTable, scan->getResultWidth(), + idTable.numColumns()} + : std::tuple{joinColTable, joinColScan, idTable.numColumns(), + scan->getResultWidth()}; + }(); - ad_utility::Timer timer{ - ad_utility::timer::Timer::InitialStatus::Started}; - bool idTableHasUndef = - !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); - std::optional> indexScanResult = - std::nullopt; - using FirstColView = ad_utility::IdTableAndFirstCol; - using GenWithDetails = - cppcoro::generator; - auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, - &indexScanResult]() - -> std::variant, GenWithDetails> { - if (idTableHasUndef) { - indexScanResult = - scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); - AD_CORRECTNESS_CHECK( - !indexScanResult.value()->isFullyMaterialized()); - return convertGenerator( - std::move(indexScanResult.value()->idTables())); - } else { - auto rightBlocksInternal = - scan->lazyScanForJoinOfColumnWithScan(permutationIdTable.col()); - return convertGenerator(std::move(rightBlocksInternal)); + auto joinColMap = ad_utility::JoinColumnMapping{ + {{jcLeft, jcRight}}, numColsLeft, numColsRight}; + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, + [&yieldTable, &joinColMap](IdTable& idTable, LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; } - }(); + idTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(idTable), + std::move(localVocab)}); + }}; + + AD_CORRECTNESS_CHECK(joinColScan == 0); + auto permutationIdTable = + ad_utility::IdTableAndFirstCol{idTable.asColumnSubsetView( + idTableIsRightInput ? joinColMap.permutationRight() + : joinColMap.permutationLeft())}; + + ad_utility::Timer timer{ad_utility::timer::Timer::InitialStatus::Started}; + bool idTableHasUndef = + !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); + std::optional> indexScanResult = std::nullopt; + using FirstColView = ad_utility::IdTableAndFirstCol; + using GenWithDetails = + cppcoro::generator; + auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, + &indexScanResult]() + -> std::variant, GenWithDetails> { + if (idTableHasUndef) { + indexScanResult = + scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); + AD_CORRECTNESS_CHECK(!indexScanResult.value()->isFullyMaterialized()); + return convertGenerator(std::move(indexScanResult.value()->idTables())); + } else { + auto rightBlocksInternal = + scan->lazyScanForJoinOfColumnWithScan(permutationIdTable.col()); + return convertGenerator(std::move(rightBlocksInternal)); + } + }(); - runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); - auto doJoin = [&rowAdder, idTableHasUndef](auto& left, - auto& right) mutable { - if (idTableHasUndef) { - ad_utility::zipperJoinForBlocksWithPotentialUndef( - left, right, std::less{}, rowAdder); + runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); + auto doJoin = [&rowAdder, idTableHasUndef](auto& left, + auto& right) mutable { + if (idTableHasUndef) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + left, right, std::less{}, rowAdder); + } else { + ad_utility::zipperJoinForBlocksWithoutUndef(left, right, std::less{}, + rowAdder); + } + }; + auto blockForIdTable = std::array{std::move(permutationIdTable)}; + std::visit( + [&doJoin, &blockForIdTable](auto& blocks) { + if constexpr (idTableIsRightInput) { + doJoin(blocks, blockForIdTable); } else { - ad_utility::zipperJoinForBlocksWithoutUndef(left, right, - std::less{}, rowAdder); + doJoin(blockForIdTable, blocks); } - }; - auto blockForIdTable = std::array{std::move(permutationIdTable)}; - std::visit( - [&doJoin, &blockForIdTable](auto& blocks) { - if constexpr (idTableIsRightInput) { - doJoin(blocks, blockForIdTable); - } else { - doJoin(blockForIdTable, blocks); - } - }, - rightBlocks); + }, + rightBlocks); - if (std::holds_alternative(rightBlocks)) { - updateRuntimeInfoForLazyScan( - *scan, std::get(rightBlocks).details()); - } + if (std::holds_alternative(rightBlocks)) { + updateRuntimeInfoForLazyScan( + *scan, std::get(rightBlocks).details()); + } - auto localVocab = std::move(rowAdder.localVocab()); - auto result = std::move(rowAdder).resultTable(); - result.setColumnSubset(joinColMap.permutationResult()); - return Result::IdTableVocabPair{std::move(result), - std::move(localVocab)}; - }); + auto localVocab = std::move(rowAdder.localVocab()); + auto result = std::move(rowAdder).resultTable(); + result.setColumnSubset(joinColMap.permutationResult()); + return Result::IdTableVocabPair{std::move(result), std::move(localVocab)}; + }); } From 08be2d8636e3c89813d8cd268a73211fb4bd2cd5 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Mon, 18 Nov 2024 22:05:21 +0100 Subject: [PATCH 03/31] Fix race condition --- src/engine/Join.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 7a1c65134b..c978d0a236 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -408,10 +408,14 @@ void Join::join(const IdTable& a, ColumnIndex jc1, const IdTable& b, // _____________________________________________________________________________ ProtoResult Join::createResult(bool requestedLaziness, auto action) const { if (requestedLaziness) { + auto startProcessing = std::make_shared(false); auto queue = std::make_shared< ad_utility::data_structures::ThreadSafeQueue>( 1); - ad_utility::JThread{[queue, action = std::move(action)]() { + ad_utility::JThread{[startProcessing, queue, action = std::move(action)]() { + // Don't start processing until the main thread has reached the generator + // to avoid race conditions. + startProcessing->wait(false); auto addValue = [&queue](Result::IdTableVocabPair value) { if (value.idTable_.empty()) { return; @@ -425,7 +429,9 @@ ProtoResult Join::createResult(bool requestedLaziness, auto action) const { queue->pushException(std::current_exception()); } }}.detach(); - return {[](auto queue) -> Result::Generator { + return {[](auto queue, auto startProcessing) -> Result::Generator { + startProcessing->test_and_set(); + startProcessing->notify_one(); while (true) { auto val = queue->pop(); if (!val.has_value()) { @@ -433,7 +439,7 @@ ProtoResult Join::createResult(bool requestedLaziness, auto action) const { } co_yield val.value(); } - }(std::move(queue)), + }(std::move(queue), std::move(startProcessing)), resultSortedOn()}; } else { auto [idTable, localVocab] = action(ad_utility::noop); From c9a5648521be84c99721feec0f5a2faebddbbcb4 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Tue, 19 Nov 2024 00:35:43 +0100 Subject: [PATCH 04/31] Fix lifetime issues and add documentation --- src/engine/Join.cpp | 28 +++++++++++++++------------- src/engine/Join.h | 21 +++++++++++++++++---- 2 files changed, 32 insertions(+), 17 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index c978d0a236..4b184eae7e 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -142,7 +142,7 @@ ProtoResult Join::computeResult(bool requestLaziness) { if (rightResIfCached && !leftResIfCached) { AD_CORRECTNESS_CHECK(rightResIfCached->isFullyMaterialized()); return computeResultForIndexScanAndIdTable( - requestLaziness, rightResIfCached->idTable(), _rightJoinCol, + requestLaziness, std::move(rightResIfCached), _rightJoinCol, leftIndexScan, _leftJoinCol); } else if (!leftResIfCached) { @@ -172,10 +172,9 @@ ProtoResult Join::computeResult(bool requestLaziness) { auto rightIndexScan = std::dynamic_pointer_cast(_right->getRootOperation()); if (rightIndexScan && !rightResIfCached && leftRes->isFullyMaterialized()) { - const auto& leftIdTable = leftRes->idTable(); return computeResultForIndexScanAndIdTable( - requestLaziness, leftIdTable, _leftJoinCol, rightIndexScan, - _rightJoinCol, leftRes); + requestLaziness, std::move(leftRes), _leftJoinCol, rightIndexScan, + _rightJoinCol); } std::shared_ptr rightRes = @@ -758,15 +757,17 @@ ProtoResult Join::computeResultForTwoIndexScans(bool requestLaziness) const { // ______________________________________________________________________________________________________ template ProtoResult Join::computeResultForIndexScanAndIdTable( - bool requestLaziness, const IdTable& idTable, ColumnIndex joinColTable, - std::shared_ptr scan, ColumnIndex joinColScan, - const std::shared_ptr& subResult) const { - return createResult(requestLaziness, [this, &idTable, joinColTable, + bool requestLaziness, std::shared_ptr resultWithIdTable, + ColumnIndex joinColTable, std::shared_ptr scan, + ColumnIndex joinColScan) const { + return createResult(requestLaziness, [this, joinColTable, scan = std::move(scan), joinColScan, - subResult = std::move(subResult)]( + resultWithIdTable = + std::move(resultWithIdTable)]( std::invocable< Result::IdTableVocabPair> auto yieldTable) { + const IdTable& idTable = resultWithIdTable->idTable(); // We first have to permute the columns. auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { return idTableIsRightInput @@ -781,12 +782,13 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( ad_utility::AddCombinedRowToIdTable rowAdder{ 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, cancellationHandle_, CHUNK_SIZE, - [&yieldTable, &joinColMap](IdTable& idTable, LocalVocab& localVocab) { - if (idTable.size() < CHUNK_SIZE) { + [&yieldTable, &joinColMap](IdTable& partialIdTable, + LocalVocab& localVocab) { + if (partialIdTable.size() < CHUNK_SIZE) { return; } - idTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(idTable), + partialIdTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(partialIdTable), std::move(localVocab)}); }}; diff --git a/src/engine/Join.h b/src/engine/Join.h index d82bda849f..73bc4e86fd 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -94,13 +94,26 @@ class Join : public Operation { void join(const IdTable& a, ColumnIndex jc1, const IdTable& b, ColumnIndex jc2, IdTable* result) const; - // action is a lambda with signature + // Helper function to compute the result of a join operation and conditionally + // return a lazy or fully materialized result depending on `requestLaziness`. + // This is achieved by running the `action` lambda in a separate thread and + // returning a lazy result that reads from the queue of the thread. If + // `requestLaziness` is false, the result is fully materialized and returned + // directly. + // `action` is a lambda with signature // Result::IdTableVocabPair(void(Result::IdTableVocabPair)) ProtoResult createResult(bool requestedLaziness, auto action) const; + // Helper function that cheaply checks if a join could contain undefined. For + // fully materialized tables it can just look at the first element. For lazy + // tables it has to look at the meta information which could potentially + // indicate undefinedness even when all values are defined. static bool couldContainUndef(const auto& blocks, const auto& tree, ColumnIndex joinColumn); + // Fallback implementation of a join that is used when at least one of the two + // inputs is not fully materialized. This represents the general case where we + // don't have any optimization left to try. ProtoResult lazyJoin(std::shared_ptr a, ColumnIndex jc1, std::shared_ptr b, ColumnIndex jc2, bool requestLaziness) const; @@ -140,9 +153,9 @@ class Join : public Operation { // determine the correct order of the columns in the result. template ProtoResult computeResultForIndexScanAndIdTable( - bool requestLaziness, const IdTable& idTable, ColumnIndex joinColTable, - std::shared_ptr scan, ColumnIndex joinColScan, - const std::shared_ptr& subResult = nullptr) const; + bool requestLaziness, std::shared_ptr resultWithIdTable, + ColumnIndex joinColTable, std::shared_ptr scan, + ColumnIndex joinColScan) const; /* * @brief Combines 2 rows like in a join and inserts the result in the From bc659c68d0c20b978d685250b73680d2cc9b5c6e Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Tue, 19 Nov 2024 16:42:28 +0100 Subject: [PATCH 05/31] Properly reset runtime parameter after test --- test/JoinTest.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index a9ec32b523..b48b754066 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -584,6 +584,11 @@ TEST(JoinTest, joinLazyAndNonLazyOperationWithAndWithoutUndefValues) { auto l = generateLocationTrace(loc); auto qec = ad_utility::testing::getQec(); RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); + absl::Cleanup cleanup{[]() { + // Reset back to original value to not influence other tests. + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( + 1'000'000); + }}; auto leftTree = ad_utility::makeExecutionTree( qec, std::move(leftTable), Vars{Variable{"?s"}}, false, From 112a7576b190105dc5a492f32e352285da02725c Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Tue, 19 Nov 2024 17:49:45 +0100 Subject: [PATCH 06/31] Properly use `LocalVocab`s from fully materialized results --- src/engine/Join.cpp | 40 ++++++++++----------- src/util/JoinAlgorithms/JoinColumnMapping.h | 1 - 2 files changed, 19 insertions(+), 22 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 4b184eae7e..f7b46997b0 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -487,30 +487,26 @@ ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, yieldTable(Result::IdTableVocabPair{std::move(idTable), std::move(localVocab)}); }}; - auto toBlockRange = - []( - T& blockOrBlocks, - [[maybe_unused]] std::span columnIndices = {}) + auto asSingleTableView = [](const Result& result, + const std::vector& permutation) { + return std::array{ad_utility::IdTableAndFirstCol{ + result.idTable().asColumnSubsetView(permutation), + result.getCopyOfLocalVocab()}}; + }; + auto toVariant = [](T variantObject) -> std::variant< cppcoro::generator>, std::array>, 1>> { - if constexpr (ad_utility::isSimilar) { - return std::array{ad_utility::IdTableAndFirstCol{ - blockOrBlocks.asColumnSubsetView(columnIndices)}}; - } else if constexpr (std::ranges::range) { - return convertGenerator(std::move(blockOrBlocks)); - } else { - static_assert(ad_utility::alwaysFalse, "Unexpected type"); - } + return variantObject; }; auto leftRange = a->isFullyMaterialized() - ? toBlockRange(a->idTable(), joinColMap.permutationLeft()) - : toBlockRange(a->idTables()); + ? toVariant(asSingleTableView(*a, joinColMap.permutationLeft())) + : toVariant(convertGenerator(std::move(a->idTables()))); auto rightRange = b->isFullyMaterialized() - ? toBlockRange(b->idTable(), joinColMap.permutationRight()) - : toBlockRange(b->idTables()); + ? toVariant(asSingleTableView(*b, joinColMap.permutationRight())) + : toVariant(convertGenerator(std::move(b->idTables()))); std::visit( [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || @@ -671,7 +667,8 @@ convertGenerator(Permutation::IdTableGenerator gen) { co_await cppcoro::getDetails = gen.details(); gen.setDetailsPointer(&co_await cppcoro::getDetails); for (auto& table : gen) { - ad_utility::IdTableAndFirstCol t{std::move(table)}; + // IndexScans don't have a local vocabulary, so we can just use an empty one + ad_utility::IdTableAndFirstCol t{std::move(table), LocalVocab{}}; co_yield t; } } @@ -793,10 +790,11 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( }}; AD_CORRECTNESS_CHECK(joinColScan == 0); - auto permutationIdTable = - ad_utility::IdTableAndFirstCol{idTable.asColumnSubsetView( - idTableIsRightInput ? joinColMap.permutationRight() - : joinColMap.permutationLeft())}; + auto permutationIdTable = ad_utility::IdTableAndFirstCol{ + idTable.asColumnSubsetView(idTableIsRightInput + ? joinColMap.permutationRight() + : joinColMap.permutationLeft()), + resultWithIdTable->getCopyOfLocalVocab()}; ad_utility::Timer timer{ad_utility::timer::Timer::InitialStatus::Started}; bool idTableHasUndef = diff --git a/src/util/JoinAlgorithms/JoinColumnMapping.h b/src/util/JoinAlgorithms/JoinColumnMapping.h index 110256bc8a..26d9be15db 100644 --- a/src/util/JoinAlgorithms/JoinColumnMapping.h +++ b/src/util/JoinAlgorithms/JoinColumnMapping.h @@ -118,7 +118,6 @@ struct IdTableAndFirstCol { std::decay_t; // Construct by taking ownership of the table. - explicit IdTableAndFirstCol(Table t) : table_{std::move(t)}, localVocab_{} {} IdTableAndFirstCol(Table t, LocalVocab localVocab) : table_{std::move(t)}, localVocab_{std::move(localVocab)} {} From d091bbad613084e84454836862ee7e6dba0b093e Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Tue, 19 Nov 2024 21:05:16 +0100 Subject: [PATCH 07/31] Properly apply permutation --- src/engine/Join.cpp | 44 ++++++++++++++++++++++++++++++-------------- test/JoinTest.cpp | 5 +++++ 2 files changed, 35 insertions(+), 14 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index f7b46997b0..28d2fa90c5 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -28,6 +28,19 @@ using std::string; namespace { // Convert a `generator` to a `generator` for +// more efficient access in the join columns below and apply the given +// permutation to each table. +cppcoro::generator> convertGenerator( + Result::Generator gen, std::vector permutation) { + for (auto& [table, localVocab] : gen) { + // Make sure to actually move the table into the wrapper so that the tables + // live as long as the wrapper. + table.setColumnSubset(permutation); + ad_utility::IdTableAndFirstCol t{std::move(table), std::move(localVocab)}; + co_yield t; + } +} +// Convert a `generator` to a `generator` for // more efficient access in the join columns below. cppcoro::generator> convertGenerator( Result::Generator gen) { @@ -483,8 +496,10 @@ ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, if (idTable.size() < CHUNK_SIZE) { return; } - idTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(idTable), + // Don't modify the columns of the reference. + IdTable movedTable = std::move(idTable); + movedTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(movedTable), std::move(localVocab)}); }}; auto asSingleTableView = [](const Result& result, @@ -493,20 +508,19 @@ ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, result.idTable().asColumnSubsetView(permutation), result.getCopyOfLocalVocab()}}; }; - auto toVariant = [](T variantObject) - -> std::variant< - cppcoro::generator>, - std::array>, 1>> { - return variantObject; - }; + using Blocks = std::variant< + cppcoro::generator>, + std::array>, 1>>; auto leftRange = a->isFullyMaterialized() - ? toVariant(asSingleTableView(*a, joinColMap.permutationLeft())) - : toVariant(convertGenerator(std::move(a->idTables()))); + ? Blocks{asSingleTableView(*a, joinColMap.permutationLeft())} + : Blocks{convertGenerator(std::move(a->idTables()), + joinColMap.permutationLeft())}; auto rightRange = b->isFullyMaterialized() - ? toVariant(asSingleTableView(*b, joinColMap.permutationRight())) - : toVariant(convertGenerator(std::move(b->idTables()))); + ? Blocks{asSingleTableView(*b, joinColMap.permutationRight())} + : Blocks{convertGenerator(std::move(b->idTables()), + joinColMap.permutationRight())}; std::visit( [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || @@ -784,8 +798,10 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( if (partialIdTable.size() < CHUNK_SIZE) { return; } - partialIdTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(partialIdTable), + // Don't modify the columns of the reference. + IdTable movedTable = std::move(partialIdTable); + movedTable.setColumnSubset(joinColMap.permutationResult()); + yieldTable(Result::IdTableVocabPair{std::move(movedTable), std::move(localVocab)}); }}; diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index b48b754066..5c5bdb70f0 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -513,6 +513,11 @@ TEST(JoinTest, joinTwoLazyOperationsWithAndWithoutUndefValues) { auto l = generateLocationTrace(loc); auto qec = ad_utility::testing::getQec(); RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); + absl::Cleanup cleanup{[]() { + // Reset back to original value to not influence other tests. + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( + 1'000'000); + }}; auto leftTree = ad_utility::makeExecutionTree( qec, std::move(leftTables), Vars{Variable{"?s"}}, false, std::vector{0}); From f5859876b25690cf6383b71295313148b4a86057 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 13:34:45 +0100 Subject: [PATCH 08/31] Spawn thread only if generator is consumed --- src/engine/Join.cpp | 58 ++++++++++++++++++++------------------------- 1 file changed, 26 insertions(+), 32 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 28d2fa90c5..249e81b540 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -420,39 +420,33 @@ void Join::join(const IdTable& a, ColumnIndex jc1, const IdTable& b, // _____________________________________________________________________________ ProtoResult Join::createResult(bool requestedLaziness, auto action) const { if (requestedLaziness) { - auto startProcessing = std::make_shared(false); - auto queue = std::make_shared< - ad_utility::data_structures::ThreadSafeQueue>( - 1); - ad_utility::JThread{[startProcessing, queue, action = std::move(action)]() { - // Don't start processing until the main thread has reached the generator - // to avoid race conditions. - startProcessing->wait(false); - auto addValue = [&queue](Result::IdTableVocabPair value) { - if (value.idTable_.empty()) { - return; - } - queue->push(std::move(value)); - }; - try { - addValue(action(addValue)); - queue->finish(); - } catch (...) { - queue->pushException(std::current_exception()); - } - }}.detach(); - return {[](auto queue, auto startProcessing) -> Result::Generator { - startProcessing->test_and_set(); - startProcessing->notify_one(); - while (true) { - auto val = queue->pop(); - if (!val.has_value()) { - break; - } - co_yield val.value(); + return { + [](auto innerAction) -> Result::Generator { + ad_utility::data_structures::ThreadSafeQueue + queue{1}; + ad_utility::JThread{[&queue, &innerAction]() { + auto addValue = [&queue](Result::IdTableVocabPair value) { + if (value.idTable_.empty()) { + return; } - }(std::move(queue), std::move(startProcessing)), - resultSortedOn()}; + queue.push(std::move(value)); + }; + try { + addValue(innerAction(addValue)); + queue.finish(); + } catch (...) { + queue.pushException(std::current_exception()); + } + }}; + while (true) { + auto val = queue.pop(); + if (!val.has_value()) { + break; + } + co_yield val.value(); + } + }(std::move(action)), + resultSortedOn()}; } else { auto [idTable, localVocab] = action(ad_utility::noop); return {std::move(idTable), resultSortedOn(), std::move(localVocab)}; From 8ecc82113b51a5fbe8520dbc50b4d01c15398610 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 14:05:01 +0100 Subject: [PATCH 09/31] Fix issue with join aggregation --- src/engine/Join.cpp | 335 ++++++++++++++++++++++---------------------- src/engine/Join.h | 14 +- test/JoinTest.cpp | 19 +++ 3 files changed, 197 insertions(+), 171 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 249e81b540..991c55b1f1 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -418,21 +418,38 @@ void Join::join(const IdTable& a, ColumnIndex jc1, const IdTable& b, } // _____________________________________________________________________________ -ProtoResult Join::createResult(bool requestedLaziness, auto action) const { +ProtoResult Join::createResult( + bool requestedLaziness, + ad_utility::InvocableWithExactReturnType< + Result::IdTableVocabPair, + std::function> auto action, + std::vector permutation) const { if (requestedLaziness) { return { - [](auto innerAction) -> Result::Generator { + [](auto innerAction, auto innerPermutation) -> Result::Generator { ad_utility::data_structures::ThreadSafeQueue queue{1}; - ad_utility::JThread{[&queue, &innerAction]() { - auto addValue = [&queue](Result::IdTableVocabPair value) { - if (value.idTable_.empty()) { + ad_utility::JThread{[&queue, &innerAction, &innerPermutation]() { + auto addValue = [&queue, &innerPermutation]( + IdTable& idTable, LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { return; } - queue.push(std::move(value)); + Result::IdTableVocabPair pair{std::move(idTable), + std::move(localVocab)}; + if (!innerPermutation.empty()) { + pair.idTable_.setColumnSubset(innerPermutation); + } + queue.push(std::move(pair)); }; try { - addValue(innerAction(addValue)); + auto finalValue = innerAction(addValue); + if (!finalValue.idTable_.empty()) { + if (!innerPermutation.empty()) { + finalValue.idTable_.setColumnSubset(innerPermutation); + } + queue.push(std::move(finalValue)); + } queue.finish(); } catch (...) { queue.pushException(std::current_exception()); @@ -445,10 +462,13 @@ ProtoResult Join::createResult(bool requestedLaziness, auto action) const { } co_yield val.value(); } - }(std::move(action)), + }(std::move(action), std::move(permutation)), resultSortedOn()}; } else { auto [idTable, localVocab] = action(ad_utility::noop); + if (!permutation.empty()) { + idTable.setColumnSubset(permutation); + } return {std::move(idTable), resultSortedOn(), std::move(localVocab)}; } } @@ -471,68 +491,60 @@ bool Join::couldContainUndef(const auto& blocks, const auto& tree, ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, std::shared_ptr b, ColumnIndex jc2, bool requestLaziness) const { - return createResult(requestLaziness, [this, a = std::move(a), jc1, - b = std::move(b), - jc2](std::invocable< - Result::IdTableVocabPair> auto - yieldTable) { - // If both inputs are fully materialized, we can join them more - // efficiently. - AD_CONTRACT_CHECK(!a->isFullyMaterialized() || !b->isFullyMaterialized()); - auto joinColMap = ad_utility::JoinColumnMapping{ - {{jc1, jc2}}, - _left->getRootOperation()->getResultWidth(), - _right->getRootOperation()->getResultWidth()}; - ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, - [&yieldTable, &joinColMap](IdTable& idTable, LocalVocab& localVocab) { - if (idTable.size() < CHUNK_SIZE) { - return; - } - // Don't modify the columns of the reference. - IdTable movedTable = std::move(idTable); - movedTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(movedTable), - std::move(localVocab)}); - }}; - auto asSingleTableView = [](const Result& result, - const std::vector& permutation) { - return std::array{ad_utility::IdTableAndFirstCol{ - result.idTable().asColumnSubsetView(permutation), - result.getCopyOfLocalVocab()}}; - }; - using Blocks = std::variant< - cppcoro::generator>, - std::array>, 1>>; - auto leftRange = - a->isFullyMaterialized() - ? Blocks{asSingleTableView(*a, joinColMap.permutationLeft())} - : Blocks{convertGenerator(std::move(a->idTables()), - joinColMap.permutationLeft())}; - auto rightRange = - b->isFullyMaterialized() - ? Blocks{asSingleTableView(*b, joinColMap.permutationRight())} - : Blocks{convertGenerator(std::move(b->idTables()), - joinColMap.permutationRight())}; - std::visit( - [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { - bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || - couldContainUndef(rightBlocks, _right, jc2); - if (containsUndef) { - ad_utility::zipperJoinForBlocksWithPotentialUndef( - leftBlocks, rightBlocks, std::less{}, rowAdder); - } else { - ad_utility::zipperJoinForBlocksWithoutUndef(leftBlocks, rightBlocks, - std::less{}, rowAdder); - } - }, - leftRange, rightRange); - auto localVocab = std::move(rowAdder.localVocab()); - auto result = std::move(rowAdder).resultTable(); - result.setColumnSubset(joinColMap.permutationResult()); - return Result::IdTableVocabPair{std::move(result), std::move(localVocab)}; - }); + // If both inputs are fully materialized, we can join them more + // efficiently. + AD_CONTRACT_CHECK(!a->isFullyMaterialized() || !b->isFullyMaterialized()); + ad_utility::JoinColumnMapping joinColMap{ + {{jc1, jc2}}, + _left->getRootOperation()->getResultWidth(), + _right->getRootOperation()->getResultWidth()}; + auto resultPermutation = joinColMap.permutationResult(); + return createResult( + requestLaziness, + [this, a = std::move(a), jc1, b = std::move(b), jc2, + joinColMap = std::move(joinColMap)]( + std::function yieldTable) { + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; + auto asSingleTableView = + [](const Result& result, + const std::vector& permutation) { + return std::array{ad_utility::IdTableAndFirstCol{ + result.idTable().asColumnSubsetView(permutation), + result.getCopyOfLocalVocab()}}; + }; + using Blocks = std::variant< + cppcoro::generator>, + std::array>, 1>>; + auto leftRange = + a->isFullyMaterialized() + ? Blocks{asSingleTableView(*a, joinColMap.permutationLeft())} + : Blocks{convertGenerator(std::move(a->idTables()), + joinColMap.permutationLeft())}; + auto rightRange = + b->isFullyMaterialized() + ? Blocks{asSingleTableView(*b, joinColMap.permutationRight())} + : Blocks{convertGenerator(std::move(b->idTables()), + joinColMap.permutationRight())}; + std::visit( + [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { + bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || + couldContainUndef(rightBlocks, _right, jc2); + if (containsUndef) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + leftBlocks, rightBlocks, std::less{}, rowAdder); + } else { + ad_utility::zipperJoinForBlocksWithoutUndef( + leftBlocks, rightBlocks, std::less{}, rowAdder); + } + }, + leftRange, rightRange); + auto localVocab = std::move(rowAdder.localVocab()); + return Result::IdTableVocabPair{std::move(rowAdder).resultTable(), + std::move(localVocab)}; + }, + std::move(resultPermutation)); } // ______________________________________________________________________________ @@ -713,7 +725,7 @@ void updateRuntimeInfoForLazyScan( ProtoResult Join::computeResultForTwoIndexScans(bool requestLaziness) const { return createResult( requestLaziness, - [this](std::invocable auto yieldTable) { + [this](std::function yieldTable) { auto leftScan = std::dynamic_pointer_cast(_left->getRootOperation()); auto rightScan = @@ -725,14 +737,7 @@ ProtoResult Join::computeResultForTwoIndexScans(bool requestLaziness) const { AD_CORRECTNESS_CHECK(_leftJoinCol == 0 && _rightJoinCol == 0); ad_utility::AddCombinedRowToIdTable rowAdder{ 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, - [&yieldTable](IdTable& idTable, LocalVocab& localVocab) { - if (idTable.size() < CHUNK_SIZE) { - return; - } - yieldTable(Result::IdTableVocabPair{std::move(idTable), - std::move(localVocab)}); - }}; + cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; ad_utility::Timer timer{ ad_utility::timer::Timer::InitialStatus::Started}; @@ -765,100 +770,94 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( bool requestLaziness, std::shared_ptr resultWithIdTable, ColumnIndex joinColTable, std::shared_ptr scan, ColumnIndex joinColScan) const { - return createResult(requestLaziness, [this, joinColTable, - scan = std::move(scan), joinColScan, - resultWithIdTable = - std::move(resultWithIdTable)]( - std::invocable< - Result::IdTableVocabPair> auto - yieldTable) { - const IdTable& idTable = resultWithIdTable->idTable(); - // We first have to permute the columns. - auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { - return idTableIsRightInput - ? std::tuple{joinColScan, joinColTable, scan->getResultWidth(), - idTable.numColumns()} - : std::tuple{joinColTable, joinColScan, idTable.numColumns(), - scan->getResultWidth()}; - }(); + const IdTable& idTable = resultWithIdTable->idTable(); + // We first have to permute the columns. + auto [jcLeft, jcRight, numColsLeft, numColsRight] = [&]() { + return idTableIsRightInput + ? std::tuple{joinColScan, joinColTable, scan->getResultWidth(), + idTable.numColumns()} + : std::tuple{joinColTable, joinColScan, idTable.numColumns(), + scan->getResultWidth()}; + }(); + + ad_utility::JoinColumnMapping joinColMap{ + {{jcLeft, jcRight}}, numColsLeft, numColsRight}; + auto resultPermutation = joinColMap.permutationResult(); + return createResult( + requestLaziness, + [this, joinColTable, scan = std::move(scan), joinColScan, + resultWithIdTable = std::move(resultWithIdTable), + joinColMap = std::move(joinColMap)]( + std::function yieldTable) { + const IdTable& idTable = resultWithIdTable->idTable(); + ad_utility::AddCombinedRowToIdTable rowAdder{ + 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, + cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; + + AD_CORRECTNESS_CHECK(joinColScan == 0); + auto permutationIdTable = ad_utility::IdTableAndFirstCol{ + idTable.asColumnSubsetView(idTableIsRightInput + ? joinColMap.permutationRight() + : joinColMap.permutationLeft()), + resultWithIdTable->getCopyOfLocalVocab()}; - auto joinColMap = ad_utility::JoinColumnMapping{ - {{jcLeft, jcRight}}, numColsLeft, numColsRight}; - ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, - [&yieldTable, &joinColMap](IdTable& partialIdTable, - LocalVocab& localVocab) { - if (partialIdTable.size() < CHUNK_SIZE) { - return; + ad_utility::Timer timer{ + ad_utility::timer::Timer::InitialStatus::Started}; + bool idTableHasUndef = + !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); + std::optional> indexScanResult = + std::nullopt; + using FirstColView = ad_utility::IdTableAndFirstCol; + using GenWithDetails = + cppcoro::generator; + auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, + &indexScanResult]() + -> std::variant, GenWithDetails> { + if (idTableHasUndef) { + indexScanResult = + scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); + AD_CORRECTNESS_CHECK( + !indexScanResult.value()->isFullyMaterialized()); + return convertGenerator( + std::move(indexScanResult.value()->idTables())); + } else { + auto rightBlocksInternal = + scan->lazyScanForJoinOfColumnWithScan(permutationIdTable.col()); + return convertGenerator(std::move(rightBlocksInternal)); } - // Don't modify the columns of the reference. - IdTable movedTable = std::move(partialIdTable); - movedTable.setColumnSubset(joinColMap.permutationResult()); - yieldTable(Result::IdTableVocabPair{std::move(movedTable), - std::move(localVocab)}); - }}; - - AD_CORRECTNESS_CHECK(joinColScan == 0); - auto permutationIdTable = ad_utility::IdTableAndFirstCol{ - idTable.asColumnSubsetView(idTableIsRightInput - ? joinColMap.permutationRight() - : joinColMap.permutationLeft()), - resultWithIdTable->getCopyOfLocalVocab()}; - - ad_utility::Timer timer{ad_utility::timer::Timer::InitialStatus::Started}; - bool idTableHasUndef = - !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); - std::optional> indexScanResult = std::nullopt; - using FirstColView = ad_utility::IdTableAndFirstCol; - using GenWithDetails = - cppcoro::generator; - auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, - &indexScanResult]() - -> std::variant, GenWithDetails> { - if (idTableHasUndef) { - indexScanResult = - scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); - AD_CORRECTNESS_CHECK(!indexScanResult.value()->isFullyMaterialized()); - return convertGenerator(std::move(indexScanResult.value()->idTables())); - } else { - auto rightBlocksInternal = - scan->lazyScanForJoinOfColumnWithScan(permutationIdTable.col()); - return convertGenerator(std::move(rightBlocksInternal)); - } - }(); + }(); - runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); - auto doJoin = [&rowAdder, idTableHasUndef](auto& left, - auto& right) mutable { - if (idTableHasUndef) { - ad_utility::zipperJoinForBlocksWithPotentialUndef( - left, right, std::less{}, rowAdder); - } else { - ad_utility::zipperJoinForBlocksWithoutUndef(left, right, std::less{}, - rowAdder); - } - }; - auto blockForIdTable = std::array{std::move(permutationIdTable)}; - std::visit( - [&doJoin, &blockForIdTable](auto& blocks) { - if constexpr (idTableIsRightInput) { - doJoin(blocks, blockForIdTable); + runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); + auto doJoin = [&rowAdder, idTableHasUndef](auto& left, + auto& right) mutable { + if (idTableHasUndef) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + left, right, std::less{}, rowAdder); } else { - doJoin(blockForIdTable, blocks); + ad_utility::zipperJoinForBlocksWithoutUndef(left, right, + std::less{}, rowAdder); } - }, - rightBlocks); + }; + auto blockForIdTable = std::array{std::move(permutationIdTable)}; + std::visit( + [&doJoin, &blockForIdTable](auto& blocks) { + if constexpr (idTableIsRightInput) { + doJoin(blocks, blockForIdTable); + } else { + doJoin(blockForIdTable, blocks); + } + }, + rightBlocks); - if (std::holds_alternative(rightBlocks)) { - updateRuntimeInfoForLazyScan( - *scan, std::get(rightBlocks).details()); - } + if (std::holds_alternative(rightBlocks)) { + updateRuntimeInfoForLazyScan( + *scan, std::get(rightBlocks).details()); + } - auto localVocab = std::move(rowAdder.localVocab()); - auto result = std::move(rowAdder).resultTable(); - result.setColumnSubset(joinColMap.permutationResult()); - return Result::IdTableVocabPair{std::move(result), std::move(localVocab)}; - }); + auto localVocab = std::move(rowAdder.localVocab()); + return Result::IdTableVocabPair{std::move(rowAdder).resultTable(), + std::move(localVocab)}; + }, + std::move(resultPermutation)); } diff --git a/src/engine/Join.h b/src/engine/Join.h index 73bc4e86fd..5ba01e55b8 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -100,9 +100,17 @@ class Join : public Operation { // returning a lazy result that reads from the queue of the thread. If // `requestLaziness` is false, the result is fully materialized and returned // directly. - // `action` is a lambda with signature - // Result::IdTableVocabPair(void(Result::IdTableVocabPair)) - ProtoResult createResult(bool requestedLaziness, auto action) const; + // `permutation` indicates a permutation to apply to the result columns before + // yielding/returning them. An empty vector means no permutation is applied. + // `action` is a lambda that can be used to send partial chunks to a consumer + // in addition to returning the remaining result. If lazyness is not required + // it is a no-op. + ProtoResult createResult( + bool requestedLaziness, + ad_utility::InvocableWithExactReturnType< + Result::IdTableVocabPair, + std::function> auto action, + std::vector permutation = {}) const; // Helper function that cheaply checks if a join could contain undefined. For // fully materialized tables it can just look at the first element. For lazy diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index 5c5bdb70f0..f3e5a25edb 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -208,6 +208,13 @@ std::vector createJoinTestSet() { return myTestSet; } + +IdTable createIdTableOfSizeWithValue(size_t size, Id value) { + IdTable idTable{1, ad_utility::testing::makeAllocator()}; + idTable.resize(size); + std::ranges::fill(idTable.getColumn(0), value); + return idTable; +} } // namespace TEST(JoinTest, joinTest) { @@ -577,6 +584,12 @@ TEST(JoinTest, joinTwoLazyOperationsWithAndWithoutUndefValues) { rightTables.push_back(makeIdTableFromVector({{I(2)}})); auto expected6 = makeIdTableFromVector({{I(1)}}); performJoin(std::move(leftTables), std::move(rightTables), expected6, false); + + leftTables.push_back(makeIdTableFromVector({{U}})); + leftTables.push_back(makeIdTableFromVector({{I(2)}})); + rightTables.push_back(createIdTableOfSizeWithValue(Join::CHUNK_SIZE, I(1))); + auto expected7 = createIdTableOfSizeWithValue(Join::CHUNK_SIZE, I(1)); + performJoin(std::move(leftTables), std::move(rightTables), expected7, false); } // _____________________________________________________________________________ @@ -652,4 +665,10 @@ TEST(JoinTest, joinLazyAndNonLazyOperationWithAndWithoutUndefValues) { auto expected6 = makeIdTableFromVector({{I(1)}}); performJoin(makeIdTableFromVector({{I(0)}, {I(1)}}), std::move(rightTables), expected6, false); + + rightTables.push_back(makeIdTableFromVector({{U}})); + rightTables.push_back(makeIdTableFromVector({{I(2)}})); + auto expected7 = createIdTableOfSizeWithValue(Join::CHUNK_SIZE, I(1)); + performJoin(createIdTableOfSizeWithValue(Join::CHUNK_SIZE, I(1)), + std::move(rightTables), expected7, false); } From 190b8dec09e06407df67b46f7c386c5cc4c744e4 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 14:08:18 +0100 Subject: [PATCH 10/31] Fix typo --- src/engine/Join.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/engine/Join.h b/src/engine/Join.h index 5ba01e55b8..8654d039cc 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -103,7 +103,7 @@ class Join : public Operation { // `permutation` indicates a permutation to apply to the result columns before // yielding/returning them. An empty vector means no permutation is applied. // `action` is a lambda that can be used to send partial chunks to a consumer - // in addition to returning the remaining result. If lazyness is not required + // in addition to returning the remaining result. If laziness is not required // it is a no-op. ProtoResult createResult( bool requestedLaziness, From 3bbc4662ac0385dadff373648c73ea9f56fd0cf8 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 14:15:58 +0100 Subject: [PATCH 11/31] Unify two generators --- src/engine/Join.cpp | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 991c55b1f1..69a23177d9 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -31,20 +31,13 @@ namespace { // more efficient access in the join columns below and apply the given // permutation to each table. cppcoro::generator> convertGenerator( - Result::Generator gen, std::vector permutation) { + Result::Generator gen, std::vector permutation = {}) { for (auto& [table, localVocab] : gen) { // Make sure to actually move the table into the wrapper so that the tables // live as long as the wrapper. - table.setColumnSubset(permutation); - ad_utility::IdTableAndFirstCol t{std::move(table), std::move(localVocab)}; - co_yield t; - } -} -// Convert a `generator` to a `generator` for -// more efficient access in the join columns below. -cppcoro::generator> convertGenerator( - Result::Generator gen) { - for (auto& [table, localVocab] : gen) { + if (!permutation.empty()) { + table.setColumnSubset(permutation); + } ad_utility::IdTableAndFirstCol t{std::move(table), std::move(localVocab)}; co_yield t; } From 3e6561cd6e70f8078cb44549168785bf45466d68 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 14:48:22 +0100 Subject: [PATCH 12/31] Fix deadlock --- src/engine/Join.cpp | 51 +++++++++++++++++++++++---------------------- 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 69a23177d9..8a1b4e6632 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -422,32 +422,33 @@ ProtoResult Join::createResult( [](auto innerAction, auto innerPermutation) -> Result::Generator { ad_utility::data_structures::ThreadSafeQueue queue{1}; - ad_utility::JThread{[&queue, &innerAction, &innerPermutation]() { - auto addValue = [&queue, &innerPermutation]( - IdTable& idTable, LocalVocab& localVocab) { - if (idTable.size() < CHUNK_SIZE) { - return; - } - Result::IdTableVocabPair pair{std::move(idTable), - std::move(localVocab)}; - if (!innerPermutation.empty()) { - pair.idTable_.setColumnSubset(innerPermutation); - } - queue.push(std::move(pair)); - }; - try { - auto finalValue = innerAction(addValue); - if (!finalValue.idTable_.empty()) { - if (!innerPermutation.empty()) { - finalValue.idTable_.setColumnSubset(innerPermutation); + ad_utility::JThread thread{ + [&queue, &innerAction, &innerPermutation]() { + auto addValue = [&queue, &innerPermutation]( + IdTable& idTable, LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; + } + Result::IdTableVocabPair pair{std::move(idTable), + std::move(localVocab)}; + if (!innerPermutation.empty()) { + pair.idTable_.setColumnSubset(innerPermutation); + } + queue.push(std::move(pair)); + }; + try { + auto finalValue = innerAction(addValue); + if (!finalValue.idTable_.empty()) { + if (!innerPermutation.empty()) { + finalValue.idTable_.setColumnSubset(innerPermutation); + } + queue.push(std::move(finalValue)); + } + queue.finish(); + } catch (...) { + queue.pushException(std::current_exception()); } - queue.push(std::move(finalValue)); - } - queue.finish(); - } catch (...) { - queue.pushException(std::current_exception()); - } - }}; + }}; while (true) { auto val = queue.pop(); if (!val.has_value()) { From ec02b9f26310425506ce8bc7afaf40a974991bf7 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 16:35:45 +0100 Subject: [PATCH 13/31] Use robuster approach for thread safety --- src/engine/Join.cpp | 60 +++++++++++++++++++++++++++++---------------- 1 file changed, 39 insertions(+), 21 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 8a1b4e6632..8db4ce55ef 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -21,7 +21,6 @@ #include "util/Exception.h" #include "util/HashMap.h" #include "util/JoinAlgorithms/JoinAlgorithms.h" -#include "util/ThreadSafeQueue.h" using std::endl; using std::string; @@ -418,14 +417,21 @@ ProtoResult Join::createResult( std::function> auto action, std::vector permutation) const { if (requestedLaziness) { - return { - [](auto innerAction, auto innerPermutation) -> Result::Generator { - ad_utility::data_structures::ThreadSafeQueue - queue{1}; - ad_utility::JThread thread{ - [&queue, &innerAction, &innerPermutation]() { - auto addValue = [&queue, &innerPermutation]( + return {[](auto innerAction, auto innerPermutation) -> Result::Generator { + std::atomic_flag write = true; + std::variant + storage; + ad_utility::JThread thread{[&write, &storage, &innerAction, + &innerPermutation]() { + auto writeValue = [&write, &storage](auto value) noexcept { + storage = std::move(value); + write.clear(); + write.notify_one(); + }; + auto addValue = [&write, &writeValue, &innerPermutation]( IdTable& idTable, LocalVocab& localVocab) { + AD_CORRECTNESS_CHECK(write.test()); if (idTable.size() < CHUNK_SIZE) { return; } @@ -434,30 +440,42 @@ ProtoResult Join::createResult( if (!innerPermutation.empty()) { pair.idTable_.setColumnSubset(innerPermutation); } - queue.push(std::move(pair)); + writeValue(std::move(pair)); + // Wait until we are allowed to write again. + write.wait(false); }; try { auto finalValue = innerAction(addValue); + AD_CORRECTNESS_CHECK(write.test()); if (!finalValue.idTable_.empty()) { if (!innerPermutation.empty()) { finalValue.idTable_.setColumnSubset(innerPermutation); } - queue.push(std::move(finalValue)); + writeValue(std::move(finalValue)); + // Wait until we are allowed to write again. + write.wait(false); } - queue.finish(); + writeValue(std::monostate{}); } catch (...) { - queue.pushException(std::current_exception()); + writeValue(std::current_exception()); } }}; - while (true) { - auto val = queue.pop(); - if (!val.has_value()) { - break; - } - co_yield val.value(); - } - }(std::move(action), std::move(permutation)), - resultSortedOn()}; + while (true) { + // Wait for read phase. + write.wait(true); + if (std::holds_alternative(storage)) { + break; + } + if (std::holds_alternative(storage)) { + std::rethrow_exception(std::get(storage)); + } + co_yield std::get(storage); + // Initiate write phase. + write.test_and_set(); + write.notify_one(); + } + }(std::move(action), std::move(permutation)), + resultSortedOn()}; } else { auto [idTable, localVocab] = action(ad_utility::noop); if (!permutation.empty()) { From cbc62529565ddc7c3ebbd6df0ba60aafe77b93a1 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 17:52:11 +0100 Subject: [PATCH 14/31] Add test for operation failure propagation --- test/JoinTest.cpp | 28 ++++++++++++++++++++++++++++ test/util/OperationTestHelpers.h | 26 ++++++++++++++++++++++---- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index f3e5a25edb..abf9682dc2 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -28,6 +28,7 @@ #include "global/RuntimeParameters.h" #include "util/Forward.h" #include "util/IndexTestHelpers.h" +#include "util/OperationTestHelpers.h" #include "util/Random.h" #include "util/SourceLocation.h" @@ -672,3 +673,30 @@ TEST(JoinTest, joinLazyAndNonLazyOperationWithAndWithoutUndefValues) { performJoin(createIdTableOfSizeWithValue(Join::CHUNK_SIZE, I(1)), std::move(rightTables), expected7, false); } + +// _____________________________________________________________________________ +TEST(JoinTest, errorInSeparateThreadIsPropagatedCorrectly) { + auto qec = ad_utility::testing::getQec(); + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); + absl::Cleanup cleanup{[]() { + // Reset back to original value to not influence other tests. + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( + 1'000'000); + }}; + auto leftTree = + ad_utility::makeExecutionTree(qec, Variable{"?s"}); + auto rightTree = ad_utility::makeExecutionTree( + qec, makeIdTableFromVector({{I(1)}}), Vars{Variable{"?s"}}, false, + std::vector{0}); + VariableToColumnMap expectedVariables{ + {Variable{"?s"}, makeAlwaysDefinedColumn(0)}}; + Join join{qec, leftTree, rightTree, 0, 0}; + + auto result = join.getResult(false, ComputationMode::LAZY_IF_SUPPORTED); + ASSERT_FALSE(result->isFullyMaterialized()); + + auto& idTables = result->idTables(); + AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE(idTables.begin(), + testing::StrEq("AlwaysFailOperation"), + std::runtime_error); +} diff --git a/test/util/OperationTestHelpers.h b/test/util/OperationTestHelpers.h index a2d0fdc816..08a9dcdc29 100644 --- a/test/util/OperationTestHelpers.h +++ b/test/util/OperationTestHelpers.h @@ -87,21 +87,37 @@ class ShallowParentOperation : public Operation { // Operation that will throw on `computeResult` for testing. class AlwaysFailOperation : public Operation { + static std::atomic_uint32_t cacheCounter; + + std::optional variable_ = std::nullopt; + std::vector getChildren() override { return {}; } - string getCacheKeyImpl() const override { AD_FAIL(); } + string getCacheKeyImpl() const override { + // Every operation gets a unique cache key + return absl::StrCat("AlwaysFailCacheKey_", cacheCounter++); + } string getDescriptor() const override { return "AlwaysFailOperationDescriptor"; } - size_t getResultWidth() const override { return 0; } + size_t getResultWidth() const override { return 1; } size_t getCostEstimate() override { return 0; } uint64_t getSizeEstimateBeforeLimit() override { return 0; } float getMultiplicity([[maybe_unused]] size_t) override { return 0; } bool knownEmptyResult() override { return false; } - vector resultSortedOn() const override { return {}; } - VariableToColumnMap computeVariableToColumnMap() const override { return {}; } + vector resultSortedOn() const override { return {0}; } + VariableToColumnMap computeVariableToColumnMap() const override { + if (!variable_.has_value()) { + return {}; + } + return {{variable_.value(), + ColumnIndexAndTypeInfo{ + 0, ColumnIndexAndTypeInfo::UndefStatus::AlwaysDefined}}}; + } public: using Operation::Operation; + AlwaysFailOperation(QueryExecutionContext* qec, Variable variable) + : Operation{qec}, variable_{std::move(variable)} {} ProtoResult computeResult(bool requestLaziness) override { if (!requestLaziness) { throw std::runtime_error{"AlwaysFailOperation"}; @@ -115,6 +131,8 @@ class AlwaysFailOperation : public Operation { } }; +std::atomic_uint32_t AlwaysFailOperation::cacheCounter = 0; + // Lazy operation that will yield a result with a custom generator you can // provide via the constructor. class CustomGeneratorOperation : public Operation { From 1f54327ce7f36457fdd39ef06d7c9ddc042c10fe Mon Sep 17 00:00:00 2001 From: Hannah Bast Date: Wed, 20 Nov 2024 18:05:09 +0100 Subject: [PATCH 15/31] Show num-local-vocabs in runtime information --- src/engine/LocalVocab.h | 6 ++++++ src/engine/Operation.cpp | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/src/engine/LocalVocab.h b/src/engine/LocalVocab.h index 72745746b9..b6cf0c0f06 100644 --- a/src/engine/LocalVocab.h +++ b/src/engine/LocalVocab.h @@ -97,12 +97,18 @@ class LocalVocab { // Return true if and only if the local vocabulary is empty. bool empty() const { return size() == 0; } + // The number of set stores (primary set and other sets). + size_t numSets() const { return 1 + otherWordSets_.size(); } + // Get the `LocalVocabEntry` corresponding to the given `LocalVocabIndex`. // // NOTE: This used to be a more complex function but is now a simple // dereference. It could be thrown out in the future. const LocalVocabEntry& getWord(LocalVocabIndex localVocabIndex) const; + // Return a const reference to the word. + const LiteralOrIri& getWord(LocalVocabIndex localVocabIndex) const; + // Add all sets (primary and other) of the given local vocabs as other sets // to this local vocab. The purpose is to keep all the contained // `LocalVocabEntry`s alive as long as this `LocalVocab` is alive. The diff --git a/src/engine/Operation.cpp b/src/engine/Operation.cpp index 9c6ae814aa..21bfecc411 100644 --- a/src/engine/Operation.cpp +++ b/src/engine/Operation.cpp @@ -120,6 +120,10 @@ ProtoResult Operation::runComputation(const ad_utility::Timer& timer, // correctly because the result was computed, so we can pass `nullopt` as // the last argument. if (result.isFullyMaterialized()) { + size_t numLocalVocabs = result.localVocab().numSets(); + if (numLocalVocabs > 1) { + runtimeInfo().addDetail("num-local-vocabs", numLocalVocabs); + } updateRuntimeInformationOnSuccess(result.idTable().size(), ad_utility::CacheStatus::computed, timer.msecs(), std::nullopt); From 1d5dcda4de3d0943f053d53bc846168843b2f067 Mon Sep 17 00:00:00 2001 From: Hannah Bast Date: Wed, 20 Nov 2024 18:08:00 +0100 Subject: [PATCH 16/31] Fix problem from previous conflict resolution --- src/engine/LocalVocab.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/engine/LocalVocab.h b/src/engine/LocalVocab.h index b6cf0c0f06..506686caec 100644 --- a/src/engine/LocalVocab.h +++ b/src/engine/LocalVocab.h @@ -106,9 +106,6 @@ class LocalVocab { // dereference. It could be thrown out in the future. const LocalVocabEntry& getWord(LocalVocabIndex localVocabIndex) const; - // Return a const reference to the word. - const LiteralOrIri& getWord(LocalVocabIndex localVocabIndex) const; - // Add all sets (primary and other) of the given local vocabs as other sets // to this local vocab. The purpose is to keep all the contained // `LocalVocabEntry`s alive as long as this `LocalVocab` is alive. The From b2b944f7641e1eb779cfa526884226ef85df7b70 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 20:10:04 +0100 Subject: [PATCH 17/31] Add unit test to ensure correct permutations --- test/JoinTest.cpp | 63 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index abf9682dc2..8d7a451b75 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -700,3 +700,66 @@ TEST(JoinTest, errorInSeparateThreadIsPropagatedCorrectly) { testing::StrEq("AlwaysFailOperation"), std::runtime_error); } + +// _____________________________________________________________________________ +TEST(JoinTest, verifyColumnPermutationsAreAppliedCorrectly) { + auto qec = + ad_utility::testing::getQec("

.

. ."); + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); + absl::Cleanup cleanup{[]() { + // Reset back to original value to not influence other tests. + RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( + 1'000'000); + }}; + auto U = Id::makeUndefined(); + { + auto leftTree = ad_utility::makeExecutionTree( + qec, makeIdTableFromVector({{U, I(1), U}, {U, I(3), U}}), + Vars{Variable{"?t"}, Variable{"?s"}, Variable{"?u"}}, false, + std::vector{1}); + auto rightTree = ad_utility::makeExecutionTree( + qec, makeIdTableFromVector({{U, I(10), I(1)}, {U, U, I(2)}}), + Vars{Variable{"?v"}, Variable{"?w"}, Variable{"?s"}}, false, + std::vector{2}); + VariableToColumnMap expectedVariables{ + {Variable{"?s"}, makeAlwaysDefinedColumn(0)}, + {Variable{"?t"}, makePossiblyUndefinedColumn(1)}, + {Variable{"?u"}, makePossiblyUndefinedColumn(2)}, + {Variable{"?v"}, makePossiblyUndefinedColumn(3)}, + {Variable{"?w"}, makePossiblyUndefinedColumn(4)}}; + auto expected = makeIdTableFromVector({{I(1), U, U, U, I(10)}}); + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + auto join = Join{qec, leftTree, rightTree, 1, 2}; + EXPECT_EQ(join.getDescriptor(), "Join on ?s"); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); + } + { + auto leftTree = ad_utility::makeExecutionTree( + qec, makeIdTableFromVector({{I(1), I(2), U}}), + Vars{Variable{"?p"}, Variable{"?q"}, Variable{"?s"}}, false, + std::vector{2}, LocalVocab{}, std::nullopt, true); + auto fullScanPSO = ad_utility::makeExecutionTree( + qec, PSO, SparqlTriple{Variable{"?s"}, "

", Var{"?o"}}); + VariableToColumnMap expectedVariables{ + {Variable{"?s"}, makeAlwaysDefinedColumn(0)}, + {Variable{"?p"}, makeAlwaysDefinedColumn(1)}, + {Variable{"?q"}, makeAlwaysDefinedColumn(2)}, + {Variable{"?o"}, makeAlwaysDefinedColumn(3)}}; + auto id = ad_utility::testing::makeGetId(qec->getIndex()); + auto expected = + makeIdTableFromVector({{id(""), I(1), I(2), id("")}, + {id(""), I(1), I(2), id("")}}); + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + auto join = Join{qec, leftTree, fullScanPSO, 2, 0}; + EXPECT_EQ(join.getDescriptor(), "Join on ?s"); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); + } +} From 1b55f452557d075e7176b88ae3dcafbc3ae524c7 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Wed, 20 Nov 2024 20:12:28 +0100 Subject: [PATCH 18/31] Avoid out-of-line definition --- test/util/OperationTestHelpers.h | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/test/util/OperationTestHelpers.h b/test/util/OperationTestHelpers.h index 08a9dcdc29..14c455960c 100644 --- a/test/util/OperationTestHelpers.h +++ b/test/util/OperationTestHelpers.h @@ -87,14 +87,12 @@ class ShallowParentOperation : public Operation { // Operation that will throw on `computeResult` for testing. class AlwaysFailOperation : public Operation { - static std::atomic_uint32_t cacheCounter; - std::optional variable_ = std::nullopt; std::vector getChildren() override { return {}; } string getCacheKeyImpl() const override { - // Every operation gets a unique cache key - return absl::StrCat("AlwaysFailCacheKey_", cacheCounter++); + // Because this operation always fails, it should never be cached. + return "AlwaysFailOperationCacheKey"; } string getDescriptor() const override { return "AlwaysFailOperationDescriptor"; @@ -131,8 +129,6 @@ class AlwaysFailOperation : public Operation { } }; -std::atomic_uint32_t AlwaysFailOperation::cacheCounter = 0; - // Lazy operation that will yield a result with a custom generator you can // provide via the constructor. class CustomGeneratorOperation : public Operation { From b9b7caefb4531f9d45375193d3d98363756a140b Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Thu, 21 Nov 2024 18:08:19 +0100 Subject: [PATCH 19/31] Address PR comments --- src/engine/Join.cpp | 267 +++++++++++++++++++++----------------------- src/engine/Join.h | 29 +++-- 2 files changed, 148 insertions(+), 148 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index 8db4ce55ef..af38151091 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -41,6 +41,26 @@ cppcoro::generator> convertGenerator( co_yield t; } } + +// Wrap a fully materialized result in a `IdTableAndFirstCol` and an array. +std::array>, 1> asSingleTableView( + const Result& result, const std::vector& permutation) { + return std::array{ad_utility::IdTableAndFirstCol{ + result.idTable().asColumnSubsetView(permutation), + result.getCopyOfLocalVocab()}}; +} + +// Wrap a result either in an array with a single element or in a range wrapping +// the lazy result generator. +std::variant())), + std::array>, 1>> +resultToRange(const Result& result, + const std::vector& permutation) { + if (result.isFullyMaterialized()) { + return asSingleTableView(result, permutation); + } + return convertGenerator(std::move(result.idTables()), permutation); +} } // namespace // _____________________________________________________________________________ @@ -114,8 +134,7 @@ ProtoResult Join::computeResult(bool requestLaziness) { if (_left->knownEmptyResult() || _right->knownEmptyResult()) { _left->getRootOperation()->updateRuntimeInformationWhenOptimizedOut(); _right->getRootOperation()->updateRuntimeInformationWhenOptimizedOut(); - return {IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - resultSortedOn(), LocalVocab()}; + return createEmptyResult(); } // Always materialize results that meet one of the following criteria: @@ -165,11 +184,7 @@ ProtoResult Join::computeResult(bool requestLaziness) { checkCancellation(); if (leftRes->isFullyMaterialized() && leftRes->idTable().empty()) { _right->getRootOperation()->updateRuntimeInformationWhenOptimizedOut(); - // TODO When we add triples to the - // index, the vocabularies of index scans will not necessarily be empty and - // we need a mechanism to still retrieve them when using the lazy scan. - return {IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - resultSortedOn(), LocalVocab()}; + return createEmptyResult(); } // Note: If only one of the children is a scan, then we have made sure in the @@ -186,15 +201,8 @@ ProtoResult Join::computeResult(bool requestLaziness) { rightResIfCached ? rightResIfCached : _right->getResult(true); checkCancellation(); if (leftRes->isFullyMaterialized() && rightRes->isFullyMaterialized()) { - IdTable idTable{getResultWidth(), getExecutionContext()->getAllocator()}; - join(leftRes->idTable(), _leftJoinCol, rightRes->idTable(), _rightJoinCol, - &idTable); - checkCancellation(); - - // If only one of the two operands has a non-empty local vocabulary, share - // with that one (otherwise, throws an exception). - return {std::move(idTable), resultSortedOn(), - Result::getMergedLocalVocab(*leftRes, *rightRes)}; + return computeResultForTwoMaterializedInputs(std::move(leftRes), + std::move(rightRes)); } return lazyJoin(std::move(leftRes), _leftJoinCol, std::move(rightRes), _rightJoinCol, requestLaziness); @@ -409,6 +417,65 @@ void Join::join(const IdTable& a, ColumnIndex jc1, const IdTable& b, << ", size = " << result->size() << "\n"; } +// _____________________________________________________________________________ +Result::Generator Join::yieldOnCallbackCalled( + ad_utility::InvocableWithExactReturnType< + Result::IdTableVocabPair, + std::function> auto action, + std::vector permutation) const { + std::atomic_flag write = true; + std::variant + storage; + ad_utility::JThread thread{[&write, &storage, &action, &permutation]() { + auto writeValue = [&write, &storage](auto value) noexcept { + storage = std::move(value); + write.clear(); + write.notify_one(); + }; + auto writeValueAndWait = [&permutation, &write, + &writeValue](Result::IdTableVocabPair value) { + AD_CORRECTNESS_CHECK(write.test()); + if (!permutation.empty()) { + value.idTable_.setColumnSubset(permutation); + } + writeValue(std::move(value)); + // Wait until we are allowed to write again. + write.wait(false); + }; + auto addValue = [&writeValueAndWait](IdTable& idTable, + LocalVocab& localVocab) { + if (idTable.size() < CHUNK_SIZE) { + return; + } + writeValueAndWait( + Result::IdTableVocabPair{std::move(idTable), std::move(localVocab)}); + }; + try { + auto finalValue = action(addValue); + if (!finalValue.idTable_.empty()) { + writeValueAndWait(std::move(finalValue)); + } + writeValue(std::monostate{}); + } catch (...) { + writeValue(std::current_exception()); + } + }}; + while (true) { + // Wait for read phase. + write.wait(true); + if (std::holds_alternative(storage)) { + break; + } + if (std::holds_alternative(storage)) { + std::rethrow_exception(std::get(storage)); + } + co_yield std::get(storage); + // Initiate write phase. + write.test_and_set(); + write.notify_one(); + } +} + // _____________________________________________________________________________ ProtoResult Join::createResult( bool requestedLaziness, @@ -417,64 +484,7 @@ ProtoResult Join::createResult( std::function> auto action, std::vector permutation) const { if (requestedLaziness) { - return {[](auto innerAction, auto innerPermutation) -> Result::Generator { - std::atomic_flag write = true; - std::variant - storage; - ad_utility::JThread thread{[&write, &storage, &innerAction, - &innerPermutation]() { - auto writeValue = [&write, &storage](auto value) noexcept { - storage = std::move(value); - write.clear(); - write.notify_one(); - }; - auto addValue = [&write, &writeValue, &innerPermutation]( - IdTable& idTable, LocalVocab& localVocab) { - AD_CORRECTNESS_CHECK(write.test()); - if (idTable.size() < CHUNK_SIZE) { - return; - } - Result::IdTableVocabPair pair{std::move(idTable), - std::move(localVocab)}; - if (!innerPermutation.empty()) { - pair.idTable_.setColumnSubset(innerPermutation); - } - writeValue(std::move(pair)); - // Wait until we are allowed to write again. - write.wait(false); - }; - try { - auto finalValue = innerAction(addValue); - AD_CORRECTNESS_CHECK(write.test()); - if (!finalValue.idTable_.empty()) { - if (!innerPermutation.empty()) { - finalValue.idTable_.setColumnSubset(innerPermutation); - } - writeValue(std::move(finalValue)); - // Wait until we are allowed to write again. - write.wait(false); - } - writeValue(std::monostate{}); - } catch (...) { - writeValue(std::current_exception()); - } - }}; - while (true) { - // Wait for read phase. - write.wait(true); - if (std::holds_alternative(storage)) { - break; - } - if (std::holds_alternative(storage)) { - std::rethrow_exception(std::get(storage)); - } - co_yield std::get(storage); - // Initiate write phase. - write.test_and_set(); - write.notify_one(); - } - }(std::move(action), std::move(permutation)), + return {yieldOnCallbackCalled(std::move(action), std::move(permutation)), resultSortedOn()}; } else { auto [idTable, localVocab] = action(ad_utility::noop); @@ -485,20 +495,6 @@ ProtoResult Join::createResult( } } -// _____________________________________________________________________________ -bool Join::couldContainUndef(const auto& blocks, const auto& tree, - ColumnIndex joinColumn) { - if constexpr (std::ranges::random_access_range) { - AD_CORRECTNESS_CHECK(!std::ranges::empty(blocks)); - return !blocks[0].empty() && blocks[0][0].isUndefined(); - } else { - auto undefStatus = tree->getVariableAndInfoByColumnIndex(joinColumn) - .second.mightContainUndef_; - return undefStatus == - ColumnIndexAndTypeInfo::UndefStatus::PossiblyUndefined; - } -} - // ______________________________________________________________________________ ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, std::shared_ptr b, ColumnIndex jc2, @@ -513,43 +509,18 @@ ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, auto resultPermutation = joinColMap.permutationResult(); return createResult( requestLaziness, - [this, a = std::move(a), jc1, b = std::move(b), jc2, + [this, a = std::move(a), b = std::move(b), joinColMap = std::move(joinColMap)]( std::function yieldTable) { ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; - auto asSingleTableView = - [](const Result& result, - const std::vector& permutation) { - return std::array{ad_utility::IdTableAndFirstCol{ - result.idTable().asColumnSubsetView(permutation), - result.getCopyOfLocalVocab()}}; - }; - using Blocks = std::variant< - cppcoro::generator>, - std::array>, 1>>; - auto leftRange = - a->isFullyMaterialized() - ? Blocks{asSingleTableView(*a, joinColMap.permutationLeft())} - : Blocks{convertGenerator(std::move(a->idTables()), - joinColMap.permutationLeft())}; - auto rightRange = - b->isFullyMaterialized() - ? Blocks{asSingleTableView(*b, joinColMap.permutationRight())} - : Blocks{convertGenerator(std::move(b->idTables()), - joinColMap.permutationRight())}; + 1, IdTable{getResultWidth(), allocator()}, cancellationHandle_, + CHUNK_SIZE, std::move(yieldTable)}; + auto leftRange = resultToRange(*a, joinColMap.permutationLeft()); + auto rightRange = resultToRange(*b, joinColMap.permutationRight()); std::visit( - [this, &rowAdder, jc1, jc2](auto& leftBlocks, auto& rightBlocks) { - bool containsUndef = couldContainUndef(leftBlocks, _left, jc1) || - couldContainUndef(rightBlocks, _right, jc2); - if (containsUndef) { - ad_utility::zipperJoinForBlocksWithPotentialUndef( - leftBlocks, rightBlocks, std::less{}, rowAdder); - } else { - ad_utility::zipperJoinForBlocksWithoutUndef( - leftBlocks, rightBlocks, std::less{}, rowAdder); - } + [&rowAdder](auto& leftBlocks, auto& rightBlocks) { + ad_utility::zipperJoinForBlocksWithPotentialUndef( + leftBlocks, rightBlocks, std::less{}, rowAdder); }, leftRange, rightRange); auto localVocab = std::move(rowAdder.localVocab()); @@ -668,7 +639,7 @@ void Join::hashJoin(const IdTable& dynA, ColumnIndex jc1, const IdTable& dynB, // ___________________________________________________________________________ template void Join::addCombinedRowToIdTable(const ROW_A& rowA, const ROW_B& rowB, - const ColumnIndex jcRowB, + ColumnIndex jcRowB, IdTableStatic* table) { // Add a new, empty row. const size_t backIndex = table->size(); @@ -748,8 +719,8 @@ ProtoResult Join::computeResultForTwoIndexScans(bool requestLaziness) const { // `AddCombinedRowToIdTable` class to work correctly. AD_CORRECTNESS_CHECK(_leftJoinCol == 0 && _rightJoinCol == 0); ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; + 1, IdTable{getResultWidth(), allocator()}, cancellationHandle_, + CHUNK_SIZE, std::move(yieldTable)}; ad_utility::Timer timer{ ad_utility::timer::Timer::InitialStatus::Started}; @@ -803,8 +774,8 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( std::function yieldTable) { const IdTable& idTable = resultWithIdTable->idTable(); ad_utility::AddCombinedRowToIdTable rowAdder{ - 1, IdTable{getResultWidth(), getExecutionContext()->getAllocator()}, - cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; + 1, IdTable{getResultWidth(), allocator()}, cancellationHandle_, + CHUNK_SIZE, std::move(yieldTable)}; AD_CORRECTNESS_CHECK(joinColScan == 0); auto permutationIdTable = ad_utility::IdTableAndFirstCol{ @@ -819,13 +790,13 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); std::optional> indexScanResult = std::nullopt; - using FirstColView = ad_utility::IdTableAndFirstCol; - using GenWithDetails = - cppcoro::generator; + using GenWithDetails = decltype(convertGenerator( + std::declval())); auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, &indexScanResult]() - -> std::variant, GenWithDetails> { + -> std::variant())), + GenWithDetails> { if (idTableHasUndef) { indexScanResult = scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); @@ -841,15 +812,13 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( }(); runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); - auto doJoin = [&rowAdder, idTableHasUndef](auto& left, - auto& right) mutable { - if (idTableHasUndef) { - ad_utility::zipperJoinForBlocksWithPotentialUndef( - left, right, std::less{}, rowAdder); - } else { - ad_utility::zipperJoinForBlocksWithoutUndef(left, right, - std::less{}, rowAdder); - } + auto doJoin = [&rowAdder](auto& left, auto& right) mutable { + // Technically we could use the zipperJoinForBlocksWithoutUndef when + // checking for `idTableHasUndef`, but the implementation of the join + // algorithm consumes all undef blocks at the start and falls back to + // the other implementation when no undef blocks could be found. + ad_utility::zipperJoinForBlocksWithPotentialUndef( + left, right, std::less{}, rowAdder); }; auto blockForIdTable = std::array{std::move(permutationIdTable)}; std::visit( @@ -873,3 +842,21 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( }, std::move(resultPermutation)); } +// _____________________________________________________________________________ +ProtoResult Join::computeResultForTwoMaterializedInputs( + std::shared_ptr leftRes, + std::shared_ptr rightRes) const { + IdTable idTable{getResultWidth(), allocator()}; + join(leftRes->idTable(), _leftJoinCol, rightRes->idTable(), _rightJoinCol, + &idTable); + checkCancellation(); + + return {std::move(idTable), resultSortedOn(), + Result::getMergedLocalVocab(*leftRes, *rightRes)}; +} + +// _____________________________________________________________________________ +ProtoResult Join::createEmptyResult() const { + return {IdTable{getResultWidth(), allocator()}, resultSortedOn(), + LocalVocab{}}; +} diff --git a/src/engine/Join.h b/src/engine/Join.h index 8654d039cc..d5ac8f679d 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -94,6 +94,18 @@ class Join : public Operation { void join(const IdTable& a, ColumnIndex jc1, const IdTable& b, ColumnIndex jc2, IdTable* result) const; + private: + // Part of the implementation of `createResult`. This function is called when + // the result should be yielded lazily. + // The semantics of action can be seen as + // runJoinAndReturnFinalResult(callbackForIntermediateResults). + Result::Generator yieldOnCallbackCalled( + ad_utility::InvocableWithExactReturnType< + Result::IdTableVocabPair, + std::function> auto action, + std::vector permutation) const; + + public: // Helper function to compute the result of a join operation and conditionally // return a lazy or fully materialized result depending on `requestLaziness`. // This is achieved by running the `action` lambda in a separate thread and @@ -112,13 +124,6 @@ class Join : public Operation { std::function> auto action, std::vector permutation = {}) const; - // Helper function that cheaply checks if a join could contain undefined. For - // fully materialized tables it can just look at the first element. For lazy - // tables it has to look at the meta information which could potentially - // indicate undefinedness even when all values are defined. - static bool couldContainUndef(const auto& blocks, const auto& tree, - ColumnIndex joinColumn); - // Fallback implementation of a join that is used when at least one of the two // inputs is not fully materialized. This represents the general case where we // don't have any optimization left to try. @@ -165,6 +170,11 @@ class Join : public Operation { ColumnIndex joinColTable, std::shared_ptr scan, ColumnIndex joinColScan) const; + // Default case where both inputs are fully materialized. + ProtoResult computeResultForTwoMaterializedInputs( + std::shared_ptr leftRes, + std::shared_ptr rightRes) const; + /* * @brief Combines 2 rows like in a join and inserts the result in the * given table. @@ -179,7 +189,7 @@ class Join : public Operation { */ template static void addCombinedRowToIdTable(const ROW_A& rowA, const ROW_B& rowB, - const ColumnIndex jcRowB, + ColumnIndex jcRowB, IdTableStatic* table); /* @@ -189,4 +199,7 @@ class Join : public Operation { static void hashJoinImpl(const IdTable& dynA, ColumnIndex jc1, const IdTable& dynB, ColumnIndex jc2, IdTable* dynRes); + + // Commonly used code for the various known-to-be-empty cases. + ProtoResult createEmptyResult() const; }; From 0f4f8593107abcaab80a419ee8e058859590ff0c Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Fri, 22 Nov 2024 16:01:48 +0100 Subject: [PATCH 20/31] Address simple PR comments --- src/engine/Join.cpp | 85 +++++++------- src/engine/Join.h | 13 ++- test/JoinTest.cpp | 135 +++++++++++------------ test/util/RuntimeParametersTestHelpers.h | 18 +++ 4 files changed, 135 insertions(+), 116 deletions(-) create mode 100644 test/util/RuntimeParametersTestHelpers.h diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index af38151091..ea1a4c0594 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -26,24 +26,38 @@ using std::endl; using std::string; namespace { +void applyPermutation( + IdTable& idTable, + const std::optional>& permutation) { + if (permutation.has_value()) { + idTable.setColumnSubset(permutation.value()); + } +} + +using LazyInputView = + cppcoro::generator>; // Convert a `generator` to a `generator` for // more efficient access in the join columns below and apply the given // permutation to each table. -cppcoro::generator> convertGenerator( - Result::Generator gen, std::vector permutation = {}) { +LazyInputView convertGenerator( + Result::Generator gen, + std::optional> permutation = {}) { for (auto& [table, localVocab] : gen) { + applyPermutation(table, permutation); // Make sure to actually move the table into the wrapper so that the tables // live as long as the wrapper. - if (!permutation.empty()) { - table.setColumnSubset(permutation); - } ad_utility::IdTableAndFirstCol t{std::move(table), std::move(localVocab)}; co_yield t; } } -// Wrap a fully materialized result in a `IdTableAndFirstCol` and an array. -std::array>, 1> asSingleTableView( +using MaterializedInputView = + std::array>, 1>; +// Wrap a fully materialized result in a `IdTableAndFirstCol` and an array. It +// then fulfills the concept `view` which is required by the +// lazy join algorithms. Note: The `convertGenerator` function above +// conceptually does exactly the same for lazy inputs. +MaterializedInputView asSingleTableView( const Result& result, const std::vector& permutation) { return std::array{ad_utility::IdTableAndFirstCol{ result.idTable().asColumnSubsetView(permutation), @@ -51,11 +65,10 @@ std::array>, 1> asSingleTableView( } // Wrap a result either in an array with a single element or in a range wrapping -// the lazy result generator. -std::variant())), - std::array>, 1>> -resultToRange(const Result& result, - const std::vector& permutation) { +// the lazy result generator. Note that the lifetime of the view is coupled to +// the lifetime of the result. +std::variant resultToView( + const Result& result, const std::vector& permutation) { if (result.isFullyMaterialized()) { return asSingleTableView(result, permutation); } @@ -418,11 +431,11 @@ void Join::join(const IdTable& a, ColumnIndex jc1, const IdTable& b, } // _____________________________________________________________________________ -Result::Generator Join::yieldOnCallbackCalled( +Result::Generator Join::runLazyJoinAndConvertToGenerator( ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::vector permutation) const { + std::optional> permutation) const { std::atomic_flag write = true; std::variant storage; @@ -435,9 +448,7 @@ Result::Generator Join::yieldOnCallbackCalled( auto writeValueAndWait = [&permutation, &write, &writeValue](Result::IdTableVocabPair value) { AD_CORRECTNESS_CHECK(write.test()); - if (!permutation.empty()) { - value.idTable_.setColumnSubset(permutation); - } + applyPermutation(value.idTable_, permutation); writeValue(std::move(value)); // Wait until we are allowed to write again. write.wait(false); @@ -447,8 +458,7 @@ Result::Generator Join::yieldOnCallbackCalled( if (idTable.size() < CHUNK_SIZE) { return; } - writeValueAndWait( - Result::IdTableVocabPair{std::move(idTable), std::move(localVocab)}); + writeValueAndWait({std::move(idTable), std::move(localVocab)}); }; try { auto finalValue = action(addValue); @@ -482,15 +492,14 @@ ProtoResult Join::createResult( ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::vector permutation) const { + std::optional> permutation) const { if (requestedLaziness) { - return {yieldOnCallbackCalled(std::move(action), std::move(permutation)), + return {runLazyJoinAndConvertToGenerator(std::move(action), + std::move(permutation)), resultSortedOn()}; } else { auto [idTable, localVocab] = action(ad_utility::noop); - if (!permutation.empty()) { - idTable.setColumnSubset(permutation); - } + applyPermutation(idTable, permutation); return {std::move(idTable), resultSortedOn(), std::move(localVocab)}; } } @@ -515,8 +524,8 @@ ProtoResult Join::lazyJoin(std::shared_ptr a, ColumnIndex jc1, ad_utility::AddCombinedRowToIdTable rowAdder{ 1, IdTable{getResultWidth(), allocator()}, cancellationHandle_, CHUNK_SIZE, std::move(yieldTable)}; - auto leftRange = resultToRange(*a, joinColMap.permutationLeft()); - auto rightRange = resultToRange(*b, joinColMap.permutationRight()); + auto leftRange = resultToView(*a, joinColMap.permutationLeft()); + auto rightRange = resultToView(*b, joinColMap.permutationRight()); std::visit( [&rowAdder](auto& leftBlocks, auto& rightBlocks) { ad_utility::zipperJoinForBlocksWithPotentialUndef( @@ -662,11 +671,12 @@ void Join::addCombinedRowToIdTable(const ROW_A& rowA, const ROW_B& rowB, } namespace { +using GeneratorWithDetails = + cppcoro::generator, + CompressedRelationReader::LazyScanMetadata>; // Convert a `generator` for more // efficient access in the join columns below. -cppcoro::generator, - CompressedRelationReader::LazyScanMetadata> -convertGenerator(Permutation::IdTableGenerator gen) { +GeneratorWithDetails convertGenerator(Permutation::IdTableGenerator gen) { co_await cppcoro::getDetails = gen.details(); gen.setDetailsPointer(&co_await cppcoro::getDetails); for (auto& table : gen) { @@ -790,13 +800,9 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( !idTable.empty() && idTable.at(0, joinColTable).isUndefined(); std::optional> indexScanResult = std::nullopt; - using GenWithDetails = decltype(convertGenerator( - std::declval())); auto rightBlocks = [&scan, idTableHasUndef, &permutationIdTable, &indexScanResult]() - -> std::variant())), - GenWithDetails> { + -> std::variant { if (idTableHasUndef) { indexScanResult = scan->getResult(false, ComputationMode::LAZY_IF_SUPPORTED); @@ -813,10 +819,9 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( runtimeInfo().addDetail("time-for-filtering-blocks", timer.msecs()); auto doJoin = [&rowAdder](auto& left, auto& right) mutable { - // Technically we could use the zipperJoinForBlocksWithoutUndef when - // checking for `idTableHasUndef`, but the implementation of the join - // algorithm consumes all undef blocks at the start and falls back to - // the other implementation when no undef blocks could be found. + // Note: The `zipperJoinForBlocksWithPotentialUndef` automatically + // switches to a more efficient implementation if there are no UNDEF + // values in any of the inputs. ad_utility::zipperJoinForBlocksWithPotentialUndef( left, right, std::less{}, rowAdder); }; @@ -831,9 +836,9 @@ ProtoResult Join::computeResultForIndexScanAndIdTable( }, rightBlocks); - if (std::holds_alternative(rightBlocks)) { + if (std::holds_alternative(rightBlocks)) { updateRuntimeInfoForLazyScan( - *scan, std::get(rightBlocks).details()); + *scan, std::get(rightBlocks).details()); } auto localVocab = std::move(rowAdder.localVocab()); diff --git a/src/engine/Join.h b/src/engine/Join.h index d5ac8f679d..3d243c4332 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -97,13 +97,16 @@ class Join : public Operation { private: // Part of the implementation of `createResult`. This function is called when // the result should be yielded lazily. - // The semantics of action can be seen as - // runJoinAndReturnFinalResult(callbackForIntermediateResults). - Result::Generator yieldOnCallbackCalled( + // Action is a lambda that itself runs the join operation in a blocking + // manner. It is passed a special function that is supposed to be the callback + // being passed to the `AddCombinedRowToIdTable` so that the partial results + // can be yielded during execution. This is achieved by spawning a separate + // thread. + Result::Generator runLazyJoinAndConvertToGenerator( ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::vector permutation) const; + std::optional> permutation) const; public: // Helper function to compute the result of a join operation and conditionally @@ -122,7 +125,7 @@ class Join : public Operation { ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::vector permutation = {}) const; + std::optional> permutation = {}) const; // Fallback implementation of a join that is used when at least one of the two // inputs is not fully materialized. This represents the general case where we diff --git a/test/JoinTest.cpp b/test/JoinTest.cpp index 8d7a451b75..f1b24c2eb4 100644 --- a/test/JoinTest.cpp +++ b/test/JoinTest.cpp @@ -25,11 +25,11 @@ #include "engine/Values.h" #include "engine/ValuesForTesting.h" #include "engine/idTable/IdTable.h" -#include "global/RuntimeParameters.h" #include "util/Forward.h" #include "util/IndexTestHelpers.h" #include "util/OperationTestHelpers.h" #include "util/Random.h" +#include "util/RuntimeParametersTestHelpers.h" #include "util/SourceLocation.h" using ad_utility::testing::makeAllocator; @@ -238,6 +238,9 @@ using ExpectedColumns = ad_utility::HashMap< std::pair, ColumnIndexAndTypeInfo::UndefStatus>>; // Test that the result of the `join` matches the `expected` outcome. +// If `requestLaziness` is true, the join is requested to be lazy. If +// `expectLazinessParityWhenNonEmpty` is true, the laziness of the result is +// expected to be the same as `requestLaziness` if the result is not empty. void testJoinOperation(Join& join, const ExpectedColumns& expected, bool requestLaziness = false, bool expectLazinessParityWhenNonEmpty = false, @@ -253,12 +256,11 @@ void testJoinOperation(Join& join, const ExpectedColumns& expected, (!res->isFullyMaterialized() || !res->idTable().empty())) { EXPECT_EQ(res->isFullyMaterialized(), !requestLaziness); } - const auto& table = res->isFullyMaterialized() - ? res->idTable() - : static_cast( - aggregateTables(std::move(res->idTables()), - join.getResultWidth()) - .first); + IdTable table = + res->isFullyMaterialized() + ? res->idTable().clone() + : aggregateTables(std::move(res->idTables()), join.getResultWidth()) + .first; ASSERT_EQ(table.numColumns(), expected.size()); for (const auto& [var, columnAndStatus] : expected) { const auto& [colIndex, undefStatus] = varToCols.at(var); @@ -357,8 +359,9 @@ TEST(JoinTest, joinWithFullScanPSO) { TEST(JoinTest, joinWithColumnAndScan) { auto test = [](size_t materializationThreshold) { auto qec = ad_utility::testing::getQec("

1.

2. 3."); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - materializationThreshold); + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">( + materializationThreshold); qec->getQueryTreeCache().clearAll(); auto fullScanPSO = ad_utility::makeExecutionTree( qec, PSO, SparqlTriple{Var{"?s"}, "

", Var{"?o"}}); @@ -389,8 +392,9 @@ TEST(JoinTest, joinWithColumnAndScan) { TEST(JoinTest, joinWithColumnAndScanEmptyInput) { auto test = [](size_t materializationThreshold) { auto qec = ad_utility::testing::getQec("

1.

2. 3."); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - materializationThreshold); + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">( + materializationThreshold); qec->getQueryTreeCache().clearAll(); auto fullScanPSO = ad_utility::makeExecutionTree( qec, PSO, SparqlTriple{Var{"?s"}, "

", Var{"?o"}}); @@ -420,8 +424,9 @@ TEST(JoinTest, joinWithColumnAndScanEmptyInput) { TEST(JoinTest, joinWithColumnAndScanUndefValues) { auto test = [](size_t materializationThreshold) { auto qec = ad_utility::testing::getQec("

1.

2. 3."); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - materializationThreshold); + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">( + materializationThreshold); qec->getQueryTreeCache().clearAll(); auto fullScanPSO = ad_utility::makeExecutionTree( qec, PSO, SparqlTriple{Var{"?s"}, "

", Var{"?o"}}); @@ -464,8 +469,9 @@ TEST(JoinTest, joinTwoScans) { auto test = [](size_t materializationThreshold) { auto qec = ad_utility::testing::getQec( "

1.

2. 3 . 4. 7. "); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - materializationThreshold); + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">( + materializationThreshold); auto scanP = ad_utility::makeExecutionTree( qec, PSO, SparqlTriple{Var{"?s"}, "

", Var{"?o"}}); auto scanP2 = ad_utility::makeExecutionTree( @@ -513,44 +519,42 @@ TEST(JoinTest, invalidJoinVariable) { // _____________________________________________________________________________ TEST(JoinTest, joinTwoLazyOperationsWithAndWithoutUndefValues) { - auto performJoin = - [](std::vector leftTables, std::vector rightTables, - const IdTable& expected, bool expectPossiblyUndefinedResult, - ad_utility::source_location loc = - ad_utility::source_location::current()) { - auto l = generateLocationTrace(loc); - auto qec = ad_utility::testing::getQec(); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); - absl::Cleanup cleanup{[]() { - // Reset back to original value to not influence other tests. - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - 1'000'000); - }}; - auto leftTree = ad_utility::makeExecutionTree( - qec, std::move(leftTables), Vars{Variable{"?s"}}, false, - std::vector{0}); - auto rightTree = ad_utility::makeExecutionTree( - qec, std::move(rightTables), Vars{Variable{"?s"}}, false, - std::vector{0}); - VariableToColumnMap expectedVariables{ - {Variable{"?s"}, expectPossiblyUndefinedResult - ? makePossiblyUndefinedColumn(0) - : makeAlwaysDefinedColumn(0)}}; - auto expectedColumns = makeExpectedColumns(expectedVariables, expected); - auto join = Join{qec, leftTree, rightTree, 0, 0}; - EXPECT_EQ(join.getDescriptor(), "Join on ?s"); - - qec->getQueryTreeCache().clearAll(); - testJoinOperation(join, expectedColumns, true, true); - qec->getQueryTreeCache().clearAll(); - testJoinOperation(join, expectedColumns, false); - - auto joinSwitched = Join{qec, rightTree, leftTree, 0, 0}; - qec->getQueryTreeCache().clearAll(); - testJoinOperation(joinSwitched, expectedColumns, true, true); - qec->getQueryTreeCache().clearAll(); - testJoinOperation(joinSwitched, expectedColumns, false); - }; + auto performJoin = [](std::vector leftTables, + std::vector rightTables, + const IdTable& expected, + bool expectPossiblyUndefinedResult, + ad_utility::source_location loc = + ad_utility::source_location::current()) { + auto l = generateLocationTrace(loc); + auto qec = ad_utility::testing::getQec(); + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">( + 0); + auto leftTree = ad_utility::makeExecutionTree( + qec, std::move(leftTables), Vars{Variable{"?s"}}, false, + std::vector{0}); + auto rightTree = ad_utility::makeExecutionTree( + qec, std::move(rightTables), Vars{Variable{"?s"}}, false, + std::vector{0}); + VariableToColumnMap expectedVariables{ + {Variable{"?s"}, expectPossiblyUndefinedResult + ? makePossiblyUndefinedColumn(0) + : makeAlwaysDefinedColumn(0)}}; + auto expectedColumns = makeExpectedColumns(expectedVariables, expected); + auto join = Join{qec, leftTree, rightTree, 0, 0}; + EXPECT_EQ(join.getDescriptor(), "Join on ?s"); + + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, true, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(join, expectedColumns, false); + + auto joinSwitched = Join{qec, rightTree, leftTree, 0, 0}; + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, true, true); + qec->getQueryTreeCache().clearAll(); + testJoinOperation(joinSwitched, expectedColumns, false); + }; auto U = Id::makeUndefined(); std::vector leftTables; std::vector rightTables; @@ -602,12 +606,9 @@ TEST(JoinTest, joinLazyAndNonLazyOperationWithAndWithoutUndefValues) { ad_utility::source_location::current()) { auto l = generateLocationTrace(loc); auto qec = ad_utility::testing::getQec(); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); - absl::Cleanup cleanup{[]() { - // Reset back to original value to not influence other tests. - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - 1'000'000); - }}; + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">( + 0); auto leftTree = ad_utility::makeExecutionTree( qec, std::move(leftTable), Vars{Variable{"?s"}}, false, @@ -677,12 +678,8 @@ TEST(JoinTest, joinLazyAndNonLazyOperationWithAndWithoutUndefValues) { // _____________________________________________________________________________ TEST(JoinTest, errorInSeparateThreadIsPropagatedCorrectly) { auto qec = ad_utility::testing::getQec(); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); - absl::Cleanup cleanup{[]() { - // Reset back to original value to not influence other tests. - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - 1'000'000); - }}; + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">(0); auto leftTree = ad_utility::makeExecutionTree(qec, Variable{"?s"}); auto rightTree = ad_utility::makeExecutionTree( @@ -705,12 +702,8 @@ TEST(JoinTest, errorInSeparateThreadIsPropagatedCorrectly) { TEST(JoinTest, verifyColumnPermutationsAreAppliedCorrectly) { auto qec = ad_utility::testing::getQec("

.

. ."); - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">(0); - absl::Cleanup cleanup{[]() { - // Reset back to original value to not influence other tests. - RuntimeParameters().set<"lazy-index-scan-max-size-materialization">( - 1'000'000); - }}; + auto cleanup = + setRuntimeParameterForTest<"lazy-index-scan-max-size-materialization">(0); auto U = Id::makeUndefined(); { auto leftTree = ad_utility::makeExecutionTree( diff --git a/test/util/RuntimeParametersTestHelpers.h b/test/util/RuntimeParametersTestHelpers.h new file mode 100644 index 0000000000..097bc93265 --- /dev/null +++ b/test/util/RuntimeParametersTestHelpers.h @@ -0,0 +1,18 @@ +// Copyright 2024, University of Freiburg, +// Chair of Algorithms and Data Structures. +// Author: Robin Textor-Falconi + +#pragma once + +#include + +#include "global/RuntimeParameters.h" + +template +[[nodiscard]] auto setRuntimeParameterForTest(Value&& value) { + auto originalValue = RuntimeParameters().get(); + RuntimeParameters().set(AD_FWD(value)); + return absl::Cleanup{[originalValue = std::move(originalValue)]() { + RuntimeParameters().set(originalValue); + }}; +} From 62d9295485c2096d37c9e88363925228b9ff8155 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Fri, 22 Nov 2024 16:45:26 +0100 Subject: [PATCH 21/31] Allow more ranges to be used in `LocalVocab` --- src/engine/LocalVocab.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/engine/LocalVocab.h b/src/engine/LocalVocab.h index 506686caec..b266f46714 100644 --- a/src/engine/LocalVocab.h +++ b/src/engine/LocalVocab.h @@ -111,10 +111,11 @@ class LocalVocab { // `LocalVocabEntry`s alive as long as this `LocalVocab` is alive. The // primary set of this `LocalVocab` remains unchanged. template - void mergeWith(const R& vocabs) { + void mergeWith(R&& vocabs) { auto inserter = std::back_inserter(otherWordSets_); using std::views::filter; - for (const auto& vocab : vocabs | filter(std::not_fn(&LocalVocab::empty))) { + for (const auto& vocab : + AD_FWD(vocabs) | filter(std::not_fn(&LocalVocab::empty))) { std::ranges::copy(vocab.otherWordSets_, inserter); *inserter = vocab.primaryWordSet_; size_ += vocab.size_; From c7889a663084220b69bd8e88049e4dd013e8f745 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Fri, 22 Nov 2024 16:48:39 +0100 Subject: [PATCH 22/31] Merge missing vocabs --- src/engine/AddCombinedRowToTable.h | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index 41d0d0d73e..9631351e1a 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -27,6 +27,7 @@ class AddCombinedRowToIdTable { std::optional, 2>> inputLeftAndRight_; IdTable resultTable_; LocalVocab mergedVocab_{}; + std::array currentVocabs_{nullptr, nullptr}; // This struct stores the information, which row indices from the input are // combined into a given row index in the output, i.e. "To obtain the @@ -139,17 +140,21 @@ class AddCombinedRowToIdTable { return table; } }; - auto mergeVocab = [this](const T& table) { + auto mergeVocab = [this](const T& table, + const LocalVocab*& currentVocab) { if constexpr (requires { table.getLocalVocab(); }) { + currentVocab = &table.getLocalVocab(); mergedVocab_.mergeWith(std::span{&table.getLocalVocab(), 1}); + } else { + currentVocab = nullptr; } }; if (nextIndex_ != 0) { AD_CORRECTNESS_CHECK(inputLeftAndRight_.has_value()); flush(); } - mergeVocab(inputLeft); - mergeVocab(inputRight); + mergeVocab(inputLeft, currentVocabs_.at(0)); + mergeVocab(inputRight, currentVocabs_.at(1)); inputLeftAndRight_ = std::array{toView(inputLeft), toView(inputRight)}; checkNumColumns(); } @@ -331,6 +336,13 @@ class AddCombinedRowToIdTable { optionalIndexBuffer_.clear(); nextIndex_ = 0; std::invoke(blockwiseCallback_, result, mergedVocab_); + // The current `IdTable`s might still be active, so we have to merge the + // local vocabs again if all other sets were moved-out. + if (mergedVocab_.numSets() == 1) { + // Only merge non-null vocabs. + mergedVocab_.mergeWith(currentVocabs_ | std::views::filter(toBool) | + std::views::transform(dereference)); + } } const IdTableView<0>& inputLeft() const { return inputLeftAndRight_.value()[0]; From 9e357cdf00d7b971a6a096974928874c74020ac5 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Fri, 22 Nov 2024 17:07:20 +0100 Subject: [PATCH 23/31] Revert "Allow more ranges to be used in `LocalVocab`" This reverts commit 62d9295485c2096d37c9e88363925228b9ff8155. --- src/engine/LocalVocab.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/engine/LocalVocab.h b/src/engine/LocalVocab.h index 956cf49a24..4cc47c5e55 100644 --- a/src/engine/LocalVocab.h +++ b/src/engine/LocalVocab.h @@ -113,11 +113,10 @@ class LocalVocab { // `LocalVocabEntry`s alive as long as this `LocalVocab` is alive. The // primary set of this `LocalVocab` remains unchanged. template - void mergeWith(R&& vocabs) { + void mergeWith(const R& vocabs) { auto inserter = std::back_inserter(otherWordSets_); using std::views::filter; - for (const auto& vocab : - AD_FWD(vocabs) | filter(std::not_fn(&LocalVocab::empty))) { + for (const auto& vocab : vocabs | filter(std::not_fn(&LocalVocab::empty))) { std::ranges::copy(vocab.otherWordSets_, inserter); *inserter = vocab.primaryWordSet_; size_ += vocab.size_; From 711212a1837907bf3d1fbe9c694aa2df9b42b1d3 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Fri, 22 Nov 2024 17:13:39 +0100 Subject: [PATCH 24/31] Fix compilation --- src/engine/AddCombinedRowToTable.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index 9631351e1a..dcd57e5c7c 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -340,8 +340,9 @@ class AddCombinedRowToIdTable { // local vocabs again if all other sets were moved-out. if (mergedVocab_.numSets() == 1) { // Only merge non-null vocabs. - mergedVocab_.mergeWith(currentVocabs_ | std::views::filter(toBool) | - std::views::transform(dereference)); + auto range = currentVocabs_ | std::views::filter(toBool) | + std::views::transform(dereference); + mergedVocab_.mergeWith(std::ranges::ref_view{range}); } } const IdTableView<0>& inputLeft() const { From 4b51d1d138b9e957f5e1657caae47d31dbfafaf4 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Sun, 24 Nov 2024 23:45:00 +0100 Subject: [PATCH 25/31] Address minor PR comments --- src/engine/Join.cpp | 13 ++++++------- src/engine/Join.h | 6 ++++-- test/util/RuntimeParametersTestHelpers.h | 2 ++ 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/engine/Join.cpp b/src/engine/Join.cpp index ea1a4c0594..6d2376bb9c 100644 --- a/src/engine/Join.cpp +++ b/src/engine/Join.cpp @@ -26,9 +26,9 @@ using std::endl; using std::string; namespace { -void applyPermutation( - IdTable& idTable, - const std::optional>& permutation) { +using OptionalPermutation = Join::OptionalPermutation; +void applyPermutation(IdTable& idTable, + const OptionalPermutation& permutation) { if (permutation.has_value()) { idTable.setColumnSubset(permutation.value()); } @@ -39,9 +39,8 @@ using LazyInputView = // Convert a `generator` to a `generator` for // more efficient access in the join columns below and apply the given // permutation to each table. -LazyInputView convertGenerator( - Result::Generator gen, - std::optional> permutation = {}) { +LazyInputView convertGenerator(Result::Generator gen, + OptionalPermutation permutation = {}) { for (auto& [table, localVocab] : gen) { applyPermutation(table, permutation); // Make sure to actually move the table into the wrapper so that the tables @@ -435,7 +434,7 @@ Result::Generator Join::runLazyJoinAndConvertToGenerator( ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::optional> permutation) const { + OptionalPermutation permutation) const { std::atomic_flag write = true; std::variant storage; diff --git a/src/engine/Join.h b/src/engine/Join.h index 3d243c4332..a65d10f87b 100644 --- a/src/engine/Join.h +++ b/src/engine/Join.h @@ -33,6 +33,8 @@ class Join : public Operation { std::shared_ptr t2, ColumnIndex t1JoinCol, ColumnIndex t2JoinCol); + using OptionalPermutation = std::optional>; + static constexpr size_t CHUNK_SIZE = 100'000; // A very explicit constructor, which initializes an invalid join object (it // has no subtrees, which violates class invariants). These invalid Join @@ -106,7 +108,7 @@ class Join : public Operation { ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::optional> permutation) const; + OptionalPermutation permutation) const; public: // Helper function to compute the result of a join operation and conditionally @@ -125,7 +127,7 @@ class Join : public Operation { ad_utility::InvocableWithExactReturnType< Result::IdTableVocabPair, std::function> auto action, - std::optional> permutation = {}) const; + OptionalPermutation permutation = {}) const; // Fallback implementation of a join that is used when at least one of the two // inputs is not fully materialized. This represents the general case where we diff --git a/test/util/RuntimeParametersTestHelpers.h b/test/util/RuntimeParametersTestHelpers.h index 097bc93265..9b75aaa350 100644 --- a/test/util/RuntimeParametersTestHelpers.h +++ b/test/util/RuntimeParametersTestHelpers.h @@ -8,6 +8,8 @@ #include "global/RuntimeParameters.h" +// Set a runtime parameter to a specific value for the duration of the current +// scope. The original value is restored when the scope is left. template [[nodiscard]] auto setRuntimeParameterForTest(Value&& value) { auto originalValue = RuntimeParameters().get(); From 8e85cc7d5c5a79f832192236fdc1ff87eafdda3f Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Sun, 24 Nov 2024 23:47:27 +0100 Subject: [PATCH 26/31] Fix issues with vocab and add unit tests --- src/engine/AddCombinedRowToTable.h | 72 ++++++++++++---------- test/AddCombinedRowToTableTest.cpp | 97 ++++++++++++++++++++++++++++++ 2 files changed, 138 insertions(+), 31 deletions(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index dcd57e5c7c..70a004cbb7 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -126,6 +126,42 @@ class AddCombinedRowToIdTable { } } + // Unwrap type `T` to get an `IdTableView<0>`, even if it's not an + // `IdTableView<0>`. Identity for `IdTableView<0>`. + template + static IdTableView<0> toView(const T& table) { + if constexpr (requires { table.template asStaticView<0>(); }) { + return table.template asStaticView<0>(); + } else { + return table; + } + } + + // Merge the vocab contained in `T` with the `mergedVocab_` and set the passed + // pointer reference to the proper value. + template + void mergeVocab(const T& table, const LocalVocab*& currentVocab) { + if constexpr (requires { table.getLocalVocab(); }) { + currentVocab = &table.getLocalVocab(); + mergedVocab_.mergeWith(std::span{&table.getLocalVocab(), 1}); + } else { + currentVocab = nullptr; + } + } + + // Flush remaining pending entries before changing the input. + void flushBeforeInputChange() { + // Clear to avoid unecessary merge. + currentVocabs_ = {nullptr, nullptr}; + if (nextIndex_ != 0) { + AD_CORRECTNESS_CHECK(inputLeftAndRight_.has_value()); + flush(); + } else { + // Clear vocab when no rows were written. + mergedVocab_ = LocalVocab{}; + } + } + // Set or reset the input. All following calls to `addRow` then refer to // indices in the new input. Before resetting, `flush()` is called, so all the // rows from the previous inputs get materialized before deleting the old @@ -133,26 +169,7 @@ class AddCombinedRowToIdTable { // `IdTable` or `IdTableView<0>`, or any other type that has a // `asStaticView<0>` method that returns an `IdTableView<0>`. void setInput(const auto& inputLeft, const auto& inputRight) { - auto toView = [](const T& table) { - if constexpr (requires { table.template asStaticView<0>(); }) { - return table.template asStaticView<0>(); - } else { - return table; - } - }; - auto mergeVocab = [this](const T& table, - const LocalVocab*& currentVocab) { - if constexpr (requires { table.getLocalVocab(); }) { - currentVocab = &table.getLocalVocab(); - mergedVocab_.mergeWith(std::span{&table.getLocalVocab(), 1}); - } else { - currentVocab = nullptr; - } - }; - if (nextIndex_ != 0) { - AD_CORRECTNESS_CHECK(inputLeftAndRight_.has_value()); - flush(); - } + flushBeforeInputChange(); mergeVocab(inputLeft, currentVocabs_.at(0)); mergeVocab(inputRight, currentVocabs_.at(1)); inputLeftAndRight_ = std::array{toView(inputLeft), toView(inputRight)}; @@ -162,17 +179,8 @@ class AddCombinedRowToIdTable { // Only set the left input. After this it is only allowed to call // `addOptionalRow` and not `addRow` until `setInput` has been called again. void setOnlyLeftInputForOptionalJoin(const auto& inputLeft) { - auto toView = [](const T& table) { - if constexpr (requires { table.template asStaticView<0>(); }) { - return table.template asStaticView<0>(); - } else { - return table; - } - }; - if (nextIndex_ != 0) { - AD_CORRECTNESS_CHECK(inputLeftAndRight_.has_value()); - flush(); - } + flushBeforeInputChange(); + mergeVocab(inputLeft, currentVocabs_.at(0)); // The right input will be empty, but with the correct number of columns. inputLeftAndRight_ = std::array{ toView(inputLeft), @@ -339,6 +347,8 @@ class AddCombinedRowToIdTable { // The current `IdTable`s might still be active, so we have to merge the // local vocabs again if all other sets were moved-out. if (mergedVocab_.numSets() == 1) { + // Make sure to reset `mergedVocab_` so it is in a valid state again. + mergedVocab_ = LocalVocab{}; // Only merge non-null vocabs. auto range = currentVocabs_ | std::views::filter(toBool) | std::views::transform(dereference); diff --git a/test/AddCombinedRowToTableTest.cpp b/test/AddCombinedRowToTableTest.cpp index 63d3d5a8be..e659f3219d 100644 --- a/test/AddCombinedRowToTableTest.cpp +++ b/test/AddCombinedRowToTableTest.cpp @@ -198,3 +198,100 @@ TEST(AddCombinedRowToTable, flushDoesCheckCancellation) { cancellationHandle->cancel(ad_utility::CancellationState::MANUAL); EXPECT_THROW(adder.flush(), ad_utility::CancellationException); } + +namespace { +struct IdTableWithVocab { + IdTable idTable_; + LocalVocab localVocab_; + + const LocalVocab& getLocalVocab() const { return localVocab_; } + + template + IdTableView asStaticView() const { + return idTable_.asStaticView(); + } +}; + +using ad_utility::triple_component::Literal; + +Literal fromString(std::string_view string) { + return Literal::fromStringRepresentation(absl::StrCat("\"", string, "\"")); +} + +// _____________________________________________________________________________ +LocalVocab createVocabWithSingleString(std::string_view string) { + LocalVocab localVocab; + localVocab.getIndexAndAddIfNotContained(LocalVocabEntry{fromString(string)}); + return localVocab; +} + +// _____________________________________________________________________________ +bool vocabContainsString(const LocalVocab& vocab, std::string_view string) { + return ad_utility::contains(vocab.getAllWordsForTesting(), + LocalVocabEntry{fromString(string)}); +} +} // namespace + +// _____________________________________________________________________________ +TEST(AddCombinedRowToTable, verifyLocalVocabIsUpdatedCorrectly) { + auto outputTable = makeIdTableFromVector({}); + outputTable.setNumColumns(3); + std::vector localVocabs; + ad_utility::AddCombinedRowToIdTable adder{ + 1, std::move(outputTable), + std::make_shared>(), 1, + [&localVocabs](IdTable&, LocalVocab& localVocab) { + localVocabs.push_back(std::move(localVocab)); + }}; + + IdTableWithVocab input1{makeIdTableFromVector({{0, 1}}), + createVocabWithSingleString("a")}; + IdTableWithVocab input2{makeIdTableFromVector({{0, 2}}), + createVocabWithSingleString("b")}; + IdTableWithVocab input3{makeIdTableFromVector({{0, 3}}), + createVocabWithSingleString("c")}; + + using ::testing::SizeIs; + + adder.setInput(input1, input2); + adder.addRow(0, 0); + adder.addRow(0, 0); + EXPECT_THAT(localVocabs, SizeIs(1)); + adder.addRow(0, 0); + + adder.setInput(input2, input3); + EXPECT_THAT(localVocabs, SizeIs(2)); + adder.addRow(0, 0); + adder.flush(); + EXPECT_THAT(localVocabs, SizeIs(3)); + + adder.setOnlyLeftInputForOptionalJoin(input1); + adder.addOptionalRow(0); + adder.addOptionalRow(0); + EXPECT_THAT(localVocabs, SizeIs(4)); + adder.addOptionalRow(0); + + localVocabs.push_back(std::move(adder.localVocab())); + + ASSERT_THAT(localVocabs, SizeIs(5)); + + EXPECT_TRUE(vocabContainsString(localVocabs[0], "a")); + EXPECT_TRUE(vocabContainsString(localVocabs[0], "b")); + EXPECT_FALSE(vocabContainsString(localVocabs[0], "c")); + + EXPECT_TRUE(vocabContainsString(localVocabs[1], "a")); + EXPECT_TRUE(vocabContainsString(localVocabs[1], "b")); + EXPECT_FALSE(vocabContainsString(localVocabs[1], "c")); + + EXPECT_FALSE(vocabContainsString(localVocabs[2], "a")); + EXPECT_TRUE(vocabContainsString(localVocabs[2], "b")); + EXPECT_TRUE(vocabContainsString(localVocabs[2], "c")); + + EXPECT_TRUE(vocabContainsString(localVocabs[3], "a")); + EXPECT_FALSE(vocabContainsString(localVocabs[3], "b")); + EXPECT_FALSE(vocabContainsString(localVocabs[3], "c")); + + EXPECT_TRUE(vocabContainsString(localVocabs[4], "a")); + EXPECT_FALSE(vocabContainsString(localVocabs[4], "b")); + EXPECT_FALSE(vocabContainsString(localVocabs[4], "c")); +} From a8e3240ad6bc81b9cd3987968cebf481a99c05e0 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Sun, 24 Nov 2024 23:55:45 +0100 Subject: [PATCH 27/31] Add another unit test --- test/AddCombinedRowToTableTest.cpp | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/test/AddCombinedRowToTableTest.cpp b/test/AddCombinedRowToTableTest.cpp index e659f3219d..6b07b7c1df 100644 --- a/test/AddCombinedRowToTableTest.cpp +++ b/test/AddCombinedRowToTableTest.cpp @@ -295,3 +295,28 @@ TEST(AddCombinedRowToTable, verifyLocalVocabIsUpdatedCorrectly) { EXPECT_FALSE(vocabContainsString(localVocabs[4], "b")); EXPECT_FALSE(vocabContainsString(localVocabs[4], "c")); } + +// _____________________________________________________________________________ +TEST(AddCombinedRowToTable, verifyLocalVocabIsRetainedWhenNotMoving) { + auto outputTable = makeIdTableFromVector({}); + outputTable.setNumColumns(3); + ad_utility::AddCombinedRowToIdTable adder{ + 1, std::move(outputTable), + std::make_shared>(), 1}; + + IdTableWithVocab input1{makeIdTableFromVector({{0, 1}}), + createVocabWithSingleString("a")}; + IdTableWithVocab input2{makeIdTableFromVector({{0, 2}}), + createVocabWithSingleString("b")}; + + adder.setInput(input1, input2); + adder.addRow(0, 0); + adder.flush(); + adder.addRow(0, 0); + + LocalVocab localVocab = std::move(adder.localVocab()); + + EXPECT_TRUE(vocabContainsString(localVocab, "a")); + EXPECT_TRUE(vocabContainsString(localVocab, "b")); + EXPECT_THAT(localVocab.getAllWordsForTesting(), ::testing::SizeIs(2)); +} From 3f817fa48b16aeaa009a5b74049a1218af458daa Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Sun, 24 Nov 2024 23:57:57 +0100 Subject: [PATCH 28/31] Fix typo --- src/engine/AddCombinedRowToTable.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index 70a004cbb7..43e4a696cb 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -151,7 +151,7 @@ class AddCombinedRowToIdTable { // Flush remaining pending entries before changing the input. void flushBeforeInputChange() { - // Clear to avoid unecessary merge. + // Clear to avoid unnecessary merge. currentVocabs_ = {nullptr, nullptr}; if (nextIndex_ != 0) { AD_CORRECTNESS_CHECK(inputLeftAndRight_.has_value()); From ad0128a9d43537443ecc1e6804692ca479b74fe4 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Mon, 25 Nov 2024 00:08:35 +0100 Subject: [PATCH 29/31] Replace else case with assertion --- src/engine/AddCombinedRowToTable.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index 43e4a696cb..7e47cfbe7c 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -141,11 +141,10 @@ class AddCombinedRowToIdTable { // pointer reference to the proper value. template void mergeVocab(const T& table, const LocalVocab*& currentVocab) { + AD_CORRECTNESS_CHECK(currentVocab == nullptr); if constexpr (requires { table.getLocalVocab(); }) { currentVocab = &table.getLocalVocab(); mergedVocab_.mergeWith(std::span{&table.getLocalVocab(), 1}); - } else { - currentVocab = nullptr; } } From f88d4b838d3042f5984cedaac221ddbf14bcbfe6 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Mon, 25 Nov 2024 12:14:36 +0100 Subject: [PATCH 30/31] Check empty table instead of vocab --- src/engine/AddCombinedRowToTable.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/engine/AddCombinedRowToTable.h b/src/engine/AddCombinedRowToTable.h index 7e47cfbe7c..8c8939f64c 100644 --- a/src/engine/AddCombinedRowToTable.h +++ b/src/engine/AddCombinedRowToTable.h @@ -137,8 +137,8 @@ class AddCombinedRowToIdTable { } } - // Merge the vocab contained in `T` with the `mergedVocab_` and set the passed - // pointer reference to the proper value. + // Merge the local vocab contained in `T` with the `mergedVocab_` and set the + // passed pointer reference to that vocab. template void mergeVocab(const T& table, const LocalVocab*& currentVocab) { AD_CORRECTNESS_CHECK(currentVocab == nullptr); @@ -345,7 +345,7 @@ class AddCombinedRowToIdTable { std::invoke(blockwiseCallback_, result, mergedVocab_); // The current `IdTable`s might still be active, so we have to merge the // local vocabs again if all other sets were moved-out. - if (mergedVocab_.numSets() == 1) { + if (resultTable_.empty()) { // Make sure to reset `mergedVocab_` so it is in a valid state again. mergedVocab_ = LocalVocab{}; // Only merge non-null vocabs. From 66cb836e6c702233c5d108d673de6f4d3602fa33 Mon Sep 17 00:00:00 2001 From: RobinTF <83676088+RobinTF@users.noreply.github.com> Date: Mon, 25 Nov 2024 12:52:06 +0100 Subject: [PATCH 31/31] Fix unit test --- test/AddCombinedRowToTableTest.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/AddCombinedRowToTableTest.cpp b/test/AddCombinedRowToTableTest.cpp index 6b07b7c1df..c4a6b1c874 100644 --- a/test/AddCombinedRowToTableTest.cpp +++ b/test/AddCombinedRowToTableTest.cpp @@ -240,8 +240,11 @@ TEST(AddCombinedRowToTable, verifyLocalVocabIsUpdatedCorrectly) { ad_utility::AddCombinedRowToIdTable adder{ 1, std::move(outputTable), std::make_shared>(), 1, - [&localVocabs](IdTable&, LocalVocab& localVocab) { + [&localVocabs](IdTable& idTable, LocalVocab& localVocab) { localVocabs.push_back(std::move(localVocab)); + // Clear to trigger new merging of local vocabs, in practice + // `localVocab` is not altered without altering `idTable` as well. + idTable.clear(); }}; IdTableWithVocab input1{makeIdTableFromVector({{0, 1}}),