fix bug in create volume from snapshot

This commit is contained in:
Edison Su 2013-05-14 16:20:20 -07:00
parent 564d3d6fa0
commit 68ffe1c706
8 changed files with 266 additions and 22 deletions

View File

@ -42,4 +42,5 @@ public interface StorageProcessor {
public Answer createVolume(CreateObjectCommand cmd); public Answer createVolume(CreateObjectCommand cmd);
public Answer createSnapshot(CreateObjectCommand cmd); public Answer createSnapshot(CreateObjectCommand cmd);
public Answer deleteVolume(DeleteCommand cmd); public Answer deleteVolume(DeleteCommand cmd);
public Answer createVolumeFromSnapshot(CopyCommand cmd);
} }

View File

@ -82,6 +82,8 @@ public class StorageSubsystemCommandHandlerBase implements StorageSubsystemComma
} }
} else if (srcData.getObjectType() == DataObjectType.SNAPSHOT && srcData.getDataStore().getRole() == DataStoreRole.Primary) { } else if (srcData.getObjectType() == DataObjectType.SNAPSHOT && srcData.getDataStore().getRole() == DataStoreRole.Primary) {
return processor.backupSnasphot(cmd); return processor.backupSnasphot(cmd);
} else if (srcData.getObjectType() == DataObjectType.SNAPSHOT && destData.getObjectType() == DataObjectType.VOLUME) {
return processor.createVolumeFromSnapshot(cmd);
} }
return new Answer(cmd, false, "not implemented yet"); return new Answer(cmd, false, "not implemented yet");

View File

@ -231,12 +231,11 @@ public class AncientDataMotionStrategy implements DataMotionStrategy {
.getDefaultValue())); .getDefaultValue()));
CopyCommand cmd = new CopyCommand(srcData.getTO(), volObj.getTO(), _createVolumeFromSnapshotWait); CopyCommand cmd = new CopyCommand(srcData.getTO(), volObj.getTO(), _createVolumeFromSnapshotWait);
EndPoint ep = selector.select(snapObj, volObj);
Answer answer = ep.sendMessage(cmd);
Answer answer = storageMgr
.sendToPool(pool, cmd);
return answer; return answer;
} catch (StorageUnavailableException e) { } catch (Exception e) {
s_logger.error(basicErrMsg, e); s_logger.error(basicErrMsg, e);
throw new CloudRuntimeException(basicErrMsg); throw new CloudRuntimeException(basicErrMsg);
} finally { } finally {

View File

@ -360,7 +360,7 @@ public class SnapshotTest extends CloudStackTestNGBase {
return null; return null;
} }
@Test //@Test
public void createSnapshot() { public void createSnapshot() {
VolumeInfo vol = createCopyBaseImage(); VolumeInfo vol = createCopyBaseImage();
SnapshotVO snapshotVO = createSnapshotInDb(vol); SnapshotVO snapshotVO = createSnapshotInDb(vol);
@ -394,7 +394,7 @@ public class SnapshotTest extends CloudStackTestNGBase {
return image; return image;
} }
@Test //@Test
public void createTemplateFromSnapshot() { public void createTemplateFromSnapshot() {
VolumeInfo vol = createCopyBaseImage(); VolumeInfo vol = createCopyBaseImage();
SnapshotVO snapshotVO = createSnapshotInDb(vol); SnapshotVO snapshotVO = createSnapshotInDb(vol);
@ -416,4 +416,24 @@ public class SnapshotTest extends CloudStackTestNGBase {
DataStore imageStore = this.dataStoreMgr.getImageStore(this.dcId); DataStore imageStore = this.dataStoreMgr.getImageStore(this.dcId);
this.imageService.createTemplateFromSnapshotAsync(snapshot, tmpl, imageStore); this.imageService.createTemplateFromSnapshotAsync(snapshot, tmpl, imageStore);
} }
@Test
public void createVolumeFromSnapshot() {
VolumeInfo vol = createCopyBaseImage();
SnapshotVO snapshotVO = createSnapshotInDb(vol);
SnapshotInfo snapshot = this.snapshotFactory.getSnapshot(snapshotVO.getId(), vol.getDataStore());
boolean result = false;
for (SnapshotStrategy strategy : this.snapshotStrategies) {
if (strategy.canHandle(snapshot)) {
snapshot = strategy.takeSnapshot(snapshot);
result = true;
}
}
AssertJUnit.assertTrue(result);
VolumeVO volVO = createVolume(vol.getTemplateId(), vol.getPoolId());
VolumeInfo newVol = this.volFactory.getVolume(volVO.getId());
this.volumeService.createVolumeFromSnapshot(newVol, newVol.getDataStore(), snapshot);
}
} }

View File

@ -539,7 +539,11 @@ public class VolumeServiceImpl implements VolumeService {
} }
try { try {
if (result.isSuccess()) {
volume.processEvent(event, result.getAnswer());
} else {
volume.processEvent(event); volume.processEvent(event);
}
snapshot.processEvent(event); snapshot.processEvent(event);
} catch (Exception e) { } catch (Exception e) {
s_logger.debug("create volume from snapshot failed", e); s_logger.debug("create volume from snapshot failed", e);

View File

@ -20,7 +20,6 @@ package com.cloud.hypervisor.kvm.storage;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.IOException;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import java.text.DateFormat; import java.text.DateFormat;
import java.text.MessageFormat; import java.text.MessageFormat;
@ -47,9 +46,9 @@ import org.apache.cloudstack.storage.to.SnapshotObjectTO;
import org.apache.cloudstack.storage.to.TemplateObjectTO; import org.apache.cloudstack.storage.to.TemplateObjectTO;
import org.apache.cloudstack.storage.to.VolumeObjectTO; import org.apache.cloudstack.storage.to.VolumeObjectTO;
import org.apache.cloudstack.utils.qemu.QemuImg; import org.apache.cloudstack.utils.qemu.QemuImg;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import org.apache.cloudstack.utils.qemu.QemuImgException; import org.apache.cloudstack.utils.qemu.QemuImgException;
import org.apache.cloudstack.utils.qemu.QemuImgFile; import org.apache.cloudstack.utils.qemu.QemuImgFile;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.libvirt.Connect; import org.libvirt.Connect;
import org.libvirt.Domain; import org.libvirt.Domain;
@ -58,37 +57,28 @@ import org.libvirt.DomainSnapshot;
import org.libvirt.LibvirtException; import org.libvirt.LibvirtException;
import com.cloud.agent.api.Answer; import com.cloud.agent.api.Answer;
import com.cloud.agent.api.AttachVolumeAnswer;
import com.cloud.agent.api.BackupSnapshotAnswer;
import com.cloud.agent.api.ManageSnapshotAnswer;
import com.cloud.agent.api.ManageSnapshotCommand;
import com.cloud.agent.api.storage.CreateAnswer;
import com.cloud.agent.api.storage.CreatePrivateTemplateAnswer;
import com.cloud.agent.api.storage.PrimaryStorageDownloadAnswer; import com.cloud.agent.api.storage.PrimaryStorageDownloadAnswer;
import com.cloud.agent.api.to.DataStoreTO; import com.cloud.agent.api.to.DataStoreTO;
import com.cloud.agent.api.to.DataTO; import com.cloud.agent.api.to.DataTO;
import com.cloud.agent.api.to.DiskTO; import com.cloud.agent.api.to.DiskTO;
import com.cloud.agent.api.to.NfsTO; import com.cloud.agent.api.to.NfsTO;
import com.cloud.agent.api.to.StorageFilerTO;
import com.cloud.agent.api.to.VolumeTO;
import com.cloud.exception.InternalErrorException; import com.cloud.exception.InternalErrorException;
import com.cloud.hypervisor.kvm.resource.LibvirtComputingResource; import com.cloud.hypervisor.kvm.resource.LibvirtComputingResource;
import com.cloud.hypervisor.kvm.resource.LibvirtConnection; import com.cloud.hypervisor.kvm.resource.LibvirtConnection;
import com.cloud.hypervisor.kvm.resource.LibvirtDomainXMLParser; import com.cloud.hypervisor.kvm.resource.LibvirtDomainXMLParser;
import com.cloud.hypervisor.kvm.resource.LibvirtVMDef.DiskDef; import com.cloud.hypervisor.kvm.resource.LibvirtVMDef.DiskDef;
import com.cloud.storage.JavaStorageLayer; import com.cloud.storage.JavaStorageLayer;
import com.cloud.storage.StorageLayer;
import com.cloud.storage.Storage.ImageFormat; import com.cloud.storage.Storage.ImageFormat;
import com.cloud.storage.Storage.StoragePoolType; import com.cloud.storage.Storage.StoragePoolType;
import com.cloud.storage.StorageLayer;
import com.cloud.storage.resource.StorageProcessor; import com.cloud.storage.resource.StorageProcessor;
import com.cloud.storage.template.Processor; import com.cloud.storage.template.Processor;
import com.cloud.storage.template.Processor.FormatInfo;
import com.cloud.storage.template.QCOW2Processor; import com.cloud.storage.template.QCOW2Processor;
import com.cloud.storage.template.TemplateLocation; import com.cloud.storage.template.TemplateLocation;
import com.cloud.storage.template.Processor.FormatInfo;
import com.cloud.utils.NumbersUtil; import com.cloud.utils.NumbersUtil;
import com.cloud.utils.exception.CloudRuntimeException; import com.cloud.utils.exception.CloudRuntimeException;
import com.cloud.utils.script.Script; import com.cloud.utils.script.Script;
import com.cloud.vm.DiskProfile;
public class KVMStorageProcessor implements StorageProcessor { public class KVMStorageProcessor implements StorageProcessor {
private static final Logger s_logger = Logger.getLogger(KVMStorageProcessor.class); private static final Logger s_logger = Logger.getLogger(KVMStorageProcessor.class);
@ -870,4 +860,46 @@ public class KVMStorageProcessor implements StorageProcessor {
} }
} }
@Override
public Answer createVolumeFromSnapshot(CopyCommand cmd) {
try {
DataTO srcData = cmd.getSrcTO();
SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData;
DataTO destData = cmd.getDestTO();
PrimaryDataStoreTO pool = (PrimaryDataStoreTO)destData.getDataStore();
DataStoreTO imageStore = srcData.getDataStore();
if (!(imageStore instanceof NfsTO)) {
return new CopyCmdAnswer("unsupported protocol");
}
NfsTO nfsImageStore = (NfsTO)imageStore;
String snapshotPath = snapshot.getPath();
int index = snapshotPath.lastIndexOf("/");
snapshotPath = snapshotPath.substring(0, index);
String snapshotName = snapshotPath.substring(index + 1);
KVMStoragePool secondaryPool = storagePoolMgr.getStoragePoolByURI(
nfsImageStore.getUrl() + File.separator + snapshotPath);
KVMPhysicalDisk snapshotDisk = secondaryPool.getPhysicalDisk(snapshotName);
String primaryUuid = pool.getUuid();
KVMStoragePool primaryPool = storagePoolMgr
.getStoragePool(pool.getPoolType(),
primaryUuid);
String volUuid = UUID.randomUUID().toString();
KVMPhysicalDisk disk = storagePoolMgr.copyPhysicalDisk(snapshotDisk,
volUuid, primaryPool);
VolumeObjectTO newVol = new VolumeObjectTO();
newVol.setPath(disk.getName());
newVol.setSize(disk.getVirtualSize());
return new CopyCmdAnswer(
newVol);
} catch (CloudRuntimeException e) {
return new CopyCmdAnswer(e.toString()
);
}
}
} }

View File

@ -44,6 +44,7 @@ import org.apache.log4j.Logger;
import com.cloud.agent.api.Answer; import com.cloud.agent.api.Answer;
import com.cloud.agent.api.BackupSnapshotAnswer; import com.cloud.agent.api.BackupSnapshotAnswer;
import com.cloud.agent.api.Command; import com.cloud.agent.api.Command;
import com.cloud.agent.api.CreateVolumeFromSnapshotAnswer;
import com.cloud.agent.api.ManageSnapshotAnswer; import com.cloud.agent.api.ManageSnapshotAnswer;
import com.cloud.agent.api.ManageSnapshotCommand; import com.cloud.agent.api.ManageSnapshotCommand;
import com.cloud.agent.api.storage.CreatePrivateTemplateAnswer; import com.cloud.agent.api.storage.CreatePrivateTemplateAnswer;
@ -980,6 +981,7 @@ public class VmwareStorageProcessor implements StorageProcessor {
VolumeObjectTO newVol = new VolumeObjectTO(); VolumeObjectTO newVol = new VolumeObjectTO();
newVol.setPath(volumeUuid); newVol.setPath(volumeUuid);
newVol.setSize(volume.getSize() / (1024L * 1024L));
return new CreateObjectAnswer(newVol); return new CreateObjectAnswer(newVol);
} finally { } finally {
s_logger.info("Destroy dummy VM after volume creation"); s_logger.info("Destroy dummy VM after volume creation");
@ -1154,4 +1156,120 @@ public class VmwareStorageProcessor implements StorageProcessor {
return new Answer(cmd, false, msg); return new Answer(cmd, false, msg);
} }
} }
private Long restoreVolumeFromSecStorage(VmwareHypervisorHost hyperHost, DatastoreMO primaryDsMo, String newVolumeName,
String secStorageUrl, String secStorageDir, String backupName) throws Exception {
String secondaryMountPoint = mountService.getMountPoint(secStorageUrl);
String srcOVAFileName = secondaryMountPoint + "/" + secStorageDir + "/"
+ backupName + "." + ImageFormat.OVA.getFileExtension();
String snapshotDir = "";
if (backupName.contains("/")){
snapshotDir = backupName.split("/")[0];
}
File ovafile = new File(srcOVAFileName);
String srcOVFFileName = secondaryMountPoint + "/" + secStorageDir + "/"
+ backupName + ".ovf";
File ovfFile = new File(srcOVFFileName);
// String srcFileName = getOVFFilePath(srcOVAFileName);
if (!ovfFile.exists()) {
srcOVFFileName = getOVFFilePath(srcOVAFileName);
if(srcOVFFileName == null && ovafile.exists() ) { // volss: ova file exists; o/w can't do tar
Script command = new Script("tar", 0, s_logger);
command.add("--no-same-owner");
command.add("-xf", srcOVAFileName);
command.setWorkDir(secondaryMountPoint + "/" + secStorageDir + "/" + snapshotDir);
s_logger.info("Executing command: " + command.toString());
String result = command.execute();
if(result != null) {
String msg = "Unable to unpack snapshot OVA file at: " + srcOVAFileName;
s_logger.error(msg);
throw new Exception(msg);
}
} else {
String msg = "Unable to find snapshot OVA file at: " + srcOVAFileName;
s_logger.error(msg);
throw new Exception(msg);
}
srcOVFFileName = getOVFFilePath(srcOVAFileName);
}
if(srcOVFFileName == null) {
String msg = "Unable to locate OVF file in template package directory: " + srcOVAFileName;
s_logger.error(msg);
throw new Exception(msg);
}
VirtualMachineMO clonedVm = null;
try {
hyperHost.importVmFromOVF(srcOVFFileName, newVolumeName, primaryDsMo, "thin");
clonedVm = hyperHost.findVmOnHyperHost(newVolumeName);
if(clonedVm == null)
throw new Exception("Unable to create container VM for volume creation");
clonedVm.moveAllVmDiskFiles(primaryDsMo, "", false);
clonedVm.detachAllDisks();
return _storage.getSize(srcOVFFileName);
} finally {
if(clonedVm != null) {
clonedVm.detachAllDisks();
clonedVm.destroy();
}
}
}
@Override
public Answer createVolumeFromSnapshot(CopyCommand cmd) {
DataTO srcData = cmd.getSrcTO();
SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData;
DataTO destData = cmd.getDestTO();
PrimaryDataStoreTO pool = (PrimaryDataStoreTO)destData.getDataStore();
DataStoreTO imageStore = srcData.getDataStore();
if (!(imageStore instanceof NfsTO)) {
return new CopyCmdAnswer("unsupported protocol");
}
NfsTO nfsImageStore = (NfsTO)imageStore;
String primaryStorageNameLabel = pool.getUuid();
String secondaryStorageUrl = nfsImageStore.getUrl();
String backedUpSnapshotUuid = snapshot.getPath();
int index = backedUpSnapshotUuid.lastIndexOf(File.separator);
String backupPath = backedUpSnapshotUuid.substring(0, index);
backedUpSnapshotUuid = backedUpSnapshotUuid.substring(index + 1);
String details = null;
String newVolumeName = UUID.randomUUID().toString().replaceAll("-", "");
VmwareContext context = hostService.getServiceContext(cmd);
try {
VmwareHypervisorHost hyperHost = hostService.getHyperHost(context, cmd);
ManagedObjectReference morPrimaryDs = HypervisorHostHelper.findDatastoreWithBackwardsCompatibility(hyperHost,
primaryStorageNameLabel);
if (morPrimaryDs == null) {
String msg = "Unable to find datastore: " + primaryStorageNameLabel;
s_logger.error(msg);
throw new Exception(msg);
}
DatastoreMO primaryDsMo = new DatastoreMO(hyperHost.getContext(), morPrimaryDs);
Long size = restoreVolumeFromSecStorage(hyperHost, primaryDsMo,
newVolumeName, secondaryStorageUrl, backupPath, backedUpSnapshotUuid);
VolumeObjectTO newVol = new VolumeObjectTO();
newVol.setPath(newVolumeName);
newVol.setSize(size);
return new CopyCmdAnswer(newVol);
} catch (Throwable e) {
if (e instanceof RemoteException) {
hostService.invalidateServiceContext(context);
}
s_logger.error("Unexpecpted exception ", e);
details = "CreateVolumeFromSnapshotCommand exception: " + StringUtils.getExceptionStackInfo(e);
}
return new CopyCmdAnswer(details);
}
} }

View File

@ -55,6 +55,7 @@ import org.apache.xmlrpc.XmlRpcException;
import com.cloud.agent.api.Answer; import com.cloud.agent.api.Answer;
import com.cloud.agent.api.CreateStoragePoolCommand; import com.cloud.agent.api.CreateStoragePoolCommand;
import com.cloud.agent.api.CreateVolumeFromSnapshotAnswer;
import com.cloud.agent.api.to.DataObjectType; import com.cloud.agent.api.to.DataObjectType;
import com.cloud.agent.api.to.DataStoreTO; import com.cloud.agent.api.to.DataStoreTO;
import com.cloud.agent.api.to.DataTO; import com.cloud.agent.api.to.DataTO;
@ -1402,4 +1403,71 @@ public class XenServerStorageProcessor implements StorageProcessor {
} }
return new CopyCmdAnswer(details); return new CopyCmdAnswer(details);
} }
@Override
public Answer createVolumeFromSnapshot(CopyCommand cmd) {
Connection conn = this.hypervisorResource.getConnection();
DataTO srcData = cmd.getSrcTO();
SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData;
DataTO destData = cmd.getDestTO();
PrimaryDataStoreTO pool = (PrimaryDataStoreTO)destData.getDataStore();
DataStoreTO imageStore = srcData.getDataStore();
if (!(imageStore instanceof NfsTO)) {
return new CopyCmdAnswer("unsupported protocol");
}
NfsTO nfsImageStore = (NfsTO)imageStore;
String primaryStorageNameLabel = pool.getUuid();
String secondaryStorageUrl = nfsImageStore.getUrl();
int wait = cmd.getWait();
boolean result = false;
// Generic error message.
String details = null;
String volumeUUID = null;
if (secondaryStorageUrl == null) {
details += " because the URL passed: " + secondaryStorageUrl + " is invalid.";
return new CopyCmdAnswer(details);
}
try {
SR primaryStorageSR = this.hypervisorResource.getSRByNameLabelandHost(conn, primaryStorageNameLabel);
if (primaryStorageSR == null) {
throw new InternalErrorException("Could not create volume from snapshot because the primary Storage SR could not be created from the name label: "
+ primaryStorageNameLabel);
}
// Get the absolute path of the snapshot on the secondary storage.
String snapshotInstallPath = snapshot.getPath();
int index = snapshotInstallPath.lastIndexOf(File.separator);
String snapshotName = snapshotInstallPath.substring(index + 1);
if (!snapshotName.startsWith("VHD-") && !snapshotName.endsWith(".vhd")) {
snapshotInstallPath = snapshotInstallPath + ".vhd";
}
URI snapshotURI = new URI(secondaryStorageUrl + File.separator + snapshotInstallPath);
String snapshotPath = snapshotURI.getHost() + ":" + snapshotURI.getPath();
String srUuid = primaryStorageSR.getUuid(conn);
volumeUUID = copy_vhd_from_secondarystorage(conn, snapshotPath, srUuid, wait);
result = true;
VDI volume = VDI.getByUuid(conn, volumeUUID);
VDI.Record vdir = volume.getRecord(conn);
VolumeObjectTO newVol = new VolumeObjectTO();
newVol.setPath(volumeUUID);
newVol.setSize(vdir.virtualSize);
return new CopyCmdAnswer(newVol);
} catch (XenAPIException e) {
details += " due to " + e.toString();
s_logger.warn(details, e);
} catch (Exception e) {
details += " due to " + e.getMessage();
s_logger.warn(details, e);
}
if (!result) {
// Is this logged at a higher level?
s_logger.error(details);
}
// In all cases return something.
return new CopyCmdAnswer(details);
}
} }