Add inspection PXE filter service

The logic to handle dnsmasq hostfiles is moved from ironic-inspector
with only cosmetic changes. The logic to purge the hostsdir is not
copied since it relies on running commands with root privileges.
A documentation example is added instead.

The change is missing the RPC call to notify the filter about changes.
It will be done in a follow-up.

Change-Id: Ie32018c760c39873ead1da54cfaeae87eaaaf043
This commit is contained in:
Dmitry Tantsur 2024-02-06 12:17:21 +01:00
parent a9397f49d5
commit 89fe0396af
No known key found for this signature in database
GPG Key ID: 315B2AF9FD216C60
13 changed files with 925 additions and 7 deletions

View File

@ -27,6 +27,7 @@ ironic-inspector_ service.
data
hooks
discovery
pxe_filter
Configuration
-------------

View File

@ -0,0 +1,81 @@
PXE filter service
==================
The PXE filter service is responsible for managing the dnsmasq instance
that is responsible for :ref:`unmanaged-inspection`. Running it allows
this dnsmasq instance to co-exist with the OpenStack Networking service's DHCP
server on the same physical network.
.. warning::
The PXE filter service is currently experimental. For a production grade
solution, please stay with ironic-inspector for the time being.
How it works?
-------------
At the core of the PXE filter service is a periodic task that fetches all ports
and compares the node ID's with the ID's of the nodes undergoing in-band
inspection. All of the MAC addresses are added to the dnsmasq host files:
to the allowlist of nodes on inspection and to the denylist for the rest.
Additionally, when any nodes are on inspection, unknown MACs are also allowed.
Otherwise, access from unknown MACs to the dnsmasq service is denied.
Installation
------------
Start with :ref:`configure-unmanaged-inspection`. Then create a *hostsdir*
writable by the PXE filter service and readable by dnsmasq. Configure it in the
dnsmasq configuration file
.. code-block:: ini
dhcp-hostsdir=/var/lib/ironic/hostsdir
and in the Bare Metal service configuration
.. code-block:: ini
[pxe_filter]
dhcp_hostsdir = /var/lib/ironic/hostsdir
Then create a systemd service to start ``ironic-pxe-filter`` alongside dnsmasq,
e.g.
.. code-block:: ini
[Unit]
Description=Ironic PXE filter
[Service]
Type=notify
Restart=on-failure
ExecStart=/usr/bin/ironic-pxe-filter --config-file /etc/ironic/ironic.conf
User=ironic
Group=ironic
Note that because of technical limitations, the PXE filter process cannot clean
up the *hostsdir* itself. You may want to do it on the service start-up, e.g.
like this (assuming the dnsmasq service is ``ironic-dnsmasq`` and its PID is
stored in ``/run/ironic/dnsmasq.pid``):
.. code-block:: ini
[Unit]
Description=Ironic PXE filter
Requires=ironic-dnsmasq.service
After=ironic-dnsmasq.service
[Service]
Type=notify
Restart=on-failure
ExecStartPre=+/bin/bash -c "rm -f /usr/lib/ironic/hostsdir/* && kill -HUP $(cat /run/ironic/dnsmasq.pid) || true"
ExecStart=/usr/bin/ironic-pxe-filter --config-file /etc/ironic/ironic.conf
User=ironic
Group=ironic
Scale considerations
--------------------
The PXE filter service should be run once per each dnsmasq instance dedicated
to unmanaged inspection. In most clouds, that will be 1 instance.

View File

@ -535,13 +535,10 @@ Networking service won't be able to handle them. For instance, you can install
dhcp-boot=pxelinux.0
dhcp-sequential-ip
.. warning::
Ironic currently lacks `PXE filters
<https://docs.openstack.org/ironic-inspector/latest/admin/dnsmasq-pxe-filter.html>`_
used by ironic-inspector to allow its DHCP server to co-exist with
OpenStack Networking (neutron) on the same network. Unless you can
physically isolation the inspection network, you may want to stay with
ironic-inspector for the time being.
If you need this dnsmasq instance to co-exist with the OpenStack Networking
service, some measures must be taken to prevent them from clashing over DHCP
requests. One way to do it is to physically separate the inspection network.
Another - to configure the :doc:`/admin/inspection/pxe_filter`.
Finally, build or download IPA images into
``/tftpboot/ironic-python-agent.kernel`` and

73
ironic/cmd/pxe_filter.py Normal file
View File

@ -0,0 +1,73 @@
# 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 sys
from oslo_config import cfg
from oslo_log import log
from oslo_service import service
from ironic.common import rpc_service
from ironic.common import service as ironic_service
CONF = cfg.CONF
LOG = log.getLogger(__name__)
class RPCService(rpc_service.BaseRPCService):
def stop(self):
try:
self.manager.del_host()
except Exception as e:
LOG.exception('Service error occurred when cleaning up '
'the RPC manager. Error: %s', e)
try:
if self.rpcserver is not None:
self.rpcserver.stop()
self.rpcserver.wait()
except Exception as e:
LOG.exception('Service error occurred when stopping the '
'RPC server. Error: %s', e)
super().stop(graceful=True)
LOG.info('Stopped RPC server for service %(service)s on host '
'%(host)s.',
{'service': self.topic, 'host': self.host})
def main():
assert 'ironic.pxe_filter.service' not in sys.modules
# Parse config file and command line options, then start logging
ironic_service.prepare_service('ironic_pxe_filter', sys.argv)
if CONF.rpc_transport == 'json-rpc':
raise RuntimeError("This service is not designed to work with "
"rpc_transport = json-rpc. Please use another "
"RPC transport.")
mgr = RPCService(
CONF.host, 'ironic.pxe_filter.service', 'PXEFilterManager')
launcher = service.launch(CONF, mgr, restart_method='mutate')
# NOTE(dtantsur): handling start-up failures before launcher.wait() helps
# notify systemd about them. Otherwise the launcher will report successful
# service start-up before checking the threads.
mgr.wait_for_start()
sys.exit(launcher.wait())
if __name__ == '__main__':
sys.exit(main())

View File

@ -157,10 +157,28 @@ discovery_opts = [
"Must be set when enabling auto-discovery.")),
]
pxe_filter_opts = [
cfg.StrOpt('dhcp_hostsdir',
help=_('The MAC address cache directory, exposed to dnsmasq.'
'This directory is expected to be in exclusive control '
'of the driver but must be purged by the operator. '
'Required.')),
cfg.ListOpt('supported_inspect_interfaces',
default=['agent'], mutable=True,
help=_("List of inspect interfaces that will be considered "
"by the PXE filter. Only nodes with these interfaces "
"will be enabled.")),
cfg.IntOpt('sync_period',
default=45, mutable=True,
help=_("Period (in seconds) between synchronizing the state "
"if dnsmasq with the database.")),
]
def register_opts(conf):
conf.register_opts(opts, group='inspector')
conf.register_opts(discovery_opts, group='auto_discovery')
conf.register_opts(pxe_filter_opts, group='pxe_filter')
auth.register_auth_opts(conf, 'inspector',
service_type='baremetal-introspection')

View File

View File

@ -0,0 +1,215 @@
# 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 fcntl
import os
import time
from oslo_log import log
from ironic.conf import CONF
LOG = log.getLogger(__name__)
def update(allow_macs, deny_macs, allow_unknown=None):
"""Update only the given MACs.
MACs not in either lists are ignored.
:param allow_macs: MACs to allow in dnsmasq.
:param deny_macs: MACs to disallow in dnsmasq.
:param allow_unknown: If set to True, unknown MACs are also allowed.
Setting it to False does nothing in this call.
"""
for mac in allow_macs:
_add_mac_to_allowlist(mac)
for mac in deny_macs:
_add_mac_to_denylist(mac)
if allow_unknown:
_configure_unknown_hosts(True)
def sync(allow_macs, deny_macs, allow_unknown):
"""Conduct a complete sync of the state.
Unlike ``update``, MACs not in either list are handled according
to ``allow_unknown``.
:param allow_macs: MACs to allow in dnsmasq.
:param deny_macs: MACs to disallow in dnsmasq.
:param allow_unknown: Whether to allow access to dnsmasq to unknown
MACs.
"""
allow_macs = set(allow_macs)
deny_macs = set(deny_macs)
known_macs = allow_macs.union(deny_macs)
current_denylist, current_allowlist = _get_deny_allow_lists()
removed_macs = current_denylist.union(current_allowlist).difference(
known_macs)
update(allow_macs=allow_macs.difference(current_allowlist),
deny_macs=deny_macs.difference(current_denylist))
# Allow or deny unknown hosts and MACs not kept in ironic
# NOTE(hjensas): Treat unknown hosts and MACs not kept in ironic the
# same. Neither should boot the inspection image unless inspection
# is active. Deleted MACs must be added to the allow list when
# inspection is active in case the host is re-enrolled.
_configure_unknown_hosts(allow_unknown)
_configure_removedlist(removed_macs, allow_unknown)
_EXCLUSIVE_WRITE_ATTEMPTS = 10
_EXCLUSIVE_WRITE_ATTEMPTS_DELAY = 0.01
_MAC_DENY_LEN = len('ff:ff:ff:ff:ff:ff,ignore\n')
_MAC_ALLOW_LEN = len('ff:ff:ff:ff:ff:ff\n')
_UNKNOWN_HOSTS_FILE = 'unknown_hosts_filter'
_DENY_UNKNOWN_HOSTS = '*:*:*:*:*:*,ignore\n'
_ALLOW_UNKNOWN_HOSTS = '*:*:*:*:*:*\n'
def _get_deny_allow_lists():
"""Get addresses currently denied by dnsmasq.
:raises: FileNotFoundError in case the dhcp_hostsdir is invalid.
:returns: tuple with 2 elements: a set of MACs currently denied by dnsmasq
and a set of allowed MACs.
"""
hostsdir = CONF.pxe_filter.dhcp_hostsdir
# MACs in the allow list lack the ,ignore directive
denylist = set()
allowlist = set()
for mac in os.listdir(hostsdir):
if os.stat(os.path.join(hostsdir, mac)).st_size == _MAC_DENY_LEN:
denylist.add(mac)
if os.stat(os.path.join(hostsdir, mac)).st_size == _MAC_ALLOW_LEN:
allowlist.add(mac)
return denylist, allowlist
def _exclusive_write_or_pass(path, buf):
"""Write exclusively or pass if path locked.
The intention is to be able to run multiple instances of the filter on the
same node in multiple inspector processes.
:param path: where to write to
:param buf: the content to write
:raises: FileNotFoundError, IOError
:returns: True if the write was successful.
"""
# NOTE(milan) line-buffering enforced to ensure dnsmasq record update
# through inotify, which reacts on f.close()
with open(path, 'w', 1) as f:
for attempt in range(_EXCLUSIVE_WRITE_ATTEMPTS):
try:
fcntl.flock(f, fcntl.LOCK_EX | fcntl.LOCK_NB)
f.write(buf)
# Go ahead and flush the data now instead of waiting until
# after the automatic flush with the file close after the
# file lock is released.
f.flush()
return True
except BlockingIOError:
LOG.debug('%s locked; will try again (later)', path)
time.sleep(_EXCLUSIVE_WRITE_ATTEMPTS_DELAY)
continue
finally:
fcntl.flock(f, fcntl.LOCK_UN)
LOG.debug('Failed to write the exclusively-locked path: %(path)s for '
'%(attempts)s times', {'attempts': _EXCLUSIVE_WRITE_ATTEMPTS,
'path': path})
return False
def _configure_removedlist(macs, allowed):
"""Manages a dhcp_hostsdir allow/deny record for removed macs
:raises: FileNotFoundError in case the dhcp_hostsdir is invalid,
:returns: None.
"""
hostsdir = CONF.pxe_filter.dhcp_hostsdir
for mac in macs:
file_size = os.stat(os.path.join(hostsdir, mac)).st_size
if allowed:
if file_size != _MAC_ALLOW_LEN:
_add_mac_to_allowlist(mac)
else:
if file_size != _MAC_DENY_LEN:
_add_mac_to_denylist(mac)
def _configure_unknown_hosts(enabled):
"""Manages a dhcp_hostsdir allow/deny record for unknown macs.
:raises: FileNotFoundError in case the dhcp_hostsdir is invalid,
IOError in case the dhcp host unknown file isn't writable.
:returns: None.
"""
path = os.path.join(CONF.pxe_filter.dhcp_hostsdir, _UNKNOWN_HOSTS_FILE)
if enabled:
wildcard_filter = _ALLOW_UNKNOWN_HOSTS
log_wildcard_filter = 'allow'
else:
wildcard_filter = _DENY_UNKNOWN_HOSTS
log_wildcard_filter = 'deny'
# Don't update if unknown hosts are already in the deny/allow-list
try:
if os.stat(path).st_size == len(wildcard_filter):
return
except FileNotFoundError:
pass
if _exclusive_write_or_pass(path, '%s' % wildcard_filter):
LOG.debug('A %s record for all unknown hosts using wildcard mac '
'created', log_wildcard_filter)
else:
LOG.warning('Failed to %s unknown hosts using wildcard mac; '
'retrying next periodic sync time', log_wildcard_filter)
def _add_mac_to_denylist(mac):
"""Creates a dhcp_hostsdir deny record for the MAC.
:raises: FileNotFoundError in case the dhcp_hostsdir is invalid,
IOError in case the dhcp host MAC file isn't writable.
:returns: None.
"""
path = os.path.join(CONF.pxe_filter.dhcp_hostsdir, mac)
if _exclusive_write_or_pass(path, '%s,ignore\n' % mac):
LOG.debug('MAC %s added to the deny list', mac)
else:
LOG.warning('Failed to add MAC %s to the deny list; retrying next '
'periodic sync time', mac)
def _add_mac_to_allowlist(mac):
"""Update the dhcp_hostsdir record for the MAC adding it to allow list
:raises: FileNotFoundError in case the dhcp_hostsdir is invalid,
IOError in case the dhcp host MAC file isn't writable.
:returns: None.
"""
path = os.path.join(CONF.pxe_filter.dhcp_hostsdir, mac)
# remove the ,ignore directive
if _exclusive_write_or_pass(path, '%s\n' % mac):
LOG.debug('MAC %s removed from the deny list', mac)
else:
LOG.warning('Failed to remove MAC %s from the deny list; retrying '
'next periodic sync time', mac)

View File

@ -0,0 +1,104 @@
# 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 os
import time
import eventlet
from eventlet import event
from ironic_lib import metrics_utils
from oslo_log import log
from ironic.common.i18n import _
from ironic.common import states
from ironic.conf import CONF
from ironic.db import api as dbapi
from ironic.pxe_filter import dnsmasq
LOG = log.getLogger(__name__)
METRICS = metrics_utils.get_metrics_logger(__name__)
_START_DELAY = 1.0
class PXEFilterManager:
topic = 'ironic.pxe_filter'
def __init__(self, host):
self.host = host or CONF.host
self._started = False
def prepare_host(self):
if not CONF.pxe_filter.dhcp_hostsdir:
raise RuntimeError(_('The [pxe_filter]dhcp_hostsdir option '
'is required'))
if not os.path.isdir(CONF.pxe_filter.dhcp_hostsdir):
# FIXME(dtantsur): should we try to create it? The permissions will
# most likely be wrong.
raise RuntimeError(_('The path in [pxe_filter]dhcp_hostsdir '
'does not exist'))
def init_host(self, admin_context):
if self._started:
raise RuntimeError(_('Attempt to start an already running '
'PXE filter manager'))
self._shutdown = event.Event()
self._thread = eventlet.spawn_after(_START_DELAY, self._periodic_sync)
self._started = True
def del_host(self):
self._shutdown.send(True)
eventlet.sleep(0)
self._thread.wait()
self._started = False
def _periodic_sync(self):
db = dbapi.get_instance()
self._try_sync(db)
while not self._shutdown.wait(timeout=CONF.pxe_filter.sync_period):
self._try_sync(db)
def _try_sync(self, db):
try:
return self._sync(db)
except Exception:
LOG.exception('Sync failed, will retry')
@METRICS.timer('PXEFilterManager._sync')
def _sync(self, db):
LOG.debug('Starting periodic sync of the filter')
ts = time.time()
nodeinfo_list = db.get_nodeinfo_list(
columns=['id', 'inspect_interface'],
filters={
'provision_state_in': [states.INSPECTWAIT, states.INSPECTING],
})
nodes_on_inspection = {
node[0] for node in nodeinfo_list
if node[1] in CONF.pxe_filter.supported_inspect_interfaces
}
all_ports = db.get_port_list()
LOG.debug("Found %d nodes on inspection, handling %d ports",
len(nodes_on_inspection), len(all_ports))
allow = [port.address for port in all_ports
if port.node_id in nodes_on_inspection]
deny = [port.address for port in all_ports
if port.node_id not in nodes_on_inspection]
allow_unknown = (CONF.auto_discovery.enabled
or bool(nodes_on_inspection))
dnsmasq.sync(allow, deny, allow_unknown)
LOG.info('Finished periodic sync of the filter, took %.2f seconds',
time.time() - ts)

View File

View File

@ -0,0 +1,293 @@
# 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 builtins
import errno
import os
from unittest import mock
import fixtures
from ironic.conf import CONF
from ironic.pxe_filter import dnsmasq
from ironic.tests import base as test_base
class TestExclusiveWriteOrPass(test_base.TestCase):
def setUp(self):
super().setUp()
self.mock_open = self.useFixture(fixtures.MockPatchObject(
builtins, 'open', new=mock.mock_open())).mock
self.mock_fd = self.mock_open.return_value
self.mock_fcntl = self.useFixture(fixtures.MockPatchObject(
dnsmasq.fcntl, 'flock', autospec=True)).mock
self.path = '/foo/bar/baz'
self.buf = 'spam'
self.fcntl_lock_call = mock.call(
self.mock_fd, dnsmasq.fcntl.LOCK_EX | dnsmasq.fcntl.LOCK_NB)
self.fcntl_unlock_call = mock.call(self.mock_fd, dnsmasq.fcntl.LOCK_UN)
self.mock_log = self.useFixture(fixtures.MockPatchObject(
dnsmasq.LOG, 'debug')).mock
self.mock_sleep = self.useFixture(fixtures.MockPatchObject(
dnsmasq.time, 'sleep')).mock
def test_write(self):
wrote = dnsmasq._exclusive_write_or_pass(self.path, self.buf)
self.assertTrue(wrote)
self.mock_open.assert_called_once_with(self.path, 'w', 1)
self.mock_fcntl.assert_has_calls(
[self.fcntl_lock_call, self.fcntl_unlock_call])
self.mock_fd.write.assert_called_once_with(self.buf)
self.mock_log.assert_not_called()
def test_write_would_block(self):
# lock/unlock paired calls
self.mock_fcntl.side_effect = [
# first try
BlockingIOError, None,
# second try
None, None]
wrote = dnsmasq._exclusive_write_or_pass(self.path, self.buf)
self.assertTrue(wrote)
self.mock_open.assert_called_once_with(self.path, 'w', 1)
self.mock_fcntl.assert_has_calls(
[self.fcntl_lock_call, self.fcntl_unlock_call],
[self.fcntl_lock_call, self.fcntl_unlock_call])
self.mock_fd.write.assert_called_once_with(self.buf)
self.mock_log.assert_called_once_with(
'%s locked; will try again (later)', self.path)
self.mock_sleep.assert_called_once_with(
dnsmasq._EXCLUSIVE_WRITE_ATTEMPTS_DELAY)
@mock.patch.object(dnsmasq, '_EXCLUSIVE_WRITE_ATTEMPTS', 1)
def test_write_would_block_too_many_times(self):
self.mock_fcntl.side_effect = [BlockingIOError, None]
wrote = dnsmasq._exclusive_write_or_pass(self.path, self.buf)
self.assertFalse(wrote)
self.mock_open.assert_called_once_with(self.path, 'w', 1)
self.mock_fcntl.assert_has_calls(
[self.fcntl_lock_call, self.fcntl_unlock_call])
self.mock_fd.write.assert_not_called()
retry_log_call = mock.call('%s locked; will try again (later)',
self.path)
failed_log_call = mock.call(
'Failed to write the exclusively-locked path: %(path)s for '
'%(attempts)s times', {
'attempts': dnsmasq._EXCLUSIVE_WRITE_ATTEMPTS,
'path': self.path
})
self.mock_log.assert_has_calls([retry_log_call, failed_log_call])
self.mock_sleep.assert_called_once_with(
dnsmasq._EXCLUSIVE_WRITE_ATTEMPTS_DELAY)
def test_write_custom_ioerror(self):
err = IOError('Oops!')
err.errno = errno.EBADF
self.mock_fcntl.side_effect = [err, None]
self.assertRaisesRegex(
IOError, 'Oops!', dnsmasq._exclusive_write_or_pass, self.path,
self.buf)
self.mock_open.assert_called_once_with(self.path, 'w', 1)
self.mock_fcntl.assert_has_calls(
[self.fcntl_lock_call, self.fcntl_unlock_call])
self.mock_fd.write.assert_not_called()
self.mock_log.assert_not_called()
class TestHelpers(test_base.TestCase):
def setUp(self):
super().setUp()
self.mac = 'ff:ff:ff:ff:ff:ff'
self.dhcp_hostsdir = '/far'
self.path = os.path.join(self.dhcp_hostsdir, self.mac)
self.unknown_path = os.path.join(self.dhcp_hostsdir,
dnsmasq._UNKNOWN_HOSTS_FILE)
CONF.set_override('dhcp_hostsdir', self.dhcp_hostsdir,
'pxe_filter')
self.mock__exclusive_write_or_pass = self.useFixture(
fixtures.MockPatchObject(dnsmasq, '_exclusive_write_or_pass')).mock
self.mock_stat = self.useFixture(
fixtures.MockPatchObject(os, 'stat')).mock
self.mock_listdir = self.useFixture(
fixtures.MockPatchObject(os, 'listdir')).mock
self.mock_log = self.useFixture(
fixtures.MockPatchObject(dnsmasq, 'LOG')).mock
def test__allowlist_unknown_hosts(self):
dnsmasq._configure_unknown_hosts(True)
self.mock__exclusive_write_or_pass.assert_called_once_with(
self.unknown_path, '%s' % dnsmasq._ALLOW_UNKNOWN_HOSTS)
self.mock_log.debug.assert_called_once_with(
'A %s record for all unknown hosts using wildcard mac '
'created', 'allow')
def test__denylist_unknown_hosts(self):
dnsmasq._configure_unknown_hosts(False)
self.mock__exclusive_write_or_pass.assert_called_once_with(
self.unknown_path, '%s' % dnsmasq._DENY_UNKNOWN_HOSTS)
self.mock_log.debug.assert_called_once_with(
'A %s record for all unknown hosts using wildcard mac '
'created', 'deny')
def test__configure_removedlist_allowlist(self):
self.mock_stat.return_value.st_size = dnsmasq._MAC_DENY_LEN
dnsmasq._configure_removedlist({self.mac}, True)
self.mock__exclusive_write_or_pass.assert_called_once_with(
self.path, '%s\n' % self.mac)
def test__configure_removedlist_denylist(self):
self.mock_stat.return_value.st_size = dnsmasq._MAC_ALLOW_LEN
dnsmasq._configure_removedlist({self.mac}, False)
self.mock__exclusive_write_or_pass.assert_called_once_with(
self.path, '%s,ignore\n' % self.mac)
def test__allowlist_mac(self):
dnsmasq._add_mac_to_allowlist(self.mac)
self.mock__exclusive_write_or_pass.assert_called_once_with(
self.path, '%s\n' % self.mac)
def test__denylist_mac(self):
dnsmasq._add_mac_to_denylist(self.mac)
self.mock__exclusive_write_or_pass.assert_called_once_with(
self.path, '%s,ignore\n' % self.mac)
def test__get_denylist(self):
self.mock_listdir.return_value = [self.mac]
self.mock_stat.return_value.st_size = len('%s,ignore\n' % self.mac)
denylist, allowlist = dnsmasq._get_deny_allow_lists()
self.assertEqual({self.mac}, denylist)
self.mock_listdir.assert_called_once_with(self.dhcp_hostsdir)
self.mock_stat.assert_called_with(self.path)
def test__get_allowlist(self):
self.mock_listdir.return_value = [self.mac]
self.mock_stat.return_value.st_size = len('%s\n' % self.mac)
denylist, allowlist = dnsmasq._get_deny_allow_lists()
self.assertEqual({self.mac}, allowlist)
self.mock_listdir.assert_called_once_with(self.dhcp_hostsdir)
self.mock_stat.assert_called_with(self.path)
def test__get_no_denylist(self):
self.mock_listdir.return_value = [self.mac]
self.mock_stat.return_value.st_size = len('%s\n' % self.mac)
denylist, allowlist = dnsmasq._get_deny_allow_lists()
self.assertEqual(set(), denylist)
self.mock_listdir.assert_called_once_with(self.dhcp_hostsdir)
self.mock_stat.assert_called_with(self.path)
def test__get_no_allowlist(self):
self.mock_listdir.return_value = [self.mac]
self.mock_stat.return_value.st_size = len('%s,ignore\n' % self.mac)
denylist, allowlist = dnsmasq._get_deny_allow_lists()
self.assertEqual(set(), allowlist)
self.mock_listdir.assert_called_once_with(self.dhcp_hostsdir)
self.mock_stat.assert_called_with(self.path)
@mock.patch.object(dnsmasq, '_configure_unknown_hosts', autospec=True)
@mock.patch.object(dnsmasq, '_add_mac_to_denylist', autospec=True)
@mock.patch.object(dnsmasq, '_add_mac_to_allowlist', autospec=True)
class TestUpdate(test_base.TestCase):
def test_no_update(self, mock_allow, mock_deny, mock_configure_unknown):
dnsmasq.update([], [])
mock_allow.assert_not_called()
mock_deny.assert_not_called()
mock_configure_unknown.assert_not_called()
def test_only_allow(self, mock_allow, mock_deny, mock_configure_unknown):
dnsmasq.update(['mac1', 'mac2'], [], allow_unknown=True)
mock_allow.assert_has_calls([mock.call(f'mac{i}') for i in (1, 2)])
mock_deny.assert_not_called()
mock_configure_unknown.assert_called_once_with(True)
def test_only_deny(self, mock_allow, mock_deny, mock_configure_unknown):
dnsmasq.update([], ['mac1', 'mac2'])
mock_allow.assert_not_called()
mock_deny.assert_has_calls([mock.call(f'mac{i}') for i in (1, 2)])
mock_configure_unknown.assert_not_called()
@mock.patch.object(dnsmasq, '_configure_removedlist', autospec=True)
@mock.patch.object(dnsmasq, '_configure_unknown_hosts', autospec=True)
@mock.patch.object(dnsmasq, '_add_mac_to_denylist', autospec=True)
@mock.patch.object(dnsmasq, '_add_mac_to_allowlist', autospec=True)
@mock.patch.object(dnsmasq, '_get_deny_allow_lists', autospec=True)
class TestSync(test_base.TestCase):
def test_no_macs(self, mock_get_lists, mock_allow, mock_deny,
mock_configure_unknown, mock_configure_removedlist):
mock_get_lists.return_value = set(), set()
dnsmasq.sync([], [], False)
mock_allow.assert_not_called()
mock_deny.assert_not_called()
mock_configure_unknown.assert_called_once_with(False)
mock_configure_removedlist.assert_called_once_with(set(), False)
def test_only_new_macs(self, mock_get_lists, mock_allow, mock_deny,
mock_configure_unknown, mock_configure_removedlist):
mock_get_lists.return_value = set(), set()
dnsmasq.sync(['allow1', 'allow2'], [], True)
mock_allow.assert_has_calls(
[mock.call(f'allow{i}') for i in (1, 2)],
any_order=True)
mock_deny.assert_not_called()
mock_configure_unknown.assert_called_once_with(True)
mock_configure_removedlist.assert_called_once_with(set(), True)
def test_deny_macs(self, mock_get_lists, mock_allow, mock_deny,
mock_configure_unknown, mock_configure_removedlist):
mock_get_lists.return_value = set(), {'deny1', 'allow1'}
dnsmasq.sync(['allow1'], ['deny1', 'deny2'], False)
mock_allow.assert_not_called()
mock_deny.assert_has_calls(
[mock.call(f'deny{i}') for i in (1, 2)],
any_order=True)
mock_configure_unknown.assert_called_once_with(False)
mock_configure_removedlist.assert_called_once_with(set(), False)
def test_removed_nodes(self, mock_get_lists, mock_allow, mock_deny,
mock_configure_unknown, mock_configure_removedlist):
mock_get_lists.return_value = {'mac1'}, {'mac2', 'mac3'}
dnsmasq.sync(['mac2'], [], True)
mock_allow.assert_not_called()
mock_deny.assert_not_called()
mock_configure_unknown.assert_called_once_with(True)
mock_configure_removedlist.assert_called_once_with(
{'mac1', 'mac3'}, True)
def test_change_state(self, mock_get_lists, mock_allow, mock_deny,
mock_configure_unknown, mock_configure_removedlist):
# MAC1 from denied to allowed, MAC2 from allowed to denied, drop MAC3
mock_get_lists.return_value = {'mac1'}, {'mac2', 'mac3'}
dnsmasq.sync(['mac1'], ['mac2'], False)
mock_allow.assert_called_once_with('mac1')
mock_deny.assert_called_once_with('mac2')
mock_configure_unknown.assert_called_once_with(False)
mock_configure_removedlist.assert_called_once_with({'mac3'}, False)

View File

@ -0,0 +1,128 @@
# 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 random
import string
from unittest import mock
from oslo_utils import uuidutils
from ironic.common import states
from ironic.conf import CONF
from ironic.pxe_filter import dnsmasq
from ironic.pxe_filter import service as pxe_filter_service
from ironic.tests.unit.db import base as test_base
from ironic.tests.unit.db import utils as db_utils
def generate_mac():
return ':'.join(''.join(random.choice(string.hexdigits) for _ in range(2))
for _ in range(6))
@mock.patch.object(dnsmasq, 'sync', autospec=True)
class TestSync(test_base.DbTestCase):
def setUp(self):
super().setUp()
self.service = pxe_filter_service.PXEFilterManager('host')
def test_no_nodes(self, mock_sync):
self.service._sync(self.dbapi)
mock_sync.assert_called_once_with([], [], False)
def test_no_nodes_with_discovery(self, mock_sync):
CONF.set_override('enabled', True, group='auto_discovery')
self.service._sync(self.dbapi)
mock_sync.assert_called_once_with([], [], True)
def test_sync(self, mock_sync):
on_inspection = [
db_utils.create_test_node(uuid=uuidutils.generate_uuid(),
provision_state=state,
inspect_interface='agent')
for state in (states.INSPECTWAIT, states.INSPECTING)
]
not_on_inspection = [
db_utils.create_test_node(uuid=uuidutils.generate_uuid(),
provision_state=state,
inspect_interface='agent')
for state in (states.ACTIVE, states.AVAILABLE, states.INSPECTFAIL)
]
ignored = db_utils.create_test_node(uuid=uuidutils.generate_uuid(),
provision_state=states.INSPECTING,
inspect_interface='no-inspect')
ignored_port = db_utils.create_test_port(
uuid=uuidutils.generate_uuid(),
node_id=ignored.id,
address=generate_mac())
allow_macs, deny_macs = set(), {ignored_port.address}
for count, node in enumerate(on_inspection):
for _i in range(count):
port = db_utils.create_test_port(
uuid=uuidutils.generate_uuid(),
node_id=node.id,
address=generate_mac())
allow_macs.add(port.address)
for count, node in enumerate(not_on_inspection):
for _i in range(count):
port = db_utils.create_test_port(
uuid=uuidutils.generate_uuid(),
node_id=node.id,
address=generate_mac())
deny_macs.add(port.address)
self.service._sync(self.dbapi)
mock_sync.assert_called_once_with(mock.ANY, mock.ANY, True)
self.assertEqual(allow_macs, set(mock_sync.call_args.args[0]))
self.assertEqual(deny_macs, set(mock_sync.call_args.args[1]))
def test_nothing_on_inspection(self, mock_sync):
not_on_inspection = [
db_utils.create_test_node(uuid=uuidutils.generate_uuid(),
provision_state=state,
inspect_interface='agent')
for state in (states.ACTIVE, states.AVAILABLE, states.INSPECTFAIL)
]
deny_macs = set()
for count, node in enumerate(not_on_inspection):
for _i in range(count):
port = db_utils.create_test_port(
uuid=uuidutils.generate_uuid(),
node_id=node.id,
address=generate_mac())
deny_macs.add(port.address)
self.service._sync(self.dbapi)
mock_sync.assert_called_once_with([], mock.ANY, False)
self.assertEqual(deny_macs, set(mock_sync.call_args.args[1]))
class TestManager(test_base.DbTestCase):
@mock.patch('eventlet.spawn_after', lambda delay, func: func())
@mock.patch('eventlet.event.Event', autospec=True)
@mock.patch.object(pxe_filter_service.PXEFilterManager, '_sync',
autospec=True)
def test_init_and_run(self, mock_sync, mock_event):
mock_wait = mock_event.return_value.wait
mock_wait.side_effect = [None, None, True]
mock_sync.side_effect = [None, RuntimeError(), None]
service = pxe_filter_service.PXEFilterManager('example.com')
service.init_host(mock.sentinel.context)
mock_sync.assert_called_with(service, mock.ANY)
self.assertEqual(3, mock_sync.call_count)
mock_wait.assert_called_with(timeout=45)

View File

@ -0,0 +1,7 @@
---
features:
- |
Adds a new service ``ironic-pxe-filter`` that is designed to work with
the ``agent`` inspect interface to conduct "unmanaged" inspection. It is
adapted from the ironic-inspector's ``dnsmasq`` PXE filter and can be used
as its replacement. See documentation for more details.

View File

@ -50,6 +50,7 @@ console_scripts =
ironic-conductor = ironic.cmd.conductor:main
ironic-rootwrap = oslo_rootwrap.cmd:main
ironic-status = ironic.cmd.status:main
ironic-pxe-filter = ironic.cmd.pxe_filter:main
wsgi_scripts =
ironic-api-wsgi = ironic.api.wsgi:initialize_wsgi_app