Update cinder-volume to enable multi volume support
* Added the service wrapper for multi binscripts * Added a service_name to manager * Added the configuration wrapper so the impls do not need to know if they are accessing config values from a option group or from the DEFAULT group * Updated drivers to use the new configuration wrapper * Added an example config set for multi backend * Modified service to comply w/ common * Added get_volume_stats to iscsi * Added config for volume_backend_name implements blueprint multi-volume-backends Change-Id: Idb86ac5f1c06b9cd5061bef6a24f17727e9a58c1
This commit is contained in:
parent
4d6b1d4adb
commit
6c708d12f5
@ -39,10 +39,21 @@ from cinder.openstack.common import log as logging
|
||||
from cinder import service
|
||||
from cinder import utils
|
||||
|
||||
FLAGS = flags.FLAGS
|
||||
|
||||
if __name__ == '__main__':
|
||||
flags.parse_args(sys.argv)
|
||||
logging.setup("cinder")
|
||||
utils.monkey_patch()
|
||||
server = service.Service.create(binary='cinder-volume')
|
||||
service.serve(server)
|
||||
service.wait()
|
||||
launcher = service.ProcessLauncher()
|
||||
if FLAGS.enabled_backends:
|
||||
for backend in FLAGS.enabled_backends:
|
||||
host = "%s@%s" % (FLAGS.host, backend)
|
||||
server = service.Service.create(
|
||||
host=host,
|
||||
service_name=backend)
|
||||
launcher.launch_server(server)
|
||||
else:
|
||||
server = service.Service.create(binary='cinder-volume')
|
||||
launcher.launch_server(server)
|
||||
launcher.wait()
|
||||
|
@ -222,6 +222,11 @@ global_opts = [
|
||||
'and deprecated.'),
|
||||
cfg.StrOpt('control_exchange',
|
||||
default='cinder',
|
||||
help='AMQP exchange to connect to if using RabbitMQ or Qpid'), ]
|
||||
help='AMQP exchange to connect to if using RabbitMQ or Qpid'),
|
||||
cfg.ListOpt('enabled_backends',
|
||||
default=None,
|
||||
help='A list of backend names to use. These backend names '
|
||||
'should be backed by a unique [CONFIG] group '
|
||||
'with its options'), ]
|
||||
|
||||
FLAGS.register_opts(global_opts)
|
||||
|
@ -50,7 +50,8 @@ class SchedulerManager(manager.Manager):
|
||||
|
||||
RPC_API_VERSION = '1.2'
|
||||
|
||||
def __init__(self, scheduler_driver=None, *args, **kwargs):
|
||||
def __init__(self, scheduler_driver=None, service_name=None,
|
||||
*args, **kwargs):
|
||||
if not scheduler_driver:
|
||||
scheduler_driver = FLAGS.scheduler_driver
|
||||
self.driver = importutils.import_object(scheduler_driver)
|
||||
|
@ -19,10 +19,13 @@
|
||||
|
||||
"""Generic Node base class for all workers that run on hosts."""
|
||||
|
||||
import errno
|
||||
import inspect
|
||||
import os
|
||||
import random
|
||||
import signal
|
||||
import sys
|
||||
import time
|
||||
|
||||
import eventlet
|
||||
import greenlet
|
||||
@ -65,6 +68,12 @@ FLAGS = flags.FLAGS
|
||||
FLAGS.register_opts(service_opts)
|
||||
|
||||
|
||||
class SignalExit(SystemExit):
|
||||
def __init__(self, signo, exccode=1):
|
||||
super(SignalExit, self).__init__(exccode)
|
||||
self.signo = signo
|
||||
|
||||
|
||||
class Launcher(object):
|
||||
"""Launch one or more services and wait for them to complete."""
|
||||
|
||||
@ -128,6 +137,180 @@ class Launcher(object):
|
||||
pass
|
||||
|
||||
|
||||
class ServerWrapper(object):
|
||||
def __init__(self, server, workers):
|
||||
self.server = server
|
||||
self.workers = workers
|
||||
self.children = set()
|
||||
self.forktimes = []
|
||||
|
||||
|
||||
class ProcessLauncher(object):
|
||||
def __init__(self):
|
||||
self.children = {}
|
||||
self.sigcaught = None
|
||||
self.running = True
|
||||
rfd, self.writepipe = os.pipe()
|
||||
self.readpipe = eventlet.greenio.GreenPipe(rfd, 'r')
|
||||
|
||||
signal.signal(signal.SIGTERM, self._handle_signal)
|
||||
signal.signal(signal.SIGINT, self._handle_signal)
|
||||
|
||||
def _handle_signal(self, signo, frame):
|
||||
self.sigcaught = signo
|
||||
self.running = False
|
||||
|
||||
# Allow the process to be killed again and die from natural causes
|
||||
signal.signal(signal.SIGTERM, signal.SIG_DFL)
|
||||
signal.signal(signal.SIGINT, signal.SIG_DFL)
|
||||
|
||||
def _pipe_watcher(self):
|
||||
# This will block until the write end is closed when the parent
|
||||
# dies unexpectedly
|
||||
self.readpipe.read()
|
||||
|
||||
LOG.info(_('Parent process has died unexpectedly, exiting'))
|
||||
|
||||
sys.exit(1)
|
||||
|
||||
def _child_process(self, server):
|
||||
# Setup child signal handlers differently
|
||||
def _sigterm(*args):
|
||||
signal.signal(signal.SIGTERM, signal.SIG_DFL)
|
||||
raise SignalExit(signal.SIGTERM)
|
||||
|
||||
signal.signal(signal.SIGTERM, _sigterm)
|
||||
# Block SIGINT and let the parent send us a SIGTERM
|
||||
# signal.signal(signal.SIGINT, signal.SIG_IGN)
|
||||
# This differs from the behavior in nova in that we dont ignore this
|
||||
# It allows the non-wsgi services to be terminated properly
|
||||
signal.signal(signal.SIGINT, _sigterm)
|
||||
|
||||
# Reopen the eventlet hub to make sure we don't share an epoll
|
||||
# fd with parent and/or siblings, which would be bad
|
||||
eventlet.hubs.use_hub()
|
||||
|
||||
# Close write to ensure only parent has it open
|
||||
os.close(self.writepipe)
|
||||
# Create greenthread to watch for parent to close pipe
|
||||
eventlet.spawn(self._pipe_watcher)
|
||||
|
||||
# Reseed random number generator
|
||||
random.seed()
|
||||
|
||||
launcher = Launcher()
|
||||
launcher.run_server(server)
|
||||
|
||||
def _start_child(self, wrap):
|
||||
if len(wrap.forktimes) > wrap.workers:
|
||||
# Limit ourselves to one process a second (over the period of
|
||||
# number of workers * 1 second). This will allow workers to
|
||||
# start up quickly but ensure we don't fork off children that
|
||||
# die instantly too quickly.
|
||||
if time.time() - wrap.forktimes[0] < wrap.workers:
|
||||
LOG.info(_('Forking too fast, sleeping'))
|
||||
time.sleep(1)
|
||||
|
||||
wrap.forktimes.pop(0)
|
||||
|
||||
wrap.forktimes.append(time.time())
|
||||
|
||||
pid = os.fork()
|
||||
if pid == 0:
|
||||
# NOTE(johannes): All exceptions are caught to ensure this
|
||||
# doesn't fallback into the loop spawning children. It would
|
||||
# be bad for a child to spawn more children.
|
||||
status = 0
|
||||
try:
|
||||
self._child_process(wrap.server)
|
||||
except SignalExit as exc:
|
||||
signame = {signal.SIGTERM: 'SIGTERM',
|
||||
signal.SIGINT: 'SIGINT'}[exc.signo]
|
||||
LOG.info(_('Caught %s, exiting'), signame)
|
||||
status = exc.code
|
||||
except SystemExit as exc:
|
||||
status = exc.code
|
||||
except BaseException:
|
||||
LOG.exception(_('Unhandled exception'))
|
||||
status = 2
|
||||
finally:
|
||||
wrap.server.stop()
|
||||
|
||||
os._exit(status)
|
||||
|
||||
LOG.info(_('Started child %d'), pid)
|
||||
|
||||
wrap.children.add(pid)
|
||||
self.children[pid] = wrap
|
||||
|
||||
return pid
|
||||
|
||||
def launch_server(self, server, workers=1):
|
||||
wrap = ServerWrapper(server, workers)
|
||||
|
||||
LOG.info(_('Starting %d workers'), wrap.workers)
|
||||
while self.running and len(wrap.children) < wrap.workers:
|
||||
self._start_child(wrap)
|
||||
|
||||
def _wait_child(self):
|
||||
try:
|
||||
# Don't block if no child processes have exited
|
||||
pid, status = os.waitpid(0, os.WNOHANG)
|
||||
if not pid:
|
||||
return None
|
||||
except OSError as exc:
|
||||
if exc.errno not in (errno.EINTR, errno.ECHILD):
|
||||
raise
|
||||
return None
|
||||
|
||||
if os.WIFSIGNALED(status):
|
||||
sig = os.WTERMSIG(status)
|
||||
LOG.info(_('Child %(pid)d killed by signal %(sig)d'), locals())
|
||||
else:
|
||||
code = os.WEXITSTATUS(status)
|
||||
LOG.info(_('Child %(pid)d exited with status %(code)d'), locals())
|
||||
|
||||
if pid not in self.children:
|
||||
LOG.warning(_('pid %d not in child list'), pid)
|
||||
return None
|
||||
|
||||
wrap = self.children.pop(pid)
|
||||
wrap.children.remove(pid)
|
||||
return wrap
|
||||
|
||||
def wait(self):
|
||||
"""Loop waiting on children to die and respawning as necessary."""
|
||||
while self.running:
|
||||
wrap = self._wait_child()
|
||||
if not wrap:
|
||||
# Yield to other threads if no children have exited
|
||||
# Sleep for a short time to avoid excessive CPU usage
|
||||
# (see bug #1095346)
|
||||
eventlet.greenthread.sleep(.01)
|
||||
continue
|
||||
|
||||
while self.running and len(wrap.children) < wrap.workers:
|
||||
self._start_child(wrap)
|
||||
|
||||
if self.sigcaught:
|
||||
signame = {signal.SIGTERM: 'SIGTERM',
|
||||
signal.SIGINT: 'SIGINT'}[self.sigcaught]
|
||||
LOG.info(_('Caught %s, stopping children'), signame)
|
||||
|
||||
for pid in self.children:
|
||||
try:
|
||||
os.kill(pid, signal.SIGTERM)
|
||||
except OSError as exc:
|
||||
if exc.errno != errno.ESRCH:
|
||||
raise
|
||||
|
||||
# Wait for children to die
|
||||
if self.children:
|
||||
LOG.info(_('Waiting on %d children to exit'), len(self.children))
|
||||
while self.children:
|
||||
self._wait_child()
|
||||
|
||||
|
||||
class Service(object):
|
||||
"""Service object for binaries running on hosts.
|
||||
|
||||
@ -137,13 +320,15 @@ class Service(object):
|
||||
|
||||
def __init__(self, host, binary, topic, manager, report_interval=None,
|
||||
periodic_interval=None, periodic_fuzzy_delay=None,
|
||||
*args, **kwargs):
|
||||
service_name=None, *args, **kwargs):
|
||||
self.host = host
|
||||
self.binary = binary
|
||||
self.topic = topic
|
||||
self.manager_class_name = manager
|
||||
manager_class = importutils.import_class(self.manager_class_name)
|
||||
self.manager = manager_class(host=self.host, *args, **kwargs)
|
||||
self.manager = manager_class(host=self.host,
|
||||
service_name=service_name,
|
||||
*args, **kwargs)
|
||||
self.report_interval = report_interval
|
||||
self.periodic_interval = periodic_interval
|
||||
self.periodic_fuzzy_delay = periodic_fuzzy_delay
|
||||
@ -217,7 +402,7 @@ class Service(object):
|
||||
@classmethod
|
||||
def create(cls, host=None, binary=None, topic=None, manager=None,
|
||||
report_interval=None, periodic_interval=None,
|
||||
periodic_fuzzy_delay=None):
|
||||
periodic_fuzzy_delay=None, service_name=None):
|
||||
"""Instantiates class and passes back application object.
|
||||
|
||||
:param host: defaults to FLAGS.host
|
||||
@ -247,7 +432,8 @@ class Service(object):
|
||||
service_obj = cls(host, binary, topic, manager,
|
||||
report_interval=report_interval,
|
||||
periodic_interval=periodic_interval,
|
||||
periodic_fuzzy_delay=periodic_fuzzy_delay)
|
||||
periodic_fuzzy_delay=periodic_fuzzy_delay,
|
||||
service_name=service_name)
|
||||
|
||||
return service_obj
|
||||
|
||||
|
@ -11,9 +11,13 @@
|
||||
# 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 mox
|
||||
|
||||
from cinder import exception
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import test
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers.san.hp_lefthand import HpSanISCSIDriver
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
@ -27,7 +31,11 @@ class HpSanISCSITestCase(test.TestCase):
|
||||
self._fake_cliq_run)
|
||||
self.stubs.Set(HpSanISCSIDriver, "_get_iscsi_properties",
|
||||
self._fake_get_iscsi_properties)
|
||||
self.driver = HpSanISCSIDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.san_is_local = False
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
self.driver = HpSanISCSIDriver(configuration=configuration)
|
||||
self.volume_name = "fakevolume"
|
||||
self.connector = {'ip': '10.0.0.2',
|
||||
'initiator': 'iqn.1993-08.org.debian:01:222',
|
||||
|
@ -15,9 +15,12 @@
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
import mox
|
||||
|
||||
from cinder import exception
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import test
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers import coraid
|
||||
from cinder.volume.drivers.coraid import CoraidDriver
|
||||
from cinder.volume.drivers.coraid import CoraidRESTClient
|
||||
@ -91,7 +94,13 @@ class TestCoraidDriver(test.TestCase):
|
||||
self.esm_mock = self.mox.CreateMockAnything()
|
||||
self.stubs.Set(coraid, 'CoraidRESTClient',
|
||||
lambda *_, **__: self.esm_mock)
|
||||
self.drv = CoraidDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
configuration.coraid_esm_address = fake_esm_ipaddress
|
||||
configuration.coraid_user = fake_esm_username
|
||||
configuration.coraid_password = fake_esm_password
|
||||
|
||||
self.drv = CoraidDriver(configuration=configuration)
|
||||
self.drv.do_setup({})
|
||||
|
||||
def test_create_volume(self):
|
||||
|
@ -16,6 +16,7 @@
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
import mox
|
||||
import os
|
||||
from xml.dom.minidom import Document
|
||||
|
||||
@ -23,11 +24,11 @@ from cinder import exception
|
||||
from cinder import flags
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import test
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers.emc.emc_smis_common import EMCSMISCommon
|
||||
from cinder.volume.drivers.emc.emc_smis_iscsi import EMCSMISISCSIDriver
|
||||
|
||||
FLAGS = flags.FLAGS
|
||||
|
||||
CINDER_EMC_CONFIG_FILE = '/etc/cinder/cinder_emc_config.xml'
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
config_file_name = 'cinder_emc_config.xml'
|
||||
@ -579,12 +580,16 @@ class EMCSMISISCSIDriverTestCase(test.TestCase):
|
||||
super(EMCSMISISCSIDriverTestCase, self).setUp()
|
||||
self.config_file_path = None
|
||||
self.create_fake_config_file()
|
||||
FLAGS.cinder_emc_config_file = self.config_file_path
|
||||
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.cinder_emc_config_file = self.config_file_path
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
self.stubs.Set(EMCSMISISCSIDriver, '_get_iscsi_properties',
|
||||
self.fake_get_iscsi_properties)
|
||||
self.stubs.Set(EMCSMISCommon, '_get_ecom_connection',
|
||||
self.fake_ecom_connection)
|
||||
driver = EMCSMISISCSIDriver()
|
||||
driver = EMCSMISISCSIDriver(configuration=configuration)
|
||||
self.driver = driver
|
||||
|
||||
def create_fake_config_file(self):
|
||||
|
@ -30,6 +30,7 @@ from cinder import exception
|
||||
from cinder.exception import ProcessExecutionError
|
||||
from cinder import test
|
||||
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers import glusterfs
|
||||
|
||||
|
||||
@ -57,9 +58,19 @@ class GlusterFsDriverTestCase(test.TestCase):
|
||||
ONE_GB_IN_BYTES = 1024 * 1024 * 1024
|
||||
|
||||
def setUp(self):
|
||||
self._driver = glusterfs.GlusterfsDriver()
|
||||
self._mox = mox_lib.Mox()
|
||||
self._configuration = mox_lib.MockObject(conf.Configuration)
|
||||
self._configuration.append_config_values(mox_lib.IgnoreArg())
|
||||
self._configuration.glusterfs_shares_config = \
|
||||
self.TEST_SHARES_CONFIG_FILE
|
||||
self._configuration.glusterfs_mount_point_base = \
|
||||
self.TEST_MNT_POINT_BASE
|
||||
self._configuration.glusterfs_disk_util = 'df'
|
||||
self._configuration.glusterfs_sparsed_volumes = True
|
||||
|
||||
self.stubs = stubout.StubOutForTesting()
|
||||
self._driver = glusterfs.GlusterfsDriver(
|
||||
configuration=self._configuration)
|
||||
|
||||
def tearDown(self):
|
||||
self._mox.UnsetStubs()
|
||||
@ -276,7 +287,8 @@ class GlusterFsDriverTestCase(test.TestCase):
|
||||
mox = self._mox
|
||||
drv = self._driver
|
||||
|
||||
setattr(glusterfs.FLAGS, 'glusterfs_disk_util', 'du')
|
||||
old_value = self._configuration.glusterfs_disk_util
|
||||
self._configuration.glusterfs_disk_util = 'du'
|
||||
|
||||
df_total_size = 2620544
|
||||
df_used_size = 996864
|
||||
@ -313,7 +325,7 @@ class GlusterFsDriverTestCase(test.TestCase):
|
||||
|
||||
mox.VerifyAll()
|
||||
|
||||
delattr(glusterfs.FLAGS, 'glusterfs_disk_util')
|
||||
self._configuration.glusterfs_disk_util = old_value
|
||||
|
||||
def test_load_shares_config(self):
|
||||
mox = self._mox
|
||||
@ -505,7 +517,8 @@ class GlusterFsDriverTestCase(test.TestCase):
|
||||
drv = self._driver
|
||||
volume = self._simple_volume()
|
||||
|
||||
setattr(glusterfs.FLAGS, 'glusterfs_sparsed_volumes', False)
|
||||
old_value = self._configuration.glusterfs_sparsed_volumes
|
||||
self._configuration.glusterfs_sparsed_volumes = False
|
||||
|
||||
mox.StubOutWithMock(drv, '_create_regular_file')
|
||||
mox.StubOutWithMock(drv, '_set_rw_permissions_for_all')
|
||||
@ -519,7 +532,7 @@ class GlusterFsDriverTestCase(test.TestCase):
|
||||
|
||||
mox.VerifyAll()
|
||||
|
||||
delattr(glusterfs.FLAGS, 'glusterfs_sparsed_volumes')
|
||||
self._configuration.glusterfs_sparsed_volumes = old_value
|
||||
|
||||
def test_create_volume_should_ensure_glusterfs_mounted(self):
|
||||
"""create_volume ensures shares provided in config are mounted."""
|
||||
|
@ -20,6 +20,7 @@ from cinder import context
|
||||
from cinder import exception
|
||||
from cinder import test
|
||||
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers.netapp import api
|
||||
from cinder.volume.drivers.netapp import iscsi
|
||||
from cinder.volume.drivers.netapp import nfs as netapp_nfs
|
||||
@ -34,6 +35,12 @@ import suds
|
||||
import types
|
||||
|
||||
|
||||
def create_configuration():
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
return configuration
|
||||
|
||||
|
||||
class FakeVolume(object):
|
||||
def __init__(self, size=0):
|
||||
self.size = size
|
||||
@ -72,8 +79,9 @@ class NetappNfsDriverTestCase(test.TestCase):
|
||||
"""Test case for NetApp specific NFS clone driver."""
|
||||
|
||||
def setUp(self):
|
||||
self._driver = netapp_nfs.NetAppNFSDriver()
|
||||
self._mox = mox.Mox()
|
||||
self._driver = netapp_nfs.NetAppNFSDriver(
|
||||
configuration=create_configuration())
|
||||
|
||||
def tearDown(self):
|
||||
self._mox.UnsetStubs()
|
||||
@ -278,7 +286,8 @@ class NetappCmodeNfsDriverTestCase(test.TestCase):
|
||||
self._custom_setup()
|
||||
|
||||
def _custom_setup(self):
|
||||
self._driver = netapp_nfs.NetAppCmodeNfsDriver()
|
||||
self._driver = netapp_nfs.NetAppCmodeNfsDriver(
|
||||
configuration=create_configuration())
|
||||
|
||||
def tearDown(self):
|
||||
self._mox.UnsetStubs()
|
||||
@ -460,7 +469,8 @@ class NetappCmodeNfsDriverTestCase(test.TestCase):
|
||||
class NetappDirectCmodeNfsDriverTestCase(NetappCmodeNfsDriverTestCase):
|
||||
"""Test direct NetApp C Mode driver"""
|
||||
def _custom_setup(self):
|
||||
self._driver = netapp_nfs.NetAppDirectCmodeNfsDriver()
|
||||
self._driver = netapp_nfs.NetAppDirectCmodeNfsDriver(
|
||||
configuration=create_configuration())
|
||||
|
||||
def test_check_for_setup_error(self):
|
||||
mox = self._mox
|
||||
@ -579,7 +589,8 @@ class NetappDirectCmodeNfsDriverTestCase(NetappCmodeNfsDriverTestCase):
|
||||
class NetappDirect7modeNfsDriverTestCase(NetappDirectCmodeNfsDriverTestCase):
|
||||
"""Test direct NetApp C Mode driver"""
|
||||
def _custom_setup(self):
|
||||
self._driver = netapp_nfs.NetAppDirect7modeNfsDriver()
|
||||
self._driver = netapp_nfs.NetAppDirect7modeNfsDriver(
|
||||
configuration=create_configuration())
|
||||
|
||||
def test_check_for_setup_error(self):
|
||||
mox = self._mox
|
||||
|
@ -30,6 +30,7 @@ from cinder import exception
|
||||
from cinder.exception import ProcessExecutionError
|
||||
from cinder import test
|
||||
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers import nfs
|
||||
|
||||
|
||||
@ -148,9 +149,15 @@ class NfsDriverTestCase(test.TestCase):
|
||||
ONE_GB_IN_BYTES = 1024 * 1024 * 1024
|
||||
|
||||
def setUp(self):
|
||||
self._driver = nfs.NfsDriver()
|
||||
self._mox = mox_lib.Mox()
|
||||
self.stubs = stubout.StubOutForTesting()
|
||||
self.configuration = mox_lib.MockObject(conf.Configuration)
|
||||
self.configuration.append_config_values(mox_lib.IgnoreArg())
|
||||
self.configuration.nfs_shares_config = None
|
||||
self.configuration.nfs_mount_point_base = '$state_path/mnt'
|
||||
self.configuration.nfs_disk_util = 'df'
|
||||
self.configuration.nfs_sparsed_volumes = True
|
||||
self._driver = nfs.NfsDriver(configuration=self.configuration)
|
||||
|
||||
def tearDown(self):
|
||||
self._mox.UnsetStubs()
|
||||
@ -196,7 +203,7 @@ class NfsDriverTestCase(test.TestCase):
|
||||
|
||||
def test_local_path(self):
|
||||
"""local_path common use case."""
|
||||
nfs.FLAGS.nfs_mount_point_base = self.TEST_MNT_POINT_BASE
|
||||
self.configuration.nfs_mount_point_base = self.TEST_MNT_POINT_BASE
|
||||
drv = self._driver
|
||||
|
||||
volume = DumbVolume()
|
||||
@ -318,7 +325,7 @@ class NfsDriverTestCase(test.TestCase):
|
||||
"""_get_mount_point_for_share should calculate correct value."""
|
||||
drv = self._driver
|
||||
|
||||
nfs.FLAGS.nfs_mount_point_base = self.TEST_MNT_POINT_BASE
|
||||
self.configuration.nfs_mount_point_base = self.TEST_MNT_POINT_BASE
|
||||
|
||||
self.assertEqual('/mnt/test/2f4f60214cf43c595666dd815f0360a4',
|
||||
drv._get_mount_point_for_share(self.TEST_NFS_EXPORT1))
|
||||
@ -333,7 +340,7 @@ class NfsDriverTestCase(test.TestCase):
|
||||
df_data = 'nfs-host:/export 2620544 996864 %d 41%% /mnt' % df_avail
|
||||
df_output = df_head + df_data
|
||||
|
||||
setattr(nfs.FLAGS, 'nfs_disk_util', 'df')
|
||||
self.configuration.nfs_disk_util = 'df'
|
||||
|
||||
mox.StubOutWithMock(drv, '_get_mount_point_for_share')
|
||||
drv._get_mount_point_for_share(self.TEST_NFS_EXPORT1).\
|
||||
@ -350,14 +357,11 @@ class NfsDriverTestCase(test.TestCase):
|
||||
|
||||
mox.VerifyAll()
|
||||
|
||||
delattr(nfs.FLAGS, 'nfs_disk_util')
|
||||
|
||||
def test_get_available_capacity_with_du(self):
|
||||
"""_get_available_capacity should calculate correct value."""
|
||||
mox = self._mox
|
||||
drv = self._driver
|
||||
|
||||
setattr(nfs.FLAGS, 'nfs_disk_util', 'du')
|
||||
self.configuration.nfs_disk_util = 'du'
|
||||
|
||||
df_total_size = 2620544
|
||||
df_used_size = 996864
|
||||
@ -391,13 +395,11 @@ class NfsDriverTestCase(test.TestCase):
|
||||
|
||||
mox.VerifyAll()
|
||||
|
||||
delattr(nfs.FLAGS, 'nfs_disk_util')
|
||||
|
||||
def test_load_shares_config(self):
|
||||
mox = self._mox
|
||||
drv = self._driver
|
||||
|
||||
nfs.FLAGS.nfs_shares_config = self.TEST_SHARES_CONFIG_FILE
|
||||
self.configuration.nfs_shares_config = self.TEST_SHARES_CONFIG_FILE
|
||||
|
||||
mox.StubOutWithMock(__builtin__, 'open')
|
||||
config_data = []
|
||||
@ -481,7 +483,7 @@ class NfsDriverTestCase(test.TestCase):
|
||||
"""do_setup should throw error if nfs client is not installed."""
|
||||
mox = self._mox
|
||||
drv = self._driver
|
||||
|
||||
self.configuration.nfs_shares_config = self.TEST_SHARES_CONFIG_FILE
|
||||
nfs.FLAGS.nfs_shares_config = self.TEST_SHARES_CONFIG_FILE
|
||||
|
||||
mox.StubOutWithMock(os.path, 'exists')
|
||||
@ -578,6 +580,7 @@ class NfsDriverTestCase(test.TestCase):
|
||||
def test_create_nonsparsed_volume(self):
|
||||
mox = self._mox
|
||||
drv = self._driver
|
||||
self.configuration.nfs_sparsed_volumes = False
|
||||
volume = self._simple_volume()
|
||||
|
||||
setattr(nfs.FLAGS, 'nfs_sparsed_volumes', False)
|
||||
|
@ -16,6 +16,7 @@
|
||||
# under the License.
|
||||
|
||||
import contextlib
|
||||
import mox
|
||||
import os
|
||||
import tempfile
|
||||
|
||||
@ -27,6 +28,7 @@ from cinder.openstack.common import timeutils
|
||||
from cinder import test
|
||||
from cinder.tests.image import fake as fake_image
|
||||
from cinder.tests.test_volume import DriverTestCase
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers.rbd import RBDDriver
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
@ -44,7 +46,17 @@ class RBDTestCase(test.TestCase):
|
||||
|
||||
def fake_execute(*args, **kwargs):
|
||||
return '', ''
|
||||
self.driver = RBDDriver(execute=fake_execute)
|
||||
self._mox = mox.Mox()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.volume_tmp_dir = None
|
||||
configuration.rbd_pool = 'rbd'
|
||||
configuration.rbd_secret_uuid = None
|
||||
configuration.rbd_user = None
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
self.driver = RBDDriver(execute=fake_execute,
|
||||
configuration=configuration)
|
||||
self._mox.ReplayAll()
|
||||
|
||||
def test_good_locations(self):
|
||||
locations = ['rbd://fsid/pool/image/snap',
|
||||
|
@ -49,6 +49,11 @@ flags.FLAGS.register_opts(test_service_opts)
|
||||
|
||||
class FakeManager(manager.Manager):
|
||||
"""Fake manager for tests"""
|
||||
def __init__(self, host=None,
|
||||
db_driver=None, service_name=None):
|
||||
super(FakeManager, self).__init__(host=host,
|
||||
db_driver=db_driver)
|
||||
|
||||
def test_method(self):
|
||||
return 'manager'
|
||||
|
||||
|
@ -15,14 +15,24 @@
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
import mox
|
||||
|
||||
from cinder import exception
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import test
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers.solidfire import SolidFire
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
|
||||
def create_configuration():
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.san_is_local = False
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
return configuration
|
||||
|
||||
|
||||
class SolidFireVolumeTestCase(test.TestCase):
|
||||
def setUp(self):
|
||||
super(SolidFireVolumeTestCase, self).setUp()
|
||||
@ -124,7 +134,8 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'size': 1,
|
||||
'id': 'a720b3c0-d1f0-11e1-9b23-0800200c9a66',
|
||||
'volume_type_id': 'fast'}
|
||||
sfv = SolidFire()
|
||||
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
model_update = sfv.create_volume(testvol)
|
||||
self.assertNotEqual(model_update, None)
|
||||
|
||||
@ -136,7 +147,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'size': 1,
|
||||
'id': 'a720b3c0-d1f0-11e1-9b23-0800200c9a66',
|
||||
'volume_type_id': None}
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
model_update = sfv.create_volume(testvol)
|
||||
self.assertNotEqual(model_update, None)
|
||||
|
||||
@ -153,7 +164,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'metadata': [preset_qos],
|
||||
'volume_type_id': None}
|
||||
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
model_update = sfv.create_volume(testvol)
|
||||
self.assertNotEqual(model_update, None)
|
||||
|
||||
@ -168,7 +179,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'name': 'testvol',
|
||||
'size': 1,
|
||||
'id': 'a720b3c0-d1f0-11e1-9b23-0800200c9a66'}
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
try:
|
||||
sfv.create_volume(testvol)
|
||||
self.fail("Should have thrown Error")
|
||||
@ -176,28 +187,28 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
pass
|
||||
|
||||
def test_create_sfaccount(self):
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
self.stubs.Set(SolidFire, '_issue_api_request',
|
||||
self.fake_issue_api_request)
|
||||
account = sfv._create_sfaccount('project-id')
|
||||
self.assertNotEqual(account, None)
|
||||
|
||||
def test_create_sfaccount_fails(self):
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
self.stubs.Set(SolidFire, '_issue_api_request',
|
||||
self.fake_issue_api_request_fails)
|
||||
account = sfv._create_sfaccount('project-id')
|
||||
self.assertEqual(account, None)
|
||||
|
||||
def test_get_sfaccount_by_name(self):
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
self.stubs.Set(SolidFire, '_issue_api_request',
|
||||
self.fake_issue_api_request)
|
||||
account = sfv._get_sfaccount_by_name('some-name')
|
||||
self.assertNotEqual(account, None)
|
||||
|
||||
def test_get_sfaccount_by_name_fails(self):
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
self.stubs.Set(SolidFire, '_issue_api_request',
|
||||
self.fake_issue_api_request_fails)
|
||||
account = sfv._get_sfaccount_by_name('some-name')
|
||||
@ -210,7 +221,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'name': 'test_volume',
|
||||
'size': 1,
|
||||
'id': 'a720b3c0-d1f0-11e1-9b23-0800200c9a66'}
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
sfv.delete_volume(testvol)
|
||||
|
||||
def test_delete_volume_fails_no_volume(self):
|
||||
@ -220,7 +231,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'name': 'no-name',
|
||||
'size': 1,
|
||||
'id': 'a720b3c0-d1f0-11e1-9b23-0800200c9a66'}
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
try:
|
||||
sfv.delete_volume(testvol)
|
||||
self.fail("Should have thrown Error")
|
||||
@ -238,7 +249,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
'name': 'no-name',
|
||||
'size': 1,
|
||||
'id': 'a720b3c0-d1f0-11e1-9b23-0800200c9a66'}
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
self.assertRaises(exception.SfAccountNotFound,
|
||||
sfv.delete_volume,
|
||||
testvol)
|
||||
@ -246,7 +257,7 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
def test_get_cluster_info(self):
|
||||
self.stubs.Set(SolidFire, '_issue_api_request',
|
||||
self.fake_issue_api_request)
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
sfv._get_cluster_info()
|
||||
|
||||
def test_get_cluster_info_fail(self):
|
||||
@ -256,6 +267,6 @@ class SolidFireVolumeTestCase(test.TestCase):
|
||||
self.fake_update_cluster_status)
|
||||
self.stubs.Set(SolidFire, '_issue_api_request',
|
||||
self.fake_issue_api_request_fails)
|
||||
sfv = SolidFire()
|
||||
sfv = SolidFire(configuration=create_configuration())
|
||||
self.assertRaises(exception.SolidFireAPIException,
|
||||
sfv._get_cluster_info)
|
||||
|
@ -24,6 +24,7 @@
|
||||
Tests for the IBM Storwize V7000 and SVC volume driver.
|
||||
"""
|
||||
|
||||
import mox
|
||||
import random
|
||||
import socket
|
||||
|
||||
@ -32,6 +33,7 @@ from cinder import flags
|
||||
from cinder.openstack.common import excutils
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import test
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers import storwize_svc
|
||||
|
||||
FLAGS = flags.FLAGS
|
||||
@ -962,7 +964,11 @@ class StorwizeSVCDriverTestCase(test.TestCase):
|
||||
storwize_svc_flashcopy_timeout="20",
|
||||
)
|
||||
self.sim = StorwizeSVCManagementSimulator("volpool")
|
||||
self.driver = StorwizeSVCFakeDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.san_is_local = False
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
self.driver = StorwizeSVCFakeDriver(configuration=configuration)
|
||||
self.driver.set_fake_storage(self.sim)
|
||||
else:
|
||||
self.flags(
|
||||
|
@ -41,6 +41,7 @@ from cinder import quota
|
||||
from cinder import test
|
||||
from cinder.tests import fake_flags
|
||||
from cinder.tests.image import fake as fake_image
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume import driver
|
||||
from cinder.volume import iscsi
|
||||
|
||||
@ -942,7 +943,11 @@ class ISCSITestCase(DriverTestCase):
|
||||
return volume_id_list
|
||||
|
||||
def test_do_iscsi_discovery(self):
|
||||
iscsi_driver = driver.ISCSIDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.iscsi_ip_address = '0.0.0.0'
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
iscsi_driver = driver.ISCSIDriver(configuration=configuration)
|
||||
iscsi_driver._execute = lambda *a, **kw: \
|
||||
("%s dummy" % FLAGS.iscsi_ip_address, '')
|
||||
volume = {"name": "dummy",
|
||||
|
71
cinder/tests/test_volume_configuration.py
Normal file
71
cinder/tests/test_volume_configuration.py
Normal file
@ -0,0 +1,71 @@
|
||||
#!/usr/bin/env python
|
||||
# vim: tabstop=4 shiftwidth=4 softtabstop=4
|
||||
#
|
||||
# Copyright (c) 2012 Rackspace Hosting
|
||||
# All Rights Reserved.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
"""Tests for the configuration wrapper in volume drivers."""
|
||||
|
||||
from cinder import flags
|
||||
from cinder.openstack.common import cfg
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import test
|
||||
from cinder.volume import configuration
|
||||
from cinder.volume import driver
|
||||
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
FLAGS = flags.FLAGS
|
||||
|
||||
|
||||
volume_opts = [
|
||||
cfg.StrOpt('str_opt', default='STR_OPT'),
|
||||
cfg.BoolOpt('bool_opt', default=False)
|
||||
]
|
||||
more_volume_opts = [
|
||||
cfg.IntOpt('int_opt', default=1),
|
||||
]
|
||||
|
||||
FLAGS.register_opts(volume_opts)
|
||||
FLAGS.register_opts(more_volume_opts)
|
||||
|
||||
|
||||
class VolumeConfigurationTest(test.TestCase):
|
||||
def setUp(self):
|
||||
super(VolumeConfigurationTest, self).setUp()
|
||||
|
||||
def tearDown(self):
|
||||
super(VolumeConfigurationTest, self).tearDown()
|
||||
|
||||
def test_group_grafts_opts(self):
|
||||
c = configuration.Configuration(volume_opts, config_group='foo')
|
||||
self.assertEquals(c.str_opt, FLAGS.foo.str_opt)
|
||||
self.assertEquals(c.bool_opt, FLAGS.foo.bool_opt)
|
||||
|
||||
def test_opts_no_group(self):
|
||||
c = configuration.Configuration(volume_opts)
|
||||
self.assertEquals(c.str_opt, FLAGS.str_opt)
|
||||
self.assertEquals(c.bool_opt, FLAGS.bool_opt)
|
||||
|
||||
def test_grafting_multiple_opts(self):
|
||||
c = configuration.Configuration(volume_opts, config_group='foo')
|
||||
c.append_config_values(more_volume_opts)
|
||||
self.assertEquals(c.str_opt, FLAGS.foo.str_opt)
|
||||
self.assertEquals(c.bool_opt, FLAGS.foo.bool_opt)
|
||||
self.assertEquals(c.int_opt, FLAGS.foo.int_opt)
|
||||
|
||||
def test_safe_get(self):
|
||||
c = configuration.Configuration(volume_opts, config_group='foo')
|
||||
self.assertEquals(c.safe_get('none_opt'), None)
|
@ -34,6 +34,12 @@ FLAGS = flags.FLAGS
|
||||
|
||||
class UsageInfoTestCase(test.TestCase):
|
||||
|
||||
QUEUE_NAME = 'cinder-volume'
|
||||
HOSTNAME = 'my-host.com'
|
||||
HOSTIP = '10.0.0.1'
|
||||
BACKEND = 'test_backend'
|
||||
MULTI_AT_BACKEND = 'test_b@ckend'
|
||||
|
||||
def setUp(self):
|
||||
super(UsageInfoTestCase, self).setUp()
|
||||
self.flags(connection_type='fake',
|
||||
@ -87,3 +93,25 @@ class UsageInfoTestCase(test.TestCase):
|
||||
self.assertTrue(attr in payload,
|
||||
msg="Key %s not in payload" % attr)
|
||||
db.volume_destroy(context.get_admin_context(), volume['id'])
|
||||
|
||||
def test_get_host_from_queue_simple(self):
|
||||
fullname = "%s.%s@%s" % (self.QUEUE_NAME, self.HOSTNAME, self.BACKEND)
|
||||
self.assertEquals(volume_utils.get_host_from_queue(fullname),
|
||||
self.HOSTNAME)
|
||||
|
||||
def test_get_host_from_queue_ip(self):
|
||||
fullname = "%s.%s@%s" % (self.QUEUE_NAME, self.HOSTIP, self.BACKEND)
|
||||
self.assertEquals(volume_utils.get_host_from_queue(fullname),
|
||||
self.HOSTIP)
|
||||
|
||||
def test_get_host_from_queue_multi_at_symbol(self):
|
||||
fullname = "%s.%s@%s" % (self.QUEUE_NAME, self.HOSTNAME,
|
||||
self.MULTI_AT_BACKEND)
|
||||
self.assertEquals(volume_utils.get_host_from_queue(fullname),
|
||||
self.HOSTNAME)
|
||||
|
||||
def test_get_host_from_queue_ip_multi_at_symbol(self):
|
||||
fullname = "%s.%s@%s" % (self.QUEUE_NAME, self.HOSTIP,
|
||||
self.MULTI_AT_BACKEND)
|
||||
self.assertEquals(volume_utils.get_host_from_queue(fullname),
|
||||
self.HOSTIP)
|
||||
|
@ -18,6 +18,9 @@
|
||||
|
||||
from cinder.db import api as db_api
|
||||
from cinder import exception
|
||||
from cinder.openstack.common import cfg
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume import driver as parent_driver
|
||||
from cinder.volume.drivers.xenapi import lib
|
||||
from cinder.volume.drivers.xenapi import sm as driver
|
||||
import mox
|
||||
@ -45,11 +48,13 @@ class DriverTestCase(unittest.TestCase):
|
||||
def test_do_setup(self):
|
||||
mock = mox.Mox()
|
||||
mock.StubOutWithMock(driver, 'xenapi_lib')
|
||||
mock.StubOutWithMock(driver, 'FLAGS')
|
||||
mock.StubOutWithMock(driver, 'xenapi_opts')
|
||||
|
||||
driver.FLAGS.xenapi_connection_url = 'url'
|
||||
driver.FLAGS.xenapi_connection_username = 'user'
|
||||
driver.FLAGS.xenapi_connection_password = 'pass'
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.xenapi_connection_url = 'url'
|
||||
configuration.xenapi_connection_username = 'user'
|
||||
configuration.xenapi_connection_password = 'pass'
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
session_factory = object()
|
||||
nfsops = object()
|
||||
@ -60,7 +65,7 @@ class DriverTestCase(unittest.TestCase):
|
||||
driver.xenapi_lib.NFSBasedVolumeOperations(
|
||||
session_factory).AndReturn(nfsops)
|
||||
|
||||
drv = driver.XenAPINFSDriver()
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
|
||||
mock.ReplayAll()
|
||||
drv.do_setup('context')
|
||||
@ -71,12 +76,13 @@ class DriverTestCase(unittest.TestCase):
|
||||
def test_create_volume(self):
|
||||
mock = mox.Mox()
|
||||
|
||||
mock.StubOutWithMock(driver, 'FLAGS')
|
||||
driver.FLAGS.xenapi_nfs_server = 'server'
|
||||
driver.FLAGS.xenapi_nfs_serverpath = 'path'
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.xenapi_nfs_server = 'server'
|
||||
configuration.xenapi_nfs_serverpath = 'path'
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
ops = mock.CreateMock(lib.NFSBasedVolumeOperations)
|
||||
drv = driver.XenAPINFSDriver()
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
drv.nfs_ops = ops
|
||||
|
||||
volume_details = dict(
|
||||
@ -96,12 +102,13 @@ class DriverTestCase(unittest.TestCase):
|
||||
def test_delete_volume(self):
|
||||
mock = mox.Mox()
|
||||
|
||||
mock.StubOutWithMock(driver, 'FLAGS')
|
||||
driver.FLAGS.xenapi_nfs_server = 'server'
|
||||
driver.FLAGS.xenapi_nfs_serverpath = 'path'
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.xenapi_nfs_server = 'server'
|
||||
configuration.xenapi_nfs_serverpath = 'path'
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
ops = mock.CreateMock(lib.NFSBasedVolumeOperations)
|
||||
drv = driver.XenAPINFSDriver()
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
drv.nfs_ops = ops
|
||||
|
||||
ops.delete_volume('server', 'path', 'sr_uuid', 'vdi_uuid')
|
||||
@ -112,21 +119,23 @@ class DriverTestCase(unittest.TestCase):
|
||||
mock.VerifyAll()
|
||||
|
||||
def test_create_export_does_not_raise_exception(self):
|
||||
drv = driver.XenAPINFSDriver()
|
||||
configuration = conf.Configuration([])
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
drv.create_export('context', 'volume')
|
||||
|
||||
def test_remove_export_does_not_raise_exception(self):
|
||||
drv = driver.XenAPINFSDriver()
|
||||
configuration = conf.Configuration([])
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
drv.remove_export('context', 'volume')
|
||||
|
||||
def test_initialize_connection(self):
|
||||
mock = mox.Mox()
|
||||
|
||||
mock.StubOutWithMock(driver, 'FLAGS')
|
||||
driver.FLAGS.xenapi_nfs_server = 'server'
|
||||
driver.FLAGS.xenapi_nfs_serverpath = 'path'
|
||||
|
||||
drv = driver.XenAPINFSDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.xenapi_nfs_server = 'server'
|
||||
configuration.xenapi_nfs_serverpath = 'path'
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
|
||||
mock.ReplayAll()
|
||||
result = drv.initialize_connection(
|
||||
@ -157,12 +166,12 @@ class DriverTestCase(unittest.TestCase):
|
||||
|
||||
def test_initialize_connection_null_values(self):
|
||||
mock = mox.Mox()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.xenapi_nfs_server = 'server'
|
||||
configuration.xenapi_nfs_serverpath = 'path'
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
mock.StubOutWithMock(driver, 'FLAGS')
|
||||
driver.FLAGS.xenapi_nfs_server = 'server'
|
||||
driver.FLAGS.xenapi_nfs_serverpath = 'path'
|
||||
|
||||
drv = driver.XenAPINFSDriver()
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
|
||||
mock.ReplayAll()
|
||||
result = drv.initialize_connection(
|
||||
@ -194,7 +203,12 @@ class DriverTestCase(unittest.TestCase):
|
||||
def _setup_mock_driver(self, server, serverpath, sr_base_path="_srbp"):
|
||||
mock = mox.Mox()
|
||||
|
||||
drv = driver.XenAPINFSDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.xenapi_nfs_server = server
|
||||
configuration.xenapi_nfs_serverpath = serverpath
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
drv = driver.XenAPINFSDriver(configuration=configuration)
|
||||
ops = mock.CreateMock(lib.NFSBasedVolumeOperations)
|
||||
db = mock.CreateMock(db_api)
|
||||
drv.nfs_ops = ops
|
||||
|
@ -20,9 +20,12 @@
|
||||
# Erik Zaadi <erikz@il.ibm.com>
|
||||
# Avishay Traeger <avishay@il.ibm.com>
|
||||
|
||||
import mox
|
||||
|
||||
from cinder import exception
|
||||
from cinder import flags
|
||||
from cinder import test
|
||||
from cinder.volume import configuration as conf
|
||||
from cinder.volume.drivers import xiv
|
||||
|
||||
|
||||
@ -114,7 +117,11 @@ class XIVVolumeDriverTest(test.TestCase):
|
||||
"""Initialize IVM XIV Driver."""
|
||||
super(XIVVolumeDriverTest, self).setUp()
|
||||
|
||||
self.driver = xiv.XIVDriver()
|
||||
configuration = mox.MockObject(conf.Configuration)
|
||||
configuration.san_is_local = False
|
||||
configuration.append_config_values(mox.IgnoreArg())
|
||||
|
||||
self.driver = xiv.XIVDriver(configuration=configuration)
|
||||
|
||||
def test_initialized_should_set_xiv_info(self):
|
||||
"""Test that the san flags are passed to the XIV proxy."""
|
||||
|
82
cinder/volume/configuration.py
Normal file
82
cinder/volume/configuration.py
Normal file
@ -0,0 +1,82 @@
|
||||
#!/usr/bin/env python
|
||||
# vim: tabstop=4 shiftwidth=4 softtabstop=4
|
||||
#
|
||||
# Copyright (c) 2012 Rackspace Hosting
|
||||
# All Rights Reserved.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
"""
|
||||
Configuration support for all drivers.
|
||||
|
||||
This module allows support for setting configurations either from default
|
||||
or from a particular FLAGS group, to be able to set multiple configurations
|
||||
for a given set of values.
|
||||
|
||||
For instance, two lvm configurations can be set by naming them in groups as
|
||||
|
||||
[lvm1]
|
||||
volume_group=lvm-group-1
|
||||
...
|
||||
|
||||
[lvm2]
|
||||
volume_group=lvm-group-2
|
||||
...
|
||||
|
||||
And the configuration group name will be passed in so that all calls to
|
||||
configuration.volume_group within that instance will be mapped to the proper
|
||||
named group.
|
||||
|
||||
This class also ensures the implementation's configuration is grafted into the
|
||||
option group. This is due to the way cfg works. All cfg options must be defined
|
||||
and registered in the group in which they are used.
|
||||
"""
|
||||
|
||||
from cinder import flags
|
||||
from cinder.openstack.common import cfg
|
||||
from cinder.openstack.common import log as logging
|
||||
|
||||
|
||||
FLAGS = flags.FLAGS
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
|
||||
class Configuration(object):
|
||||
|
||||
def __init__(self, volume_opts, config_group=None):
|
||||
"""This takes care of grafting the implementation's config
|
||||
values into the config group"""
|
||||
self.config_group = config_group
|
||||
|
||||
# set the local conf so that __call__'s know what to use
|
||||
if self.config_group:
|
||||
self._ensure_config_values(volume_opts)
|
||||
self.local_conf = FLAGS._get(self.config_group)
|
||||
else:
|
||||
self.local_conf = FLAGS
|
||||
|
||||
def _ensure_config_values(self, volume_opts):
|
||||
FLAGS.register_opts(volume_opts,
|
||||
group=self.config_group)
|
||||
|
||||
def append_config_values(self, volume_opts):
|
||||
self._ensure_config_values(volume_opts)
|
||||
|
||||
def safe_get(self, value):
|
||||
try:
|
||||
return self.__getattr__(value)
|
||||
except cfg.NoSuchOptError:
|
||||
return None
|
||||
|
||||
def __getattr__(self, value):
|
||||
return getattr(self.local_conf, value)
|
@ -29,6 +29,7 @@ from cinder.image import image_utils
|
||||
from cinder.openstack.common import cfg
|
||||
from cinder.openstack.common import log as logging
|
||||
from cinder import utils
|
||||
from cinder.volume.configuration import Configuration
|
||||
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
@ -54,7 +55,10 @@ volume_opts = [
|
||||
help='The port that the iSCSI daemon is listening on'),
|
||||
cfg.IntOpt('iscsi_port',
|
||||
default=3260,
|
||||
help='The port that the iSCSI daemon is listening on'), ]
|
||||
help='The port that the iSCSI daemon is listening on'),
|
||||
cfg.StrOpt('volume_backend_name',
|
||||
default=None,
|
||||
help='The backend name for a given driver implementation'), ]
|
||||
|
||||
FLAGS = flags.FLAGS
|
||||
FLAGS.register_opts(volume_opts)
|
||||
@ -66,6 +70,9 @@ class VolumeDriver(object):
|
||||
def __init__(self, execute=utils.execute, *args, **kwargs):
|
||||
# NOTE(vish): db is set by Manager
|
||||
self.db = None
|
||||
self.configuration = kwargs.get('configuration', None)
|
||||
if self.configuration:
|
||||
self.configuration.append_config_values(volume_opts)
|
||||
self.set_execute(execute)
|
||||
self._stats = {}
|
||||
|
||||
@ -83,7 +90,7 @@ class VolumeDriver(object):
|
||||
return True
|
||||
except exception.ProcessExecutionError:
|
||||
tries = tries + 1
|
||||
if tries >= FLAGS.num_shell_tries:
|
||||
if tries >= self.configuration.num_shell_tries:
|
||||
raise
|
||||
LOG.exception(_("Recovering from a failed execute. "
|
||||
"Try number %s"), tries)
|
||||
@ -208,7 +215,8 @@ class ISCSIDriver(VolumeDriver):
|
||||
'-t', 'sendtargets', '-p', volume['host'],
|
||||
run_as_root=True)
|
||||
for target in out.splitlines():
|
||||
if FLAGS.iscsi_ip_address in target and volume_name in target:
|
||||
if (self.configuration.iscsi_ip_address in target
|
||||
and volume_name in target):
|
||||
return target
|
||||
return None
|
||||
|
||||
@ -260,7 +268,7 @@ class ISCSIDriver(VolumeDriver):
|
||||
try:
|
||||
properties['target_lun'] = int(results[2])
|
||||
except (IndexError, ValueError):
|
||||
if FLAGS.iscsi_helper == 'tgtadm':
|
||||
if self.configuration.iscsi_helper == 'tgtadm':
|
||||
properties['target_lun'] = 1
|
||||
else:
|
||||
properties['target_lun'] = 0
|
||||
@ -409,7 +417,7 @@ class ISCSIDriver(VolumeDriver):
|
||||
|
||||
tries = 0
|
||||
while not os.path.exists(host_device):
|
||||
if tries >= FLAGS.num_iscsi_scan_tries:
|
||||
if tries >= self.configuration.num_iscsi_scan_tries:
|
||||
raise exception.CinderException(
|
||||
_("iSCSI device not found at %s") % (host_device))
|
||||
|
||||
@ -431,6 +439,32 @@ class ISCSIDriver(VolumeDriver):
|
||||
|
||||
return iscsi_properties, host_device
|
||||
|
||||
def get_volume_stats(self, refresh=False):
|
||||
"""Get volume status.
|
||||
|
||||
If 'refresh' is True, run update the stats first."""
|
||||
if refresh:
|
||||
self._update_volume_status()
|
||||
|
||||
return self._stats
|
||||
|
||||
def _update_volume_status(self):
|
||||
"""Retrieve status info from volume group."""
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = backend_name or 'Generic_iSCSI'
|
||||
data["vendor_name"] = 'Open Source'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'iSCSI'
|
||||
|
||||
data['total_capacity_gb'] = 'infinite'
|
||||
data['free_capacity_gb'] = 'infinite'
|
||||
data['reserved_percentage'] = 100
|
||||
data['QoS_support'] = False
|
||||
self._stats = data
|
||||
|
||||
|
||||
class FakeISCSIDriver(ISCSIDriver):
|
||||
"""Logs calls instead of executing."""
|
||||
|
@ -246,12 +246,13 @@ class CoraidDriver(driver.VolumeDriver):
|
||||
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(CoraidDriver, self).__init__(*args, **kwargs)
|
||||
self.configuration.append_config_values(coraid_opts)
|
||||
|
||||
def do_setup(self, context):
|
||||
"""Initialize the volume driver."""
|
||||
self.esm = CoraidRESTClient(FLAGS.coraid_esm_address,
|
||||
FLAGS.coraid_user,
|
||||
FLAGS.coraid_password)
|
||||
self.esm = CoraidRESTClient(self.configuration.coraid_esm_address,
|
||||
self.configuration.coraid_user,
|
||||
self.configuration.coraid_password)
|
||||
|
||||
def check_for_setup_error(self):
|
||||
"""Return an error if prerequisites aren't met."""
|
||||
@ -264,7 +265,7 @@ class CoraidDriver(driver.VolumeDriver):
|
||||
The ESM Repository is stored into a volume_type_extra_specs key.
|
||||
"""
|
||||
volume_type_id = volume_type['id']
|
||||
repository_key_name = FLAGS.coraid_repository_key
|
||||
repository_key_name = self.configuration.coraid_repository_key
|
||||
repository = volume_types.get_volume_type_extra_specs(
|
||||
volume_type_id, repository_key_name)
|
||||
return repository
|
||||
@ -296,10 +297,12 @@ class CoraidDriver(driver.VolumeDriver):
|
||||
def create_snapshot(self, snapshot):
|
||||
"""Create a Snapshot."""
|
||||
try:
|
||||
volume_name = FLAGS.volume_name_template % snapshot['volume_id']
|
||||
snapshot_name = FLAGS.snapshot_name_template % snapshot['id']
|
||||
volume_name = (FLAGS.volume_name_template
|
||||
% snapshot['volume_id'])
|
||||
snapshot_name = (FLAGS.snapshot_name_template
|
||||
% snapshot['id'])
|
||||
self.esm.create_snapshot(volume_name, snapshot_name)
|
||||
except Exception:
|
||||
except Exception, e:
|
||||
msg = _('Failed to Create Snapshot %(snapname)s')
|
||||
LOG.debug(msg % dict(snapname=snapshot_name))
|
||||
raise
|
||||
@ -308,7 +311,8 @@ class CoraidDriver(driver.VolumeDriver):
|
||||
def delete_snapshot(self, snapshot):
|
||||
"""Delete a Snapshot."""
|
||||
try:
|
||||
snapshot_name = FLAGS.snapshot_name_template % snapshot['id']
|
||||
snapshot_name = (FLAGS.snapshot_name_template
|
||||
% snapshot['id'])
|
||||
self.esm.delete_snapshot(snapshot_name)
|
||||
except Exception:
|
||||
msg = _('Failed to Delete Snapshot %(snapname)s')
|
||||
@ -319,7 +323,8 @@ class CoraidDriver(driver.VolumeDriver):
|
||||
def create_volume_from_snapshot(self, volume, snapshot):
|
||||
"""Create a Volume from a Snapshot."""
|
||||
try:
|
||||
snapshot_name = FLAGS.snapshot_name_template % snapshot['id']
|
||||
snapshot_name = (FLAGS.snapshot_name_template
|
||||
% snapshot['id'])
|
||||
repository = self._get_repository(volume['volume_type'])
|
||||
self.esm.create_volume_from_snapshot(snapshot_name,
|
||||
volume['name'],
|
||||
|
@ -55,7 +55,7 @@ class EMCSMISCommon():
|
||||
'vendor_name': 'EMC',
|
||||
'volume_backend_name': None}
|
||||
|
||||
def __init__(self, prtcl):
|
||||
def __init__(self, prtcl, configuration=None):
|
||||
|
||||
opt = cfg.StrOpt('cinder_emc_config_file',
|
||||
default=CINDER_EMC_CONFIG_FILE,
|
||||
@ -63,6 +63,9 @@ class EMCSMISCommon():
|
||||
'config data')
|
||||
FLAGS.register_opt(opt)
|
||||
self.protocol = prtcl
|
||||
self.configuration = configuration
|
||||
self.configuration.append_config_values([opt])
|
||||
|
||||
ip, port = self._get_ecom_server()
|
||||
self.user, self.passwd = self._get_ecom_cred()
|
||||
self.url = 'http://' + ip + ':' + port
|
||||
@ -862,7 +865,7 @@ class EMCSMISCommon():
|
||||
def _get_storage_type(self, filename=None):
|
||||
"""Get the storage type from the config file."""
|
||||
if filename == None:
|
||||
filename = FLAGS.cinder_emc_config_file
|
||||
filename = self.configuration.cinder_emc_config_file
|
||||
|
||||
file = open(filename, 'r')
|
||||
data = file.read()
|
||||
@ -882,7 +885,7 @@ class EMCSMISCommon():
|
||||
|
||||
def _get_masking_view(self, filename=None):
|
||||
if filename == None:
|
||||
filename = FLAGS.cinder_emc_config_file
|
||||
filename = self.configuration.cinder_emc_config_file
|
||||
|
||||
file = open(filename, 'r')
|
||||
data = file.read()
|
||||
@ -900,7 +903,7 @@ class EMCSMISCommon():
|
||||
|
||||
def _get_ecom_cred(self, filename=None):
|
||||
if filename == None:
|
||||
filename = FLAGS.cinder_emc_config_file
|
||||
filename = self.configuration.cinder_emc_config_file
|
||||
|
||||
file = open(filename, 'r')
|
||||
data = file.read()
|
||||
@ -922,7 +925,7 @@ class EMCSMISCommon():
|
||||
|
||||
def _get_ecom_server(self, filename=None):
|
||||
if filename == None:
|
||||
filename = FLAGS.cinder_emc_config_file
|
||||
filename = self.configuration.cinder_emc_config_file
|
||||
|
||||
file = open(filename, 'r')
|
||||
data = file.read()
|
||||
|
@ -41,7 +41,9 @@ class EMCSMISISCSIDriver(driver.ISCSIDriver):
|
||||
def __init__(self, *args, **kwargs):
|
||||
|
||||
super(EMCSMISISCSIDriver, self).__init__(*args, **kwargs)
|
||||
self.common = emc_smis_common.EMCSMISCommon('iSCSI')
|
||||
self.common = emc_smis_common.EMCSMISCommon(
|
||||
'iSCSI',
|
||||
configuration=self.configuration)
|
||||
|
||||
def check_for_setup_error(self):
|
||||
pass
|
||||
@ -118,10 +120,10 @@ class EMCSMISISCSIDriver(driver.ISCSIDriver):
|
||||
|
||||
(out, _err) = self._execute('iscsiadm', '-m', 'discovery',
|
||||
'-t', 'sendtargets', '-p',
|
||||
FLAGS.iscsi_ip_address,
|
||||
self.configuration.iscsi_ip_address,
|
||||
run_as_root=True)
|
||||
for target in out.splitlines():
|
||||
index = target.find(FLAGS.iscsi_ip_address)
|
||||
index = target.find(self.configuration.iscsi_ip_address)
|
||||
if index != -1:
|
||||
return target
|
||||
return None
|
||||
|
@ -50,11 +50,15 @@ class GlusterfsDriver(nfs.RemoteFsDriver):
|
||||
"""Gluster based cinder driver. Creates file on Gluster share for using it
|
||||
as block device on hypervisor."""
|
||||
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(GlusterfsDriver, self).__init__(*args, **kwargs)
|
||||
self.configuration.append_config_values(volume_opts)
|
||||
|
||||
def do_setup(self, context):
|
||||
"""Any initialization the volume driver does while starting."""
|
||||
super(GlusterfsDriver, self).do_setup(context)
|
||||
|
||||
config = FLAGS.glusterfs_shares_config
|
||||
config = self.configuration.glusterfs_shares_config
|
||||
if not config:
|
||||
msg = (_("There's no Gluster config file configured (%s)") %
|
||||
'glusterfs_shares_config')
|
||||
@ -149,7 +153,7 @@ class GlusterfsDriver(nfs.RemoteFsDriver):
|
||||
volume_path = self.local_path(volume)
|
||||
volume_size = volume['size']
|
||||
|
||||
if FLAGS.glusterfs_sparsed_volumes:
|
||||
if self.configuration.glusterfs_sparsed_volumes:
|
||||
self._create_sparsed_file(volume_path, volume_size)
|
||||
else:
|
||||
self._create_regular_file(volume_path, volume_size)
|
||||
@ -171,7 +175,8 @@ class GlusterfsDriver(nfs.RemoteFsDriver):
|
||||
LOG.debug('Available shares %s' % str(self._mounted_shares))
|
||||
|
||||
def _load_shares_config(self):
|
||||
return [share.strip() for share in open(FLAGS.glusterfs_shares_config)
|
||||
return [share.strip() for share
|
||||
in open(self.configuration.glusterfs_shares_config)
|
||||
if share and not share.startswith('#')]
|
||||
|
||||
def _ensure_share_mounted(self, glusterfs_share):
|
||||
@ -208,7 +213,7 @@ class GlusterfsDriver(nfs.RemoteFsDriver):
|
||||
"""Return mount point for share.
|
||||
:param glusterfs_share: example 172.18.194.100:/var/glusterfs
|
||||
"""
|
||||
return os.path.join(FLAGS.glusterfs_mount_point_base,
|
||||
return os.path.join(self.configuration.glusterfs_mount_point_base,
|
||||
self._get_hash_str(glusterfs_share))
|
||||
|
||||
def _get_available_capacity(self, glusterfs_share):
|
||||
@ -223,7 +228,7 @@ class GlusterfsDriver(nfs.RemoteFsDriver):
|
||||
|
||||
available = 0
|
||||
|
||||
if FLAGS.glusterfs_disk_util == 'df':
|
||||
if self.configuration.glusterfs_disk_util == 'df':
|
||||
available = int(out.split()[3])
|
||||
else:
|
||||
size = int(out.split()[1])
|
||||
|
@ -64,22 +64,23 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
"""Executes commands relating to Volumes."""
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(LVMVolumeDriver, self).__init__(*args, **kwargs)
|
||||
self.configuration.append_config_values(volume_opts)
|
||||
|
||||
def check_for_setup_error(self):
|
||||
"""Returns an error if prerequisites aren't met"""
|
||||
out, err = self._execute('vgs', '--noheadings', '-o', 'name',
|
||||
run_as_root=True)
|
||||
volume_groups = out.split()
|
||||
if FLAGS.volume_group not in volume_groups:
|
||||
if self.configuration.volume_group not in volume_groups:
|
||||
exception_message = (_("volume group %s doesn't exist")
|
||||
% FLAGS.volume_group)
|
||||
% self.configuration.volume_group)
|
||||
raise exception.VolumeBackendAPIException(data=exception_message)
|
||||
|
||||
def _create_volume(self, volume_name, sizestr):
|
||||
cmd = ['lvcreate', '-L', sizestr, '-n', volume_name,
|
||||
FLAGS.volume_group]
|
||||
if FLAGS.lvm_mirrors:
|
||||
cmd += ['-m', FLAGS.lvm_mirrors, '--nosync']
|
||||
self.configuration.volume_group]
|
||||
if self.configuration.lvm_mirrors:
|
||||
cmd += ['-m', self.configuration.lvm_mirrors, '--nosync']
|
||||
terras = int(sizestr[:-1]) / 1024.0
|
||||
if terras >= 1.5:
|
||||
rsize = int(2 ** math.ceil(math.log(terras) / math.log(2)))
|
||||
@ -112,7 +113,7 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
*extra_flags, run_as_root=True)
|
||||
|
||||
def _volume_not_present(self, volume_name):
|
||||
path_name = '%s/%s' % (FLAGS.volume_group, volume_name)
|
||||
path_name = '%s/%s' % (self.configuration.volume_group, volume_name)
|
||||
try:
|
||||
self._try_execute('lvdisplay', path_name, run_as_root=True)
|
||||
except Exception as e:
|
||||
@ -129,7 +130,7 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
self.clear_volume(volume)
|
||||
|
||||
self._try_execute('lvremove', '-f', "%s/%s" %
|
||||
(FLAGS.volume_group,
|
||||
(self.configuration.volume_group,
|
||||
self._escape_snapshot(volume['name'])),
|
||||
run_as_root=True)
|
||||
|
||||
@ -166,7 +167,7 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
# deleting derived snapshots. Can we do something fancy?
|
||||
out, err = self._execute('lvdisplay', '--noheading',
|
||||
'-C', '-o', 'Attr',
|
||||
'%s/%s' % (FLAGS.volume_group,
|
||||
'%s/%s' % (self.configuration.volume_group,
|
||||
volume['name']),
|
||||
run_as_root=True)
|
||||
# fake_execute returns None resulting unit test error
|
||||
@ -182,29 +183,29 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
|
||||
vol_path = self.local_path(volume)
|
||||
size_in_g = volume.get('size')
|
||||
size_in_m = FLAGS.volume_clear_size
|
||||
size_in_m = self.configuration.volume_clear_size
|
||||
|
||||
if not size_in_g:
|
||||
return
|
||||
|
||||
if FLAGS.volume_clear == 'none':
|
||||
if self.configuration.volume_clear == 'none':
|
||||
return
|
||||
|
||||
LOG.info(_("Performing secure delete on volume: %s") % volume['id'])
|
||||
|
||||
if FLAGS.volume_clear == 'zero':
|
||||
if self.configuration.volume_clear == 'zero':
|
||||
if size_in_m == 0:
|
||||
return self._copy_volume('/dev/zero', vol_path, size_in_g,
|
||||
clearing=True)
|
||||
else:
|
||||
clear_cmd = ['shred', '-n0', '-z', '-s%dMiB' % size_in_m]
|
||||
elif FLAGS.volume_clear == 'shred':
|
||||
elif self.configuration.volume_clear == 'shred':
|
||||
clear_cmd = ['shred', '-n3']
|
||||
if size_in_m:
|
||||
clear_cmd.append('-s%dMiB' % size_in_m)
|
||||
else:
|
||||
LOG.error(_("Error unrecognized volume_clear option: %s"),
|
||||
FLAGS.volume_clear)
|
||||
self.configuration.volume_clear)
|
||||
return
|
||||
|
||||
clear_cmd.append(vol_path)
|
||||
@ -212,7 +213,8 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
|
||||
def create_snapshot(self, snapshot):
|
||||
"""Creates a snapshot."""
|
||||
orig_lv_name = "%s/%s" % (FLAGS.volume_group, snapshot['volume_name'])
|
||||
orig_lv_name = "%s/%s" % (self.configuration.volume_group,
|
||||
snapshot['volume_name'])
|
||||
self._try_execute('lvcreate', '-L',
|
||||
self._sizestr(snapshot['volume_size']),
|
||||
'--name', self._escape_snapshot(snapshot['name']),
|
||||
@ -230,7 +232,7 @@ class LVMVolumeDriver(driver.VolumeDriver):
|
||||
|
||||
def local_path(self, volume):
|
||||
# NOTE(vish): stops deprecation warning
|
||||
escaped_group = FLAGS.volume_group.replace('-', '--')
|
||||
escaped_group = self.configuration.volume_group.replace('-', '--')
|
||||
escaped_name = self._escape_snapshot(volume['name']).replace('-', '--')
|
||||
return "/dev/mapper/%s-%s" % (escaped_group, escaped_name)
|
||||
|
||||
@ -352,8 +354,10 @@ class LVMISCSIDriver(LVMVolumeDriver, driver.ISCSIDriver):
|
||||
volume_name = old_name
|
||||
old_name = None
|
||||
|
||||
iscsi_name = "%s%s" % (FLAGS.iscsi_target_prefix, volume_name)
|
||||
volume_path = "/dev/%s/%s" % (FLAGS.volume_group, volume_name)
|
||||
iscsi_name = "%s%s" % (self.configuration.iscsi_target_prefix,
|
||||
volume_name)
|
||||
volume_path = "/dev/%s/%s" % (self.configuration.volume_group,
|
||||
volume_name)
|
||||
|
||||
# NOTE(jdg): For TgtAdm case iscsi_name is the ONLY param we need
|
||||
# should clean this all up at some point in the future
|
||||
@ -388,12 +392,13 @@ class LVMISCSIDriver(LVMVolumeDriver, driver.ISCSIDriver):
|
||||
self.db.volume_update(context, volume['id'], model_update)
|
||||
|
||||
start = os.getcwd()
|
||||
os.chdir('/dev/%s' % FLAGS.volume_group)
|
||||
os.chdir('/dev/%s' % self.configuration.volume_group)
|
||||
|
||||
try:
|
||||
(out, err) = self._execute('readlink', old_name)
|
||||
except exception.ProcessExecutionError:
|
||||
link_path = '/dev/%s/%s' % (FLAGS.volume_group, old_name)
|
||||
link_path = '/dev/%s/%s' % (self.configuration.volume_group,
|
||||
old_name)
|
||||
LOG.debug(_('Symbolic link %s not found') % link_path)
|
||||
os.chdir(start)
|
||||
return
|
||||
@ -414,19 +419,22 @@ class LVMISCSIDriver(LVMVolumeDriver, driver.ISCSIDriver):
|
||||
if not isinstance(self.tgtadm, iscsi.TgtAdm):
|
||||
host_iscsi_targets = self.db.iscsi_target_count_by_host(context,
|
||||
host)
|
||||
if host_iscsi_targets >= FLAGS.iscsi_num_targets:
|
||||
if host_iscsi_targets >= self.configuration.iscsi_num_targets:
|
||||
return
|
||||
|
||||
# NOTE(vish): Target ids start at 1, not 0.
|
||||
for target_num in xrange(1, FLAGS.iscsi_num_targets + 1):
|
||||
target_end = self.configuration.iscsi_num_targets + 1
|
||||
for target_num in xrange(1, target_end):
|
||||
target = {'host': host, 'target_num': target_num}
|
||||
self.db.iscsi_target_create_safe(context, target)
|
||||
|
||||
def create_export(self, context, volume):
|
||||
"""Creates an export for a logical volume."""
|
||||
|
||||
iscsi_name = "%s%s" % (FLAGS.iscsi_target_prefix, volume['name'])
|
||||
volume_path = "/dev/%s/%s" % (FLAGS.volume_group, volume['name'])
|
||||
iscsi_name = "%s%s" % (self.configuration.iscsi_target_prefix,
|
||||
volume['name'])
|
||||
volume_path = "/dev/%s/%s" % (self.configuration.volume_group,
|
||||
volume['name'])
|
||||
model_update = {}
|
||||
|
||||
# TODO(jdg): In the future move all of the dependent stuff into the
|
||||
@ -454,7 +462,7 @@ class LVMISCSIDriver(LVMVolumeDriver, driver.ISCSIDriver):
|
||||
volume_path,
|
||||
chap_auth)
|
||||
model_update['provider_location'] = self._iscsi_location(
|
||||
FLAGS.iscsi_ip_address, tid, iscsi_name, lun)
|
||||
self.configuration.iscsi_ip_address, tid, iscsi_name, lun)
|
||||
model_update['provider_auth'] = self._iscsi_authentication(
|
||||
'CHAP', chap_username, chap_password)
|
||||
return model_update
|
||||
@ -527,20 +535,22 @@ class LVMISCSIDriver(LVMVolumeDriver, driver.ISCSIDriver):
|
||||
# Note(zhiteng): These information are driver/backend specific,
|
||||
# each driver may define these values in its own config options
|
||||
# or fetch from driver specific configuration file.
|
||||
data["volume_backend_name"] = 'LVM_iSCSI'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = backend_name or 'LVM_iSCSI'
|
||||
data["vendor_name"] = 'Open Source'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'iSCSI'
|
||||
|
||||
data['total_capacity_gb'] = 0
|
||||
data['free_capacity_gb'] = 0
|
||||
data['reserved_percentage'] = FLAGS.reserved_percentage
|
||||
data['reserved_percentage'] = self.configuration.reserved_percentage
|
||||
data['QoS_support'] = False
|
||||
|
||||
try:
|
||||
out, err = self._execute('vgs', '--noheadings', '--nosuffix',
|
||||
'--unit=G', '-o', 'name,size,free',
|
||||
FLAGS.volume_group, run_as_root=True)
|
||||
self.configuration.volume_group,
|
||||
run_as_root=True)
|
||||
except exception.ProcessExecutionError as exc:
|
||||
LOG.error(_("Error retrieving volume status: "), exc.stderr)
|
||||
out = False
|
||||
@ -553,7 +563,8 @@ class LVMISCSIDriver(LVMVolumeDriver, driver.ISCSIDriver):
|
||||
self._stats = data
|
||||
|
||||
def _iscsi_location(self, ip, target, iqn, lun=None):
|
||||
return "%s:%s,%s %s %s" % (ip, FLAGS.iscsi_port, target, iqn, lun)
|
||||
return "%s:%s,%s %s %s" % (ip, self.configuration.iscsi_port,
|
||||
target, iqn, lun)
|
||||
|
||||
def _iscsi_authentication(self, chap, name, password):
|
||||
return "%s %s %s" % (chap, name, password)
|
||||
|
@ -1080,14 +1080,15 @@ class NetAppISCSIDriver(driver.ISCSIDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_iSCSI_7mode'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = backend_name or 'NetApp_iSCSI_7mode'
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'iSCSI'
|
||||
|
||||
data['total_capacity_gb'] = 'infinite'
|
||||
data['free_capacity_gb'] = 'infinite'
|
||||
data['reserved_percentage'] = 100
|
||||
data['reserved_percentage'] = 0
|
||||
data['QoS_support'] = False
|
||||
self._stats = data
|
||||
|
||||
@ -1437,7 +1438,8 @@ class NetAppCmodeISCSIDriver(driver.ISCSIDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_iSCSI_Cluster'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = backend_name or 'NetApp_iSCSI_Cluster'
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'iSCSI'
|
||||
@ -2193,7 +2195,9 @@ class NetAppDirectCmodeISCSIDriver(NetAppDirectISCSIDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_iSCSI_Cluster_direct'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = (backend_name
|
||||
or 'NetApp_iSCSI_Cluster_direct')
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'iSCSI'
|
||||
@ -2448,7 +2452,9 @@ class NetAppDirect7modeISCSIDriver(NetAppDirectISCSIDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_iSCSI_7mode_direct'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = (backend_name
|
||||
or 'NetApp_iSCSI_7mode_direct')
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'iSCSI'
|
||||
|
@ -295,7 +295,8 @@ class NetAppNFSDriver(nfs.NfsDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_NFS_7mode'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = backend_name or 'NetApp_NFS_7mode'
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'NFS'
|
||||
@ -366,7 +367,8 @@ class NetAppCmodeNfsDriver (NetAppNFSDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_NFS_Cluster'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = backend_name or 'NetApp_NFS_Cluster'
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'NFS'
|
||||
@ -540,7 +542,9 @@ class NetAppDirectCmodeNfsDriver (NetAppDirectNfsDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_NFS_cluster_direct'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = (backend_name
|
||||
or 'NetApp_NFS_cluster_direct')
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'NFS'
|
||||
@ -668,7 +672,9 @@ class NetAppDirect7modeNfsDriver (NetAppDirectNfsDriver):
|
||||
|
||||
LOG.debug(_("Updating volume status"))
|
||||
data = {}
|
||||
data["volume_backend_name"] = 'NetApp_NFS_7mode_direct'
|
||||
backend_name = self.configuration.safe_get('volume_backend_name')
|
||||
data["volume_backend_name"] = (backend_name
|
||||
or 'NetApp_NFS_7mode_direct')
|
||||
data["vendor_name"] = 'NetApp'
|
||||
data["driver_version"] = '1.0'
|
||||
data["storage_protocol"] = 'NFS'
|
||||
|
@ -75,8 +75,8 @@ FLAGS.register_opts(nexenta_opts)
|
||||
class NexentaDriver(driver.ISCSIDriver): # pylint: disable=R0921
|
||||
"""Executes volume driver commands on Nexenta Appliance."""
|
||||
|
||||
def __init__(self):
|
||||
super(NexentaDriver, self).__init__()
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(NexentaDriver, self).__init__(*args, **kwargs)
|
||||
|
||||
def do_setup(self, context):
|
||||
protocol = FLAGS.nexenta_rest_protocol
|
||||
|
@ -120,12 +120,15 @@ class RemoteFsDriver(driver.VolumeDriver):
|
||||
class NfsDriver(RemoteFsDriver):
|
||||
"""NFS based cinder driver. Creates file on NFS share for using it
|
||||
as block device on hypervisor."""
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(NfsDriver, self).__init__(*args, **kwargs)
|
||||
self.configuration.append_config_values(volume_opts)
|
||||
|
||||
def do_setup(self, context):
|
||||
"""Any initialization the volume driver does while starting"""
|
||||
super(NfsDriver, self).do_setup(context)
|
||||
|
||||
config = FLAGS.nfs_shares_config
|
||||
config = self.configuration.nfs_shares_config
|
||||
if not config:
|
||||
msg = (_("There's no NFS config file configured (%s)") %
|
||||
'nfs_shares_config')
|
||||
@ -214,7 +217,7 @@ class NfsDriver(RemoteFsDriver):
|
||||
volume_path = self.local_path(volume)
|
||||
volume_size = volume['size']
|
||||
|
||||
if FLAGS.nfs_sparsed_volumes:
|
||||
if self.configuration.nfs_sparsed_volumes:
|
||||
self._create_sparsed_file(volume_path, volume_size)
|
||||
else:
|
||||
self._create_regular_file(volume_path, volume_size)
|
||||
@ -235,7 +238,8 @@ class NfsDriver(RemoteFsDriver):
|
||||
LOG.debug('Available shares %s' % str(self._mounted_shares))
|
||||
|
||||
def _load_shares_config(self):
|
||||
return [share.strip() for share in open(FLAGS.nfs_shares_config)
|
||||
return [share.strip() for share in
|
||||
open(self.configuration.nfs_shares_config)
|
||||
if share and not share.startswith('#')]
|
||||
|
||||
def _ensure_share_mounted(self, nfs_share):
|
||||
@ -272,7 +276,7 @@ class NfsDriver(RemoteFsDriver):
|
||||
"""
|
||||
:param nfs_share: example 172.18.194.100:/var/nfs
|
||||
"""
|
||||
return os.path.join(FLAGS.nfs_mount_point_base,
|
||||
return os.path.join(self.configuration.nfs_mount_point_base,
|
||||
self._get_hash_str(nfs_share))
|
||||
|
||||
def _get_available_capacity(self, nfs_share):
|
||||
@ -287,7 +291,7 @@ class NfsDriver(RemoteFsDriver):
|
||||
|
||||
available = 0
|
||||
|
||||
if FLAGS.nfs_disk_util == 'df':
|
||||
if self.configuration.nfs_disk_util == 'df':
|
||||
available = int(out.split()[3])
|
||||
else:
|
||||
size = int(out.split()[1])
|
||||
|
@ -52,14 +52,17 @@ FLAGS.register_opts(rbd_opts)
|
||||
|
||||
class RBDDriver(driver.VolumeDriver):
|
||||
"""Implements RADOS block device (RBD) volume commands"""
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(RBDDriver, self).__init__(*args, **kwargs)
|
||||
self.configuration.append_config_values(rbd_opts)
|
||||
|
||||
def check_for_setup_error(self):
|
||||
"""Returns an error if prerequisites aren't met"""
|
||||
(stdout, stderr) = self._execute('rados', 'lspools')
|
||||
pools = stdout.split("\n")
|
||||
if FLAGS.rbd_pool not in pools:
|
||||
if self.configuration.rbd_pool not in pools:
|
||||
exception_message = (_("rbd has no pool %s") %
|
||||
FLAGS.rbd_pool)
|
||||
self.configuration.rbd_pool)
|
||||
raise exception.VolumeBackendAPIException(data=exception_message)
|
||||
|
||||
def _supports_layering(self):
|
||||
@ -76,7 +79,7 @@ class RBDDriver(driver.VolumeDriver):
|
||||
else:
|
||||
size = int(volume['size']) * 1024
|
||||
args = ['rbd', 'create',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
'--size', size,
|
||||
volume['name']]
|
||||
if self._supports_layering():
|
||||
@ -88,19 +91,19 @@ class RBDDriver(driver.VolumeDriver):
|
||||
'--pool', src_pool,
|
||||
'--image', src_image,
|
||||
'--snap', src_snap,
|
||||
'--dest-pool', FLAGS.rbd_pool,
|
||||
'--dest-pool', self.configuration.rbd_pool,
|
||||
'--dest', volume['name'])
|
||||
|
||||
def _resize(self, volume):
|
||||
size = int(volume['size']) * 1024
|
||||
self._try_execute('rbd', 'resize',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
'--image', volume['name'],
|
||||
'--size', size)
|
||||
|
||||
def create_volume_from_snapshot(self, volume, snapshot):
|
||||
"""Creates a volume from a snapshot."""
|
||||
self._clone(volume, FLAGS.rbd_pool,
|
||||
self._clone(volume, self.configuration.rbd_pool,
|
||||
snapshot['volume_name'], snapshot['name'])
|
||||
if int(volume['size']):
|
||||
self._resize(volume)
|
||||
@ -108,23 +111,23 @@ class RBDDriver(driver.VolumeDriver):
|
||||
def delete_volume(self, volume):
|
||||
"""Deletes a logical volume."""
|
||||
stdout, _ = self._execute('rbd', 'snap', 'ls',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
volume['name'])
|
||||
if stdout.count('\n') > 1:
|
||||
raise exception.VolumeIsBusy(volume_name=volume['name'])
|
||||
self._try_execute('rbd', 'rm',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
volume['name'])
|
||||
|
||||
def create_snapshot(self, snapshot):
|
||||
"""Creates an rbd snapshot"""
|
||||
self._try_execute('rbd', 'snap', 'create',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
'--snap', snapshot['name'],
|
||||
snapshot['volume_name'])
|
||||
if self._supports_layering():
|
||||
self._try_execute('rbd', 'snap', 'protect',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
'--snap', snapshot['name'],
|
||||
snapshot['volume_name'])
|
||||
|
||||
@ -133,13 +136,13 @@ class RBDDriver(driver.VolumeDriver):
|
||||
if self._supports_layering():
|
||||
try:
|
||||
self._try_execute('rbd', 'snap', 'unprotect',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
'--snap', snapshot['name'],
|
||||
snapshot['volume_name'])
|
||||
except exception.ProcessExecutionError:
|
||||
raise exception.SnapshotIsBusy(snapshot_name=snapshot['name'])
|
||||
self._try_execute('rbd', 'snap', 'rm',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
'--snap', snapshot['name'],
|
||||
snapshot['volume_name'])
|
||||
|
||||
@ -147,7 +150,7 @@ class RBDDriver(driver.VolumeDriver):
|
||||
"""Returns the path of the rbd volume."""
|
||||
# This is the same as the remote path
|
||||
# since qemu accesses it directly.
|
||||
return "rbd:%s/%s" % (FLAGS.rbd_pool, volume['name'])
|
||||
return "rbd:%s/%s" % (self.configuration.rbd_pool, volume['name'])
|
||||
|
||||
def ensure_export(self, context, volume):
|
||||
"""Synchronously recreates an export for a logical volume."""
|
||||
@ -165,11 +168,13 @@ class RBDDriver(driver.VolumeDriver):
|
||||
return {
|
||||
'driver_volume_type': 'rbd',
|
||||
'data': {
|
||||
'name': '%s/%s' % (FLAGS.rbd_pool, volume['name']),
|
||||
'auth_enabled': FLAGS.rbd_secret_uuid is not None,
|
||||
'auth_username': FLAGS.rbd_user,
|
||||
'name': '%s/%s' % (self.configuration.rbd_pool,
|
||||
volume['name']),
|
||||
'auth_enabled': (self.configuration.rbd_secret_uuid
|
||||
is not None),
|
||||
'auth_username': self.configuration.rbd_user,
|
||||
'secret_type': 'ceph',
|
||||
'secret_uuid': FLAGS.rbd_secret_uuid, }
|
||||
'secret_uuid': self.configuration.rbd_secret_uuid, }
|
||||
}
|
||||
|
||||
def terminate_connection(self, volume, connector, **kwargs):
|
||||
@ -225,25 +230,27 @@ class RBDDriver(driver.VolumeDriver):
|
||||
return True
|
||||
|
||||
def _ensure_tmp_exists(self):
|
||||
if FLAGS.volume_tmp_dir and not os.path.exists(FLAGS.volume_tmp_dir):
|
||||
os.makedirs(FLAGS.volume_tmp_dir)
|
||||
tmp_dir = self.configuration.volume_tmp_dir
|
||||
if tmp_dir and not os.path.exists(tmp_dir):
|
||||
os.makedirs(tmp_dir)
|
||||
|
||||
def copy_image_to_volume(self, context, volume, image_service, image_id):
|
||||
# TODO(jdurgin): replace with librbd
|
||||
# this is a temporary hack, since rewriting this driver
|
||||
# to use librbd would take too long
|
||||
self._ensure_tmp_exists()
|
||||
tmp_dir = self.configuration.volume_tmp_dir
|
||||
|
||||
with tempfile.NamedTemporaryFile(dir=FLAGS.volume_tmp_dir) as tmp:
|
||||
with tempfile.NamedTemporaryFile(dir=tmp_dir) as tmp:
|
||||
image_utils.fetch_to_raw(context, image_service, image_id,
|
||||
tmp.name)
|
||||
# import creates the image, so we must remove it first
|
||||
self._try_execute('rbd', 'rm',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
volume['name'])
|
||||
|
||||
args = ['rbd', 'import',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
tmp.name, volume['name']]
|
||||
if self._supports_layering():
|
||||
args += ['--new-format']
|
||||
@ -253,12 +260,12 @@ class RBDDriver(driver.VolumeDriver):
|
||||
def copy_volume_to_image(self, context, volume, image_service, image_meta):
|
||||
self._ensure_tmp_exists()
|
||||
|
||||
tmp_dir = FLAGS.volume_tmp_dir or '/tmp'
|
||||
tmp_dir = self.configuration.volume_tmp_dir or '/tmp'
|
||||
tmp_file = os.path.join(tmp_dir,
|
||||
volume['name'] + '-' + image_meta['id'])
|
||||
with utils.remove_path_on_error(tmp_file):
|
||||
self._try_execute('rbd', 'export',
|
||||
'--pool', FLAGS.rbd_pool,
|
||||
'--pool', self.configuration.rbd_pool,
|
||||
volume['name'], tmp_file)
|
||||
image_utils.upload_volume(context, image_service,
|
||||
image_meta, tmp_file)
|
||||
|
@ -87,11 +87,13 @@ class SanISCSIDriver(ISCSIDriver):
|
||||
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(SanISCSIDriver, self).__init__(*args, **kwargs)
|
||||
self.run_local = FLAGS.san_is_local
|
||||
self.configuration.append_config_values(san_opts)
|
||||
self.run_local = self.configuration.san_is_local
|
||||
self.sshpool = None
|
||||
|
||||
def _build_iscsi_target_name(self, volume):
|
||||
return "%s%s" % (FLAGS.iscsi_target_prefix, volume['name'])
|
||||
return "%s%s" % (self.configuration.iscsi_target_prefix,
|
||||
volume['name'])
|
||||
|
||||
def _execute(self, *cmd, **kwargs):
|
||||
if self.run_local:
|
||||
@ -103,14 +105,18 @@ class SanISCSIDriver(ISCSIDriver):
|
||||
|
||||
def _run_ssh(self, command, check_exit_code=True, attempts=1):
|
||||
if not self.sshpool:
|
||||
self.sshpool = utils.SSHPool(FLAGS.san_ip,
|
||||
FLAGS.san_ssh_port,
|
||||
FLAGS.ssh_conn_timeout,
|
||||
FLAGS.san_login,
|
||||
password=FLAGS.san_password,
|
||||
privatekey=FLAGS.san_private_key,
|
||||
min_size=FLAGS.ssh_min_pool_conn,
|
||||
max_size=FLAGS.ssh_max_pool_conn)
|
||||
password = self.configuration.san_password
|
||||
privatekey = self.configuration.san_private_key
|
||||
min_size = self.configuration.ssh_min_pool_conn
|
||||
max_size = self.configuration.ssh_max_pool_conn
|
||||
self.sshpool = utils.SSHPool(self.configuration.san_ip,
|
||||
self.configuration.san_ssh_port,
|
||||
self.configuration.ssh_conn_timeout,
|
||||
self.configuration.san_login,
|
||||
password=password,
|
||||
privatekey=privatekey,
|
||||
min_size=min_size,
|
||||
max_size=max_size)
|
||||
last_exception = None
|
||||
try:
|
||||
total_attempts = attempts
|
||||
@ -158,12 +164,13 @@ class SanISCSIDriver(ISCSIDriver):
|
||||
def check_for_setup_error(self):
|
||||
"""Returns an error if prerequisites aren't met."""
|
||||
if not self.run_local:
|
||||
if not (FLAGS.san_password or FLAGS.san_private_key):
|
||||
if not (self.configuration.san_password or
|
||||
self.configuration.san_private_key):
|
||||
raise exception.InvalidInput(
|
||||
reason=_('Specify san_password or san_private_key'))
|
||||
|
||||
# The san_ip must always be set, because we use it for the target
|
||||
if not FLAGS.san_ip:
|
||||
if not self.configuration.san_ip:
|
||||
raise exception.InvalidInput(reason=_("san_ip must be set"))
|
||||
|
||||
def create_cloned_volume(self, volume, src_vref):
|
||||
|
@ -50,8 +50,8 @@ FLAGS.register_opts(windows_opts)
|
||||
class WindowsDriver(driver.ISCSIDriver):
|
||||
"""Executes volume driver commands on Windows Storage server."""
|
||||
|
||||
def __init__(self):
|
||||
super(WindowsDriver, self).__init__()
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(WindowsDriver, self).__init__(*args, **kwargs)
|
||||
|
||||
def do_setup(self, context):
|
||||
"""Setup the Windows Volume driver.
|
||||
|
@ -59,12 +59,16 @@ FLAGS.register_opts(xenapi_nfs_opts)
|
||||
|
||||
|
||||
class XenAPINFSDriver(driver.VolumeDriver):
|
||||
def __init__(self, *args, **kwargs):
|
||||
super(XenAPINFSDriver, self).__init__(*args, **kwargs)
|
||||
self.configuration.append_config_values(xenapi_opts)
|
||||
self.configuration.append_config_values(xenapi_nfs_opts)
|
||||
|
||||
def do_setup(self, context):
|
||||
session_factory = xenapi_lib.SessionFactory(
|
||||
FLAGS.xenapi_connection_url,
|
||||
FLAGS.xenapi_connection_username,
|
||||
FLAGS.xenapi_connection_password
|
||||
self.configuration.xenapi_connection_url,
|
||||
self.configuration.xenapi_connection_username,
|
||||
self.configuration.xenapi_connection_password
|
||||
)
|
||||
self.nfs_ops = xenapi_lib.NFSBasedVolumeOperations(session_factory)
|
||||
|
||||
@ -73,8 +77,8 @@ class XenAPINFSDriver(driver.VolumeDriver):
|
||||
|
||||
def create_volume(self, volume):
|
||||
volume_details = self.nfs_ops.create_volume(
|
||||
FLAGS.xenapi_nfs_server,
|
||||
FLAGS.xenapi_nfs_serverpath,
|
||||
self.configuration.xenapi_nfs_server,
|
||||
self.configuration.xenapi_nfs_serverpath,
|
||||
volume['size'],
|
||||
volume['display_name'],
|
||||
volume['display_description']
|
||||
@ -89,8 +93,8 @@ class XenAPINFSDriver(driver.VolumeDriver):
|
||||
sr_uuid, vdi_uuid = volume['provider_location'].split('/')
|
||||
|
||||
self.nfs_ops.delete_volume(
|
||||
FLAGS.xenapi_nfs_server,
|
||||
FLAGS.xenapi_nfs_serverpath,
|
||||
self.configuration.xenapi_nfs_server,
|
||||
self.configuration.xenapi_nfs_serverpath,
|
||||
sr_uuid,
|
||||
vdi_uuid
|
||||
)
|
||||
@ -109,8 +113,8 @@ class XenAPINFSDriver(driver.VolumeDriver):
|
||||
sr_uuid=sr_uuid,
|
||||
vdi_uuid=vdi_uuid,
|
||||
sr_type='nfs',
|
||||
server=FLAGS.xenapi_nfs_server,
|
||||
serverpath=FLAGS.xenapi_nfs_serverpath,
|
||||
server=self.configuration.xenapi_nfs_server,
|
||||
serverpath=self.configuration.xenapi_nfs_serverpath,
|
||||
introduce_sr_keys=['sr_type', 'server', 'serverpath']
|
||||
)
|
||||
)
|
||||
@ -135,8 +139,8 @@ class XenAPINFSDriver(driver.VolumeDriver):
|
||||
sr_uuid, vdi_uuid = volume['provider_location'].split('/')
|
||||
|
||||
volume_details = self.nfs_ops.copy_volume(
|
||||
FLAGS.xenapi_nfs_server,
|
||||
FLAGS.xenapi_nfs_serverpath,
|
||||
self.configuration.xenapi_nfs_server,
|
||||
self.configuration.xenapi_nfs_serverpath,
|
||||
sr_uuid,
|
||||
vdi_uuid,
|
||||
target_name,
|
||||
|
@ -54,7 +54,9 @@ from cinder.openstack.common import log as logging
|
||||
from cinder.openstack.common import timeutils
|
||||
from cinder.openstack.common import uuidutils
|
||||
from cinder import quota
|
||||
from cinder.volume.configuration import Configuration
|
||||
from cinder.volume import utils as volume_utils
|
||||
from cinder.volume import volume_types
|
||||
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
@ -109,16 +111,23 @@ class VolumeManager(manager.SchedulerDependentManager):
|
||||
|
||||
RPC_API_VERSION = '1.4'
|
||||
|
||||
def __init__(self, volume_driver=None, *args, **kwargs):
|
||||
def __init__(self, volume_driver=None, service_name=None,
|
||||
*args, **kwargs):
|
||||
"""Load the driver from the one specified in args, or from flags."""
|
||||
self.configuration = Configuration(volume_manager_opts,
|
||||
config_group=service_name)
|
||||
if not volume_driver:
|
||||
volume_driver = FLAGS.volume_driver
|
||||
# Get from configuration, which will get the default
|
||||
# if its not using the multi backend
|
||||
volume_driver = self.configuration.volume_driver
|
||||
if volume_driver in MAPPING:
|
||||
LOG.warn(_("Driver path %s is deprecated, update your "
|
||||
"configuration to the new path."), volume_driver)
|
||||
self.driver = importutils.import_object(MAPPING[volume_driver])
|
||||
else:
|
||||
self.driver = importutils.import_object(volume_driver)
|
||||
volume_driver = MAPPING[volume_driver]
|
||||
self.driver = importutils.import_object(
|
||||
volume_driver,
|
||||
configuration=self.configuration)
|
||||
# update_service_capabilities needs service_name to be volume
|
||||
super(VolumeManager, self).__init__(service_name='volume',
|
||||
*args, **kwargs)
|
||||
# NOTE(vish): Implementation specific db handling is done
|
||||
|
@ -42,9 +42,9 @@ class VolumeAPI(cinder.openstack.common.rpc.proxy.RpcProxy):
|
||||
|
||||
BASE_RPC_API_VERSION = '1.0'
|
||||
|
||||
def __init__(self):
|
||||
def __init__(self, topic=None):
|
||||
super(VolumeAPI, self).__init__(
|
||||
topic=FLAGS.volume_topic,
|
||||
topic=topic or FLAGS.volume_topic,
|
||||
default_version=self.BASE_RPC_API_VERSION)
|
||||
|
||||
def create_volume(self, ctxt, volume, host,
|
||||
|
@ -27,6 +27,12 @@ FLAGS = flags.FLAGS
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
|
||||
def get_host_from_queue(queuename):
|
||||
# This assumes the queue is named something like cinder-volume
|
||||
# and does not have dot separators in the queue name
|
||||
return queuename.split('@', 1)[0].split('.', 1)[1]
|
||||
|
||||
|
||||
def notify_usage_exists(context, volume_ref, current_period=False):
|
||||
""" Generates 'exists' notification for a volume for usage auditing
|
||||
purposes.
|
||||
|
@ -727,6 +727,9 @@
|
||||
# value)
|
||||
#iscsi_port=3260
|
||||
|
||||
# Optional override to the capacity based volume backend name
|
||||
#
|
||||
#volume_backend_name=LVM_iSCSI_unique1
|
||||
|
||||
#
|
||||
# Options defined in cinder.volume.drivers.glusterfs
|
||||
@ -1125,4 +1128,20 @@
|
||||
#volume_driver=cinder.volume.drivers.lvm.LVMISCSIDriver
|
||||
|
||||
|
||||
# Total option count: 249
|
||||
#
|
||||
# Multi backend options
|
||||
#
|
||||
|
||||
# Define the names of the groups for multiple volume backends
|
||||
#enabled_backends=fakedriver,lvmdriver
|
||||
|
||||
# Define the groups as above
|
||||
#[lvmdriver]
|
||||
#volume_group=lvm-group-1
|
||||
#volume_driver=cinder.volume.drivers.lvm.LVMISCSIDriver
|
||||
#volume_backend_name=LVM_iSCSI_unique1
|
||||
#[fakedriver]
|
||||
#volume_driver=cinder.volume.driver.FakeISCSIDriver
|
||||
|
||||
|
||||
# Total option count: 251
|
||||
|
Loading…
x
Reference in New Issue
Block a user