Mount iSCSI target and 'dd' in PXE driver

This patch does integration deploy-helper functions
into PXE driver.

Partially implements blueprint pxe-mount-and-dd

Change-Id: I9df927dd3910e0bb88e9c028b674314650b98774
This commit is contained in:
Yuriy Zveryanskyy 2013-09-05 19:45:29 +03:00
parent cc316aa439
commit 535426efaf
7 changed files with 313 additions and 526 deletions

View File

@ -51,6 +51,7 @@ class FakePXEDriver(base.BaseDriver):
self.power = fake.FakePower()
self.deploy = pxe.PXEDeploy()
self.rescue = self.deploy
self.vendor = pxe.VendorPassthru()
class FakeSSHDriver(base.BaseDriver):

View File

@ -15,32 +15,20 @@
# License for the specific language governing permissions and limitations
# under the License.
"""Starter script for Bare-Metal Deployment Service."""
import os
import sys
import threading
import time
import cgi
import Queue
import re
import socket
import stat
from wsgiref import simple_server
from ironic.common import config
from ironic.common import exception
from ironic.common import states
from ironic.common import utils
from ironic import db
from ironic.openstack.common import context as ironic_context
from ironic.openstack.common import excutils
from ironic.openstack.common import log as logging
QUEUE = Queue.Queue()
LOG = logging.getLogger(__name__)
@ -136,7 +124,7 @@ def switch_pxe_config(path, root_uuid):
with open(path) as f:
lines = f.readlines()
root = 'UUID=%s' % root_uuid
rre = re.compile(r'\$\{ROOT\}')
rre = re.compile(r'\{\{ ROOT \}\}')
dre = re.compile('^default .*$')
with open(path, 'w') as f:
for line in lines:
@ -177,14 +165,14 @@ def work_on_disk(dev, root_mb, swap_mb, image_path):
swap_part = "%s-part2" % dev
if not is_block_device(dev):
LOG.warn(_("parent device '%s' not found") % dev)
LOG.warn(_("parent device '%s' not found"), dev)
return
make_partitions(dev, root_mb, swap_mb)
if not is_block_device(root_part):
LOG.warn(_("root device '%s' not found") % root_part)
LOG.warn(_("root device '%s' not found"), root_part)
return
if not is_block_device(swap_part):
LOG.warn(_("swap device '%s' not found") % swap_part)
LOG.warn(_("swap device '%s' not found"), swap_part)
return
dd(image_path, root_part)
mkswap(swap_part)
@ -193,7 +181,7 @@ def work_on_disk(dev, root_mb, swap_mb, image_path):
root_uuid = block_uuid(root_part)
except exception.ProcessExecutionError:
with excutils.save_and_reraise_exception():
LOG.error("Failed to detect root device UUID.")
LOG.error(_("Failed to detect root device UUID."))
return root_uuid
@ -211,125 +199,13 @@ def deploy(address, port, iqn, lun, image_path, pxe_config_path,
except exception.ProcessExecutionError as err:
with excutils.save_and_reraise_exception():
# Log output if there was a error
LOG.error("Cmd : %s" % err.cmd)
LOG.error("StdOut : %s" % err.stdout)
LOG.error("StdErr : %s" % err.stderr)
LOG.error(_("Deploy to address %s failed.") % address)
LOG.error(_("Command: %s") % err.cmd)
LOG.error(_("StdOut: %r") % err.stdout)
LOG.error(_("StdErr: %r") % err.stderr)
finally:
logout_iscsi(address, port, iqn)
switch_pxe_config(pxe_config_path, root_uuid)
# Ensure the node started netcat on the port after POST the request.
time.sleep(3)
notify(address, 10000)
class Worker(threading.Thread):
"""Thread that handles requests in queue."""
def __init__(self):
super(Worker, self).__init__()
self.setDaemon(True)
self.stop = False
self.queue_timeout = 1
def run(self):
while not self.stop:
try:
# Set timeout to check self.stop periodically
(node_id, params) = QUEUE.get(block=True,
timeout=self.queue_timeout)
except Queue.Empty:
pass
else:
# Requests comes here from BareMetalDeploy.post()
LOG.info(_('start deployment for node %(node_id)s, '
'params %(params)s') %
{'node_id': node_id, 'params': params})
context = ironic_context.get_admin_context()
try:
db.bm_node_update(context, node_id,
{'task_state': states.DEPLOYING})
deploy(**params)
except Exception:
LOG.error(_('deployment to node %s failed') % node_id)
db.bm_node_update(context, node_id,
{'task_state': states.DEPLOYFAIL})
else:
LOG.info(_('deployment to node %s done') % node_id)
db.bm_node_update(context, node_id,
{'task_state': states.DEPLOYDONE})
class BareMetalDeploy(object):
"""WSGI server for bare-metal deployment."""
def __init__(self):
self.worker = Worker()
self.worker.start()
def __call__(self, environ, start_response):
method = environ['REQUEST_METHOD']
if method == 'POST':
return self.post(environ, start_response)
else:
start_response('501 Not Implemented',
[('Content-type', 'text/plain')])
return 'Not Implemented'
def post(self, environ, start_response):
LOG.info(_("post: environ=%s") % environ)
inpt = environ['wsgi.input']
length = int(environ.get('CONTENT_LENGTH', 0))
x = inpt.read(length)
q = dict(cgi.parse_qsl(x))
try:
node_id = q['i']
deploy_key = q['k']
address = q['a']
port = q.get('p', '3260')
iqn = q['n']
lun = q.get('l', '1')
err_msg = q.get('e')
except KeyError as e:
start_response('400 Bad Request', [('Content-type', 'text/plain')])
return "parameter '%s' is not defined" % e
if err_msg:
LOG.error(_('Deploy agent error message: %s'), err_msg)
context = ironic_context.get_admin_context()
d = db.bm_node_get(context, node_id)
if d['deploy_key'] != deploy_key:
start_response('400 Bad Request', [('Content-type', 'text/plain')])
return 'key is not match'
params = {'address': address,
'port': port,
'iqn': iqn,
'lun': lun,
'image_path': d['image_path'],
'pxe_config_path': d['pxe_config_path'],
'root_mb': int(d['root_mb']),
'swap_mb': int(d['swap_mb']),
}
# Restart worker, if needed
if not self.worker.isAlive():
self.worker = Worker()
self.worker.start()
LOG.info(_("request is queued: node %(node_id)s, params %(params)s") %
{'node_id': node_id, 'params': params})
QUEUE.put((node_id, params))
# Requests go to Worker.run()
start_response('200 OK', [('Content-type', 'text/plain')])
return ''
def main():
config.parse_args(sys.argv)
logging.setup("nova")
global LOG
LOG = logging.getLogger('nova.virt.baremetal.deploy_helper')
app = BareMetalDeploy()
srv = simple_server.make_server('', 10000, app)
srv.serve_forever()

View File

@ -18,7 +18,6 @@
PXE Driver and supporting meta-classes.
"""
import datetime
import os
import tempfile
@ -33,12 +32,12 @@ from ironic.common import states
from ironic.common import utils
from ironic.drivers import base
from ironic.drivers.modules import deploy_utils
from ironic.openstack.common import context
from ironic.openstack.common import fileutils
from ironic.openstack.common import lockutils
from ironic.openstack.common import log as logging
from ironic.openstack.common import loopingcall
from ironic.openstack.common import timeutils
pxe_opts = [
@ -432,13 +431,14 @@ class PXEDeploy(base.DeployInterface):
_parse_driver_info(node)
def deploy(self, task, node):
"""Perform a deployment to a node.
"""Perform start deployment a node.
Given a node with complete metadata, deploy the indicated image
to the node.
:param task: a TaskManager instance.
:param node: the Node to act upon.
:returns: deploy state DEPLOYING.
"""
pxe_info = _get_tftp_image_info(node)
@ -446,46 +446,7 @@ class PXEDeploy(base.DeployInterface):
_create_pxe_config(task, node, pxe_info)
_cache_images(node, pxe_info)
local_status = {'error': '', 'started': False}
def _wait_for_deploy():
"""Called at an interval until the deployment completes."""
try:
node.refresh()
status = node['provision_state']
if (status == states.DEPLOYING
and local_status['started'] is False):
LOG.info(_("PXE deploy started for instance %s")
% node['instance_uuid'])
local_status['started'] = True
elif status in (states.DEPLOYDONE,
states.ACTIVE):
LOG.info(_("PXE deploy completed for instance %s")
% node['instance_uuid'])
raise loopingcall.LoopingCallDone()
elif status == states.DEPLOYFAIL:
local_status['error'] = _("PXE deploy failed for"
" instance %s")
except exception.NodeNotFound:
local_status['error'] = _("Baremetal node deleted"
"while waiting for deployment"
" of instance %s")
if (CONF.pxe.pxe_deploy_timeout and
timeutils.utcnow() > expiration):
local_status['error'] = _("Timeout reached while waiting for "
"PXE deploy of instance %s")
if local_status['error']:
raise loopingcall.LoopingCallDone()
expiration = timeutils.utcnow() + datetime.timedelta(
seconds=CONF.pxe.pxe_deploy_timeout)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_deploy)
timer.start(interval=1).wait()
if local_status['error']:
raise exception.InstanceDeployFailure(
local_status['error'] % node['instance_uuid'])
return states.DEPLOYING
def tear_down(self, task, node):
"""Tear down a previous deployment.
@ -495,6 +456,7 @@ class PXEDeploy(base.DeployInterface):
:param task: a TaskManager instance.
:param node: the Node to act upon.
:returns: deploy state DELETED.
"""
#FIXME(ghe): Possible error to get image info if eliminated from glance
# Retrieve image info and store in db
@ -518,6 +480,8 @@ class PXEDeploy(base.DeployInterface):
_destroy_images(d_info)
return states.DELETED
class PXERescue(base.RescueInterface):
@ -531,18 +495,83 @@ class PXERescue(base.RescueInterface):
pass
class IPMIVendorPassthru(base.VendorInterface):
class VendorPassthru(base.VendorInterface):
"""Interface to mix IPMI and PXE vendor-specific interfaces."""
def validate(self, node):
pass
def _get_deploy_info(self, node, **kwargs):
d_info = _parse_driver_info(node)
def vendor_passthru(self, task, node, *args, **kwargs):
method = kwargs.get('method')
params = {'address': kwargs.get('address'),
'port': kwargs.get('port', '3260'),
'iqn': kwargs.get('iqn'),
'lun': kwargs.get('lun', '1'),
'image_path': _get_image_file_path(d_info),
'pxe_config_path': _get_pxe_config_file_path(
node['instance_uuid']),
'root_mb': 1024 * int(d_info['root_gb']),
'swap_mb': int(d_info['swap_mb'])
}
missing = [key for key in params.keys() if params[key] is None]
if missing:
raise exception.InvalidParameterValue(_(
"Parameters %s were not passed to ironic"
" for deploy.") % missing)
return params
def validate(self, node, **kwargs):
method = kwargs['method']
if method == 'pass_deploy_info':
self._get_deploy_info(node, **kwargs)
elif method == 'set_boot_device':
# todo
pass
else:
raise exception.InvalidParameterValue(_(
"Unsupported method (%s) passed to PXE driver.")
% method)
return True
def _continue_deploy(self, task, node, **kwargs):
params = self._get_deploy_info(node, **kwargs)
ctx = context.get_admin_context()
node_id = node['uuid']
err_msg = kwargs.get('error')
if err_msg:
LOG.error(_('Node %(node_id)s deploy error message: %(error)s') %
{'node_id': node_id, 'error': err_msg})
LOG.info(_('start deployment for node %(node_id)s, '
'params %(params)s') %
{'node_id': node_id, 'params': params})
try:
node['provision_state'] = states.DEPLOYING
node.save(ctx)
deploy_utils.deploy(**params)
except Exception as e:
LOG.error(_('deployment to node %s failed') % node_id)
node['provision_state'] = states.DEPLOYFAIL
node.save(ctx)
raise exception.InstanceDeployFailure(_(
'Deploy error: "%(error)s" for node %(node_id)s') %
{'error': e.message, 'node_id': node_id})
else:
LOG.info(_('deployment to node %s done') % node_id)
node['provision_state'] = states.DEPLOYDONE
node.save(ctx)
def vendor_passthru(self, task, node, **kwargs):
method = kwargs['method']
if method == 'set_boot_device':
return node.driver.vendor._set_boot_device(
task, node,
kwargs.get('device'),
kwargs.get('persistent'))
else:
return
elif method == 'pass_deploy_info':
self._continue_deploy(task, node, **kwargs)

View File

@ -38,7 +38,7 @@ class PXEAndIPMIToolDriver(base.BaseDriver):
self.power = ipmitool.IPMIPower()
self.deploy = pxe.PXEDeploy()
self.rescue = self.deploy
self.vendor = pxe.IPMIVendorPassthru()
self.vendor = pxe.VendorPassthru()
class PXEAndSSHDriver(base.BaseDriver):
@ -75,4 +75,4 @@ class PXEAndIPMINativeDriver(base.BaseDriver):
self.power = ipminative.NativeIPMIPower()
self.deploy = pxe.PXEDeploy()
self.rescue = self.deploy
self.vendor = pxe.IPMIVendorPassthru()
self.vendor = pxe.VendorPassthru()

View File

@ -0,0 +1,192 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright (c) 2012 NTT DOCOMO, INC.
# Copyright 2011 OpenStack Foundation
# Copyright 2011 Ilya Alekseyev
#
# 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 tempfile
import time
from ironic.drivers.modules import deploy_utils as utils
from ironic.tests import base as tests_base
_PXECONF_DEPLOY = """
default deploy
label deploy
kernel deploy_kernel
append initrd=deploy_ramdisk
ipappend 3
label boot
kernel kernel
append initrd=ramdisk root={{ ROOT }}
"""
_PXECONF_BOOT = """
default boot
label deploy
kernel deploy_kernel
append initrd=deploy_ramdisk
ipappend 3
label boot
kernel kernel
append initrd=ramdisk root=UUID=12345678-1234-1234-1234-1234567890abcdef
"""
class PhysicalWorkTestCase(tests_base.TestCase):
def setUp(self):
super(PhysicalWorkTestCase, self).setUp()
def noop(*args, **kwargs):
pass
self.stubs.Set(time, 'sleep', noop)
def test_deploy(self):
"""Check loosely all functions are called with right args."""
address = '127.0.0.1'
port = 3306
iqn = 'iqn.xyz'
lun = 1
image_path = '/tmp/xyz/image'
pxe_config_path = '/tmp/abc/pxeconfig'
root_mb = 128
swap_mb = 64
dev = '/dev/fake'
root_part = '/dev/fake-part1'
swap_part = '/dev/fake-part2'
root_uuid = '12345678-1234-1234-12345678-12345678abcdef'
self.mox.StubOutWithMock(utils, 'get_dev')
self.mox.StubOutWithMock(utils, 'get_image_mb')
self.mox.StubOutWithMock(utils, 'discovery')
self.mox.StubOutWithMock(utils, 'login_iscsi')
self.mox.StubOutWithMock(utils, 'logout_iscsi')
self.mox.StubOutWithMock(utils, 'make_partitions')
self.mox.StubOutWithMock(utils, 'is_block_device')
self.mox.StubOutWithMock(utils, 'dd')
self.mox.StubOutWithMock(utils, 'mkswap')
self.mox.StubOutWithMock(utils, 'block_uuid')
self.mox.StubOutWithMock(utils, 'switch_pxe_config')
self.mox.StubOutWithMock(utils, 'notify')
utils.get_dev(address, port, iqn, lun).AndReturn(dev)
utils.get_image_mb(image_path).AndReturn(1) # < root_mb
utils.discovery(address, port)
utils.login_iscsi(address, port, iqn)
utils.is_block_device(dev).AndReturn(True)
utils.make_partitions(dev, root_mb, swap_mb)
utils.is_block_device(root_part).AndReturn(True)
utils.is_block_device(swap_part).AndReturn(True)
utils.dd(image_path, root_part)
utils.mkswap(swap_part)
utils.block_uuid(root_part).AndReturn(root_uuid)
utils.logout_iscsi(address, port, iqn)
utils.switch_pxe_config(pxe_config_path, root_uuid)
utils.notify(address, 10000)
self.mox.ReplayAll()
utils.deploy(address, port, iqn, lun, image_path, pxe_config_path,
root_mb, swap_mb)
self.mox.VerifyAll()
def test_always_logout_iscsi(self):
"""logout_iscsi() must be called once login_iscsi() is called."""
address = '127.0.0.1'
port = 3306
iqn = 'iqn.xyz'
lun = 1
image_path = '/tmp/xyz/image'
pxe_config_path = '/tmp/abc/pxeconfig'
root_mb = 128
swap_mb = 64
dev = '/dev/fake'
self.mox.StubOutWithMock(utils, 'get_dev')
self.mox.StubOutWithMock(utils, 'get_image_mb')
self.mox.StubOutWithMock(utils, 'discovery')
self.mox.StubOutWithMock(utils, 'login_iscsi')
self.mox.StubOutWithMock(utils, 'logout_iscsi')
self.mox.StubOutWithMock(utils, 'work_on_disk')
class TestException(Exception):
pass
utils.get_dev(address, port, iqn, lun).AndReturn(dev)
utils.get_image_mb(image_path).AndReturn(1) # < root_mb
utils.discovery(address, port)
utils.login_iscsi(address, port, iqn)
utils.work_on_disk(dev, root_mb, swap_mb, image_path).\
AndRaise(TestException)
utils.logout_iscsi(address, port, iqn)
self.mox.ReplayAll()
self.assertRaises(TestException,
utils.deploy,
address, port, iqn, lun, image_path,
pxe_config_path, root_mb, swap_mb)
class SwitchPxeConfigTestCase(tests_base.TestCase):
def setUp(self):
super(SwitchPxeConfigTestCase, self).setUp()
(fd, self.fname) = tempfile.mkstemp()
os.write(fd, _PXECONF_DEPLOY)
os.close(fd)
def tearDown(self):
os.unlink(self.fname)
super(SwitchPxeConfigTestCase, self).tearDown()
def test_switch_pxe_config(self):
utils.switch_pxe_config(self.fname,
'12345678-1234-1234-1234-1234567890abcdef')
with open(self.fname, 'r') as f:
pxeconf = f.read()
self.assertEqual(pxeconf, _PXECONF_BOOT)
class OtherFunctionTestCase(tests_base.TestCase):
def test_get_dev(self):
expected = '/dev/disk/by-path/ip-1.2.3.4:5678-iscsi-iqn.fake-lun-9'
actual = utils.get_dev('1.2.3.4', 5678, 'iqn.fake', 9)
self.assertEqual(expected, actual)
def test_get_image_mb(self):
mb = 1024 * 1024
size = None
def fake_getsize(path):
return size
self.stubs.Set(os.path, 'getsize', fake_getsize)
size = 0
self.assertEqual(utils.get_image_mb('x'), 0)
size = 1
self.assertEqual(utils.get_image_mb('x'), 1)
size = mb
self.assertEqual(utils.get_image_mb('x'), 1)
size = mb + 1
self.assertEqual(utils.get_image_mb('x'), 2)

View File

@ -35,6 +35,7 @@ from ironic.common import states
from ironic.common import utils
from ironic.conductor import task_manager
from ironic.db import api as dbapi
from ironic.drivers.modules import deploy_utils
from ironic.drivers.modules import pxe
from ironic.openstack.common import context
from ironic.openstack.common import fileutils
@ -415,7 +416,7 @@ class PXEDriverTestCase(db_base.DbTestCase):
def setUp(self):
super(PXEDriverTestCase, self).setUp()
self.driver = mgr_utils.get_mocked_node_manager(driver='fake_pxe')
mgr_utils.get_mocked_node_manager(driver='fake_pxe')
n = db_utils.get_test_node(
driver='fake_pxe',
driver_info=json.loads(db_utils.pxe_info),
@ -456,13 +457,18 @@ class PXEDriverTestCase(db_base.DbTestCase):
node_macs = pxe._get_node_mac_addresses(task, self.node)
self.assertEqual(node_macs, ['aa:bb:cc', 'dd:ee:ff'])
def test_deploy_good(self):
def test_vendor_passthru_validate_good(self):
with task_manager.acquire([self.node['uuid']], shared=True) as task:
task.resources[0].driver.vendor.validate(self.node,
method='pass_deploy_info', address='123456', iqn='aaa-bbb')
def refresh():
pass
self.node.refresh = refresh
def test_vendor_passthru_validate_fail(self):
with task_manager.acquire([self.node['uuid']], shared=True) as task:
self.assertRaises(exception.InvalidParameterValue,
task.resources[0].driver.vendor.validate,
self.node, method='pass_deploy_info')
def test_start_deploy(self):
self.mox.StubOutWithMock(pxe, '_create_pxe_config')
self.mox.StubOutWithMock(pxe, '_cache_images')
self.mox.StubOutWithMock(pxe, '_get_tftp_image_info')
@ -472,82 +478,34 @@ class PXEDriverTestCase(db_base.DbTestCase):
pxe._cache_images(self.node, None).AndReturn(None)
self.mox.ReplayAll()
class handler_deploying(threading.Thread):
def __init__(self, node):
threading.Thread.__init__(self)
self.node = node
def run(self):
self.node['provision_state'] = states.DEPLOYING
time.sleep(2)
self.node['provision_state'] = states.ACTIVE
handler = handler_deploying(self.node)
handler.start()
with task_manager.acquire([self.node['uuid']], shared=False) as task:
task.resources[0].driver.deploy.deploy(task, self.node)
state = task.resources[0].driver.deploy.deploy(task, self.node)
self.assertEqual(state, states.DEPLOYING)
self.mox.VerifyAll()
def test_deploy_fail(self):
def test_continue_deploy_good(self):
def refresh():
def fake_deploy(**kwargs):
pass
self.node.refresh = refresh
self.stubs.Set(deploy_utils, 'deploy', fake_deploy)
with task_manager.acquire([self.node['uuid']], shared=True) as task:
task.resources[0].driver.vendor.vendor_passthru(task, self.node,
method='pass_deploy_info', address='123456', iqn='aaa-bbb')
self.assertEqual(self.node['provision_state'], states.DEPLOYDONE)
self.mox.StubOutWithMock(pxe, '_create_pxe_config')
self.mox.StubOutWithMock(pxe, '_cache_images')
self.mox.StubOutWithMock(pxe, '_get_tftp_image_info')
pxe._get_tftp_image_info(self.node).AndReturn(None)
pxe._create_pxe_config(mox.IgnoreArg(), self.node, None).\
AndReturn(None)
pxe._cache_images(self.node, None).AndReturn(None)
self.mox.ReplayAll()
def test_continue_deploy_fail(self):
class handler_deploying(threading.Thread):
def __init__(self, node):
threading.Thread.__init__(self)
self.node = node
def fake_deploy(**kwargs):
raise exception.InstanceDeployFailure()
def run(self):
self.node['provision_state'] = states.DEPLOYING
time.sleep(2)
self.node['provision_state'] = states.DEPLOYFAIL
handler = handler_deploying(self.node)
handler.start()
with task_manager.acquire([self.node['uuid']], shared=False) as task:
self.stubs.Set(deploy_utils, 'deploy', fake_deploy)
with task_manager.acquire([self.node['uuid']], shared=True) as task:
self.assertRaises(exception.InstanceDeployFailure,
task.resources[0].driver.deploy.deploy,
task,
self.node)
self.mox.VerifyAll()
def test_deploy_timeout_fail(self):
def refresh():
pass
self.node.refresh = refresh
self.mox.StubOutWithMock(pxe, '_create_pxe_config')
self.mox.StubOutWithMock(pxe, '_cache_images')
self.mox.StubOutWithMock(pxe, '_get_tftp_image_info')
pxe._get_tftp_image_info(self.node).AndReturn(None)
pxe._create_pxe_config(mox.IgnoreArg(), self.node, None).\
AndReturn(None)
pxe._cache_images(self.node, None).AndReturn(None)
self.mox.ReplayAll()
CONF.set_default('pxe_deploy_timeout', 2, group='pxe')
with task_manager.acquire([self.node['uuid']], shared=False) as task:
self.assertRaises(exception.InstanceDeployFailure,
task.resources[0].driver.deploy.deploy,
task,
self.node)
self.mox.VerifyAll()
task.resources[0].driver.vendor.vendor_passthru,
task, self.node, method='pass_deploy_info',
address='123456', iqn='aaa-bbb')
self.assertEqual(self.node['provision_state'], states.DEPLOYFAIL)
def tear_down_config(self, master=None):
temp_dir = tempfile.mkdtemp()

View File

@ -1,269 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright (c) 2012 NTT DOCOMO, INC.
# Copyright 2011 OpenStack Foundation
# Copyright 2011 Ilya Alekseyev
#
# 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 tempfile
import testtools
import time
import mox
from ironic.cmd import ironic_deploy_helper as bmdh
from ironic import db
from ironic.openstack.common import log as logging
from ironic.tests import base as tests_base
from ironic.tests.db import base
bmdh.LOG = logging.getLogger('ironic.deploy_helper')
_PXECONF_DEPLOY = """
default deploy
label deploy
kernel deploy_kernel
append initrd=deploy_ramdisk
ipappend 3
label boot
kernel kernel
append initrd=ramdisk root=${ROOT}
"""
_PXECONF_BOOT = """
default boot
label deploy
kernel deploy_kernel
append initrd=deploy_ramdisk
ipappend 3
label boot
kernel kernel
append initrd=ramdisk root=UUID=12345678-1234-1234-1234-1234567890abcdef
"""
class WorkerTestCase(base.DbTestCase):
def setUp(self):
super(WorkerTestCase, self).setUp()
self.worker = bmdh.Worker()
# Make tearDown() fast
self.worker.queue_timeout = 0.1
self.worker.start()
def tearDown(self):
if self.worker.isAlive():
self.worker.stop = True
self.worker.join(timeout=1)
super(WorkerTestCase, self).tearDown()
def wait_queue_empty(self, timeout):
for _ in xrange(int(timeout / 0.1)):
if bmdh.QUEUE.empty():
break
time.sleep(0.1)
@testtools.skip("not compatible with Ironic db")
def test_run_calls_deploy(self):
"""Check all queued requests are passed to deploy()."""
history = []
def fake_deploy(**params):
history.append(params)
self.stubs.Set(bmdh, 'deploy', fake_deploy)
self.mox.StubOutWithMock(db, 'bm_node_update')
# update is called twice inside Worker.run
for i in range(6):
db.bm_node_update(mox.IgnoreArg(), mox.IgnoreArg(),
mox.IgnoreArg())
self.mox.ReplayAll()
params_list = [{'fake1': ''}, {'fake2': ''}, {'fake3': ''}]
for (dep_id, params) in enumerate(params_list):
bmdh.QUEUE.put((dep_id, params))
self.wait_queue_empty(1)
self.assertEqual(params_list, history)
self.mox.VerifyAll()
@testtools.skip("not compatible with Ironic db")
def test_run_with_failing_deploy(self):
"""Check a worker keeps on running even if deploy() raises
an exception.
"""
history = []
def fake_deploy(**params):
history.append(params)
# always fail
raise Exception('test')
self.stubs.Set(bmdh, 'deploy', fake_deploy)
self.mox.StubOutWithMock(db, 'bm_node_update')
# update is called twice inside Worker.run
for i in range(6):
db.bm_node_update(mox.IgnoreArg(), mox.IgnoreArg(),
mox.IgnoreArg())
self.mox.ReplayAll()
params_list = [{'fake1': ''}, {'fake2': ''}, {'fake3': ''}]
for (dep_id, params) in enumerate(params_list):
bmdh.QUEUE.put((dep_id, params))
self.wait_queue_empty(1)
self.assertEqual(params_list, history)
self.mox.VerifyAll()
class PhysicalWorkTestCase(tests_base.TestCase):
def setUp(self):
super(PhysicalWorkTestCase, self).setUp()
def noop(*args, **kwargs):
pass
self.stubs.Set(time, 'sleep', noop)
def test_deploy(self):
"""Check loosely all functions are called with right args."""
address = '127.0.0.1'
port = 3306
iqn = 'iqn.xyz'
lun = 1
image_path = '/tmp/xyz/image'
pxe_config_path = '/tmp/abc/pxeconfig'
root_mb = 128
swap_mb = 64
dev = '/dev/fake'
root_part = '/dev/fake-part1'
swap_part = '/dev/fake-part2'
root_uuid = '12345678-1234-1234-12345678-12345678abcdef'
self.mox.StubOutWithMock(bmdh, 'get_dev')
self.mox.StubOutWithMock(bmdh, 'get_image_mb')
self.mox.StubOutWithMock(bmdh, 'discovery')
self.mox.StubOutWithMock(bmdh, 'login_iscsi')
self.mox.StubOutWithMock(bmdh, 'logout_iscsi')
self.mox.StubOutWithMock(bmdh, 'make_partitions')
self.mox.StubOutWithMock(bmdh, 'is_block_device')
self.mox.StubOutWithMock(bmdh, 'dd')
self.mox.StubOutWithMock(bmdh, 'mkswap')
self.mox.StubOutWithMock(bmdh, 'block_uuid')
self.mox.StubOutWithMock(bmdh, 'switch_pxe_config')
self.mox.StubOutWithMock(bmdh, 'notify')
bmdh.get_dev(address, port, iqn, lun).AndReturn(dev)
bmdh.get_image_mb(image_path).AndReturn(1) # < root_mb
bmdh.discovery(address, port)
bmdh.login_iscsi(address, port, iqn)
bmdh.is_block_device(dev).AndReturn(True)
bmdh.make_partitions(dev, root_mb, swap_mb)
bmdh.is_block_device(root_part).AndReturn(True)
bmdh.is_block_device(swap_part).AndReturn(True)
bmdh.dd(image_path, root_part)
bmdh.mkswap(swap_part)
bmdh.block_uuid(root_part).AndReturn(root_uuid)
bmdh.logout_iscsi(address, port, iqn)
bmdh.switch_pxe_config(pxe_config_path, root_uuid)
bmdh.notify(address, 10000)
self.mox.ReplayAll()
bmdh.deploy(address, port, iqn, lun, image_path, pxe_config_path,
root_mb, swap_mb)
self.mox.VerifyAll()
def test_always_logout_iscsi(self):
"""logout_iscsi() must be called once login_iscsi() is called."""
address = '127.0.0.1'
port = 3306
iqn = 'iqn.xyz'
lun = 1
image_path = '/tmp/xyz/image'
pxe_config_path = '/tmp/abc/pxeconfig'
root_mb = 128
swap_mb = 64
dev = '/dev/fake'
self.mox.StubOutWithMock(bmdh, 'get_dev')
self.mox.StubOutWithMock(bmdh, 'get_image_mb')
self.mox.StubOutWithMock(bmdh, 'discovery')
self.mox.StubOutWithMock(bmdh, 'login_iscsi')
self.mox.StubOutWithMock(bmdh, 'logout_iscsi')
self.mox.StubOutWithMock(bmdh, 'work_on_disk')
class TestException(Exception):
pass
bmdh.get_dev(address, port, iqn, lun).AndReturn(dev)
bmdh.get_image_mb(image_path).AndReturn(1) # < root_mb
bmdh.discovery(address, port)
bmdh.login_iscsi(address, port, iqn)
bmdh.work_on_disk(dev, root_mb, swap_mb, image_path).\
AndRaise(TestException)
bmdh.logout_iscsi(address, port, iqn)
self.mox.ReplayAll()
self.assertRaises(TestException,
bmdh.deploy,
address, port, iqn, lun, image_path,
pxe_config_path, root_mb, swap_mb)
class SwitchPxeConfigTestCase(tests_base.TestCase):
def setUp(self):
super(SwitchPxeConfigTestCase, self).setUp()
(fd, self.fname) = tempfile.mkstemp()
os.write(fd, _PXECONF_DEPLOY)
os.close(fd)
def tearDown(self):
os.unlink(self.fname)
super(SwitchPxeConfigTestCase, self).tearDown()
def test_switch_pxe_config(self):
bmdh.switch_pxe_config(self.fname,
'12345678-1234-1234-1234-1234567890abcdef')
with open(self.fname, 'r') as f:
pxeconf = f.read()
self.assertEqual(pxeconf, _PXECONF_BOOT)
class OtherFunctionTestCase(tests_base.TestCase):
def test_get_dev(self):
expected = '/dev/disk/by-path/ip-1.2.3.4:5678-iscsi-iqn.fake-lun-9'
actual = bmdh.get_dev('1.2.3.4', 5678, 'iqn.fake', 9)
self.assertEqual(expected, actual)
def test_get_image_mb(self):
mb = 1024 * 1024
size = None
def fake_getsize(path):
return size
self.stubs.Set(os.path, 'getsize', fake_getsize)
size = 0
self.assertEqual(bmdh.get_image_mb('x'), 0)
size = 1
self.assertEqual(bmdh.get_image_mb('x'), 1)
size = mb
self.assertEqual(bmdh.get_image_mb('x'), 1)
size = mb + 1
self.assertEqual(bmdh.get_image_mb('x'), 2)