Add native OVSDB implementation for bridge monitor

This patch implements an OVS bridge monitor based in the OVSDB
native implementation (OVSDB IDL, Open vSwitch Database Interface
Definition Language). This new implementation supersedes the CLI
OVSDB monitor.

Partial-Bug: #1789592

Change-Id: I9c512d4cbd4cebf94c339231f83bbe89b37650ba
This commit is contained in:
Rodolfo Alonso Hernandez 2018-11-09 17:49:02 +00:00
parent 8db1a47fa8
commit 22c9cf3d95
10 changed files with 301 additions and 207 deletions

View File

@ -12,10 +12,6 @@
# License for the specific language governing permissions and limitations # License for the specific language governing permissions and limitations
# under the License. # under the License.
import contextlib
import eventlet
from oslo_config import cfg
from oslo_log import log as logging from oslo_log import log as logging
from oslo_serialization import jsonutils from oslo_serialization import jsonutils
@ -23,7 +19,6 @@ from neutron.agent.common import async_process
from neutron.agent.ovsdb import api as ovsdb from neutron.agent.ovsdb import api as ovsdb
from neutron.agent.ovsdb.native import helpers from neutron.agent.ovsdb.native import helpers
from neutron.common import utils from neutron.common import utils
from neutron.plugins.ml2.drivers.openvswitch.agent.common import constants
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
@ -34,22 +29,6 @@ OVSDB_ACTION_DELETE = 'delete'
OVSDB_ACTION_NEW = 'new' OVSDB_ACTION_NEW = 'new'
@contextlib.contextmanager
def get_bridges_monitor(
bridges, ovsdb_monitor_respawn_interval=(
constants.DEFAULT_OVSDBMON_RESPAWN)):
mon = SimpleBridgesMonitor(
bridges,
respawn_interval=ovsdb_monitor_respawn_interval,
ovsdb_connection=cfg.CONF.OVS.ovsdb_connection)
mon.start()
try:
yield mon
finally:
mon.stop()
class OvsdbMonitor(async_process.AsyncProcess): class OvsdbMonitor(async_process.AsyncProcess):
"""Manages an invocation of 'ovsdb-client monitor'.""" """Manages an invocation of 'ovsdb-client monitor'."""
@ -144,36 +123,3 @@ class SimpleInterfaceMonitor(OvsdbMonitor):
# update any events with ofports received from 'new' action # update any events with ofports received from 'new' action
for event in self.new_events['added']: for event in self.new_events['added']:
event['ofport'] = dev_to_ofport.get(event['name'], event['ofport']) event['ofport'] = dev_to_ofport.get(event['name'], event['ofport'])
class SimpleBridgesMonitor(OvsdbMonitor):
"""Monitors the Bridge table of the local host's ovsdb for changes.
The bridges_added() method returns all newly created bridges in ovsdb
since the monitor started or since the previous access.
"""
def __init__(self, bridges, respawn_interval=None, ovsdb_connection=None):
super(SimpleBridgesMonitor, self).__init__(
'Bridge',
columns=['name'],
format='json',
respawn_interval=respawn_interval,
ovsdb_connection=ovsdb_connection
)
self.bridges = bridges
@property
def bridges_added(self):
eventlet.sleep()
return self.get_events()['added']
def process_events(self):
bridges_added = []
for row in self.iter_stdout():
json = jsonutils.loads(row).get('data')
for ovs_id, action, name in json:
if name in self.bridges and action == OVSDB_ACTION_INSERT:
bridges_added.append(name)
self.new_events['added'].extend(bridges_added)

View File

@ -40,15 +40,18 @@ Transaction = moves.moved_class(transaction.Transaction,
ovs_conf.register_ovs_agent_opts() ovs_conf.register_ovs_agent_opts()
_connection = None _connection = None
_idl_monitor = None
def api_factory(): def api_factory():
global _connection global _connection
global _idl_monitor
if _connection is None: if _connection is None:
_idl_monitor = n_connection.OvsIdlMonitor()
_connection = connection.Connection( _connection = connection.Connection(
idl=n_connection.idl_factory(), idl=_idl_monitor,
timeout=cfg.CONF.OVS.ovsdb_timeout) timeout=cfg.CONF.OVS.ovsdb_timeout)
return NeutronOvsdbIdl(_connection) return NeutronOvsdbIdl(_connection, _idl_monitor)
class OvsCleanup(command.BaseCommand): class OvsCleanup(command.BaseCommand):
@ -81,9 +84,10 @@ class OvsCleanup(command.BaseCommand):
class NeutronOvsdbIdl(impl_idl.OvsdbIdl): class NeutronOvsdbIdl(impl_idl.OvsdbIdl):
def __init__(self, connection): def __init__(self, connection, idl_monitor):
max_level = None if cfg.CONF.OVS.ovsdb_debug else vlog.INFO max_level = None if cfg.CONF.OVS.ovsdb_debug else vlog.INFO
vlog.use_python_logger(max_level=max_level) vlog.use_python_logger(max_level=max_level)
self.idl_monitor = idl_monitor
super(NeutronOvsdbIdl, self).__init__(connection) super(NeutronOvsdbIdl, self).__init__(connection)
def ovs_cleanup(self, bridges, all_ports=False): def ovs_cleanup(self, bridges, all_ports=False):

View File

@ -13,13 +13,16 @@
# under the License. # under the License.
import os import os
import threading
from debtcollector import moves from debtcollector import moves
from oslo_config import cfg from oslo_config import cfg
from ovs.db import idl from ovs.db import idl
from ovs.stream import Stream from ovs.stream import Stream
from ovsdbapp.backend.ovs_idl import connection as _connection from ovsdbapp.backend.ovs_idl import connection as _connection
from ovsdbapp.backend.ovs_idl import event as idl_event
from ovsdbapp.backend.ovs_idl import idlutils from ovsdbapp.backend.ovs_idl import idlutils
from ovsdbapp import event as ovsdb_event
import tenacity import tenacity
from neutron.agent.ovsdb.native import exceptions as ovsdb_exc from neutron.agent.ovsdb.native import exceptions as ovsdb_exc
@ -54,24 +57,72 @@ def configure_ssl_conn():
Stream.ssl_set_ca_cert_file(req_ssl_opts['ssl_ca_cert_file']) Stream.ssl_set_ca_cert_file(req_ssl_opts['ssl_ca_cert_file'])
def idl_factory(): class BridgeCreateEvent(idl_event.RowEvent):
conn = cfg.CONF.OVS.ovsdb_connection
schema_name = 'Open_vSwitch'
if conn.startswith('ssl:'):
configure_ssl_conn()
try:
helper = idlutils.get_schema_helper(conn, schema_name)
except Exception:
helpers.enable_connection_uri(conn)
@tenacity.retry(wait=tenacity.wait_exponential(multiplier=0.01), def __init__(self, metadata_agent):
stop=tenacity.stop_after_delay(1), self.agent = metadata_agent
reraise=True) table = 'Bridge'
def do_get_schema_helper(): super(BridgeCreateEvent, self).__init__((self.ROW_CREATE, ),
return idlutils.get_schema_helper(conn, schema_name) table, None)
self.event_name = 'BridgeCreateEvent'
helper = do_get_schema_helper() def run(self, event, row, old):
self.agent.add_bridge(str(row.name))
# TODO(twilson) We should still select only the tables/columns we use
helper.register_all() class OvsIdl(idl.Idl):
return idl.Idl(conn, helper)
SCHEMA = 'Open_vSwitch'
def __init__(self):
self._ovsdb_connection = cfg.CONF.OVS.ovsdb_connection
if self._ovsdb_connection.startswith('ssl:'):
configure_ssl_conn()
helper = self._get_ovsdb_helper(self._ovsdb_connection)
helper.register_all()
super(OvsIdl, self).__init__(self._ovsdb_connection, helper)
self.notify_handler = ovsdb_event.RowEventHandler()
@tenacity.retry(wait=tenacity.wait_exponential(multiplier=0.01),
stop=tenacity.stop_after_delay(1),
reraise=True)
def _do_get_schema_helper(self, connection):
return idlutils.get_schema_helper(connection, self.SCHEMA)
def _get_ovsdb_helper(self, connection):
try:
return idlutils.get_schema_helper(connection, self.SCHEMA)
except Exception:
helpers.enable_connection_uri(connection)
return self._do_get_schema_helper(connection)
def notify(self, event, row, updates=None):
self.notify_handler.notify(event, row, updates)
class OvsIdlMonitor(OvsIdl):
def __init__(self):
super(OvsIdlMonitor, self).__init__()
self._lock = threading.Lock()
self._bridges_to_monitor = []
self._bridges_added_list = []
def start_bridge_monitor(self, bridge_names):
if not bridge_names:
return
self._bridges_to_monitor = bridge_names
event = BridgeCreateEvent(self)
self.notify_handler.watch_event(event)
def add_bridge(self, bridge_name):
with self._lock:
if bridge_name in self._bridges_to_monitor:
self._bridges_added_list.append(bridge_name)
@property
def bridges_added(self):
with self._lock:
bridges = self._bridges_added_list
self._bridges_added_list = []
return bridges

View File

@ -45,7 +45,6 @@ import six
from neutron._i18n import _ from neutron._i18n import _
from neutron.agent.common import ip_lib from neutron.agent.common import ip_lib
from neutron.agent.common import ovs_lib from neutron.agent.common import ovs_lib
from neutron.agent.common import ovsdb_monitor
from neutron.agent.common import polling from neutron.agent.common import polling
from neutron.agent.common import utils from neutron.agent.common import utils
from neutron.agent.l2 import l2_agent_extensions_manager as ext_manager from neutron.agent.l2 import l2_agent_extensions_manager as ext_manager
@ -2070,7 +2069,8 @@ class OVSNeutronAgent(l2population_rpc.L2populationRpcCallBackTunnelMixin,
polling_manager.stop() polling_manager.stop()
polling_manager.start() polling_manager.start()
def rpc_loop(self, polling_manager, bridges_monitor=None): def rpc_loop(self, polling_manager):
idl_monitor = self.ovs.ovsdb.idl_monitor
sync = False sync = False
ports = set() ports = set()
updated_ports_copy = set() updated_ports_copy = set()
@ -2089,7 +2089,6 @@ class OVSNeutronAgent(l2population_rpc.L2populationRpcCallBackTunnelMixin,
LOG.info("rpc_loop doing a full sync.") LOG.info("rpc_loop doing a full sync.")
sync = True sync = True
self.fullsync = False self.fullsync = False
bridges_recreated = False
port_info = {} port_info = {}
ancillary_port_info = {} ancillary_port_info = {}
start = time.time() start = time.time()
@ -2107,10 +2106,9 @@ class OVSNeutronAgent(l2population_rpc.L2populationRpcCallBackTunnelMixin,
self.loop_count_and_wait(start, port_stats) self.loop_count_and_wait(start, port_stats)
continue continue
# Check if any physical bridge wasn't recreated recently # Check if any physical bridge wasn't recreated recently
if bridges_monitor: bridges_recreated = self._reconfigure_physical_bridges(
bridges_recreated = self._reconfigure_physical_bridges( idl_monitor.bridges_added)
bridges_monitor.bridges_added) sync |= bridges_recreated
sync |= bridges_recreated
# Notify the plugin of tunnel IP # Notify the plugin of tunnel IP
if self.enable_tunneling and tunnel_sync: if self.enable_tunneling and tunnel_sync:
try: try:
@ -2214,14 +2212,12 @@ class OVSNeutronAgent(l2population_rpc.L2populationRpcCallBackTunnelMixin,
if hasattr(signal, 'SIGHUP'): if hasattr(signal, 'SIGHUP'):
signal.signal(signal.SIGHUP, self._handle_sighup) signal.signal(signal.SIGHUP, self._handle_sighup)
br_names = [br.br_name for br in self.phys_brs.values()] br_names = [br.br_name for br in self.phys_brs.values()]
self.ovs.ovsdb.idl_monitor.start_bridge_monitor(br_names)
with polling.get_polling_manager( with polling.get_polling_manager(
self.minimize_polling, self.minimize_polling,
self.ovsdb_monitor_respawn_interval) as pm,\ self.ovsdb_monitor_respawn_interval) as pm:
ovsdb_monitor.get_bridges_monitor( self.rpc_loop(polling_manager=pm)
br_names,
self.ovsdb_monitor_respawn_interval) as bm:
self.rpc_loop(polling_manager=pm, bridges_monitor=bm)
def _handle_sigterm(self, signum, frame): def _handle_sigterm(self, signum, frame):
self.catch_sigterm = True self.catch_sigterm = True

View File

@ -0,0 +1,47 @@
# Copyright (c) 2018 Red Hat, Inc.
# All Rights Reserved.
#
# 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 neutron.agent.common import ovs_lib
from neutron.common import utils as common_utils
from neutron.tests.functional import base
class BridgeMonitorTestCase(base.BaseLoggingTestCase):
def _delete_bridges(self, bridges):
for bridge in bridges:
self.ovs.delete_bridge(bridge)
def test_create_bridges(self):
bridges_added = []
bridges_to_monitor = ['br01', 'br02', 'br03']
bridges_to_create = ['br01', 'br02', 'br03', 'br04', 'br05']
self.ovs = ovs_lib.BaseOVS()
self.ovs.ovsdb.idl_monitor.start_bridge_monitor(bridges_to_monitor)
self.addCleanup(self._delete_bridges, bridges_to_create)
for bridge in bridges_to_create:
self.ovs.add_bridge(bridge)
def retrieve_bridges(bridges_added):
bridges_added += self.ovs.ovsdb.idl_monitor.bridges_added
return len(bridges_added)
common_utils.wait_until_true(
lambda: retrieve_bridges(bridges_added) == len(bridges_to_monitor),
timeout=5)
bridges_added.sort()
self.assertEqual(bridges_to_monitor, bridges_added)
self.assertEqual([], self.ovs.ovsdb.idl_monitor.bridges_added)

View File

@ -14,96 +14,152 @@
import mock import mock
from ovs.db import idl
from ovs import jsonrpc
from ovsdbapp.backend.ovs_idl import connection from ovsdbapp.backend.ovs_idl import connection
from ovsdbapp.backend.ovs_idl import idlutils from ovsdbapp import event as ovsdb_event
from neutron.agent.ovsdb.native import connection as native_conn from neutron.agent.ovsdb.native import connection as native_conn
from neutron.agent.ovsdb.native import exceptions as ovsdb_exc from neutron.agent.ovsdb.native import exceptions as ovsdb_exc
from neutron.agent.ovsdb.native import helpers
from neutron.tests import base from neutron.tests import base
SSL_KEY_FILE = '/tmp/dummy.pem' SSL_KEY_FILE = '/tmp/dummy.pem'
SSL_CERT_FILE = '/tmp/dummy.crt' SSL_CERT_FILE = '/tmp/dummy.crt'
SSL_CA_FILE = '/tmp/ca.crt' SSL_CA_FILE = '/tmp/ca.crt'
COLUMN_NAME = {'name': {'mutable': False, 'type': 'string'}}
SCHEMA = {'tables': {'Bridge': {'columns': COLUMN_NAME},
'Open_vSwitch': {'columns': COLUMN_NAME},
'Port': {'columns': COLUMN_NAME},
'Interface': {'columns': COLUMN_NAME}},
'version': '7.15.1', 'name': 'Open_vSwitch',
'cksum': '3682332033 23608'}
class TestOVSNativeConnection(base.BaseTestCase):
@mock.patch.object(connection, 'threading') class ConfigureSslConnTestCase(base.BaseTestCase):
@mock.patch.object(idlutils, 'wait_for_change')
@mock.patch.object(native_conn, 'idl') def setUp(self):
@mock.patch.object(helpers, 'enable_connection_uri') super(ConfigureSslConnTestCase, self).setUp()
@mock.patch.object(idlutils, 'get_schema_helper') self._mock_cfg = mock.patch.object(native_conn, 'cfg')
def test_do_get_schema_helper_retry(self, mock_get_schema_helper, self.mock_cfg = self._mock_cfg.start()
mock_enable_conn, self._mock_os = mock.patch.object(native_conn, 'os')
mock_idl, self.mock_os = self._mock_os.start()
mock_wait_for_change, self._mock_stream = mock.patch.object(native_conn, 'Stream')
mock_threading): self.mock_stream = self._mock_stream.start()
mock_helper = mock.Mock() self._mock_has_ever_connected = mock.patch.object(
# raise until 3rd retry attempt idl.Idl, 'has_ever_connected')
mock_get_schema_helper.side_effect = [Exception(), Exception(), self.mock_has_ever_connected = self._mock_has_ever_connected.start()
mock_helper] self.addCleanup(self._clean_mocks)
try:
conn = connection.Connection(idl_factory=native_conn.idl_factory, def _get_ovs_idl_monitor(self):
timeout=mock.Mock()) with mock.patch.object(ovsdb_event, 'RowEventHandler'), \
except TypeError: mock.patch.object(
conn = connection.Connection(idl=native_conn.idl_factory(), native_conn.OvsIdl, '_get_ovsdb_helper',
timeout=mock.Mock()) return_value=idl.SchemaHelper(None, SCHEMA)), \
mock.patch.object(jsonrpc.Session, 'open'), \
mock.patch.object(connection.OvsdbIdl, '__init__'):
return native_conn.OvsIdlMonitor()
def _clean_mocks(self):
self._mock_cfg.stop()
self._mock_os.stop()
self._mock_stream.stop()
self._mock_has_ever_connected.stop()
def test_ssl_connection(self):
self.mock_os.path.isfile.return_value = True
self.mock_cfg.CONF.OVS.ovsdb_connection = 'ssl:127.0.0.1:6640'
self.mock_cfg.CONF.OVS.ssl_key_file = SSL_KEY_FILE
self.mock_cfg.CONF.OVS.ssl_cert_file = SSL_CERT_FILE
self.mock_cfg.CONF.OVS.ssl_ca_cert_file = SSL_CA_FILE
ovs_idl_monitor = self._get_ovs_idl_monitor()
conn = connection.Connection(idl=ovs_idl_monitor,
timeout=1)
conn.start() conn.start()
self.assertEqual(3, len(mock_get_schema_helper.mock_calls)) self.mock_stream.ssl_set_private_key_file.assert_called_once_with(
mock_helper.register_all.assert_called_once_with() SSL_KEY_FILE)
self.mock_stream.ssl_set_certificate_file.assert_called_once_with(
@mock.patch.object(native_conn, 'Stream') SSL_CERT_FILE)
@mock.patch.object(connection, 'threading') self.mock_stream.ssl_set_ca_cert_file.assert_called_once_with(
@mock.patch.object(native_conn, 'idl') SSL_CA_FILE)
@mock.patch.object(idlutils, 'get_schema_helper')
@mock.patch.object(native_conn, 'os')
@mock.patch.object(native_conn, 'cfg')
def test_ssl_connection(self, mock_cfg, mock_os, mock_get_schema_helper,
mock_idl, mock_threading, mock_stream):
mock_os.path.isfile.return_value = True
mock_cfg.CONF.OVS.ovsdb_connection = 'ssl:127.0.0.1:6640'
mock_cfg.CONF.OVS.ssl_key_file = SSL_KEY_FILE
mock_cfg.CONF.OVS.ssl_cert_file = SSL_CERT_FILE
mock_cfg.CONF.OVS.ssl_ca_cert_file = SSL_CA_FILE
conn = connection.Connection(idl=native_conn.idl_factory(),
timeout=1)
conn.start()
mock_stream.ssl_set_private_key_file.assert_called_once_with(
SSL_KEY_FILE
)
mock_stream.ssl_set_certificate_file.assert_called_once_with(
SSL_CERT_FILE
)
mock_stream.ssl_set_ca_cert_file.assert_called_once_with(
SSL_CA_FILE
)
@mock.patch.object(native_conn, 'Stream')
@mock.patch.object(connection, 'threading')
@mock.patch.object(native_conn, 'idl')
@mock.patch.object(idlutils, 'get_schema_helper')
@mock.patch.object(native_conn, 'cfg')
def test_ssl_conn_file_missing(self, mock_cfg, mock_get_schema_helper,
mock_idl, mock_threading, mock_stream):
mock_cfg.CONF.OVS.ovsdb_connection = 'ssl:127.0.0.1:6640'
mock_cfg.CONF.OVS.ssl_key_file = SSL_KEY_FILE
mock_cfg.CONF.OVS.ssl_cert_file = SSL_CERT_FILE
mock_cfg.CONF.OVS.ssl_ca_cert_file = SSL_CA_FILE
def test_ssl_conn_file_missing(self):
self.mock_cfg.CONF.OVS.ovsdb_connection = 'ssl:127.0.0.1:6640'
self.mock_cfg.CONF.OVS.ssl_key_file = SSL_KEY_FILE
self.mock_cfg.CONF.OVS.ssl_cert_file = SSL_CERT_FILE
self.mock_cfg.CONF.OVS.ssl_ca_cert_file = SSL_CA_FILE
self.mock_os.path.exists.return_value = False
self.assertRaises(ovsdb_exc.OvsdbSslConfigNotFound, self.assertRaises(ovsdb_exc.OvsdbSslConfigNotFound,
native_conn.idl_factory) self._get_ovs_idl_monitor)
@mock.patch.object(native_conn, 'Stream') def test_ssl_conn_cfg_missing(self):
@mock.patch.object(connection, 'threading') self.mock_cfg.CONF.OVS.ovsdb_connection = 'ssl:127.0.0.1:6640'
@mock.patch.object(native_conn, 'idl') self.mock_cfg.CONF.OVS.ssl_key_file = None
@mock.patch.object(idlutils, 'get_schema_helper') self.mock_cfg.CONF.OVS.ssl_cert_file = None
@mock.patch.object(native_conn, 'cfg') self.mock_cfg.CONF.OVS.ssl_ca_cert_file = None
def test_ssl_conn_cfg_missing(self, mock_cfg, mock_get_schema_helper,
mock_idl, mock_threading, mock_stream):
mock_cfg.CONF.OVS.ovsdb_connection = 'ssl:127.0.0.1:6640'
mock_cfg.CONF.OVS.ssl_key_file = None
mock_cfg.CONF.OVS.ssl_cert_file = None
mock_cfg.CONF.OVS.ssl_ca_cert_file = None
self.assertRaises(ovsdb_exc.OvsdbSslRequiredOptError, self.assertRaises(ovsdb_exc.OvsdbSslRequiredOptError,
native_conn.idl_factory) self._get_ovs_idl_monitor)
class BridgeCreateEventTestCase(base.BaseTestCase):
class MetadataAgent(object):
bridges = []
def add_bridge(self, row_name):
self.bridges.append(row_name)
def test_run(self):
agent = self.MetadataAgent()
mock_row = mock.Mock()
mock_row.name = 'row_name'
bridge_create_event = native_conn.BridgeCreateEvent(agent)
bridge_create_event.run(mock.ANY, mock_row, mock.ANY)
self.assertEqual([mock_row.name], agent.bridges)
class OvsIdlMonitorTestCase(base.BaseTestCase):
def setUp(self):
super(OvsIdlMonitorTestCase, self).setUp()
self._mock_get_ovsdb_helper = mock.patch.object(
native_conn.OvsIdl, '_get_ovsdb_helper')
self._mock_get_ovsdb_helper.start()
self._mock_row_event_handler = mock.patch.object(ovsdb_event,
'RowEventHandler')
self._mock_row_event_handler.start()
self._mock_idl = mock.patch.object(idl.Idl, '__init__')
self._mock_idl.start()
self.addCleanup(self._stop_mocks)
self.ovs_idl_monitor = native_conn.OvsIdlMonitor()
def _stop_mocks(self):
self._mock_get_ovsdb_helper.stop()
self._mock_row_event_handler.stop()
self._mock_idl.stop()
@mock.patch.object(native_conn, 'BridgeCreateEvent')
def test_start_bridge_monitor(self, mock_bridge_event):
mock_bridge_event.return_value = 'bridge_event'
self.ovs_idl_monitor.start_bridge_monitor(['br01', 'br02'])
self.assertEqual(['br01', 'br02'],
self.ovs_idl_monitor._bridges_to_monitor)
self.ovs_idl_monitor.notify_handler.\
watch_event.assert_called_once_with('bridge_event')
def test_add_bridge(self):
self.ovs_idl_monitor.start_bridge_monitor(['br01', 'br02'])
self.ovs_idl_monitor.add_bridge('br01')
self.ovs_idl_monitor.add_bridge('br02')
self.ovs_idl_monitor.add_bridge('br03')
self.assertEqual(['br01', 'br02'],
self.ovs_idl_monitor._bridges_added_list)
def test_bridged_added(self):
self.ovs_idl_monitor.start_bridge_monitor(['br01', 'br02'])
self.ovs_idl_monitor.add_bridge('br01')
self.ovs_idl_monitor.add_bridge('br02')
self.assertEqual(['br01', 'br02'], self.ovs_idl_monitor.bridges_added)
self.assertEqual([], self.ovs_idl_monitor.bridges_added)

View File

@ -26,6 +26,7 @@ import testtools
from neutron._i18n import _ from neutron._i18n import _
from neutron.agent.common import async_process from neutron.agent.common import async_process
from neutron.agent.common import ovs_lib from neutron.agent.common import ovs_lib
from neutron.agent.common import polling
from neutron.agent.common import utils from neutron.agent.common import utils
from neutron.agent.linux import ip_lib from neutron.agent.linux import ip_lib
from neutron.common import constants as c_const from neutron.common import constants as c_const
@ -1768,52 +1769,39 @@ class TestOvsNeutronAgent(object):
mock.Mock(br_name='br-ex1')] mock.Mock(br_name='br-ex1')]
phys_bridges = {'physnet0': ex_br_mocks[0], phys_bridges = {'physnet0': ex_br_mocks[0],
'physnet1': ex_br_mocks[1]}, 'physnet1': ex_br_mocks[1]},
bm_mock = mock.Mock() with mock.patch.object(self.agent, 'check_ovs_status',
with mock.patch( return_value=constants.OVS_NORMAL), \
'neutron.agent.common.ovsdb_monitor.get_bridges_monitor', mock.patch.object(self.agent, '_agent_has_updates',
return_value=bm_mock),\ side_effect=TypeError('loop exit')), \
mock.patch.object( mock.patch.dict(self.agent.bridge_mappings, bridge_mappings,
self.agent, clear=True), \
'check_ovs_status', mock.patch.dict(self.agent.phys_brs, phys_bridges,
return_value=constants.OVS_NORMAL),\ clear=True), \
mock.patch.object( mock.patch.object(self.agent, 'setup_physical_bridges') as \
self.agent, setup_physical_bridges, \
'_agent_has_updates', mock.patch.object(self.agent.ovs.ovsdb, 'idl_monitor') as \
side_effect=TypeError('loop exit')),\ mock_idl_monitor:
mock.patch.dict( mock_idl_monitor.bridges_added = ['br-ex0']
self.agent.bridge_mappings, bridge_mappings, clear=True),\
mock.patch.dict(
self.agent.phys_brs, phys_bridges, clear=True),\
mock.patch.object(
self.agent,
'setup_physical_bridges') as setup_physical_bridges:
bm_mock.bridges_added = ['br-ex0']
try: try:
self.agent.rpc_loop(polling_manager=mock.Mock(), self.agent.rpc_loop(polling_manager=mock.Mock())
bridges_monitor=bm_mock)
except TypeError: except TypeError:
pass pass
setup_physical_bridges.assert_called_once_with( setup_physical_bridges.assert_called_once_with({'physnet0': 'br-ex0'})
{'physnet0': 'br-ex0'})
def test_daemon_loop_uses_polling_manager(self): def test_daemon_loop_uses_polling_manager(self):
ex_br_mock = mock.Mock(br_name="br-ex0") ex_br_mock = mock.Mock(br_name="br-ex0")
with mock.patch( with mock.patch.object(polling, 'get_polling_manager') as \
'neutron.agent.common.polling.get_polling_manager' mock_get_pm, \
) as mock_get_pm, mock.patch( mock.patch.object(self.agent, 'rpc_loop') as mock_loop, \
'neutron.agent.common.ovsdb_monitor.get_bridges_monitor' mock.patch.dict(self.agent.phys_brs, {'physnet0': ex_br_mock},
) as mock_get_bm, mock.patch.object( clear=True), \
self.agent, 'rpc_loop' mock.patch.object(self.agent.ovs.ovsdb, 'idl_monitor') as \
) as mock_loop, mock.patch.dict( mock_idl_monitor:
self.agent.phys_brs, {'physnet0': ex_br_mock}, clear=True):
self.agent.daemon_loop() self.agent.daemon_loop()
mock_get_pm.assert_called_with(True, mock_get_pm.assert_called_with(True,
constants.DEFAULT_OVSDBMON_RESPAWN) constants.DEFAULT_OVSDBMON_RESPAWN)
mock_get_bm.assert_called_once_with( mock_loop.assert_called_once_with(polling_manager=mock.ANY)
['br-ex0'], constants.DEFAULT_OVSDBMON_RESPAWN) mock_idl_monitor.start_bridge_monitor.assert_called()
mock_loop.assert_called_once_with(
polling_manager=mock.ANY, bridges_monitor=mock.ANY)
def test_setup_tunnel_port_invalid_ofport(self): def test_setup_tunnel_port_invalid_ofport(self):
remote_ip = '1.2.3.4' remote_ip = '1.2.3.4'
@ -2054,7 +2042,8 @@ class TestOvsNeutronAgent(object):
'setup_tunnel_br_flows') as setup_tunnel_br_flows,\ 'setup_tunnel_br_flows') as setup_tunnel_br_flows,\
mock.patch.object( mock.patch.object(
self.mod_agent.OVSNeutronAgent, self.mod_agent.OVSNeutronAgent,
'_reset_tunnel_ofports') as reset_tunnel_ofports: '_reset_tunnel_ofports') as reset_tunnel_ofports, \
mock.patch.object(self.agent.ovs.ovsdb, 'idl_monitor'):
log_exception.side_effect = Exception( log_exception.side_effect = Exception(
'Fake exception to get out of the loop') 'Fake exception to get out of the loop')
devices_not_ready = set() devices_not_ready = set()
@ -2125,7 +2114,8 @@ class TestOvsNeutronAgent(object):
'cleanup_stale_flows') as cleanup,\ 'cleanup_stale_flows') as cleanup,\
mock.patch.object( mock.patch.object(
self.mod_agent.OVSNeutronAgent, self.mod_agent.OVSNeutronAgent,
'_check_and_handle_signal') as check_and_handle_signal: '_check_and_handle_signal') as check_and_handle_signal, \
mock.patch.object(self.agent.ovs.ovsdb, 'idl_monitor'):
process_network_ports.side_effect = Exception("Trigger resync") process_network_ports.side_effect = Exception("Trigger resync")
check_ovs_status.return_value = constants.OVS_NORMAL check_ovs_status.return_value = constants.OVS_NORMAL
check_and_handle_signal.side_effect = [True, False] check_and_handle_signal.side_effect = [True, False]
@ -3473,7 +3463,8 @@ class TestOvsDvrNeutronAgent(object):
mock.patch.object(self.agent, 'setup_physical_bridges'),\ mock.patch.object(self.agent, 'setup_physical_bridges'),\
mock.patch.object(self.agent, 'setup_integration_br'),\ mock.patch.object(self.agent, 'setup_integration_br'),\
mock.patch.object(self.agent, 'setup_tunnel_br'),\ mock.patch.object(self.agent, 'setup_tunnel_br'),\
mock.patch.object(self.agent, 'state_rpc'): mock.patch.object(self.agent, 'state_rpc'), \
mock.patch.object(self.agent.ovs.ovsdb, 'idl_monitor'):
try: try:
self.agent.rpc_loop(polling_manager=mock.Mock()) self.agent.rpc_loop(polling_manager=mock.Mock())
except TypeError: except TypeError:
@ -3500,7 +3491,8 @@ class TestOvsDvrNeutronAgent(object):
side_effect=[True, False]),\ side_effect=[True, False]),\
mock.patch.object(self.agent, 'setup_physical_bridges'),\ mock.patch.object(self.agent, 'setup_physical_bridges'),\
mock.patch.object(self.agent, 'setup_integration_br'),\ mock.patch.object(self.agent, 'setup_integration_br'),\
mock.patch.object(self.agent, 'state_rpc'): mock.patch.object(self.agent, 'state_rpc'), \
mock.patch.object(self.agent.ovs.ovsdb, 'idl_monitor'):
# block RPC calls and bridge calls # block RPC calls and bridge calls
self.agent.rpc_loop(polling_manager=mock.Mock()) self.agent.rpc_loop(polling_manager=mock.Mock())

View File

@ -302,8 +302,10 @@ class TunnelTest(object):
cfg.CONF.set_override(k, v, 'AGENT') cfg.CONF.set_override(k, v, 'AGENT')
ext_mgr = mock.Mock() ext_mgr = mock.Mock()
return self.mod_agent.OVSNeutronAgent( agent = self.mod_agent.OVSNeutronAgent(
bridge_classes, ext_mgr, cfg.CONF) bridge_classes, ext_mgr, cfg.CONF)
mock.patch.object(agent.ovs.ovsdb, 'idl_monitor').start()
return agent
def _verify_mock_call(self, mock_obj, expected): def _verify_mock_call(self, mock_obj, expected):
mock_obj.assert_has_calls(expected) mock_obj.assert_has_calls(expected)