Merge "Add QoS minimum egress bandwidth rule into ovs-agent"

This commit is contained in:
Zuul 2019-03-04 12:12:54 +00:00 committed by Gerrit Code Review
commit c3d34901dd
14 changed files with 769 additions and 29 deletions

View File

@ -53,7 +53,7 @@ traffic directions (from the VM point of view).
Rule \\ back end Open vSwitch SR-IOV Linux bridge
==================== =================== ================ ===================
Bandwidth limit Egress \\ Ingress Egress (1) Egress \\ Ingress
Minimum bandwidth - Egress -
Minimum bandwidth Egress (2) Egress -
DSCP marking Egress - Egress
==================== =================== ================ ===================
@ -61,6 +61,8 @@ traffic directions (from the VM point of view).
(1) Max burst parameter is skipped because it is not supported by the
IP tool.
(2) Only for physical bridges (tenant networks, provider networks), tunneled
traffic is not shaped yet.
In the most simple case, the property can be represented by a simple Python
list defined on the class.

View File

@ -22,8 +22,10 @@ import uuid
from neutron_lib import constants as p_const
from neutron_lib import exceptions
from neutron_lib.services.qos import constants as qos_constants
from oslo_config import cfg
from oslo_log import log as logging
from oslo_utils import uuidutils
import six
import tenacity
@ -223,6 +225,7 @@ class OVSBridge(BaseOVS):
self.datapath_type = datapath_type
self._default_cookie = generate_random_cookie()
self._highest_protocol_needed = constants.OPENFLOW10
self._min_bw_qos_id = uuidutils.generate_uuid()
@property
def default_cookie(self):
@ -928,6 +931,194 @@ class OVSBridge(BaseOVS):
self.set_db_attribute('Bridge', self.br_name, 'other_config', dpid_cfg,
check_error=True)
def get_egress_min_bw_for_port(self, port_id):
queue = self._find_queue(port_id)
if not queue:
return
min_bps = queue['other_config'].get('min-rate')
return int(int(min_bps) / 1000) if min_bps else None
def _set_queue_for_minimum_bandwidth(self, queue_num):
# reg4 is used to memoize if queue was set or not. If it is first visit
# to table 0 for a packet (i.e. reg4 == 0), set queue and memoize (i.e.
# load 1 to reg4), then goto table 0 again. The packet will be handled
# as usual when the second visit to table 0.
self.add_flow(
table=constants.LOCAL_SWITCHING,
in_port=queue_num,
reg4=0,
priority=200,
actions=("set_queue:%s,load:1->NXM_NX_REG4[0],"
"resubmit(,%s)" % (queue_num, constants.LOCAL_SWITCHING)))
def _unset_queue_for_minimum_bandwidth(self, queue_num):
self.delete_flows(
table=constants.LOCAL_SWITCHING,
in_port=queue_num,
reg4=0)
def update_minimum_bandwidth_queue(self, port_id, egress_port_names,
queue_num, min_kbps):
queue_num = int(queue_num)
queue_id = self._update_queue(port_id, queue_num, min_kbps=min_kbps)
qos_id, qos_queues = self._find_qos()
if qos_queues:
qos_queues[queue_num] = queue_id
else:
qos_queues = {queue_num: queue_id}
qos_id = self._update_qos(
qos_id=qos_id, queues=qos_queues)
for egress_port_name in egress_port_names:
self._set_port_qos(egress_port_name, qos_id=qos_id)
self._set_queue_for_minimum_bandwidth(queue_num)
return qos_id
def delete_minimum_bandwidth_queue(self, port_id):
queue = self._find_queue(port_id)
if not queue:
return
queue_num = int(queue['external_ids']['queue-num'])
self._unset_queue_for_minimum_bandwidth(queue_num)
qos_id, qos_queues = self._find_qos()
if queue_num in qos_queues.keys():
qos_queues.pop(queue_num)
self._update_qos(
qos_id=qos_id, queues=qos_queues)
self._delete_queue(queue['_uuid'])
def clear_minimum_bandwidth_qos(self):
qoses = self._list_qos(
qos_type=qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH)
for qos in qoses:
qos_id = qos['_uuid']
queues = {num: queue.uuid
for num, queue in qos['queues'].items()}
ports = self.ovsdb.db_find(
'Port',
('qos', '=', qos_id),
colmuns=['name']).execute(check_error=True)
for port in ports:
self._set_port_qos(port['name'])
self.ovsdb.db_destroy('QoS', qos_id).execute(check_error=True)
for queue_uuid in queues.values():
self.ovsdb.db_destroy('Queue', queue_uuid).execute(
check_error=True)
def _update_queue(self, port_id, queue_num, max_kbps=None,
max_burst_kbps=None, min_kbps=None):
other_config = {}
if max_kbps:
other_config[six.u('max-rate')] = six.u(str(max_kbps * 1000))
if max_burst_kbps:
other_config[six.u('burst')] = six.u(str(max_burst_kbps * 1000))
if min_kbps:
other_config[six.u('min-rate')] = six.u(str(min_kbps * 1000))
queue = self._find_queue(port_id)
if queue and queue['_uuid']:
if queue['other_config'] != other_config:
self.set_db_attribute('Queue', queue['_uuid'], 'other_config',
other_config, check_error=True)
else:
# NOTE(ralonsoh): "external_ids" is a map of string-string pairs
external_ids = {
'port': str(port_id),
'type': str(qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH),
'queue-num': str(queue_num)}
self.ovsdb.db_create(
'Queue', other_config=other_config,
external_ids=external_ids).execute(check_error=True)
queue = self._find_queue(port_id)
return queue['_uuid']
def _find_queue(self, port_id, _type=None):
# NOTE(ralonsoh): in ovsdb native library, '{>=}' operator is not
# implemented yet. This is a workaround: list all queues and compare
# the external_ids key needed.
_type = _type or qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH
queues = self._list_queues(port=port_id, _type=_type)
if queues:
return queues[0]
return None
def _list_queues(self, _type=None, port=None):
queues = self.ovsdb.db_list(
'Queue',
columns=['_uuid', 'external_ids', 'other_config']).execute(
check_error=True)
if port:
queues = [queue for queue in queues
if queue['external_ids'].get('port') == str(port)]
if _type:
queues = [queue for queue in queues
if queue['external_ids'].get('type') == str(_type)]
return queues
def _delete_queue(self, queue_id):
self.ovsdb.db_destroy('Queue', queue_id).execute(check_error=True)
def _update_qos(self, qos_id=None, queues=None):
queues = queues or {}
if not qos_id:
external_ids = {'id': self._min_bw_qos_id,
'_type': qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH}
self.ovsdb.db_create(
'QoS',
type='linux-htb',
queues=queues,
external_ids=external_ids).execute(check_error=True)
qos_id, _ = self._find_qos()
else:
self.clear_db_attribute('QoS', qos_id, 'queues')
if queues:
self.set_db_attribute('QoS', qos_id, 'queues', queues,
check_error=True)
return qos_id
def _list_qos(self, _id=None, qos_type=None):
external_ids = {}
if _id:
external_ids['id'] = _id
if qos_type:
external_ids['_type'] = qos_type
if external_ids:
return self.ovsdb.db_find(
'QoS',
('external_ids', '=', external_ids),
colmuns=['_uuid', 'queues']).execute(check_error=True)
return self.ovsdb.db_find(
'QoS', colmuns=['_uuid', 'queues']).execute(check_error=True)
def _find_qos(self):
qos_regs = self._list_qos(_id=self._min_bw_qos_id)
if qos_regs:
queues = {num: queue.uuid
for num, queue in qos_regs[0]['queues'].items()}
return qos_regs[0]['_uuid'], queues
return None, None
def _set_port_qos(self, port_name, qos_id=None):
if qos_id:
self.set_db_attribute('Port', port_name, 'qos', qos_id,
check_error=True)
else:
self.clear_db_attribute('Port', port_name, 'qos')
def get_bridge_ports(self, port_type=None):
port_names = self.get_port_name_list() + [self.br_name]
ports = self.get_ports_attributes('Interface',
ports=port_names,
columns=['name', 'type'],
if_exists=True) or []
if port_type is None:
return ports
elif not isinstance(port_type, list):
port_type = [port_type]
return [port['name'] for port in ports if port['type'] in port_type]
def __enter__(self):
self.create()
return self

View File

@ -40,9 +40,21 @@ class QosOVSAgentDriver(qos.QosLinuxAgentDriver):
def consume_api(self, agent_api):
self.agent_api = agent_api
def _minimum_bandwidth_initialize(self):
"""Clear QoS setting at agent restart.
This is for clearing stale settings (such as ports and QoS tables
deleted while the agent is down). The current implementation
can not find stale settings. The solution is to clear everything and
rebuild. There is no performance impact however the QoS feature will
be down until the QoS rules are rebuilt.
"""
self.br_int.clear_minimum_bandwidth_qos()
def initialize(self):
self.br_int = self.agent_api.request_int_br()
self.cookie = self.br_int.default_cookie
self._minimum_bandwidth_initialize()
def create_bandwidth_limit(self, port, rule):
self.update_bandwidth_limit(port, rule)
@ -174,24 +186,36 @@ class QosOVSAgentDriver(qos.QosLinuxAgentDriver):
max_burst_kbps
)
# Note(lajoskatona): As minimum bandwidth rule was allowed to be used by
# OVS and SRIOV even with ingress direction for the placement based
# enforcement, but the dataplane enforcement implementation is not yet
# ready these methods are empty.
# For details see:
# RFE for placement based enforcement:
# https://bugs.launchpad.net/neutron/+bug/1578989
# RFE for dataplane based enforcement:
# https://bugs.launchpad.net/neutron/+bug/1560963
def create_minimum_bandwidth(self, port, rule):
LOG.debug("Minimum bandwidth rule was created for port %s and "
"rule %s.", port['port_id'], rule.id)
self.update_minimum_bandwidth(port, rule)
def update_minimum_bandwidth(self, port, rule):
LOG.debug("Minimum bandwidth rule was updated for port %s and "
"rule %s.", port['port_id'], rule.id)
vif_port = port.get('vif_port')
if not vif_port:
LOG.debug('update_minimum_bandwidth was received for port %s but '
'vif_port was not found. It seems that port is already '
'deleted', port.get('port_id'))
return
# queue_num is used to identify the port which traffic come from,
# it needs to be unique across br-int. It is convenient to use ofport
# as queue_num because it is unique in br-int and start from 1.
egress_port_names = []
for phy_br in self.agent_api.request_phy_brs():
ports = phy_br.get_bridge_ports('')
if not ports:
LOG.warning('Bridge %s does not have a physical port '
'connected', phy_br.br_name)
egress_port_names.extend(ports)
qos_id = self.br_int.update_minimum_bandwidth_queue(
port['port_id'], egress_port_names, vif_port.ofport, rule.min_kbps)
LOG.debug('Minimum bandwidth rule was updated/created for port '
'%(port_id)s and rule %(rule_id)s. QoS ID: %(qos_id)s. '
'Egress ports with QoS applied: %(ports)s',
{'port_id': port['port_id'], 'rule_id': rule.id,
'qos_id': qos_id, 'ports': egress_port_names})
def delete_minimum_bandwidth(self, port):
self.br_int.delete_minimum_bandwidth_queue(port['port_id'])
LOG.debug("Minimum bandwidth rule was deleted for port: %s.",
port['port_id'])

View File

@ -40,10 +40,11 @@ class OVSAgentExtensionAPI(object):
method which has been added to the AgentExtension class.
'''
def __init__(self, int_br, tun_br):
def __init__(self, int_br, tun_br, phys_brs):
super(OVSAgentExtensionAPI, self).__init__()
self.br_int = int_br
self.br_tun = tun_br
self.br_phys = phys_brs
def request_int_br(self):
"""Allows extensions to request an integration bridge to use for
@ -61,3 +62,13 @@ class OVSAgentExtensionAPI(object):
return None
return OVSCookieBridge(self.br_tun)
def request_phy_brs(self):
"""Allows extensions to request all physical bridges to use for
extension specific flows.
This a generator function which returns all existing physical bridges
in the switch.
"""
for phy_br in self.br_phys.values():
yield OVSCookieBridge(phy_br)

View File

@ -227,10 +227,6 @@ class OVSNeutronAgent(l2population_rpc.L2populationRpcCallBackTunnelMixin,
self.setup_tunnel_br(ovs_conf.tunnel_bridge)
self.setup_tunnel_br_flows()
agent_api = ovs_ext_api.OVSAgentExtensionAPI(self.int_br, self.tun_br)
self.ext_manager.initialize(
self.connection, constants.EXTENSION_DRIVER_TYPE, agent_api)
self.dvr_agent = ovs_dvr_neutron_agent.OVSDVRNeutronAgent(
self.context,
self.dvr_plugin_rpc,
@ -253,6 +249,12 @@ class OVSNeutronAgent(l2population_rpc.L2populationRpcCallBackTunnelMixin,
self.ancillary_brs = self.setup_ancillary_bridges(
ovs_conf.integration_bridge, ovs_conf.tunnel_bridge)
agent_api = ovs_ext_api.OVSAgentExtensionAPI(self.int_br,
self.tun_br,
self.phys_brs)
self.ext_manager.initialize(
self.connection, constants.EXTENSION_DRIVER_TYPE, agent_api)
# In order to keep existed device's local vlan unchanged,
# restore local vlan mapping at start
self._restore_local_vlan_map()

View File

@ -203,6 +203,22 @@ class ClientFixture(fixtures.Fixture):
return rule['bandwidth_limit_rule']
def create_minimum_bandwidth_rule(self, tenant_id, qos_policy_id,
min_bw, direction=None):
rule = {'tenant_id': tenant_id,
'min_kbps': min_bw}
if direction:
rule['direction'] = direction
rule = self.client.create_minimum_bandwidth_rule(
policy=qos_policy_id,
body={'minimum_bandwidth_rule': rule})
self.addCleanup(_safe_method(
self.client.delete_minimum_bandwidth_rule),
rule['minimum_bandwidth_rule']['id'], qos_policy_id)
return rule['minimum_bandwidth_rule']
def create_dscp_marking_rule(self, tenant_id, qos_policy_id, dscp_mark=0):
rule = {'tenant_id': tenant_id}
if dscp_mark:

View File

@ -42,6 +42,7 @@ load_tests = testlib_api.module_load_tests
BANDWIDTH_BURST = 100
BANDWIDTH_LIMIT = 500
MIN_BANDWIDTH = 300
DSCP_MARK = 16
@ -609,3 +610,88 @@ class TestQoSPolicyIsDefault(base.BaseFullStackTestCase):
self.assertFalse(qos_policy_2['is_default'])
self.assertRaises(exceptions.Conflict,
self._update_qos_policy, qos_policy_2['id'], True)
class _TestMinBwQoS(BaseQoSRuleTestCase):
number_of_hosts = 1
def _wait_for_min_bw_rule_removed(self, vm, direction):
# No values are provided when port doesn't have qos policy
self._wait_for_min_bw_rule_applied(vm, None, direction)
def _add_min_bw_rule(self, min_bw, direction, qos_policy):
qos_policy_id = qos_policy['id']
rule = self.safe_client.create_minimum_bandwidth_rule(
self.tenant_id, qos_policy_id, min_bw, direction)
# Make it consistent with GET reply
rule['type'] = qos_consts.RULE_TYPE_MINIMUM_BANDWIDTH
rule['qos_policy_id'] = qos_policy_id
qos_policy['rules'].append(rule)
def test_min_bw_qos_policy_rule_lifecycle(self):
new_limit = MIN_BANDWIDTH - 100
# Create port with qos policy attached
vm, qos_policy = self._prepare_vm_with_qos_policy(
[functools.partial(
self._add_min_bw_rule, MIN_BANDWIDTH, self.direction)])
bw_rule = qos_policy['rules'][0]
self._wait_for_min_bw_rule_applied(vm, MIN_BANDWIDTH, self.direction)
qos_policy_id = qos_policy['id']
self.client.delete_minimum_bandwidth_rule(bw_rule['id'], qos_policy_id)
self._wait_for_min_bw_rule_removed(vm, self.direction)
new_rule = self.safe_client.create_minimum_bandwidth_rule(
self.tenant_id, qos_policy_id, new_limit, direction=self.direction)
self._wait_for_min_bw_rule_applied(vm, new_limit, self.direction)
# Update qos policy rule id
self.client.update_minimum_bandwidth_rule(
new_rule['id'], qos_policy_id,
body={'minimum_bandwidth_rule': {'min_kbps': MIN_BANDWIDTH}})
self._wait_for_min_bw_rule_applied(vm, MIN_BANDWIDTH, self.direction)
# Remove qos policy from port
self.client.update_port(
vm.neutron_port['id'],
body={'port': {'qos_policy_id': None}})
self._wait_for_min_bw_rule_removed(vm, self.direction)
class TestMinBwQoSOvs(_TestMinBwQoS, base.BaseFullStackTestCase):
l2_agent_type = constants.AGENT_TYPE_OVS
direction_scenarios = [
('egress', {'direction': constants.EGRESS_DIRECTION})
]
scenarios = testscenarios.multiply_scenarios(
direction_scenarios, fullstack_utils.get_ovs_interface_scenarios())
def _wait_for_min_bw_rule_applied(self, vm, min_bw, direction):
if direction == constants.EGRESS_DIRECTION:
utils.wait_until_true(
lambda: vm.bridge.get_egress_min_bw_for_port(
vm.neutron_port['id']) == min_bw)
elif direction == constants.INGRESS_DIRECTION:
self.fail('"%s" direction not implemented'
% constants.INGRESS_DIRECTION)
def test_bw_limit_qos_port_removed(self):
"""Test if rate limit config is properly removed when whole port is
removed.
"""
# Create port with qos policy attached
vm, qos_policy = self._prepare_vm_with_qos_policy(
[functools.partial(
self._add_min_bw_rule, MIN_BANDWIDTH, self.direction)])
self._wait_for_min_bw_rule_applied(
vm, MIN_BANDWIDTH, self.direction)
# Delete port with qos policy attached
vm.destroy()
self._wait_for_min_bw_rule_removed(vm, self.direction)
self.assertIsNone(vm.bridge.find_qos(vm.port.name))
self.assertIsNone(vm.bridge.find_queue(vm.port.name,
ovs_lib.QOS_DEFAULT_QUEUE))

View File

@ -0,0 +1,337 @@
# 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.
import mock
from neutron_lib.services.qos import constants as qos_constants
from oslo_utils import uuidutils
import six
from neutron.agent.common import ovs_lib
from neutron.agent.linux import ip_lib
from neutron.plugins.ml2.drivers.openvswitch.agent.common \
import constants as ovs_constants
from neutron.tests.functional import base
class BaseOVSTestCase(base.BaseSudoTestCase):
def setUp(self):
super(BaseOVSTestCase, self).setUp()
self.br_name = ('br-' + uuidutils.generate_uuid())[:10]
self.port_id = ('port-' + uuidutils.generate_uuid())[:8]
self.ovs = ovs_lib.OVSBridge(self.br_name)
self.elements_to_clean = {'bridges': [], 'devices': [],
'qoses': [], 'queues': []}
self.addCleanup(self._clean_system)
def _clean_system(self):
# NOTE(ralonsoh): the deletion order is important. First we need to
# delete any bridge (and the ports attached); then the physical devices
# created. QoS registers can be deleted if no port has those rules
# assigned. Queues registers can be deleted if no QoS register refers
# to those Queues.
for bridge in self.elements_to_clean['bridges']:
self.ovs.ovsdb.del_br(bridge).execute()
for device in self.elements_to_clean['devices']:
ip_lib.IPDevice(device).link.delete()
for qos in self.elements_to_clean['qoses']:
self.ovs.ovsdb.db_destroy('QoS', qos).execute()
for queue in self.elements_to_clean['queues']:
self.ovs.ovsdb.db_destroy('Queue', queue).execute()
def _list_queues(self, queue_id=None):
queues = self.ovs.ovsdb.db_list(
'Queue',
columns=('_uuid', 'other_config', 'external_ids')).execute()
if queue_id:
for queue in (queue for queue in queues
if queue['_uuid'] == queue_id):
return queue
else:
return None
return queues
def _create_queue(self, max_kbps=3000, max_burst_kbps=2000, min_kbps=1000,
neutron_port_id=None, queue_num=None):
neutron_port_id = (('port-' + uuidutils.generate_uuid())[:13]
if not neutron_port_id else neutron_port_id)
queue_num = 'queue_num' if not queue_num else queue_num
queue_id = self.ovs._update_queue(neutron_port_id, queue_num,
max_kbps=max_kbps,
max_burst_kbps=max_burst_kbps,
min_kbps=min_kbps)
self.elements_to_clean['queues'].append(queue_id)
return queue_id, neutron_port_id
def _create_qos(self, qos_id=None, queues=None):
qos_id = self.ovs._update_qos(qos_id=qos_id, queues=queues)
self.elements_to_clean['qoses'].append(qos_id)
return qos_id
def _list_qos(self, qos_id=None):
qoses = self.ovs.ovsdb.db_list(
'QoS',
columns=('_uuid', 'queues', 'external_ids', 'type')).execute()
if qos_id:
for qos in (qos for qos in qoses if qos['_uuid'] == qos_id):
return qos
else:
return None
return qoses
def _create_bridge(self):
self.ovs.ovsdb.add_br(self.br_name).execute()
self.elements_to_clean['bridges'].append(self.br_name)
def _create_port(self, port_name):
self.ovs.ovsdb.add_port(self.br_name, port_name).execute()
def _find_port_uuid(self, port_name):
return self.ovs.ovsdb.db_get('Port', port_name, '_uuid').execute()
def _find_port_qos(self, port_name):
return self.ovs.ovsdb.db_get('Port', port_name, 'qos').execute()
def _create_dummy_device(self):
device_name = ('dummy-' + uuidutils.generate_uuid())[:12]
ip_lib.IPWrapper().add_dummy(device_name)
self.elements_to_clean['devices'].append(device_name)
return device_name
def test__update_queue_new(self):
queue_id, neutron_port_id = self._create_queue()
self.assertIsNotNone(queue_id)
other_config = {six.u('max-rate'): six.u('3000000'),
six.u('burst'): six.u('2000000'),
six.u('min-rate'): six.u('1000000')}
external_ids = {six.u('port'): six.u(neutron_port_id),
six.u('queue-num'): six.u('queue_num'),
six.u('type'):
six.u(qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH)}
queue = self._list_queues(queue_id)
self.assertIsNotNone(queue)
self.assertEqual(queue['_uuid'], queue_id)
self.assertEqual(other_config, queue['other_config'])
self.assertEqual(external_ids, queue['external_ids'])
def test__update_queue_update(self):
queue_id, neutron_port_id = self._create_queue()
self.assertIsNotNone(queue_id)
other_config = {six.u('max-rate'): six.u('6000000'),
six.u('burst'): six.u('5000000'),
six.u('min-rate'): six.u('4000000')}
external_ids = {six.u('port'): six.u(neutron_port_id),
six.u('queue-num'): six.u('queue_num'),
six.u('type'):
six.u(qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH)}
queue = self._list_queues(queue_id)
self.assertIsNotNone(queue)
queue_id, _ = self._create_queue(max_kbps=6000, max_burst_kbps=5000,
min_kbps=4000, queue_num=queue_id,
neutron_port_id=neutron_port_id)
self.assertIsNotNone(queue_id)
queue = self._list_queues(queue_id)
self.assertEqual(queue['_uuid'], queue_id)
self.assertEqual(other_config, queue['other_config'])
self.assertEqual(external_ids, queue['external_ids'])
def test__find_queue(self):
queue_id, neutron_port_id = self._create_queue()
queue_found = self.ovs._find_queue(neutron_port_id)
self.assertEqual(queue_id, queue_found['_uuid'])
def test__list_queues(self):
ports = []
queue_ids = []
for _ in range(5):
queue_id, neutron_port_id = self._create_queue()
queue_ids.append(queue_id)
ports.append(neutron_port_id)
for idx, port in enumerate(ports):
queue_list = self.ovs._list_queues(
port=port, _type=qos_constants.RULE_TYPE_MINIMUM_BANDWIDTH)
self.assertEqual(1, len(queue_list))
self.assertEqual(queue_ids[idx], queue_list[0]['_uuid'])
self.assertEqual(port, queue_list[0]['external_ids']['port'])
queue_list = self.ovs._list_queues(port=port, _type='other_type')
self.assertEqual(0, len(queue_list))
def test__delete_queue(self):
queue_id, _ = self._create_queue()
self.assertIsNotNone(self._list_queues(queue_id=queue_id))
self.ovs._delete_queue(queue_id)
self.assertIsNone(self._list_queues(queue_id=queue_id))
def test__update_qos_new(self):
queue_id, _ = self._create_queue()
queues = {1: queue_id}
qos_id = self._create_qos(queues=queues)
qos = self._list_qos(qos_id)
self.assertEqual(qos_id, qos['_uuid'])
self.assertEqual(queues[1], qos['queues'][1].uuid)
def test__update_qos_update(self):
queue_id_1, _ = self._create_queue()
queues = {1: queue_id_1}
qos_id = self._create_qos(queues=queues)
qos = self._list_qos(qos_id)
self.assertEqual(qos_id, qos['_uuid'])
self.assertEqual(1, len(qos['queues']))
self.assertEqual(queues[1], qos['queues'][1].uuid)
queue_id_2, _ = self._create_queue()
queues[2] = queue_id_2
self._create_qos(qos_id=qos_id, queues=queues)
qos = self._list_qos(qos_id)
self.assertEqual(qos_id, qos['_uuid'])
self.assertEqual(2, len(qos['queues']))
self.assertEqual(queues[1], qos['queues'][1].uuid)
self.assertEqual(queues[2], qos['queues'][2].uuid)
def test__find_qos(self):
queue_id, _ = self._create_queue()
queues = {1: queue_id}
qos_id = self._create_qos()
qos_ret, qos_queues = self.ovs._find_qos()
self.assertEqual(qos_id, qos_ret)
self.assertEqual(queues[1], queues[1])
def test__set_port_qos(self):
port_name = 'test_port'
self._create_bridge()
self._create_port(port_name)
port_qos = self._find_port_qos(port_name)
self.assertEqual([], port_qos)
qos_id = self._create_qos()
self.ovs._set_port_qos(port_name, qos_id=qos_id)
port_qos = self._find_port_qos(port_name)
self.assertEqual(qos_id, port_qos)
self.ovs._set_port_qos(port_name)
port_qos = self._find_port_qos(port_name)
self.assertEqual([], port_qos)
def test_get_bridge_ports(self):
self._create_bridge()
device_names = []
for _ in range(5):
device_name = self._create_dummy_device()
device_names.append(device_name)
self._create_port(device_name)
bridge_ports = self.ovs.get_bridge_ports('')
device_names.sort()
bridge_ports.sort()
self.assertEqual(device_names, bridge_ports)
def test__set_queue_for_minimum_bandwidth(self):
self._create_bridge()
self.ovs._set_queue_for_minimum_bandwidth(1234)
flows = self.ovs.dump_flows_for_table(ovs_constants.LOCAL_SWITCHING)
expected = 'priority=200,reg4=0,in_port=1234 actions=set_queue:1234,' \
'load:0x1->NXM_NX_REG4[0],resubmit(,0)'
self.assertIn(expected, flows)
def test__unset_queue_for_minimum_bandwidth(self):
self.test__set_queue_for_minimum_bandwidth()
self.ovs._unset_queue_for_minimum_bandwidth(1234)
flows = self.ovs.dump_flows_for_table(ovs_constants.LOCAL_SWITCHING)
expected = 'in_port=1234'
self.assertNotIn(expected, flows)
def test_update_minimum_bandwidth_queue(self):
port_name = 'test_output_port_1'
self._create_bridge()
self._create_port(port_name)
queue_num = 1
queue_id, _ = self._create_queue(neutron_port_id=self.port_id)
queues = {queue_num: queue_id}
qos_id = self._create_qos(queues=queues)
self.ovs.update_minimum_bandwidth_queue(self.port_id, [port_name],
queue_num, 1800)
port_qos = self._find_port_qos(port_name)
self.assertEqual(qos_id, port_qos)
queue = self._list_queues(queue_id)
self.assertEqual(six.u('1800000'), queue['other_config']['min-rate'])
def test_update_minimum_bandwidth_queue_no_qos_no_queue(self):
port_name = 'test_output_port_2'
self._create_bridge()
self._create_port(port_name)
queue_num = 1
self.ovs.update_minimum_bandwidth_queue(self.port_id, [port_name],
queue_num, 1700)
qos_id = self._find_port_qos(port_name)
qos = self._list_qos(qos_id)
queue_id = qos['queues'][1].uuid
queue = self._list_queues(queue_id)
self.elements_to_clean['qoses'].append(qos_id)
self.elements_to_clean['queues'].append(queue_id)
self.assertEqual(six.u('1700000'), queue['other_config']['min-rate'])
def test_delete_minimum_bandwidth_queue(self):
queue_id_1, neutron_port_id_1 = self._create_queue(queue_num=1)
queue_id_2, neutron_port_id_2 = self._create_queue(queue_num=2)
queues = {1: queue_id_1, 2: queue_id_2}
qos_id = self._create_qos(queues=queues)
qos = self._list_qos(qos_id)
self.assertEqual(queue_id_1, qos['queues'][1].uuid)
self.assertEqual(queue_id_2, qos['queues'][2].uuid)
self.ovs.delete_minimum_bandwidth_queue(neutron_port_id_2)
qos = self._list_qos(qos_id)
self.assertEqual(1, len(qos['queues']))
self.assertEqual(queue_id_1, qos['queues'][1].uuid)
self.ovs.delete_minimum_bandwidth_queue(neutron_port_id_1)
qos = self._list_qos(qos_id)
self.assertEqual(0, len(qos['queues']))
def test_clear_minimum_bandwidth_qos(self):
queue_id_1, port_id_1 = self._create_queue(queue_num=1)
queue_id_2, port_id_2 = self._create_queue(queue_num=2)
queue_id_3, _ = self._create_queue()
queues = {1: queue_id_1, 2: queue_id_2}
qos_id = self._create_qos(queues=queues)
# NOTE(ralonsoh): we need to clean only the QoS rule created in this
# test in order to avoid any interference with other tests.
qoses = self.ovs._list_qos(_id=self.ovs._min_bw_qos_id)
with mock.patch.object(self.ovs, '_list_qos') as mock_list_qos:
mock_list_qos.return_value = qoses
self.ovs.clear_minimum_bandwidth_qos()
self.assertIsNone(self._list_qos(qos_id=qos_id))
self.assertIsNone(self._list_queues(queue_id=queue_id_1))
self.assertIsNone(self._list_queues(queue_id=queue_id_2))
self.assertIsNotNone(self._list_queues(queue_id=queue_id_3))
def test_get_egress_min_bw_for_port(self):
self.ovs.update_minimum_bandwidth_queue(self.port_id, [], 1, 2800)
self.assertEqual(
2800,
self.ovs.get_egress_min_bw_for_port(port_id=self.port_id))

View File

@ -65,7 +65,8 @@ class LoggingExtensionTestFramework(test_firewall.BaseFirewallTestCase):
mock.patch('os_ken.base.app_manager.AppManager.get_instance').start()
agent_api = ovs_ext_api.OVSAgentExtensionAPI(
ovs_bridge.OVSAgentBridge(self.tester.bridge.br_name),
ovs_bridge.OVSAgentBridge('br-tun'))
ovs_bridge.OVSAgentBridge('br-tun'),
{'physnet1': ovs_bridge.OVSAgentBridge('br-physnet1')})
log_driver = ovs_fw_log.OVSFirewallLoggingDriver(agent_api)
log_driver.initialize(self.resource_rpc)
return log_driver

View File

@ -226,7 +226,8 @@ class QosExtensionBaseTestCase(base.BaseTestCase):
self.connection = mock.Mock()
self.agent_api = ovs_ext_api.OVSAgentExtensionAPI(
ovs_bridge.OVSAgentBridge('br-int'),
ovs_bridge.OVSAgentBridge('br-tun'))
ovs_bridge.OVSAgentBridge('br-tun'),
{'phynet1': ovs_bridge.OVSAgentBridge('br-phynet1')})
self.qos_ext.consume_api(self.agent_api)
# Don't rely on used driver

View File

@ -39,9 +39,12 @@ class QosOVSAgentDriverTestCase(ovs_test_base.OVSAgentConfigTestBase):
self.addCleanup(conn_patcher.stop)
self.context = context.get_admin_context()
self.qos_driver = qos_driver.QosOVSAgentDriver()
self.mock_clear_minimum_bandwidth_qos = mock.patch.object(
self.qos_driver, '_minimum_bandwidth_initialize').start()
self.agent_api = ovs_ext_api.OVSAgentExtensionAPI(
ovs_bridge.OVSAgentBridge('br-int'),
ovs_bridge.OVSAgentBridge('br-tun'))
ovs_bridge.OVSAgentBridge('br-tun'),
{'phys1': ovs_bridge.OVSAgentBridge('br-phys1')})
self.qos_driver.consume_api(self.agent_api)
self.qos_driver.initialize()
self.qos_driver.br_int = mock.Mock()
@ -189,3 +192,58 @@ class QosOVSAgentDriverTestCase(ovs_test_base.OVSAgentConfigTestBase):
self.qos_driver.br_int.add_flow.assert_called_once_with(
actions='mod_nw_tos:128,load:55->NXM_NX_REG2[0..5],resubmit(,0)',
in_port=mock.ANY, priority=65535, reg2=0, table=0)
def test_create_minimum_bandwidth(self):
with mock.patch.object(self.qos_driver, 'update_minimum_bandwidth') \
as mock_update_minimum_bandwidth:
self.qos_driver.create_minimum_bandwidth('port_name', 'rule')
mock_update_minimum_bandwidth.assert_called_once_with('port_name',
'rule')
def test_delete_minimum_bandwidth(self):
with mock.patch.object(self.qos_driver.br_int,
'delete_minimum_bandwidth_queue') \
as mock_delete_minimum_bandwidth_queue:
self.qos_driver.delete_minimum_bandwidth({'port_id': 'port_id'})
mock_delete_minimum_bandwidth_queue.assert_called_once_with(
'port_id')
def test_update_minimum_bandwidth_no_vif_port(self):
with mock.patch.object(self.qos_driver.br_int,
'update_minimum_bandwidth_queue') \
as mock_delete_minimum_bandwidth_queue:
self.qos_driver.update_minimum_bandwidth({}, mock.ANY)
mock_delete_minimum_bandwidth_queue.assert_not_called()
def test_update_minimum_bandwidth_no_phy_brs(self):
vif_port = mock.Mock()
vif_port.ofport = 'ofport'
rule = mock.Mock()
rule.min_kbps = 1500
port = {'port_id': 'port_id', 'vif_port': vif_port}
with mock.patch.object(self.qos_driver.br_int,
'update_minimum_bandwidth_queue') \
as mock_delete_minimum_bandwidth_queue, \
mock.patch.object(self.qos_driver.agent_api,
'request_phy_brs'):
self.qos_driver.update_minimum_bandwidth(port, rule)
mock_delete_minimum_bandwidth_queue.assert_called_once_with(
'port_id', [], 'ofport', 1500)
def test_update_minimum_bandwidth(self):
vif_port = mock.Mock()
vif_port.ofport = 'ofport'
rule = mock.Mock()
rule.min_kbps = 1500
port = {'port_id': 'port_id', 'vif_port': vif_port}
with mock.patch.object(self.qos_driver.br_int,
'update_minimum_bandwidth_queue') \
as mock_delete_minimum_bandwidth_queue, \
mock.patch.object(self.qos_driver.agent_api,
'request_phy_brs') as mock_request_phy_brs:
phy_br = mock.Mock()
phy_br.get_bridge_ports.return_value = ['port1', 'port2']
mock_request_phy_brs.return_value = [phy_br]
self.qos_driver.update_minimum_bandwidth(port, rule)
mock_delete_minimum_bandwidth_queue.assert_called_once_with(
'port_id', ['port1', 'port2'], 'ofport', 1500)

View File

@ -32,6 +32,8 @@ class TestOVSAgentExtensionAPI(ovs_test_base.OVSOFCtlTestBase):
super(TestOVSAgentExtensionAPI, self).setUp()
self.br_int = self.br_int_cls("br-int")
self.br_tun = self.br_tun_cls("br-tun")
self.br_phys = {'br-phys1': self.br_phys_cls('br-phys1'),
'br-phys2': self.br_phys_cls('br-phys2')}
def _test_bridge(self, orig_bridge, new_bridge):
self.assertIsNotNone(new_bridge)
@ -42,22 +44,30 @@ class TestOVSAgentExtensionAPI(ovs_test_base.OVSOFCtlTestBase):
new_bridge._default_cookie)
def test_request_int_br(self):
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(self.br_int,
self.br_tun)
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(
self.br_int, self.br_tun, {'phys': self.br_phys['br-phys1']})
new_int_br = agent_extension_api.request_int_br()
self._test_bridge(self.br_int, new_int_br)
def test_request_tun_br(self):
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(self.br_int,
self.br_tun)
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(
self.br_int, self.br_tun, {'phys': self.br_phys['br-phys1']})
new_tun_br = agent_extension_api.request_tun_br()
self._test_bridge(self.br_tun, new_tun_br)
def test_request_tun_br_tunneling_disabled(self):
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(self.br_int,
None)
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(
self.br_int, None, {'phys': self.br_phys['br-phys1']})
self.assertIsNone(agent_extension_api.request_tun_br())
def test_request_phys_brs(self):
agent_extension_api = ovs_ext_agt.OVSAgentExtensionAPI(
self.br_int, self.br_tun,
{'phys1': self.br_phys['br-phys1'],
'phys2': self.br_phys['br-phys2']})
for phys_br in agent_extension_api.request_phy_brs():
self._test_bridge(self.br_phys[phys_br.br_name], phys_br)
class TestOVSCookieBridgeOFCtl(ovs_test_base.OVSOFCtlTestBase):

View File

@ -56,7 +56,8 @@ class LoggingExtensionBaseTestCase(base.BaseTestCase):
self.connection = mock.Mock()
agent_api = ovs_ext_api.OVSAgentExtensionAPI(
ovs_bridge.OVSAgentBridge('br-int'),
ovs_bridge.OVSAgentBridge('br-tun'))
ovs_bridge.OVSAgentBridge('br-tun'),
{'physnet1': ovs_bridge.OVSAgentBridge('br-physnet1')})
self.agent_ext.consume_api(agent_api)
mock.patch(
'neutron.manager.NeutronManager.load_class_for_provider').start()