Renaming instance_actions.instance_id column to instance_uuid.

blueprint: internal-uuids

Change-Id: I204c3fd49278c509a9bc32353445ede086cfadb2
This commit is contained in:
Naveed Massjouni 2011-12-06 16:56:12 -05:00
parent ff753cd608
commit 88b1d7739b
12 changed files with 133 additions and 66 deletions

View File

@ -1375,7 +1375,7 @@ class API(base.Base):
def get_actions(self, context, instance):
"""Retrieve actions for the given instance."""
return self.db.instance_get_actions(context, instance['id'])
return self.db.instance_get_actions(context, instance['uuid'])
@check_vm_state(state_checker.SUSPEND)
@scheduler_api.reroute_compute(state_checker.SUSPEND)

View File

@ -636,9 +636,9 @@ def instance_action_create(context, values):
return IMPL.instance_action_create(context, values)
def instance_get_actions(context, instance_id):
"""Get instance actions by instance id."""
return IMPL.instance_get_actions(context, instance_id)
def instance_get_actions(context, instance_uuid):
"""Get instance actions by instance uuid."""
return IMPL.instance_get_actions(context, instance_uuid)
def instance_get_id_to_uuid_mapping(context, ids):

View File

@ -1528,17 +1528,12 @@ def instance_action_create(context, values):
@require_admin_context
def instance_get_actions(context, instance_id):
def instance_get_actions(context, instance_uuid):
"""Return the actions associated to the given instance id"""
session = get_session()
if utils.is_uuid_like(instance_id):
instance = instance_get_by_uuid(context, instance_id, session)
instance_id = instance.id
return session.query(models.InstanceActions).\
filter_by(instance_id=instance_id).\
all()
filter_by(instance_uuid=instance_uuid).\
all()
@require_context

View File

@ -0,0 +1,71 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2011 OpenStack LLC.
# 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.
import sqlalchemy
from sqlalchemy import select, Column, ForeignKey, Integer, String
from nova import log as logging
LOG = logging.getLogger('nova.db.sqlalchemy.migrate_repo.versions')
meta = sqlalchemy.MetaData()
def _get_table(name):
return sqlalchemy.Table(name, meta, autoload=True)
def upgrade(migrate_engine):
meta.bind = migrate_engine
instance_actions = _get_table('instance_actions')
instances = _get_table('instances')
uuid_column = Column('instance_uuid', String(36),
ForeignKey('instances.uuid'))
uuid_column = Column('instance_uuid', String(36))
uuid_column.create(instance_actions)
try:
instance_actions.update().values(
instance_uuid=select(
[instances.c.uuid],
instances.c.id == instance_actions.c.instance_id)
).execute()
except Exception:
uuid_column.drop()
raise
instance_actions.c.instance_id.drop()
def downgrade(migrate_engine):
meta.bind = migrate_engine
instance_actions = _get_table('instance_actions')
instances = _get_table('instances')
id_column = Column('instance_id', Integer, ForeignKey('instances.id'))
id_column.create(instance_actions)
try:
instance_actions.update().values(
instance_id=select(
[instances.c.id],
instances.c.uuid == instance_actions.c.instance_uuid)
).execute()
except Exception:
id_column.drop()
raise
instance_actions.c.instance_uuid.drop()

View File

@ -308,8 +308,7 @@ class InstanceActions(BASE, NovaBase):
"""Represents a guest VM's actions and results"""
__tablename__ = "instance_actions"
id = Column(Integer, primary_key=True)
instance_id = Column(Integer, ForeignKey('instances.id'))
instance_uuid = Column(String(36), ForeignKey('instances.uuid'))
action = Column(String(255))
error = Column(Text)

View File

@ -62,6 +62,7 @@ def stub_out_db_instance_api(stubs):
base_options = {
'name': values['name'],
'id': values['id'],
'uuid': utils.gen_uuid(),
'reservation_id': utils.generate_uid('r'),
'image_ref': values['image_ref'],
'kernel_id': values['kernel_id'],

View File

@ -35,12 +35,11 @@ def stubout_instance_snapshot(stubs):
stubs.Set(vm_utils.VMHelper, 'fetch_image', fake_fetch_image)
def fake_wait_for_vhd_coalesce(session, instance_id, sr_ref, vdi_ref,
original_parent_uuid):
def fake_wait_for_vhd_coalesce(*args):
#TODO(sirp): Should we actually fake out the data here
return "fakeparent"
stubs.Set(vm_utils, 'wait_for_vhd_coalesce', fake_wait_for_vhd_coalesce)
stubs.Set(vm_utils, '_wait_for_vhd_coalesce', fake_wait_for_vhd_coalesce)
def stubout_session(stubs, cls, product_version=None):

View File

@ -196,7 +196,7 @@ class VMWareVMOps(object):
self._session._get_vim(),
"CreateVM_Task", vm_folder_mor,
config=config_spec, pool=res_pool_mor)
self._session._wait_for_task(instance.id, vm_create_task)
self._session._wait_for_task(instance['uuid'], vm_create_task)
LOG.debug(_("Created VM with the name %s on the ESX host") %
instance.name)
@ -243,7 +243,7 @@ class VMWareVMOps(object):
name=uploaded_vmdk_path,
datacenter=self._get_datacenter_name_and_ref()[0],
spec=vmdk_create_spec)
self._session._wait_for_task(instance.id, vmdk_create_task)
self._session._wait_for_task(instance['uuid'], vmdk_create_task)
LOG.debug(_("Created Virtual Disk of size %(vmdk_file_size_in_kb)s"
" KB on the ESX host local store "
"%(data_store_name)s") %
@ -264,7 +264,7 @@ class VMWareVMOps(object):
"DeleteDatastoreFile_Task",
service_content.fileManager,
name=flat_uploaded_vmdk_path)
self._session._wait_for_task(instance.id, vmdk_delete_task)
self._session._wait_for_task(instance['uuid'], vmdk_delete_task)
LOG.debug(_("Deleted the file %(flat_uploaded_vmdk_path)s on the "
"ESX host local store %(data_store_name)s") %
{"flat_uploaded_vmdk_path": flat_uploaded_vmdk_path,
@ -314,7 +314,7 @@ class VMWareVMOps(object):
self._session._get_vim(),
"ReconfigVM_Task", vm_ref,
spec=vmdk_attach_config_spec)
self._session._wait_for_task(instance.id, reconfig_task)
self._session._wait_for_task(instance['uuid'], reconfig_task)
LOG.debug(_("Reconfigured VM instance %s to attach the image "
"disk") % instance.name)
@ -327,7 +327,7 @@ class VMWareVMOps(object):
power_on_task = self._session._call_method(
self._session._get_vim(),
"PowerOnVM_Task", vm_ref)
self._session._wait_for_task(instance.id, power_on_task)
self._session._wait_for_task(instance['uuid'], power_on_task)
LOG.debug(_("Powered on the VM instance %s") % instance.name)
_power_on_vm()
@ -382,7 +382,7 @@ class VMWareVMOps(object):
description="Taking Snapshot of the VM",
memory=True,
quiesce=True)
self._session._wait_for_task(instance.id, snapshot_task)
self._session._wait_for_task(instance['uuid'], snapshot_task)
LOG.debug(_("Created Snapshot of the VM instance %s ") %
instance.name)
@ -438,7 +438,7 @@ class VMWareVMOps(object):
destDatacenter=dc_ref,
destSpec=copy_spec,
force=False)
self._session._wait_for_task(instance.id, copy_disk_task)
self._session._wait_for_task(instance['uuid'], copy_disk_task)
LOG.debug(_("Copied disk data before snapshot of the VM "
"instance %s") % instance.name)
@ -479,7 +479,7 @@ class VMWareVMOps(object):
service_content.virtualDiskManager,
name=dest_vmdk_file_location,
datacenter=dc_ref)
self._session._wait_for_task(instance.id, remove_disk_task)
self._session._wait_for_task(instance['uuid'], remove_disk_task)
LOG.debug(_("Deleted temporary vmdk file %s")
% dest_vmdk_file_location)
@ -527,7 +527,7 @@ class VMWareVMOps(object):
LOG.debug(_("Doing hard reboot of VM %s") % instance.name)
reset_task = self._session._call_method(self._session._get_vim(),
"ResetVM_Task", vm_ref)
self._session._wait_for_task(instance.id, reset_task)
self._session._wait_for_task(instance['uuid'], reset_task)
LOG.debug(_("Did hard reboot of VM %s") % instance.name)
def destroy(self, instance, network_info):
@ -563,7 +563,7 @@ class VMWareVMOps(object):
poweroff_task = self._session._call_method(
self._session._get_vim(),
"PowerOffVM_Task", vm_ref)
self._session._wait_for_task(instance.id, poweroff_task)
self._session._wait_for_task(instance['uuid'], poweroff_task)
LOG.debug(_("Powered off the VM %s") % instance.name)
# Un-register the VM
@ -593,7 +593,7 @@ class VMWareVMOps(object):
"DeleteDatastoreFile_Task",
self._session._get_vim().get_service_content().fileManager,
name=dir_ds_compliant_path)
self._session._wait_for_task(instance.id, delete_task)
self._session._wait_for_task(instance['uuid'], delete_task)
LOG.debug(_("Deleted contents of the VM %(name)s from "
"datastore %(datastore_name)s") %
({'name': instance.name,
@ -628,7 +628,7 @@ class VMWareVMOps(object):
LOG.debug(_("Suspending the VM %s ") % instance.name)
suspend_task = self._session._call_method(self._session._get_vim(),
"SuspendVM_Task", vm_ref)
self._session._wait_for_task(instance.id, suspend_task)
self._session._wait_for_task(instance['uuid'], suspend_task)
LOG.debug(_("Suspended the VM %s ") % instance.name)
# Raise Exception if VM is poweredOff
elif pwr_state == "poweredOff":
@ -652,7 +652,7 @@ class VMWareVMOps(object):
suspend_task = self._session._call_method(
self._session._get_vim(),
"PowerOnVM_Task", vm_ref)
self._session._wait_for_task(instance.id, suspend_task)
self._session._wait_for_task(instance['uuid'], suspend_task)
LOG.debug(_("Resumed the VM %s ") % instance.name)
else:
reason = _("instance is not in a suspended state")
@ -761,7 +761,7 @@ class VMWareVMOps(object):
reconfig_task = self._session._call_method(self._session._get_vim(),
"ReconfigVM_Task", vm_ref,
spec=machine_id_change_spec)
self._session._wait_for_task(instance.id, reconfig_task)
self._session._wait_for_task(instance['uuid'], reconfig_task)
LOG.debug(_("Reconfigured VM instance %(name)s to set the machine id "
"with ip - %(ip_addr)s") %
({'name': instance.name,

View File

@ -348,20 +348,20 @@ class VMWareAPISession(object):
self._create_session()
return self.vim
def _wait_for_task(self, instance_id, task_ref):
def _wait_for_task(self, instance_uuid, task_ref):
"""
Return a Deferred that will give the result of the given task.
The task is polled until it completes.
"""
done = event.Event()
loop = utils.LoopingCall(self._poll_task, instance_id, task_ref,
loop = utils.LoopingCall(self._poll_task, instance_uuid, task_ref,
done)
loop.start(FLAGS.vmwareapi_task_poll_interval, now=True)
ret_val = done.wait()
loop.stop()
return ret_val
def _poll_task(self, instance_id, task_ref, done):
def _poll_task(self, instance_uuid, task_ref, done):
"""
Poll the given task, and fires the given Deferred if we
get a result.
@ -371,7 +371,7 @@ class VMWareAPISession(object):
task_ref, "Task", "info")
task_name = task_info.name
action = dict(
instance_id=int(instance_id),
instance_uuid=instance_uuid,
action=task_name[0:255],
error=None)
if task_info.state in ['queued', 'running']:

View File

@ -316,7 +316,7 @@ class VMHelper(HelperBase):
"%(vm_ref)s") % locals())
@classmethod
def create_snapshot(cls, session, instance_id, vm_ref, label):
def create_snapshot(cls, session, instance, vm_ref, label):
"""Creates Snapshot (Template) VM, Snapshot VBD, Snapshot VDI,
Snapshot VHD"""
LOG.debug(_("Snapshotting VM %(vm_ref)s with label '%(label)s'...")
@ -328,7 +328,7 @@ class VMHelper(HelperBase):
original_parent_uuid = get_vhd_parent_uuid(session, vm_vdi_ref)
task = session.call_xenapi('Async.VM.snapshot', vm_ref, label)
template_vm_ref = session.wait_for_task(task, instance_id)
template_vm_ref = session.wait_for_task(task, instance['uuid'])
template_vdi_rec = cls.get_vdi_for_vm_safely(session,
template_vm_ref)[1]
template_vdi_uuid = template_vdi_rec["uuid"]
@ -336,8 +336,8 @@ class VMHelper(HelperBase):
LOG.debug(_('Created snapshot %(template_vm_ref)s from'
' VM %(vm_ref)s.') % locals())
parent_uuid = wait_for_vhd_coalesce(
session, instance_id, sr_ref, vm_vdi_ref, original_parent_uuid)
parent_uuid = _wait_for_vhd_coalesce(
session, instance, sr_ref, vm_vdi_ref, original_parent_uuid)
#TODO(sirp): we need to assert only one parent, not parents two deep
template_vdi_uuids = {'image': parent_uuid,
@ -382,7 +382,7 @@ class VMHelper(HelperBase):
kwargs = {'params': pickle.dumps(params)}
task = session.async_call_plugin('glance', 'upload_vhd', kwargs)
session.wait_for_task(task, instance.id)
session.wait_for_task(task, instance['uuid'])
@classmethod
def resize_disk(cls, session, vdi_ref, instance_type):
@ -550,7 +550,7 @@ class VMHelper(HelperBase):
kwargs = {'params': pickle.dumps(params)}
task = session.async_call_plugin('glance', 'download_vhd', kwargs)
result = session.wait_for_task(task, instance_id)
result = session.wait_for_task(task, instance['uuid'])
# 'download_vhd' will return a json encoded string containing
# a list of dictionaries describing VDIs. The dictionary will
# contain 'vdi_type' and 'vdi_uuid' keys. 'vdi_type' can be
@ -560,7 +560,7 @@ class VMHelper(HelperBase):
LOG.debug(_("xapi 'download_vhd' returned VDI of "
"type '%(vdi_type)s' with UUID '%(vdi_uuid)s'" % vdi))
cls.scan_sr(session, instance_id, sr_ref)
cls.scan_sr(session, instance, sr_ref)
# Pull out the UUID of the first VDI (which is the os VDI)
os_vdi_uuid = vdis[0]['vdi_uuid']
@ -675,7 +675,7 @@ class VMHelper(HelperBase):
# Let the plugin copy the correct number of bytes.
args['image-size'] = str(vdi_size)
task = session.async_call_plugin('glance', fn, args)
filename = session.wait_for_task(task, instance_id)
filename = session.wait_for_task(task, instance['uuid'])
# Remove the VDI as it is not needed anymore.
session.call_xenapi("VDI.destroy", vdi_ref)
LOG.debug(_("Kernel/Ramdisk VDI %s destroyed"), vdi_ref)
@ -909,12 +909,13 @@ class VMHelper(HelperBase):
raise exception.CouldNotFetchMetrics()
@classmethod
def scan_sr(cls, session, instance_id=None, sr_ref=None):
def scan_sr(cls, session, instance=None, sr_ref=None):
"""Scans the SR specified by sr_ref"""
if sr_ref:
LOG.debug(_("Re-scanning SR %s"), sr_ref)
task = session.call_xenapi('Async.SR.scan', sr_ref)
session.wait_for_task(task, instance_id)
instance_uuid = instance['uuid'] if instance else None
session.wait_for_task(task, instance_uuid)
@classmethod
def scan_default_sr(cls, session):
@ -1128,8 +1129,8 @@ def walk_vdi_chain(session, vdi_uuid):
break
def wait_for_vhd_coalesce(session, instance_id, sr_ref, vdi_ref,
original_parent_uuid):
def _wait_for_vhd_coalesce(session, instance, sr_ref, vdi_ref,
original_parent_uuid):
""" Spin until the parent VHD is coalesced into its parent VHD
Before coalesce:
@ -1152,7 +1153,7 @@ def wait_for_vhd_coalesce(session, instance_id, sr_ref, vdi_ref,
" %(max_attempts)d), giving up...") % locals())
raise exception.Error(msg)
VMHelper.scan_sr(session, instance_id, sr_ref)
VMHelper.scan_sr(session, instance, sr_ref)
parent_uuid = get_vhd_parent_uuid(session, vdi_ref)
if original_parent_uuid and (parent_uuid != original_parent_uuid):
LOG.debug(_("Parent %(parent_uuid)s doesn't match original parent"

View File

@ -615,7 +615,7 @@ class VMOps(object):
label = "%s-snapshot" % instance.name
try:
template_vm_ref, template_vdi_uuids = VMHelper.create_snapshot(
self._session, instance.id, vm_ref, label)
self._session, instance, vm_ref, label)
return template_vm_ref, template_vdi_uuids
except self.XenAPI.Failure, exc:
logging.error(_("Unable to Snapshot %(vm_ref)s: %(exc)s")
@ -635,7 +635,7 @@ class VMOps(object):
task = self._session.async_call_plugin('migration',
'transfer_vhd',
_params)
self._session.wait_for_task(task, instance_id)
self._session.wait_for_task(task, instance_uuid)
except self.XenAPI.Failure:
msg = _("Failed to transfer vhd to new host")
raise exception.MigrationError(reason=msg)
@ -797,7 +797,7 @@ class VMOps(object):
task = self._session.async_call_plugin('migration',
'move_vhds_into_sr', {'params': pickle.dumps(params)})
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
# Now we rescan the SR so we find the VHDs
VMHelper.scan_default_sr(self._session)
@ -848,7 +848,7 @@ class VMOps(object):
else:
task = self._session.call_xenapi('Async.VM.clean_reboot', vm_ref)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
def get_agent_version(self, instance):
"""Get the version of the agent running on the VM instance."""
@ -989,7 +989,7 @@ class VMOps(object):
else:
task = self._session.call_xenapi("Async.VM.clean_shutdown",
vm_ref)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
except self.XenAPI.Failure, exc:
LOG.exception(exc)
@ -1021,7 +1021,7 @@ class VMOps(object):
for vdi_ref in vdi_refs:
try:
task = self._session.call_xenapi('Async.VDI.destroy', vdi_ref)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
except self.XenAPI.Failure, exc:
LOG.exception(exc)
@ -1091,7 +1091,7 @@ class VMOps(object):
instance_uuid = instance['uuid']
try:
task = self._session.call_xenapi('Async.VM.destroy', vm_ref)
self._session.wait_for_task(task, instance_id)
self._session.wait_for_task(task, instance_uuid)
except self.XenAPI.Failure, exc:
LOG.exception(exc)
@ -1147,26 +1147,26 @@ class VMOps(object):
"""Pause VM instance."""
vm_ref = self._get_vm_opaque_ref(instance)
task = self._session.call_xenapi('Async.VM.pause', vm_ref)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
def unpause(self, instance):
"""Unpause VM instance."""
vm_ref = self._get_vm_opaque_ref(instance)
task = self._session.call_xenapi('Async.VM.unpause', vm_ref)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
def suspend(self, instance):
"""Suspend the specified instance."""
vm_ref = self._get_vm_opaque_ref(instance)
task = self._session.call_xenapi('Async.VM.suspend', vm_ref)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
def resume(self, instance):
"""Resume the specified instance."""
vm_ref = self._get_vm_opaque_ref(instance)
task = self._session.call_xenapi('Async.VM.resume',
vm_ref, False, True)
self._session.wait_for_task(task, instance.id)
self._session.wait_for_task(task, instance['uuid'])
def rescue(self, context, instance, network_info, image_meta):
"""Rescue the specified instance.
@ -1402,7 +1402,7 @@ class VMOps(object):
task = self._session.async_call_plugin("xenhost", method,
args=arg_dict)
#args={"params": arg_dict})
ret = self._session.wait_for_task(task, task_id)
ret = self._session.wait_for_task(task, str(task_id))
except self.XenAPI.Failure as e:
ret = e
LOG.error(_("The call to %(method)s returned an error: %(e)s.")
@ -1557,7 +1557,7 @@ class VMOps(object):
args.update(addl_args or {})
try:
task = self._session.async_call_plugin(plugin, method, args)
ret = self._session.wait_for_task(task, instance_id)
ret = self._session.wait_for_task(task, instance_uuid)
except self.XenAPI.Failure, e:
ret = None
err_msg = e.details[-1].splitlines()[-1]

View File

@ -508,7 +508,7 @@ class XenAPISession(object):
session.xenapi.Async.host.call_plugin,
self.get_xenapi_host(), plugin, fn, args)
def wait_for_task(self, task, id=None):
def wait_for_task(self, task, uuid=None):
"""Return the result of the given task. The task is polled
until it completes."""
done = event.Event()
@ -525,9 +525,10 @@ class XenAPISession(object):
# Ensure action is never > 255
action = dict(action=name[:255], error=None)
log_instance_actions = FLAGS.xenapi_log_instance_actions and id
log_instance_actions = (FLAGS.xenapi_log_instance_actions and
uuid)
if log_instance_actions:
action["instance_id"] = int(id)
action["instance_uuid"] = uuid
if status == "pending":
return
@ -608,10 +609,10 @@ class HostState(object):
we can get host status information using xenapi.
"""
LOG.debug(_("Updating host stats"))
# Make it something unlikely to match any actual instance ID
# Make it something unlikely to match any actual instance UUID
task_id = random.randint(-80000, -70000)
task = self._session.async_call_plugin("xenhost", "host_data", {})
task_result = self._session.wait_for_task(task, task_id)
task_result = self._session.wait_for_task(task, str(task_id))
if not task_result:
task_result = json.dumps("")
try: