Skip to content

Commit

Permalink
Add multi-label set
Browse files Browse the repository at this point in the history
  • Loading branch information
andyfengHKU committed Aug 2, 2023
1 parent baf9e56 commit a33357d
Show file tree
Hide file tree
Showing 17 changed files with 516 additions and 216 deletions.
2 changes: 0 additions & 2 deletions src/binder/bind/bind_updating_clause.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -207,12 +207,10 @@ std::unique_ptr<BoundSetPropertyInfo> Binder::bindSetPropertyInfo(
auto left = expressionBinder.bindExpression(*setItem.first->getChild(0));
switch (left->dataType.getLogicalTypeID()) {
case LogicalTypeID::NODE: {
validateSetNodeProperty(*left);
return std::make_unique<BoundSetPropertyInfo>(
UpdateTableType::NODE, left, bindSetItem(setItem));
}
case LogicalTypeID::REL: {
validateSetRelProperty(*left);
return std::make_unique<BoundSetPropertyInfo>(
UpdateTableType::REL, left, bindSetItem(setItem));
}
Expand Down
90 changes: 46 additions & 44 deletions src/include/planner/logical_plan/logical_operator/logical_set.h
Original file line number Diff line number Diff line change
@@ -1,69 +1,71 @@
#pragma once

#include "base_logical_operator.h"
#include "logical_update.h"
#include "planner/logical_plan/logical_operator/flatten_resolver.h"

namespace kuzu {
namespace planner {

class LogicalSetNodeProperty : public LogicalUpdateNode {
public:
LogicalSetNodeProperty(std::vector<std::shared_ptr<binder::NodeExpression>> nodes,
std::vector<binder::expression_pair> setItems, std::shared_ptr<LogicalOperator> child)
: LogicalUpdateNode{LogicalOperatorType::SET_NODE_PROPERTY, std::move(nodes),
std::move(child)},
setItems{std::move(setItems)} {}

inline void computeFactorizedSchema() override { copyChildSchema(0); }
inline void computeFlatSchema() override { copyChildSchema(0); }

inline std::string getExpressionsForPrinting() const override {
std::string result;
for (auto& [lhs, rhs] : setItems) {
result += lhs->toString() + " = " + rhs->toString() + ",";
}
return result;
struct LogicalSetPropertyInfo {
std::shared_ptr<binder::Expression> nodeOrRel;
binder::expression_pair setItem;

LogicalSetPropertyInfo(
std::shared_ptr<binder::Expression> nodeOrRel, binder::expression_pair setItem)
: nodeOrRel{std::move(nodeOrRel)}, setItem{std::move(setItem)} {}
LogicalSetPropertyInfo(const LogicalSetPropertyInfo& other)
: nodeOrRel{other.nodeOrRel}, setItem{other.setItem} {}

inline std::unique_ptr<LogicalSetPropertyInfo> copy() const {
return std::make_unique<LogicalSetPropertyInfo>(*this);
}
};

class LogicalSetNodeProperty : public LogicalOperator {
public:
LogicalSetNodeProperty(std::vector<std::unique_ptr<LogicalSetPropertyInfo>> infos,
std::shared_ptr<LogicalOperator> child)
: LogicalOperator{LogicalOperatorType::SET_NODE_PROPERTY, std::move(child)},
infos{std::move(infos)} {}

inline binder::expression_pair getSetItem(size_t idx) const { return setItems[idx]; }
inline void computeFactorizedSchema() final { copyChildSchema(0); }
inline void computeFlatSchema() final { copyChildSchema(0); }

inline std::unique_ptr<LogicalOperator> copy() override {
return make_unique<LogicalSetNodeProperty>(nodes, setItems, children[0]->copy());
inline const std::vector<std::unique_ptr<LogicalSetPropertyInfo>>& getInfosRef() const {
return infos;
}

std::string getExpressionsForPrinting() const final;

std::unique_ptr<LogicalOperator> copy() final;

private:
std::vector<binder::expression_pair> setItems;
std::vector<std::unique_ptr<LogicalSetPropertyInfo>> infos;
};

class LogicalSetRelProperty : public LogicalUpdateRel {
class LogicalSetRelProperty : public LogicalOperator {
public:
LogicalSetRelProperty(std::vector<std::shared_ptr<binder::RelExpression>> rels,
std::vector<binder::expression_pair> setItems, std::shared_ptr<LogicalOperator> child)
: LogicalUpdateRel{LogicalOperatorType::SET_REL_PROPERTY, std::move(rels),
std::move(child)},
setItems{std::move(setItems)} {}

inline void computeFactorizedSchema() override { copyChildSchema(0); }
inline void computeFlatSchema() override { copyChildSchema(0); }

f_group_pos_set getGroupsPosToFlatten(uint32_t setItemIdx);

inline std::string getExpressionsForPrinting() const override {
std::string result;
for (auto& [lhs, rhs] : setItems) {
result += lhs->toString() + " = " + rhs->toString() + ",";
}
return result;
}
LogicalSetRelProperty(std::vector<std::unique_ptr<LogicalSetPropertyInfo>> infos,
std::shared_ptr<LogicalOperator> child)
: LogicalOperator{LogicalOperatorType::SET_REL_PROPERTY, std::move(child)}, infos{std::move(
infos)} {}

inline binder::expression_pair getSetItem(size_t idx) const { return setItems[idx]; }
inline void computeFactorizedSchema() final { copyChildSchema(0); }
inline void computeFlatSchema() final { copyChildSchema(0); }

inline std::unique_ptr<LogicalOperator> copy() override {
return make_unique<LogicalSetRelProperty>(rels, setItems, children[0]->copy());
inline const std::vector<std::unique_ptr<LogicalSetPropertyInfo>>& getInfosRef() const {
return infos;
}

f_group_pos_set getGroupsPosToFlatten(uint32_t idx);

std::string getExpressionsForPrinting() const final;

std::unique_ptr<LogicalOperator> copy() final;

private:
std::vector<binder::expression_pair> setItems;
std::vector<std::unique_ptr<LogicalSetPropertyInfo>> infos;
};

} // namespace planner
Expand Down
4 changes: 2 additions & 2 deletions src/include/planner/query_planner.h
Original file line number Diff line number Diff line change
Expand Up @@ -100,9 +100,9 @@ class QueryPlanner {
void appendCreateRel(
const std::vector<binder::BoundCreateInfo*>& createInfos, LogicalPlan& plan);
void appendSetNodeProperty(
const std::vector<binder::BoundSetPropertyInfo*>& infos, LogicalPlan& plan);
const std::vector<binder::BoundSetPropertyInfo*>& boundInfos, LogicalPlan& plan);
void appendSetRelProperty(
const std::vector<binder::BoundSetPropertyInfo*>& infos, LogicalPlan& plan);
const std::vector<binder::BoundSetPropertyInfo*>& boundInfos, LogicalPlan& plan);
void appendDeleteNode(
const std::vector<binder::BoundDeleteInfo*>& boundInfos, LogicalPlan& plan);
void appendDeleteRel(
Expand Down
75 changes: 13 additions & 62 deletions src/include/processor/operator/update/set.h
Original file line number Diff line number Diff line change
@@ -1,94 +1,45 @@
#pragma once

#include "expression_evaluator/base_evaluator.h"
#include "processor/operator/physical_operator.h"
#include "storage/store/node_table.h"
#include "storage/store/rel_table.h"
#include "set_executor.h"

namespace kuzu {
namespace processor {

class SetNodePropertyInfo {
public:
SetNodePropertyInfo(storage::NodeTable* table, common::property_id_t propertyID,
const DataPos& nodeIDPos, std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: table{table}, propertyID{propertyID}, nodeIDPos{nodeIDPos}, evaluator{
std::move(evaluator)} {}

inline std::unique_ptr<SetNodePropertyInfo> clone() const {
return make_unique<SetNodePropertyInfo>(table, propertyID, nodeIDPos, evaluator->clone());
}

storage::NodeTable* table;
common::property_id_t propertyID;
DataPos nodeIDPos;
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator;
};

class SetNodeProperty : public PhysicalOperator {
public:
SetNodeProperty(std::vector<std::unique_ptr<SetNodePropertyInfo>> infos,
SetNodeProperty(std::vector<std::unique_ptr<NodeSetExecutor>> executors,
std::unique_ptr<PhysicalOperator> child, uint32_t id, const std::string& paramsString)
: PhysicalOperator{PhysicalOperatorType::SET_NODE_PROPERTY, std::move(child), id,
paramsString},
infos{std::move(infos)} {}

~SetNodeProperty() override = default;
executors{std::move(executors)} {}

void initLocalStateInternal(ResultSet* resultSet, ExecutionContext* context) override;
void initLocalStateInternal(ResultSet* resultSet, ExecutionContext* context) final;

bool getNextTuplesInternal(ExecutionContext* context) override;
bool getNextTuplesInternal(ExecutionContext* context) final;

std::unique_ptr<PhysicalOperator> clone() override;
std::unique_ptr<PhysicalOperator> clone() final;

private:
std::vector<std::unique_ptr<SetNodePropertyInfo>> infos;

std::vector<common::ValueVector*> nodeIDVectors;
};

struct SetRelPropertyInfo {
storage::RelTable* table;
DataPos srcNodePos;
DataPos dstNodePos;
DataPos relIDPos;
common::property_id_t propertyId;
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator;

SetRelPropertyInfo(storage::RelTable* table, const DataPos& srcNodePos,
const DataPos& dstNodePos, const DataPos& relIDPos, common::property_id_t propertyId,
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: table{table}, srcNodePos{srcNodePos}, dstNodePos{dstNodePos}, relIDPos{relIDPos},
propertyId{propertyId}, evaluator{std::move(evaluator)} {}

inline std::unique_ptr<SetRelPropertyInfo> clone() const {
return make_unique<SetRelPropertyInfo>(
table, srcNodePos, dstNodePos, relIDPos, propertyId, evaluator->clone());
}
std::vector<std::unique_ptr<NodeSetExecutor>> executors;
};

class SetRelProperty : public PhysicalOperator {
public:
SetRelProperty(std::vector<std::unique_ptr<SetRelPropertyInfo>> infos,
SetRelProperty(std::vector<std::unique_ptr<RelSetExecutor>> executors,
std::unique_ptr<PhysicalOperator> child, uint32_t id, const std::string& paramsString)
: PhysicalOperator{PhysicalOperatorType::SET_NODE_PROPERTY, std::move(child), id,
paramsString},
infos{std::move(infos)} {}

~SetRelProperty() override = default;
executors{std::move(executors)} {}

void initLocalStateInternal(ResultSet* resultSet, ExecutionContext* context) override;
void initLocalStateInternal(ResultSet* resultSet, ExecutionContext* context) final;

bool getNextTuplesInternal(ExecutionContext* context) override;
bool getNextTuplesInternal(ExecutionContext* context) final;

std::unique_ptr<PhysicalOperator> clone() override;
std::unique_ptr<PhysicalOperator> clone() final;

private:
std::vector<std::unique_ptr<SetRelPropertyInfo>> infos;

std::vector<common::ValueVector*> srcNodeVectors;
std::vector<common::ValueVector*> dstNodeVectors;
std::vector<common::ValueVector*> relIDVectors;
std::vector<std::unique_ptr<RelSetExecutor>> executors;
};

} // namespace processor
Expand Down
147 changes: 147 additions & 0 deletions src/include/processor/operator/update/set_executor.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
#pragma once

#include "expression_evaluator/base_evaluator.h"
#include "processor/execution_context.h"
#include "processor/result/result_set.h"
#include "storage/store/node_table.h"
#include "storage/store/rel_table.h"

namespace kuzu {
namespace processor {

class NodeSetExecutor {
public:
NodeSetExecutor(
const DataPos& nodeIDPos, std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: nodeIDPos{nodeIDPos}, evaluator{std::move(evaluator)} {}
virtual ~NodeSetExecutor() = default;

Check warning on line 17 in src/include/processor/operator/update/set_executor.h

View check run for this annotation

Codecov / codecov/patch

src/include/processor/operator/update/set_executor.h#L17

Added line #L17 was not covered by tests

void init(ResultSet* resultSet, ExecutionContext* context);

virtual void set() = 0;

virtual std::unique_ptr<NodeSetExecutor> copy() const = 0;

protected:
DataPos nodeIDPos;
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator;

common::ValueVector* nodeIDVector;
common::ValueVector* rhsVector;
};

class SingleLabelNodeSetExecutor : public NodeSetExecutor {
public:
SingleLabelNodeSetExecutor(storage::NodeColumn* column, const DataPos& nodeIDPos,
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: NodeSetExecutor{nodeIDPos, std::move(evaluator)}, column{column} {}
SingleLabelNodeSetExecutor(const SingleLabelNodeSetExecutor& other)
: NodeSetExecutor{other.nodeIDPos, other.evaluator->clone()}, column{other.column} {}

void set() final;

inline std::unique_ptr<NodeSetExecutor> copy() const final {
return std::make_unique<SingleLabelNodeSetExecutor>(*this);
}

private:
storage::NodeColumn* column;
};

class MultiLabelNodeSetExecutor : public NodeSetExecutor {
public:
MultiLabelNodeSetExecutor(
std::unordered_map<common::table_id_t, storage::NodeColumn*> tableIDToColumn,
const DataPos& nodeIDPos, std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: NodeSetExecutor{nodeIDPos, std::move(evaluator)}, tableIDToColumn{
std::move(tableIDToColumn)} {}
MultiLabelNodeSetExecutor(const MultiLabelNodeSetExecutor& other)
: NodeSetExecutor{other.nodeIDPos, other.evaluator->clone()}, tableIDToColumn{
other.tableIDToColumn} {}

void set() final;

inline std::unique_ptr<NodeSetExecutor> copy() const final {
return std::make_unique<MultiLabelNodeSetExecutor>(*this);
}

private:
std::unordered_map<common::table_id_t, storage::NodeColumn*> tableIDToColumn;
};

class RelSetExecutor {
public:
RelSetExecutor(const DataPos& srcNodeIDPos, const DataPos& dstNodeIDPos,
const DataPos& relIDPos, std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: srcNodeIDPos{srcNodeIDPos},
dstNodeIDPos{dstNodeIDPos}, relIDPos{relIDPos}, evaluator{std::move(evaluator)} {}
virtual ~RelSetExecutor() = default;

Check warning on line 78 in src/include/processor/operator/update/set_executor.h

View check run for this annotation

Codecov / codecov/patch

src/include/processor/operator/update/set_executor.h#L78

Added line #L78 was not covered by tests

void init(ResultSet* resultSet, ExecutionContext* context);

virtual void set() = 0;

virtual std::unique_ptr<RelSetExecutor> copy() const = 0;

protected:
DataPos srcNodeIDPos;
DataPos dstNodeIDPos;
DataPos relIDPos;
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator;

common::ValueVector* srcNodeIDVector;
common::ValueVector* dstNodeIDVector;
common::ValueVector* relIDVector;
common::ValueVector* rhsVector;
};

class SingleLabelRelSetExecutor : public RelSetExecutor {
public:
SingleLabelRelSetExecutor(storage::RelTable* table, common::property_id_t propertyID,
const DataPos& srcNodeIDPos, const DataPos& dstNodeIDPos, const DataPos& relIDPos,
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: RelSetExecutor{srcNodeIDPos, dstNodeIDPos, relIDPos, std::move(evaluator)}, table{table},
propertyID{propertyID} {}
SingleLabelRelSetExecutor(const SingleLabelRelSetExecutor& other)
: RelSetExecutor{other.srcNodeIDPos, other.dstNodeIDPos, other.relIDPos,
other.evaluator->clone()},
table{other.table}, propertyID{other.propertyID} {}

void set() final;

inline std::unique_ptr<RelSetExecutor> copy() const final {
return std::make_unique<SingleLabelRelSetExecutor>(*this);
}

private:
storage::RelTable* table;
common::property_id_t propertyID;
};

class MultiLabelRelSetExecutor : public RelSetExecutor {
public:
MultiLabelRelSetExecutor(
std::unordered_map<common::table_id_t, std::pair<storage::RelTable*, common::property_id_t>>
tableIDToTableAndPropertyID,
const DataPos& srcNodeIDPos, const DataPos& dstNodeIDPos, const DataPos& relIDPos,
std::unique_ptr<evaluator::BaseExpressionEvaluator> evaluator)
: RelSetExecutor{srcNodeIDPos, dstNodeIDPos, relIDPos, std::move(evaluator)},
tableIDToTableAndPropertyID{std::move(tableIDToTableAndPropertyID)} {}
MultiLabelRelSetExecutor(const MultiLabelRelSetExecutor& other)
: RelSetExecutor{other.srcNodeIDPos, other.dstNodeIDPos, other.relIDPos,
other.evaluator->clone()},
tableIDToTableAndPropertyID{other.tableIDToTableAndPropertyID} {}

void set() final;

inline std::unique_ptr<RelSetExecutor> copy() const final {
return std::make_unique<MultiLabelRelSetExecutor>(*this);
}

private:
std::unordered_map<common::table_id_t, std::pair<storage::RelTable*, common::property_id_t>>
tableIDToTableAndPropertyID;
};

} // namespace processor
} // namespace kuzu
Loading

0 comments on commit a33357d

Please sign in to comment.