diff --git a/src/storage/v2/CMakeLists.txt b/src/storage/v2/CMakeLists.txt
index 3b9794015..1a3776624 100644
--- a/src/storage/v2/CMakeLists.txt
+++ b/src/storage/v2/CMakeLists.txt
@@ -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
diff --git a/src/storage/v2/constraint_verification_info.cpp b/src/storage/v2/constraint_verification_info.cpp
new file mode 100644
index 000000000..8de712e11
--- /dev/null
+++ b/src/storage/v2/constraint_verification_info.cpp
@@ -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
diff --git a/src/storage/v2/constraint_verification_info.hpp b/src/storage/v2/constraint_verification_info.hpp
new file mode 100644
index 000000000..5fe42c197
--- /dev/null
+++ b/src/storage/v2/constraint_verification_info.hpp
@@ -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
diff --git a/src/storage/v2/inmemory/storage.cpp b/src/storage/v2/inmemory/storage.cpp
index d9257d673..2bbb1888a 100644
--- a/src/storage/v2/inmemory/storage.cpp
+++ b/src/storage/v2/inmemory/storage.cpp
@@ -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;
+          }
         }
       }
 
diff --git a/src/storage/v2/inmemory/unique_constraints.hpp b/src/storage/v2/inmemory/unique_constraints.hpp
index 64565056b..45472ca74 100644
--- a/src/storage/v2/inmemory/unique_constraints.hpp
+++ b/src/storage/v2/inmemory/unique_constraints.hpp
@@ -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
diff --git a/src/storage/v2/transaction.hpp b/src/storage/v2/transaction.hpp
index dab484095..2bdd68a94 100644
--- a/src/storage/v2/transaction.hpp
+++ b/src/storage/v2/transaction.hpp
@@ -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
diff --git a/src/storage/v2/vertex_accessor.cpp b/src/storage/v2/vertex_accessor.cpp
index 3d70a0051..924c305ad 100644
--- a/src/storage/v2/vertex_accessor.cpp
+++ b/src/storage/v2/vertex_accessor.cpp
@@ -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);
   }
 
diff --git a/tests/e2e/CMakeLists.txt b/tests/e2e/CMakeLists.txt
index 8ebd66dda..dc3c8f5b9 100644
--- a/tests/e2e/CMakeLists.txt
+++ b/tests/e2e/CMakeLists.txt
@@ -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})
diff --git a/tests/e2e/constraints/CMakeLists.txt b/tests/e2e/constraints/CMakeLists.txt
new file mode 100644
index 000000000..0c4ff72d9
--- /dev/null
+++ b/tests/e2e/constraints/CMakeLists.txt
@@ -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)
diff --git a/tests/e2e/constraints/common.py b/tests/e2e/constraints/common.py
new file mode 100644
index 000000000..9fcc3d613
--- /dev/null
+++ b/tests/e2e/constraints/common.py
@@ -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()
diff --git a/tests/e2e/constraints/constraints_validation.py b/tests/e2e/constraints/constraints_validation.py
new file mode 100644
index 000000000..bf101ae5c
--- /dev/null
+++ b/tests/e2e/constraints/constraints_validation.py
@@ -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"]))
diff --git a/tests/e2e/constraints/workloads.yaml b/tests/e2e/constraints/workloads.yaml
new file mode 100644
index 000000000..ca506c690
--- /dev/null
+++ b/tests/e2e/constraints/workloads.yaml
@@ -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