Add constraint verification update only on necessary actions (#1341)

This commit is contained in:
Josipmrden 2023-10-25 16:01:02 +02:00 committed by GitHub
parent 2426d7980d
commit 3d4d841753
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 375 additions and 38 deletions

View File

@ -6,6 +6,7 @@ add_library(mg-storage-v2 STATIC
commit_log.cpp
constraints/existence_constraints.cpp
constraints/constraints.cpp
constraint_verification_info.cpp
temporal.cpp
durability/durability.cpp
durability/serialization.cpp

View File

@ -0,0 +1,54 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/constraint_verification_info.hpp"
#include <algorithm>
namespace memgraph::storage {
ConstraintVerificationInfo::ConstraintVerificationInfo() = default;
ConstraintVerificationInfo::~ConstraintVerificationInfo() = default;
ConstraintVerificationInfo::ConstraintVerificationInfo(ConstraintVerificationInfo &&) noexcept = default;
ConstraintVerificationInfo &ConstraintVerificationInfo::operator=(ConstraintVerificationInfo &&) noexcept = default;
void ConstraintVerificationInfo::AddedLabel(Vertex const *vertex) { added_labels_.insert(vertex); }
void ConstraintVerificationInfo::AddedProperty(Vertex const *vertex) { added_properties_.insert(vertex); }
void ConstraintVerificationInfo::RemovedProperty(Vertex const *vertex) { removed_properties_.insert(vertex); }
auto ConstraintVerificationInfo::GetVerticesForUniqueConstraintChecking() const -> std::unordered_set<Vertex const *> {
std::unordered_set<Vertex const *> updated_vertices;
updated_vertices.insert(added_labels_.begin(), added_labels_.end());
updated_vertices.insert(added_properties_.begin(), added_properties_.end());
return updated_vertices;
}
auto ConstraintVerificationInfo::GetVerticesForExistenceConstraintChecking() const
-> std::unordered_set<Vertex const *> {
std::unordered_set<Vertex const *> updated_vertices;
updated_vertices.insert(added_labels_.begin(), added_labels_.end());
updated_vertices.insert(removed_properties_.begin(), removed_properties_.end());
return updated_vertices;
}
bool ConstraintVerificationInfo::NeedsUniqueConstraintVerification() const {
return !added_labels_.empty() || !added_properties_.empty();
}
bool ConstraintVerificationInfo::NeedsExistenceConstraintVerification() const {
return !added_labels_.empty() || !removed_properties_.empty();
}
} // namespace memgraph::storage

View File

@ -0,0 +1,61 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include "storage/v2/vertex.hpp"
namespace memgraph::storage {
// forward declarations
struct Vertex;
struct Transaction;
class PropertyValue;
/**
*/
struct ConstraintVerificationInfo final {
ConstraintVerificationInfo();
~ConstraintVerificationInfo();
// By design would be a mistake to copy the cache
ConstraintVerificationInfo(ConstraintVerificationInfo const &) = delete;
ConstraintVerificationInfo &operator=(ConstraintVerificationInfo const &) = delete;
ConstraintVerificationInfo(ConstraintVerificationInfo &&) noexcept;
ConstraintVerificationInfo &operator=(ConstraintVerificationInfo &&) noexcept;
void AddedLabel(Vertex const *vertex);
void AddedProperty(Vertex const *vertex);
void RemovedProperty(Vertex const *vertex);
auto GetVerticesForUniqueConstraintChecking() const -> std::unordered_set<Vertex const *>;
auto GetVerticesForExistenceConstraintChecking() const -> std::unordered_set<Vertex const *>;
bool NeedsUniqueConstraintVerification() const;
bool NeedsExistenceConstraintVerification() const;
private:
// Update unique constraints to check whether any vertex already has that value
// Update existence constraints to check whether for that label the node has all the properties present
std::unordered_set<Vertex const *> added_labels_;
// Update unique constraints to check whether any vertex already has that property
// No update to existence constraints because we only added a property
std::unordered_set<Vertex const *> added_properties_;
// No update to unique constraints because uniqueness is preserved
// Update existence constraints because it might be the referenced property of the constraint
std::unordered_set<Vertex const *> removed_properties_;
};
} // namespace memgraph::storage

View File

@ -760,21 +760,18 @@ utils::BasicResult<StorageManipulationError, void> InMemoryStorage::InMemoryAcce
// it.
mem_storage->commit_log_->MarkFinished(transaction_.start_timestamp);
} else {
// Validate that existence constraints are satisfied for all modified
// vertices.
for (const auto &delta : transaction_.deltas.use()) {
auto prev = delta.prev.Get();
MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
if (prev.type != PreviousPtr::Type::VERTEX) {
continue;
}
// No need to take any locks here because we modified this vertex and no
// one else can touch it until we commit.
auto validation_result = storage_->constraints_.existence_constraints_->Validate(*prev.vertex);
if (validation_result) {
Abort();
DMG_ASSERT(!commit_timestamp_.has_value());
return StorageManipulationError{*validation_result};
if (transaction_.constraint_verification_info.NeedsExistenceConstraintVerification()) {
const auto vertices_to_update =
transaction_.constraint_verification_info.GetVerticesForExistenceConstraintChecking();
for (auto const *vertex : vertices_to_update) {
// No need to take any locks here because we modified this vertex and no
// one else can touch it until we commit.
auto validation_result = storage_->constraints_.existence_constraints_->Validate(*vertex);
if (validation_result) {
Abort();
DMG_ASSERT(!commit_timestamp_.has_value());
return StorageManipulationError{*validation_result};
}
}
}
@ -792,32 +789,24 @@ utils::BasicResult<StorageManipulationError, void> InMemoryStorage::InMemoryAcce
static_cast<InMemoryUniqueConstraints *>(storage_->constraints_.unique_constraints_.get());
commit_timestamp_.emplace(mem_storage->CommitTimestamp(desired_commit_timestamp));
// Before committing and validating vertices against unique constraints,
// we have to update unique constraints with the vertices that are going
// to be validated/committed.
for (const auto &delta : transaction_.deltas.use()) {
auto prev = delta.prev.Get();
MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
if (prev.type != PreviousPtr::Type::VERTEX) {
continue;
}
mem_unique_constraints->UpdateBeforeCommit(prev.vertex, transaction_);
}
if (transaction_.constraint_verification_info.NeedsUniqueConstraintVerification()) {
// Before committing and validating vertices against unique constraints,
// we have to update unique constraints with the vertices that are going
// to be validated/committed.
const auto vertices_to_update =
transaction_.constraint_verification_info.GetVerticesForUniqueConstraintChecking();
// Validate that unique constraints are satisfied for all modified
// vertices.
for (const auto &delta : transaction_.deltas.use()) {
auto prev = delta.prev.Get();
MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
if (prev.type != PreviousPtr::Type::VERTEX) {
continue;
for (auto const *vertex : vertices_to_update) {
mem_unique_constraints->UpdateBeforeCommit(vertex, transaction_);
}
// No need to take any locks here because we modified this vertex and no
// one else can touch it until we commit.
unique_constraint_violation = mem_unique_constraints->Validate(*prev.vertex, transaction_, *commit_timestamp_);
if (unique_constraint_violation) {
break;
for (auto const *vertex : vertices_to_update) {
// No need to take any locks here because we modified this vertex and no
// one else can touch it until we commit.
unique_constraint_violation = mem_unique_constraints->Validate(*vertex, transaction_, *commit_timestamp_);
if (unique_constraint_violation) {
break;
}
}
}

View File

@ -51,6 +51,9 @@ class InMemoryUniqueConstraints : public UniqueConstraints {
/// @throw std::bad_alloc
void UpdateBeforeCommit(const Vertex *vertex, const Transaction &tx);
void UpdateBeforeCommit(const Vertex *vertex, std::unordered_set<LabelId> &added_labels,
std::unordered_set<PropertyId> &added_properties, const Transaction &tx);
/// Creates unique constraint on the given `label` and a list of `properties`.
/// Returns constraint violation if there are multiple vertices with the same
/// label and property values. Returns `CreationStatus::ALREADY_EXISTS` if

View File

@ -19,6 +19,7 @@
#include "utils/memory.hpp"
#include "utils/skip_list.hpp"
#include "storage/v2/constraint_verification_info.hpp"
#include "storage/v2/delta.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/isolation_level.hpp"
@ -101,6 +102,7 @@ struct Transaction {
// Used to speedup getting info about a vertex when there is a long delta
// chain involved in rebuilding that info.
mutable VertexInfoCache manyDeltasCache{};
mutable ConstraintVerificationInfo constraint_verification_info{};
// Store modified edges GID mapped to changed Delta and serialized edge key
// Only for disk storage

View File

@ -112,6 +112,7 @@ Result<bool> VertexAccessor::AddLabel(LabelId label) {
/// TODO: some by pointers, some by reference => not good, make it better
storage_->constraints_.unique_constraints_->UpdateOnAddLabel(label, *vertex_, transaction_->start_timestamp);
transaction_->constraint_verification_info.AddedLabel(vertex_);
storage_->indices_.UpdateOnAddLabel(label, vertex_, *transaction_);
transaction_->manyDeltasCache.Invalidate(vertex_, label);
@ -260,6 +261,11 @@ Result<PropertyValue> VertexAccessor::SetProperty(PropertyId property, const Pro
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, current_value);
vertex_->properties.SetProperty(property, value);
if (!value.IsNull()) {
transaction_->constraint_verification_info.AddedProperty(vertex_);
} else {
transaction_->constraint_verification_info.RemovedProperty(vertex_);
}
storage_->indices_.UpdateOnSetProperty(property, value, vertex_, *transaction_);
transaction_->manyDeltasCache.Invalidate(vertex_, property);
@ -283,6 +289,11 @@ Result<bool> VertexAccessor::InitProperties(const std::map<storage::PropertyId,
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, PropertyValue());
storage_->indices_.UpdateOnSetProperty(property, value, vertex_, *transaction_);
transaction_->manyDeltasCache.Invalidate(vertex_, property);
if (!value.IsNull()) {
transaction_->constraint_verification_info.AddedProperty(vertex_);
} else {
transaction_->constraint_verification_info.RemovedProperty(vertex_);
}
}
return true;
@ -307,6 +318,11 @@ Result<std::vector<std::tuple<PropertyId, PropertyValue, PropertyValue>>> Vertex
storage_->indices_.UpdateOnSetProperty(id, new_value, vertex_, *transaction_);
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), id, std::move(old_value));
transaction_->manyDeltasCache.Invalidate(vertex_, id);
if (!new_value.IsNull()) {
transaction_->constraint_verification_info.AddedProperty(vertex_);
} else {
transaction_->constraint_verification_info.RemovedProperty(vertex_);
}
}
return id_old_new_change;
@ -326,6 +342,7 @@ Result<std::map<PropertyId, PropertyValue>> VertexAccessor::ClearProperties() {
for (const auto &[property, value] : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, value);
storage_->indices_.UpdateOnSetProperty(property, PropertyValue(), vertex_, *transaction_);
transaction_->constraint_verification_info.RemovedProperty(vertex_);
transaction_->manyDeltasCache.Invalidate(vertex_, property);
}

View File

@ -66,6 +66,7 @@ add_subdirectory(concurrent_query_modules)
add_subdirectory(show_index_info)
add_subdirectory(set_properties)
add_subdirectory(transaction_rollback)
add_subdirectory(constraints)
copy_e2e_python_files(pytest_runner pytest_runner.sh "")
file(COPY ${CMAKE_CURRENT_SOURCE_DIR}/memgraph-selfsigned.crt DESTINATION ${CMAKE_CURRENT_BINARY_DIR})

View File

@ -0,0 +1,6 @@
function(copy_constraint_validation_e2e_python_files FILE_NAME)
copy_e2e_python_files(constraint_validation ${FILE_NAME})
endfunction()
copy_constraint_validation_e2e_python_files(common.py)
copy_constraint_validation_e2e_python_files(constraints_validation.py)

View File

@ -0,0 +1,26 @@
# Copyright 2023 Memgraph Ltd.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
# License, and you may not use this file except in compliance with the Business Source License.
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0, included in the file
# licenses/APL.txt.
import pytest
from gqlalchemy import Memgraph
@pytest.fixture
def memgraph(**kwargs) -> Memgraph:
memgraph = Memgraph()
yield memgraph
memgraph.drop_indexes()
memgraph.ensure_constraints([])
ts = list(memgraph.execute_and_fetch("SHOW TRANSACTIONS;"))
memgraph.drop_database()

View File

@ -0,0 +1,163 @@
# Copyright 2023 Memgraph Ltd.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
# License, and you may not use this file except in compliance with the Business Source License.
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0, included in the file
# licenses/APL.txt.
import sys
import pytest
from common import memgraph
from gqlalchemy import GQLAlchemyError
def test_cant_add_2_nodes_with_same_property_and_unique_constraint(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
with pytest.raises(GQLAlchemyError):
memgraph.execute("CREATE (:Node {prop: 1})")
def test_unique_constraint_fails_when_adding_label(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node2 {prop: 1})")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n:Node2) SET n:Node;")
def test_unique_constraint_fails_when_setting_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node:Node2 {prop: 2})")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n:Node2) SET n.prop = 1;")
def test_unique_constraint_fails_when_updating_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node {prop: 2})")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) WHERE n.prop = 2 SET n.prop = 1;")
def test_unique_constraint_passes_when_setting_property_to_null(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node {prop: 2})")
memgraph.execute("MATCH (n) SET n.prop = null;")
def test_unique_constraint_passes_when_removing_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node {prop: 2})")
memgraph.execute("MATCH (n) REMOVE n.prop;")
def test_unique_constraint_fails_when_setting_properties(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node {prop: 2})")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) SET n += {prop: 1, prop2: 1};")
def test_unique_constraint_fails_when_setting_properties_alternative(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT n.prop IS UNIQUE;")
memgraph.execute("CREATE (:Node {prop: 1})")
memgraph.execute("CREATE (:Node {prop: 2})")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) SET n = {prop: 1, prop2: 1};")
def test_existence_constraint_fails_when_adding_label_with_nonexistent_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
memgraph.execute("CREATE ();")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) SET n:Node")
def test_existence_constraint_fails_when_creating_node_with_nonexistent_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
with pytest.raises(GQLAlchemyError):
memgraph.execute("CREATE (:Node);")
def test_existence_constraint_passes_when_updating_node_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
memgraph.execute("CREATE (:Node {prop: 1});")
memgraph.execute("MATCH (n) SET n.prop = 2;")
def test_existence_constraint_fails_when_updating_node_property_to_null(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
memgraph.execute("CREATE (:Node {prop: 1});")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) SET n.prop = null;")
def test_existence_constraint_fails_when_removing_node_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
memgraph.execute("CREATE (:Node {prop: 1});")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) REMOVE n.prop;")
def test_existence_constraint_fails_when_setting_properties(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
memgraph.execute("CREATE (:Node {prop: 1});")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) SET n += {prop: null, prop2: 2};")
def test_existence_constraint_fails_when_setting_properties_alternative(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
memgraph.execute("CREATE (:Node {prop: 1});")
with pytest.raises(GQLAlchemyError):
memgraph.execute("MATCH (n) SET n = {prop: null, prop2: 2};")
def test_existence_constraint_fails_when_creating_node_null_property(memgraph):
memgraph.execute("CREATE CONSTRAINT ON (n:Node) ASSERT EXISTS (n.prop);")
with pytest.raises(GQLAlchemyError):
memgraph.execute("CREATE (:Node {prop: null});")
if __name__ == "__main__":
sys.exit(pytest.main([__file__, "-rA"]))

View File

@ -0,0 +1,14 @@
analyze_graph_cluster: &analyze_graph_cluster
cluster:
main:
args: ["--bolt-port", "7687", "--log-level=TRACE"]
log_file: "analyze_graph.log"
setup_queries: []
validation_queries: []
workloads:
- name: "Constraint logic validation"
binary: "tests/e2e/pytest_runner.sh"
args: ["constraints/constraints_validation.py"]
<<: *analyze_graph_cluster