Skip to content

Commit

Permalink
Merge pull request #2453 from kuzudb/detach-delete
Browse files Browse the repository at this point in the history
Detach delete
  • Loading branch information
ray6080 committed Nov 19, 2023
2 parents bd4f0e2 + fc4a49a commit 53772d3
Show file tree
Hide file tree
Showing 31 changed files with 2,237 additions and 1,957 deletions.
4 changes: 3 additions & 1 deletion src/antlr4/Cypher.g4
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,9 @@ oC_SetItem
: ( oC_PropertyExpression SP? '=' SP? oC_Expression ) ;

oC_Delete
: DELETE SP? oC_Expression ( SP? ',' SP? oC_Expression )*;
: ( DETACH SP )? DELETE SP? oC_Expression ( SP? ',' SP? oC_Expression )*;

DETACH : ( 'D' | 'd' ) ( 'E' | 'e' ) ( 'T' | 't' ) ( 'A' | 'a' ) ( 'C' | 'c' ) ( 'H' | 'h' ) ;

DELETE : ( 'D' | 'd' ) ( 'E' | 'e' ) ( 'L' | 'l' ) ( 'E' | 'e' ) ( 'T' | 't' ) ( 'E' | 'e' ) ;

Expand Down
11 changes: 8 additions & 3 deletions src/binder/bind/bind_updating_clause.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -231,15 +231,20 @@ std::unique_ptr<BoundUpdatingClause> Binder::bindDeleteClause(
for (auto i = 0u; i < deleteClause.getNumExpressions(); ++i) {
auto nodeOrRel = expressionBinder.bindExpression(*deleteClause.getExpression(i));
if (ExpressionUtil::isNodePattern(*nodeOrRel)) {
auto deleteNodeInfo =
std::make_unique<BoundDeleteInfo>(UpdateTableType::NODE, nodeOrRel);
auto deleteNodeInfo = std::make_unique<BoundDeleteInfo>(
UpdateTableType::NODE, nodeOrRel, deleteClause.getDeleteClauseType());
boundDeleteClause->addInfo(std::move(deleteNodeInfo));
} else if (ExpressionUtil::isRelPattern(*nodeOrRel)) {
if (deleteClause.getDeleteClauseType() == DeleteClauseType::DETACH_DELETE) {
// TODO(Xiyang): Dummy check here. Make sure this is the correct semantic.
throw BinderException("Detach delete on rel tables is not supported.");
}
auto rel = (RelExpression*)nodeOrRel.get();
if (rel->getDirectionType() == RelDirectionType::BOTH) {
throw BinderException("Delete undirected rel is not supported.");
}
auto deleteRel = std::make_unique<BoundDeleteInfo>(UpdateTableType::REL, nodeOrRel);
auto deleteRel = std::make_unique<BoundDeleteInfo>(
UpdateTableType::REL, nodeOrRel, DeleteClauseType::DELETE);
boundDeleteClause->addInfo(std::move(deleteRel));
} else {
throw BinderException(stringFormat(
Expand Down
4 changes: 2 additions & 2 deletions src/common/CMakeLists.txt
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
add_subdirectory(arrow)
add_subdirectory(copier_config)
add_subdirectory(data_chunk)
add_subdirectory(enums)
add_subdirectory(exception)
add_subdirectory(serializer)
add_subdirectory(task_system)
Expand All @@ -18,8 +19,7 @@ add_library(kuzu_common
profiler.cpp
type_utils.cpp
utils.cpp
string_utils.cpp
table_type.cpp)
string_utils.cpp)

target_link_libraries(kuzu_common Glob)

Expand Down
8 changes: 8 additions & 0 deletions src/common/enums/CMakeLists.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
add_library(kuzu_common_enums
OBJECT
rel_direction.cpp
table_type.cpp)

set(ALL_OBJECT_FILES
${ALL_OBJECT_FILES} $<TARGET_OBJECTS:kuzu_common_enums>
PARENT_SCOPE)
20 changes: 20 additions & 0 deletions src/common/enums/rel_direction.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
#include "common/enums/rel_direction.h"

#include "common/assert.h"

namespace kuzu {
namespace common {

std::string RelDataDirectionUtils::relDirectionToString(RelDataDirection direction) {
switch (direction) {
case RelDataDirection::FWD:
return "forward";
case RelDataDirection::BWD:
return "backward";
default:
KU_UNREACHABLE;
}
}

} // namespace common
} // namespace kuzu
File renamed without changes.
11 changes: 8 additions & 3 deletions src/include/binder/query/updating_clause/bound_delete_info.h
Original file line number Diff line number Diff line change
@@ -1,19 +1,24 @@
#pragma once

#include "binder/expression/expression.h"
#include "common/enums/clause_type.h"
#include "update_table_type.h"

namespace kuzu {
namespace binder {

struct BoundDeleteInfo {
common::DeleteClauseType deleteClauseType;
UpdateTableType updateTableType;
std::shared_ptr<Expression> nodeOrRel;

BoundDeleteInfo(UpdateTableType updateTableType, std::shared_ptr<Expression> nodeOrRel)
: updateTableType{updateTableType}, nodeOrRel{std::move(nodeOrRel)} {}
BoundDeleteInfo(UpdateTableType updateTableType, std::shared_ptr<Expression> nodeOrRel,
common::DeleteClauseType deleteClauseType)
: updateTableType{updateTableType}, nodeOrRel{std::move(nodeOrRel)},
deleteClauseType{deleteClauseType} {}
BoundDeleteInfo(const BoundDeleteInfo& other)
: updateTableType{other.updateTableType}, nodeOrRel{other.nodeOrRel} {}
: updateTableType{other.updateTableType}, nodeOrRel{other.nodeOrRel},
deleteClauseType{other.deleteClauseType} {}

inline std::unique_ptr<BoundDeleteInfo> copy() {
return std::make_unique<BoundDeleteInfo>(*this);
Expand Down
2 changes: 1 addition & 1 deletion src/include/common/enums/alter_type.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,5 +13,5 @@ enum class AlterType : uint8_t {
RENAME_PROPERTY = 12,
};

}
} // namespace common
} // namespace kuzu
5 changes: 5 additions & 0 deletions src/include/common/enums/clause_type.h
Original file line number Diff line number Diff line change
Expand Up @@ -24,5 +24,10 @@ enum class MatchClauseType : uint8_t {
OPTIONAL_MATCH = 1,
};

enum class DeleteClauseType : uint8_t {
DELETE = 0,
DETACH_DELETE = 1,
};

} // namespace common
} // namespace kuzu
14 changes: 14 additions & 0 deletions src/include/common/enums/delete_type.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
#pragma once

#include <cstdint>

namespace kuzu {
namespace common {

enum class DeleteNodeType : uint8_t {
DELETE = 0,
DETACH_DELETE = 1,
};

} // namespace common
} // namespace kuzu
5 changes: 5 additions & 0 deletions src/include/common/enums/rel_direction.h
Original file line number Diff line number Diff line change
@@ -1,11 +1,16 @@
#pragma once

#include <cstdint>
#include <string>

namespace kuzu {
namespace common {

enum class RelDataDirection : uint8_t { FWD = 0, BWD = 1 };

struct RelDataDirectionUtils {
static std::string relDirectionToString(RelDataDirection direction);
};

} // namespace common
} // namespace kuzu
8 changes: 5 additions & 3 deletions src/include/parser/query/updating_clause/delete_clause.h
Original file line number Diff line number Diff line change
Expand Up @@ -6,18 +6,20 @@
namespace kuzu {
namespace parser {

class DeleteClause : public UpdatingClause {
class DeleteClause final : public UpdatingClause {
public:
DeleteClause() : UpdatingClause{common::ClauseType::DELETE_} {};
~DeleteClause() override = default;
DeleteClause(common::DeleteClauseType deleteClauseType)
: UpdatingClause{common::ClauseType::DELETE_}, deleteClauseType{deleteClauseType} {};

inline void addExpression(std::unique_ptr<ParsedExpression> expression) {
expressions.push_back(std::move(expression));
}
inline common::DeleteClauseType getDeleteClauseType() const { return deleteClauseType; }
inline uint32_t getNumExpressions() const { return expressions.size(); }
inline ParsedExpression* getExpression(uint32_t idx) const { return expressions[idx].get(); }

private:
common::DeleteClauseType deleteClauseType;
std::vector<std::unique_ptr<ParsedExpression>> expressions;
};

Expand Down
37 changes: 30 additions & 7 deletions src/include/planner/operator/persistent/logical_delete.h
Original file line number Diff line number Diff line change
@@ -1,35 +1,58 @@
#pragma once

#include "binder/expression/rel_expression.h"
#include "common/enums/delete_type.h"
#include "planner/operator/logical_operator.h"

namespace kuzu {
namespace planner {

struct LogicalDeleteNodeInfo {
std::shared_ptr<binder::NodeExpression> node;
common::DeleteNodeType deleteType;

LogicalDeleteNodeInfo(
std::shared_ptr<binder::NodeExpression> node, common::DeleteNodeType deleteType)
: node{std::move(node)}, deleteType{deleteType} {}

inline std::unique_ptr<LogicalDeleteNodeInfo> copy() const {
return std::make_unique<LogicalDeleteNodeInfo>(node, deleteType);
}

static std::vector<std::unique_ptr<LogicalDeleteNodeInfo>> copy(
const std::vector<std::unique_ptr<LogicalDeleteNodeInfo>>& infos);
};

class LogicalDeleteNode : public LogicalOperator {
public:
LogicalDeleteNode(std::vector<std::shared_ptr<binder::NodeExpression>> nodes,
LogicalDeleteNode(std::vector<std::unique_ptr<LogicalDeleteNodeInfo>> infos,
std::shared_ptr<LogicalOperator> child)
: LogicalOperator{LogicalOperatorType::DELETE_NODE, std::move(child)}, nodes{std::move(
nodes)} {}
: LogicalOperator{LogicalOperatorType::DELETE_NODE, std::move(child)}, infos{std::move(
infos)} {}

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

std::string getExpressionsForPrinting() const final;

inline const std::vector<std::shared_ptr<binder::NodeExpression>>& getNodesRef() const {
return nodes;
inline std::vector<LogicalDeleteNodeInfo*> getInfos() const {
std::vector<LogicalDeleteNodeInfo*> result;
result.reserve(infos.size());
for (auto& info : infos) {
result.push_back(info.get());
}
return result;
}

f_group_pos_set getGroupsPosToFlatten();

inline std::unique_ptr<LogicalOperator> copy() final {
return std::make_unique<LogicalDeleteNode>(nodes, children[0]->copy());
return std::make_unique<LogicalDeleteNode>(
LogicalDeleteNodeInfo::copy(infos), children[0]->copy());
}

private:
std::vector<std::shared_ptr<binder::NodeExpression>> nodes;
std::vector<std::unique_ptr<LogicalDeleteNodeInfo>> infos;
};

class LogicalDeleteRel : public LogicalOperator {
Expand Down
25 changes: 16 additions & 9 deletions src/include/processor/operator/persistent/delete_executor.h
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
#pragma once

#include "common/enums/delete_type.h"
#include "processor/execution_context.h"
#include "processor/result/result_set.h"
#include "storage/store/node_table.h"
Expand All @@ -10,7 +11,8 @@ namespace processor {

class NodeDeleteExecutor {
public:
NodeDeleteExecutor(const DataPos& nodeIDPos) : nodeIDPos{nodeIDPos}, nodeIDVector(nullptr) {}
NodeDeleteExecutor(common::DeleteNodeType deleteType, const DataPos& nodeIDPos)
: deleteType{deleteType}, nodeIDPos{nodeIDPos}, nodeIDVector(nullptr) {}
virtual ~NodeDeleteExecutor() = default;

virtual void init(ResultSet* resultSet, ExecutionContext* context);
Expand All @@ -20,20 +22,23 @@ class NodeDeleteExecutor {
virtual std::unique_ptr<NodeDeleteExecutor> copy() const = 0;

protected:
common::DeleteNodeType deleteType;
DataPos nodeIDPos;
common::ValueVector* nodeIDVector;
std::unique_ptr<storage::RelDetachDeleteState> detachDeleteState;
};

class SingleLabelNodeDeleteExecutor final : public NodeDeleteExecutor {
public:
SingleLabelNodeDeleteExecutor(storage::NodeTable* table,
std::unordered_set<storage::RelTable*> fwdRelTables,
std::unordered_set<storage::RelTable*> bwdRelTables, const DataPos& nodeIDPos)
: NodeDeleteExecutor(nodeIDPos), table{table}, fwdRelTables{fwdRelTables},
std::unordered_set<storage::RelTable*> bwdRelTables, common::DeleteNodeType deleteType,
const DataPos& nodeIDPos)
: NodeDeleteExecutor(deleteType, nodeIDPos), table{table}, fwdRelTables{fwdRelTables},
bwdRelTables{bwdRelTables} {}
SingleLabelNodeDeleteExecutor(const SingleLabelNodeDeleteExecutor& other)
: NodeDeleteExecutor(other.nodeIDPos), table{other.table}, fwdRelTables{other.fwdRelTables},
bwdRelTables{other.bwdRelTables} {}
: NodeDeleteExecutor(other.deleteType, other.nodeIDPos), table{other.table},
fwdRelTables{other.fwdRelTables}, bwdRelTables{other.bwdRelTables} {}

void init(ResultSet* resultSet, ExecutionContext* context) override;
void delete_(ExecutionContext* context) override;
Expand All @@ -57,12 +62,14 @@ class MultiLabelNodeDeleteExecutor final : public NodeDeleteExecutor {
std::unordered_map<common::table_id_t, storage::NodeTable*> tableIDToTableMap,
std::unordered_map<common::table_id_t, rel_tables_set_t> tableIDToFwdRelTablesMap,
std::unordered_map<common::table_id_t, rel_tables_set_t> tableIDToBwdRelTablesMap,
const DataPos& nodeIDPos)
: NodeDeleteExecutor(nodeIDPos), tableIDToTableMap{std::move(tableIDToTableMap)},
tableIDToFwdRelTablesMap{std::move(tableIDToFwdRelTablesMap)},
common::DeleteNodeType deleteType, const DataPos& nodeIDPos)
: NodeDeleteExecutor(deleteType, nodeIDPos),
tableIDToTableMap{std::move(tableIDToTableMap)}, tableIDToFwdRelTablesMap{std::move(
tableIDToFwdRelTablesMap)},
tableIDToBwdRelTablesMap{std::move(tableIDToBwdRelTablesMap)} {}
MultiLabelNodeDeleteExecutor(const MultiLabelNodeDeleteExecutor& other)
: NodeDeleteExecutor(other.nodeIDPos), tableIDToTableMap{other.tableIDToTableMap},
: NodeDeleteExecutor(other.deleteType, other.nodeIDPos),
tableIDToTableMap{other.tableIDToTableMap},
tableIDToFwdRelTablesMap{other.tableIDToFwdRelTablesMap},
tableIDToBwdRelTablesMap{other.tableIDToBwdRelTablesMap} {}

Expand Down
19 changes: 18 additions & 1 deletion src/include/storage/store/rel_table.h
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,13 @@
namespace kuzu {
namespace storage {

struct RelDetachDeleteState {
std::unique_ptr<common::ValueVector> dstNodeIDVector;
std::unique_ptr<common::ValueVector> relIDVector;

explicit RelDetachDeleteState();
};

class RelTable : public Table {
public:
RelTable(BMFileHandle* dataFH, BMFileHandle* metadataFH, RelsStoreStats* relsStoreStats,
Expand Down Expand Up @@ -36,7 +43,8 @@ class RelTable : public Table {
common::ValueVector* relIDVector, common::ValueVector* propertyVector);
void delete_(transaction::Transaction* transaction, common::ValueVector* srcNodeIDVector,
common::ValueVector* dstNodeIDVector, common::ValueVector* relIDVector);

void detachDelete(transaction::Transaction* transaction, common::RelDataDirection direction,
common::ValueVector* srcNodeIDVector, RelDetachDeleteState* deleteState);
inline bool checkIfNodeHasRels(transaction::Transaction* transaction,
common::RelDataDirection direction, common::ValueVector* srcNodeIDVector) {
return direction == common::RelDataDirection::FWD ?
Expand Down Expand Up @@ -73,6 +81,15 @@ class RelTable : public Table {
common::ValueVector* inNodeIDVector,
const std::vector<common::ValueVector*>& outputVectors);

common::row_idx_t detachDeleteForRegularRels(transaction::Transaction* transaction,
RelTableData* tableData, RelTableData* reverseTableData,
common::ValueVector* srcNodeIDVector, RelDataReadState* relDataReadState,
RelDetachDeleteState* deleteState);
common::row_idx_t detachDeleteForCSRRels(transaction::Transaction* transaction,
RelTableData* tableData, RelTableData* reverseTableData,
common::ValueVector* srcNodeIDVector, RelDataReadState* relDataReadState,
RelDetachDeleteState* deleteState);

inline RelTableData* getDirectedTableData(common::RelDataDirection direction) {
return direction == common::RelDataDirection::FWD ? fwdRelTableData.get() :
bwdRelTableData.get();
Expand Down
5 changes: 3 additions & 2 deletions src/include/storage/store/rel_table_data.h
Original file line number Diff line number Diff line change
Expand Up @@ -63,12 +63,13 @@ class RelTableData final : public TableData {
void update(transaction::Transaction* transaction, common::column_id_t columnID,
common::ValueVector* srcNodeIDVector, common::ValueVector* relIDVector,
common::ValueVector* propertyVector);

// Return true if deletion succeeds. Note that we should return num of rels deleted later when
// we remove the restriction of flatten all tuples.
bool delete_(transaction::Transaction* transaction, common::ValueVector* srcNodeIDVector,
common::ValueVector* dstNodeIDVector, common::ValueVector* relIDVector);

bool checkIfNodeHasRels(
transaction::Transaction* transaction, common::ValueVector* srcNodeIDVector);

void append(NodeGroup* nodeGroup);

inline Column* getAdjColumn() const { return adjColumn.get(); }
Expand Down
4 changes: 2 additions & 2 deletions src/optimizer/projection_push_down_optimizer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -181,8 +181,8 @@ void ProjectionPushDownOptimizer::visitInsertRel(planner::LogicalOperator* op) {

void ProjectionPushDownOptimizer::visitDeleteNode(planner::LogicalOperator* op) {
auto deleteNode = (LogicalDeleteNode*)op;
for (auto& node : deleteNode->getNodesRef()) {
collectExpressionsInUse(node->getInternalID());
for (auto info : deleteNode->getInfos()) {
collectExpressionsInUse(info->node->getInternalID());
}
}

Expand Down
5 changes: 4 additions & 1 deletion src/parser/transform/transform_updating_clause.cpp
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
#include "common/assert.h"
#include "common/enums/clause_type.h"
#include "parser/query/updating_clause/delete_clause.h"
#include "parser/query/updating_clause/insert_clause.h"
#include "parser/query/updating_clause/merge_clause.h"
Expand Down Expand Up @@ -56,7 +57,9 @@ parsed_expression_pair Transformer::transformSetItem(CypherParser::OC_SetItemCon
}

std::unique_ptr<UpdatingClause> Transformer::transformDelete(CypherParser::OC_DeleteContext& ctx) {
auto deleteClause = std::make_unique<DeleteClause>();
auto deleteClauseType =
ctx.DETACH() ? common::DeleteClauseType::DETACH_DELETE : common::DeleteClauseType::DELETE;
auto deleteClause = std::make_unique<DeleteClause>(deleteClauseType);
for (auto& expression : ctx.oC_Expression()) {
deleteClause->addExpression(transformExpression(*expression));
}
Expand Down
Loading

0 comments on commit 53772d3

Please sign in to comment.