Skip to content

Commit

Permalink
This seems to work, but copies IdTables etc.
Browse files Browse the repository at this point in the history
Signed-off-by: Johannes Kalmbach <[email protected]>
  • Loading branch information
joka921 committed Jan 30, 2025
1 parent 0d4fa20 commit befc33d
Show file tree
Hide file tree
Showing 13 changed files with 234 additions and 70 deletions.
8 changes: 5 additions & 3 deletions src/engine/CheckUsePatternTrick.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,11 @@ bool isVariableContainedInGraphPatternOperation(
} else if constexpr (std::is_same_v<T, p::Service>) {
return ad_utility::contains(arg.visibleVariables_, variable);
} else {
static_assert(
std::is_same_v<T, p::TransPath> || std::is_same_v<T, p::PathQuery> ||
std::is_same_v<T, p::Describe> || std::is_same_v<T, p::SpatialQuery>);
static_assert(std::is_same_v<T, p::TransPath> ||
std::is_same_v<T, p::PathQuery> ||
std::is_same_v<T, p::Describe> ||
std::is_same_v<T, p::SpatialQuery> ||
std::is_same_v<T, p::NamedCachedQuery>);
// The `TransPath` is set up later in the query planning, when this
// function should not be called anymore.
AD_FAIL();
Expand Down
23 changes: 18 additions & 5 deletions src/engine/NamedQueryCache.h
Original file line number Diff line number Diff line change
Expand Up @@ -8,21 +8,34 @@
#include "util/Synchronized.h"

class NamedQueryCache {
public:
struct Value {
IdTable result_;
VariableToColumnMap varToColMap_;
std::vector<ColumnIndex> resultSortedOn_;
};
using Key = std::string;
using Value = std::shared_ptr<ValuesForTesting>;
using Cache =
ad_utility::HashMap<std::string, std::shared_ptr<ValuesForTesting>>;
using Cache = ad_utility::HashMap<std::string, Value>;

private:
ad_utility::Synchronized<Cache> cache_;

public:
void store(const Key& key, Value value) {
(*cache_.wlock())[key] = std::move(value);
(*cache_.wlock()).insert_or_assign(key, std::move(value));
}
Value get(const Key& key) {
const Value& get(const Key& key) {
auto l = cache_.wlock();
auto it = l->find(key);
// TODO<joka921> Proper error message.
AD_CONTRACT_CHECK(it != l->end());
return it->second;
}

std::shared_ptr<ValuesForTesting> getOperation(const Key& key,
QueryExecutionContext* ctx) {
const auto& [table, map, sortedOn] = get(key);
return std::make_shared<ValuesForTesting>(
ctx, std::make_shared<IdTable>(table.clone()), map);
}
};
12 changes: 12 additions & 0 deletions src/engine/QueryPlanner.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
#include <type_traits>
#include <variant>

#include "NamedQueryCache.h"
#include "backports/algorithm.h"
#include "engine/Bind.h"
#include "engine/CartesianProductJoin.h"
Expand Down Expand Up @@ -2408,6 +2409,8 @@ void QueryPlanner::GraphPatternPlanner::graphPatternOperationVisitor(Arg& arg) {
visitDescribe(arg);
} else if constexpr (std::is_same_v<T, p::SpatialQuery>) {
visitSpatialSearch(arg);
} else if constexpr (std::is_same_v<T, p::NamedCachedQuery>) {
visitNamedCachedQuery(arg);
} else {
static_assert(std::is_same_v<T, p::BasicGraphPattern>);
visitBasicGraphPattern(arg);
Expand Down Expand Up @@ -2581,6 +2584,15 @@ void QueryPlanner::GraphPatternPlanner::visitSpatialSearch(
visitGroupOptionalOrMinus(std::move(candidatesOut));
}

// _____________________________________________________________________________
void QueryPlanner::GraphPatternPlanner::visitNamedCachedQuery(
parsedQuery::NamedCachedQuery& arg) {
auto candidate = SubtreePlan{
planner_._qec, planner_._qec->namedQueryCache().getOperation(
arg.validateAndGetIdentifier(), planner_._qec)};
visitGroupOptionalOrMinus(std::vector{std::move(candidate)});
}

// _______________________________________________________________
void QueryPlanner::GraphPatternPlanner::visitUnion(parsedQuery::Union& arg) {
// TODO<joka921> here we could keep all the candidates, and create a
Expand Down
1 change: 1 addition & 0 deletions src/engine/QueryPlanner.h
Original file line number Diff line number Diff line change
Expand Up @@ -543,6 +543,7 @@ class QueryPlanner {
void visitTransitivePath(parsedQuery::TransPath& transitivePath);
void visitPathSearch(parsedQuery::PathQuery& config);
void visitSpatialSearch(parsedQuery::SpatialQuery& config);
void visitNamedCachedQuery(parsedQuery::NamedCachedQuery& config);
void visitUnion(parsedQuery::Union& un);
void visitSubquery(parsedQuery::Subquery& subquery);
void visitDescribe(parsedQuery::Describe& describe);
Expand Down
34 changes: 30 additions & 4 deletions src/engine/Server.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,11 @@ Awaitable<void> Server::process(
const auto parsedHttpRequest = parseHttpRequest(request);
const auto& parameters = parsedHttpRequest.parameters_;

LOG(INFO) << "Logging all the parameters" << std::endl;
for (const auto& [key, value] : parameters) {
LOG(INFO) << key << ":" << value.at(0) << std::endl;
}

// We always want to call `Server::checkParameter` with the same first
// parameter.
auto checkParameter = std::bind_front(&ad_utility::url_parser::checkParameter,
Expand Down Expand Up @@ -809,9 +814,17 @@ Awaitable<void> Server::processQuery(
// Do the query planning. This creates a `QueryExecutionTree`, which will
// then be used to process the query.
auto [pinSubtrees, pinResult] = determineResultPinning(params);
for (auto [key, value] : params) {
LOG(INFO) << "key : " << key << ": " << value.at(0) << std::endl;
}
std::optional<std::string> pinNamed =
ad_utility::url_parser::checkParameter(params, "pin-named-query", {});
LOG(INFO) << "Processing the following SPARQL query:"
<< (pinResult ? " [pin result]" : "")
<< (pinSubtrees ? " [pin subresults]" : "") << "\n"
<< (pinNamed ? absl::StrCat(" [pin named as ]", pinNamed.value())
: "")
<< "\n"
<< query.query_ << std::endl;
QueryExecutionContext qec(index_, &cache_, allocator_,
sortPerformanceEstimator_, &namedQueryCache_,
Expand Down Expand Up @@ -866,10 +879,23 @@ Awaitable<void> Server::processQuery(
qet.getRootOperation()->getLimit()._offset);
limitOffset._offset -= qet.getRootOperation()->getLimit()._offset;

// This actually processes the query and sends the result in the requested
// format.
co_await sendStreamableResponse(request, send, mediaType, plannedQuery, qet,
requestTimer, cancellationHandle);
if (pinNamed.has_value()) {
auto result = qet.getResult(false);
auto t =
NamedQueryCache::Value(result->idTable().clone(),
qet.getVariableColumns(), result->sortedBy());
qec.namedQueryCache().store(pinNamed.value(), std::move(t));

auto response = ad_utility::httpUtils::createOkResponse(
"successfully pinned the query result", request,
ad_utility::MediaType::textPlain);
co_await send(response);
} else {
// This actually processes the query and sends the result in the requested
// format.
co_await sendStreamableResponse(request, send, mediaType, plannedQuery, qet,
requestTimer, cancellationHandle);
}

// Print the runtime info. This needs to be done after the query
// was computed.
Expand Down
93 changes: 60 additions & 33 deletions src/engine/ValuesForTesting.h
Original file line number Diff line number Diff line change
Expand Up @@ -4,19 +4,24 @@

#pragma once

#include <util/TransparentFunctors.h>

#include "engine/Operation.h"
#include "engine/QueryExecutionContext.h"
#include "engine/Result.h"
#include "util/Algorithm.h"
#include "util/Random.h"

auto tables(auto& tables_) {
return ql::views::transform(tables_, ad_utility::dereference);
}
// An operation that yields a given `IdTable` as its result. It is used for
// unit testing purposes when we need to specify the subtrees of another
// operation.
class ValuesForTesting : public Operation {
private:
std::vector<IdTable> tables_;
std::vector<std::optional<Variable>> variables_;
std::vector<std::shared_ptr<const IdTable>> tables_;
VariableToColumnMap variables_;
bool supportsLimit_;
// Those can be manually overwritten for testing using the respective getters.
size_t sizeEstimate_;
Expand All @@ -27,16 +32,14 @@ class ValuesForTesting : public Operation {
// Create an operation that has as its result the given `table` and the given
// `variables`. The number of variables must be equal to the number
// of columns in the table.
explicit ValuesForTesting(QueryExecutionContext* ctx, IdTable table,
std::vector<std::optional<Variable>> variables,
bool supportsLimit = false,
std::vector<ColumnIndex> sortedColumns = {},
LocalVocab localVocab = LocalVocab{},
std::optional<float> multiplicity = std::nullopt,
bool forceFullyMaterialized = false)
explicit ValuesForTesting(
QueryExecutionContext* ctx, IdTable table,
const std::vector<std::optional<Variable>>& variables,
bool supportsLimit = false, std::vector<ColumnIndex> sortedColumns = {},
LocalVocab localVocab = LocalVocab{},
std::optional<float> multiplicity = std::nullopt,
bool forceFullyMaterialized = false)
: Operation{ctx},
tables_{},
variables_{std::move(variables)},
supportsLimit_{supportsLimit},
sizeEstimate_{table.numRows()},
costEstimate_{table.numRows()},
Expand All @@ -45,33 +48,53 @@ class ValuesForTesting : public Operation {
multiplicity_{multiplicity},
forceFullyMaterialized_{forceFullyMaterialized} {
AD_CONTRACT_CHECK(variables_.size() == table.numColumns());
tables_.push_back(std::move(table));
tables_.push_back(std::make_shared<const IdTable>(std::move(table)));
variables_ = computeVarMapFromVector(variables);
}

ValuesForTesting(QueryExecutionContext* ctx,
std::shared_ptr<const IdTable> table,
VariableToColumnMap variables,
std::vector<ColumnIndex> sortedColumns = {},
LocalVocab localVocab = LocalVocab{})
: Operation{ctx},
tables_{std::move(table)},
variables_{std::move(variables)},
supportsLimit_{false},
sizeEstimate_{tables_.at(0)->numRows()},
costEstimate_{0},
resultSortedColumns_{std::move(sortedColumns)},
localVocab_{std::move(localVocab)},
multiplicity_{},
forceFullyMaterialized_{false} {}
explicit ValuesForTesting(QueryExecutionContext* ctx,
std::vector<IdTable> tables,
std::vector<IdTable> idTables,
std::vector<std::optional<Variable>> variables,
bool unlikelyToFitInCache = false,
std::vector<ColumnIndex> sortedColumns = {},
LocalVocab localVocab = LocalVocab{})
: Operation{ctx},
tables_{std::move(tables)},
variables_{std::move(variables)},
supportsLimit_{false},
sizeEstimate_{0},
costEstimate_{0},
unlikelyToFitInCache_{unlikelyToFitInCache},
resultSortedColumns_{std::move(sortedColumns)},
localVocab_{std::move(localVocab)},
multiplicity_{std::nullopt} {
AD_CONTRACT_CHECK(ql::ranges::all_of(tables_, [this](const IdTable& table) {
return variables_.size() == table.numColumns();
}));
for (auto& table : idTables) {
tables_.push_back(std::make_shared<const IdTable>(std::move(table)));
}
AD_CONTRACT_CHECK(
ql::ranges::all_of(tables(tables_), [this](const IdTable& table) {
return variables_.size() == table.numColumns();
}));
size_t totalRows = 0;
for (const IdTable& idTable : tables_) {
for (const IdTable& idTable : tables(tables_)) {
totalRows += idTable.numRows();
}
sizeEstimate_ = totalRows;
costEstimate_ = totalRows;
variables_ = computeVarMapFromVector(variables);
}

// Accessors for the estimates for manual testing.
Expand All @@ -85,7 +108,7 @@ class ValuesForTesting : public Operation {
AD_CORRECTNESS_CHECK(!supportsLimit_);
std::vector<IdTable> clones;
clones.reserve(tables_.size());
for (const IdTable& idTable : tables_) {
for (const IdTable& idTable : tables(tables_)) {
clones.push_back(idTable.clone());
}
auto generator = [](auto idTables,
Expand All @@ -98,15 +121,15 @@ class ValuesForTesting : public Operation {
}
std::optional<IdTable> optionalTable;
if (tables_.size() > 1) {
IdTable aggregateTable{tables_.at(0).numColumns(),
tables_.at(0).getAllocator()};
for (const IdTable& idTable : tables_) {
IdTable aggregateTable{tables(tables_)[0].numColumns(),
tables(tables_)[0].getAllocator()};
for (const IdTable& idTable : tables(tables_)) {
aggregateTable.insertAtEnd(idTable);
}
optionalTable = std::move(aggregateTable);
}
auto table = optionalTable.has_value() ? std::move(optionalTable).value()
: tables_.at(0).clone();
: tables(tables_)[0].clone();
if (supportsLimit_) {
table.erase(table.begin() + getLimit().upperBound(table.size()),
table.end());
Expand All @@ -128,13 +151,13 @@ class ValuesForTesting : public Operation {
std::stringstream str;
auto numRowsView = tables_ | ql::views::transform(&IdTable::numRows);
auto totalNumRows = std::reduce(numRowsView.begin(), numRowsView.end(), 0);
auto numCols = tables_.empty() ? 0 : tables_.at(0).numColumns();
auto numCols = tables_.empty() ? 0 : tables_.at(0)->numColumns();
str << "Values for testing with " << numCols << " columns and "
<< totalNumRows << " rows. ";
if (totalNumRows > 1000) {
str << ad_utility::FastRandomIntGenerator<int64_t>{}();
} else {
for (const IdTable& idTable : tables_) {
for (const IdTable& idTable : tables(tables_)) {
for (size_t i = 0; i < idTable.numColumns(); ++i) {
for (Id entry : idTable.getColumn(i)) {
str << entry << ' ';
Expand All @@ -154,7 +177,7 @@ class ValuesForTesting : public Operation {
size_t getResultWidth() const override {
// Assume a width of 1 if we have no tables and no other information to base
// it on because 0 would otherwise cause stuff to break.
return tables_.empty() ? 1 : tables_.at(0).numColumns();
return tables_.empty() ? 1 : tables_.at(0)->numColumns();
}

vector<ColumnIndex> resultSortedOn() const override {
Expand All @@ -179,27 +202,31 @@ class ValuesForTesting : public Operation {

bool knownEmptyResult() override {
return ql::ranges::all_of(
tables_, [](const IdTable& table) { return table.empty(); });
tables(tables_), [](const IdTable& table) { return table.empty(); });
}

private:
VariableToColumnMap computeVariableToColumnMap() const override {
VariableToColumnMap computeVarMapFromVector(
const std::vector<std::optional<Variable>>& vars) const {
VariableToColumnMap m;
for (auto i = ColumnIndex{0}; i < variables_.size(); ++i) {
if (!variables_.at(i).has_value()) {
for (auto i = ColumnIndex{0}; i < vars.size(); ++i) {
if (!vars.at(i).has_value()) {
continue;
}
bool containsUndef =
ql::ranges::any_of(tables_, [&i](const IdTable& table) {
ql::ranges::any_of(tables(tables_), [&i](const IdTable& table) {
return ql::ranges::any_of(table.getColumn(i),
[](Id id) { return id.isUndefined(); });
});
using enum ColumnIndexAndTypeInfo::UndefStatus;
m[variables_.at(i).value()] = ColumnIndexAndTypeInfo{
m[vars.at(i).value()] = ColumnIndexAndTypeInfo{
i, containsUndef ? PossiblyUndefined : AlwaysDefined};
}
return m;
}
VariableToColumnMap computeVariableToColumnMap() const override {
return variables_;
}

std::vector<ColumnIndex> resultSortedColumns_;
LocalVocab localVocab_;
Expand Down
4 changes: 4 additions & 0 deletions src/global/Constants.h
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,10 @@ constexpr inline std::string_view DEFAULT_GRAPH_IRI =
constexpr inline std::string_view QLEVER_INTERNAL_GRAPH_IRI =
makeQleverInternalIriConst<"internal-graph">();

constexpr inline std::string_view NAMED_CACHED_QUERY_PREFIX =
ad_utility::constexprStrCat<"<", QLEVER_INTERNAL_PREFIX_URL,
"named-cached-query-">();

constexpr inline std::pair<std::string_view, std::string_view> GEOF_PREFIX = {
"geof:", "http://www.opengis.net/def/function/geosparql/"};
constexpr inline std::pair<std::string_view, std::string_view> MATH_PREFIX = {
Expand Down
2 changes: 2 additions & 0 deletions src/parser/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ add_library(parser
Literal.cpp
LiteralOrIri.cpp
DatasetClauses.cpp
NamedCachedQuery.cpp
NamedCachedQuery.h
)
qlever_target_link_libraries(parser sparqlParser parserData sparqlExpressions rdfEscaping re2::re2 util engine index)

5 changes: 3 additions & 2 deletions src/parser/GraphPatternOperation.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
#include "engine/sparqlExpressions/SparqlExpressionPimpl.h"
#include "parser/DatasetClauses.h"
#include "parser/GraphPattern.h"
#include "parser/NamedCachedQuery.h"
#include "parser/PathQuery.h"
#include "parser/SpatialQuery.h"
#include "parser/TripleComponent.h"
Expand Down Expand Up @@ -178,8 +179,8 @@ struct Bind {
// class actually becomes `using GraphPatternOperation = std::variant<...>`
using GraphPatternOperationVariant =
std::variant<Optional, Union, Subquery, TransPath, Bind, BasicGraphPattern,
Values, Service, PathQuery, SpatialQuery, Minus,
GroupGraphPattern, Describe>;
Values, Service, PathQuery, SpatialQuery, NamedCachedQuery,
Minus, GroupGraphPattern, Describe>;
struct GraphPatternOperation
: public GraphPatternOperationVariant,
public VisitMixin<GraphPatternOperation, GraphPatternOperationVariant> {
Expand Down
Loading

0 comments on commit befc33d

Please sign in to comment.