Merge "Initialize the network segment ranges only in first WSGI worker" into stable/2025.1
This commit is contained in:
@@ -20,6 +20,9 @@ from oslo_utils import timeutils
|
||||
from neutron.common import utils
|
||||
|
||||
|
||||
FIRST_WORKER_ID = 1
|
||||
|
||||
|
||||
def get_start_time(default=None, current_time=False):
|
||||
"""Return the 'start-time=%t' config varible in the WSGI config
|
||||
|
||||
|
||||
@@ -15,7 +15,6 @@
|
||||
|
||||
import functools
|
||||
|
||||
from neutron_lib import context
|
||||
from neutron_lib.db import api as db_api
|
||||
from neutron_lib import exceptions
|
||||
from neutron_lib.plugins import constants as plugin_constants
|
||||
@@ -165,8 +164,7 @@ class SegmentTypeDriver(BaseTypeDriver):
|
||||
context.elevated()):
|
||||
LOG.debug(' - %s', srange)
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def _delete_expired_default_network_segment_ranges(self, start_time):
|
||||
ns_range.NetworkSegmentRange.\
|
||||
delete_expired_default_network_segment_ranges(
|
||||
context.get_admin_context(), self.get_type(), start_time)
|
||||
def _delete_expired_default_network_segment_ranges(self, ctx, start_time):
|
||||
(ns_range.NetworkSegmentRange.
|
||||
delete_expired_default_network_segment_ranges(
|
||||
ctx, self.get_type(), start_time))
|
||||
|
||||
@@ -127,7 +127,7 @@ class _TunnelTypeDriverBase(helpers.SegmentTypeDriver, metaclass=abc.ABCMeta):
|
||||
# allocation during driver initialization, instead of using the
|
||||
# directory.get_plugin() method - the normal way used elsewhere to
|
||||
# check if a plugin is loaded.
|
||||
self.sync_allocations()
|
||||
self._sync_allocations()
|
||||
|
||||
def _parse_tunnel_ranges(self, tunnel_ranges, current_range):
|
||||
for entry in tunnel_ranges:
|
||||
@@ -145,17 +145,15 @@ class _TunnelTypeDriverBase(helpers.SegmentTypeDriver, metaclass=abc.ABCMeta):
|
||||
{'type': self.get_type(), 'range': current_range})
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def _populate_new_default_network_segment_ranges(self, start_time):
|
||||
ctx = context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(ctx):
|
||||
for tun_min, tun_max in self.tunnel_ranges:
|
||||
range_obj.NetworkSegmentRange.new_default(
|
||||
ctx, self.get_type(), None, tun_min, tun_max, start_time)
|
||||
def _populate_new_default_network_segment_ranges(self, ctx, start_time):
|
||||
for tun_min, tun_max in self.tunnel_ranges:
|
||||
range_obj.NetworkSegmentRange.new_default(
|
||||
ctx, self.get_type(), None, tun_min, tun_max, start_time)
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def _get_network_segment_ranges_from_db(self):
|
||||
def _get_network_segment_ranges_from_db(self, ctx=None):
|
||||
ranges = []
|
||||
ctx = context.get_admin_context()
|
||||
ctx = ctx or context.get_admin_context()
|
||||
with db_api.CONTEXT_READER.using(ctx):
|
||||
range_objs = (range_obj.NetworkSegmentRange.get_objects(
|
||||
ctx, network_type=self.get_type()))
|
||||
@@ -164,21 +162,27 @@ class _TunnelTypeDriverBase(helpers.SegmentTypeDriver, metaclass=abc.ABCMeta):
|
||||
|
||||
return ranges
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def initialize_network_segment_range_support(self, start_time):
|
||||
self._delete_expired_default_network_segment_ranges(start_time)
|
||||
self._populate_new_default_network_segment_ranges(start_time)
|
||||
# Override self.tunnel_ranges with the network segment range
|
||||
# information from DB and then do a sync_allocations since the
|
||||
# segment range service plugin has not yet been loaded at this
|
||||
# initialization time.
|
||||
self.tunnel_ranges = self._get_network_segment_ranges_from_db()
|
||||
self.sync_allocations()
|
||||
admin_context = context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(admin_context):
|
||||
self._delete_expired_default_network_segment_ranges(
|
||||
admin_context, start_time)
|
||||
self._populate_new_default_network_segment_ranges(
|
||||
admin_context, start_time)
|
||||
# Override self.tunnel_ranges with the network segment range
|
||||
# information from DB and then do a sync_allocations since the
|
||||
# segment range service plugin has not yet been loaded at this
|
||||
# initialization time.
|
||||
self.tunnel_ranges = self._get_network_segment_ranges_from_db(
|
||||
ctx=admin_context)
|
||||
self._sync_allocations(ctx=admin_context)
|
||||
|
||||
def update_network_segment_range_allocations(self):
|
||||
self.sync_allocations()
|
||||
self._sync_allocations()
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def sync_allocations(self):
|
||||
def _sync_allocations(self, ctx=None):
|
||||
# determine current configured allocatable tunnel ids
|
||||
tunnel_ids = set()
|
||||
ranges = self.get_network_segment_ranges()
|
||||
@@ -187,7 +191,7 @@ class _TunnelTypeDriverBase(helpers.SegmentTypeDriver, metaclass=abc.ABCMeta):
|
||||
|
||||
tunnel_id_getter = operator.attrgetter(self.segmentation_key)
|
||||
tunnel_col = getattr(self.model, self.segmentation_key)
|
||||
ctx = context.get_admin_context()
|
||||
ctx = ctx or context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(ctx):
|
||||
# Check if the allocations are updated: if the total number of
|
||||
# allocations for this tunnel type matches the allocations of the
|
||||
|
||||
@@ -56,16 +56,13 @@ class VlanTypeDriver(helpers.SegmentTypeDriver):
|
||||
self.model_segmentation_id = vlan_alloc_model.VlanAllocation.vlan_id
|
||||
self._parse_network_vlan_ranges()
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def _populate_new_default_network_segment_ranges(self, start_time):
|
||||
ctx = context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(ctx):
|
||||
for (physical_network, vlan_ranges) in (
|
||||
self.network_vlan_ranges.items()):
|
||||
for vlan_min, vlan_max in vlan_ranges:
|
||||
range_obj.NetworkSegmentRange.new_default(
|
||||
ctx, self.get_type(), physical_network, vlan_min,
|
||||
vlan_max, start_time)
|
||||
def _populate_new_default_network_segment_ranges(self, ctx, start_time):
|
||||
for (physical_network, vlan_ranges) in (
|
||||
self.network_vlan_ranges.items()):
|
||||
for vlan_min, vlan_max in vlan_ranges:
|
||||
range_obj.NetworkSegmentRange.new_default(
|
||||
ctx, self.get_type(), physical_network, vlan_min,
|
||||
vlan_max, start_time)
|
||||
|
||||
def _parse_network_vlan_ranges(self):
|
||||
try:
|
||||
@@ -78,8 +75,8 @@ class VlanTypeDriver(helpers.SegmentTypeDriver):
|
||||
LOG.info("Network VLAN ranges: %s", self.network_vlan_ranges)
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def _sync_vlan_allocations(self):
|
||||
ctx = context.get_admin_context()
|
||||
def _sync_vlan_allocations(self, ctx=None):
|
||||
ctx = ctx or context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(ctx):
|
||||
# VLAN ranges per physical network:
|
||||
# {phy1: [(1, 10), (30, 50)], ...}
|
||||
@@ -142,9 +139,9 @@ class VlanTypeDriver(helpers.SegmentTypeDriver):
|
||||
vlan_ids)
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def _get_network_segment_ranges_from_db(self):
|
||||
def _get_network_segment_ranges_from_db(self, ctx=None):
|
||||
ranges = {}
|
||||
ctx = context.get_admin_context()
|
||||
ctx = ctx or context.get_admin_context()
|
||||
with db_api.CONTEXT_READER.using(ctx):
|
||||
range_objs = (range_obj.NetworkSegmentRange.get_objects(
|
||||
ctx, network_type=self.get_type()))
|
||||
@@ -171,15 +168,21 @@ class VlanTypeDriver(helpers.SegmentTypeDriver):
|
||||
self._sync_vlan_allocations()
|
||||
LOG.info("VlanTypeDriver initialization complete")
|
||||
|
||||
@db_api.retry_db_errors
|
||||
def initialize_network_segment_range_support(self, start_time):
|
||||
self._delete_expired_default_network_segment_ranges(start_time)
|
||||
self._populate_new_default_network_segment_ranges(start_time)
|
||||
# Override self.network_vlan_ranges with the network segment range
|
||||
# information from DB and then do a sync_allocations since the
|
||||
# segment range service plugin has not yet been loaded at this
|
||||
# initialization time.
|
||||
self.network_vlan_ranges = self._get_network_segment_ranges_from_db()
|
||||
self._sync_vlan_allocations()
|
||||
admin_context = context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(admin_context):
|
||||
self._delete_expired_default_network_segment_ranges(
|
||||
admin_context, start_time)
|
||||
self._populate_new_default_network_segment_ranges(
|
||||
admin_context, start_time)
|
||||
# Override self.network_vlan_ranges with the network segment range
|
||||
# information from DB and then do a sync_allocations since the
|
||||
# segment range service plugin has not yet been loaded at this
|
||||
# initialization time.
|
||||
self.network_vlan_ranges = (
|
||||
self._get_network_segment_ranges_from_db(ctx=admin_context))
|
||||
self._sync_vlan_allocations(ctx=admin_context)
|
||||
|
||||
def update_network_segment_range_allocations(self):
|
||||
self._sync_vlan_allocations()
|
||||
|
||||
@@ -32,6 +32,7 @@ from oslo_utils import excutils
|
||||
import stevedore
|
||||
|
||||
from neutron._i18n import _
|
||||
from neutron.common import wsgi_utils
|
||||
from neutron.conf.plugins.ml2 import config
|
||||
from neutron.db import segments_db
|
||||
from neutron.objects import ports
|
||||
@@ -205,6 +206,9 @@ class TypeManager(stevedore.named.NamedExtensionManager):
|
||||
driver.obj.initialize()
|
||||
|
||||
def initialize_network_segment_range_support(self, start_time):
|
||||
if wsgi_utils.get_api_worker_id() != wsgi_utils.FIRST_WORKER_ID:
|
||||
return
|
||||
|
||||
for network_type, driver in self.drivers.items():
|
||||
if network_type in constants.NETWORK_SEGMENT_RANGE_TYPES:
|
||||
LOG.info("Initializing driver network segment range support "
|
||||
|
||||
@@ -0,0 +1,92 @@
|
||||
# Copyright 2025 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 concurrent import futures
|
||||
import time
|
||||
|
||||
from neutron_lib import constants
|
||||
from neutron_lib import context
|
||||
from neutron_lib.db import api as db_api
|
||||
from oslo_config import cfg
|
||||
|
||||
from neutron.conf import common as common_config
|
||||
from neutron.conf.plugins.ml2 import config as ml2_config
|
||||
from neutron.conf.plugins.ml2.drivers import driver_type as driver_type_config
|
||||
from neutron.objects import network_segment_range as range_obj
|
||||
from neutron.plugins.ml2.drivers import type_geneve
|
||||
from neutron.tests.unit import testlib_api
|
||||
|
||||
|
||||
def _initialize_network_segment_range_support(type_driver, start_time):
|
||||
# This method is similar to
|
||||
# ``_TunnelTypeDriverBase.initialize_network_segment_range_support``.
|
||||
# The method first deletes the existing default network ranges and then
|
||||
# creates the new ones. It also adds an extra second before closing the
|
||||
# DB transaction.
|
||||
admin_context = context.get_admin_context()
|
||||
with db_api.CONTEXT_WRITER.using(admin_context):
|
||||
type_driver._delete_expired_default_network_segment_ranges(
|
||||
admin_context, start_time)
|
||||
type_driver._populate_new_default_network_segment_ranges(
|
||||
admin_context, start_time)
|
||||
time.sleep(1)
|
||||
|
||||
|
||||
class TunnelTypeDriverBaseTestCase(testlib_api.SqlTestCase):
|
||||
def setUp(self):
|
||||
super().setUp()
|
||||
cfg.CONF.register_opts(common_config.core_opts)
|
||||
ml2_config.register_ml2_plugin_opts()
|
||||
driver_type_config.register_ml2_drivers_geneve_opts()
|
||||
ml2_config.cfg.CONF.set_override(
|
||||
'service_plugins', 'network_segment_range')
|
||||
self.min = 1001
|
||||
self.max = 1020
|
||||
self.net_type = constants.TYPE_GENEVE
|
||||
ml2_config.cfg.CONF.set_override(
|
||||
'vni_ranges', f'{self.min}:{self.max}', group='ml2_type_geneve')
|
||||
self.admin_ctx = context.get_admin_context()
|
||||
self.type_driver = type_geneve.GeneveTypeDriver()
|
||||
self.type_driver.initialize()
|
||||
|
||||
def test_initialize_network_segment_range_support(self):
|
||||
# Execute the initialization several times with different start times.
|
||||
for start_time in range(3):
|
||||
self.type_driver.initialize_network_segment_range_support(
|
||||
start_time)
|
||||
sranges = range_obj.NetworkSegmentRange.get_objects(self.admin_ctx)
|
||||
self.assertEqual(1, len(sranges))
|
||||
self.assertEqual(self.net_type, sranges[0].network_type)
|
||||
self.assertEqual(self.min, sranges[0].minimum)
|
||||
self.assertEqual(self.max, sranges[0].maximum)
|
||||
self.assertEqual([(self.min, self.max)],
|
||||
self.type_driver.tunnel_ranges)
|
||||
|
||||
def test_initialize_network_segment_range_support_parallel_execution(self):
|
||||
max_workers = 3
|
||||
_futures = []
|
||||
with futures.ThreadPoolExecutor(max_workers=max_workers) as executor:
|
||||
for idx in range(max_workers):
|
||||
_futures.append(executor.submit(
|
||||
_initialize_network_segment_range_support,
|
||||
self.type_driver, idx))
|
||||
for _future in _futures:
|
||||
_future.result()
|
||||
|
||||
sranges = range_obj.NetworkSegmentRange.get_objects(self.admin_ctx)
|
||||
self.assertEqual(1, len(sranges))
|
||||
self.assertEqual(self.net_type, sranges[0].network_type)
|
||||
self.assertEqual(self.min, sranges[0].minimum)
|
||||
self.assertEqual(self.max, sranges[0].maximum)
|
||||
@@ -49,7 +49,7 @@ class TunnelTypeTestMixin:
|
||||
super().setUp()
|
||||
self.driver = self.DRIVER_CLASS()
|
||||
self.driver.tunnel_ranges = TUNNEL_RANGES
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
self.context = context.Context()
|
||||
|
||||
def test_tunnel_type(self):
|
||||
@@ -84,7 +84,7 @@ class TunnelTypeTestMixin:
|
||||
self.driver.get_allocation(self.context, (TUN_MAX + 1)))
|
||||
|
||||
self.driver.tunnel_ranges = UPDATED_TUNNEL_RANGES
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
|
||||
self.assertIsNone(
|
||||
self.driver.get_allocation(self.context, (TUN_MIN + 5 - 1)))
|
||||
@@ -108,7 +108,7 @@ class TunnelTypeTestMixin:
|
||||
self.driver.reserve_provider_segment(self.context, segment)
|
||||
|
||||
self.driver.tunnel_ranges = UPDATED_TUNNEL_RANGES
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
|
||||
self.assertTrue(
|
||||
self.driver.get_allocation(self.context, tunnel_id).allocated)
|
||||
@@ -127,7 +127,7 @@ class TunnelTypeTestMixin:
|
||||
return []
|
||||
with mock.patch.object(
|
||||
type_tunnel, 'chunks', side_effect=verify_no_chunk) as chunks:
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
# No writing operation is done, fast exit: current allocations
|
||||
# already present.
|
||||
self.assertEqual(0, len(chunks.mock_calls))
|
||||
@@ -295,7 +295,7 @@ class TunnelTypeMultiRangeTestMixin:
|
||||
super().setUp()
|
||||
self.driver = self.DRIVER_CLASS()
|
||||
self.driver.tunnel_ranges = self.TUNNEL_MULTI_RANGES
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
self.context = context.Context()
|
||||
|
||||
def test_release_segment(self):
|
||||
@@ -486,7 +486,7 @@ class TunnelTypeNetworkSegmentRangeTestMixin:
|
||||
# one of the `service_plugins`
|
||||
self.driver._initialize(RAW_TUNNEL_RANGES)
|
||||
self.driver.initialize_network_segment_range_support(self.start_time)
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
ret = obj_network_segment_range.NetworkSegmentRange.get_objects(
|
||||
self.context)
|
||||
self.assertEqual(1, len(ret))
|
||||
@@ -502,9 +502,9 @@ class TunnelTypeNetworkSegmentRangeTestMixin:
|
||||
|
||||
def test__delete_expired_default_network_segment_ranges(self):
|
||||
self.driver.tunnel_ranges = TUNNEL_RANGES
|
||||
self.driver.sync_allocations()
|
||||
self.driver._sync_allocations()
|
||||
self.driver._delete_expired_default_network_segment_ranges(
|
||||
self.start_time)
|
||||
self.context, self.start_time)
|
||||
ret = obj_network_segment_range.NetworkSegmentRange.get_objects(
|
||||
self.context, network_type=self.driver.get_type())
|
||||
self.assertEqual(0, len(ret))
|
||||
|
||||
@@ -25,11 +25,14 @@ from neutron_lib.plugins import utils as plugin_utils
|
||||
from oslo_config import cfg
|
||||
from testtools import matchers
|
||||
|
||||
from neutron.common import wsgi_utils
|
||||
from neutron.conf.plugins.ml2 import config as ml2_config
|
||||
from neutron.objects import network_segment_range as obj_network_segment_range
|
||||
from neutron.objects.plugins.ml2 import vlanallocation as vlan_alloc_obj
|
||||
from neutron.plugins.ml2.drivers import type_vlan
|
||||
from neutron.tests.unit import testlib_api
|
||||
|
||||
|
||||
PROVIDER_NET = 'phys_net1'
|
||||
TENANT_NET = 'phys_net2'
|
||||
UNCONFIGURED_NET = 'no_net'
|
||||
@@ -361,6 +364,9 @@ class VlanTypeAllocationTest(testlib_api.SqlTestCase):
|
||||
class VlanTypeTestWithNetworkSegmentRange(testlib_api.SqlTestCase):
|
||||
|
||||
def setUp(self):
|
||||
ml2_config.register_ml2_plugin_opts()
|
||||
mock.patch.object(wsgi_utils, 'get_api_worker_id',
|
||||
return_value=wsgi_utils.FIRST_WORKER_ID).start()
|
||||
super().setUp()
|
||||
cfg.CONF.set_override('network_vlan_ranges',
|
||||
NETWORK_VLAN_RANGES,
|
||||
@@ -400,7 +406,7 @@ class VlanTypeTestWithNetworkSegmentRange(testlib_api.SqlTestCase):
|
||||
|
||||
def test__delete_expired_default_network_segment_ranges(self):
|
||||
self.driver._delete_expired_default_network_segment_ranges(
|
||||
self.start_time)
|
||||
self.context, self.start_time)
|
||||
ret = obj_network_segment_range.NetworkSegmentRange.get_objects(
|
||||
self.context, network_type=self.driver.get_type())
|
||||
self.assertEqual(0, len(ret))
|
||||
|
||||
Reference in New Issue
Block a user