Skip to content

Commit

Permalink
Replace with client context (#3121)
Browse files Browse the repository at this point in the history
  • Loading branch information
hououou committed Mar 23, 2024
1 parent 365815b commit 3a6bd7e
Show file tree
Hide file tree
Showing 55 changed files with 241 additions and 223 deletions.
2 changes: 1 addition & 1 deletion src/binder/bind/bind_updating_clause.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ std::vector<BoundInsertInfo> Binder::bindInsertInfos(
const QueryGraphCollection& queryGraphCollection, const expression_set& nodeRelScope_) {
auto nodeRelScope = nodeRelScope_;
std::vector<BoundInsertInfo> result;
auto analyzer = QueryGraphLabelAnalyzer(*clientContext->getCatalog(), *clientContext);
auto analyzer = QueryGraphLabelAnalyzer(*clientContext);
for (auto i = 0u; i < queryGraphCollection.getNumQueryGraphs(); ++i) {
auto queryGraph = queryGraphCollection.getQueryGraph(i);
// Ensure query graph does not violate declared schema.
Expand Down
2 changes: 1 addition & 1 deletion src/binder/binder.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ std::unique_ptr<BoundStatement> Binder::bind(const Statement& statement) {
KU_UNREACHABLE;
}
}
BoundStatementRewriter::rewrite(*boundStatement, *clientContext->getCatalog(), *clientContext);
BoundStatementRewriter::rewrite(*boundStatement, *clientContext);
return boundStatement;
}

Expand Down
6 changes: 3 additions & 3 deletions src/binder/bound_statement_rewriter.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -7,12 +7,12 @@
namespace kuzu {
namespace binder {

void BoundStatementRewriter::rewrite(BoundStatement& boundStatement,
const catalog::Catalog& catalog, const main::ClientContext& clientContext) {
void BoundStatementRewriter::rewrite(
BoundStatement& boundStatement, const main::ClientContext& clientContext) {
auto withClauseProjectionRewriter = WithClauseProjectionRewriter();
withClauseProjectionRewriter.visitUnsafe(boundStatement);

auto matchClausePatternLabelRewriter = MatchClausePatternLabelRewriter(catalog, clientContext);
auto matchClausePatternLabelRewriter = MatchClausePatternLabelRewriter(clientContext);
matchClausePatternLabelRewriter.visit(boundStatement);

auto defaultTypeSolver = DefaultTypeSolver();
Expand Down
21 changes: 12 additions & 9 deletions src/binder/query/query_graph_label_analyzer.cpp
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
#include "binder/query/query_graph_label_analyzer.h"

#include "catalog/catalog.h"
#include "catalog/catalog_entry/rel_table_catalog_entry.h"
#include "common/cast.h"
#include "common/exception/binder.h"
Expand All @@ -22,6 +23,7 @@ void QueryGraphLabelAnalyzer::pruneLabel(const QueryGraph& graph) {
}

void QueryGraphLabelAnalyzer::pruneNode(const QueryGraph& graph, NodeExpression& node) {
auto catalog = clientContext.getCatalog();
for (auto i = 0u; i < graph.getNumQueryRels(); ++i) {
auto queryRel = graph.getQueryRel(i);
if (queryRel->isRecursive()) {
Expand All @@ -36,13 +38,13 @@ void QueryGraphLabelAnalyzer::pruneNode(const QueryGraph& graph, NodeExpression&
if (isSrcConnect || isDstConnect) {
for (auto relTableID : queryRel->getTableIDs()) {
auto relTableSchema = ku_dynamic_cast<CatalogEntry*, RelTableCatalogEntry*>(
catalog.getTableCatalogEntry(tx, relTableID));
catalog->getTableCatalogEntry(tx, relTableID));
auto srcTableID = relTableSchema->getSrcTableID();
auto dstTableID = relTableSchema->getDstTableID();
candidates.insert(srcTableID);
candidates.insert(dstTableID);
auto srcTableSchema = catalog.getTableCatalogEntry(tx, srcTableID);
auto dstTableSchema = catalog.getTableCatalogEntry(tx, dstTableID);
auto srcTableSchema = catalog->getTableCatalogEntry(tx, srcTableID);
auto dstTableSchema = catalog->getTableCatalogEntry(tx, dstTableID);
candidateNamesSet.insert(srcTableSchema->getName());
candidateNamesSet.insert(dstTableSchema->getName());
}
Expand All @@ -51,19 +53,19 @@ void QueryGraphLabelAnalyzer::pruneNode(const QueryGraph& graph, NodeExpression&
if (isSrcConnect) {
for (auto relTableID : queryRel->getTableIDs()) {
auto relTableSchema = ku_dynamic_cast<CatalogEntry*, RelTableCatalogEntry*>(
catalog.getTableCatalogEntry(tx, relTableID));
catalog->getTableCatalogEntry(tx, relTableID));
auto srcTableID = relTableSchema->getSrcTableID();
candidates.insert(srcTableID);
auto srcTableSchema = catalog.getTableCatalogEntry(tx, srcTableID);
auto srcTableSchema = catalog->getTableCatalogEntry(tx, srcTableID);
candidateNamesSet.insert(srcTableSchema->getName());
}
} else if (isDstConnect) {
for (auto relTableID : queryRel->getTableIDs()) {
auto relTableSchema = ku_dynamic_cast<CatalogEntry*, RelTableCatalogEntry*>(
catalog.getTableCatalogEntry(tx, relTableID));
catalog->getTableCatalogEntry(tx, relTableID));
auto dstTableID = relTableSchema->getDstTableID();
candidates.insert(dstTableID);
auto dstTableSchema = catalog.getTableCatalogEntry(tx, dstTableID);
auto dstTableSchema = catalog->getTableCatalogEntry(tx, dstTableID);
candidateNamesSet.insert(dstTableSchema->getName());
}
}
Expand Down Expand Up @@ -94,6 +96,7 @@ void QueryGraphLabelAnalyzer::pruneNode(const QueryGraph& graph, NodeExpression&
}

void QueryGraphLabelAnalyzer::pruneRel(RelExpression& rel) {
auto catalog = clientContext.getCatalog();
if (rel.isRecursive()) {
return;
}
Expand All @@ -108,7 +111,7 @@ void QueryGraphLabelAnalyzer::pruneRel(RelExpression& rel) {
}
for (auto& relTableID : rel.getTableIDs()) {
auto relTableSchema = ku_dynamic_cast<CatalogEntry*, RelTableCatalogEntry*>(
catalog.getTableCatalogEntry(clientContext.getTx(), relTableID));
catalog->getTableCatalogEntry(clientContext.getTx(), relTableID));
auto srcTableID = relTableSchema->getSrcTableID();
auto dstTableID = relTableSchema->getDstTableID();
if (!boundTableIDSet.contains(srcTableID) || !boundTableIDSet.contains(dstTableID)) {
Expand All @@ -121,7 +124,7 @@ void QueryGraphLabelAnalyzer::pruneRel(RelExpression& rel) {
auto dstTableIDSet = rel.getDstNode()->getTableIDsSet();
for (auto& relTableID : rel.getTableIDs()) {
auto relTableSchema = ku_dynamic_cast<CatalogEntry*, RelTableCatalogEntry*>(
catalog.getTableCatalogEntry(clientContext.getTx(), relTableID));
catalog->getTableCatalogEntry(clientContext.getTx(), relTableID));
auto srcTableID = relTableSchema->getSrcTableID();
auto dstTableID = relTableSchema->getDstTableID();
if (!srcTableIDSet.contains(srcTableID) || !dstTableIDSet.contains(dstTableID)) {
Expand Down
6 changes: 2 additions & 4 deletions src/include/binder/bound_statement_rewriter.h
Original file line number Diff line number Diff line change
@@ -1,17 +1,15 @@
#pragma once

#include "bound_statement.h"
#include "catalog/catalog.h"
#include "main/client_context.h"

namespace kuzu {
namespace binder {

// Perform semantic rewrite over bound statement.
class BoundStatementRewriter {
public:
// TODO(Jiamin): remove catalog
static void rewrite(BoundStatement& boundStatement, const catalog::Catalog& catalog,
const main::ClientContext& clientContext);
static void rewrite(BoundStatement& boundStatement, const main::ClientContext& clientContext);
};

} // namespace binder
Expand Down
8 changes: 3 additions & 5 deletions src/include/binder/query/query_graph_label_analyzer.h
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
#pragma once

#include "catalog/catalog.h"
#include "main/client_context.h"
#include "query_graph.h"

namespace kuzu {
Expand All @@ -9,9 +9,8 @@ namespace binder {
class QueryGraphLabelAnalyzer {
public:
// TODO(Jiamin): remove catalog
explicit QueryGraphLabelAnalyzer(
const catalog::Catalog& catalog, const main::ClientContext& clientContext)
: catalog{catalog}, clientContext{clientContext} {}
explicit QueryGraphLabelAnalyzer(const main::ClientContext& clientContext)
: clientContext{clientContext} {}

void pruneLabel(const QueryGraph& graph);

Expand All @@ -20,7 +19,6 @@ class QueryGraphLabelAnalyzer {
void pruneRel(RelExpression& rel);

private:
const catalog::Catalog& catalog;
const main::ClientContext& clientContext;
};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,8 @@ namespace binder {

class MatchClausePatternLabelRewriter : public BoundStatementVisitor {
public:
// TODO(Jiamin): remove catalog
explicit MatchClausePatternLabelRewriter(
const catalog::Catalog& catalog, const main::ClientContext& clientContext)
: analyzer{catalog, clientContext} {}
explicit MatchClausePatternLabelRewriter(const main::ClientContext& clientContext)
: analyzer{clientContext} {}

void visitMatch(const BoundReadingClause& readingClause) final;

Expand Down
6 changes: 1 addition & 5 deletions src/include/planner/planner.h
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,7 @@ struct LogicalSetPropertyInfo;

class Planner {
public:
// TODO(Jiamin): Remove catalog and storageManager
Planner(catalog::Catalog* catalog, storage::StorageManager* storageManager,
main::ClientContext* clientContext);
explicit Planner(main::ClientContext* clientContext);
DELETE_COPY_AND_MOVE(Planner);

std::unique_ptr<LogicalPlan> getBestPlan(const binder::BoundStatement& statement);
Expand Down Expand Up @@ -282,9 +280,7 @@ class Planner {
void exitContext(JoinOrderEnumeratorContext prevContext);

private:
catalog::Catalog* catalog;
main::ClientContext* clientContext;
storage::StorageManager* storageManager;
binder::expression_vector propertiesToScan;
CardinalityEstimator cardinalityEstimator;
JoinOrderEnumeratorContext context;
Expand Down
13 changes: 6 additions & 7 deletions src/include/processor/operator/ddl/add_node_property.h
Original file line number Diff line number Diff line change
Expand Up @@ -7,19 +7,18 @@ namespace processor {

class AddNodeProperty final : public AddProperty {
public:
AddNodeProperty(catalog::Catalog* catalog, common::table_id_t tableID, std::string propertyName,
AddNodeProperty(common::table_id_t tableID, std::string propertyName,
std::unique_ptr<common::LogicalType> dataType,
std::unique_ptr<evaluator::ExpressionEvaluator> defaultValueEvaluator,
storage::StorageManager& storageManager, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
: AddProperty{catalog, tableID, std::move(propertyName), std::move(dataType),
std::move(defaultValueEvaluator), storageManager, outputPos, id, paramsString} {}
const DataPos& outputPos, uint32_t id, const std::string& paramsString)
: AddProperty{tableID, std::move(propertyName), std::move(dataType),
std::move(defaultValueEvaluator), outputPos, id, paramsString} {}

void executeDDLInternal(ExecutionContext* context) final;

std::unique_ptr<PhysicalOperator> clone() override {
return make_unique<AddNodeProperty>(catalog, tableID, propertyName, dataType->copy(),
defaultValueEvaluator->clone(), storageManager, outputPos, id, paramsString);
return make_unique<AddNodeProperty>(tableID, propertyName, dataType->copy(),
defaultValueEvaluator->clone(), outputPos, id, paramsString);
}
};

Expand Down
13 changes: 5 additions & 8 deletions src/include/processor/operator/ddl/add_property.h
Original file line number Diff line number Diff line change
Expand Up @@ -2,21 +2,19 @@

#include "ddl.h"
#include "expression_evaluator/expression_evaluator.h"
#include "storage/storage_manager.h"

namespace kuzu {
namespace processor {

class AddProperty : public DDL {
public:
AddProperty(catalog::Catalog* catalog, common::table_id_t tableID, std::string propertyName,
AddProperty(common::table_id_t tableID, std::string propertyName,
std::unique_ptr<common::LogicalType> dataType,
std::unique_ptr<evaluator::ExpressionEvaluator> defaultValueEvaluator,
storage::StorageManager& storageManager, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
: DDL{PhysicalOperatorType::ADD_PROPERTY, catalog, outputPos, id, paramsString},
tableID{tableID}, propertyName{std::move(propertyName)}, dataType{std::move(dataType)},
defaultValueEvaluator{std::move(defaultValueEvaluator)}, storageManager{storageManager} {}
const DataPos& outputPos, uint32_t id, const std::string& paramsString)
: DDL{PhysicalOperatorType::ADD_PROPERTY, outputPos, id, paramsString}, tableID{tableID},
propertyName{std::move(propertyName)}, dataType{std::move(dataType)},
defaultValueEvaluator{std::move(defaultValueEvaluator)} {}

void initLocalStateInternal(ResultSet* resultSet, ExecutionContext* context) override {
DDL::initLocalStateInternal(resultSet, context);
Expand All @@ -35,7 +33,6 @@ class AddProperty : public DDL {
std::string propertyName;
std::unique_ptr<common::LogicalType> dataType;
std::unique_ptr<evaluator::ExpressionEvaluator> defaultValueEvaluator;
storage::StorageManager& storageManager;
};

} // namespace processor
Expand Down
13 changes: 6 additions & 7 deletions src/include/processor/operator/ddl/add_rel_property.h
Original file line number Diff line number Diff line change
Expand Up @@ -7,19 +7,18 @@ namespace processor {

class AddRelProperty final : public AddProperty {
public:
AddRelProperty(catalog::Catalog* catalog, common::table_id_t tableID, std::string propertyName,
AddRelProperty(common::table_id_t tableID, std::string propertyName,
std::unique_ptr<common::LogicalType> dataType,
std::unique_ptr<evaluator::ExpressionEvaluator> expressionEvaluator,
storage::StorageManager& storageManager, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
: AddProperty(catalog, tableID, std::move(propertyName), std::move(dataType),
std::move(expressionEvaluator), storageManager, outputPos, id, paramsString) {}
const DataPos& outputPos, uint32_t id, const std::string& paramsString)
: AddProperty(tableID, std::move(propertyName), std::move(dataType),
std::move(expressionEvaluator), outputPos, id, paramsString) {}

void executeDDLInternal(ExecutionContext* context) override;

std::unique_ptr<PhysicalOperator> clone() override {
return make_unique<AddRelProperty>(catalog, tableID, propertyName, dataType->copy(),
defaultValueEvaluator->clone(), storageManager, outputPos, id, paramsString);
return make_unique<AddRelProperty>(tableID, propertyName, dataType->copy(),
defaultValueEvaluator->clone(), outputPos, id, paramsString);
}
};

Expand Down
12 changes: 5 additions & 7 deletions src/include/processor/operator/ddl/create_node_table.h
Original file line number Diff line number Diff line change
@@ -1,29 +1,27 @@
#pragma once

#include "binder/ddl/bound_create_table_info.h"
#include "processor/operator/ddl/ddl.h"

namespace kuzu {
namespace processor {

class CreateNodeTable : public DDL {
public:
CreateNodeTable(catalog::Catalog* catalog, storage::StorageManager* storageManager,
binder::BoundCreateTableInfo info, const DataPos& outputPos, uint32_t id,
CreateNodeTable(binder::BoundCreateTableInfo info, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
: DDL{PhysicalOperatorType::CREATE_NODE_TABLE, catalog, outputPos, id, paramsString},
storageManager{storageManager}, info{std::move(info)} {}
: DDL{PhysicalOperatorType::CREATE_NODE_TABLE, outputPos, id, paramsString}, info{std::move(
info)} {}

void executeDDLInternal(ExecutionContext* context) final;

std::string getOutputMsg() final;

std::unique_ptr<PhysicalOperator> clone() final {
return std::make_unique<CreateNodeTable>(
catalog, storageManager, info.copy(), outputPos, id, paramsString);
return std::make_unique<CreateNodeTable>(info.copy(), outputPos, id, paramsString);
}

private:
storage::StorageManager* storageManager;
binder::BoundCreateTableInfo info;
};

Expand Down
9 changes: 4 additions & 5 deletions src/include/processor/operator/ddl/create_rdf_graph.h
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,9 @@ namespace processor {

class CreateRdfGraph final : public DDL {
public:
CreateRdfGraph(catalog::Catalog* catalog, storage::StorageManager* storageManager,
binder::BoundCreateTableInfo info, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
: DDL{PhysicalOperatorType::CREATE_RDF_GRAPH, catalog, outputPos, id, paramsString},
CreateRdfGraph(storage::StorageManager* storageManager, binder::BoundCreateTableInfo info,
const DataPos& outputPos, uint32_t id, const std::string& paramsString)
: DDL{PhysicalOperatorType::CREATE_RDF_GRAPH, outputPos, id, paramsString},
storageManager{storageManager},
nodesStatistics{storageManager->getNodesStatisticsAndDeletedIDs()},
relsStatistics{storageManager->getRelsStatistics()}, info{std::move(info)} {}
Expand All @@ -22,7 +21,7 @@ class CreateRdfGraph final : public DDL {

inline std::unique_ptr<PhysicalOperator> clone() override {
return std::make_unique<CreateRdfGraph>(
catalog, storageManager, info.copy(), outputPos, id, paramsString);
storageManager, info.copy(), outputPos, id, paramsString);
}

private:
Expand Down
12 changes: 5 additions & 7 deletions src/include/processor/operator/ddl/create_rel_table.h
Original file line number Diff line number Diff line change
@@ -1,29 +1,27 @@
#pragma once

#include "binder/ddl/bound_create_table_info.h"
#include "processor/operator/ddl/ddl.h"

namespace kuzu {
namespace processor {

class CreateRelTable final : public DDL {
public:
CreateRelTable(catalog::Catalog* catalog, storage::StorageManager* storageManager,
binder::BoundCreateTableInfo info, const DataPos& outputPos, uint32_t id,
CreateRelTable(binder::BoundCreateTableInfo info, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
: DDL{PhysicalOperatorType::CREATE_REL_TABLE, catalog, outputPos, id, paramsString},
storageManager{storageManager}, info{std::move(info)} {}
: DDL{PhysicalOperatorType::CREATE_REL_TABLE, outputPos, id, paramsString}, info{std::move(
info)} {}

void executeDDLInternal(ExecutionContext* context) override;

std::string getOutputMsg() override;

std::unique_ptr<PhysicalOperator> clone() override {
return make_unique<CreateRelTable>(
catalog, storageManager, info.copy(), outputPos, id, paramsString);
return make_unique<CreateRelTable>(info.copy(), outputPos, id, paramsString);
}

private:
storage::StorageManager* storageManager;
binder::BoundCreateTableInfo info;
};

Expand Down
Loading

0 comments on commit 3a6bd7e

Please sign in to comment.