add consistency with periodic process

implements: blueprint entity-graph-consistency-validator

Change-Id: Ifaff9117d0164b3b96253092bace3a56b6cdd778
This commit is contained in:
Alexey Weyl 2016-02-10 17:12:23 +02:00
parent 841f0f1c68
commit 18135a4dba
23 changed files with 465 additions and 96 deletions

View File

@ -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()

View File

@ -58,6 +58,7 @@ class SynchronizerProperties(object):
class EntityType(object):
NODE = 'node'
NOVA_INSTANCE = 'nova.instance'
NOVA_HOST = 'nova.host'
NOVA_ZONE = 'nova.zone'

View File

@ -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]})

View File

@ -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)'),
]

View 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

View 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")

View File

@ -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
"""

View File

@ -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):

View File

@ -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)
]

View File

@ -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'

View File

@ -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'

View File

@ -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'

View File

@ -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'

View File

@ -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

View File

@ -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
)

View 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

View File

@ -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

View 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)

View 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'

View File

@ -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):

View File

@ -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]

View File

@ -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: