diff --git a/src/engine/Operation.cpp b/src/engine/Operation.cpp index 686c897df3..9518bb46a7 100644 --- a/src/engine/Operation.cpp +++ b/src/engine/Operation.cpp @@ -122,12 +122,12 @@ shared_ptr Operation::getResult(bool isRoot, } }); auto computeLambda = [this, &timer] { - checkCancellation([this]() { return "Before " + getDescriptor(); }); + checkCancellation(); runtimeInfo().status_ = RuntimeInformation::Status::inProgress; signalQueryUpdate(); ResultTable result = computeResult(); - checkCancellation([this]() { return "After " + getDescriptor(); }); + checkCancellation(); // Compute the datatypes that occur in each column of the result. // Also assert, that if a column contains UNDEF values, then the // `mightContainUndef` flag for that columns is set. diff --git a/src/engine/Operation.h b/src/engine/Operation.h index 8f1e764e2d..662ed84f20 100644 --- a/src/engine/Operation.h +++ b/src/engine/Operation.h @@ -222,15 +222,9 @@ class Operation { // as lightweight as possible because of that. The `detailSupplier` allows to // pass a message to add to any potential exception that might be thrown. AD_ALWAYS_INLINE void checkCancellation( - const ad_utility::InvocableWithConvertibleReturnType< - std::string_view> auto& detailSupplier) const { - cancellationHandle_->throwIfCancelled(detailSupplier); - } - - // Same as checkCancellation, but with the descriptor of this operation - // as string. - AD_ALWAYS_INLINE void checkCancellation() const { - cancellationHandle_->throwIfCancelled(&Operation::getDescriptor, this); + ad_utility::source_location location = + ad_utility::source_location::current()) const { + cancellationHandle_->throwIfCancelled(location); } std::chrono::milliseconds remainingTime() const; diff --git a/src/engine/QueryPlanner.cpp b/src/engine/QueryPlanner.cpp index ead9fe7799..23eb78c528 100644 --- a/src/engine/QueryPlanner.cpp +++ b/src/engine/QueryPlanner.cpp @@ -1109,7 +1109,7 @@ vector QueryPlanner::merge( candidates[getPruningKey(plan, plan._qet->resultSortedOn())] .emplace_back(std::move(plan)); } - cancellationHandle_->throwIfCancelled("Creating join candidates"); + checkCancellation(); } } @@ -1126,7 +1126,7 @@ vector QueryPlanner::merge( (void)key; // silence unused warning size_t minIndex = findCheapestExecutionTree(value); prunedPlans.push_back(std::move(value[minIndex])); - cancellationHandle_->throwIfCancelled("Pruning candidates"); + checkCancellation(); } }; @@ -1330,7 +1330,7 @@ QueryPlanner::runDynamicProgrammingOnConnectedComponent( << std::endl; dpTab.emplace_back(vector()); for (size_t i = 1; i * 2 <= k; ++i) { - cancellationHandle_->throwIfCancelled("QueryPlanner producing plans"); + checkCancellation(); auto newPlans = merge(dpTab[i - 1], dpTab[k - i - 1], tg); dpTab[k - 1].insert(dpTab[k - 1].end(), newPlans.begin(), newPlans.end()); applyFiltersIfPossible(dpTab.back(), filters, false); @@ -1943,6 +1943,7 @@ std::vector QueryPlanner::QueryGraph::dfsForAllNodes() { } // _______________________________________________________________ -void QueryPlanner::checkCancellation() const { - cancellationHandle_->throwIfCancelled("Query planning"); +void QueryPlanner::checkCancellation( + ad_utility::source_location location) const { + cancellationHandle_->throwIfCancelled(location); } diff --git a/src/engine/QueryPlanner.h b/src/engine/QueryPlanner.h index cbc0c110a0..631157f07e 100644 --- a/src/engine/QueryPlanner.h +++ b/src/engine/QueryPlanner.h @@ -460,5 +460,6 @@ class QueryPlanner { /// Helper function to check if the assigned `cancellationHandle_` has /// been cancelled yet. - void checkCancellation() const; + void checkCancellation(ad_utility::source_location location = + ad_utility::source_location::current()) const; }; diff --git a/src/engine/Server.cpp b/src/engine/Server.cpp index f641072d1a..5538ea92b1 100644 --- a/src/engine/Server.cpp +++ b/src/engine/Server.cpp @@ -807,11 +807,11 @@ net::awaitable Server::parseAndPlan( [&query, &qec, enablePatternTrick = enablePatternTrick_, handle = std::move(handle), timeLimit]() mutable { auto pq = SparqlParser::parseQuery(query); - handle->throwIfCancelled("After parsing"); + handle->throwIfCancelled(); QueryPlanner qp(&qec, handle); qp.setEnablePatternTrick(enablePatternTrick); auto qet = qp.createExecutionTree(pq); - handle->throwIfCancelled("After query planning"); + handle->throwIfCancelled(); PlannedQuery plannedQuery{std::move(pq), std::move(qet)}; plannedQuery.queryExecutionTree_.getRootOperation() diff --git a/src/engine/sparqlExpressions/AggregateExpression.h b/src/engine/sparqlExpressions/AggregateExpression.h index 5dedf1b2dd..353583727c 100644 --- a/src/engine/sparqlExpressions/AggregateExpression.h +++ b/src/engine/sparqlExpressions/AggregateExpression.h @@ -114,13 +114,14 @@ class AggregateExpression : public SparqlExpression { // The operands *without* applying the `valueGetter`. auto operands = makeGenerator(std::forward(operand), inputSize, context); - - auto impl = [&valueGetter, context, &finalOperation, - &callFunction](auto&& inputs) { - auto checkCancellation = [context]() { - context->cancellationHandle_->throwIfCancelled( - "AggregateExpression evaluate on child operand"); - }; + auto checkCancellation = + [context](ad_utility::source_location location = + ad_utility::source_location::current()) { + context->cancellationHandle_->throwIfCancelled(location); + }; + + auto impl = [&valueGetter, context, &finalOperation, &callFunction, + &checkCancellation](auto&& inputs) { auto it = inputs.begin(); AD_CORRECTNESS_CHECK(it != inputs.end()); @@ -144,8 +145,7 @@ class AggregateExpression : public SparqlExpression { if (distinct) { auto uniqueValues = getUniqueElements(context, inputSize, std::move(operands)); - context->cancellationHandle_->throwIfCancelled( - "AggregateExpression after filtering unique values"); + checkCancellation(); return impl(std::move(uniqueValues)); } else { return impl(std::move(operands)); diff --git a/src/engine/sparqlExpressions/ConditionalExpressions.cpp b/src/engine/sparqlExpressions/ConditionalExpressions.cpp index 5c5c3c4bae..29e8e1d44c 100644 --- a/src/engine/sparqlExpressions/ConditionalExpressions.cpp +++ b/src/engine/sparqlExpressions/ConditionalExpressions.cpp @@ -49,14 +49,12 @@ class CoalesceExpression : public VariadicExpression { ad_utility::chunkedForLoop( 0, ctx->size(), [&unboundIndices](size_t i) { unboundIndices.push_back(i); }, - [ctx]() { - ctx->cancellationHandle_->throwIfCancelled("CoalesceExpression"); - }); + [ctx]() { ctx->cancellationHandle_->throwIfCancelled(); }); VectorWithMemoryLimit result{ctx->_allocator}; std::fill_n(std::back_inserter(result), ctx->size(), IdOrString{Id::makeUndefined()}); - ctx->cancellationHandle_->throwIfCancelled("CoalesceExpression"); + ctx->cancellationHandle_->throwIfCancelled(); auto isUnbound = [](const IdOrString& x) { return (std::holds_alternative(x) && @@ -83,10 +81,7 @@ class CoalesceExpression : public VariadicExpression { DISABLE_UNINITIALIZED_WARNINGS result[unboundIndices[idx]] = constantResult; }, - [ctx]() { - ctx->cancellationHandle_->throwIfCancelled( - "CoalesceExpression constant expression result"); - }); + [ctx]() { ctx->cancellationHandle_->throwIfCancelled(); }); }; ENABLE_UNINITIALIZED_WARNINGS @@ -125,10 +120,7 @@ class CoalesceExpression : public VariadicExpression { } ++generatorIterator; }, - [ctx]() { - ctx->cancellationHandle_->throwIfCancelled( - "CoalesceExpression vector expression result"); - }); + [ctx]() { ctx->cancellationHandle_->throwIfCancelled(); }); }; auto visitExpressionResult = [ @@ -150,7 +142,7 @@ class CoalesceExpression : public VariadicExpression { std::visit(visitExpressionResult, child->evaluate(ctx)); unboundIndices = std::move(nextUnboundIndices); nextUnboundIndices.clear(); - ctx->cancellationHandle_->throwIfCancelled("CoalesceExpression"); + ctx->cancellationHandle_->throwIfCancelled(); // Early stopping if no more unbound result remain. if (unboundIndices.empty()) { break; diff --git a/src/engine/sparqlExpressions/GroupConcatExpression.h b/src/engine/sparqlExpressions/GroupConcatExpression.h index d2778d1004..f6b985386b 100644 --- a/src/engine/sparqlExpressions/GroupConcatExpression.h +++ b/src/engine/sparqlExpressions/GroupConcatExpression.h @@ -40,14 +40,13 @@ class GroupConcatExpression : public SparqlExpression { } result.append(s.value()); } - context->cancellationHandle_->throwIfCancelled( - "GroupConcatExpression"); + context->cancellationHandle_->throwIfCancelled(); } }; auto generator = detail::makeGenerator(AD_FWD(el), context->size(), context); if (distinct_) { - context->cancellationHandle_->throwIfCancelled("GroupConcatExpression"); + context->cancellationHandle_->throwIfCancelled(); groupConcatImpl(detail::getUniqueElements(context, context->size(), std::move(generator))); } else { diff --git a/src/engine/sparqlExpressions/LiteralExpression.h b/src/engine/sparqlExpressions/LiteralExpression.h index 40973f3e5b..a040f855fd 100644 --- a/src/engine/sparqlExpressions/LiteralExpression.h +++ b/src/engine/sparqlExpressions/LiteralExpression.h @@ -50,7 +50,7 @@ class LiteralExpression : public SparqlExpression { auto ptrForCache = std::make_unique(result); ptrForCache.reset(std::atomic_exchange_explicit( &cachedResult_, ptrForCache.release(), std::memory_order_relaxed)); - context->cancellationHandle_->throwIfCancelled("LiteralExpression"); + context->cancellationHandle_->throwIfCancelled(); return result; }; if constexpr (std::is_same_v) { diff --git a/src/engine/sparqlExpressions/RandomExpression.h b/src/engine/sparqlExpressions/RandomExpression.h index ce97424f51..2f1afc9ef8 100644 --- a/src/engine/sparqlExpressions/RandomExpression.h +++ b/src/engine/sparqlExpressions/RandomExpression.h @@ -37,7 +37,7 @@ class RandomExpression : public SparqlExpression { result.push_back(Id::makeFromInt(randInt() >> Id::numDatatypeBits)); }, [context]() { - context->cancellationHandle_->throwIfCancelled("RandomExpression"); + context->cancellationHandle_->throwIfCancelled(); }); return result; } diff --git a/src/engine/sparqlExpressions/RegexExpression.cpp b/src/engine/sparqlExpressions/RegexExpression.cpp index 4647eaa566..3b4e20f995 100644 --- a/src/engine/sparqlExpressions/RegexExpression.cpp +++ b/src/engine/sparqlExpressions/RegexExpression.cpp @@ -320,8 +320,9 @@ auto RegexExpression::getEstimatesForFilterExpression( // ____________________________________________________________________________ void RegexExpression::checkCancellation( - const sparqlExpression::EvaluationContext* context) { - context->cancellationHandle_->throwIfCancelled("RegexExpression"); + const sparqlExpression::EvaluationContext* context, + ad_utility::source_location location) { + context->cancellationHandle_->throwIfCancelled(location); } } // namespace sparqlExpression diff --git a/src/engine/sparqlExpressions/RegexExpression.h b/src/engine/sparqlExpressions/RegexExpression.h index 2b00c9153f..d258b011bc 100644 --- a/src/engine/sparqlExpressions/RegexExpression.h +++ b/src/engine/sparqlExpressions/RegexExpression.h @@ -57,7 +57,9 @@ class RegexExpression : public SparqlExpression { /// Helper function to check if the `CancellationHandle` of the passed /// `EvaluationContext` has been cancelled. static void checkCancellation( - const sparqlExpression::EvaluationContext* context); + const sparqlExpression::EvaluationContext* context, + ad_utility::source_location location = + ad_utility::source_location::current()); }; namespace detail { // Check if `regex` is a prefix regex which means that it starts with `^` and diff --git a/src/engine/sparqlExpressions/RelationalExpressions.cpp b/src/engine/sparqlExpressions/RelationalExpressions.cpp index dacfd4ed37..424f5670b8 100644 --- a/src/engine/sparqlExpressions/RelationalExpressions.cpp +++ b/src/engine/sparqlExpressions/RelationalExpressions.cpp @@ -155,7 +155,7 @@ requires AreComparable ExpressionResult evaluateRelationalExpression( context); } } - context->cancellationHandle_->throwIfCancelled("RelationalExpression"); + context->cancellationHandle_->throwIfCancelled(); return std::nullopt; }; std::optional resultFromBinarySearch; @@ -196,7 +196,7 @@ requires AreComparable ExpressionResult evaluateRelationalExpression( ad_utility::visitWithVariantsAndParameters(impl, base(*itA), base(*itB)); ++itA; ++itB; - context->cancellationHandle_->throwIfCancelled("RelationalExpression"); + context->cancellationHandle_->throwIfCancelled(); } if constexpr (resultIsConstant) { diff --git a/src/engine/sparqlExpressions/SparqlExpressionGenerators.h b/src/engine/sparqlExpressions/SparqlExpressionGenerators.h index c78d52531e..6197cf0e32 100644 --- a/src/engine/sparqlExpressions/SparqlExpressionGenerators.h +++ b/src/engine/sparqlExpressions/SparqlExpressionGenerators.h @@ -108,7 +108,7 @@ inline auto valueGetterGenerator = [](I && i) requires std::invocable { - context->cancellationHandle_->throwIfCancelled("Value getter generator"); + context->cancellationHandle_->throwIfCancelled(); return valueGetter(AD_FWD(i), context); }; return makeGenerator(std::forward(input), numElements, context, transformation); diff --git a/src/engine/sparqlExpressions/StringExpressions.cpp b/src/engine/sparqlExpressions/StringExpressions.cpp index fac0d4f6ea..43c3272dc4 100644 --- a/src/engine/sparqlExpressions/StringExpressions.cpp +++ b/src/engine/sparqlExpressions/StringExpressions.cpp @@ -341,11 +341,11 @@ class ConcatExpression : public detail::VariadicExpression { str.has_value()) { resultAsVec[i].append(str.value()); } - ctx->cancellationHandle_->throwIfCancelled("ConcatExpression"); + ctx->cancellationHandle_->throwIfCancelled(); ++i; } } - ctx->cancellationHandle_->throwIfCancelled("ConcatExpression"); + ctx->cancellationHandle_->throwIfCancelled(); }; std::ranges::for_each( childrenVec(), [&ctx, &visitSingleExpressionResult](const auto& child) { diff --git a/src/index/CompressedRelation.cpp b/src/index/CompressedRelation.cpp index 07703eec16..87f1b3f26c 100644 --- a/src/index/CompressedRelation.cpp +++ b/src/index/CompressedRelation.cpp @@ -40,7 +40,7 @@ CompressedRelationReader::asyncParallelBlockGenerator( std::mutex blockIteratorMutex; auto readAndDecompressBlock = [&]() -> std::optional> { - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); std::unique_lock lock{blockIteratorMutex}; if (blockIterator == endBlock) { return std::nullopt; @@ -77,7 +77,7 @@ CompressedRelationReader::asyncParallelBlockGenerator( queueSize, numThreads, readAndDecompressBlock); for (IdTable& block : queue) { popTimer.stop(); - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); ++details.numBlocksRead_; details.numElementsRead_ += block.numRows(); co_yield block; @@ -110,7 +110,7 @@ CompressedRelationReader::IdTableGenerator CompressedRelationReader::lazyScan( auto getIncompleteBlock = [&](auto it) { auto result = readPossiblyIncompleteBlock(metadata, col1Id, *it, std::ref(details), columnIndices); - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); return result; }; @@ -327,13 +327,13 @@ IdTable CompressedRelationReader::scan( firstBlockResult = readIncompleteBlock(*beginBlock); totalResultSize += firstBlockResult.value().size(); ++beginBlock; - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); } if (beginBlock < endBlock) { lastBlockResult = readIncompleteBlock(*(endBlock - 1)); totalResultSize += lastBlockResult.value().size(); endBlock--; - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); } // Determine the total size of the result. @@ -343,7 +343,7 @@ IdTable CompressedRelationReader::scan( return count + block.numRows_; }); result.resize(totalResultSize); - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); size_t rowIndexOfNextBlockStart = 0; // Lambda that appends a possibly incomplete block (the first or last block) @@ -365,7 +365,7 @@ IdTable CompressedRelationReader::scan( }; addIncompleteBlockIfExists(firstBlockResult); - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); // Insert the complete blocks from the middle in parallel if (beginBlock < endBlock) { @@ -394,7 +394,7 @@ IdTable CompressedRelationReader::scan( // block in parallel #pragma omp task { - if (!cancellationHandle->isCancelled("CompressedRelation scan")) { + if (!cancellationHandle->isCancelled()) { decompressLambda(); } } @@ -403,11 +403,11 @@ IdTable CompressedRelationReader::scan( rowIndexOfNextBlockStart += block.numRows_; } // end of parallel region } - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); // Add the last block. addIncompleteBlockIfExists(lastBlockResult); AD_CORRECTNESS_CHECK(rowIndexOfNextBlockStart == result.size()); - checkCancellation(cancellationHandle); + cancellationHandle->throwIfCancelled(); return result; } diff --git a/src/index/CompressedRelation.h b/src/index/CompressedRelation.h index 67bd946610..a4149e1b0d 100644 --- a/src/index/CompressedRelation.h +++ b/src/index/CompressedRelation.h @@ -531,15 +531,6 @@ class CompressedRelationReader { auto beginBlock, auto endBlock, ColumnIndices columnIndices, ad_utility::SharedCancellationHandle cancellationHandle) const; - // A helper function to abstract away the timeout check: - static void checkCancellation( - const ad_utility::SharedCancellationHandle& cancellationHandle) { - // Not really expensive but since this should be called - // very often, try to avoid any extra checks. - AD_EXPENSIVE_CHECK(cancellationHandle); - cancellationHandle->throwIfCancelled("IndexScan"sv); - } - // Return a vector that consists of the concatenation of `baseColumns` and // `additionalColumns` static std::vector prepareColumnIndices( diff --git a/src/index/IndexImpl.cpp b/src/index/IndexImpl.cpp index 7e05c0ba5e..3cd0ffad4b 100644 --- a/src/index/IndexImpl.cpp +++ b/src/index/IndexImpl.cpp @@ -1455,7 +1455,7 @@ IdTable IndexImpl::scan( : std::nullopt; if (!col0Id.has_value() || (col1String.has_value() && !col1Id.has_value())) { size_t numColumns = col1String.has_value() ? 1 : 2; - cancellationHandle->throwIfCancelled("IndexImpl scan"); + cancellationHandle->throwIfCancelled(); return IdTable{numColumns, allocator_}; } return scan(col0Id.value(), col1Id, permutation, additionalColumns, diff --git a/src/index/Permutation.cpp b/src/index/Permutation.cpp index 3897dd7556..d0b47b7312 100644 --- a/src/index/Permutation.cpp +++ b/src/index/Permutation.cpp @@ -49,7 +49,7 @@ IdTable Permutation::scan( if (!meta_.col0IdExists(col0Id)) { size_t numColumns = col1Id.has_value() ? 1 : 2; - cancellationHandle->throwIfCancelled("Permutation scan"); + cancellationHandle->throwIfCancelled(); return IdTable{numColumns, reader().allocator()}; } const auto& metaData = meta_.getMetaData(col0Id); diff --git a/src/index/TriplesView.h b/src/index/TriplesView.h index fca09fb371..fe525140c6 100644 --- a/src/index/TriplesView.h +++ b/src/index/TriplesView.h @@ -84,7 +84,7 @@ cppcoro::generator> TriplesView( co_yield triple; } } - cancellationHandle->throwIfCancelled("TriplesView generator"); + cancellationHandle->throwIfCancelled(); } } } diff --git a/src/util/CancellationHandle.h b/src/util/CancellationHandle.h index 79fe48879a..94d01e52cb 100644 --- a/src/util/CancellationHandle.h +++ b/src/util/CancellationHandle.h @@ -18,6 +18,7 @@ #include "util/Exception.h" #include "util/Log.h" #include "util/ParseableDuration.h" +#include "util/SourceLocation.h" #include "util/TypeTraits.h" #include "util/jthread.h" @@ -81,12 +82,13 @@ struct PseudoStopToken { class CancellationException : public std::runtime_error { public: using std::runtime_error::runtime_error; - CancellationException(CancellationState reason, std::string_view details) - : std::runtime_error{absl::StrCat("Cancelled due to ", - reason == CancellationState::TIMEOUT - ? "timeout" - : "manual cancellation", - ". Stage: ", details)} { + CancellationException(CancellationState reason, + ad_utility::source_location location) + : std::runtime_error{absl::StrCat( + "Cancelled due to ", + reason == CancellationState::TIMEOUT ? "timeout" + : "manual cancellation", + " at ", location.file_name(), ":", location.line())} { AD_CONTRACT_CHECK(detail::isCancelled(reason)); } }; @@ -123,11 +125,9 @@ class CancellationHandle { /// in the console that would otherwise be triggered by the watchdog. /// NOTE: The parameter state is expected to be one of `CHECK_WINDOW_MISSED` /// or `WAITING_FOR_CHECK`, otherwise it will violate the correctness check. - template void pleaseWatchDog(CancellationState state, - const InvocableWithConvertibleReturnType< - std::string_view, ArgTypes...> auto& detailSupplier, - ArgTypes&&... argTypes) requires WatchDogEnabled { + ad_utility::source_location location) + requires WatchDogEnabled { using DurationType = std::remove_const_t; AD_CORRECTNESS_CHECK(!detail::isCancelled(state) && @@ -148,9 +148,8 @@ class CancellationHandle { DESIRED_CANCELLATION_CHECK_INTERVAL} << ", should be at most " << ParseableDuration{DESIRED_CANCELLATION_CHECK_INTERVAL} - << ". Stage: " - << std::invoke(detailSupplier, AD_FWD(argTypes)...) - << std::endl; + << ". Checked at " << location.file_name() << ":" + << location.line() << std::endl; } break; } @@ -176,24 +175,13 @@ class CancellationHandle { /// throw. No-op if this instance is already in a cancelled state. void cancel(CancellationState reason); - /// Overload for static exception messages, make sure the string is a constant - /// expression, or computed in advance. If that's not the case do not use - /// this overload and use the overload that takes a callable that creates - /// the exception message (see below). - AD_ALWAYS_INLINE void throwIfCancelled(std::string_view detail) { - throwIfCancelled(std::identity{}, detail); - } - /// Throw an `CancellationException` when this handle has been cancelled. Do - /// nothing otherwise. The arg types are passed to the `detailSupplier` only - /// if an exception is about to be thrown. If no exception is thrown, - /// `detailSupplier` will not be evaluated. If `WatchDogEnabled` is true, - /// this will log a warning if this check is not called frequently enough. - template + /// nothing otherwise. If `WatchDogEnabled` is true, this will log a warning + /// if this check is not called frequently enough. It will contain the + /// filename and line of the caller of this method. AD_ALWAYS_INLINE void throwIfCancelled( - [[maybe_unused]] const InvocableWithConvertibleReturnType< - std::string_view, ArgTypes...> auto& detailSupplier, - [[maybe_unused]] ArgTypes&&... argTypes) { + [[maybe_unused]] ad_utility::source_location location = + ad_utility::source_location::current()) { if constexpr (CancellationEnabled) { auto state = cancellationState_.load(std::memory_order_relaxed); if (state == CancellationState::NOT_CANCELLED) [[likely]] { @@ -201,12 +189,11 @@ class CancellationHandle { } if constexpr (WatchDogEnabled) { if (!detail::isCancelled(state)) { - pleaseWatchDog(state, detailSupplier, AD_FWD(argTypes)...); + pleaseWatchDog(state, location); return; } } - throw CancellationException{ - state, std::invoke(detailSupplier, AD_FWD(argTypes)...)}; + throw CancellationException{state, location}; } } @@ -214,14 +201,16 @@ class CancellationHandle { /// otherwise. Note: Make sure to not use this value to set any other atomic /// value with relaxed memory ordering, as this may lead to out-of-thin-air /// values. If the watchdog is enabled, this will please it and print - /// a warning with the passed detail string. - AD_ALWAYS_INLINE bool isCancelled(std::string_view details) { + /// a warning with the filename and line of the caller. + AD_ALWAYS_INLINE bool isCancelled( + [[maybe_unused]] ad_utility::source_location location = + ad_utility::source_location::current()) { if constexpr (CancellationEnabled) { auto state = cancellationState_.load(std::memory_order_relaxed); bool isCancelled = detail::isCancelled(state); if constexpr (WatchDogEnabled) { if (!isCancelled && state != CancellationState::NOT_CANCELLED) { - pleaseWatchDog(state, std::identity{}, details); + pleaseWatchDog(state, location); } } return isCancelled; @@ -238,7 +227,8 @@ class CancellationHandle { /// If this `CancellationHandle` is not cancelled, reset the internal /// `cancellationState_` to `CancellationState::NOT_CANCELED`. - /// Useful to ignore expected gaps in the execution flow. + /// Useful to ignore expected gaps in the execution flow, but typically + /// indicates that there's code that cannot be interrupted, so use with care! void resetWatchDogState(); // Explicit move-semantics diff --git a/test/CancellationHandleTest.cpp b/test/CancellationHandleTest.cpp index 7b0683dc62..27910932ed 100644 --- a/test/CancellationHandleTest.cpp +++ b/test/CancellationHandleTest.cpp @@ -39,7 +39,8 @@ TEST(CancellationHandle, verifyConstructorMessageIsPassed) { // _____________________________________________________________________________ TEST(CancellationHandle, verifyConstructorDoesNotAcceptNoReason) { - EXPECT_THROW(CancellationException exception(NOT_CANCELLED, ""), + EXPECT_THROW(CancellationException exception( + NOT_CANCELLED, ad_utility::source_location::current()), ad_utility::Exception); } @@ -48,9 +49,9 @@ TEST(CancellationHandle, verifyConstructorDoesNotAcceptNoReason) { TYPED_TEST(CancellationHandleFixture, verifyNotCancelledByDefault) { auto& handle = this->handle_; - EXPECT_FALSE(handle.isCancelled("")); - EXPECT_NO_THROW(handle.throwIfCancelled("")); - EXPECT_NO_THROW(handle.throwIfCancelled([]() { return ""; })); + EXPECT_FALSE(handle.isCancelled()); + EXPECT_NO_THROW(handle.throwIfCancelled()); + EXPECT_NO_THROW(handle.throwIfCancelled()); } // _____________________________________________________________________________ @@ -61,22 +62,20 @@ TYPED_TEST(CancellationHandleFixture, verifyCancelWithWrongReasonThrows) { } // _____________________________________________________________________________ - -auto detail = "Some Detail"; +ad_utility::source_location location = ad_utility::source_location::current(); TYPED_TEST(CancellationHandleFixture, verifyTimeoutCancellationWorks) { auto& handle = this->handle_; handle.cancel(TIMEOUT); - auto timeoutMessageMatcher = AllOf(HasSubstr(detail), HasSubstr("timeout")); - EXPECT_TRUE(handle.isCancelled("")); - AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE(handle.throwIfCancelled(detail), + auto timeoutMessageMatcher = + AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line())), HasSubstr("timeout")); + EXPECT_TRUE(handle.isCancelled()); + AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE(handle.throwIfCancelled(location), timeoutMessageMatcher, CancellationException); - AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE( - handle.throwIfCancelled([]() { return detail; }), timeoutMessageMatcher, - CancellationException); } // _____________________________________________________________________________ @@ -87,14 +86,13 @@ TYPED_TEST(CancellationHandleFixture, verifyManualCancellationWorks) { handle.cancel(MANUAL); auto cancellationMessageMatcher = - AllOf(HasSubstr(detail), HasSubstr("manual cancellation")); - EXPECT_TRUE(handle.isCancelled("")); - AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE(handle.throwIfCancelled(detail), + AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line())), + HasSubstr("manual cancellation")); + EXPECT_TRUE(handle.isCancelled()); + AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE(handle.throwIfCancelled(location), cancellationMessageMatcher, CancellationException); - AD_EXPECT_THROW_WITH_MESSAGE_AND_TYPE( - handle.throwIfCancelled([]() { return detail; }), - cancellationMessageMatcher, CancellationException); } // _____________________________________________________________________________ @@ -112,11 +110,11 @@ TYPED_TEST(CancellationHandleFixture, { auto end = std::chrono::steady_clock::now() + 100ms; while (std::chrono::steady_clock::now() < end) { - handle.throwIfCancelled("Some Detail"); + handle.throwIfCancelled(); } }, CancellationException); - EXPECT_TRUE(handle.isCancelled("")); + EXPECT_TRUE(handle.isCancelled()); } // _____________________________________________________________________________ @@ -239,11 +237,11 @@ TEST(CancellationHandle, verifyCheckDoesPleaseWatchDog) { CancellationHandle handle; handle.cancellationState_ = WAITING_FOR_CHECK; - EXPECT_NO_THROW(handle.throwIfCancelled("")); + EXPECT_NO_THROW(handle.throwIfCancelled()); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); handle.cancellationState_ = CHECK_WINDOW_MISSED; - EXPECT_NO_THROW(handle.throwIfCancelled("")); + EXPECT_NO_THROW(handle.throwIfCancelled()); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); } @@ -253,11 +251,11 @@ TEST(CancellationHandle, verifyCheckDoesNotOverrideCancelledState) { CancellationHandle handle; handle.cancellationState_ = MANUAL; - EXPECT_THROW(handle.throwIfCancelled(""), CancellationException); + EXPECT_THROW(handle.throwIfCancelled(), CancellationException); EXPECT_EQ(handle.cancellationState_, MANUAL); handle.cancellationState_ = TIMEOUT; - EXPECT_THROW(handle.throwIfCancelled(""), CancellationException); + EXPECT_THROW(handle.throwIfCancelled(), CancellationException); EXPECT_EQ(handle.cancellationState_, TIMEOUT); } @@ -277,12 +275,13 @@ TEST(CancellationHandle, verifyCheckAfterDeadlineMissDoesReportProperly) { handle.startTimeoutWindow_ = std::chrono::steady_clock::now(); handle.cancellationState_ = CHECK_WINDOW_MISSED; - EXPECT_NO_THROW(handle.throwIfCancelled("my-detail")); + EXPECT_NO_THROW(handle.throwIfCancelled(location)); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); EXPECT_THAT( std::move(testStream).str(), - AllOf(HasSubstr("my-detail"), + AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line())), HasSubstr(ParseableDuration{DESIRED_CANCELLATION_CHECK_INTERVAL} .toString()), // Check for small miss window @@ -309,40 +308,44 @@ TEST(CancellationHandle, verifyPleaseWatchDogReportsOnlyWhenNecessary) { handle.cancellationState_ = CHECK_WINDOW_MISSED; // The first call should trigger a log - handle.pleaseWatchDog(CHECK_WINDOW_MISSED, std::identity{}, "my-detail"); + handle.pleaseWatchDog(CHECK_WINDOW_MISSED, location); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); - EXPECT_THAT(std::move(testStream).str(), HasSubstr("my-detail")); + EXPECT_THAT(std::move(testStream).str(), + AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line())))); testStream.str(""); // The second call should not trigger a log because the state has already // been reset - handle.pleaseWatchDog(CHECK_WINDOW_MISSED, std::identity{}, "other-detail"); + handle.pleaseWatchDog(CHECK_WINDOW_MISSED, location); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); - EXPECT_THAT(std::move(testStream).str(), Not(HasSubstr("other-detail"))); + EXPECT_THAT(std::move(testStream).str(), + Not(AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line()))))); handle.cancellationState_ = CHECK_WINDOW_MISSED; testStream.str(""); // WAITING_FOR_CHECK should not trigger a log - handle.pleaseWatchDog(WAITING_FOR_CHECK, std::identity{}, "my-detail"); + handle.pleaseWatchDog(WAITING_FOR_CHECK, location); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); - EXPECT_THAT(std::move(testStream).str(), Not(HasSubstr("my-detail"))); + EXPECT_THAT(std::move(testStream).str(), + Not(AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line()))))); } // _____________________________________________________________________________ TEST(CancellationHandle, verifyPleaseWatchDogDoesNotAcceptInvalidState) { CancellationHandle handle; - EXPECT_THROW(handle.pleaseWatchDog(NOT_CANCELLED, std::identity{}, ""), - ad_utility::Exception); - EXPECT_THROW(handle.pleaseWatchDog(MANUAL, std::identity{}, ""), - ad_utility::Exception); - EXPECT_THROW(handle.pleaseWatchDog(TIMEOUT, std::identity{}, ""), + EXPECT_THROW(handle.pleaseWatchDog(NOT_CANCELLED, location), ad_utility::Exception); + EXPECT_THROW(handle.pleaseWatchDog(MANUAL, location), ad_utility::Exception); + EXPECT_THROW(handle.pleaseWatchDog(TIMEOUT, location), ad_utility::Exception); } // _____________________________________________________________________________ @@ -362,18 +365,22 @@ TEST(CancellationHandle, verifyIsCancelledDoesPleaseWatchDog) { handle.startTimeoutWindow_ = std::chrono::steady_clock::now(); handle.cancellationState_ = CHECK_WINDOW_MISSED; - handle.isCancelled("my-detail"); + handle.isCancelled(location); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); - EXPECT_THAT(std::move(testStream).str(), HasSubstr("my-detail")); + EXPECT_THAT(std::move(testStream).str(), + AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line())))); handle.cancellationState_ = WAITING_FOR_CHECK; testStream.str(""); - handle.isCancelled("my-detail"); + handle.isCancelled(location); EXPECT_EQ(handle.cancellationState_, NOT_CANCELLED); - EXPECT_THAT(std::move(testStream).str(), Not(HasSubstr("my-detail"))); + EXPECT_THAT(std::move(testStream).str(), + Not(AllOf(HasSubstr(location.file_name()), + HasSubstr(std::to_string(location.line()))))); } // _____________________________________________________________________________ @@ -381,8 +388,8 @@ TEST(CancellationHandle, verifyIsCancelledDoesPleaseWatchDog) { TEST(CancellationHandle, expectDisabledHandleIsAlwaysFalse) { CancellationHandle handle; - EXPECT_FALSE(handle.isCancelled("")); - EXPECT_NO_THROW(handle.throwIfCancelled("Abc")); + EXPECT_FALSE(handle.isCancelled()); + EXPECT_NO_THROW(handle.throwIfCancelled()); } consteval bool isMemberFunction([[maybe_unused]] auto funcPtr) {