Add support for inspection using ironic-inspector
Adds a new module ironic_python_agent.inspector and new entry point for extensions, which will allow vendor-specific inspection. Inspection is run on service start up just before the lookup. Due to this early start, and due to the fact we don't even know MAC address of nodes on inspection (to say nothing about IP addresses), exception handling is a bit different from other agent features: we try hard not to error out until we send at least something to inspector. Change-Id: I00932463d41819fd0a050782e2c88eddf6fc08c6
This commit is contained in:
parent
cdb4b520eb
commit
096830414b
@ -63,6 +63,19 @@ After the agent heartbeats, the conductor performs any actions needed against
|
||||
the node, including querying status of an already run command. For example,
|
||||
initiating in-band cleaning tasks or deploying an image to the node.
|
||||
|
||||
Inspection
|
||||
~~~~~~~~~~
|
||||
IPA can conduct hardware inspection on start up and post data to the `Ironic
|
||||
Inspector`_. Edit your default PXE/iPXE configuration or kernel command
|
||||
options baked in the image, and set ``ipa-inspection-callback-url`` to the
|
||||
full endpoint of Ironic Inspector, for example::
|
||||
|
||||
ipa-inspection-callback-url=http://IP:5050/v1/continue
|
||||
|
||||
Make sure your DHCP environment is set to boot IPA by default.
|
||||
|
||||
.. _Ironic Inspector: https://github.com/openstack/ironic-inspector
|
||||
|
||||
Image Builders
|
||||
--------------
|
||||
Unlike most other python software, you must build an IPA ramdisk image before
|
||||
|
@ -28,6 +28,7 @@ from ironic_python_agent import encoding
|
||||
from ironic_python_agent import errors
|
||||
from ironic_python_agent.extensions import base
|
||||
from ironic_python_agent import hardware
|
||||
from ironic_python_agent import inspector
|
||||
from ironic_python_agent import ironic_api_client
|
||||
|
||||
|
||||
@ -280,11 +281,16 @@ class IronicPythonAgent(base.ExecuteCommandMixin):
|
||||
hardware.load_managers()
|
||||
|
||||
if not self.standalone:
|
||||
# Inspection should be started before call to lookup, otherwise
|
||||
# lookup will fail due to unknown MAC.
|
||||
uuid = inspector.inspect()
|
||||
|
||||
content = self.api_client.lookup_node(
|
||||
hardware_info=hardware.dispatch_to_managers(
|
||||
'list_hardware_info'),
|
||||
timeout=self.lookup_timeout,
|
||||
starting_interval=self.lookup_interval)
|
||||
starting_interval=self.lookup_interval,
|
||||
node_uuid=uuid)
|
||||
|
||||
self.node = content['node']
|
||||
self.heartbeat_timeout = content['heartbeat_timeout']
|
||||
|
@ -18,6 +18,7 @@ from oslo_config import cfg
|
||||
from oslo_log import log
|
||||
|
||||
from ironic_python_agent import agent
|
||||
from ironic_python_agent import inspector
|
||||
from ironic_python_agent import utils
|
||||
|
||||
CONF = cfg.CONF
|
||||
@ -99,6 +100,19 @@ cli_opts = [
|
||||
default=APARAMS.get('ipa-standalone', False),
|
||||
help='Note: for debugging only. Start the Agent but suppress '
|
||||
'any calls to Ironic API.'),
|
||||
|
||||
cfg.StrOpt('inspection_callback_url',
|
||||
default=APARAMS.get('ipa-inspection-callback-url'),
|
||||
help='Endpoint of ironic-inspector. If set, hardware inventory '
|
||||
'will be collected and sent to ironic-inspector '
|
||||
'on start up.'),
|
||||
|
||||
cfg.StrOpt('inspection_collectors',
|
||||
default=APARAMS.get('ipa-inspection-collectors',
|
||||
inspector.DEFAULT_COLLECTOR),
|
||||
help='Comma-separated list of plugins providing additional '
|
||||
'hardware data for inspection, empty value gives '
|
||||
'a minimum required set of plugins.'),
|
||||
]
|
||||
|
||||
CONF.register_cli_opts(cli_opts)
|
||||
|
@ -321,3 +321,9 @@ class DeviceNotFound(NotFound):
|
||||
|
||||
def __init__(self, details):
|
||||
super(DeviceNotFound, self).__init__(details)
|
||||
|
||||
|
||||
# This is not something we return to a user, so we don't inherit it from
|
||||
# RESTError.
|
||||
class InspectionError(Exception):
|
||||
"""Failure during inspection."""
|
||||
|
@ -391,20 +391,7 @@ class GenericHardwareManager(HardwareManager):
|
||||
root_device_hints = utils.parse_root_device_hints()
|
||||
|
||||
if not root_device_hints:
|
||||
# If no hints are passed find the first device larger than
|
||||
# 4GiB, assume it is the OS disk
|
||||
min_size_required = 4 * units.Gi
|
||||
# TODO(russellhaering): This isn't a valid assumption in
|
||||
# all cases, is there a more reasonable default behavior?
|
||||
block_devices.sort(key=lambda device: device.size)
|
||||
if block_devices[-1].size < min_size_required:
|
||||
raise errors.DeviceNotFound("No suitable device was found "
|
||||
"for deployment - root device hints were not provided "
|
||||
"and all found block devices are smaller than %iB."
|
||||
% min_size_required)
|
||||
for device in block_devices:
|
||||
if device.size >= min_size_required:
|
||||
return device.name
|
||||
return utils.guess_root_disk(block_devices).name
|
||||
else:
|
||||
|
||||
def match(hint, current_value, device):
|
||||
|
219
ironic_python_agent/inspector.py
Normal file
219
ironic_python_agent/inspector.py
Normal file
@ -0,0 +1,219 @@
|
||||
# Copyright 2015 Red Hat, Inc.
|
||||
#
|
||||
# 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 logging
|
||||
|
||||
from oslo_concurrency import processutils
|
||||
from oslo_config import cfg
|
||||
from oslo_utils import excutils
|
||||
from oslo_utils import units
|
||||
import requests
|
||||
import stevedore
|
||||
|
||||
from ironic_python_agent import encoding
|
||||
from ironic_python_agent import errors
|
||||
from ironic_python_agent import hardware
|
||||
from ironic_python_agent import utils
|
||||
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
CONF = cfg.CONF
|
||||
DEFAULT_COLLECTOR = 'default'
|
||||
_COLLECTOR_NS = 'ironic_python_agent.inspector.collectors'
|
||||
|
||||
|
||||
def extension_manager(names):
|
||||
try:
|
||||
return stevedore.NamedExtensionManager(_COLLECTOR_NS,
|
||||
names=names,
|
||||
name_order=True)
|
||||
except KeyError as exc:
|
||||
raise errors.InspectionError('Failed to load collector %s' % exc)
|
||||
|
||||
|
||||
def inspect():
|
||||
"""Optionally run inspection on the current node.
|
||||
|
||||
If ``inspection_callback_url`` is set in the configuration, get
|
||||
the hardware inventory from the node and post it back to the inspector.
|
||||
|
||||
:return: node UUID if inspection was successful, None if associated node
|
||||
was not found in inspector cache. None is also returned if
|
||||
inspector support is not enabled.
|
||||
"""
|
||||
if not CONF.inspection_callback_url:
|
||||
LOG.info('Inspection is disabled, skipping')
|
||||
return
|
||||
collector_names = [x.strip() for x in CONF.inspection_collectors.split(',')
|
||||
if x.strip()]
|
||||
LOG.info('inspection is enabled with collectors %s', collector_names)
|
||||
|
||||
# NOTE(dtantsur): inspection process tries to delay raising any exceptions
|
||||
# until after we posted some data back to inspector. This is because
|
||||
# inspection is run automatically on (mostly) unknown nodes, so if it
|
||||
# fails, we don't have much information for debugging.
|
||||
failures = utils.AccumulatedFailures(exc_class=errors.InspectionError)
|
||||
data = {}
|
||||
|
||||
try:
|
||||
ext_mgr = extension_manager(collector_names)
|
||||
collectors = [(ext.name, ext.plugin) for ext in ext_mgr]
|
||||
except Exception as exc:
|
||||
with excutils.save_and_reraise_exception():
|
||||
failures.add(exc)
|
||||
call_inspector(data, failures)
|
||||
|
||||
for name, collector in collectors:
|
||||
try:
|
||||
collector(data, failures)
|
||||
except Exception as exc:
|
||||
# No reraise here, try to keep going
|
||||
failures.add('collector %s failed: %s', name, exc)
|
||||
|
||||
resp = call_inspector(data, failures)
|
||||
|
||||
# Now raise everything we were delaying
|
||||
failures.raise_if_needed()
|
||||
|
||||
if resp is None:
|
||||
LOG.info('stopping inspection, as inspector returned an error')
|
||||
return
|
||||
|
||||
# Optionally update IPMI credentials
|
||||
setup_ipmi_credentials(resp)
|
||||
|
||||
LOG.info('inspection finished successfully')
|
||||
return resp.get('uuid')
|
||||
|
||||
|
||||
def call_inspector(data, failures):
|
||||
"""Post data to inspector."""
|
||||
data['error'] = failures.get_error()
|
||||
|
||||
LOG.info('posting collected data to %s', CONF.inspection_callback_url)
|
||||
LOG.debug('collected data: %s', data)
|
||||
|
||||
encoder = encoding.RESTJSONEncoder()
|
||||
data = encoder.encode(data)
|
||||
|
||||
resp = requests.post(CONF.inspection_callback_url, data=data)
|
||||
if resp.status_code >= 400:
|
||||
LOG.error('inspector error %d: %s, proceeding with lookup',
|
||||
resp.status_code, resp.content.decode('utf-8'))
|
||||
return
|
||||
|
||||
return resp.json()
|
||||
|
||||
|
||||
def setup_ipmi_credentials(resp):
|
||||
"""Setup IPMI credentials, if requested.
|
||||
|
||||
:param resp: JSON response from inspector.
|
||||
"""
|
||||
if not resp.get('ipmi_setup_credentials'):
|
||||
LOG.info('setting IPMI credentials was not requested')
|
||||
return
|
||||
|
||||
user, password = resp['ipmi_username'], resp['ipmi_password']
|
||||
LOG.debug('setting IPMI credentials: user %s', user)
|
||||
|
||||
commands = [
|
||||
('user', 'set', 'name', '2', user),
|
||||
('user', 'set', 'password', '2', password),
|
||||
('user', 'enable', '2'),
|
||||
('channel', 'setaccess', '1', '2',
|
||||
'link=on', 'ipmi=on', 'callin=on', 'privilege=4'),
|
||||
]
|
||||
|
||||
for cmd in commands:
|
||||
try:
|
||||
utils.execute('ipmitool', *cmd)
|
||||
except processutils.ProcessExecutionError:
|
||||
LOG.exception('failed to update IPMI credentials')
|
||||
raise errors.InspectionError('failed to update IPMI credentials')
|
||||
|
||||
LOG.info('successfully set IPMI credentials: user %s', user)
|
||||
|
||||
|
||||
def discover_network_properties(inventory, data, failures):
|
||||
"""Discover network and BMC related properties.
|
||||
|
||||
Populates 'boot_interface', 'ipmi_address' and 'interfaces' keys.
|
||||
"""
|
||||
# Both boot interface and IPMI address might not be present,
|
||||
# we don't count it as failure
|
||||
data['boot_interface'] = utils.get_agent_params().get('BOOTIF')
|
||||
LOG.info('boot devices was %s', data['boot_interface'])
|
||||
data['ipmi_address'] = inventory.get('bmc_address')
|
||||
LOG.info('BMC IP address: %s', data['ipmi_address'])
|
||||
|
||||
data.setdefault('interfaces', {})
|
||||
for iface in inventory['interfaces']:
|
||||
if iface.name == 'lo' or iface.ipv4_address == '127.0.0.1':
|
||||
LOG.debug('ignoring local network interface %s', iface.name)
|
||||
continue
|
||||
|
||||
LOG.debug('found network interface %s', iface.name)
|
||||
|
||||
if not iface.mac_address:
|
||||
LOG.debug('no link information for interface %s', iface.name)
|
||||
continue
|
||||
|
||||
if not iface.ipv4_address:
|
||||
LOG.debug('no IP address for interface %s', iface.name)
|
||||
|
||||
data['interfaces'][iface.name] = {'mac': iface.mac_address,
|
||||
'ip': iface.ipv4_address}
|
||||
|
||||
if data['interfaces']:
|
||||
LOG.info('network interfaces: %s', data['interfaces'])
|
||||
else:
|
||||
failures.add('no network interfaces found')
|
||||
|
||||
|
||||
def discover_scheduling_properties(inventory, data):
|
||||
data['cpus'] = inventory['cpu'].count
|
||||
data['cpu_arch'] = inventory['cpu'].architecture
|
||||
data['memory_mb'] = inventory['memory'].physical_mb
|
||||
|
||||
# Replicate the same logic as in deploy. This logic will be moved to
|
||||
# inspector itself, but we need it for backward compatibility.
|
||||
try:
|
||||
disk = utils.guess_root_disk(inventory['disks'])
|
||||
except errors.DeviceNotFound:
|
||||
LOG.warn('no suitable root device detected')
|
||||
else:
|
||||
# -1 is required to give Ironic some spacing for partitioning
|
||||
data['local_gb'] = disk.size / units.Gi - 1
|
||||
|
||||
for key in ('cpus', 'local_gb', 'memory_mb'):
|
||||
try:
|
||||
data[key] = int(data[key])
|
||||
except (KeyError, ValueError, TypeError):
|
||||
LOG.warn('value for %s is missing or malformed: %s',
|
||||
key, data.get(key))
|
||||
else:
|
||||
LOG.info('value for %s is %s', key, data[key])
|
||||
|
||||
|
||||
def collect_default(data, failures):
|
||||
inventory = hardware.dispatch_to_managers('list_hardware_info')
|
||||
# These 2 calls are required for backward compatibility and should be
|
||||
# dropped after inspector is ready (probably in Mitaka cycle).
|
||||
discover_network_properties(inventory, data, failures)
|
||||
discover_scheduling_properties(inventory, data)
|
||||
# In the future we will only need the current version of inventory,
|
||||
# everything else will be done by inspector itself and its plugins
|
||||
data['inventory'] = inventory
|
@ -16,6 +16,7 @@ import json
|
||||
import time
|
||||
|
||||
import mock
|
||||
from oslo_config import cfg
|
||||
from oslotest import base as test_base
|
||||
import pkg_resources
|
||||
from stevedore import extension
|
||||
@ -26,9 +27,12 @@ from ironic_python_agent import encoding
|
||||
from ironic_python_agent import errors
|
||||
from ironic_python_agent.extensions import base
|
||||
from ironic_python_agent import hardware
|
||||
from ironic_python_agent import inspector
|
||||
|
||||
EXPECTED_ERROR = RuntimeError('command execution failed')
|
||||
|
||||
CONF = cfg.CONF
|
||||
|
||||
|
||||
def foo_execute(*args, **kwargs):
|
||||
if kwargs['fail']:
|
||||
@ -164,6 +168,7 @@ class TestBaseAgent(test_base.BaseTestCase):
|
||||
@mock.patch('wsgiref.simple_server.make_server', autospec=True)
|
||||
@mock.patch.object(hardware.HardwareManager, 'list_hardware_info')
|
||||
def test_run(self, mocked_list_hardware, wsgi_server_cls):
|
||||
CONF.set_override('inspection_callback_url', '')
|
||||
wsgi_server = wsgi_server_cls.return_value
|
||||
wsgi_server.start.side_effect = KeyboardInterrupt()
|
||||
|
||||
@ -187,6 +192,43 @@ class TestBaseAgent(test_base.BaseTestCase):
|
||||
|
||||
self.agent.heartbeater.start.assert_called_once_with()
|
||||
|
||||
@mock.patch.object(inspector, 'inspect', autospec=True)
|
||||
@mock.patch('wsgiref.simple_server.make_server', autospec=True)
|
||||
@mock.patch.object(hardware.HardwareManager, 'list_hardware_info')
|
||||
def test_run_with_inspection(self, mocked_list_hardware, wsgi_server_cls,
|
||||
mocked_inspector):
|
||||
CONF.set_override('inspection_callback_url', 'http://foo/bar')
|
||||
|
||||
wsgi_server = wsgi_server_cls.return_value
|
||||
wsgi_server.start.side_effect = KeyboardInterrupt()
|
||||
|
||||
mocked_inspector.return_value = 'uuid'
|
||||
|
||||
self.agent.heartbeater = mock.Mock()
|
||||
self.agent.api_client.lookup_node = mock.Mock()
|
||||
self.agent.api_client.lookup_node.return_value = {
|
||||
'node': {
|
||||
'uuid': 'deadbeef-dabb-ad00-b105-f00d00bab10c'
|
||||
},
|
||||
'heartbeat_timeout': 300,
|
||||
}
|
||||
self.agent.run()
|
||||
|
||||
listen_addr = ('192.0.2.1', 9999)
|
||||
wsgi_server_cls.assert_called_once_with(
|
||||
listen_addr[0],
|
||||
listen_addr[1],
|
||||
self.agent.api,
|
||||
server_class=simple_server.WSGIServer)
|
||||
wsgi_server.serve_forever.assert_called_once_with()
|
||||
mocked_inspector.assert_called_once_with()
|
||||
self.assertEqual(1, self.agent.api_client.lookup_node.call_count)
|
||||
self.assertEqual(
|
||||
'uuid',
|
||||
self.agent.api_client.lookup_node.call_args[1]['node_uuid'])
|
||||
|
||||
self.agent.heartbeater.start.assert_called_once_with()
|
||||
|
||||
@mock.patch('os.read')
|
||||
@mock.patch('select.poll')
|
||||
@mock.patch('time.sleep', return_value=None)
|
||||
|
336
ironic_python_agent/tests/unit/test_inspector.py
Normal file
336
ironic_python_agent/tests/unit/test_inspector.py
Normal file
@ -0,0 +1,336 @@
|
||||
# Copyright 2015 Red Hat, Inc.
|
||||
#
|
||||
# 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 collections
|
||||
import copy
|
||||
import unittest
|
||||
|
||||
import mock
|
||||
from oslo_concurrency import processutils
|
||||
from oslo_config import cfg
|
||||
import requests
|
||||
import stevedore
|
||||
|
||||
from ironic_python_agent import errors
|
||||
from ironic_python_agent import hardware
|
||||
from ironic_python_agent import inspector
|
||||
from ironic_python_agent import utils
|
||||
|
||||
|
||||
CONF = cfg.CONF
|
||||
|
||||
|
||||
class AcceptingFailure(mock.Mock):
|
||||
def __call__(self, *args):
|
||||
return super(mock.Mock, self).__call__(
|
||||
*(copy.deepcopy(x) for x in args))
|
||||
|
||||
def assert_called_with_failure(self, expect_error=False):
|
||||
self.assert_called_once_with({}, mock.ANY)
|
||||
failure = self.call_args[0][1]
|
||||
assert bool(failure) is expect_error, '%s is not %s' % (
|
||||
failure, expect_error)
|
||||
|
||||
|
||||
class TestMisc(unittest.TestCase):
|
||||
def test_default_collector_loadable(self):
|
||||
ext = inspector.extension_manager([inspector.DEFAULT_COLLECTOR])
|
||||
self.assertIs(ext[inspector.DEFAULT_COLLECTOR].plugin,
|
||||
inspector.collect_default)
|
||||
|
||||
def test_raise_on_wrong_collector(self):
|
||||
self.assertRaisesRegexp(errors.InspectionError,
|
||||
'foobar',
|
||||
inspector.extension_manager,
|
||||
['foobar'])
|
||||
|
||||
|
||||
@mock.patch.object(inspector, 'setup_ipmi_credentials', autospec=True)
|
||||
@mock.patch.object(inspector, 'call_inspector', new_callable=AcceptingFailure)
|
||||
@mock.patch.object(stevedore, 'NamedExtensionManager', autospec=True)
|
||||
class TestInspect(unittest.TestCase):
|
||||
def setUp(self):
|
||||
super(TestInspect, self).setUp()
|
||||
CONF.set_override('inspection_callback_url', 'http://foo/bar')
|
||||
CONF.set_override('inspection_collectors', '')
|
||||
self.mock_collect = AcceptingFailure()
|
||||
self.mock_ext = mock.Mock(spec=['plugin', 'name'],
|
||||
plugin=self.mock_collect)
|
||||
|
||||
def test_ok(self, mock_ext_mgr, mock_call, mock_setup_ipmi):
|
||||
mock_ext_mgr.return_value = [self.mock_ext]
|
||||
mock_call.return_value = {'uuid': 'uuid1'}
|
||||
|
||||
result = inspector.inspect()
|
||||
|
||||
self.mock_collect.assert_called_with_failure()
|
||||
mock_call.assert_called_with_failure()
|
||||
self.assertEqual('uuid1', result)
|
||||
mock_setup_ipmi.assert_called_once_with(mock_call.return_value)
|
||||
|
||||
def test_collectors_option(self, mock_ext_mgr, mock_call, mock_setup_ipmi):
|
||||
CONF.set_override('inspection_collectors', 'foo,bar')
|
||||
mock_ext_mgr.return_value = [
|
||||
mock.Mock(spec=['name', 'plugin'], plugin=AcceptingFailure()),
|
||||
mock.Mock(spec=['name', 'plugin'], plugin=AcceptingFailure()),
|
||||
]
|
||||
|
||||
inspector.inspect()
|
||||
|
||||
for fake_ext in mock_ext_mgr.return_value:
|
||||
fake_ext.plugin.assert_called_with_failure()
|
||||
mock_call.assert_called_with_failure()
|
||||
|
||||
def test_collector_failed(self, mock_ext_mgr, mock_call, mock_setup_ipmi):
|
||||
mock_ext_mgr.return_value = [self.mock_ext]
|
||||
self.mock_collect.side_effect = RuntimeError('boom')
|
||||
|
||||
self.assertRaisesRegexp(errors.InspectionError,
|
||||
'boom', inspector.inspect)
|
||||
|
||||
self.mock_collect.assert_called_with_failure()
|
||||
mock_call.assert_called_with_failure(expect_error=True)
|
||||
self.assertFalse(mock_setup_ipmi.called)
|
||||
|
||||
def test_extensions_failed(self, mock_ext_mgr, mock_call, mock_setup_ipmi):
|
||||
CONF.set_override('inspection_collectors', 'foo,bar')
|
||||
mock_ext_mgr.side_effect = RuntimeError('boom')
|
||||
|
||||
self.assertRaisesRegexp(RuntimeError, 'boom', inspector.inspect)
|
||||
|
||||
mock_call.assert_called_with_failure(expect_error=True)
|
||||
self.assertFalse(mock_setup_ipmi.called)
|
||||
|
||||
def test_inspector_error(self, mock_ext_mgr, mock_call, mock_setup_ipmi):
|
||||
mock_call.return_value = None
|
||||
mock_ext_mgr.return_value = [self.mock_ext]
|
||||
|
||||
result = inspector.inspect()
|
||||
|
||||
self.mock_collect.assert_called_with_failure()
|
||||
mock_call.assert_called_with_failure()
|
||||
self.assertIsNone(result)
|
||||
self.assertFalse(mock_setup_ipmi.called)
|
||||
|
||||
|
||||
@mock.patch.object(requests, 'post', autospec=True)
|
||||
class TestCallInspector(unittest.TestCase):
|
||||
def setUp(self):
|
||||
super(TestCallInspector, self).setUp()
|
||||
CONF.set_override('inspection_callback_url', 'url')
|
||||
|
||||
def test_ok(self, mock_post):
|
||||
failures = utils.AccumulatedFailures()
|
||||
data = collections.OrderedDict(data=42)
|
||||
mock_post.return_value.status_code = 200
|
||||
|
||||
res = inspector.call_inspector(data, failures)
|
||||
|
||||
mock_post.assert_called_once_with('url',
|
||||
data='{"data": 42, "error": null}')
|
||||
self.assertEqual(mock_post.return_value.json.return_value, res)
|
||||
|
||||
def test_send_failure(self, mock_post):
|
||||
failures = mock.Mock(spec=utils.AccumulatedFailures)
|
||||
failures.get_error.return_value = "boom"
|
||||
data = collections.OrderedDict(data=42)
|
||||
mock_post.return_value.status_code = 200
|
||||
|
||||
res = inspector.call_inspector(data, failures)
|
||||
|
||||
mock_post.assert_called_once_with('url',
|
||||
data='{"data": 42, "error": "boom"}')
|
||||
self.assertEqual(mock_post.return_value.json.return_value, res)
|
||||
|
||||
def test_inspector_error(self, mock_post):
|
||||
failures = utils.AccumulatedFailures()
|
||||
data = collections.OrderedDict(data=42)
|
||||
mock_post.return_value.status_code = 400
|
||||
|
||||
res = inspector.call_inspector(data, failures)
|
||||
|
||||
mock_post.assert_called_once_with('url',
|
||||
data='{"data": 42, "error": null}')
|
||||
self.assertIsNone(res)
|
||||
|
||||
|
||||
@mock.patch.object(utils, 'execute', autospec=True)
|
||||
class TestSetupIpmiCredentials(unittest.TestCase):
|
||||
def setUp(self):
|
||||
super(TestSetupIpmiCredentials, self).setUp()
|
||||
self.resp = {'ipmi_username': 'user',
|
||||
'ipmi_password': 'pwd',
|
||||
'ipmi_setup_credentials': True}
|
||||
|
||||
def test_disabled(self, mock_call):
|
||||
del self.resp['ipmi_setup_credentials']
|
||||
|
||||
inspector.setup_ipmi_credentials(self.resp)
|
||||
|
||||
self.assertFalse(mock_call.called)
|
||||
|
||||
def test_ok(self, mock_call):
|
||||
inspector.setup_ipmi_credentials(self.resp)
|
||||
|
||||
expected = [
|
||||
mock.call('ipmitool', 'user', 'set', 'name', '2', 'user'),
|
||||
mock.call('ipmitool', 'user', 'set', 'password', '2', 'pwd'),
|
||||
mock.call('ipmitool', 'user', 'enable', '2'),
|
||||
mock.call('ipmitool', 'channel', 'setaccess', '1', '2',
|
||||
'link=on', 'ipmi=on', 'callin=on', 'privilege=4'),
|
||||
]
|
||||
self.assertEqual(expected, mock_call.call_args_list)
|
||||
|
||||
def test_user_failed(self, mock_call):
|
||||
mock_call.side_effect = processutils.ProcessExecutionError()
|
||||
|
||||
self.assertRaises(errors.InspectionError,
|
||||
inspector.setup_ipmi_credentials,
|
||||
self.resp)
|
||||
|
||||
mock_call.assert_called_once_with('ipmitool', 'user', 'set', 'name',
|
||||
'2', 'user')
|
||||
|
||||
def test_password_failed(self, mock_call):
|
||||
mock_call.side_effect = iter((None,
|
||||
processutils.ProcessExecutionError))
|
||||
|
||||
self.assertRaises(errors.InspectionError,
|
||||
inspector.setup_ipmi_credentials,
|
||||
self.resp)
|
||||
|
||||
expected = [
|
||||
mock.call('ipmitool', 'user', 'set', 'name', '2', 'user'),
|
||||
mock.call('ipmitool', 'user', 'set', 'password', '2', 'pwd')
|
||||
]
|
||||
self.assertEqual(expected, mock_call.call_args_list)
|
||||
|
||||
|
||||
class BaseDiscoverTest(unittest.TestCase):
|
||||
def setUp(self):
|
||||
super(BaseDiscoverTest, self).setUp()
|
||||
self.inventory = {
|
||||
'interfaces': [
|
||||
hardware.NetworkInterface(name='em1',
|
||||
mac_addr='aa:bb:cc:dd:ee:ff',
|
||||
ipv4_address='1.1.1.1'),
|
||||
hardware.NetworkInterface(name='em2',
|
||||
mac_addr='11:22:33:44:55:66',
|
||||
ipv4_address=None),
|
||||
],
|
||||
'cpu': hardware.CPU(model_name='generic', frequency='3000',
|
||||
count=4, architecture='x86_64'),
|
||||
'memory': hardware.Memory(total=11998396 * 1024,
|
||||
physical_mb=12288),
|
||||
'disks': [
|
||||
hardware.BlockDevice(name='/dev/sdc',
|
||||
model='Disk 2',
|
||||
size=500107862016,
|
||||
rotational=False),
|
||||
hardware.BlockDevice(name='/dev/sda',
|
||||
model='Too Small Disk',
|
||||
size=4294967295,
|
||||
rotational=False),
|
||||
hardware.BlockDevice(name='/dev/sdb',
|
||||
model='Disk 1',
|
||||
size=500107862016,
|
||||
rotational=True)
|
||||
],
|
||||
'bmc_address': '1.2.3.4',
|
||||
}
|
||||
self.failures = utils.AccumulatedFailures()
|
||||
self.data = {}
|
||||
|
||||
|
||||
@mock.patch.object(utils, 'get_agent_params',
|
||||
lambda: {'BOOTIF': 'boot:if'})
|
||||
class TestDiscoverNetworkProperties(BaseDiscoverTest):
|
||||
def test_no_network_interfaces(self):
|
||||
self.inventory['interfaces'] = [
|
||||
hardware.NetworkInterface(name='lo',
|
||||
mac_addr='aa:bb:cc:dd:ee:ff',
|
||||
ipv4_address='127.0.0.1')
|
||||
]
|
||||
|
||||
inspector.discover_network_properties(self.inventory, self.data,
|
||||
self.failures)
|
||||
|
||||
self.assertIn('no network interfaces found', self.failures.get_error())
|
||||
self.assertFalse(self.data['interfaces'])
|
||||
|
||||
def test_ok(self):
|
||||
inspector.discover_network_properties(self.inventory, self.data,
|
||||
self.failures)
|
||||
|
||||
self.assertEqual({'em1': {'mac': 'aa:bb:cc:dd:ee:ff',
|
||||
'ip': '1.1.1.1'},
|
||||
'em2': {'mac': '11:22:33:44:55:66',
|
||||
'ip': None}},
|
||||
self.data['interfaces'])
|
||||
self.assertEqual('1.2.3.4', self.data['ipmi_address'])
|
||||
self.assertEqual('boot:if', self.data['boot_interface'])
|
||||
self.assertFalse(self.failures)
|
||||
|
||||
def test_missing(self):
|
||||
self.inventory['interfaces'] = [
|
||||
hardware.NetworkInterface(name='em1',
|
||||
mac_addr='aa:bb:cc:dd:ee:ff'),
|
||||
hardware.NetworkInterface(name='em2',
|
||||
mac_addr=None,
|
||||
ipv4_address='1.2.1.2'),
|
||||
]
|
||||
|
||||
inspector.discover_network_properties(self.inventory, self.data,
|
||||
self.failures)
|
||||
|
||||
self.assertEqual({'em1': {'mac': 'aa:bb:cc:dd:ee:ff', 'ip': None}},
|
||||
self.data['interfaces'])
|
||||
self.assertFalse(self.failures)
|
||||
|
||||
|
||||
class TestDiscoverSchedulingProperties(BaseDiscoverTest):
|
||||
def test_ok(self):
|
||||
inspector.discover_scheduling_properties(self.inventory, self.data)
|
||||
|
||||
self.assertEqual({'cpus': 4, 'cpu_arch': 'x86_64', 'local_gb': 464,
|
||||
'memory_mb': 12288}, self.data)
|
||||
|
||||
def test_no_local_gb(self):
|
||||
# Some DRAC servers do not have any visible hard drive until RAID is
|
||||
# built
|
||||
self.inventory['disks'] = []
|
||||
|
||||
inspector.discover_scheduling_properties(self.inventory, self.data)
|
||||
|
||||
self.assertEqual({'cpus': 4, 'cpu_arch': 'x86_64', 'memory_mb': 12288},
|
||||
self.data)
|
||||
|
||||
|
||||
@mock.patch.object(inspector, 'discover_scheduling_properties', autospec=True)
|
||||
@mock.patch.object(inspector, 'discover_network_properties', autospec=True)
|
||||
@mock.patch.object(hardware, 'dispatch_to_managers', autospec=True)
|
||||
class TestCollectDefault(BaseDiscoverTest):
|
||||
def test_ok(self, mock_dispatch, mock_discover_net, mock_discover_sched):
|
||||
mock_dispatch.return_value = self.inventory
|
||||
|
||||
inspector.collect_default(self.data, self.failures)
|
||||
|
||||
mock_dispatch.assert_called_once_with('list_hardware_info')
|
||||
mock_discover_net.assert_called_once_with(self.inventory, self.data,
|
||||
self.failures)
|
||||
mock_discover_sched.assert_called_once_with(self.inventory, self.data)
|
||||
|
||||
for key in ('memory', 'interfaces', 'cpu', 'disks'):
|
||||
self.assertTrue(self.data['inventory'][key])
|
@ -413,3 +413,27 @@ class GetAgentParamsTestCase(test_base.BaseTestCase):
|
||||
}
|
||||
self.assertRaises(errors.DeviceNotFound,
|
||||
utils.parse_root_device_hints)
|
||||
|
||||
|
||||
class TestFailures(testtools.TestCase):
|
||||
def test_get_error(self):
|
||||
f = utils.AccumulatedFailures()
|
||||
self.assertFalse(f)
|
||||
self.assertIsNone(f.get_error())
|
||||
|
||||
f.add('foo')
|
||||
f.add('%s', 'bar')
|
||||
f.add(RuntimeError('baz'))
|
||||
self.assertTrue(f)
|
||||
|
||||
exp = ('The following errors were encountered:\n* foo\n* bar\n* baz')
|
||||
self.assertEqual(exp, f.get_error())
|
||||
|
||||
def test_raise(self):
|
||||
class FakeException(Exception):
|
||||
pass
|
||||
|
||||
f = utils.AccumulatedFailures(exc_class=FakeException)
|
||||
self.assertIsNone(f.raise_if_needed())
|
||||
f.add('foo')
|
||||
self.assertRaisesRegexp(FakeException, 'foo', f.raise_if_needed)
|
||||
|
@ -20,6 +20,7 @@ import tempfile
|
||||
|
||||
from oslo_concurrency import processutils
|
||||
from oslo_log import log as logging
|
||||
from oslo_utils import units
|
||||
from six.moves.urllib import parse
|
||||
|
||||
from ironic_python_agent import errors
|
||||
@ -232,3 +233,72 @@ def parse_root_device_hints():
|
||||
hints['size'] = int(hints['size'])
|
||||
|
||||
return hints
|
||||
|
||||
|
||||
class AccumulatedFailures(object):
|
||||
"""Object to accumulate failures without raising exception."""
|
||||
|
||||
def __init__(self, exc_class=RuntimeError):
|
||||
self._failures = []
|
||||
self._exc_class = exc_class
|
||||
|
||||
def add(self, fail, *fmt):
|
||||
"""Add failure with optional formatting.
|
||||
|
||||
:param fail: exception or error string
|
||||
:param fmt: formatting arguments (only if fail is a string)
|
||||
"""
|
||||
if fmt:
|
||||
fail = fail % fmt
|
||||
LOG.error('%s', fail)
|
||||
self._failures.append(fail)
|
||||
|
||||
def get_error(self):
|
||||
"""Get error string or None."""
|
||||
if not self._failures:
|
||||
return
|
||||
|
||||
msg = ('The following errors were encountered:\n%s'
|
||||
% '\n'.join('* %s' % item for item in self._failures))
|
||||
return msg
|
||||
|
||||
def raise_if_needed(self):
|
||||
"""Raise exception if error list is not empty.
|
||||
|
||||
:raises: RuntimeError
|
||||
"""
|
||||
if self._failures:
|
||||
raise self._exc_class(self.get_error())
|
||||
|
||||
def __nonzero__(self):
|
||||
return bool(self._failures)
|
||||
|
||||
__bool__ = __nonzero__
|
||||
|
||||
def __repr__(self): # pragma: no cover
|
||||
# This is for tests
|
||||
if self:
|
||||
return '<%s: %s>' % (self.__class__.__name__,
|
||||
', '.join(self._failures))
|
||||
else:
|
||||
return '<%s: success>' % self.__class__.__name__
|
||||
|
||||
|
||||
def guess_root_disk(block_devices, min_size_required=4 * units.Gi):
|
||||
"""Find suitable disk provided that root device hints are not given.
|
||||
|
||||
If no hints are passed find the first device larger than min_size_required,
|
||||
assume it is the OS disk
|
||||
"""
|
||||
# TODO(russellhaering): This isn't a valid assumption in
|
||||
# all cases, is there a more reasonable default behavior?
|
||||
block_devices.sort(key=lambda device: device.size)
|
||||
if not block_devices or block_devices[-1].size < min_size_required:
|
||||
raise errors.DeviceNotFound(
|
||||
"No suitable device was found "
|
||||
"for deployment - root device hints were not provided "
|
||||
"and all found block devices are smaller than %iB."
|
||||
% min_size_required)
|
||||
for device in block_devices:
|
||||
if device.size >= min_size_required:
|
||||
return device
|
||||
|
@ -29,6 +29,9 @@ ironic_python_agent.extensions =
|
||||
ironic_python_agent.hardware_managers =
|
||||
generic = ironic_python_agent.hardware:GenericHardwareManager
|
||||
|
||||
ironic_python_agent.inspector.collectors =
|
||||
default = ironic_python_agent.inspector:collect_default
|
||||
|
||||
[pbr]
|
||||
autodoc_index_modules = True
|
||||
warnerrors = True
|
||||
|
Loading…
Reference in New Issue
Block a user