You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ariatosca.apache.org by mx...@apache.org on 2016/12/06 14:54:51 UTC

incubator-ariatosca git commit: review 3 [Forced Update!]

Repository: incubator-ariatosca
Updated Branches:
  refs/heads/ARIA-30-SQL-based-storage-implementation 27174e550 -> 165b9ca8a (forced update)


review 3


Project: http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/commit/165b9ca8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/tree/165b9ca8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/diff/165b9ca8

Branch: refs/heads/ARIA-30-SQL-based-storage-implementation
Commit: 165b9ca8a4f4fe7b93366ad5dfa5ee381b0161bc
Parents: 72a4e48
Author: mxmrlv <mx...@gmail.com>
Authored: Mon Dec 5 18:32:18 2016 +0200
Committer: mxmrlv <mx...@gmail.com>
Committed: Tue Dec 6 16:54:40 2016 +0200

----------------------------------------------------------------------
 aria/orchestrator/context/common.py             |  17 -
 aria/orchestrator/context/operation.py          |   1 -
 aria/orchestrator/context/workflow.py           |  28 +-
 aria/orchestrator/workflows/core/task.py        |   1 -
 aria/storage/models.py                          |  69 +-
 aria/storage/sql_mapi.py                        |  11 +-
 aria/storage/structures.py                      |  71 +-
 tests/mock/models.py                            |   3 -
 tests/orchestrator/context/test_operation.py    |   6 +-
 tests/orchestrator/context/test_toolbelt.py     |   5 +-
 tests/orchestrator/context/test_workflow.py     |  11 +-
 tests/orchestrator/workflows/api/test_task.py   |   5 +-
 .../workflows/builtin/test_execute_operation.py |   5 +-
 tests/storage/__init__.py                       |   5 +-
 tests/storage/test_models.py                    | 838 ++++++++++++++++---
 15 files changed, 837 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/orchestrator/context/common.py
----------------------------------------------------------------------
diff --git a/aria/orchestrator/context/common.py b/aria/orchestrator/context/common.py
index 75bb0fb..14efd9d 100644
--- a/aria/orchestrator/context/common.py
+++ b/aria/orchestrator/context/common.py
@@ -33,7 +33,6 @@ class BaseContext(logger.LoggerMixin):
             resource_storage,
             deployment_id,
             workflow_name,
-            execution_name=None,
             task_max_attempts=1,
             task_retry_interval=0,
             task_ignore_failure=False,
@@ -45,7 +44,6 @@ class BaseContext(logger.LoggerMixin):
         self._resource = resource_storage
         self._deployment_id = deployment_id
         self._workflow_name = workflow_name
-        self._execution_name = execution_name or str(uuid4())
         self._task_max_attempts = task_max_attempts
         self._task_retry_interval = task_retry_interval
         self._task_ignore_failure = task_ignore_failure
@@ -55,7 +53,6 @@ class BaseContext(logger.LoggerMixin):
             '{name}(name={self.name}, '
             'deployment_id={self._deployment_id}, '
             'workflow_name={self._workflow_name}, '
-            'execution_name={self._execution_name})'
             .format(name=self.__class__.__name__, self=self))
 
     @property
@@ -89,20 +86,6 @@ class BaseContext(logger.LoggerMixin):
         return self.model.deployment.get(self._deployment_id)
 
     @property
-    def execution(self):
-        """
-        The execution model
-        """
-        return self.model.execution.get_by_name(self._execution_name)
-
-    @execution.setter
-    def execution(self, value):
-        """
-        Store the execution in the model storage
-        """
-        self.model.execution.put(value)
-
-    @property
     def name(self):
         """
         The operation name

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/orchestrator/context/operation.py
----------------------------------------------------------------------
diff --git a/aria/orchestrator/context/operation.py b/aria/orchestrator/context/operation.py
index 8410c46..02469a7 100644
--- a/aria/orchestrator/context/operation.py
+++ b/aria/orchestrator/context/operation.py
@@ -33,7 +33,6 @@ class BaseOperationContext(BaseContext):
             resource_storage=workflow_context.resource,
             deployment_id=workflow_context._deployment_id,
             workflow_name=workflow_context._workflow_name,
-            execution_name=workflow_context._execution_name,
             **kwargs)
         self._task_model = task
         self._actor = self.task.actor

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/orchestrator/context/workflow.py
----------------------------------------------------------------------
diff --git a/aria/orchestrator/context/workflow.py b/aria/orchestrator/context/workflow.py
index 52ea630..e2e8e25 100644
--- a/aria/orchestrator/context/workflow.py
+++ b/aria/orchestrator/context/workflow.py
@@ -21,8 +21,6 @@ import threading
 from contextlib import contextmanager
 from datetime import datetime
 
-from aria import storage
-
 from .exceptions import ContextException
 from .common import BaseContext
 
@@ -31,28 +29,23 @@ class WorkflowContext(BaseContext):
     """
     Context object used during workflow creation and execution
     """
-    def __init__(self, parameters=None, *args, **kwargs):
+    def __init__(self, parameters=None, execution_id=None, *args, **kwargs):
         super(WorkflowContext, self).__init__(*args, **kwargs)
         self.parameters = parameters or {}
         # TODO: execution creation should happen somewhere else
         # should be moved there, when such logical place exists
-        try:
-            self.model.execution.get_by_name(self._execution_name)
-        except storage.exceptions.StorageError:
-            self._create_execution()
+        self._execution_id = self._create_execution() if execution_id is None else execution_id
 
     def __repr__(self):
         return (
             '{name}(deployment_id={self._deployment_id}, '
-            'workflow_name={self._workflow_name}, '
-            'execution_name={self._execution_name})'.format(
+            'workflow_name={self._workflow_name}'.format(
                 name=self.__class__.__name__, self=self))
 
     def _create_execution(self):
         execution_cls = self.model.execution.model_cls
         now = datetime.utcnow()
         execution = self.model.execution.model_cls(
-            name=self._execution_name,
             blueprint_id=self.blueprint.id,
             deployment_id=self.deployment.id,
             workflow_name=self._workflow_name,
@@ -61,6 +54,21 @@ class WorkflowContext(BaseContext):
             parameters=self.parameters,
         )
         self.model.execution.put(execution)
+        return execution.id
+
+    @property
+    def execution(self):
+        """
+        The execution model
+        """
+        return self.model.execution.get(self._execution_id)
+
+    @execution.setter
+    def execution(self, value):
+        """
+        Store the execution in the model storage
+        """
+        self.model.execution.put(value)
 
     @property
     def nodes(self):

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/orchestrator/workflows/core/task.py
----------------------------------------------------------------------
diff --git a/aria/orchestrator/workflows/core/task.py b/aria/orchestrator/workflows/core/task.py
index 1163722..16c0491 100644
--- a/aria/orchestrator/workflows/core/task.py
+++ b/aria/orchestrator/workflows/core/task.py
@@ -124,7 +124,6 @@ class OperationTask(BaseTask):
             instance_id=api_task.actor.id,
             inputs=api_task.inputs,
             status=base_task_model.PENDING,
-            execution_id=self._workflow_context._execution_name,
             max_attempts=api_task.max_attempts,
             retry_interval=api_task.retry_interval,
             ignore_failure=api_task.ignore_failure,

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/storage/models.py
----------------------------------------------------------------------
diff --git a/aria/storage/models.py b/aria/storage/models.py
index d58420f..781894b 100644
--- a/aria/storage/models.py
+++ b/aria/storage/models.py
@@ -40,7 +40,6 @@ from collections import namedtuple
 from datetime import datetime
 
 from sqlalchemy.ext.declarative.base import declared_attr
-from sqlalchemy.orm import validates
 
 from .structures import (
     SQLModelBase,
@@ -51,14 +50,13 @@ from .structures import (
     Boolean,
     Enum,
     String,
-    PickleType,
     Float,
+    List,
     Dict,
     foreign_key,
     one_to_many_relationship,
-    relationship,
-    orm,
-)
+    relationship_to_self,
+    orm)
 
 __all__ = (
     'Blueprint',
@@ -85,6 +83,7 @@ class Blueprint(SQLModelBase):
     """
     __tablename__ = 'blueprints'
 
+    name = Column(Text, index=True)
     created_at = Column(DateTime, nullable=False, index=True)
     main_file_name = Column(Text, nullable=False)
     plan = Column(Dict, nullable=False)
@@ -102,6 +101,7 @@ class Deployment(SQLModelBase):
 
     blueprint_id = foreign_key(Blueprint.id)
 
+    name = Column(Text, index=True)
     created_at = Column(DateTime, nullable=False, index=True)
     description = Column(Text)
     inputs = Column(Dict)
@@ -143,7 +143,7 @@ class Execution(SQLModelBase):
         CANCELLING: END_STATES
     }
 
-    @validates('status')
+    @orm.validates('status')
     def validate_status(self, key, value):
         """Validation function that verifies execution status transitions are OK"""
         try:
@@ -199,7 +199,7 @@ class DeploymentUpdate(SQLModelBase):
     _private_fields = ['execution_id', 'deployment_id']
 
     created_at = Column(DateTime, nullable=False, index=True)
-    deployment_plan = Column(Dict)
+    deployment_plan = Column(Dict, nullable=False)
     deployment_update_node_instances = Column(Dict)
     deployment_update_deployment = Column(Dict)
     deployment_update_nodes = Column(Dict)
@@ -249,9 +249,9 @@ class DeploymentUpdateStep(SQLModelBase):
     deployment_update_id = foreign_key(DeploymentUpdate.id)
     _private_fields = ['deployment_update_id']
 
-    action = Column(Enum(*ACTION_TYPES, name='action_type'))
+    action = Column(Enum(*ACTION_TYPES, name='action_type'), nullable=False)
     entity_id = Column(Text, nullable=False)
-    entity_type = Column(Enum(*ENTITY_TYPES, name='entity_type'))
+    entity_type = Column(Enum(*ENTITY_TYPES, name='entity_type'), nullable=False)
 
     @declared_attr
     def deployment_update(cls):
@@ -324,11 +324,11 @@ class Node(SQLModelBase):
     Node model representation.
     """
     __tablename__ = 'nodes'
-    id = Column(Integer, primary_key=True)
 
     # See base class for an explanation on these properties
     is_id_unique = False
 
+    name = Column(Text, index=True)
     _private_fields = ['deployment_id', 'host_id']
     deployment_id = foreign_key(Deployment.id)
     host_id = foreign_key('nodes.id', nullable=True)
@@ -340,7 +340,6 @@ class Node(SQLModelBase):
     deploy_number_of_instances = Column(Integer, nullable=False)
     # TODO: This probably should be a foreign key, but there's no guarantee
     # in the code, currently, that the host will be created beforehand
-    _host_id = foreign_key('nodes.id', nullable=True)
     max_number_of_instances = Column(Integer, nullable=False)
     min_number_of_instances = Column(Integer, nullable=False)
     number_of_instances = Column(Integer, nullable=False)
@@ -350,12 +349,12 @@ class Node(SQLModelBase):
     properties = Column(Dict)
     operations = Column(Dict)
     type = Column(Text, nullable=False, index=True)
-    type_hierarchy = Column(PickleType)
+    type_hierarchy = Column(List)
+
+    @declared_attr
+    def host(cls):
+        return relationship_to_self(cls, cls.host_id, cls.id)
 
-    host = relationship('Node',
-                        foreign_keys=[host_id],
-                        remote_side=[id],
-                        backref=orm.backref('guests'))
 
 class Relationship(SQLModelBase):
     """
@@ -383,11 +382,11 @@ class Relationship(SQLModelBase):
                                         'inbound_relationships')
 
     source_interfaces = Column(Dict)
-    source_operations = Column(Dict)
+    source_operations = Column(Dict, nullable=False)
     target_interfaces = Column(Dict)
-    target_operations = Column(Dict)
-    type = Column(String)
-    type_hierarchy = Column(PickleType)
+    target_operations = Column(Dict, nullable=False)
+    type = Column(String, nullable=False)
+    type_hierarchy = Column(List)
     properties = Column(Dict)
 
 
@@ -396,7 +395,6 @@ class NodeInstance(SQLModelBase):
     Node instance model representation.
     """
     __tablename__ = 'node_instances'
-    id = Column(Integer, primary_key=True)
 
     node_id = foreign_key(Node.id)
     deployment_id = foreign_key(Deployment.id)
@@ -404,21 +402,20 @@ class NodeInstance(SQLModelBase):
 
     _private_fields = ['node_id', 'host_id']
 
-    # TODO: This probably should be a foreign key, but there's no guarantee
-    # in the code, currently, that the host will be created beforehand
+    name = Column(Text, index=True)
     runtime_properties = Column(Dict)
     scaling_groups = Column(Dict)
     state = Column(Text, nullable=False)
     version = Column(Integer, default=1)
-    host = relationship('NodeInstance',
-                        foreign_keys=[host_id],
-                        remote_side=[id],
-                        backref=orm.backref('guests'))
 
     @declared_attr
     def node(cls):
         return one_to_many_relationship(cls, Node, cls.node_id)
 
+    @declared_attr
+    def host(cls):
+        return relationship_to_self(cls, cls.host_id, cls.id)
+
 
 class RelationshipInstance(SQLModelBase):
     """
@@ -491,8 +488,11 @@ class Task(SQLModelBase):
     __tablename__ = 'task'
     node_instance_id = foreign_key(NodeInstance.id, nullable=True)
     relationship_instance_id = foreign_key(RelationshipInstance.id, nullable=True)
+    execution_id = foreign_key(Execution.id)
 
-    _private_fields = ['node_instance_id', 'relationship_instance_id']
+    _private_fields = ['node_instance_id',
+                       'relationship_instance_id',
+                       'execution_id']
 
     @declared_attr
     def node_instance(cls):
@@ -522,7 +522,7 @@ class Task(SQLModelBase):
     WAIT_STATES = [PENDING, RETRYING]
     END_STATES = [SUCCESS, FAILED]
 
-    @validates('max_attempts')
+    @orm.validates('max_attempts')
     def validate_max_attempts(self, _, value):                                  # pylint: disable=no-self-use
         """Validates that max attempts is either -1 or a positive number"""
         if value < 1 and value != Task.INFINITE_RETRIES:
@@ -534,10 +534,9 @@ class Task(SQLModelBase):
 
     status = Column(Enum(*STATES), name='status', default=PENDING)
 
-    execution_id = Column(String)
-    due_at = Column(DateTime, default=datetime.utcnow, nullable=True)
-    started_at = Column(DateTime, default=None, nullable=True)
-    ended_at = Column(DateTime, default=None, nullable=True)
+    due_at = Column(DateTime, default=datetime.utcnow)
+    started_at = Column(DateTime, default=None)
+    ended_at = Column(DateTime, default=None)
     max_attempts = Column(Integer, default=1)
     retry_count = Column(Integer, default=0)
     retry_interval = Column(Float, default=0)
@@ -548,6 +547,10 @@ class Task(SQLModelBase):
     operation_mapping = Column(String)
     inputs = Column(Dict)
 
+    @declared_attr
+    def execution(cls):
+        return one_to_many_relationship(cls, Execution, cls.execution_id)
+
     @property
     def actor(self):
         """

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/storage/sql_mapi.py
----------------------------------------------------------------------
diff --git a/aria/storage/sql_mapi.py b/aria/storage/sql_mapi.py
index 51e4ae1..cde40c2 100644
--- a/aria/storage/sql_mapi.py
+++ b/aria/storage/sql_mapi.py
@@ -38,12 +38,10 @@ class SQLAlchemyModelAPI(api.ModelAPI):
         self._engine = engine
         self._session = session
 
-    def get(self, entry_id, include=None, locking=False, **kwargs):
+    def get(self, entry_id, include=None, **kwargs):
         """Return a single result based on the model class and element ID
         """
         query = self._get_query(include, {'id': entry_id})
-        if locking:
-            query = query.with_for_update()
         result = query.first()
 
         if not result:
@@ -54,6 +52,7 @@ class SQLAlchemyModelAPI(api.ModelAPI):
         return result
 
     def get_by_name(self, entry_name, include=None, **kwargs):
+        assert hasattr(self.model_cls, 'name')
         result = self.list(include=include, filters={'name': entry_name})
         if not result:
             raise exceptions.StorageError(
@@ -92,9 +91,7 @@ class SQLAlchemyModelAPI(api.ModelAPI):
              **kwargs):
         """Return a (possibly empty) list of `model_class` results
         """
-        query = self._get_query(include, filters, sort)
-        for result in query:
-            yield result
+        return iter(self._get_query(include, filters, sort))
 
     def put(self, entry, **kwargs):
         """Create a `model_class` instance from a serializable `model` object
@@ -155,7 +152,7 @@ class SQLAlchemyModelAPI(api.ModelAPI):
         """
         try:
             self._session.commit()
-        except SQLAlchemyError as e:
+        except (SQLAlchemyError, ValueError) as e:
             self._session.rollback()
             raise exceptions.StorageError('SQL Storage error: {0}'.format(str(e)))
 

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/aria/storage/structures.py
----------------------------------------------------------------------
diff --git a/aria/storage/structures.py b/aria/storage/structures.py
index 317daec..c2e5595 100644
--- a/aria/storage/structures.py
+++ b/aria/storage/structures.py
@@ -28,7 +28,6 @@ classes:
 """
 import json
 
-from sqlalchemy import VARCHAR
 from sqlalchemy.ext.mutable import Mutable
 from sqlalchemy.orm import relationship, backref
 from sqlalchemy.ext.declarative import declarative_base
@@ -36,6 +35,8 @@ from sqlalchemy.ext.declarative import declarative_base
 from sqlalchemy.ext.associationproxy import association_proxy
 from sqlalchemy import (
     schema,
+    VARCHAR,
+    ARRAY,
     Column,
     Integer,
     Text,
@@ -50,6 +51,8 @@ from sqlalchemy import (
     orm,
 )
 
+from aria.storage import exceptions
+
 Model = declarative_base()
 
 
@@ -87,7 +90,15 @@ def one_to_many_relationship(child_class,
     )
 
 
-class _DictType(TypeDecorator):
+def relationship_to_self(self_cls, parent_key, self_key):
+    return relationship(
+        self_cls,
+        foreign_keys=parent_key,
+        remote_side=self_key
+    )
+
+
+class _MutableType(TypeDecorator):
     """
     Dict representation of type.
     """
@@ -95,10 +106,6 @@ class _DictType(TypeDecorator):
     def process_literal_param(self, value, dialect):
         pass
 
-    @property
-    def python_type(self):
-        return dict
-
     impl = VARCHAR
 
     def process_bind_param(self, value, dialect):
@@ -112,6 +119,18 @@ class _DictType(TypeDecorator):
         return value
 
 
+class _DictType(_MutableType):
+    @property
+    def python_type(self):
+        return dict
+
+
+class _ListType(_MutableType):
+    @property
+    def python_type(self):
+        return list
+
+
 class _MutableDict(Mutable, dict):
     """
     Enables tracking for dict values.
@@ -125,7 +144,10 @@ class _MutableDict(Mutable, dict):
                 return _MutableDict(value)
 
             # this call will raise ValueError
-            return Mutable.coerce(key, value)
+            try:
+                return Mutable.coerce(key, value)
+            except ValueError as e:
+                raise exceptions.StorageError('SQL Storage error: {0}'.format(str(e)))
         else:
             return value
 
@@ -141,8 +163,36 @@ class _MutableDict(Mutable, dict):
         dict.__delitem__(self, key)
         self.changed()
 
-Dict = _MutableDict.as_mutable(_DictType)
 
+class _MutableList(Mutable, list):
+
+    @classmethod
+    def coerce(cls, key, value):
+        "Convert plain dictionaries to MutableDict."
+
+        if not isinstance(value, _MutableList):
+            if isinstance(value, list):
+                return _MutableList(value)
+
+            # this call will raise ValueError
+            try:
+                return Mutable.coerce(key, value)
+            except ValueError as e:
+                raise exceptions.StorageError('SQL Storage error: {0}'.format(str(e)))
+        else:
+            return value
+
+    def __setitem__(self, key, value):
+        list.__setitem__(self, key, value)
+        self.changed()
+
+    def __delitem__(self, key):
+        list.__delitem__(self, key)
+
+
+
+Dict = _MutableDict.as_mutable(_DictType)
+List = _MutableList.as_mutable(_ListType)
 
 class SQLModelBase(Model):
     """
@@ -157,11 +207,6 @@ class SQLModelBase(Model):
     _private_fields = []
 
     id = Column(Integer, primary_key=True, autoincrement=True)
-    name = Column(Text, index=True)
-
-    @classmethod
-    def unique_id(cls):
-        return 'id'
 
     def to_dict(self, suppress_error=False):
         """Return a dict representation of the model

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/mock/models.py
----------------------------------------------------------------------
diff --git a/tests/mock/models.py b/tests/mock/models.py
index cdda614..e2e3d2f 100644
--- a/tests/mock/models.py
+++ b/tests/mock/models.py
@@ -64,7 +64,6 @@ def get_dependency_node_instance(dependency_node):
 
 def get_relationship(source=None, target=None):
     return models.Relationship(
-        name=RELATIONSHIP_NAME,
         source_node_id=source.id,
         target_node_id=target.id,
         source_interfaces={},
@@ -79,7 +78,6 @@ def get_relationship(source=None, target=None):
 
 def get_relationship_instance(source_instance, target_instance, relationship):
     return models.RelationshipInstance(
-        name=RELATIONSHIP_INSTANCE_NAME,
         relationship_id=relationship.id,
         target_node_instance_id=target_instance.id,
         source_node_instance_id=source_instance.id,
@@ -128,7 +126,6 @@ def get_blueprint():
 
 def get_execution(deployment):
     return models.Execution(
-        name=EXECUTION_NAME,
         deployment_id=deployment.id,
         blueprint_id=deployment.blueprint.id,
         status=models.Execution.STARTED,

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/orchestrator/context/test_operation.py
----------------------------------------------------------------------
diff --git a/tests/orchestrator/context/test_operation.py b/tests/orchestrator/context/test_operation.py
index 5e81f26..ab442a4 100644
--- a/tests/orchestrator/context/test_operation.py
+++ b/tests/orchestrator/context/test_operation.py
@@ -91,14 +91,12 @@ def test_node_operation_task_execution(ctx, executor):
 
 def test_relationship_operation_task_execution(ctx, executor):
     operation_name = 'aria.interfaces.relationship_lifecycle.postconfigure'
-
-    relationship = ctx.model.relationship.get_by_name(mock.models.RELATIONSHIP_NAME)
+    relationship = ctx.model.relationship.list()[0]
     relationship.source_operations[operation_name] = {
         'operation': op_path(my_operation, module_path=__name__)
     }
     ctx.model.relationship.update(relationship)
-    relationship_instance = ctx.model.relationship_instance.get_by_name(
-        mock.models.RELATIONSHIP_INSTANCE_NAME)
+    relationship_instance = ctx.model.relationship_instance.list()[0]
 
     dependency_node = ctx.model.node.get_by_name(mock.models.DEPENDENCY_NODE_NAME)
     dependency_node_instance = \

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/orchestrator/context/test_toolbelt.py
----------------------------------------------------------------------
diff --git a/tests/orchestrator/context/test_toolbelt.py b/tests/orchestrator/context/test_toolbelt.py
index 6a742dc..7b3f1a3 100644
--- a/tests/orchestrator/context/test_toolbelt.py
+++ b/tests/orchestrator/context/test_toolbelt.py
@@ -66,9 +66,8 @@ def _get_elements(workflow_context):
     dependent_node_instance.host_id = dependent_node_instance.id
     workflow_context.model.node_instance.update(dependent_node_instance)
 
-    relationship = workflow_context.model.relationship.get_by_name(mock.models.RELATIONSHIP_NAME)
-    relationship_instance = workflow_context.model.relationship_instance.get_by_name(
-        mock.models.RELATIONSHIP_INSTANCE_NAME)
+    relationship = workflow_context.model.relationship.list()[0]
+    relationship_instance = workflow_context.model.relationship_instance.list()[0]
     return dependency_node, dependency_node_instance, dependent_node, dependent_node_instance, \
         relationship, relationship_instance
 

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/orchestrator/context/test_workflow.py
----------------------------------------------------------------------
diff --git a/tests/orchestrator/context/test_workflow.py b/tests/orchestrator/context/test_workflow.py
index 2cb24fd..84fe95a 100644
--- a/tests/orchestrator/context/test_workflow.py
+++ b/tests/orchestrator/context/test_workflow.py
@@ -27,9 +27,8 @@ from tests.mock import models
 class TestWorkflowContext(object):
 
     def test_execution_creation_on_workflow_context_creation(self, storage):
-        self._create_ctx(storage)
-        execution = storage.execution.get_by_name(models.EXECUTION_NAME)
-        assert execution.name == models.EXECUTION_NAME
+        ctx = self._create_ctx(storage)
+        execution = storage.execution.get(ctx.execution.id)
         assert execution.deployment == storage.deployment.get_by_name(models.DEPLOYMENT_NAME)
         assert execution.workflow_name == models.WORKFLOW_NAME
         assert execution.blueprint == storage.blueprint.get_by_name(models.BLUEPRINT_NAME)
@@ -43,13 +42,17 @@ class TestWorkflowContext(object):
 
     @staticmethod
     def _create_ctx(storage):
+        """
+
+        :param storage:
+        :return WorkflowContext:
+        """
         return context.workflow.WorkflowContext(
             name='simple_context',
             model_storage=storage,
             resource_storage=None,
             deployment_id=storage.deployment.get_by_name(models.DEPLOYMENT_NAME).id,
             workflow_name=models.WORKFLOW_NAME,
-            execution_name=models.EXECUTION_NAME,
             task_max_attempts=models.TASK_MAX_ATTEMPTS,
             task_retry_interval=models.TASK_RETRY_INTERVAL
         )

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/orchestrator/workflows/api/test_task.py
----------------------------------------------------------------------
diff --git a/tests/orchestrator/workflows/api/test_task.py b/tests/orchestrator/workflows/api/test_task.py
index 6b89257..22dddd8 100644
--- a/tests/orchestrator/workflows/api/test_task.py
+++ b/tests/orchestrator/workflows/api/test_task.py
@@ -70,10 +70,9 @@ class TestOperationTask(object):
     def test_relationship_operation_task_creation(self, ctx):
         operation_name = 'aria.interfaces.relationship_lifecycle.preconfigure'
         op_details = {'operation': True}
-        relationship = ctx.model.relationship.get_by_name(mock.models.RELATIONSHIP_NAME)
+        relationship = ctx.model.relationship.list()[0]
         relationship.source_operations[operation_name] = op_details
-        relationship_instance = ctx.model.relationship_instance.get_by_name(
-            mock.models.RELATIONSHIP_INSTANCE_NAME)
+        relationship_instance = ctx.model.relationship_instance.list()[0]
         inputs = {'inputs': True}
         max_attempts = 10
         retry_interval = 10

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/orchestrator/workflows/builtin/test_execute_operation.py
----------------------------------------------------------------------
diff --git a/tests/orchestrator/workflows/builtin/test_execute_operation.py b/tests/orchestrator/workflows/builtin/test_execute_operation.py
index 435ad0f..c7ab524 100644
--- a/tests/orchestrator/workflows/builtin/test_execute_operation.py
+++ b/tests/orchestrator/workflows/builtin/test_execute_operation.py
@@ -29,9 +29,8 @@ def ctx():
 
 
 def test_execute_operation(ctx):
-    node_instance = ctx.model.node_instance.list(filters={
-        'name': mock.models.DEPENDENCY_NODE_INSTANCE_NAME
-    })[0]
+    node_instance = ctx.model.node_instance.get_by_name(mock.models.DEPENDENCY_NODE_INSTANCE_NAME)
+
     operation_name = mock.operations.NODE_OPERATIONS_INSTALL[0]
 
     execute_tasks = list(

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/storage/__init__.py
----------------------------------------------------------------------
diff --git a/tests/storage/__init__.py b/tests/storage/__init__.py
index 2c848b4..ba1269d 100644
--- a/tests/storage/__init__.py
+++ b/tests/storage/__init__.py
@@ -18,10 +18,11 @@ from shutil import rmtree
 
 from sqlalchemy import (
     create_engine,
-    MetaData,
     orm)
 from sqlalchemy.pool import StaticPool
 
+from aria.storage import structures
+
 
 class TestFileSystem(object):
 
@@ -37,5 +38,5 @@ def get_sqlite_api_kwargs():
                            connect_args={'check_same_thread': False},
                            poolclass=StaticPool)
     session = orm.sessionmaker(bind=engine)()
-    MetaData().create_all(engine)
+    structures.Model.metadata.create_all(engine)
     return dict(engine=engine, session=session)

http://git-wip-us.apache.org/repos/asf/incubator-ariatosca/blob/165b9ca8/tests/storage/test_models.py
----------------------------------------------------------------------
diff --git a/tests/storage/test_models.py b/tests/storage/test_models.py
index 23a1549..6b0a5b6 100644
--- a/tests/storage/test_models.py
+++ b/tests/storage/test_models.py
@@ -23,9 +23,11 @@ from aria.storage.models import (
     DeploymentUpdateStep,
     Blueprint,
     Execution,
-    Task)
+    Task, ProviderContext, Plugin, Deployment, Node, NodeInstance, Relationship, RelationshipInstance,
+    DeploymentUpdate, DeploymentModification)
 
 
+from tests import mock
 from tests.storage import get_sqlite_api_kwargs
 
 
@@ -35,147 +37,713 @@ def empty_storage():
                                      api_kwargs=get_sqlite_api_kwargs())
 
 
-@pytest.mark.parametrize(
-    'is_valid, plan, description, created_at, updated_at, main_file_name',
-    [
-        (False, None, 'description', datetime.utcnow(), datetime.utcnow(), '/path'),
-        (False, {}, {}, datetime.utcnow(), datetime.utcnow(), '/path'),
-        (False, {}, 'description', 'error', datetime.utcnow(), '/path'),
-        (False, {}, 'description', datetime.utcnow(), 'error', '/path'),
-        (False, {}, 'description', datetime.utcnow(), datetime.utcnow(), {}),
-        (True, {}, 'description', datetime.utcnow(), datetime.utcnow(), '/path'),
-    ]
-)
-def test_blueprint_model(empty_storage, is_valid, plan, description,
-                         created_at, updated_at, main_file_name):
-    if not is_valid:
+@pytest.fixture
+def blueprint_storage():
+    storage = empty_storage()
+    blueprint = mock.models.get_blueprint()
+    storage.blueprint.put(blueprint)
+    return storage
+
+
+@pytest.fixture
+def deployment_storage():
+    storage = blueprint_storage()
+    deployment = mock.models.get_deployment(storage.blueprint.list()[0])
+    storage.deployment.put(deployment)
+    return storage
+
+
+@pytest.fixture
+def deployment_update_storage():
+    storage = deployment_storage()
+    deployment_update = DeploymentUpdate(
+        deployment_id=storage.deployment.list()[0].id,
+        created_at=now,
+        deployment_plan={},
+    )
+    storage.deployment_update.put(deployment_update)
+    return storage
+
+
+@pytest.fixture
+def node_storage():
+    storage = deployment_storage()
+    node = mock.models.get_dependency_node(storage.deployment.list()[0])
+    storage.node.put(node)
+    return storage
+
+
+@pytest.fixture
+def nodes_storage():
+    storage = deployment_storage()
+    dependent_node = mock.models.get_dependent_node(storage.deployment.list()[0])
+    dependency_node = mock.models.get_dependency_node(storage.deployment.list()[0])
+    storage.node.put(dependent_node)
+    storage.node.put(dependency_node)
+    return storage
+
+
+@pytest.fixture
+def node_instances_storage():
+    storage = nodes_storage()
+    dependent_node = storage.node.get_by_name(mock.models.DEPENDENT_NODE_NAME)
+    dependency_node = storage.node.get_by_name(mock.models.DEPENDENCY_NODE_NAME)
+    dependency_node_instance = mock.models.get_dependency_node_instance(dependency_node)
+    dependent_node_instance = mock.models.get_dependent_node_instance(dependent_node)
+    storage.node_instance.put(dependency_node_instance)
+    storage.node_instance.put(dependent_node_instance)
+    return storage
+
+
+@pytest.fixture
+def execution_storage():
+    storage = deployment_storage()
+    execution = mock.models.get_execution(storage.deployment.list()[0])
+    storage.execution.put(execution)
+    return storage
+
+m_cls = type('MockClass')
+now = datetime.utcnow()
+
+
+def _test_model(is_valid, storage, model_name, model_cls, model_kwargs):
+    if is_valid:
+        getattr(storage, model_name).put(model_cls(**model_kwargs))
+    else:
         with pytest.raises(exceptions.StorageError):
+            getattr(storage, model_name).put(model_cls(**model_kwargs))
+
+
+class TestBlueprint(object):
+
+    @pytest.mark.parametrize(
+        'is_valid, plan, description, created_at, updated_at, main_file_name',
+        [
+            (False, None, 'description', now, now, '/path'),
+            (False, {}, {}, now, now, '/path'),
+            (False, {}, 'description', 'error', now, '/path'),
+            (False, {}, 'description', now, 'error', '/path'),
+            (False, {}, 'description', now, now, {}),
+            (True, {}, 'description', now, now, '/path'),
+        ]
+    )
+    def test_blueprint_model_creation(self, empty_storage, is_valid, plan, description, created_at,
+                                      updated_at, main_file_name):
+        if not is_valid:
+            with pytest.raises(exceptions.StorageError):
+                empty_storage.blueprint.put(Blueprint(plan=plan, description=description,
+                                                      created_at=created_at, updated_at=updated_at,
+                                                      main_file_name=main_file_name))
+        else:
             empty_storage.blueprint.put(Blueprint(plan=plan, description=description,
                                                   created_at=created_at, updated_at=updated_at,
                                                   main_file_name=main_file_name))
-    else:
-        empty_storage.blueprint.put(Blueprint(plan=plan, description=description,
-                                              created_at=created_at, updated_at=updated_at,
-                                              main_file_name=main_file_name))
-
-
-def test_deployment_update_step_model():
-    add_node = DeploymentUpdateStep(
-        id='add_step',
-        action='add',
-        entity_type='node',
-        entity_id='node_id')
-
-    modify_node = DeploymentUpdateStep(
-        id='modify_step',
-        action='modify',
-        entity_type='node',
-        entity_id='node_id')
-
-    remove_node = DeploymentUpdateStep(
-        id='remove_step',
-        action='remove',
-        entity_type='node',
-        entity_id='node_id')
-
-    for step in (add_node, modify_node, remove_node):
-        assert hash((step.id, step.entity_id)) == hash(step)
-
-    assert remove_node < modify_node < add_node
-    assert not remove_node > modify_node > add_node
-
-    add_rel = DeploymentUpdateStep(
-        id='add_step',
-        action='add',
-        entity_type='relationship',
-        entity_id='relationship_id')
-
-    remove_rel = DeploymentUpdateStep(
-        id='remove_step',
-        action='remove',
-        entity_type='relationship',
-        entity_id='relationship_id')
-
-    assert remove_rel < remove_node < add_node < add_rel
-    assert not add_node < None
-
-
-def test_execution_status_transition():
-    def create_execution(status):
-        execution = Execution(
-            id='e_id',
-            workflow_name='w_name',
-            status=status,
-            parameters={},
-            created_at=datetime.utcnow(),
-        )
-        return execution
-
-    valid_transitions = {
-        Execution.PENDING: [Execution.STARTED,
-                            Execution.CANCELLED,
-                            Execution.PENDING],
-        Execution.STARTED: [Execution.FAILED,
-                            Execution.TERMINATED,
-                            Execution.CANCELLED,
-                            Execution.CANCELLING,
-                            Execution.STARTED],
-        Execution.CANCELLING: [Execution.FAILED,
-                               Execution.TERMINATED,
-                               Execution.CANCELLED,
-                               Execution.CANCELLING],
-        Execution.FAILED: [Execution.FAILED],
-        Execution.TERMINATED: [Execution.TERMINATED],
-        Execution.CANCELLED: [Execution.CANCELLED]
-    }
-
-    invalid_transitions = {
-        Execution.PENDING: [Execution.FAILED,
-                            Execution.TERMINATED,
-                            Execution.CANCELLING],
-        Execution.STARTED: [Execution.PENDING],
-        Execution.CANCELLING: [Execution.PENDING,
-                               Execution.STARTED],
-        Execution.FAILED: [Execution.PENDING,
-                           Execution.STARTED,
-                           Execution.TERMINATED,
-                           Execution.CANCELLED,
-                           Execution.CANCELLING],
-        Execution.TERMINATED: [Execution.PENDING,
+
+
+class TestDeployment(object):
+
+    @pytest.mark.parametrize(
+        'is_valid, name, created_at, description, inputs, groups, permalink, policy_triggers, policy_types, outputs, scaling_groups, updated_at, workflows',
+        [
+            (False, m_cls, now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (False, 'name', m_cls, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (False, 'name', now, m_cls, {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (False, 'name', now, 'desc', m_cls, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (False, 'name', now, 'desc', {}, m_cls, 'perlnk', {}, {}, {}, {}, now, {}),
+            (False, 'name', now, 'desc', {}, {}, m_cls, {}, {}, {}, {}, now, {}),
+            (False, 'name', now, 'desc', {}, {}, 'perlnk', m_cls, {}, {}, {}, now, {}),
+            (False, 'name', now, 'desc', {}, {}, 'perlnk', {}, m_cls, {}, {}, now, {}),
+            (False, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, m_cls, {}, now, {}),
+            (False, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, m_cls, now, {}),
+            (False, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, m_cls, {}),
+            (False, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, m_cls),
+
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (True, None, now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (True, 'name', now, None, {}, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (True, 'name', now, 'desc', None, {}, 'perlnk', {}, {}, {}, {}, now, {}),
+            (True, 'name', now, 'desc', {}, None, 'perlnk', {}, {}, {}, {}, now, {}),
+            (True, 'name', now, 'desc', {}, {}, None, {}, {}, {}, {}, now, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', None, {}, {}, {}, now, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, None, {}, {}, now, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, None, {}, now, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, None, now, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, None, {}),
+            (True, 'name', now, 'desc', {}, {}, 'perlnk', {}, {}, {}, {}, now, None),
+        ]
+    )
+    def test_deployment_model_creation(self, deployment_storage, is_valid, name, created_at,
+                                       description, inputs, groups, permalink, policy_triggers,
+                                       policy_types, outputs, scaling_groups, updated_at,
+                                       workflows):
+        _test_model(is_valid=is_valid,
+                    storage=deployment_storage,
+                    model_name='deployment',
+                    model_cls=Deployment,
+                    model_kwargs=dict(
+                        name=name,
+                        blueprint_id=deployment_storage.blueprint.list()[0].id,
+                        created_at=created_at,
+                        description=description,
+                        inputs=inputs,
+                        groups=groups,
+                        permalink=permalink,
+                        policy_triggers=policy_triggers,
+                        policy_types=policy_types,
+                        outputs=outputs,
+                        scaling_groups=scaling_groups,
+                        updated_at=updated_at,
+                        workflows=workflows
+                    ))
+
+
+class TestExecution(object):
+
+    @pytest.mark.parametrize(
+        'is_valid, created_at, started_at, ended_at, error, is_system_workflow, parameters, status, workflow_name',
+        [
+            (False, m_cls, now, now, 'error', False, {}, Execution.STARTED, 'wf_name'),
+            (False, now, m_cls, now, 'error', False, {}, Execution.STARTED, 'wf_name'),
+            (False, now, now, m_cls, 'error', False, {}, Execution.STARTED, 'wf_name'),
+            (False, now, now, now, m_cls, False, {}, Execution.STARTED, 'wf_name'),
+            (False, now, now, now, 'error', False, m_cls, Execution.STARTED, 'wf_name'),
+            (False, now, now, now, 'error', False, {}, m_cls, 'wf_name'),
+            (False, now, now, now, 'error', False, {}, Execution.STARTED, m_cls),
+
+            (True, now, now, now, 'error', False, {}, Execution.STARTED, 'wf_name'),
+            (True, now, None, now, 'error', False, {}, Execution.STARTED, 'wf_name'),
+            (True, now, now, None, 'error', False, {}, Execution.STARTED, 'wf_name'),
+            (True, now, now, now, None, False, {}, Execution.STARTED, 'wf_name'),
+            (True, now, now, now, 'error', False, None, Execution.STARTED, 'wf_name'),
+        ]
+    )
+    def test_execution_model_creation(self, deployment_storage, is_valid, created_at, started_at,
+                                      ended_at, error, is_system_workflow, parameters, status,
+                                      workflow_name):
+        _test_model(is_valid=is_valid,
+                    storage=deployment_storage,
+                    model_name='execution',
+                    model_cls=Execution,
+                    model_kwargs=dict(
+                        deployment_id=deployment_storage.deployment.list()[0].id,
+                        blueprint_id=deployment_storage.blueprint.list()[0].id,
+                        created_at=created_at,
+                        started_at=started_at,
+                        ended_at=ended_at,
+                        error=error,
+                        is_system_workflow=is_system_workflow,
+                        parameters=parameters,
+                        status=status,
+                        workflow_name=workflow_name,
+                    ))
+
+    def test_execution_status_transition(self):
+        def create_execution(status):
+            execution = Execution(
+                id='e_id',
+                workflow_name='w_name',
+                status=status,
+                parameters={},
+                created_at=now,
+            )
+            return execution
+
+        valid_transitions = {
+            Execution.PENDING: [Execution.STARTED,
+                                Execution.CANCELLED,
+                                Execution.PENDING],
+            Execution.STARTED: [Execution.FAILED,
+                                Execution.TERMINATED,
+                                Execution.CANCELLED,
+                                Execution.CANCELLING,
+                                Execution.STARTED],
+            Execution.CANCELLING: [Execution.FAILED,
+                                   Execution.TERMINATED,
+                                   Execution.CANCELLED,
+                                   Execution.CANCELLING],
+            Execution.FAILED: [Execution.FAILED],
+            Execution.TERMINATED: [Execution.TERMINATED],
+            Execution.CANCELLED: [Execution.CANCELLED]
+        }
+
+        invalid_transitions = {
+            Execution.PENDING: [Execution.FAILED,
+                                Execution.TERMINATED,
+                                Execution.CANCELLING],
+            Execution.STARTED: [Execution.PENDING],
+            Execution.CANCELLING: [Execution.PENDING,
+                                   Execution.STARTED],
+            Execution.FAILED: [Execution.PENDING,
                                Execution.STARTED,
-                               Execution.FAILED,
+                               Execution.TERMINATED,
                                Execution.CANCELLED,
                                Execution.CANCELLING],
-        Execution.CANCELLED: [Execution.PENDING,
-                              Execution.STARTED,
-                              Execution.FAILED,
-                              Execution.TERMINATED,
-                              Execution.CANCELLING],
-    }
-
-    for current_status, valid_transitioned_statues in valid_transitions.items():
-        for transitioned_status in valid_transitioned_statues:
-            execution = create_execution(current_status)
-            execution.status = transitioned_status
-
-    for current_status, invalid_transitioned_statues in invalid_transitions.items():
-        for transitioned_status in invalid_transitioned_statues:
-            execution = create_execution(current_status)
-            with pytest.raises(ValueError):
+            Execution.TERMINATED: [Execution.PENDING,
+                                   Execution.STARTED,
+                                   Execution.FAILED,
+                                   Execution.CANCELLED,
+                                   Execution.CANCELLING],
+            Execution.CANCELLED: [Execution.PENDING,
+                                  Execution.STARTED,
+                                  Execution.FAILED,
+                                  Execution.TERMINATED,
+                                  Execution.CANCELLING],
+        }
+
+        for current_status, valid_transitioned_statues in valid_transitions.items():
+            for transitioned_status in valid_transitioned_statues:
+                execution = create_execution(current_status)
                 execution.status = transitioned_status
 
+        for current_status, invalid_transitioned_statues in invalid_transitions.items():
+            for transitioned_status in invalid_transitioned_statues:
+                execution = create_execution(current_status)
+                with pytest.raises(ValueError):
+                    execution.status = transitioned_status
+
+
+class TestDeploymentUpdate(object):
+    @pytest.mark.parametrize(
+        'is_valid, created_at, deployment_plan, deployment_update_node_instances, '
+        'deployment_update_deployment, deployment_update_nodes, modified_entity_ids, state',
+        [
+            (False, m_cls, {}, {}, {}, {}, {}, 'state'),
+            (False, now, m_cls, {}, {}, {}, {}, 'state'),
+            (False, now, {}, m_cls, {}, {}, {}, 'state'),
+            (False, now, {}, {}, m_cls, {}, {}, 'state'),
+            (False, now, {}, {}, {}, m_cls, {}, 'state'),
+            (False, now, {}, {}, {}, {}, m_cls, 'state'),
+            (False, now, {}, {}, {}, {}, {}, m_cls),
+
+            (True, now, {}, {}, {}, {}, {}, 'state'),
+            (True, now, {}, None, {}, {}, {}, 'state'),
+            (True, now, {}, {}, None, {}, {}, 'state'),
+            (True, now, {}, {}, {}, None, {}, 'state'),
+            (True, now, {}, {}, {}, {}, None, 'state'),
+            (True, now, {}, {}, {}, {}, {}, None),
+        ]
+    )
+    def test_deployment_update_model_creation(self, deployment_storage, is_valid, created_at,
+                                              deployment_plan, deployment_update_node_instances,
+                                              deployment_update_deployment, deployment_update_nodes,
+                                              modified_entity_ids, state):
+        _test_model(is_valid=is_valid,
+                    storage=deployment_storage,
+                    model_name='deployment_update',
+                    model_cls=DeploymentUpdate,
+                    model_kwargs=dict(
+                        deployment_id=deployment_storage.deployment.list()[0].id,
+                        created_at=created_at,
+                        deployment_plan=deployment_plan,
+                        deployment_update_node_instances=deployment_update_node_instances,
+                        deployment_update_deployment=deployment_update_deployment,
+                        deployment_update_nodes=deployment_update_nodes,
+                        modified_entity_ids=modified_entity_ids,
+                        state=state,
+                    ))
+
+
+class TestDeploymentUpdateStep(object):
+
+    @pytest.mark.parametrize(
+        'is_valid, action, entity_id, entity_type',
+        [
+            (False, m_cls, 'id', DeploymentUpdateStep.ENTITY_TYPES.NODE),
+            (False, DeploymentUpdateStep.ACTION_TYPES.ADD, m_cls, DeploymentUpdateStep.ENTITY_TYPES.NODE),
+            (False, DeploymentUpdateStep.ACTION_TYPES.ADD, 'id', m_cls),
+
+            (True, DeploymentUpdateStep.ACTION_TYPES.ADD, 'id', DeploymentUpdateStep.ENTITY_TYPES.NODE)
+        ]
+    )
+    def test_deployment_update_step_model_creation(self, deployment_update_storage, is_valid, action,
+                                                   entity_id, entity_type):
+        _test_model(is_valid=is_valid,
+                    storage=deployment_update_storage,
+                    model_name='deployment_update_step',
+                    model_cls=DeploymentUpdateStep,
+                    model_kwargs=dict(
+                        deployment_update_id=deployment_update_storage.deployment_update.list()[0].id,
+                        action=action,
+                        entity_id=entity_id,
+                        entity_type=entity_type
+                    ))
+
+    def test_deployment_update_step_order(self):
+        add_node = DeploymentUpdateStep(
+            id='add_step',
+            action='add',
+            entity_type='node',
+            entity_id='node_id')
+
+        modify_node = DeploymentUpdateStep(
+            id='modify_step',
+            action='modify',
+            entity_type='node',
+            entity_id='node_id')
+
+        remove_node = DeploymentUpdateStep(
+            id='remove_step',
+            action='remove',
+            entity_type='node',
+            entity_id='node_id')
+
+        for step in (add_node, modify_node, remove_node):
+            assert hash((step.id, step.entity_id)) == hash(step)
+
+        assert remove_node < modify_node < add_node
+        assert not remove_node > modify_node > add_node
+
+        add_rel = DeploymentUpdateStep(
+            id='add_step',
+            action='add',
+            entity_type='relationship',
+            entity_id='relationship_id')
+
+        remove_rel = DeploymentUpdateStep(
+            id='remove_step',
+            action='remove',
+            entity_type='relationship',
+            entity_id='relationship_id')
+
+        assert remove_rel < remove_node < add_node < add_rel
+        assert not add_node < None
+
+
+class TestDeploymentModification(object):
+    @pytest.mark.parametrize(
+        'is_valid, context, created_at, ended_at, modified_nodes, node_instances, status',
+        [
+            (False, m_cls, now, now, {}, {}, DeploymentModification.STARTED),
+            (False, {}, m_cls, now, {}, {}, DeploymentModification.STARTED),
+            (False, {}, now, m_cls, {}, {}, DeploymentModification.STARTED),
+            (False, {}, now, now, m_cls, {}, DeploymentModification.STARTED),
+            (False, {}, now, now, {}, m_cls, DeploymentModification.STARTED),
+            (False, {}, now, now, {}, {}, m_cls),
+
+            (True, {}, now, now, {}, {}, DeploymentModification.STARTED),
+            (True, {}, now, None, {}, {}, DeploymentModification.STARTED),
+            (True, {}, now, now, None, {}, DeploymentModification.STARTED),
+            (True, {}, now, now, {}, None, DeploymentModification.STARTED),
+        ]
+    )
+    def test_deployment_modification_model_creation(self, deployment_storage, is_valid, context,
+                                                    created_at, ended_at, modified_nodes,
+                                                    node_instances, status):
+        _test_model(is_valid=is_valid,
+                    storage=deployment_storage,
+                    model_name='deployment_modification',
+                    model_cls=DeploymentModification,
+                    model_kwargs=dict(
+                        deployment_id=deployment_storage.deployment.list()[0].id,
+                        context=context,
+                        created_at=created_at,
+                        ended_at=ended_at,
+                        modified_nodes=modified_nodes,
+                        node_instances=node_instances,
+                        status=status,
+                    ))
+
+
+class TestNode(object):
+    @pytest.mark.parametrize(
+        'is_valid, name, deploy_number_of_instances, max_number_of_instances, '
+        'min_number_of_instances, number_of_instances, planned_number_of_instances, plugins, '
+        'plugins_to_install, properties, operations, type, type_hierarchy',
+        [
+           (False, m_cls, 1, 1, 1, 1, 1, {}, {}, {}, {}, 'type', []),
+           (False, 'name', m_cls, 1, 1, 1, 1, {}, {}, {}, {}, 'type', []),
+           (False, 'name', 1, m_cls, 1, 1, 1, {}, {}, {}, {}, 'type', []),
+           (False, 'name', 1, 1, m_cls, 1, 1, {}, {}, {}, {}, 'type', []),
+           (False, 'name', 1, 1, 1, m_cls, 1, {}, {}, {}, {}, 'type', []),
+           (False, 'name', 1, 1, 1, 1, m_cls, {}, {}, {}, {}, 'type', []),
+           (False, 'name', 1, 1, 1, 1, 1, m_cls, {}, {}, {}, 'type', []),
+           (False, 'name', 1, 1, 1, 1, 1, {}, m_cls, {}, {}, 'type', []),
+           (False, 'name', 1, 1, 1, 1, 1, {}, {}, m_cls, {}, 'type', []),
+           (False, 'name', 1, 1, 1, 1, 1, {}, {}, {}, m_cls, 'type', []),
+           (False, 'name', 1, 1, 1, 1, 1, {}, {}, {}, {}, m_cls, []),
+           (False, 'name', 1, 1, 1, 1, 1, {}, {}, {}, {}, 'type', m_cls),
+
+           (True, 'name', 1, 1, 1, 1, 1, {}, {}, {}, {}, 'type', []),
+           (True, 'name', 1, 1, 1, 1, 1, None, {}, {}, {}, 'type', []),
+           (True, 'name', 1, 1, 1, 1, 1, {}, None, {}, {}, 'type', []),
+           (True, 'name', 1, 1, 1, 1, 1, {}, {}, None, {}, 'type', []),
+           (True, 'name', 1, 1, 1, 1, 1, {}, {}, {}, None, 'type', []),
+           (True, 'name', 1, 1, 1, 1, 1, {}, {}, {}, {}, 'type', []),
+           (True, 'name', 1, 1, 1, 1, 1, {}, {}, {}, {}, 'type', None),
+       ]
+    )
+    def test_node_model_creation(self, deployment_storage, is_valid, name,
+                                 deploy_number_of_instances, max_number_of_instances,
+                                 min_number_of_instances, number_of_instances,
+                                 planned_number_of_instances, plugins, plugins_to_install,
+                                 properties, operations, type, type_hierarchy):
+        _test_model(is_valid=is_valid,
+                    storage=deployment_storage,
+                    model_name='node',
+                    model_cls=Node,
+                    model_kwargs=dict(
+                        name=name,
+                        deploy_number_of_instances=deploy_number_of_instances,
+                        max_number_of_instances=max_number_of_instances,
+                        min_number_of_instances=min_number_of_instances,
+                        number_of_instances=number_of_instances,
+                        planned_number_of_instances=planned_number_of_instances,
+                        plugins=plugins,
+                        plugins_to_install=plugins_to_install,
+                        properties=properties,
+                        operations=operations,
+                        type=type,
+                        type_hierarchy=type_hierarchy,
+                        deployment_id=deployment_storage.deployment.list()[0].id
+                    ))
+
+
+class TestRelationship(object):
+    @pytest.mark.parametrize(
+        'is_valid, source_interfaces, source_operations, target_interfaces, target_operations, type, type_hierarchy, properties',
+        [
+            (False, m_cls, {}, {}, {}, 'type', [], {}),
+            (False, {}, m_cls, {}, {}, 'type', [], {}),
+            (False, {}, {}, m_cls, {}, 'type', [], {}),
+            (False, {}, {}, {}, m_cls, 'type', [], {}),
+            (False, {}, {}, {}, {}, m_cls, [], {}),
+            (False, {}, {}, {}, {}, 'type', m_cls, {}),
+            (False, {}, {}, {}, {}, 'type', [], m_cls),
+
+            (True, {}, {}, {}, {}, 'type', [], {}),
+            (True, None, {}, {}, {}, 'type', [], {}),
+            (True, {}, {}, None, {}, 'type', [], {}),
+            (True, {}, {}, {}, {}, 'type', None, {}),
+            (True, {}, {}, {}, {}, 'type', [], None),
+        ]
+        )
+    def test_relationship_model_ceration(self, nodes_storage, is_valid, source_interfaces,
+                                         source_operations, target_interfaces, target_operations,
+                                         type, type_hierarchy, properties):
+        _test_model(is_valid=is_valid,
+                    storage=nodes_storage,
+                    model_name='relationship',
+                    model_cls=Relationship,
+                    model_kwargs=dict(
+                        source_node_id=nodes_storage.node.list()[1].id,
+                        target_node_id=nodes_storage.node.list()[0].id,
+                        source_interfaces=source_interfaces,
+                        source_operations=source_operations,
+                        target_interfaces=target_interfaces,
+                        target_operations=target_operations,
+                        type=type,
+                        type_hierarchy=type_hierarchy,
+                        properties=properties,
+                    ))
+
+
+class TestNodeInstance(object):
+    @pytest.mark.parametrize(
+        'is_valid, name, runtime_properties, scaling_groups, state, version',
+        [
+            (False, m_cls, {}, {}, 'state', 1),
+            (False, 'name', m_cls, {}, 'state', 1),
+            (False, 'name', {}, m_cls, 'state', 1),
+            (False, 'name', {}, {}, m_cls, 1),
+            (False, m_cls, {}, {}, 'state', m_cls),
+
+            (True, 'name', {}, {}, 'state', 1),
+            (True, None, {}, {}, 'state', 1),
+            (True, 'name', None, {}, 'state', 1),
+            (True, 'name', {}, None, 'state', 1),
+            (True, 'name', {}, {}, 'state', None),
+        ]
+    )
+    def test_node_instance_model_creation(self, node_storage, is_valid, name, runtime_properties,
+                                          scaling_groups, state, version):
+        _test_model(is_valid=is_valid,
+                    storage=node_storage,
+                    model_name='node_instance',
+                    model_cls=NodeInstance,
+                    model_kwargs=dict(
+                        node_id=node_storage.node.list()[0].id,
+                        deployment_id=node_storage.deployment.list()[0].id,
+                        name=name,
+                        runtime_properties=runtime_properties,
+                        scaling_groups=scaling_groups,
+                        state=state,
+                        version=version,
+
+                    ))
+
+
+class TestRelationshipInstance(object):
+    def test_relatiship_instance_model_creation(self, node_instances_storage):
+        relationship = mock.models.get_relationship(
+            source=node_instances_storage.node.get_by_name(mock.models.DEPENDENT_NODE_NAME),
+            target=node_instances_storage.node.get_by_name(mock.models.DEPENDENCY_NODE_NAME)
+        )
+        node_instances_storage.relationship.put(relationship)
+        _test_model(is_valid=True,
+                    storage=node_instances_storage,
+                    model_name='relationship_instance',
+                    model_cls=RelationshipInstance,
+                    model_kwargs=dict(
+                        relationship_id=relationship.id,
+                        source_node_instance_id=node_instances_storage.node_instance.get_by_name(mock.models.DEPENDENT_NODE_INSTANCE_NAME).id,
+                        target_node_instance_id=node_instances_storage.node_instance.get_by_name(mock.models.DEPENDENCY_NODE_INSTANCE_NAME).id
+                    ))
+
+
+class TestProviderContext(object):
+    @pytest.mark.parametrize(
+        'is_valid, name, context',
+        [
+            (False, None, {}),
+            (False, 'name', None),
+            (True, 'name', {}),
+        ]
+    )
+    def test_provider_context_model_creation(self, empty_storage, is_valid, name, context):
+        _test_model(is_valid=is_valid,
+                    storage=empty_storage,
+                    model_name='provider_context',
+                    model_cls=ProviderContext,
+                    model_kwargs=dict(name=name, context=context)
+                    )
+
+
+class TestPlugin(object):
+    @pytest.mark.parametrize(
+        'is_valid, archive_name, distribution, distribution_release, '
+        'distribution_version, excluded_wheels, package_name, package_source, '
+        'package_version, supported_platform, supported_py_versions, uploaded_at, wheels',
+        [
+            (False, m_cls, 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+            (False, 'arc_name', m_cls, 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+            (False, 'arc_name', 'dis_name', m_cls, 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {},  now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', m_cls, {}, 'pak_name', 'pak_src', 'pak_ver', {}, {},  now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', m_cls, 'pak_name', 'pak_src', 'pak_ver', {}, {},  now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, m_cls, 'pak_src', 'pak_ver', {}, {},  now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', m_cls, 'pak_ver', {}, {},  now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', m_cls, {}, {},  now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', m_cls, {}, now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, m_cls, now, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {},  m_cls, {}),
+            (False, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {},  now, m_cls),
+
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {},  now, {}),
+            (True, 'arc_name', None, 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+            (True, 'arc_name', 'dis_name', None, 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', None, {}, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', None, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', None, 'pak_ver', {}, {}, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', None, {}, {}, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', None, {}, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, None, now, {}),
+            (True, 'arc_name', 'dis_name', 'dis_rel', 'dis_ver', {}, 'pak_name', 'pak_src', 'pak_ver', {}, {}, now, {}),
+        ]
+    )
+    def test_plugin_model_creation(self, empty_storage, is_valid, archive_name, distribution,
+                                   distribution_release, distribution_version, excluded_wheels,
+                                   package_name, package_source, package_version,
+                                   supported_platform, supported_py_versions, uploaded_at, wheels):
+                _test_model(is_valid=is_valid,
+                            storage=empty_storage,
+                            model_name='plugin',
+                            model_cls=Plugin,
+                            model_kwargs=dict(
+                                archive_name=archive_name,
+                                distribution=distribution,
+                                distribution_release=distribution_release,
+                                distribution_version=distribution_version,
+                                excluded_wheels=excluded_wheels,
+                                package_name=package_name,
+                                package_source=package_source,
+                                package_version=package_version,
+                                supported_platform=supported_platform,
+                                supported_py_versions=supported_py_versions,
+                                uploaded_at=uploaded_at,
+                                wheels=wheels,
+                            ))
+
+
+class TestTask(object):
+
+    @pytest.mark.parametrize(
+        'is_valid, status, due_at, started_at, ended_at, max_attempts, retry_count, retry_interval, ignore_failure, name, operation_mapping, inputs',
+        [
+            (False, m_cls, now, now, now, 1, 1, 1, True, 'name', 'map', {}),
+            (False, Task.STARTED, m_cls, now, now, 1, 1, 1, True, 'name', 'map', {}),
+            (False, Task.STARTED, now, m_cls, now, 1, 1, 1, True, 'name', 'map', {}),
+            (False, Task.STARTED, now, now, m_cls, 1, 1, 1, True, 'name', 'map', {}),
+            (False, Task.STARTED, now, now, now, m_cls, 1, 1, True, 'name', 'map', {}),
+            (False, Task.STARTED, now, now, now, 1, m_cls, 1, True, 'name', 'map', {}),
+            (False, Task.STARTED, now, now, now, 1, 1, m_cls, True, 'name', 'map', {}),
+            (False, Task.STARTED, now, now, now, 1, 1, 1, True, m_cls, 'map', {}),
+            (False, Task.STARTED, now, now, now, 1, 1, 1, True, 'name', m_cls, {}),
+            (False, Task.STARTED, now, now, now, 1, 1, 1, True, 'name', 'map', m_cls),
+
+            (True, Task.STARTED, now, now, now, 1, 1, 1, True, 'name', 'map', {}),
+            (True, Task.STARTED, None, now, now, 1, 1, 1, True, 'name', 'map', {}),
+            (True, Task.STARTED, now, None, now, 1, 1, 1, True, 'name', 'map', {}),
+            (True, Task.STARTED, now, now, None, 1, 1, 1, True, 'name', 'map', {}),
+            (True, Task.STARTED, now, now, now, 1, None, 1, True, 'name', 'map', {}),
+            (True, Task.STARTED, now, now, now, 1, 1, None, True, 'name', 'map', {}),
+            (True, Task.STARTED, now, now, now, 1, 1, 1, None, 'name', 'map', {}),
+            (True, Task.STARTED, now, now, now, 1, 1, 1, True, None, 'map', {}),
+            (True, Task.STARTED, now, now, now, 1, 1, 1, True, 'name', None, {}),
+            (True, Task.STARTED, now, now, now, 1, 1, 1, True, 'name', 'map', None),
+        ]
+    )
+    def test_task_model_creation(self, execution_storage, is_valid, status, due_at, started_at,
+                                 ended_at, max_attempts, retry_count, retry_interval,
+                                 ignore_failure, name, operation_mapping, inputs):
+        _test_model(is_valid=is_valid,
+                    storage=execution_storage,
+                    model_name='task',
+                    model_cls=Task,
+                    model_kwargs=dict(
+                        status=status,
+                        execution_id=execution_storage.execution.list()[0].id,
+                        due_at=due_at,
+                        started_at=started_at,
+                        ended_at=ended_at,
+                        max_attempts=max_attempts,
+                        retry_count=retry_count,
+                        retry_interval=retry_interval,
+                        ignore_failure=ignore_failure,
+                        name=name,
+                        operation_mapping=operation_mapping,
+                        inputs=inputs,
+                    ))
+    def test_task_max_attempts_validation(self):
+        def create_task(max_attempts):
+            Task(execution_id='eid',
+                 name='name',
+                 operation_mapping='',
+                 inputs={},
+                 max_attempts=max_attempts)
+        create_task(max_attempts=1)
+        create_task(max_attempts=2)
+        create_task(max_attempts=Task.INFINITE_RETRIES)
+        with pytest.raises(ValueError):
+            create_task(max_attempts=0)
+        with pytest.raises(ValueError):
+            create_task(max_attempts=-2)
+
+
+def test_inner_dict_update(empty_storage):
+    inner_dict = {'inner_value': 1}
+    pc = ProviderContext(name='name', context={
+        'inner_dict': {'inner_value': inner_dict},
+        'value': 0
+    })
+    empty_storage.provider_context.put(pc)
+
+    storage_pc = empty_storage.provider_context.get(pc.id)
+    assert storage_pc == pc
+
+    storage_pc.context['inner_dict']['inner_value'] = 2
+    storage_pc.context['value'] = -1
+    empty_storage.provider_context.update(storage_pc)
+    storage_pc = empty_storage.provider_context.get(pc.id)
 
-def test_task_max_attempts_validation():
-    def create_task(max_attempts):
-        Task(execution_id='eid',
-             name='name',
-             operation_mapping='',
-             inputs={},
-             max_attempts=max_attempts)
-    create_task(max_attempts=1)
-    create_task(max_attempts=2)
-    create_task(max_attempts=Task.INFINITE_RETRIES)
-    with pytest.raises(ValueError):
-        create_task(max_attempts=0)
-    with pytest.raises(ValueError):
-        create_task(max_attempts=-2)
+    assert storage_pc.context['inner_dict']['inner_value'] == 2
+    assert storage_pc.context['value'] == -1