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:
parent
cc316aa439
commit
535426efaf
@ -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):
|
||||
|
@ -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()
|
@ -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)
|
||||
|
@ -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()
|
||||
|
192
ironic/tests/drivers/test_deploy_utils.py
Normal file
192
ironic/tests/drivers/test_deploy_utils.py
Normal 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)
|
@ -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()
|
||||
|
@ -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)
|
Loading…
x
Reference in New Issue
Block a user