mirror of
				https://github.com/apache/cloudstack.git
				synced 2025-10-26 08:42:29 +01:00 
			
		
		
		
	Support online migration of a virtual disk on XenServer from non-managed storage to managed storage
This commit is contained in:
		
							parent
							
								
									46c56eaaf9
								
							
						
					
					
						commit
						3db33b7385
					
				| @ -97,10 +97,18 @@ public class MigrateVolumeCommand extends Command { | ||||
|         return destData; | ||||
|     } | ||||
| 
 | ||||
|     public void setSrcDetails(Map<String, String> details) { | ||||
|         srcDetails = details; | ||||
|     } | ||||
| 
 | ||||
|     public Map<String, String> getSrcDetails() { | ||||
|         return srcDetails; | ||||
|     } | ||||
| 
 | ||||
|     public void setDestDetails(Map<String, String> details) { | ||||
|         destDetails = details; | ||||
|     } | ||||
| 
 | ||||
|     public Map<String, String> getDestDetails() { | ||||
|         return destDetails; | ||||
|     } | ||||
|  | ||||
| @ -25,9 +25,12 @@ import com.cloud.host.Host; | ||||
| import com.cloud.storage.StoragePool; | ||||
| 
 | ||||
| public interface PrimaryDataStoreDriver extends DataStoreDriver { | ||||
|     enum QualityOfServiceState { MIGRATION, NO_MIGRATION } | ||||
| 
 | ||||
|     String BASIC_CREATE = "basicCreate"; | ||||
|     String BASIC_DELETE = "basicDelete"; | ||||
|     String BASIC_DELETE_FAILURE = "basicDeleteFailure"; | ||||
|     String BASIC_DELETE_BY_FOLDER = "basicDeleteByFolder"; | ||||
|     String BASIC_GRANT_ACCESS = "basicGrantAccess"; | ||||
|     String BASIC_REVOKE_ACCESS = "basicRevokeAccess"; | ||||
|     String BASIC_IQN = "basicIqn"; | ||||
| @ -67,4 +70,6 @@ public interface PrimaryDataStoreDriver extends DataStoreDriver { | ||||
|     void takeSnapshot(SnapshotInfo snapshot, AsyncCompletionCallback<CreateCmdResult> callback); | ||||
| 
 | ||||
|     void revertSnapshot(SnapshotInfo snapshotOnImageStore, SnapshotInfo snapshotOnPrimaryStore, AsyncCompletionCallback<CommandResult> callback); | ||||
| 
 | ||||
|     void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState); | ||||
| } | ||||
|  | ||||
| @ -50,6 +50,7 @@ import com.cloud.storage.Snapshot; | ||||
| import com.cloud.storage.SnapshotVO; | ||||
| import com.cloud.storage.Storage.ImageFormat; | ||||
| import com.cloud.storage.StorageManager; | ||||
| import com.cloud.storage.StoragePool; | ||||
| import com.cloud.storage.VMTemplateVO; | ||||
| import com.cloud.storage.VolumeDetailVO; | ||||
| import com.cloud.storage.Volume; | ||||
| @ -85,8 +86,10 @@ import org.apache.cloudstack.engine.subsystem.api.storage.EndPoint; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.EndPointSelector; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.HostScope; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.ObjectInDataStoreStateMachine.Event; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.PrimaryDataStoreDriver; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.Scope; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.SnapshotInfo; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.StorageAction; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.StorageCacheManager; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.StrategyPriority; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.TemplateInfo; | ||||
| @ -295,7 +298,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             VolumeInfo srcVolumeInfo = (VolumeInfo)srcData; | ||||
|             TemplateInfo destTemplateInfo = (TemplateInfo)destData; | ||||
| 
 | ||||
|             handleCreateTemplateFromVolume(srcVolumeInfo, destTemplateInfo, callback); | ||||
|             handleCreateTemplateFromManagedVolume(srcVolumeInfo, destTemplateInfo, callback); | ||||
|         } | ||||
|         else { | ||||
|             handleError(OPERATION_NOT_SUPPORTED, callback); | ||||
| @ -309,7 +312,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|         if (canHandleSrc && (destData instanceof TemplateInfo || destData instanceof SnapshotInfo) && | ||||
|                 (destData.getDataStore().getRole() == DataStoreRole.Image || destData.getDataStore().getRole() == DataStoreRole.ImageCache)) { | ||||
|             handleCopyDataToSecondaryStorage(srcSnapshotInfo, destData, callback); | ||||
|             handleCopyAsyncToSecondaryStorage(srcSnapshotInfo, destData, callback); | ||||
|         } else if (destData instanceof VolumeInfo) { | ||||
|             handleCopyAsyncForSnapshotToVolume(srcSnapshotInfo, (VolumeInfo)destData, callback); | ||||
|         } else { | ||||
| @ -319,26 +322,28 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|     private void handleCopyAsyncForSnapshotToVolume(SnapshotInfo srcSnapshotInfo, VolumeInfo destVolumeInfo, | ||||
|                                                     AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         boolean canHandleSrc = canHandle(srcSnapshotInfo); | ||||
|         boolean canHandleDest = canHandle(destVolumeInfo); | ||||
| 
 | ||||
|         if (!canHandleDest) { | ||||
|             handleError(OPERATION_NOT_SUPPORTED, callback); | ||||
|         } | ||||
| 
 | ||||
|         boolean canHandleSrc = canHandle(srcSnapshotInfo); | ||||
| 
 | ||||
|         if (!canHandleSrc) { | ||||
|             handleCreateVolumeFromSnapshotOnSecondaryStorage(srcSnapshotInfo, destVolumeInfo, callback); | ||||
|         } | ||||
| 
 | ||||
|         if (canHandleSrc && canHandleDest) { | ||||
|             if (srcSnapshotInfo.getDataStore().getId() == destVolumeInfo.getDataStore().getId()) { | ||||
|             handleCreateVolumeFromSnapshotBothOnStorageSystem(srcSnapshotInfo, destVolumeInfo, callback); | ||||
|                 handleCreateManagedVolumeFromManagedSnapshot(srcSnapshotInfo, destVolumeInfo, callback); | ||||
|             } else { | ||||
|                 String errMsg = "To perform this operation, the source and destination primary storages must be the same."; | ||||
| 
 | ||||
|                 handleError(errMsg, callback); | ||||
|             } | ||||
|         } | ||||
|         else if (!canHandleSrc && !canHandleDest) { | ||||
|             handleError(OPERATION_NOT_SUPPORTED, callback); | ||||
|         } | ||||
|         else if (canHandleSrc) { | ||||
|             handleCreateNonManagedVolumeFromManagedSnapshot(srcSnapshotInfo, destVolumeInfo, callback); | ||||
|         } | ||||
|         else { | ||||
|             handleCreateManagedVolumeFromNonManagedSnapshot(srcSnapshotInfo, destVolumeInfo, callback); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void handleCopyAsyncForTemplateAndVolume(TemplateInfo srcTemplateInfo, VolumeInfo destVolumeInfo, AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         boolean canHandleSrc = canHandle(srcTemplateInfo); | ||||
| @ -436,7 +441,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                 hostVO = getHost(srcVolumeInfo.getDataCenterId(), hypervisorType, false); | ||||
|             } | ||||
| 
 | ||||
|             volumePath = copyVolumeToSecondaryStorage(srcVolumeInfo, destVolumeInfo, hostVO, | ||||
|             volumePath = copyManagedVolumeToSecondaryStorage(srcVolumeInfo, destVolumeInfo, hostVO, | ||||
|                     "Unable to copy the volume from managed storage to secondary storage"); | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
| @ -503,7 +508,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             setCertainVolumeValuesNull(destVolumeInfo.getId()); | ||||
| 
 | ||||
|             // migrate the volume via the hypervisor | ||||
|             String path = migrateVolume(srcVolumeInfo, destVolumeInfo, hostVO, "Unable to migrate the volume from managed storage to non-managed storage"); | ||||
|             String path = migrateVolumeForKVM(srcVolumeInfo, destVolumeInfo, hostVO, "Unable to migrate the volume from managed storage to non-managed storage"); | ||||
| 
 | ||||
|             updateVolumePath(destVolumeInfo.getId(), path); | ||||
|         } | ||||
| @ -625,48 +630,17 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         try { | ||||
|             HypervisorType hypervisorType = srcVolumeInfo.getHypervisorType(); | ||||
| 
 | ||||
|             if (!HypervisorType.KVM.equals(hypervisorType)) { | ||||
|                 throw new CloudRuntimeException("Currently, only the KVM hypervisor type is supported for the migration of a volume " + | ||||
|             if (!HypervisorType.XenServer.equals(hypervisorType) && !HypervisorType.KVM.equals(hypervisorType)) { | ||||
|                 throw new CloudRuntimeException("Currently, only the XenServer and KVM hypervisor types are supported for the migration of a volume " + | ||||
|                         "from non-managed storage to managed storage."); | ||||
|             } | ||||
| 
 | ||||
|             VirtualMachine vm = srcVolumeInfo.getAttachedVM(); | ||||
| 
 | ||||
|             if (vm != null && vm.getState() != VirtualMachine.State.Stopped) { | ||||
|                 throw new CloudRuntimeException("Currently, if a volume to migrate from non-managed storage to managed storage is attached to " + | ||||
|                         "a VM, the VM must be in the Stopped state."); | ||||
|             } | ||||
| 
 | ||||
|             destVolumeInfo.getDataStore().getDriver().createAsync(destVolumeInfo.getDataStore(), destVolumeInfo, null); | ||||
| 
 | ||||
|             VolumeVO volumeVO = _volumeDao.findById(destVolumeInfo.getId()); | ||||
| 
 | ||||
|             volumeVO.setPath(volumeVO.get_iScsiName()); | ||||
| 
 | ||||
|             _volumeDao.update(volumeVO.getId(), volumeVO); | ||||
| 
 | ||||
|             destVolumeInfo = _volumeDataFactory.getVolume(destVolumeInfo.getId(), destVolumeInfo.getDataStore()); | ||||
| 
 | ||||
|             long srcStoragePoolId = srcVolumeInfo.getPoolId(); | ||||
|             StoragePoolVO srcStoragePoolVO = _storagePoolDao.findById(srcStoragePoolId); | ||||
| 
 | ||||
|             HostVO hostVO; | ||||
| 
 | ||||
|             if (srcStoragePoolVO.getClusterId() != null) { | ||||
|                 hostVO = getHostInCluster(srcStoragePoolVO.getClusterId()); | ||||
|             if (HypervisorType.XenServer.equals(hypervisorType)) { | ||||
|                 handleVolumeMigrationForXenServer(srcVolumeInfo, destVolumeInfo); | ||||
|             } | ||||
|             else { | ||||
|                 hostVO = getHost(destVolumeInfo.getDataCenterId(), hypervisorType, false); | ||||
|                 handleVolumeMigrationForKVM(srcVolumeInfo, destVolumeInfo); | ||||
|             } | ||||
| 
 | ||||
|             // migrate the volume via the hypervisor | ||||
|             migrateVolume(srcVolumeInfo, destVolumeInfo, hostVO, "Unable to migrate the volume from non-managed storage to managed storage"); | ||||
| 
 | ||||
|             volumeVO = _volumeDao.findById(destVolumeInfo.getId()); | ||||
| 
 | ||||
|             volumeVO.setFormat(ImageFormat.QCOW2); | ||||
| 
 | ||||
|             _volumeDao.update(volumeVO.getId(), volumeVO); | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             errMsg = "Migration operation failed in 'StorageSystemDataMotionStrategy.handleVolumeMigrationFromNonManagedStorageToManagedStorage': " + | ||||
| @ -696,6 +670,155 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void handleVolumeMigrationForXenServer(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo) { | ||||
|         VirtualMachine vm = srcVolumeInfo.getAttachedVM(); | ||||
| 
 | ||||
|         if (vm == null || vm.getState() != VirtualMachine.State.Running) { | ||||
|             throw new CloudRuntimeException("Currently, a volume to migrate from non-managed storage to managed storage on XenServer must be attached to " + | ||||
|                     "a VM in the Running state."); | ||||
|         } | ||||
| 
 | ||||
|         destVolumeInfo.getDataStore().getDriver().createAsync(destVolumeInfo.getDataStore(), destVolumeInfo, null); | ||||
| 
 | ||||
|         destVolumeInfo = _volumeDataFactory.getVolume(destVolumeInfo.getId(), destVolumeInfo.getDataStore()); | ||||
| 
 | ||||
|         handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|         HostVO hostVO = _hostDao.findById(vm.getHostId()); | ||||
| 
 | ||||
|         _volumeService.grantAccess(destVolumeInfo, hostVO, destVolumeInfo.getDataStore()); | ||||
| 
 | ||||
|         String value = _configDao.getValue(Config.MigrateWait.key()); | ||||
|         int waitInterval = NumbersUtil.parseInt(value, Integer.parseInt(Config.MigrateWait.getDefaultValue())); | ||||
| 
 | ||||
|         StoragePool destPool = (StoragePool)dataStoreMgr.getDataStore(destVolumeInfo.getDataStore().getId(), DataStoreRole.Primary); | ||||
| 
 | ||||
|         MigrateVolumeCommand command = new MigrateVolumeCommand(srcVolumeInfo.getId(), srcVolumeInfo.getPath(), destPool, srcVolumeInfo.getAttachedVmName(), | ||||
|                 srcVolumeInfo.getVolumeType(), waitInterval); | ||||
| 
 | ||||
|         Map<String, String> details = new HashMap<>(); | ||||
| 
 | ||||
|         details.put(DiskTO.MANAGED, Boolean.TRUE.toString()); | ||||
|         details.put(DiskTO.IQN, destVolumeInfo.get_iScsiName()); | ||||
|         details.put(DiskTO.STORAGE_HOST, destPool.getHostAddress()); | ||||
| 
 | ||||
|         command.setDestDetails(details); | ||||
| 
 | ||||
|         EndPoint ep = selector.select(srcVolumeInfo, StorageAction.MIGRATEVOLUME); | ||||
| 
 | ||||
|         Answer answer; | ||||
| 
 | ||||
|         if (ep == null) { | ||||
|             String errMsg = "No remote endpoint to send command to; check if host or SSVM is down"; | ||||
| 
 | ||||
|             LOGGER.error(errMsg); | ||||
| 
 | ||||
|             answer = new Answer(command, false, errMsg); | ||||
|         } else { | ||||
|             answer = ep.sendMessage(command); | ||||
|         } | ||||
| 
 | ||||
|         handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
| 
 | ||||
|         if (answer == null || !answer.getResult()) { | ||||
|             handleFailedVolumeMigration(srcVolumeInfo, destVolumeInfo, hostVO); | ||||
| 
 | ||||
|             throw new CloudRuntimeException("Failed to migrate volume with ID " + srcVolumeInfo.getId() + " to storage pool with ID " + destPool.getId()); | ||||
|         } else { | ||||
|             handleSuccessfulVolumeMigration(srcVolumeInfo, destPool, (MigrateVolumeAnswer)answer); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void handleSuccessfulVolumeMigration(VolumeInfo srcVolumeInfo, StoragePool destPool, MigrateVolumeAnswer migrateVolumeAnswer) { | ||||
|         VolumeVO volumeVO = _volumeDao.findById(srcVolumeInfo.getId()); | ||||
| 
 | ||||
|         volumeVO.setPath(migrateVolumeAnswer.getVolumePath()); | ||||
| 
 | ||||
|         String chainInfo = migrateVolumeAnswer.getVolumeChainInfo(); | ||||
| 
 | ||||
|         if (chainInfo != null) { | ||||
|             volumeVO.setChainInfo(chainInfo); | ||||
|         } | ||||
| 
 | ||||
|         volumeVO.setPodId(destPool.getPodId()); | ||||
|         volumeVO.setPoolId(destPool.getId()); | ||||
|         volumeVO.setLastPoolId(srcVolumeInfo.getPoolId()); | ||||
| 
 | ||||
|         _volumeDao.update(srcVolumeInfo.getId(), volumeVO); | ||||
|     } | ||||
| 
 | ||||
|     private void handleFailedVolumeMigration(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo, HostVO hostVO) { | ||||
|         try { | ||||
|             _volumeService.revokeAccess(destVolumeInfo, hostVO, destVolumeInfo.getDataStore()); | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             LOGGER.warn("Failed to revoke access to the volume with the following ID: " + destVolumeInfo.getId()); | ||||
|         } | ||||
| 
 | ||||
|         try { | ||||
|             VolumeDetailVO volumeDetailVO = new VolumeDetailVO(destVolumeInfo.getId(), PrimaryDataStoreDriver.BASIC_DELETE_BY_FOLDER, | ||||
|                     Boolean.TRUE.toString(), false); | ||||
| 
 | ||||
|             volumeDetailsDao.persist(volumeDetailVO); | ||||
| 
 | ||||
|             destVolumeInfo.getDataStore().getDriver().deleteAsync(destVolumeInfo.getDataStore(), destVolumeInfo, null); | ||||
| 
 | ||||
|             volumeDetailsDao.removeDetails(srcVolumeInfo.getId()); | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             LOGGER.warn(ex.getMessage()); | ||||
|         } | ||||
| 
 | ||||
|         VolumeVO volumeVO = _volumeDao.findById(srcVolumeInfo.getId()); | ||||
| 
 | ||||
|         volumeVO.setPoolId(srcVolumeInfo.getPoolId()); | ||||
|         volumeVO.setLastPoolId(srcVolumeInfo.getLastPoolId()); | ||||
|         volumeVO.setFolder(srcVolumeInfo.getFolder()); | ||||
|         volumeVO.set_iScsiName(srcVolumeInfo.get_iScsiName()); | ||||
| 
 | ||||
|         _volumeDao.update(srcVolumeInfo.getId(), volumeVO); | ||||
|     } | ||||
| 
 | ||||
|     private void handleVolumeMigrationForKVM(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo) { | ||||
|         VirtualMachine vm = srcVolumeInfo.getAttachedVM(); | ||||
| 
 | ||||
|         if (vm != null && vm.getState() != VirtualMachine.State.Stopped) { | ||||
|             throw new CloudRuntimeException("Currently, if a volume to migrate from non-managed storage to managed storage on KVM is attached to " + | ||||
|                     "a VM, the VM must be in the Stopped state."); | ||||
|         } | ||||
| 
 | ||||
|         destVolumeInfo.getDataStore().getDriver().createAsync(destVolumeInfo.getDataStore(), destVolumeInfo, null); | ||||
| 
 | ||||
|         VolumeVO volumeVO = _volumeDao.findById(destVolumeInfo.getId()); | ||||
| 
 | ||||
|         volumeVO.setPath(volumeVO.get_iScsiName()); | ||||
| 
 | ||||
|         _volumeDao.update(volumeVO.getId(), volumeVO); | ||||
| 
 | ||||
|         destVolumeInfo = _volumeDataFactory.getVolume(destVolumeInfo.getId(), destVolumeInfo.getDataStore()); | ||||
| 
 | ||||
|         long srcStoragePoolId = srcVolumeInfo.getPoolId(); | ||||
|         StoragePoolVO srcStoragePoolVO = _storagePoolDao.findById(srcStoragePoolId); | ||||
| 
 | ||||
|         HostVO hostVO; | ||||
| 
 | ||||
|         if (srcStoragePoolVO.getClusterId() != null) { | ||||
|             hostVO = getHostInCluster(srcStoragePoolVO.getClusterId()); | ||||
|         } | ||||
|         else { | ||||
|             hostVO = getHost(destVolumeInfo.getDataCenterId(), HypervisorType.KVM, false); | ||||
|         } | ||||
| 
 | ||||
|         // migrate the volume via the hypervisor | ||||
|         migrateVolumeForKVM(srcVolumeInfo, destVolumeInfo, hostVO, "Unable to migrate the volume from non-managed storage to managed storage"); | ||||
| 
 | ||||
|         volumeVO = _volumeDao.findById(destVolumeInfo.getId()); | ||||
| 
 | ||||
|         volumeVO.setFormat(ImageFormat.QCOW2); | ||||
| 
 | ||||
|         _volumeDao.update(volumeVO.getId(), volumeVO); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * This function is responsible for copying a snapshot from managed storage to secondary storage. This is used in the following two cases: | ||||
|      * 1) When creating a template from a snapshot | ||||
| @ -705,7 +828,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|      * @param destData destination (can be template or snapshot) | ||||
|      * @param callback callback for async | ||||
|      */ | ||||
|     private void handleCopyDataToSecondaryStorage(SnapshotInfo snapshotInfo, DataObject destData, AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|     private void handleCopyAsyncToSecondaryStorage(SnapshotInfo snapshotInfo, DataObject destData, AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         String errMsg = null; | ||||
|         CopyCmdAnswer copyCmdAnswer = null; | ||||
|         boolean usingBackendSnapshot = false; | ||||
| @ -786,13 +909,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                         disconnectHostFromVolume(hostVO, srcDataStore.getId(), iqn); | ||||
|                     } | ||||
| 
 | ||||
|                     if (copyCmdAnswer == null || !copyCmdAnswer.getResult()) { | ||||
|                         if (copyCmdAnswer != null && !StringUtils.isEmpty(copyCmdAnswer.getDetails())) { | ||||
|                             throw new CloudRuntimeException(copyCmdAnswer.getDetails()); | ||||
|                         } else { | ||||
|                             throw new CloudRuntimeException("Unable to create volume from snapshot"); | ||||
|                         } | ||||
|                     } | ||||
|                     verifyCopyCmdAnswer(copyCmdAnswer, snapshotInfo); | ||||
| 
 | ||||
|                     vmdk = copyCmdAnswer.getNewData().getPath(); | ||||
|                     uuid = UUID.randomUUID().toString(); | ||||
| @ -829,9 +946,10 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                 copyCmdAnswer = (CopyCmdAnswer)_agentMgr.send(hostVO.getId(), copyCommand); | ||||
| 
 | ||||
|                 if (!copyCmdAnswer.getResult()) { | ||||
|                     // We were not able to copy. Handle it. | ||||
|                     errMsg = copyCmdAnswer.getDetails(); | ||||
| 
 | ||||
|                     LOGGER.warn(errMsg); | ||||
| 
 | ||||
|                     throw new CloudRuntimeException(errMsg); | ||||
|                 } | ||||
| 
 | ||||
| @ -925,13 +1043,154 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void handleCreateNonManagedVolumeFromManagedSnapshot(SnapshotInfo snapshotInfo, VolumeInfo volumeInfo, | ||||
|                                                                  AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         if (!HypervisorType.XenServer.equals(snapshotInfo.getHypervisorType())) { | ||||
|             String errMsg = "Creating a volume on non-managed storage from a snapshot on managed storage is currently only supported with XenServer."; | ||||
| 
 | ||||
|             handleError(errMsg, callback); | ||||
|         } | ||||
| 
 | ||||
|         long volumeStoragePoolId = volumeInfo.getDataStore().getId(); | ||||
|         StoragePoolVO volumeStoragePoolVO = _storagePoolDao.findById(volumeStoragePoolId); | ||||
| 
 | ||||
|         if (volumeStoragePoolVO.getClusterId() == null) { | ||||
|             String errMsg = "To create a non-managed volume from a managed snapshot, the destination storage pool must be cluster scoped."; | ||||
| 
 | ||||
|             handleError(errMsg, callback); | ||||
|         } | ||||
| 
 | ||||
|         String errMsg = null; | ||||
|         CopyCmdAnswer copyCmdAnswer = null; | ||||
| 
 | ||||
|         boolean usingBackendSnapshot = false; | ||||
| 
 | ||||
|         try { | ||||
|             snapshotInfo.processEvent(Event.CopyingRequested); | ||||
| 
 | ||||
|             usingBackendSnapshot = usingBackendSnapshotFor(snapshotInfo); | ||||
| 
 | ||||
|             if (usingBackendSnapshot) { | ||||
|                 boolean computeClusterSupportsVolumeClone = clusterDao.getSupportsResigning(volumeStoragePoolVO.getClusterId()); | ||||
| 
 | ||||
|                 if (!computeClusterSupportsVolumeClone) { | ||||
|                     String noSupportForResignErrMsg = "Unable to locate an applicable host with which to perform a resignature operation : Cluster ID = " + | ||||
|                             volumeStoragePoolVO.getClusterId(); | ||||
| 
 | ||||
|                     LOGGER.warn(noSupportForResignErrMsg); | ||||
| 
 | ||||
|                     throw new CloudRuntimeException(noSupportForResignErrMsg); | ||||
|                 } | ||||
| 
 | ||||
|                 createVolumeFromSnapshot(snapshotInfo); | ||||
| 
 | ||||
|                 HostVO hostVO = getHost(snapshotInfo.getDataCenterId(), HypervisorType.XenServer, true); | ||||
| 
 | ||||
|                 copyCmdAnswer = performResignature(snapshotInfo, hostVO, null, true); | ||||
| 
 | ||||
|                 verifyCopyCmdAnswer(copyCmdAnswer, snapshotInfo); | ||||
|             } | ||||
| 
 | ||||
|             String value = _configDao.getValue(Config.PrimaryStorageDownloadWait.toString()); | ||||
|             int primaryStorageDownloadWait = NumbersUtil.parseInt(value, Integer.parseInt(Config.PrimaryStorageDownloadWait.getDefaultValue())); | ||||
| 
 | ||||
|             CopyCommand copyCommand = new CopyCommand(snapshotInfo.getTO(), volumeInfo.getTO(), primaryStorageDownloadWait, | ||||
|                     VirtualMachineManager.ExecuteInSequence.value()); | ||||
| 
 | ||||
|             HostVO hostVO = getHostInCluster(volumeStoragePoolVO.getClusterId()); | ||||
| 
 | ||||
|             if (!usingBackendSnapshot) { | ||||
|                 long snapshotStoragePoolId = snapshotInfo.getDataStore().getId(); | ||||
|                 DataStore snapshotDataStore = dataStoreMgr.getDataStore(snapshotStoragePoolId, DataStoreRole.Primary); | ||||
| 
 | ||||
|                 _volumeService.grantAccess(snapshotInfo, hostVO, snapshotDataStore); | ||||
|             } | ||||
| 
 | ||||
|             Map<String, String> srcDetails = getSnapshotDetails(snapshotInfo); | ||||
| 
 | ||||
|             copyCommand.setOptions(srcDetails); | ||||
| 
 | ||||
|             copyCmdAnswer = (CopyCmdAnswer)_agentMgr.send(hostVO.getId(), copyCommand); | ||||
| 
 | ||||
|             if (!copyCmdAnswer.getResult()) { | ||||
|                 errMsg = copyCmdAnswer.getDetails(); | ||||
| 
 | ||||
|                 LOGGER.warn(errMsg); | ||||
| 
 | ||||
|                 throw new CloudRuntimeException(errMsg); | ||||
|             } | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             errMsg = "Copy operation failed in 'StorageSystemDataMotionStrategy.handleCreateNonManagedVolumeFromManagedSnapshot': " + ex.getMessage(); | ||||
| 
 | ||||
|             throw new CloudRuntimeException(errMsg); | ||||
|         } | ||||
|         finally { | ||||
|             try { | ||||
|                 HostVO hostVO = getHostInCluster(volumeStoragePoolVO.getClusterId()); | ||||
| 
 | ||||
|                 long snapshotStoragePoolId = snapshotInfo.getDataStore().getId(); | ||||
|                 DataStore snapshotDataStore = dataStoreMgr.getDataStore(snapshotStoragePoolId, DataStoreRole.Primary); | ||||
| 
 | ||||
|                 _volumeService.revokeAccess(snapshotInfo, hostVO, snapshotDataStore); | ||||
|             } | ||||
|             catch (Exception e) { | ||||
|                 LOGGER.debug("Failed to revoke access from dest volume", e); | ||||
|             } | ||||
| 
 | ||||
|             if (usingBackendSnapshot) { | ||||
|                 deleteVolumeFromSnapshot(snapshotInfo); | ||||
|             } | ||||
| 
 | ||||
|             try { | ||||
|                 if (StringUtils.isEmpty(errMsg)) { | ||||
|                     snapshotInfo.processEvent(Event.OperationSuccessed); | ||||
|                 } | ||||
|                 else { | ||||
|                     snapshotInfo.processEvent(Event.OperationFailed); | ||||
|                 } | ||||
|             } | ||||
|             catch (Exception ex) { | ||||
|                 LOGGER.warn("Error processing snapshot event: " + ex.getMessage(), ex); | ||||
|             } | ||||
| 
 | ||||
|             if (copyCmdAnswer == null) { | ||||
|                 copyCmdAnswer = new CopyCmdAnswer(errMsg); | ||||
|             } | ||||
| 
 | ||||
|             CopyCommandResult result = new CopyCommandResult(null, copyCmdAnswer); | ||||
| 
 | ||||
|             result.setResult(errMsg); | ||||
| 
 | ||||
|             callback.complete(result); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void verifyCopyCmdAnswer(CopyCmdAnswer copyCmdAnswer, DataObject dataObject) { | ||||
|         if (copyCmdAnswer == null) { | ||||
|             throw new CloudRuntimeException("Unable to create a volume from a " + dataObject.getType().toString().toLowerCase() + " (copyCmdAnswer == null)"); | ||||
|         } | ||||
| 
 | ||||
|         if (copyCmdAnswer.getResult()) { | ||||
|             return; | ||||
|         } | ||||
| 
 | ||||
|         String details = copyCmdAnswer.getDetails(); | ||||
| 
 | ||||
|         if (StringUtils.isEmpty(details)) { | ||||
|             throw new CloudRuntimeException("Unable to create a volume from a " + dataObject.getType().toString().toLowerCase() + " (no error details specified)"); | ||||
|         } | ||||
| 
 | ||||
|         throw new CloudRuntimeException(details); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Creates a volume on the storage from a snapshot that resides on the secondary storage (archived snapshot). | ||||
|      * Creates a managed volume on the storage from a snapshot that resides on the secondary storage (archived snapshot). | ||||
|      * @param snapshotInfo snapshot on secondary | ||||
|      * @param volumeInfo volume to be created on the storage | ||||
|      * @param callback for async | ||||
|      */ | ||||
|     private void handleCreateVolumeFromSnapshotOnSecondaryStorage(SnapshotInfo snapshotInfo, VolumeInfo volumeInfo, | ||||
|     private void handleCreateManagedVolumeFromNonManagedSnapshot(SnapshotInfo snapshotInfo, VolumeInfo volumeInfo, | ||||
|                                                                  AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         String errMsg = null; | ||||
|         CopyCmdAnswer copyCmdAnswer = null; | ||||
| @ -960,6 +1219,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             volumeInfo.processEvent(Event.MigrationRequested); | ||||
|             volumeInfo = _volumeDataFactory.getVolume(volumeInfo.getId(), volumeInfo.getDataStore()); | ||||
| 
 | ||||
|             handleQualityOfServiceForVolumeMigration(volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|             hostVO = getHost(snapshotInfo.getDataCenterId(), snapshotInfo.getHypervisorType(), false); | ||||
| 
 | ||||
|             // copy the volume from secondary via the hypervisor | ||||
| @ -980,12 +1241,13 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             } | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             errMsg = "Copy operation failed in 'StorageSystemDataMotionStrategy.handleCreateVolumeFromSnapshotOnSecondaryStorage': " + | ||||
|                     ex.getMessage(); | ||||
|             errMsg = "Copy operation failed in 'StorageSystemDataMotionStrategy.handleCreateManagedVolumeFromNonManagedSnapshot': " + ex.getMessage(); | ||||
| 
 | ||||
|             throw new CloudRuntimeException(errMsg); | ||||
|         } | ||||
|         finally { | ||||
|             handleQualityOfServiceForVolumeMigration(volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
| 
 | ||||
|             if (copyCmdAnswer == null) { | ||||
|                 copyCmdAnswer = new CopyCmdAnswer(errMsg); | ||||
|             } | ||||
| @ -1093,13 +1355,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|                 copyCmdAnswer = performResignature(volumeInfo, hostVO, extraDetails); | ||||
| 
 | ||||
|                 if (copyCmdAnswer == null || !copyCmdAnswer.getResult()) { | ||||
|                     if (copyCmdAnswer != null && !StringUtils.isEmpty(copyCmdAnswer.getDetails())) { | ||||
|                         throw new CloudRuntimeException(copyCmdAnswer.getDetails()); | ||||
|                     } else { | ||||
|                         throw new CloudRuntimeException("Unable to create a volume from a template"); | ||||
|                     } | ||||
|                 } | ||||
|                 verifyCopyCmdAnswer(copyCmdAnswer, templateInfo); | ||||
| 
 | ||||
|                 // If using VMware, have the host rescan its software HBA if dynamic discovery is in use. | ||||
|                 if (HypervisorType.VMware.equals(templateInfo.getHypervisorType())) { | ||||
| @ -1145,11 +1401,13 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void handleCreateVolumeFromSnapshotBothOnStorageSystem(SnapshotInfo snapshotInfo, VolumeInfo volumeInfo, | ||||
|     private void handleCreateManagedVolumeFromManagedSnapshot(SnapshotInfo snapshotInfo, VolumeInfo volumeInfo, | ||||
|                                                               AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         String errMsg = null; | ||||
|         CopyCmdAnswer copyCmdAnswer = null; | ||||
| 
 | ||||
|         boolean useCloning = true; | ||||
| 
 | ||||
|         try { | ||||
|             verifyFormat(snapshotInfo); | ||||
| 
 | ||||
| @ -1171,8 +1429,9 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                 } | ||||
|             } | ||||
| 
 | ||||
|             boolean canStorageSystemCreateVolumeFromVolume = canStorageSystemCreateVolumeFromVolume(snapshotInfo); | ||||
|             boolean useCloning = usingBackendSnapshot || (canStorageSystemCreateVolumeFromVolume && computeClusterSupportsVolumeClone); | ||||
|             boolean canStorageSystemCreateVolumeFromVolume = canStorageSystemCreateVolumeFromVolume(snapshotInfo.getDataStore().getId()); | ||||
| 
 | ||||
|             useCloning = usingBackendSnapshot || (canStorageSystemCreateVolumeFromVolume && computeClusterSupportsVolumeClone); | ||||
| 
 | ||||
|             VolumeDetailVO volumeDetail = null; | ||||
| 
 | ||||
| @ -1232,16 +1491,12 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                     // even when we don't need those hosts to do this kind of copy work | ||||
|                     hostVO = getHost(snapshotInfo.getDataCenterId(), snapshotInfo.getHypervisorType(), false); | ||||
| 
 | ||||
|                     handleQualityOfServiceForVolumeMigration(volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|                     copyCmdAnswer = performCopyOfVdi(volumeInfo, snapshotInfo, hostVO); | ||||
|                 } | ||||
| 
 | ||||
|                 if (copyCmdAnswer == null || !copyCmdAnswer.getResult()) { | ||||
|                     if (copyCmdAnswer != null && !StringUtils.isEmpty(copyCmdAnswer.getDetails())) { | ||||
|                         throw new CloudRuntimeException(copyCmdAnswer.getDetails()); | ||||
|                     } else { | ||||
|                         throw new CloudRuntimeException("Unable to create volume from snapshot"); | ||||
|                     } | ||||
|                 } | ||||
|                 verifyCopyCmdAnswer(copyCmdAnswer, snapshotInfo); | ||||
|             } | ||||
|             else if (HypervisorType.KVM.equals(snapshotInfo.getHypervisorType())) { | ||||
|                 VolumeObjectTO newVolume = new VolumeObjectTO(); | ||||
| @ -1257,12 +1512,16 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             } | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             errMsg = "Copy operation failed in 'StorageSystemDataMotionStrategy.handleCreateVolumeFromSnapshotBothOnStorageSystem': " + | ||||
|             errMsg = "Copy operation failed in 'StorageSystemDataMotionStrategy.handleCreateManagedVolumeFromManagedSnapshot': " + | ||||
|                     ex.getMessage(); | ||||
| 
 | ||||
|             throw new CloudRuntimeException(errMsg); | ||||
|         } | ||||
|         finally { | ||||
|             if (useCloning) { | ||||
|                 handleQualityOfServiceForVolumeMigration(volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
|             } | ||||
| 
 | ||||
|             if (copyCmdAnswer == null) { | ||||
|                 copyCmdAnswer = new CopyCmdAnswer(errMsg); | ||||
|             } | ||||
| @ -1289,6 +1548,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|             HostVO hostVO = getHost(dataCenterId, hypervisorType, false); | ||||
| 
 | ||||
|             handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|             // copy the volume from secondary via the hypervisor | ||||
|             copyCmdAnswer = copyImageToVolume(srcVolumeInfo, destVolumeInfo, hostVO); | ||||
| 
 | ||||
| @ -1308,6 +1569,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             throw new CloudRuntimeException(errMsg); | ||||
|         } | ||||
|         finally { | ||||
|             handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
| 
 | ||||
|             if (copyCmdAnswer == null) { | ||||
|                 copyCmdAnswer = new CopyCmdAnswer(errMsg); | ||||
|             } | ||||
| @ -1393,6 +1656,15 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState qualityOfServiceState) { | ||||
|         try { | ||||
|             ((PrimaryDataStoreDriver)volumeInfo.getDataStore().getDriver()).handleQualityOfServiceForVolumeMigration(volumeInfo, qualityOfServiceState); | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             LOGGER.warn(ex); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private SnapshotDetailsVO handleSnapshotDetails(long csSnapshotId, String value) { | ||||
|         String name = "tempVolume"; | ||||
| 
 | ||||
| @ -1452,6 +1724,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|                 destVolumeInfo = _volumeDataFactory.getVolume(destVolume.getId(), destDataStore); | ||||
| 
 | ||||
|                 handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|                 _volumeService.grantAccess(destVolumeInfo, destHost, destDataStore); | ||||
| 
 | ||||
|                 String connectedPath = connectHostToVolume(destHost, destVolumeInfo.getPoolId(), destVolumeInfo.get_iScsiName()); | ||||
| @ -1554,6 +1828,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             VolumeInfo srcVolumeInfo = entry.getKey(); | ||||
|             VolumeInfo destVolumeInfo = entry.getValue(); | ||||
| 
 | ||||
|             handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
| 
 | ||||
|             if (success) { | ||||
|                 srcVolumeInfo.processEvent(Event.OperationSuccessed); | ||||
|                 destVolumeInfo.processEvent(Event.OperationSuccessed); | ||||
| @ -1721,20 +1997,28 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private boolean canStorageSystemCreateVolumeFromVolume(SnapshotInfo snapshotInfo) { | ||||
|         boolean supportsCloningVolumeFromVolume = false; | ||||
|     private boolean canStorageSystemCreateVolumeFromVolume(long storagePoolId) { | ||||
|         return storageSystemSupportsCapability(storagePoolId, DataStoreCapabilities.CAN_CREATE_VOLUME_FROM_VOLUME.toString()); | ||||
|     } | ||||
| 
 | ||||
|         DataStore dataStore = dataStoreMgr.getDataStore(snapshotInfo.getDataStore().getId(), DataStoreRole.Primary); | ||||
|     private boolean canStorageSystemCreateVolumeFromSnapshot(long storagePoolId) { | ||||
|         return storageSystemSupportsCapability(storagePoolId, DataStoreCapabilities.CAN_CREATE_VOLUME_FROM_SNAPSHOT.toString()); | ||||
|     } | ||||
| 
 | ||||
|     private boolean storageSystemSupportsCapability(long storagePoolId, String capability) { | ||||
|         boolean supportsCapability = false; | ||||
| 
 | ||||
|         DataStore dataStore = dataStoreMgr.getDataStore(storagePoolId, DataStoreRole.Primary); | ||||
| 
 | ||||
|         Map<String, String> mapCapabilities = dataStore.getDriver().getCapabilities(); | ||||
| 
 | ||||
|         if (mapCapabilities != null) { | ||||
|             String value = mapCapabilities.get(DataStoreCapabilities.CAN_CREATE_VOLUME_FROM_VOLUME.toString()); | ||||
|             String value = mapCapabilities.get(capability); | ||||
| 
 | ||||
|             supportsCloningVolumeFromVolume = Boolean.valueOf(value); | ||||
|             supportsCapability = Boolean.valueOf(value); | ||||
|         } | ||||
| 
 | ||||
|         return supportsCloningVolumeFromVolume; | ||||
|         return supportsCapability; | ||||
|     } | ||||
| 
 | ||||
|     private String getVolumeProperty(long volumeId, String property) { | ||||
| @ -1757,7 +2041,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         return null; | ||||
|     } | ||||
| 
 | ||||
|     private void handleCreateTemplateFromVolume(VolumeInfo volumeInfo, TemplateInfo templateInfo, AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|     private void handleCreateTemplateFromManagedVolume(VolumeInfo volumeInfo, TemplateInfo templateInfo, AsyncCompletionCallback<CopyCommandResult> callback) { | ||||
|         boolean srcVolumeDetached = volumeInfo.getAttachedVM() == null; | ||||
| 
 | ||||
|         String errMsg = null; | ||||
| @ -1775,9 +2059,12 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|             String value = _configDao.getValue(Config.PrimaryStorageDownloadWait.toString()); | ||||
|             int primaryStorageDownloadWait = NumberUtils.toInt(value, Integer.parseInt(Config.PrimaryStorageDownloadWait.getDefaultValue())); | ||||
| 
 | ||||
|             CopyCommand copyCommand = new CopyCommand(volumeInfo.getTO(), templateInfo.getTO(), primaryStorageDownloadWait, VirtualMachineManager.ExecuteInSequence.value()); | ||||
| 
 | ||||
|             try { | ||||
|                 handleQualityOfServiceForVolumeMigration(volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|                 if (srcVolumeDetached) { | ||||
|                     _volumeService.grantAccess(volumeInfo, hostVO, srcDataStore); | ||||
|                 } | ||||
| @ -1789,8 +2076,10 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                 copyCmdAnswer = (CopyCmdAnswer)_agentMgr.send(hostVO.getId(), copyCommand); | ||||
| 
 | ||||
|                 if (!copyCmdAnswer.getResult()) { | ||||
|                     // We were not able to copy. Handle it. | ||||
|                     errMsg = copyCmdAnswer.getDetails(); | ||||
| 
 | ||||
|                     LOGGER.warn(errMsg); | ||||
| 
 | ||||
|                     throw new CloudRuntimeException(errMsg); | ||||
|                 } | ||||
| 
 | ||||
| @ -1808,14 +2097,17 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                 throw new CloudRuntimeException(msg + ex.getMessage(), ex); | ||||
|             } | ||||
|             finally { | ||||
|                 try { | ||||
|                 if (srcVolumeDetached) { | ||||
|                     try { | ||||
|                         _volumeService.revokeAccess(volumeInfo, hostVO, srcDataStore); | ||||
|                     } | ||||
|                 } | ||||
|                     catch (Exception ex) { | ||||
|                         LOGGER.warn("Error revoking access to volume (Volume ID = " + volumeInfo.getId() + "): " + ex.getMessage(), ex); | ||||
|                     } | ||||
|                 } | ||||
| 
 | ||||
|                 handleQualityOfServiceForVolumeMigration(volumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
| 
 | ||||
|                 if (copyCmdAnswer == null || !copyCmdAnswer.getResult()) { | ||||
|                     if (copyCmdAnswer != null && !StringUtils.isEmpty(copyCmdAnswer.getDetails())) { | ||||
|                         errMsg = copyCmdAnswer.getDetails(); | ||||
| @ -2104,7 +2396,7 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|         return leafData; | ||||
|     } | ||||
| 
 | ||||
|     private String migrateVolume(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo, HostVO hostVO, String errMsg) { | ||||
|     private String migrateVolumeForKVM(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo, HostVO hostVO, String errMsg) { | ||||
|         boolean srcVolumeDetached = srcVolumeInfo.getAttachedVM() == null; | ||||
| 
 | ||||
|         try { | ||||
| @ -2118,6 +2410,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|                 _volumeService.grantAccess(srcVolumeInfo, hostVO, srcVolumeInfo.getDataStore()); | ||||
|             } | ||||
| 
 | ||||
|             handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|             _volumeService.grantAccess(destVolumeInfo, hostVO, destVolumeInfo.getDataStore()); | ||||
| 
 | ||||
|             MigrateVolumeAnswer migrateVolumeAnswer = (MigrateVolumeAnswer)_agentMgr.send(hostVO.getId(), migrateVolumeCommand); | ||||
| @ -2164,9 +2458,12 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
| 
 | ||||
|             throw new CloudRuntimeException(msg + ex.getMessage(), ex); | ||||
|         } | ||||
|         finally { | ||||
|             handleQualityOfServiceForVolumeMigration(destVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private String copyVolumeToSecondaryStorage(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo, HostVO hostVO, String errMsg) { | ||||
|     private String copyManagedVolumeToSecondaryStorage(VolumeInfo srcVolumeInfo, VolumeInfo destVolumeInfo, HostVO hostVO, String errMsg) { | ||||
|         boolean srcVolumeDetached = srcVolumeInfo.getAttachedVM() == null; | ||||
| 
 | ||||
|         try { | ||||
| @ -2179,6 +2476,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             copyVolumeCommand.setSrcData(srcVolumeInfo.getTO()); | ||||
|             copyVolumeCommand.setSrcDetails(srcDetails); | ||||
| 
 | ||||
|             handleQualityOfServiceForVolumeMigration(srcVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.MIGRATION); | ||||
| 
 | ||||
|             if (srcVolumeDetached) { | ||||
|                 _volumeService.grantAccess(srcVolumeInfo, hostVO, srcVolumeInfo.getDataStore()); | ||||
|             } | ||||
| @ -2207,6 +2506,8 @@ public class StorageSystemDataMotionStrategy implements DataMotionStrategy { | ||||
|             if (srcVolumeDetached) { | ||||
|                 _volumeService.revokeAccess(srcVolumeInfo, hostVO, srcVolumeInfo.getDataStore()); | ||||
|             } | ||||
| 
 | ||||
|             handleQualityOfServiceForVolumeMigration(srcVolumeInfo, PrimaryDataStoreDriver.QualityOfServiceState.NO_MIGRATION); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | ||||
| @ -1517,67 +1517,114 @@ public class XenServerStorageProcessor implements StorageProcessor { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private boolean isManaged(Map<String, String> options) { | ||||
|         if (options == null) { | ||||
|             return false; | ||||
|         } | ||||
| 
 | ||||
|         String iqn = options.get(DiskTO.IQN); | ||||
| 
 | ||||
|         if (iqn == null || iqn.trim().length() == 0) { | ||||
|             return false; | ||||
|         } | ||||
| 
 | ||||
|         String storageHost = options.get(DiskTO.STORAGE_HOST); | ||||
| 
 | ||||
|         if (storageHost == null || storageHost.trim().length() == 0) { | ||||
|             return false; | ||||
|         } | ||||
| 
 | ||||
|         return true; | ||||
|     } | ||||
| 
 | ||||
|     boolean isCreateManagedVolumeFromManagedSnapshot(Map<String, String> volumeOptions, Map<String, String> snapshotOptions) { | ||||
|         return isManaged(volumeOptions) && isManaged(snapshotOptions); | ||||
|     } | ||||
| 
 | ||||
|     boolean isCreateNonManagedVolumeFromManagedSnapshot(Map<String, String> volumeOptions, Map<String, String> snapshotOptions) { | ||||
|         return !isManaged(volumeOptions) && isManaged(snapshotOptions); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public Answer createVolumeFromSnapshot(final CopyCommand cmd) { | ||||
|         final Connection conn = hypervisorResource.getConnection(); | ||||
|         final DataTO srcData = cmd.getSrcTO(); | ||||
|         final SnapshotObjectTO snapshot = (SnapshotObjectTO) srcData; | ||||
|         final DataTO destData = cmd.getDestTO(); | ||||
|         final DataStoreTO imageStore = srcData.getDataStore(); | ||||
|         Connection conn = hypervisorResource.getConnection(); | ||||
| 
 | ||||
|         if (srcData.getDataStore() instanceof PrimaryDataStoreTO && destData.getDataStore() instanceof PrimaryDataStoreTO) { | ||||
|             return createVolumeFromSnapshot2(cmd); | ||||
|         DataTO srcData = cmd.getSrcTO(); | ||||
|         SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData; | ||||
|         DataStoreTO imageStore = srcData.getDataStore(); | ||||
|         DataTO destData = cmd.getDestTO(); | ||||
| 
 | ||||
|         if (isCreateManagedVolumeFromManagedSnapshot(cmd.getOptions2(), cmd.getOptions())) { | ||||
|             return createManagedVolumeFromManagedSnapshot(cmd); | ||||
|         } | ||||
| 
 | ||||
|         if (isCreateNonManagedVolumeFromManagedSnapshot(cmd.getOptions2(), cmd.getOptions())) { | ||||
|             return createNonManagedVolumeFromManagedSnapshot(cmd); | ||||
|         } | ||||
| 
 | ||||
|         if (!(imageStore instanceof NfsTO)) { | ||||
|             return new CopyCmdAnswer("unsupported protocol"); | ||||
|         } | ||||
| 
 | ||||
|         final NfsTO nfsImageStore = (NfsTO) imageStore; | ||||
|         final String primaryStorageNameLabel = destData.getDataStore().getUuid(); | ||||
|         final String secondaryStorageUrl = nfsImageStore.getUrl(); | ||||
|         final int wait = cmd.getWait(); | ||||
|         NfsTO nfsImageStore = (NfsTO)imageStore; | ||||
|         String primaryStorageNameLabel = destData.getDataStore().getUuid(); | ||||
|         String secondaryStorageUrl = nfsImageStore.getUrl(); | ||||
| 
 | ||||
|         int wait = cmd.getWait(); | ||||
|         boolean result = false; | ||||
| 
 | ||||
|         // Generic error message. | ||||
|         String details = null; | ||||
|         String volumeUUID = null; | ||||
|         String details; | ||||
|         String volumeUUID; | ||||
| 
 | ||||
|         if (secondaryStorageUrl == null) { | ||||
|             details += " because the URL passed: " + secondaryStorageUrl + " is invalid."; | ||||
|             details = "The URL passed in 'null'."; | ||||
| 
 | ||||
|             return new CopyCmdAnswer(details); | ||||
|         } | ||||
| 
 | ||||
|         try { | ||||
|             final SR primaryStorageSR = hypervisorResource.getSRByNameLabelandHost(conn, primaryStorageNameLabel); | ||||
|             SR primaryStorageSR = 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); | ||||
|                 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(); | ||||
|             final int index = snapshotInstallPath.lastIndexOf(nfsImageStore.getPathSeparator()); | ||||
|             final String snapshotName = snapshotInstallPath.substring(index + 1); | ||||
|             int index = snapshotInstallPath.lastIndexOf(nfsImageStore.getPathSeparator()); | ||||
|             String snapshotName = snapshotInstallPath.substring(index + 1); | ||||
| 
 | ||||
|             if (!snapshotName.startsWith("VHD-") && !snapshotName.endsWith(".vhd")) { | ||||
|                 snapshotInstallPath = snapshotInstallPath + ".vhd"; | ||||
|             } | ||||
|             final URI snapshotURI = new URI(secondaryStorageUrl + nfsImageStore.getPathSeparator() + snapshotInstallPath); | ||||
|             final String snapshotPath = snapshotURI.getHost() + ":" + snapshotURI.getPath(); | ||||
|             final String srUuid = primaryStorageSR.getUuid(conn); | ||||
| 
 | ||||
|             URI snapshotURI = new URI(secondaryStorageUrl + nfsImageStore.getPathSeparator() + snapshotInstallPath); | ||||
|             String snapshotPath = snapshotURI.getHost() + ":" + snapshotURI.getPath(); | ||||
|             String srUuid = primaryStorageSR.getUuid(conn); | ||||
| 
 | ||||
|             volumeUUID = copy_vhd_from_secondarystorage(conn, snapshotPath, srUuid, wait); | ||||
|             result = true; | ||||
|             final VDI volume = VDI.getByUuid(conn, volumeUUID); | ||||
|             final VDI.Record vdir = volume.getRecord(conn); | ||||
|             final VolumeObjectTO newVol = new VolumeObjectTO(); | ||||
| 
 | ||||
|             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 (final XenAPIException e) { | ||||
|             details += " due to " + e.toString(); | ||||
|             details = "Exception due to " + e.toString(); | ||||
| 
 | ||||
|             s_logger.warn(details, e); | ||||
|         } catch (final Exception e) { | ||||
|             details += " due to " + e.getMessage(); | ||||
|             details = "Exception due to " + e.getMessage(); | ||||
| 
 | ||||
|             s_logger.warn(details, e); | ||||
|         } | ||||
| 
 | ||||
|         if (!result) { | ||||
|             // Is this logged at a higher level? | ||||
|             s_logger.error(details); | ||||
| @ -1587,7 +1634,7 @@ public class XenServerStorageProcessor implements StorageProcessor { | ||||
|         return new CopyCmdAnswer(details); | ||||
|     } | ||||
| 
 | ||||
|     protected Answer createVolumeFromSnapshot2(final CopyCommand cmd) { | ||||
|     Answer createManagedVolumeFromManagedSnapshot(final CopyCommand cmd) { | ||||
|         try { | ||||
|             final Connection conn = hypervisorResource.getConnection(); | ||||
| 
 | ||||
| @ -1632,6 +1679,51 @@ public class XenServerStorageProcessor implements StorageProcessor { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     Answer createNonManagedVolumeFromManagedSnapshot(final CopyCommand cmd) { | ||||
|         Connection conn = hypervisorResource.getConnection(); | ||||
|         SR srcSr = null; | ||||
| 
 | ||||
|         try { | ||||
|             Map<String, String> srcOptions = cmd.getOptions(); | ||||
| 
 | ||||
|             String src_iScsiName = srcOptions.get(DiskTO.IQN); | ||||
|             String srcStorageHost = srcOptions.get(DiskTO.STORAGE_HOST); | ||||
|             String srcChapInitiatorUsername = srcOptions.get(DiskTO.CHAP_INITIATOR_USERNAME); | ||||
|             String srcChapInitiatorSecret = srcOptions.get(DiskTO.CHAP_INITIATOR_SECRET); | ||||
| 
 | ||||
|             srcSr = hypervisorResource.getIscsiSR(conn, src_iScsiName, srcStorageHost, src_iScsiName, | ||||
|                     srcChapInitiatorUsername, srcChapInitiatorSecret, false); | ||||
| 
 | ||||
|             // there should only be one VDI in this SR | ||||
|             VDI srcVdi = srcSr.getVDIs(conn).iterator().next(); | ||||
| 
 | ||||
|             DataTO destData = cmd.getDestTO(); | ||||
|             String primaryStorageNameLabel = destData.getDataStore().getUuid(); | ||||
| 
 | ||||
|             SR destSr = hypervisorResource.getSRByNameLabelandHost(conn, primaryStorageNameLabel); | ||||
| 
 | ||||
|             VDI vdiCopy = srcVdi.copy(conn, destSr); | ||||
| 
 | ||||
|             VolumeObjectTO newVol = new VolumeObjectTO(); | ||||
| 
 | ||||
|             newVol.setSize(vdiCopy.getVirtualSize(conn)); | ||||
|             newVol.setPath(vdiCopy.getUuid(conn)); | ||||
|             newVol.setFormat(ImageFormat.VHD); | ||||
| 
 | ||||
|             return new CopyCmdAnswer(newVol); | ||||
|         } | ||||
|         catch (Exception ex) { | ||||
|             s_logger.warn("Failed to copy snapshot to volume: " + ex.toString(), ex); | ||||
| 
 | ||||
|             return new CopyCmdAnswer(ex.getMessage()); | ||||
|         } | ||||
|         finally { | ||||
|             if (srcSr != null) { | ||||
|                 hypervisorResource.removeSR(conn, srcSr); | ||||
|             } | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public Answer deleteSnapshot(final DeleteCommand cmd) { | ||||
|         final SnapshotObjectTO snapshot = (SnapshotObjectTO) cmd.getData(); | ||||
|  | ||||
| @ -787,8 +787,12 @@ public class Xenserver625StorageProcessor extends XenServerStorageProcessor { | ||||
|         final VolumeObjectTO volume = (VolumeObjectTO)destData; | ||||
|         final DataStoreTO imageStore = srcData.getDataStore(); | ||||
| 
 | ||||
|         if (srcData.getDataStore() instanceof PrimaryDataStoreTO && destData.getDataStore() instanceof PrimaryDataStoreTO) { | ||||
|             return createVolumeFromSnapshot2(cmd); | ||||
|         if (isCreateManagedVolumeFromManagedSnapshot(cmd.getOptions2(), cmd.getOptions())) { | ||||
|             return createManagedVolumeFromManagedSnapshot(cmd); | ||||
|         } | ||||
| 
 | ||||
|         if (isCreateNonManagedVolumeFromManagedSnapshot(cmd.getOptions2(), cmd.getOptions())) { | ||||
|             return createNonManagedVolumeFromManagedSnapshot(cmd); | ||||
|         } | ||||
| 
 | ||||
|         if (!(imageStore instanceof NfsTO)) { | ||||
|  | ||||
| @ -27,6 +27,7 @@ import org.apache.log4j.Logger; | ||||
| import com.cloud.agent.api.Answer; | ||||
| import com.cloud.agent.api.storage.MigrateVolumeAnswer; | ||||
| import com.cloud.agent.api.storage.MigrateVolumeCommand; | ||||
| import com.cloud.agent.api.to.DiskTO; | ||||
| import com.cloud.agent.api.to.StorageFilerTO; | ||||
| import com.cloud.hypervisor.xenserver.resource.XenServer610Resource; | ||||
| import com.cloud.resource.CommandWrapper; | ||||
| @ -39,34 +40,58 @@ import com.xensource.xenapi.VDI; | ||||
| 
 | ||||
| @ResourceWrapper(handles =  MigrateVolumeCommand.class) | ||||
| public final class XenServer610MigrateVolumeCommandWrapper extends CommandWrapper<MigrateVolumeCommand, Answer, XenServer610Resource> { | ||||
| 
 | ||||
|     private static final Logger s_logger = Logger.getLogger(XenServer610MigrateVolumeCommandWrapper.class); | ||||
|     private static final Logger LOGGER = Logger.getLogger(XenServer610MigrateVolumeCommandWrapper.class); | ||||
| 
 | ||||
|     @Override | ||||
|     public Answer execute(final MigrateVolumeCommand command, final XenServer610Resource xenServer610Resource) { | ||||
|         final Connection connection = xenServer610Resource.getConnection(); | ||||
|         final String volumeUUID = command.getVolumePath(); | ||||
|         final StorageFilerTO poolTO = command.getPool(); | ||||
|         Connection connection = xenServer610Resource.getConnection(); | ||||
|         String srcVolumeUuid = command.getVolumePath(); | ||||
|         SR destPool = null; | ||||
|         Map<String, String> destDetails = command.getDestDetails(); | ||||
| 
 | ||||
|         try { | ||||
|             final String uuid = poolTO.getUuid(); | ||||
|             final SR destinationPool = xenServer610Resource.getStorageRepository(connection, uuid); | ||||
|             final VDI srcVolume = xenServer610Resource.getVDIbyUuid(connection, volumeUUID); | ||||
|             final Map<String, String> other = new HashMap<String, String>(); | ||||
|             VDI srcVolume = xenServer610Resource.getVDIbyUuid(connection, srcVolumeUuid); | ||||
| 
 | ||||
|             if (destDetails != null && Boolean.parseBoolean(destDetails.get(DiskTO.MANAGED))) { | ||||
|                 String iScsiName = destDetails.get(DiskTO.IQN); | ||||
|                 String storageHost = destDetails.get(DiskTO.STORAGE_HOST); | ||||
|                 String chapInitiatorUsername = destDetails.get(DiskTO.CHAP_INITIATOR_USERNAME); | ||||
|                 String chapInitiatorSecret = destDetails.get(DiskTO.CHAP_INITIATOR_SECRET); | ||||
| 
 | ||||
|                 destPool = xenServer610Resource.getIscsiSR(connection, iScsiName, storageHost, iScsiName, | ||||
|                         chapInitiatorUsername, chapInitiatorSecret, false); | ||||
|             } | ||||
|             else { | ||||
|                 StorageFilerTO destPoolTO = command.getPool(); | ||||
|                 String destPoolUuid = destPoolTO.getUuid(); | ||||
| 
 | ||||
|                 destPool = xenServer610Resource.getStorageRepository(connection, destPoolUuid); | ||||
|             } | ||||
| 
 | ||||
|             Map<String, String> other = new HashMap<>(); | ||||
| 
 | ||||
|             other.put("live", "true"); | ||||
| 
 | ||||
|             // Live migrate the vdi across pool. | ||||
|             final Task task = srcVolume.poolMigrateAsync(connection, destinationPool, other); | ||||
|             final long timeout = xenServer610Resource.getMigrateWait() * 1000L; | ||||
|             // Live migrate the VDI. | ||||
|             Task task = srcVolume.poolMigrateAsync(connection, destPool, other); | ||||
| 
 | ||||
|             long timeout = xenServer610Resource.getMigrateWait() * 1000L; | ||||
| 
 | ||||
|             xenServer610Resource.waitForTask(connection, task, 1000, timeout); | ||||
|             xenServer610Resource.checkForSuccess(connection, task); | ||||
| 
 | ||||
|             final VDI dvdi = Types.toVDI(task, connection); | ||||
|             VDI destVdi = Types.toVDI(task, connection); | ||||
| 
 | ||||
|             return new MigrateVolumeAnswer(command, true, null, destVdi.getUuid(connection)); | ||||
|         } catch (Exception ex) { | ||||
|             if (destDetails != null && Boolean.parseBoolean(destDetails.get(DiskTO.MANAGED)) && destPool != null) { | ||||
|                 xenServer610Resource.removeSR(connection, destPool); | ||||
|             } | ||||
| 
 | ||||
|             String msg = "Caught exception " + ex.getClass().getName() + " due to the following: " + ex.toString(); | ||||
| 
 | ||||
|             LOGGER.error(msg, ex); | ||||
| 
 | ||||
|             return new MigrateVolumeAnswer(command, true, null, dvdi.getUuid(connection)); | ||||
|         } catch (final Exception e) { | ||||
|             final String msg = "Catch Exception " + e.getClass().getName() + " due to " + e.toString(); | ||||
|             s_logger.error(msg, e); | ||||
|             return new MigrateVolumeAnswer(command, false, msg, null); | ||||
|         } | ||||
|     } | ||||
|  | ||||
| @ -321,6 +321,9 @@ public class ElastistorPrimaryDataStoreDriver extends CloudStackPrimaryDataStore | ||||
| 
 | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState) {} | ||||
| 
 | ||||
|     //this method will utilize the volume details table to add third party volume properties | ||||
|     public void updateVolumeDetails(VolumeVO volume, FileSystem esvolume) { | ||||
| 
 | ||||
|  | ||||
| @ -385,4 +385,7 @@ public class CloudStackPrimaryDataStoreDriverImpl implements PrimaryDataStoreDri | ||||
| 
 | ||||
|         callback.complete(result); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState) {} | ||||
| } | ||||
|  | ||||
| @ -206,4 +206,7 @@ public class NexentaPrimaryDataStoreDriver implements PrimaryDataStoreDriver { | ||||
| 
 | ||||
|     @Override | ||||
|     public void resize(DataObject data, AsyncCompletionCallback<CreateCmdResult> callback) {} | ||||
| 
 | ||||
|     @Override | ||||
|     public void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState) {} | ||||
| } | ||||
|  | ||||
| @ -32,6 +32,7 @@ import org.apache.cloudstack.engine.subsystem.api.storage.EndPointSelector; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.PrimaryDataStoreDriver; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.SnapshotInfo; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.TemplateInfo; | ||||
| import org.apache.cloudstack.engine.subsystem.api.storage.VolumeInfo; | ||||
| import org.apache.cloudstack.framework.async.AsyncCallbackDispatcher; | ||||
| import org.apache.cloudstack.framework.async.AsyncCompletionCallback; | ||||
| import org.apache.cloudstack.framework.async.AsyncRpcContext; | ||||
| @ -227,6 +228,10 @@ public class SamplePrimaryDataStoreDriverImpl implements PrimaryDataStoreDriver | ||||
|     public void resize(DataObject data, AsyncCompletionCallback<CreateCmdResult> callback) { | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState) { | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void takeSnapshot(SnapshotInfo snapshot, AsyncCompletionCallback<CreateCmdResult> callback) { | ||||
|     } | ||||
|  | ||||
| @ -91,6 +91,7 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver { | ||||
|     private static final long MAX_IOPS_FOR_TEMPLATE_VOLUME = 20000L; | ||||
|     private static final long MIN_IOPS_FOR_TEMP_VOLUME = 100L; | ||||
|     private static final long MAX_IOPS_FOR_TEMP_VOLUME = 20000L; | ||||
|     private static final long MAX_IOPS_FOR_MIGRATING_VOLUME = 20000L; | ||||
|     private static final long MIN_IOPS_FOR_SNAPSHOT_VOLUME = 100L; | ||||
|     private static final long MAX_IOPS_FOR_SNAPSHOT_VOLUME = 20000L; | ||||
| 
 | ||||
| @ -686,6 +687,10 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver { | ||||
|         return getBooleanValueFromVolumeDetails(volumeId, BASIC_DELETE_FAILURE); | ||||
|     } | ||||
| 
 | ||||
|     private boolean isBasicDeleteByFolder(long volumeId) { | ||||
|         return getBooleanValueFromVolumeDetails(volumeId, PrimaryDataStoreDriver.BASIC_DELETE_BY_FOLDER); | ||||
|     } | ||||
| 
 | ||||
|     private boolean isBasicGrantAccess(long volumeId) { | ||||
|         return getBooleanValueFromVolumeDetails(volumeId, BASIC_GRANT_ACCESS); | ||||
|     } | ||||
| @ -1218,13 +1223,30 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver { | ||||
|         volumeDetailsDao.removeDetail(volumeId, BASIC_DELETE_FAILURE); | ||||
|     } | ||||
| 
 | ||||
|     private void performBasicDeleteByFolder(SolidFireUtil.SolidFireConnection sfConnection, long volumeId) { | ||||
|         VolumeVO volumeVO = volumeDao.findById(volumeId); | ||||
| 
 | ||||
|         Preconditions.checkNotNull(volumeVO, "'volumeVO' should not be 'null'."); | ||||
| 
 | ||||
|         String folder = volumeVO.getFolder(); | ||||
| 
 | ||||
|         Preconditions.checkNotNull(folder, "'folder' should not be 'null'."); | ||||
| 
 | ||||
|         long sfVolumeId = Long.parseLong(folder); | ||||
| 
 | ||||
|         SolidFireUtil.deleteVolume(sfConnection, sfVolumeId); | ||||
|     } | ||||
| 
 | ||||
|     private void deleteVolume(VolumeInfo volumeInfo, long storagePoolId) { | ||||
|         try { | ||||
|             long volumeId = volumeInfo.getId(); | ||||
| 
 | ||||
|             SolidFireUtil.SolidFireConnection sfConnection = SolidFireUtil.getSolidFireConnection(storagePoolId, storagePoolDetailsDao); | ||||
| 
 | ||||
|             if (isBasicDelete(volumeId)) { | ||||
|             if (isBasicDeleteByFolder(volumeId)) { | ||||
|                 performBasicDeleteByFolder(sfConnection, volumeId); | ||||
|             } | ||||
|             else if (isBasicDelete(volumeId)) { | ||||
|                 performBasicDelete(sfConnection, volumeId); | ||||
|             } | ||||
|             else if (isBasicDeleteFailure(volumeId)) { | ||||
| @ -1436,6 +1458,21 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState) { | ||||
|         SolidFireUtil.SolidFireConnection sfConnection = SolidFireUtil.getSolidFireConnection(volumeInfo.getPoolId(), storagePoolDetailsDao); | ||||
| 
 | ||||
|         Iops iops; | ||||
| 
 | ||||
|         if (QualityOfServiceState.MIGRATION.equals(qualityOfServiceState)) { | ||||
|             iops = getIops(volumeInfo.getMinIops(), MAX_IOPS_FOR_MIGRATING_VOLUME, volumeInfo.getPoolId()); | ||||
|         } else { | ||||
|             iops = getIops(volumeInfo.getMinIops(), volumeInfo.getMaxIops(), volumeInfo.getPoolId()); | ||||
|         } | ||||
| 
 | ||||
|         SolidFireUtil.modifyVolumeQoS(sfConnection, Long.parseLong(volumeInfo.getFolder()), iops.getMinIops(), iops.getMaxIops(), iops.getBurstIops()); | ||||
|     } | ||||
| 
 | ||||
|     private void verifySufficientBytesForStoragePool(long requestedBytes, long storagePoolId) { | ||||
|         StoragePoolVO storagePool = storagePoolDao.findById(storagePoolId); | ||||
| 
 | ||||
|  | ||||
| @ -857,6 +857,15 @@ public class SolidFireUtil { | ||||
|         getSolidFireElement(sfConnection).modifyVolume(request); | ||||
|     } | ||||
| 
 | ||||
|     public static void modifyVolumeQoS(SolidFireConnection sfConnection, long volumeId, long minIops, long maxIops, long burstIops) { | ||||
|         ModifyVolumeRequest request = ModifyVolumeRequest.builder() | ||||
|                 .volumeID(volumeId) | ||||
|                 .optionalQos(new QoS(Optional.of(minIops), Optional.of(maxIops), Optional.of(burstIops), Optional.EMPTY_LONG)) | ||||
|                 .build(); | ||||
| 
 | ||||
|         getSolidFireElement(sfConnection).modifyVolume(request); | ||||
|     } | ||||
| 
 | ||||
|     public static SolidFireVolume getVolume(SolidFireConnection sfConnection, long volumeId) { | ||||
|         ListVolumesRequest request = ListVolumesRequest.builder() | ||||
|                 .optionalStartVolumeID(volumeId) | ||||
|  | ||||
							
								
								
									
										402
									
								
								test/integration/plugins/solidfire/TestOnlineStorageMigration.py
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										402
									
								
								test/integration/plugins/solidfire/TestOnlineStorageMigration.py
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,402 @@ | ||||
| # Licensed to the Apache Software Foundation (ASF) under one | ||||
| # or more contributor license agreements.  See the NOTICE file | ||||
| # distributed with this work for additional information | ||||
| # regarding copyright ownership.  The ASF licenses this file | ||||
| # to you 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 logging | ||||
| import random | ||||
| import SignedAPICall | ||||
| import XenAPI | ||||
| 
 | ||||
| from solidfire.factory import ElementFactory | ||||
| 
 | ||||
| from util import sf_util | ||||
| 
 | ||||
| # All tests inherit from cloudstackTestCase | ||||
| from marvin.cloudstackTestCase import cloudstackTestCase | ||||
| 
 | ||||
| # Import Integration Libraries | ||||
| 
 | ||||
| # base - contains all resources as entities and defines create, delete, list operations on them | ||||
| from marvin.lib.base import Account, ServiceOffering, Snapshot, StoragePool, User, VirtualMachine, Volume | ||||
| 
 | ||||
| # common - commonly used methods for all tests are listed here | ||||
| from marvin.lib.common import list_clusters, get_domain, list_hosts, list_snapshots, get_template, list_volumes, get_zone | ||||
| 
 | ||||
| # utils - utility classes for common cleanup, external library wrappers, etc. | ||||
| from marvin.lib.utils import cleanup_resources | ||||
| 
 | ||||
| # Prerequisites: | ||||
| #  Only one zone | ||||
| #  Only one pod | ||||
| #  Only one cluster | ||||
| # | ||||
| # Running the tests: | ||||
| #  If using XenServer, verify the "xen_server_hostname" variable is correct. | ||||
| # | ||||
| # Note: | ||||
| #  If you do have more than one cluster, you might need to change this line: cls.cluster = list_clusters(cls.apiClient)[0] | ||||
| 
 | ||||
| 
 | ||||
| class TestData(): | ||||
|     # constants | ||||
|     account = "account" | ||||
|     capacityBytes = "capacitybytes" | ||||
|     capacityIops = "capacityiops" | ||||
|     clusterId = "clusterId" | ||||
|     computeOffering = "computeoffering" | ||||
|     domainId = "domainId" | ||||
|     email = "email" | ||||
|     firstname = "firstname" | ||||
|     hypervisor = "hypervisor" | ||||
|     lastname = "lastname" | ||||
|     managementServer = "managementServer" | ||||
|     mvip = "mvip" | ||||
|     name = "name" | ||||
|     nfs_folder = "/export/primary2" | ||||
|     nfs_path = "nfs://10.117.40.114" + nfs_folder | ||||
|     nfs_storage_tag = "NFS-123" | ||||
|     password = "password" | ||||
|     podId = "podId" | ||||
|     port = "port" | ||||
|     primaryStorage = "primarystorage" | ||||
|     primaryStorage2 = "primarystorage2" | ||||
|     provider = "provider" | ||||
|     scope = "scope" | ||||
|     solidFire = "solidfire" | ||||
|     storageTag = "SolidFire_SAN_1" | ||||
|     tags = "tags" | ||||
|     url = "url" | ||||
|     user = "user" | ||||
|     username = "username" | ||||
|     xenServer = "xenserver" | ||||
|     zoneId = "zoneId" | ||||
| 
 | ||||
|     hypervisor_type = xenServer | ||||
|     xen_server_hostname = "XenServer-6.5-1" | ||||
| 
 | ||||
|     def __init__(self): | ||||
|         self.testdata = { | ||||
|             TestData.solidFire: { | ||||
|                 TestData.mvip: "10.117.40.120", | ||||
|                 TestData.username: "admin", | ||||
|                 TestData.password: "admin", | ||||
|                 TestData.port: 443, | ||||
|                 TestData.url: "https://10.117.40.120:443" | ||||
|             }, | ||||
|             TestData.xenServer: { | ||||
|                 TestData.username: "root", | ||||
|                 TestData.password: "solidfire" | ||||
|             }, | ||||
|             TestData.managementServer: { | ||||
|                 TestData.username: "cloudstack", | ||||
|                 TestData.password: "solidfire" | ||||
|             }, | ||||
|             TestData.account: { | ||||
|                 TestData.email: "test@test.com", | ||||
|                 TestData.firstname: "John", | ||||
|                 TestData.lastname: "Doe", | ||||
|                 TestData.username: "test", | ||||
|                 TestData.password: "test" | ||||
|             }, | ||||
|             TestData.user: { | ||||
|                 TestData.email: "user@test.com", | ||||
|                 TestData.firstname: "Jane", | ||||
|                 TestData.lastname: "Doe", | ||||
|                 TestData.username: "testuser", | ||||
|                 TestData.password: "password" | ||||
|             }, | ||||
|             TestData.primaryStorage: { | ||||
|                 TestData.name: "NFS-%d" % random.randint(0, 100), | ||||
|                 TestData.scope: "CLUSTER", | ||||
|                 TestData.clusterId: 1, | ||||
|                 TestData.url: TestData.nfs_path, | ||||
|                 TestData.tags: TestData.nfs_storage_tag | ||||
|             }, | ||||
|             TestData.primaryStorage2: { | ||||
|                 TestData.name: "SolidFire-%d" % random.randint(0, 100), | ||||
|                 TestData.scope: "ZONE", | ||||
|                 TestData.url: "MVIP=10.117.40.120;SVIP=10.117.41.120;" + | ||||
|                        "clusterAdminUsername=admin;clusterAdminPassword=admin;" + | ||||
|                        "clusterDefaultMinIops=10000;clusterDefaultMaxIops=15000;" + | ||||
|                        "clusterDefaultBurstIopsPercentOfMaxIops=1.5;", | ||||
|                 TestData.provider: "SolidFire", | ||||
|                 TestData.tags: TestData.storageTag, | ||||
|                 TestData.capacityIops: 4500000, | ||||
|                 TestData.capacityBytes: 2251799813685248, # 2 PiB | ||||
|                 TestData.hypervisor: "Any" | ||||
|             }, | ||||
|             TestData.computeOffering: { | ||||
|                 TestData.name: "SF_CO_1", | ||||
|                 "displaytext": "SF_CO_1 (Min IOPS = 300; Max IOPS = 600)", | ||||
|                 "cpunumber": 1, | ||||
|                 "cpuspeed": 100, | ||||
|                 "memory": 128, | ||||
|                 "storagetype": "shared", | ||||
|                 "customizediops": False, | ||||
|                 "miniops": "300", | ||||
|                 "maxiops": "600", | ||||
|                 "hypervisorsnapshotreserve": 200, | ||||
|                 TestData.tags: TestData.nfs_storage_tag | ||||
|             }, | ||||
|             TestData.zoneId: 1, | ||||
|             TestData.podId: 1, | ||||
|             TestData.clusterId: 1, | ||||
|             TestData.domainId: 1, | ||||
|             TestData.url: "10.117.40.114" | ||||
|         } | ||||
| 
 | ||||
| 
 | ||||
| class TestOnlineStorageMigration(cloudstackTestCase): | ||||
|     _should_only_be_one_volume_in_list_err_msg = "There should only be one volume in this list." | ||||
|     _volume_not_on_correct_primary_storage = "The volume is not on the correct primary storage." | ||||
|     _snapshot_not_associated_with_correct_volume = "The snapshot is not associated with the correct volume." | ||||
| 
 | ||||
|     @classmethod | ||||
|     def setUpClass(cls): | ||||
|         # Set up API client | ||||
|         testclient = super(TestOnlineStorageMigration, cls).getClsTestClient() | ||||
| 
 | ||||
|         cls.apiClient = testclient.getApiClient() | ||||
|         cls.configData = testclient.getParsedTestDataConfig() | ||||
|         cls.dbConnection = testclient.getDbConnection() | ||||
| 
 | ||||
|         cls.testdata = TestData().testdata | ||||
| 
 | ||||
|         sf_util.set_supports_resign(True, cls.dbConnection) | ||||
| 
 | ||||
|         cls._connect_to_hypervisor() | ||||
| 
 | ||||
|         # Set up SolidFire connection | ||||
|         solidfire = cls.testdata[TestData.solidFire] | ||||
| 
 | ||||
|         cls.sfe = ElementFactory.create(solidfire[TestData.mvip], solidfire[TestData.username], solidfire[TestData.password]) | ||||
| 
 | ||||
|         # Get Resources from Cloud Infrastructure | ||||
|         cls.zone = get_zone(cls.apiClient, zone_id=cls.testdata[TestData.zoneId]) | ||||
|         cls.cluster = list_clusters(cls.apiClient)[0] | ||||
|         cls.template = get_template(cls.apiClient, cls.zone.id, hypervisor=TestData.hypervisor_type) | ||||
|         cls.domain = get_domain(cls.apiClient, cls.testdata[TestData.domainId]) | ||||
| 
 | ||||
|         # Create test account | ||||
|         cls.account = Account.create( | ||||
|             cls.apiClient, | ||||
|             cls.testdata["account"], | ||||
|             admin=1 | ||||
|         ) | ||||
| 
 | ||||
|         # Set up connection to make customized API calls | ||||
|         cls.user = User.create( | ||||
|             cls.apiClient, | ||||
|             cls.testdata["user"], | ||||
|             account=cls.account.name, | ||||
|             domainid=cls.domain.id | ||||
|         ) | ||||
| 
 | ||||
|         url = cls.testdata[TestData.url] | ||||
| 
 | ||||
|         api_url = "http://" + url + ":8080/client/api" | ||||
|         userkeys = User.registerUserKeys(cls.apiClient, cls.user.id) | ||||
| 
 | ||||
|         cls.cs_api = SignedAPICall.CloudStack(api_url, userkeys.apikey, userkeys.secretkey) | ||||
| 
 | ||||
|         primarystorage = cls.testdata[TestData.primaryStorage] | ||||
| 
 | ||||
|         cls.primary_storage = StoragePool.create( | ||||
|             cls.apiClient, | ||||
|             primarystorage, | ||||
|             scope=primarystorage[TestData.scope], | ||||
|             zoneid=cls.zone.id, | ||||
|             podid=cls.testdata[TestData.podId], | ||||
|             clusterid=cls.cluster.id, | ||||
|             tags=primarystorage[TestData.tags] | ||||
|         ) | ||||
| 
 | ||||
|         primarystorage2 = cls.testdata[TestData.primaryStorage2] | ||||
| 
 | ||||
|         cls.primary_storage_2 = StoragePool.create( | ||||
|             cls.apiClient, | ||||
|             primarystorage2, | ||||
|             scope=primarystorage2[TestData.scope], | ||||
|             zoneid=cls.zone.id, | ||||
|             provider=primarystorage2[TestData.provider], | ||||
|             tags=primarystorage2[TestData.tags], | ||||
|             capacityiops=primarystorage2[TestData.capacityIops], | ||||
|             capacitybytes=primarystorage2[TestData.capacityBytes], | ||||
|             hypervisor=primarystorage2[TestData.hypervisor] | ||||
|         ) | ||||
| 
 | ||||
|         cls.compute_offering = ServiceOffering.create( | ||||
|             cls.apiClient, | ||||
|             cls.testdata[TestData.computeOffering] | ||||
|         ) | ||||
| 
 | ||||
|         # Resources that are to be destroyed | ||||
|         cls._cleanup = [ | ||||
|             cls.compute_offering, | ||||
|             cls.user, | ||||
|             cls.account | ||||
|         ] | ||||
| 
 | ||||
|     @classmethod | ||||
|     def tearDownClass(cls): | ||||
|         try: | ||||
|             cleanup_resources(cls.apiClient, cls._cleanup) | ||||
| 
 | ||||
|             cls.primary_storage.delete(cls.apiClient) | ||||
|             cls.primary_storage_2.delete(cls.apiClient) | ||||
| 
 | ||||
|             ms = cls.testdata[TestData.managementServer] | ||||
|             ip_address = cls.testdata[TestData.url] | ||||
| 
 | ||||
|             cls._delete_all_files(ip_address, ms[TestData.username], ms[TestData.password], TestData.nfs_folder) | ||||
| 
 | ||||
|             sf_util.purge_solidfire_volumes(cls.sfe) | ||||
|         except Exception as e: | ||||
|             logging.debug("Exception in tearDownClass(cls): %s" % e) | ||||
| 
 | ||||
|     def setUp(self): | ||||
|         self.cleanup = [] | ||||
| 
 | ||||
|     def tearDown(self): | ||||
|         cleanup_resources(self.apiClient, self.cleanup) | ||||
| 
 | ||||
|     def test_online_migrate_volume_from_nfs_storage_to_managed_storage(self): | ||||
|         if TestData.hypervisor_type != TestData.xenServer: | ||||
|             return | ||||
| 
 | ||||
|         virtual_machine = VirtualMachine.create( | ||||
|             self.apiClient, | ||||
|             self._get_vm_name(), | ||||
|             accountid=self.account.name, | ||||
|             zoneid=self.zone.id, | ||||
|             serviceofferingid=self.compute_offering.id, | ||||
|             templateid=self.template.id, | ||||
|             domainid=self.domain.id, | ||||
|             startvm=True | ||||
|         ) | ||||
| 
 | ||||
|         self.cleanup.append(virtual_machine) | ||||
| 
 | ||||
|         vm_root_volume = self._get_only_volume(virtual_machine.id) | ||||
| 
 | ||||
|         self._verify_volume_on_primary_storage(vm_root_volume, self.primary_storage) | ||||
| 
 | ||||
|         # Migrate the root disk from NFS storage to managed storage. | ||||
| 
 | ||||
|         Volume.migrate(self.apiClient, livemigrate=True, volumeid=vm_root_volume.id, storageid=self.primary_storage_2.id) | ||||
| 
 | ||||
|         vm_root_volume = self._get_only_volume(virtual_machine.id) | ||||
| 
 | ||||
|         self._verify_volume_on_primary_storage(vm_root_volume, self.primary_storage_2) | ||||
| 
 | ||||
|     def test_online_migrate_volume_with_snapshot_from_nfs_storage_to_managed_storage(self): | ||||
|         if TestData.hypervisor_type != TestData.xenServer: | ||||
|             return | ||||
| 
 | ||||
|         virtual_machine = VirtualMachine.create( | ||||
|             self.apiClient, | ||||
|             self._get_vm_name(), | ||||
|             accountid=self.account.name, | ||||
|             zoneid=self.zone.id, | ||||
|             serviceofferingid=self.compute_offering.id, | ||||
|             templateid=self.template.id, | ||||
|             domainid=self.domain.id, | ||||
|             startvm=True | ||||
|         ) | ||||
| 
 | ||||
|         self.cleanup.append(virtual_machine) | ||||
| 
 | ||||
|         vm_root_volume = self._get_only_volume(virtual_machine.id) | ||||
| 
 | ||||
|         self._verify_volume_on_primary_storage(vm_root_volume, self.primary_storage) | ||||
| 
 | ||||
|         vol_snap = Snapshot.create( | ||||
|             self.apiClient, | ||||
|             volume_id=vm_root_volume.id | ||||
|         ) | ||||
| 
 | ||||
|         self.cleanup.append(vol_snap) | ||||
| 
 | ||||
|         self._verify_snapshot_belongs_to_volume(vol_snap.id, vm_root_volume.id) | ||||
| 
 | ||||
|         # Migrate the root disk from NFS storage to managed storage. | ||||
| 
 | ||||
|         Volume.migrate(self.apiClient, livemigrate=True, volumeid=vm_root_volume.id, storageid=self.primary_storage_2.id) | ||||
| 
 | ||||
|         vm_root_volume = self._get_only_volume(virtual_machine.id) | ||||
| 
 | ||||
|         self._verify_volume_on_primary_storage(vm_root_volume, self.primary_storage_2) | ||||
| 
 | ||||
|         self._verify_snapshot_belongs_to_volume(vol_snap.id, vm_root_volume.id) | ||||
| 
 | ||||
|     def _verify_snapshot_belongs_to_volume(self, snapshot_id, volume_id): | ||||
|         snapshot = list_snapshots( | ||||
|             self.apiClient, | ||||
|             id=snapshot_id | ||||
|         )[0] | ||||
| 
 | ||||
|         self.assertEqual( | ||||
|             snapshot.volumeid, | ||||
|             volume_id, | ||||
|             TestOnlineStorageMigration._snapshot_not_associated_with_correct_volume | ||||
|         ) | ||||
| 
 | ||||
|     def _get_only_volume(self, virtual_machine_id): | ||||
|         list_volumes_response = list_volumes( | ||||
|             self.apiClient, | ||||
|             virtualmachineid=virtual_machine_id, | ||||
|             listall=True | ||||
|         ) | ||||
| 
 | ||||
|         sf_util.check_list(list_volumes_response, 1, self, TestOnlineStorageMigration._should_only_be_one_volume_in_list_err_msg) | ||||
| 
 | ||||
|         return list_volumes_response[0] | ||||
| 
 | ||||
|     def _verify_volume_on_primary_storage(self, vm_root_volume, primary_storage): | ||||
|         self.assertEqual( | ||||
|             vm_root_volume.storageid, | ||||
|             primary_storage.id, | ||||
|             TestOnlineStorageMigration._volume_not_on_correct_primary_storage | ||||
|         ) | ||||
| 
 | ||||
|     def _get_vm_name(self): | ||||
|         number = random.randint(0, 1000) | ||||
| 
 | ||||
|         vm_name = { | ||||
|             TestData.name: "VM-%d" % number, | ||||
|             "displayname": "Test VM %d" % number | ||||
|         } | ||||
| 
 | ||||
|         return vm_name | ||||
| 
 | ||||
|     @classmethod | ||||
|     def _delete_all_files(cls, ip_address, username, password, path): | ||||
|         ssh_connection = sf_util.get_ssh_connection(ip_address, username, password) | ||||
| 
 | ||||
|         ssh_connection.exec_command("sudo rm -rf " + path + "/*") | ||||
| 
 | ||||
|         ssh_connection.close() | ||||
| 
 | ||||
|     @classmethod | ||||
|     def _connect_to_hypervisor(cls): | ||||
|         host_ip = "https://" + \ | ||||
|               list_hosts(cls.apiClient, clusterid=cls.testdata[TestData.clusterId], name=TestData.xen_server_hostname)[0].ipaddress | ||||
| 
 | ||||
|         cls.xen_session = XenAPI.Session(host_ip) | ||||
| 
 | ||||
|         xen_server = cls.testdata[TestData.xenServer] | ||||
| 
 | ||||
|         cls.xen_session.xenapi.login_with_password(xen_server[TestData.username], xen_server[TestData.password]) | ||||
		Loading…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user