add consistency with periodic process
implements: blueprint entity-graph-consistency-validator Change-Id: Ifaff9117d0164b3b96253092bace3a56b6cdd778
This commit is contained in:
parent
841f0f1c68
commit
18135a4dba
@ -1,4 +1,5 @@
|
||||
# Copyright 2015 - Alcatel-Lucent
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
@ -19,7 +20,7 @@ from oslo_service import service as os_service
|
||||
|
||||
from vitrage import entity_graph as entity_graph_svc
|
||||
from vitrage.entity_graph import api_handler as api_handler_svc
|
||||
from vitrage.entity_graph import consistency as consistency_svc
|
||||
from vitrage.entity_graph.consistency import service as consistency_svc
|
||||
from vitrage.entity_graph.processor import entity_graph
|
||||
from vitrage import service
|
||||
from vitrage.synchronizer import launcher as synchronizer_launcher
|
||||
@ -50,7 +51,7 @@ def main():
|
||||
synchronizer.launch()
|
||||
|
||||
launcher.launch_service(consistency_svc.VitrageGraphConsistencyService(
|
||||
e_graph))
|
||||
conf, e_graph))
|
||||
|
||||
launcher.wait()
|
||||
|
||||
|
@ -58,6 +58,7 @@ class SynchronizerProperties(object):
|
||||
|
||||
|
||||
class EntityType(object):
|
||||
NODE = 'node'
|
||||
NOVA_INSTANCE = 'nova.instance'
|
||||
NOVA_HOST = 'nova.host'
|
||||
NOVA_ZONE = 'nova.zone'
|
||||
|
@ -19,8 +19,11 @@ from oslo_log import log
|
||||
import oslo_messaging
|
||||
from oslo_service import service as os_service
|
||||
|
||||
from vitrage.common.constants import EdgeLabels
|
||||
from vitrage.common.constants import EdgeProperties as EProps
|
||||
from vitrage.common.constants import EntityCategory
|
||||
from vitrage.common.constants import VertexProperties as VProps
|
||||
from vitrage.graph import Direction
|
||||
|
||||
LOG = log.getLogger(__name__)
|
||||
|
||||
@ -85,6 +88,16 @@ class EntityGraphApis(object):
|
||||
items_list = self.entity_graph.neighbors(
|
||||
vitrage_id,
|
||||
vertex_attr_filter={VProps.CATEGORY: EntityCategory.ALARM})
|
||||
|
||||
# TODO(alexey) this should not be here, but in the transformer
|
||||
for alarm in items_list:
|
||||
related_resource = self.entity_graph.neighbors(
|
||||
v_id=alarm.vertex_id,
|
||||
edge_attr_filter={EProps.RELATIONSHIP_NAME: EdgeLabels.ON},
|
||||
direction=Direction.OUT)
|
||||
alarm["resource_id"] = related_resource.vertex_id
|
||||
alarm["resource_name"] = related_resource[VProps.NAME]
|
||||
|
||||
LOG.info("EntityGraphApis get_alarms result:%s", str(items_list))
|
||||
return json.dumps({'alarms': [v.properties for v in items_list]})
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
# Copyright 2016 - Alcatel-Lucent
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
@ -12,29 +12,16 @@
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
from oslo_log import log
|
||||
from oslo_service import service as os_service
|
||||
from oslo_config import cfg
|
||||
|
||||
|
||||
LOG = log.getLogger(__name__)
|
||||
|
||||
|
||||
class VitrageGraphConsistencyService(os_service.Service):
|
||||
|
||||
def __init__(self, entity_graph):
|
||||
super(VitrageGraphConsistencyService, self).__init__()
|
||||
self.entity_graph = entity_graph
|
||||
|
||||
def start(self):
|
||||
LOG.info("Start VitrageGraphConsistencyService")
|
||||
|
||||
super(VitrageGraphConsistencyService, self).start()
|
||||
|
||||
LOG.info("Finish start VitrageGraphConsistencyService")
|
||||
|
||||
def stop(self):
|
||||
LOG.info("Stop VitrageGraphConsistencyService")
|
||||
|
||||
super(VitrageGraphConsistencyService, self).stop()
|
||||
|
||||
LOG.info("Finish stop VitrageGraphConsistencyService")
|
||||
OPTS = [
|
||||
cfg.IntOpt('consistency_interval',
|
||||
default=600,
|
||||
min=180,
|
||||
help='interval between consistency checks (in seconds)'),
|
||||
cfg.IntOpt('min_time_to_delete',
|
||||
default=60,
|
||||
min=60,
|
||||
help='minimum time until deleting entity (in seconds)'),
|
||||
]
|
||||
|
90
vitrage/entity_graph/consistency/consistency_enforcer.py
Normal file
90
vitrage/entity_graph/consistency/consistency_enforcer.py
Normal file
@ -0,0 +1,90 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
from datetime import timedelta
|
||||
import traceback
|
||||
|
||||
from oslo_log import log
|
||||
|
||||
from vitrage.common.constants import EntityCategory
|
||||
from vitrage.common.constants import EntityType
|
||||
from vitrage.common.constants import VertexProperties as VProps
|
||||
from vitrage.common.datetime_utils import utcnow
|
||||
|
||||
LOG = log.getLogger(__name__)
|
||||
|
||||
|
||||
class ConsistencyEnforcer(object):
|
||||
|
||||
def __init__(self, cfg, entity_graph):
|
||||
self.cfg = cfg
|
||||
self.graph = entity_graph
|
||||
|
||||
def starting_process(self):
|
||||
pass
|
||||
|
||||
def periodic_process(self):
|
||||
try:
|
||||
LOG.debug('Started consistency periodic check')
|
||||
|
||||
# periodic check
|
||||
stale_entities = self._find_stale_entities()
|
||||
old_deleted_entities = self._find_old_deleted_entities()
|
||||
vertices_to_delete = stale_entities.union(old_deleted_entities)
|
||||
|
||||
LOG.debug('Found %s vertices to be deleted by consistency service',
|
||||
len(stale_entities))
|
||||
for item in vertices_to_delete:
|
||||
LOG.debug(item)
|
||||
|
||||
self._delete_vertices(vertices_to_delete)
|
||||
except Exception:
|
||||
LOG.error("Error in deleting vertices from entity_graph: %s",
|
||||
traceback.print_exc())
|
||||
|
||||
def _find_stale_entities(self):
|
||||
query = {
|
||||
'<': {VProps.UPDATE_TIMESTAMP: str(utcnow() - timedelta(
|
||||
seconds=2 * self.cfg.consistency.consistency_interval))},
|
||||
}
|
||||
|
||||
vertices = self.graph.get_vertices(query_dict=query)
|
||||
|
||||
return set(self._filter_vertices_to_be_deleted(vertices))
|
||||
|
||||
def _find_old_deleted_entities(self):
|
||||
query = {
|
||||
'and': [
|
||||
{'==': {VProps.IS_DELETED: True}},
|
||||
{'<': {VProps.UPDATE_TIMESTAMP: str(utcnow() - timedelta(
|
||||
seconds=self.cfg.consistency.min_time_to_delete))}}
|
||||
]
|
||||
}
|
||||
|
||||
vertices = self.graph.get_vertices(query_dict=query)
|
||||
|
||||
return set(self._filter_vertices_to_be_deleted(vertices))
|
||||
|
||||
def _delete_vertices(self, vertices):
|
||||
for vertex in vertices:
|
||||
self.graph.remove_vertex(vertex)
|
||||
|
||||
@staticmethod
|
||||
def _filter_vertices_to_be_deleted(vertices):
|
||||
for vertex in vertices:
|
||||
if vertex.properties[VProps.CATEGORY] == EntityCategory.RESOURCE \
|
||||
and vertex.properties[VProps.TYPE] == EntityType.NODE:
|
||||
vertices.remove(vertex)
|
||||
|
||||
return vertices
|
47
vitrage/entity_graph/consistency/service.py
Normal file
47
vitrage/entity_graph/consistency/service.py
Normal file
@ -0,0 +1,47 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
from oslo_log import log
|
||||
from oslo_service import service as os_service
|
||||
|
||||
from vitrage.entity_graph.consistency.consistency_enforcer \
|
||||
import ConsistencyEnforcer
|
||||
|
||||
LOG = log.getLogger(__name__)
|
||||
|
||||
|
||||
class VitrageGraphConsistencyService(os_service.Service):
|
||||
|
||||
def __init__(self, conf, entity_graph):
|
||||
super(VitrageGraphConsistencyService, self).__init__()
|
||||
self.cfg = conf
|
||||
self.entity_graph = entity_graph
|
||||
|
||||
def start(self):
|
||||
LOG.info("Start VitrageGraphConsistencyService")
|
||||
|
||||
super(VitrageGraphConsistencyService, self).start()
|
||||
|
||||
consistency_enf = ConsistencyEnforcer(self.cfg, self.entity_graph)
|
||||
self.tg.add_timer(self.cfg.consistency.consistency_interval,
|
||||
consistency_enf.periodic_process)
|
||||
|
||||
LOG.info("Finish start VitrageGraphConsistencyService")
|
||||
|
||||
def stop(self):
|
||||
LOG.info("Stop VitrageGraphConsistencyService")
|
||||
|
||||
super(VitrageGraphConsistencyService, self).stop()
|
||||
|
||||
LOG.info("Finish stop VitrageGraphConsistencyService")
|
@ -421,7 +421,7 @@ class Graph(object):
|
||||
pass
|
||||
|
||||
@abc.abstractmethod
|
||||
def get_vertices(self, vertex_attr_filter=None):
|
||||
def get_vertices(self, vertex_attr_filter=None, query_dict=None):
|
||||
"""Get vertices list with an optional match filter
|
||||
|
||||
To filter the vertices, specify property values for
|
||||
@ -449,6 +449,8 @@ class Graph(object):
|
||||
|
||||
:param vertex_attr_filter: expected keys and values
|
||||
:type vertex_attr_filter dict
|
||||
:param query_dict: expected query
|
||||
:type query_dict dict
|
||||
:return: A list of vertices that match the requested query
|
||||
:rtype: list of Vertex
|
||||
"""
|
||||
|
@ -16,7 +16,9 @@ import copy
|
||||
import json
|
||||
|
||||
import networkx as nx
|
||||
|
||||
from networkx.readwrite import json_graph
|
||||
|
||||
from oslo_log import log as logging
|
||||
|
||||
from driver import Direction
|
||||
@ -25,6 +27,7 @@ from driver import Graph
|
||||
from driver import Vertex # noqa
|
||||
from utils import check_filter
|
||||
from vitrage.common.constants import VertexProperties as VProps
|
||||
from vitrage.graph.query import create_predicate
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
@ -171,12 +174,23 @@ class NXGraph(Graph):
|
||||
"""
|
||||
self._g.remove_edge(u=e.source_id, v=e.target_id, key=e.label)
|
||||
|
||||
def get_vertices(self, vertex_attr_filter=None):
|
||||
def get_vertices(self, vertex_attr_filter=None, query_dict=None):
|
||||
def check_vertex((v_id, vertex_data)):
|
||||
return check_filter(vertex_data, vertex_attr_filter)
|
||||
|
||||
items = filter(check_vertex, self._g.nodes_iter(data=True))
|
||||
return [vertex_copy(node, node_data) for node, node_data in items]
|
||||
if not query_dict:
|
||||
items = filter(check_vertex, self._g.nodes_iter(data=True))
|
||||
return [vertex_copy(node, node_data) for node, node_data in items]
|
||||
elif not vertex_attr_filter:
|
||||
vertices = []
|
||||
match_func = create_predicate(query_dict)
|
||||
for node, node_data in self._g.nodes_iter(data=True):
|
||||
v = vertex_copy(node, node_data)
|
||||
if match_func(v):
|
||||
vertices.append(v)
|
||||
return vertices
|
||||
else:
|
||||
return []
|
||||
|
||||
def neighbors(self, v_id, vertex_attr_filter=None, edge_attr_filter=None,
|
||||
direction=Direction.BOTH):
|
||||
|
@ -1,4 +1,6 @@
|
||||
# Copyright 2015 - Alcatel-Lucent
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
@ -12,6 +14,7 @@
|
||||
# under the License.
|
||||
|
||||
import vitrage.api
|
||||
import vitrage.entity_graph.consistency
|
||||
import vitrage.evaluator
|
||||
import vitrage.synchronizer
|
||||
import vitrage.synchronizer.plugins
|
||||
@ -22,5 +25,6 @@ def list_opts():
|
||||
('api', vitrage.api.OPTS),
|
||||
('synchronizer', vitrage.synchronizer.OPTS),
|
||||
('evaluator', vitrage.evaluator.OPTS),
|
||||
('synchronizer_plugins', vitrage.synchronizer.plugins.OPTS)
|
||||
('synchronizer_plugins', vitrage.synchronizer.plugins.OPTS),
|
||||
('consistency', vitrage.entity_graph.consistency.OPTS)
|
||||
]
|
||||
|
@ -1 +1,15 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
__author__ = 'stack'
|
||||
|
@ -1 +1,15 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
__author__ = 'stack'
|
||||
|
@ -1 +1,15 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
__author__ = 'stack'
|
||||
|
@ -1 +1,15 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
__author__ = 'stack'
|
||||
|
@ -12,6 +12,8 @@
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
import os
|
||||
|
||||
from vitrage.common import file_utils
|
||||
from vitrage.synchronizer.base import SynchronizerBase
|
||||
|
||||
@ -26,11 +28,12 @@ class StaticPhysical(SynchronizerBase):
|
||||
|
||||
def get_instances(self):
|
||||
static_entities = []
|
||||
static_plugin_configs = file_utils.load_yaml_files(
|
||||
self.cfg.synchronizer_plugins.static_plugins_dir)
|
||||
if os.path.isdir(self.cfg.synchronizer_plugins.static_plugins_dir):
|
||||
static_plugin_configs = file_utils.load_yaml_files(
|
||||
self.cfg.synchronizer_plugins.static_plugins_dir)
|
||||
|
||||
for config in static_plugin_configs:
|
||||
for entity in config['entities']:
|
||||
static_entities.append(entity)
|
||||
for config in static_plugin_configs:
|
||||
for entity in config['entities']:
|
||||
static_entities.append(entity)
|
||||
|
||||
return static_entities
|
||||
|
@ -19,12 +19,12 @@ from oslo_log import log as logging
|
||||
import six
|
||||
|
||||
import vitrage.common.constants as cons
|
||||
from vitrage.common.constants import EntityType
|
||||
from vitrage.common.constants import SynchronizerProperties as SyncProps
|
||||
from vitrage.common.exception import VitrageTransformerError
|
||||
import vitrage.graph.utils as graph_utils
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
NODE_SUBTYPE = 'node'
|
||||
|
||||
|
||||
EntityWrapper = \
|
||||
@ -52,16 +52,16 @@ def build_key(key_values):
|
||||
|
||||
|
||||
def create_node_placeholder_vertex():
|
||||
key = build_key([cons.EntityCategory.RESOURCE, NODE_SUBTYPE])
|
||||
key = build_key([cons.EntityCategory.RESOURCE, EntityType.NODE])
|
||||
|
||||
metadata = {
|
||||
cons.VertexProperties.NAME: NODE_SUBTYPE
|
||||
cons.VertexProperties.NAME: EntityType.NODE
|
||||
}
|
||||
|
||||
return graph_utils.create_vertex(
|
||||
key,
|
||||
entity_category=cons.EntityCategory.RESOURCE,
|
||||
entity_type=NODE_SUBTYPE,
|
||||
entity_type=EntityType.NODE,
|
||||
metadata=metadata
|
||||
)
|
||||
|
||||
|
83
vitrage/tests/unit/entity_graph/__init__.py
Normal file
83
vitrage/tests/unit/entity_graph/__init__.py
Normal file
@ -0,0 +1,83 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
from vitrage.common.constants import SynchronizerProperties as SyncProps
|
||||
from vitrage.common.constants import SyncMode
|
||||
from vitrage.entity_graph.processor import processor as proc
|
||||
|
||||
from vitrage.tests import base
|
||||
|
||||
from vitrage.tests.mocks import mock_syncronizer as mock_sync
|
||||
|
||||
|
||||
class TestEntityGraph(base.BaseTest):
|
||||
|
||||
NUM_NODES = 1
|
||||
NUM_ZONES = 2
|
||||
NUM_HOSTS = 4
|
||||
NUM_INSTANCES = 15
|
||||
|
||||
def _create_processor_with_graph(self, processor=None):
|
||||
events = self._create_mock_events()
|
||||
|
||||
if not processor:
|
||||
processor = proc.Processor()
|
||||
|
||||
for event in events:
|
||||
processor.process_event(event)
|
||||
|
||||
return processor
|
||||
|
||||
def _create_mock_events(self):
|
||||
gen_list = mock_sync.simple_zone_generators(
|
||||
self.NUM_ZONES,
|
||||
self.NUM_HOSTS,
|
||||
snapshot_events=self.NUM_ZONES,
|
||||
snap_vals={SyncProps.SYNC_MODE: SyncMode.INIT_SNAPSHOT})
|
||||
gen_list += mock_sync.simple_host_generators(
|
||||
self.NUM_ZONES,
|
||||
self.NUM_HOSTS,
|
||||
self.NUM_HOSTS,
|
||||
snap_vals={SyncProps.SYNC_MODE: SyncMode.INIT_SNAPSHOT})
|
||||
gen_list += mock_sync.simple_instance_generators(
|
||||
self.NUM_HOSTS,
|
||||
self.NUM_INSTANCES,
|
||||
self.NUM_INSTANCES,
|
||||
snap_vals={SyncProps.SYNC_MODE: SyncMode.INIT_SNAPSHOT})
|
||||
return mock_sync.generate_sequential_events_list(gen_list)
|
||||
|
||||
@staticmethod
|
||||
def _create_event(spec_type=None, sync_mode=None,
|
||||
event_type=None, properties=None):
|
||||
# generate event
|
||||
spec_list = mock_sync.simple_instance_generators(1, 1, 1)
|
||||
events_list = mock_sync.generate_random_events_list(
|
||||
spec_list)
|
||||
|
||||
# update properties
|
||||
if sync_mode is not None:
|
||||
events_list[0][SyncProps.SYNC_MODE] = sync_mode
|
||||
|
||||
if event_type is not None:
|
||||
events_list[0][SyncProps.EVENT_TYPE] = event_type
|
||||
|
||||
if properties is not None:
|
||||
for key, value in properties.iteritems():
|
||||
events_list[0][key] = value
|
||||
|
||||
return events_list[0]
|
||||
|
||||
def _num_total_expected_vertices(self):
|
||||
return self.NUM_NODES + self.NUM_ZONES + self.NUM_HOSTS + \
|
||||
self.NUM_INSTANCES
|
@ -1,4 +1,4 @@
|
||||
# Copyright 2015 - Alcatel-Lucent
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
102
vitrage/tests/unit/entity_graph/consistency/test_consistency.py
Normal file
102
vitrage/tests/unit/entity_graph/consistency/test_consistency.py
Normal file
@ -0,0 +1,102 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
from datetime import timedelta
|
||||
import time
|
||||
|
||||
from oslo_config import cfg
|
||||
|
||||
from vitrage.common.constants import EntityCategory
|
||||
from vitrage.common.constants import EntityType
|
||||
from vitrage.common.constants import VertexProperties as VProps
|
||||
from vitrage.common.datetime_utils import utcnow
|
||||
from vitrage.entity_graph.consistency.consistency_enforcer \
|
||||
import ConsistencyEnforcer
|
||||
from vitrage.entity_graph.processor.processor import Processor
|
||||
from vitrage.tests.unit.entity_graph import TestEntityGraph
|
||||
|
||||
|
||||
class TestConsistency(TestEntityGraph):
|
||||
|
||||
OPTS = [
|
||||
cfg.IntOpt('consistency_interval',
|
||||
default=1,
|
||||
min=1),
|
||||
cfg.IntOpt('min_time_to_delete',
|
||||
default=1,
|
||||
min=1),
|
||||
]
|
||||
|
||||
def setUp(self):
|
||||
super(TestConsistency, self).setUp()
|
||||
self.processor = Processor()
|
||||
self.conf = cfg.ConfigOpts()
|
||||
self.conf.register_opts(self.OPTS, group='consistency')
|
||||
self.consistency_enforcer = ConsistencyEnforcer(
|
||||
self.conf, self.processor.entity_graph)
|
||||
|
||||
def test_periodic_process(self):
|
||||
self._create_processor_with_graph(processor=self.processor)
|
||||
current_time = utcnow()
|
||||
consistency_inteval = self.conf.consistency.consistency_interval
|
||||
|
||||
# set all vertices to be have timestamp that consistency won't get
|
||||
self._update_timestamp(self.processor.entity_graph.get_vertices(),
|
||||
current_time +
|
||||
timedelta(seconds=1.5 * consistency_inteval))
|
||||
|
||||
# check number of instances in graph
|
||||
instance_vertices = self.processor.entity_graph.get_vertices(
|
||||
{VProps.CATEGORY: EntityCategory.RESOURCE,
|
||||
VProps.TYPE: EntityType.NOVA_INSTANCE}
|
||||
)
|
||||
self.assertEqual(self.NUM_INSTANCES, len(instance_vertices))
|
||||
|
||||
# set current timestamp of part of the instances
|
||||
self._update_timestamp(instance_vertices[0:3], current_time)
|
||||
|
||||
# set part of the instances as deleted + update to current timestamp
|
||||
for i in range(3, 6):
|
||||
instance_vertices[i][VProps.IS_DELETED] = True
|
||||
self.processor.entity_graph.update_vertex(instance_vertices[i])
|
||||
|
||||
# set part of the instances as deleted
|
||||
for i in range(6, 9):
|
||||
instance_vertices[i][VProps.IS_DELETED] = True
|
||||
instance_vertices[i][VProps.UPDATE_TIMESTAMP] = str(
|
||||
current_time + timedelta(seconds=2 * consistency_inteval + 1))
|
||||
self.processor.entity_graph.update_vertex(instance_vertices[i])
|
||||
|
||||
# sleep
|
||||
time.sleep(2 * consistency_inteval + 1)
|
||||
|
||||
# run periodic check
|
||||
self.consistency_enforcer.periodic_process()
|
||||
|
||||
# check number of instances
|
||||
instance_vertices = self.processor.entity_graph.get_vertices(
|
||||
{VProps.CATEGORY: EntityCategory.RESOURCE,
|
||||
VProps.TYPE: EntityType.NOVA_INSTANCE}
|
||||
)
|
||||
self.assertEqual(self.NUM_INSTANCES - 6, len(instance_vertices))
|
||||
self.assertEqual(self._num_total_expected_vertices() - 6,
|
||||
len(self.processor.entity_graph.get_vertices()))
|
||||
|
||||
def test_starting_process(self):
|
||||
pass
|
||||
|
||||
def _update_timestamp(self, list, timestamp):
|
||||
for vertex in list:
|
||||
vertex[VProps.UPDATE_TIMESTAMP] = str(timestamp)
|
||||
self.processor.entity_graph.update_vertex(vertex)
|
15
vitrage/tests/unit/entity_graph/processor/__init__.py
Normal file
15
vitrage/tests/unit/entity_graph/processor/__init__.py
Normal file
@ -0,0 +1,15 @@
|
||||
# Copyright 2016 - Nokia
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
__author__ = 'stack'
|
@ -14,7 +14,7 @@
|
||||
|
||||
from vitrage.common.constants import VertexProperties
|
||||
from vitrage.entity_graph.processor import entity_graph as entity_g
|
||||
from vitrage.tests.unit.processor import base
|
||||
from vitrage.tests.unit.entity_graph.processor import base
|
||||
|
||||
|
||||
class TestEntityGraphManager(base.BaseProcessor):
|
@ -19,16 +19,11 @@ from vitrage.common.constants import SyncMode
|
||||
from vitrage.common.constants import VertexProperties
|
||||
from vitrage.common.datetime_utils import utcnow
|
||||
from vitrage.entity_graph.processor import processor as proc
|
||||
from vitrage.tests import base
|
||||
from vitrage.tests.mocks import mock_syncronizer as mock_sync
|
||||
from vitrage.tests.unit.entity_graph import TestEntityGraph
|
||||
|
||||
|
||||
class TestProcessor(base.BaseTest):
|
||||
class TestProcessor(TestEntityGraph):
|
||||
|
||||
NUM_NODES = 1
|
||||
NUM_ZONES = 2
|
||||
NUM_HOSTS = 4
|
||||
NUM_INSTANCES = 15
|
||||
ZONE_SPEC = 'ZONE_SPEC'
|
||||
HOST_SPEC = 'HOST_SPEC'
|
||||
INSTANCE_SPEC = 'INSTANCE_SPEC'
|
||||
@ -45,7 +40,7 @@ class TestProcessor(base.BaseTest):
|
||||
|
||||
# check number of entities
|
||||
num_vertices = len(processor.entity_graph)
|
||||
self.assertEqual(self._num_resources_in_initial_graph(), num_vertices)
|
||||
self.assertEqual(self._num_total_expected_vertices(), num_vertices)
|
||||
|
||||
# TODO(Alexey): add this check and to check also the number of edges
|
||||
# check all entities create a tree and no free floating vertices exists
|
||||
@ -209,47 +204,3 @@ class TestProcessor(base.BaseTest):
|
||||
def _check_graph(self, processor, num_vertices, num_edges):
|
||||
self.assertEqual(num_vertices, len(processor.entity_graph))
|
||||
self.assertEqual(num_edges, processor.entity_graph.num_edges())
|
||||
|
||||
def _num_resources_in_initial_graph(self):
|
||||
return self.NUM_NODES + self.NUM_ZONES + \
|
||||
self.NUM_HOSTS + self.NUM_INSTANCES
|
||||
|
||||
def _create_processor_with_graph(self):
|
||||
events = self._create_mock_events()
|
||||
processor = proc.Processor()
|
||||
|
||||
for event in events:
|
||||
processor.process_event(event)
|
||||
|
||||
return processor
|
||||
|
||||
@staticmethod
|
||||
def _create_mock_events():
|
||||
gen_list = mock_sync.simple_zone_generators(
|
||||
2, 4, snapshot_events=2,
|
||||
snap_vals={SyncProps.SYNC_MODE: SyncMode.INIT_SNAPSHOT})
|
||||
gen_list += mock_sync.simple_host_generators(
|
||||
2, 4, 4, snap_vals={SyncProps.SYNC_MODE: SyncMode.INIT_SNAPSHOT})
|
||||
gen_list += mock_sync.simple_instance_generators(
|
||||
4, 15, 15, snap_vals={SyncProps.SYNC_MODE: SyncMode.INIT_SNAPSHOT})
|
||||
return mock_sync.generate_sequential_events_list(gen_list)
|
||||
|
||||
def _create_event(self, spec_type=None, sync_mode=None,
|
||||
event_type=None, properties=None):
|
||||
# generate event
|
||||
spec_list = mock_sync.simple_instance_generators(1, 1, 1)
|
||||
events_list = mock_sync.generate_random_events_list(
|
||||
spec_list)
|
||||
|
||||
# update properties
|
||||
if sync_mode is not None:
|
||||
events_list[0][SyncProps.SYNC_MODE] = sync_mode
|
||||
|
||||
if event_type is not None:
|
||||
events_list[0][SyncProps.EVENT_TYPE] = event_type
|
||||
|
||||
if properties is not None:
|
||||
for key, value in properties.iteritems():
|
||||
events_list[0][key] = value
|
||||
|
||||
return events_list[0]
|
@ -128,7 +128,7 @@ class NovaZoneTransformerTest(base.BaseTest):
|
||||
for neighbor in neighbors:
|
||||
vertex_type = neighbor.vertex.get(VertexProperties.TYPE)
|
||||
|
||||
if tbase.NODE_SUBTYPE == vertex_type:
|
||||
if EntityType.NODE == vertex_type:
|
||||
node_neighbors_counter += 1
|
||||
self._validate_node_neighbor(neighbor, zone_vertex_id)
|
||||
else:
|
||||
|
Loading…
x
Reference in New Issue
Block a user