mirror of https://github.com/apache/cloudstack.git
Fix exceeding of resource limits with powerflex (#9008)
* Fix exceeding of resource limits with powerflex * Add e2e tests * Update server/src/main/java/com/cloud/vm/UserVmManagerImpl.java Co-authored-by: Suresh Kumar Anaparti <sureshkumar.anaparti@gmail.com> * fixup --------- Co-authored-by: Suresh Kumar Anaparti <sureshkumar.anaparti@gmail.com>
This commit is contained in:
parent
e9ff2707bb
commit
21af134087
|
|
@ -243,6 +243,8 @@ public interface ResourceLimitService {
|
|||
void checkVolumeResourceLimitForDiskOfferingChange(Account owner, Boolean display, Long currentSize, Long newSize,
|
||||
DiskOffering currentOffering, DiskOffering newOffering) throws ResourceAllocationException;
|
||||
|
||||
void checkPrimaryStorageResourceLimit(Account owner, Boolean display, Long size, DiskOffering diskOffering) throws ResourceAllocationException;
|
||||
|
||||
void incrementVolumeResourceCount(long accountId, Boolean display, Long size, DiskOffering diskOffering);
|
||||
void decrementVolumeResourceCount(long accountId, Boolean display, Long size, DiskOffering diskOffering);
|
||||
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import com.cloud.exception.ResourceAllocationException;
|
||||
import org.apache.cloudstack.engine.subsystem.api.storage.DataObject;
|
||||
import org.apache.cloudstack.engine.subsystem.api.storage.DataStore;
|
||||
import org.apache.cloudstack.engine.subsystem.api.storage.VolumeInfo;
|
||||
|
|
@ -126,7 +127,7 @@ public interface VolumeOrchestrationService {
|
|||
|
||||
void prepareForMigration(VirtualMachineProfile vm, DeployDestination dest);
|
||||
|
||||
void prepare(VirtualMachineProfile vm, DeployDestination dest) throws StorageUnavailableException, InsufficientStorageCapacityException, ConcurrentOperationException, StorageAccessException;
|
||||
void prepare(VirtualMachineProfile vm, DeployDestination dest) throws StorageUnavailableException, InsufficientStorageCapacityException, ConcurrentOperationException, StorageAccessException, ResourceAllocationException;
|
||||
|
||||
boolean canVmRestartOnAnotherServer(long vmId);
|
||||
|
||||
|
|
|
|||
|
|
@ -157,4 +157,20 @@ public interface PrimaryDataStoreDriver extends DataStoreDriver {
|
|||
default boolean zoneWideVolumesAvailableWithoutClusterMotion() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method returns the actual size required on the pool for a volume.
|
||||
*
|
||||
* @param volumeSize
|
||||
* Size of volume to be created on the store
|
||||
* @param templateSize
|
||||
* Size of template, if any, which will be used to create the volume
|
||||
* @param isEncryptionRequired
|
||||
* true if volume is encrypted
|
||||
*
|
||||
* @return the size required on the pool for the volume
|
||||
*/
|
||||
default long getVolumeSizeRequiredOnPool(long volumeSize, Long templateSize, boolean isEncryptionRequired) {
|
||||
return volumeSize;
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -52,6 +52,7 @@ import javax.persistence.EntityExistsException;
|
|||
import com.cloud.configuration.Resource;
|
||||
import com.cloud.domain.Domain;
|
||||
import com.cloud.domain.dao.DomainDao;
|
||||
import com.cloud.exception.ResourceAllocationException;
|
||||
import com.cloud.network.vpc.VpcVO;
|
||||
import com.cloud.network.vpc.dao.VpcDao;
|
||||
import com.cloud.user.dao.AccountDao;
|
||||
|
|
@ -1403,7 +1404,7 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
|
|||
logger.warn("unexpected InsufficientCapacityException : {}", e.getScope().getName(), e);
|
||||
}
|
||||
}
|
||||
} catch (ExecutionException | NoTransitionException e) {
|
||||
} catch (ExecutionException | NoTransitionException | ResourceAllocationException e) {
|
||||
logger.error("Failed to start instance {}", vm, e);
|
||||
throw new AgentUnavailableException("Unable to start instance due to " + e.getMessage(), destHostId, e);
|
||||
} catch (final StorageAccessException e) {
|
||||
|
|
|
|||
|
|
@ -38,6 +38,11 @@ import java.util.stream.Collectors;
|
|||
import javax.inject.Inject;
|
||||
import javax.naming.ConfigurationException;
|
||||
|
||||
import com.cloud.exception.ResourceAllocationException;
|
||||
import com.cloud.storage.DiskOfferingVO;
|
||||
import com.cloud.storage.VMTemplateVO;
|
||||
import com.cloud.storage.dao.VMTemplateDao;
|
||||
import com.cloud.user.AccountManager;
|
||||
import org.apache.cloudstack.api.ApiCommandResourceType;
|
||||
import org.apache.cloudstack.api.ApiConstants.IoDriverPolicy;
|
||||
import org.apache.cloudstack.api.command.admin.vm.MigrateVMCmd;
|
||||
|
|
@ -180,6 +185,8 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
}
|
||||
|
||||
|
||||
@Inject
|
||||
private AccountManager _accountMgr;
|
||||
@Inject
|
||||
EntityManager _entityMgr;
|
||||
@Inject
|
||||
|
|
@ -195,6 +202,8 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
@Inject
|
||||
protected VolumeDao _volumeDao;
|
||||
@Inject
|
||||
protected VMTemplateDao _templateDao;
|
||||
@Inject
|
||||
protected SnapshotDao _snapshotDao;
|
||||
@Inject
|
||||
protected SnapshotDataStoreDao _snapshotDataStoreDao;
|
||||
|
|
@ -1677,7 +1686,7 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
}
|
||||
}
|
||||
|
||||
private Pair<VolumeVO, DataStore> recreateVolume(VolumeVO vol, VirtualMachineProfile vm, DeployDestination dest) throws StorageUnavailableException, StorageAccessException {
|
||||
private Pair<VolumeVO, DataStore> recreateVolume(VolumeVO vol, VirtualMachineProfile vm, DeployDestination dest) throws StorageUnavailableException, StorageAccessException, ResourceAllocationException {
|
||||
String volToString = getReflectOnlySelectedFields(vol);
|
||||
|
||||
VolumeVO newVol;
|
||||
|
|
@ -1710,6 +1719,7 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
}
|
||||
logger.debug("Created new volume [{}] from old volume [{}].", newVolToString, volToString);
|
||||
}
|
||||
updateVolumeSize(destPool, newVol);
|
||||
VolumeInfo volume = volFactory.getVolume(newVol.getId(), destPool);
|
||||
Long templateId = newVol.getTemplateId();
|
||||
for (int i = 0; i < 2; i++) {
|
||||
|
|
@ -1841,8 +1851,39 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method checks if size of volume on the data store would be different.
|
||||
* If it's different it verifies the resource limits and updates the volume's size
|
||||
*/
|
||||
protected void updateVolumeSize(DataStore store, VolumeVO vol) throws ResourceAllocationException {
|
||||
if (store == null || !(store.getDriver() instanceof PrimaryDataStoreDriver)) {
|
||||
return;
|
||||
}
|
||||
|
||||
VMTemplateVO template = vol.getTemplateId() != null ? _templateDao.findById(vol.getTemplateId()) : null;
|
||||
PrimaryDataStoreDriver driver = (PrimaryDataStoreDriver) store.getDriver();
|
||||
long newSize = driver.getVolumeSizeRequiredOnPool(vol.getSize(),
|
||||
template == null ? null : template.getSize(),
|
||||
vol.getPassphraseId() != null);
|
||||
|
||||
if (newSize != vol.getSize()) {
|
||||
DiskOfferingVO diskOffering = diskOfferingDao.findByIdIncludingRemoved(vol.getDiskOfferingId());
|
||||
if (newSize > vol.getSize()) {
|
||||
_resourceLimitMgr.checkPrimaryStorageResourceLimit(_accountMgr.getActiveAccountById(vol.getAccountId()),
|
||||
vol.isDisplay(), newSize - vol.getSize(), diskOffering);
|
||||
_resourceLimitMgr.incrementVolumePrimaryStorageResourceCount(vol.getAccountId(), vol.isDisplay(),
|
||||
newSize - vol.getSize(), diskOffering);
|
||||
} else {
|
||||
_resourceLimitMgr.decrementVolumePrimaryStorageResourceCount(vol.getAccountId(), vol.isDisplay(),
|
||||
vol.getSize() - newSize, diskOffering);
|
||||
}
|
||||
vol.setSize(newSize);
|
||||
_volsDao.persist(vol);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void prepare(VirtualMachineProfile vm, DeployDestination dest) throws StorageUnavailableException, InsufficientStorageCapacityException, ConcurrentOperationException, StorageAccessException {
|
||||
public void prepare(VirtualMachineProfile vm, DeployDestination dest) throws StorageUnavailableException, InsufficientStorageCapacityException, ConcurrentOperationException, StorageAccessException, ResourceAllocationException {
|
||||
if (dest == null) {
|
||||
String msg = String.format("Unable to prepare volumes for the VM [%s] because DeployDestination is null.", vm.getVirtualMachine());
|
||||
logger.error(msg);
|
||||
|
|
@ -1865,7 +1906,7 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
|
||||
String volToString = getReflectOnlySelectedFields(vol);
|
||||
|
||||
store = (PrimaryDataStore)dataStoreMgr.getDataStore(task.pool.getId(), DataStoreRole.Primary);
|
||||
store = (PrimaryDataStore) dataStoreMgr.getDataStore(task.pool.getId(), DataStoreRole.Primary);
|
||||
|
||||
// For zone-wide managed storage, it is possible that the VM can be started in another
|
||||
// cluster. In that case, make sure that the volume is in the right access group.
|
||||
|
|
@ -1876,6 +1917,8 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
long lastClusterId = lastHost == null || lastHost.getClusterId() == null ? -1 : lastHost.getClusterId();
|
||||
long clusterId = host == null || host.getClusterId() == null ? -1 : host.getClusterId();
|
||||
|
||||
updateVolumeSize(store, (VolumeVO) vol);
|
||||
|
||||
if (lastClusterId != clusterId) {
|
||||
if (lastHost != null) {
|
||||
storageMgr.removeStoragePoolFromCluster(lastHost.getId(), vol.get_iScsiName(), store);
|
||||
|
|
@ -1895,6 +1938,7 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
|
|||
}
|
||||
} else if (task.type == VolumeTaskType.MIGRATE) {
|
||||
store = (PrimaryDataStore) dataStoreMgr.getDataStore(task.pool.getId(), DataStoreRole.Primary);
|
||||
updateVolumeSize(store, task.volume);
|
||||
vol = migrateVolume(task.volume, store);
|
||||
} else if (task.type == VolumeTaskType.RECREATE) {
|
||||
Pair<VolumeVO, DataStore> result = recreateVolume(task.volume, vm, dest);
|
||||
|
|
|
|||
|
|
@ -1340,6 +1340,16 @@ public class ScaleIOPrimaryDataStoreDriver implements PrimaryDataStoreDriver {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getVolumeSizeRequiredOnPool(long volumeSize, Long templateSize, boolean isEncryptionRequired) {
|
||||
long newSizeInGB = volumeSize / (1024 * 1024 * 1024);
|
||||
if (templateSize != null && isEncryptionRequired && needsExpansionForEncryptionHeader(templateSize, volumeSize)) {
|
||||
newSizeInGB = (volumeSize + (1<<30)) / (1024 * 1024 * 1024);
|
||||
}
|
||||
long newSizeIn8gbBoundary = (long) (Math.ceil(newSizeInGB / 8.0) * 8.0);
|
||||
return newSizeIn8gbBoundary * (1024 * 1024 * 1024);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleQualityOfServiceForVolumeMigration(VolumeInfo volumeInfo, QualityOfServiceState qualityOfServiceState) {
|
||||
}
|
||||
|
|
|
|||
|
|
@ -542,4 +542,37 @@ public class ScaleIOPrimaryDataStoreDriverTest {
|
|||
|
||||
Assert.assertEquals(false, answer.getResult());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetVolumeSizeRequiredOnPool() {
|
||||
Assert.assertEquals(16L * (1024 * 1024 * 1024),
|
||||
scaleIOPrimaryDataStoreDriver.getVolumeSizeRequiredOnPool(
|
||||
10L * (1024 * 1024 * 1024),
|
||||
null,
|
||||
true));
|
||||
|
||||
Assert.assertEquals(16L * (1024 * 1024 * 1024),
|
||||
scaleIOPrimaryDataStoreDriver.getVolumeSizeRequiredOnPool(
|
||||
10L * (1024 * 1024 * 1024),
|
||||
null,
|
||||
false));
|
||||
|
||||
Assert.assertEquals(16L * (1024 * 1024 * 1024),
|
||||
scaleIOPrimaryDataStoreDriver.getVolumeSizeRequiredOnPool(
|
||||
16L * (1024 * 1024 * 1024),
|
||||
null,
|
||||
false));
|
||||
|
||||
Assert.assertEquals(16L * (1024 * 1024 * 1024),
|
||||
scaleIOPrimaryDataStoreDriver.getVolumeSizeRequiredOnPool(
|
||||
16L * (1024 * 1024 * 1024),
|
||||
16L * (1024 * 1024 * 1024),
|
||||
false));
|
||||
|
||||
Assert.assertEquals(24L * (1024 * 1024 * 1024),
|
||||
scaleIOPrimaryDataStoreDriver.getVolumeSizeRequiredOnPool(
|
||||
16L * (1024 * 1024 * 1024),
|
||||
16L * (1024 * 1024 * 1024),
|
||||
true));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1641,6 +1641,19 @@ public class ResourceLimitManagerImpl extends ManagerBase implements ResourceLim
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkPrimaryStorageResourceLimit(Account owner, Boolean display, Long size, DiskOffering diskOffering) throws ResourceAllocationException {
|
||||
List<String> tags = getResourceLimitStorageTagsForResourceCountOperation(display, diskOffering);
|
||||
if (CollectionUtils.isEmpty(tags)) {
|
||||
return;
|
||||
}
|
||||
if (size != null) {
|
||||
for (String tag : tags) {
|
||||
checkResourceLimitWithTag(owner, ResourceType.primary_storage, tag, size);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkVolumeResourceLimitForDiskOfferingChange(Account owner, Boolean display, Long currentSize, Long newSize,
|
||||
DiskOffering currentOffering, DiskOffering newOffering
|
||||
|
|
|
|||
|
|
@ -1240,6 +1240,16 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
|
|||
}
|
||||
|
||||
long currentSize = volume.getSize();
|
||||
VolumeInfo volInfo = volFactory.getVolume(volume.getId());
|
||||
boolean isEncryptionRequired = volume.getPassphraseId() != null;
|
||||
if (newDiskOffering != null) {
|
||||
isEncryptionRequired = newDiskOffering.getEncrypt();
|
||||
}
|
||||
|
||||
DataStore dataStore = volInfo.getDataStore();
|
||||
if (dataStore != null && dataStore.getDriver() instanceof PrimaryDataStoreDriver) {
|
||||
newSize = ((PrimaryDataStoreDriver) dataStore.getDriver()).getVolumeSizeRequiredOnPool(newSize, null, isEncryptionRequired);
|
||||
}
|
||||
validateVolumeResizeWithSize(volume, currentSize, newSize, shrinkOk, diskOffering, newDiskOffering);
|
||||
|
||||
// Note: The storage plug-in in question should perform validation on the IOPS to check if a sufficient number of IOPS is available to perform
|
||||
|
|
@ -1982,6 +1992,14 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
|
|||
newMaxIops = updateNewMaxIops[0];
|
||||
newHypervisorSnapshotReserve = updateNewHypervisorSnapshotReserve[0];
|
||||
long currentSize = volume.getSize();
|
||||
|
||||
VolumeInfo volInfo = volFactory.getVolume(volume.getId());
|
||||
|
||||
DataStore dataStore = volInfo.getDataStore();
|
||||
if (dataStore != null && dataStore.getDriver() instanceof PrimaryDataStoreDriver) {
|
||||
newSize = ((PrimaryDataStoreDriver) dataStore.getDriver()).getVolumeSizeRequiredOnPool(newSize, null, newDiskOffering.getEncrypt());
|
||||
}
|
||||
|
||||
validateVolumeResizeWithSize(volume, currentSize, newSize, shrinkOk, existingDiskOffering, newDiskOffering);
|
||||
|
||||
/* If this volume has never been beyond allocated state, short circuit everything and simply update the database. */
|
||||
|
|
|
|||
|
|
@ -7980,7 +7980,7 @@ public class UserVmManagerImpl extends ManagerBase implements UserVmManager, Vir
|
|||
}
|
||||
|
||||
for (VolumeVO root : rootVols) {
|
||||
if ( !Volume.State.Allocated.equals(root.getState()) || newTemplateId != null ) {
|
||||
if ( !Volume.State.Allocated.equals(root.getState()) || newTemplateId != null || diskOffering != null) {
|
||||
_volumeService.validateDestroyVolume(root, caller, Volume.State.Allocated.equals(root.getState()) || expunge, false);
|
||||
final UserVmVO userVm = vm;
|
||||
Pair<UserVmVO, Volume> vmAndNewVol = Transaction.execute(new TransactionCallbackWithException<Pair<UserVmVO, Volume>, CloudRuntimeException>() {
|
||||
|
|
@ -8013,7 +8013,7 @@ public class UserVmManagerImpl extends ManagerBase implements UserVmManager, Vir
|
|||
newVol = volumeMgr.allocateDuplicateVolume(root, diskOffering, null);
|
||||
}
|
||||
|
||||
updateVolume(newVol, template, userVm, diskOffering, details);
|
||||
getRootVolumeSizeForVmRestore(newVol, template, userVm, diskOffering, details, true);
|
||||
volumeMgr.saveVolumeDetails(newVol.getDiskOfferingId(), newVol.getId());
|
||||
|
||||
// 1. Save usage event and update resource count for user vm volumes
|
||||
|
|
@ -8112,62 +8112,87 @@ public class UserVmManagerImpl extends ManagerBase implements UserVmManager, Vir
|
|||
|
||||
}
|
||||
|
||||
private void updateVolume(Volume vol, VMTemplateVO template, UserVmVO userVm, DiskOffering diskOffering, Map<String, String> details) {
|
||||
Long getRootVolumeSizeForVmRestore(Volume vol, VMTemplateVO template, UserVmVO userVm, DiskOffering diskOffering, Map<String, String> details, boolean update) {
|
||||
VolumeVO resizedVolume = (VolumeVO) vol;
|
||||
|
||||
Long size = null;
|
||||
if (template != null && template.getSize() != null) {
|
||||
UserVmDetailVO vmRootDiskSizeDetail = userVmDetailsDao.findDetail(userVm.getId(), VmDetailConstants.ROOT_DISK_SIZE);
|
||||
if (vmRootDiskSizeDetail == null) {
|
||||
resizedVolume.setSize(template.getSize());
|
||||
size = template.getSize();
|
||||
} else {
|
||||
long rootDiskSize = Long.parseLong(vmRootDiskSizeDetail.getValue()) * GiB_TO_BYTES;
|
||||
if (template.getSize() >= rootDiskSize) {
|
||||
resizedVolume.setSize(template.getSize());
|
||||
userVmDetailsDao.remove(vmRootDiskSizeDetail.getId());
|
||||
size = template.getSize();
|
||||
if (update) {
|
||||
userVmDetailsDao.remove(vmRootDiskSizeDetail.getId());
|
||||
}
|
||||
} else {
|
||||
resizedVolume.setSize(rootDiskSize);
|
||||
size = rootDiskSize;
|
||||
}
|
||||
}
|
||||
if (update) {
|
||||
resizedVolume.setSize(size);
|
||||
}
|
||||
}
|
||||
|
||||
if (diskOffering != null) {
|
||||
resizedVolume.setDiskOfferingId(diskOffering.getId());
|
||||
if (update) {
|
||||
resizedVolume.setDiskOfferingId(diskOffering.getId());
|
||||
}
|
||||
// Size of disk offering should be greater than or equal to the template's size and this should be validated before this
|
||||
if (!diskOffering.isCustomized()) {
|
||||
resizedVolume.setSize(diskOffering.getDiskSize());
|
||||
}
|
||||
if (diskOffering.getMinIops() != null) {
|
||||
resizedVolume.setMinIops(diskOffering.getMinIops());
|
||||
}
|
||||
if (diskOffering.getMaxIops() != null) {
|
||||
resizedVolume.setMaxIops(diskOffering.getMaxIops());
|
||||
}
|
||||
}
|
||||
|
||||
if (MapUtils.isNotEmpty(details)) {
|
||||
if (StringUtils.isNumeric(details.get(VmDetailConstants.ROOT_DISK_SIZE))) {
|
||||
Long rootDiskSize = Long.parseLong(details.get(VmDetailConstants.ROOT_DISK_SIZE)) * GiB_TO_BYTES;
|
||||
resizedVolume.setSize(rootDiskSize);
|
||||
UserVmDetailVO vmRootDiskSizeDetail = userVmDetailsDao.findDetail(userVm.getId(), VmDetailConstants.ROOT_DISK_SIZE);
|
||||
if (vmRootDiskSizeDetail != null) {
|
||||
vmRootDiskSizeDetail.setValue(details.get(VmDetailConstants.ROOT_DISK_SIZE));
|
||||
userVmDetailsDao.update(vmRootDiskSizeDetail.getId(), vmRootDiskSizeDetail);
|
||||
} else {
|
||||
userVmDetailsDao.persist(new UserVmDetailVO(userVm.getId(), VmDetailConstants.ROOT_DISK_SIZE,
|
||||
details.get(VmDetailConstants.ROOT_DISK_SIZE), true));
|
||||
size = diskOffering.getDiskSize();
|
||||
if (update) {
|
||||
resizedVolume.setSize(diskOffering.getDiskSize());
|
||||
}
|
||||
}
|
||||
|
||||
String minIops = details.get(MIN_IOPS);
|
||||
String maxIops = details.get(MAX_IOPS);
|
||||
|
||||
if (StringUtils.isNumeric(minIops)) {
|
||||
resizedVolume.setMinIops(Long.parseLong(minIops));
|
||||
}
|
||||
if (StringUtils.isNumeric(maxIops)) {
|
||||
resizedVolume.setMinIops(Long.parseLong(maxIops));
|
||||
if (update) {
|
||||
if (diskOffering.getMinIops() != null) {
|
||||
resizedVolume.setMinIops(diskOffering.getMinIops());
|
||||
}
|
||||
if (diskOffering.getMaxIops() != null) {
|
||||
resizedVolume.setMaxIops(diskOffering.getMaxIops());
|
||||
}
|
||||
}
|
||||
}
|
||||
_volsDao.update(resizedVolume.getId(), resizedVolume);
|
||||
|
||||
// Size of disk should be greater than or equal to the template's size and this should be validated before this
|
||||
if (MapUtils.isNotEmpty(details)) {
|
||||
if (StringUtils.isNumeric(details.get(VmDetailConstants.ROOT_DISK_SIZE))) {
|
||||
Long rootDiskSize = Long.parseLong(details.get(VmDetailConstants.ROOT_DISK_SIZE)) * GiB_TO_BYTES;
|
||||
size = rootDiskSize;
|
||||
if (update) {
|
||||
resizedVolume.setSize(rootDiskSize);
|
||||
}
|
||||
UserVmDetailVO vmRootDiskSizeDetail = userVmDetailsDao.findDetail(userVm.getId(), VmDetailConstants.ROOT_DISK_SIZE);
|
||||
if (update) {
|
||||
if (vmRootDiskSizeDetail != null) {
|
||||
vmRootDiskSizeDetail.setValue(details.get(VmDetailConstants.ROOT_DISK_SIZE));
|
||||
userVmDetailsDao.update(vmRootDiskSizeDetail.getId(), vmRootDiskSizeDetail);
|
||||
} else {
|
||||
userVmDetailsDao.persist(new UserVmDetailVO(userVm.getId(), VmDetailConstants.ROOT_DISK_SIZE,
|
||||
details.get(VmDetailConstants.ROOT_DISK_SIZE), true));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (update) {
|
||||
String minIops = details.get(MIN_IOPS);
|
||||
String maxIops = details.get(MAX_IOPS);
|
||||
|
||||
if (StringUtils.isNumeric(minIops)) {
|
||||
resizedVolume.setMinIops(Long.parseLong(minIops));
|
||||
}
|
||||
if (StringUtils.isNumeric(maxIops)) {
|
||||
resizedVolume.setMinIops(Long.parseLong(maxIops));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (update) {
|
||||
_volsDao.update(resizedVolume.getId(), resizedVolume);
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
private void updateVMDynamicallyScalabilityUsingTemplate(UserVmVO vm, Long newTemplateId) {
|
||||
|
|
@ -8185,7 +8210,7 @@ public class UserVmManagerImpl extends ManagerBase implements UserVmManager, Vir
|
|||
* @param template template
|
||||
* @throws InvalidParameterValueException if restore is not possible
|
||||
*/
|
||||
private void checkRestoreVmFromTemplate(UserVmVO vm, VMTemplateVO template, List<VolumeVO> volumes, DiskOffering newDiskOffering, Map<String,String> details) throws ResourceAllocationException {
|
||||
private void checkRestoreVmFromTemplate(UserVmVO vm, VMTemplateVO template, List<VolumeVO> rootVolumes, DiskOffering newDiskOffering, Map<String,String> details) throws ResourceAllocationException {
|
||||
TemplateDataStoreVO tmplStore;
|
||||
if (!template.isDirectDownload()) {
|
||||
tmplStore = _templateStoreDao.findByTemplateZoneReady(template.getId(), vm.getDataCenterId());
|
||||
|
|
@ -8206,17 +8231,15 @@ public class UserVmManagerImpl extends ManagerBase implements UserVmManager, Vir
|
|||
_resourceLimitMgr.checkVmResourceLimitsForTemplateChange(owner, vm.isDisplay(), serviceOffering, currentTemplate, template);
|
||||
}
|
||||
|
||||
Long newSize = newDiskOffering != null ? newDiskOffering.getDiskSize() : null;
|
||||
if (MapUtils.isNotEmpty(details) && StringUtils.isNumeric(details.get(VmDetailConstants.ROOT_DISK_SIZE))) {
|
||||
newSize = Long.parseLong(details.get(VmDetailConstants.ROOT_DISK_SIZE)) * GiB_TO_BYTES;
|
||||
}
|
||||
if (newDiskOffering != null || newSize != null) {
|
||||
for (Volume vol : volumes) {
|
||||
if (newDiskOffering != null || !vol.getSize().equals(newSize)) {
|
||||
DiskOffering currentOffering = _diskOfferingDao.findById(vol.getDiskOfferingId());
|
||||
_resourceLimitMgr.checkVolumeResourceLimitForDiskOfferingChange(owner, vol.isDisplay(),
|
||||
vol.getSize(), newSize, currentOffering, newDiskOffering);
|
||||
}
|
||||
for (Volume vol : rootVolumes) {
|
||||
Long newSize = getRootVolumeSizeForVmRestore(vol, template, vm, newDiskOffering, details, false);
|
||||
if (newSize == null) {
|
||||
newSize = vol.getSize();
|
||||
}
|
||||
if (newDiskOffering != null || !vol.getSize().equals(newSize)) {
|
||||
DiskOffering currentOffering = _diskOfferingDao.findById(vol.getDiskOfferingId());
|
||||
_resourceLimitMgr.checkVolumeResourceLimitForDiskOfferingChange(owner, vol.isDisplay(),
|
||||
vol.getSize(), newSize, currentOffering, newDiskOffering);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -40,6 +40,7 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.cloud.offering.DiskOffering;
|
||||
import org.apache.cloudstack.api.BaseCmd.HTTPMethod;
|
||||
import org.apache.cloudstack.api.command.user.vm.DeployVMCmd;
|
||||
import org.apache.cloudstack.api.command.user.vm.DeployVnfApplianceCmd;
|
||||
|
|
@ -1563,4 +1564,37 @@ public class UserVmManagerImplTest {
|
|||
Assert.fail(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRootVolumeSizeForVmRestore() {
|
||||
VMTemplateVO template = Mockito.mock(VMTemplateVO.class);
|
||||
Mockito.when(template.getSize()).thenReturn(10L * GiB_TO_BYTES);
|
||||
UserVmVO userVm = Mockito.mock(UserVmVO.class);
|
||||
Mockito.when(userVm.getId()).thenReturn(1L);
|
||||
DiskOffering diskOffering = Mockito.mock(DiskOffering.class);
|
||||
Mockito.when(diskOffering.isCustomized()).thenReturn(false);
|
||||
Mockito.when(diskOffering.getDiskSize()).thenReturn(8L * GiB_TO_BYTES);
|
||||
Map<String, String> details = new HashMap<>();
|
||||
details.put(VmDetailConstants.ROOT_DISK_SIZE, "16");
|
||||
UserVmDetailVO vmRootDiskSizeDetail = Mockito.mock(UserVmDetailVO.class);
|
||||
Mockito.when(vmRootDiskSizeDetail.getValue()).thenReturn("20");
|
||||
Mockito.when(userVmDetailsDao.findDetail(1L, VmDetailConstants.ROOT_DISK_SIZE)).thenReturn(vmRootDiskSizeDetail);
|
||||
Long actualSize = userVmManagerImpl.getRootVolumeSizeForVmRestore(null, template, userVm, diskOffering, details, false);
|
||||
Assert.assertEquals(16 * GiB_TO_BYTES, actualSize.longValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRootVolumeSizeForVmRestoreNullDiskOfferingAndEmptyDetails() {
|
||||
VMTemplateVO template = Mockito.mock(VMTemplateVO.class);
|
||||
Mockito.when(template.getSize()).thenReturn(10L * GiB_TO_BYTES);
|
||||
UserVmVO userVm = Mockito.mock(UserVmVO.class);
|
||||
Mockito.when(userVm.getId()).thenReturn(1L);
|
||||
DiskOffering diskOffering = null;
|
||||
Map<String, String> details = new HashMap<>();
|
||||
UserVmDetailVO vmRootDiskSizeDetail = Mockito.mock(UserVmDetailVO.class);
|
||||
Mockito.when(vmRootDiskSizeDetail.getValue()).thenReturn("20");
|
||||
Mockito.when(userVmDetailsDao.findDetail(1L, VmDetailConstants.ROOT_DISK_SIZE)).thenReturn(vmRootDiskSizeDetail);
|
||||
Long actualSize = userVmManagerImpl.getRootVolumeSizeForVmRestore(null, template, userVm, diskOffering, details, false);
|
||||
Assert.assertEquals(20 * GiB_TO_BYTES, actualSize.longValue());
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -278,6 +278,12 @@ public class MockResourceLimitManagerImpl extends ManagerBase implements Resourc
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkPrimaryStorageResourceLimit(Account owner, Boolean display, Long size,
|
||||
DiskOffering diskOffering) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrementVolumeResourceCount(long accountId, Boolean display, Long size, DiskOffering diskOffering) {
|
||||
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ from marvin.lib.base import (Host,
|
|||
Domain,
|
||||
Zone,
|
||||
ServiceOffering,
|
||||
Template,
|
||||
DiskOffering,
|
||||
VirtualMachine,
|
||||
Volume,
|
||||
|
|
@ -56,6 +57,7 @@ class TestResourceLimitTags(cloudstackTestCase):
|
|||
def setUpClass(cls):
|
||||
testClient = super(TestResourceLimitTags, cls).getClsTestClient()
|
||||
cls.apiclient = testClient.getApiClient()
|
||||
cls.hypervisor = testClient.getHypervisorInfo()
|
||||
cls.services = testClient.getParsedTestDataConfig()
|
||||
|
||||
# Get Zone, Domain and templates
|
||||
|
|
@ -646,3 +648,45 @@ class TestResourceLimitTags(cloudstackTestCase):
|
|||
expected_usage_total = 2 * expected_usage_total
|
||||
self.assertTrue(usage.total == expected_usage_total, "Usage for %s with tag %s is not matching for target account" % (usage.resourcetypename, usage.tag))
|
||||
return
|
||||
|
||||
@attr(tags=["devcloud", "advanced", "advancedns", "smoke", "basic", "sg"], required_hardware="false")
|
||||
def test_13_verify_restore_vm_limit(self):
|
||||
"""Test to verify limits are updated on restoring VM
|
||||
"""
|
||||
hypervisor = self.hypervisor.lower()
|
||||
restore_template_service = self.services["test_templates"][
|
||||
hypervisor if hypervisor != 'simulator' else 'xenserver'].copy()
|
||||
restore_template = Template.register(self.apiclient, restore_template_service, zoneid=self.zone.id, hypervisor=hypervisor, templatetag=self.host_tags[1])
|
||||
restore_template.download(self.apiclient)
|
||||
self.cleanup.append(restore_template)
|
||||
|
||||
self.vm = VirtualMachine.create(
|
||||
self.userapiclient,
|
||||
self.services["virtual_machine"],
|
||||
templateid=restore_template.id,
|
||||
serviceofferingid=self.host_storage_tagged_compute_offering.id,
|
||||
mode=self.services["mode"]
|
||||
)
|
||||
self.cleanup.append(self.vm)
|
||||
old_root_vol = Volume.list(self.userapiclient, virtualmachineid=self.vm.id)[0]
|
||||
|
||||
acc = Account.list(
|
||||
self.userapiclient,
|
||||
id=self.account.id
|
||||
)[0]
|
||||
tags = [self.host_storage_tagged_compute_offering.hosttags, self.host_storage_tagged_compute_offering.storagetags]
|
||||
account_usage_before = list(filter(lambda x: x.tag in tags, acc['taggedresources']))
|
||||
|
||||
self.vm.restore(self.userapiclient, restore_template.id, rootdisksize=16, expunge=True)
|
||||
acc = Account.list(
|
||||
self.userapiclient,
|
||||
id=self.account.id
|
||||
)[0]
|
||||
|
||||
account_usage_after = list(filter(lambda x: x.tag in tags, acc['taggedresources']))
|
||||
for idx, usage in enumerate(account_usage_after):
|
||||
expected_usage_total = account_usage_before[idx].total
|
||||
if usage.resourcetype in [10]:
|
||||
expected_usage_total = expected_usage_total - old_root_vol.size + 16 * 1024 * 1024 * 1024
|
||||
self.assertTrue(usage.total == expected_usage_total, "Usage for %s with tag %s is not matching for target account" % (usage.resourcetypename, usage.tag))
|
||||
return
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@
|
|||
"""
|
||||
# Import Local Modules
|
||||
from marvin.cloudstackTestCase import cloudstackTestCase
|
||||
from marvin.lib.base import (VirtualMachine, Volume, ServiceOffering, Template)
|
||||
from marvin.lib.base import (VirtualMachine, Volume, DiskOffering, ServiceOffering, Template)
|
||||
from marvin.lib.common import (get_zone, get_domain)
|
||||
from nose.plugins.attrib import attr
|
||||
|
||||
|
|
@ -45,16 +45,19 @@ class TestRestoreVM(cloudstackTestCase):
|
|||
cls.service_offering = ServiceOffering.create(cls.apiclient, cls.services["service_offering"])
|
||||
cls._cleanup.append(cls.service_offering)
|
||||
|
||||
cls.disk_offering = DiskOffering.create(cls.apiclient, cls.services["disk_offering"], disksize='8')
|
||||
cls._cleanup.append(cls.disk_offering)
|
||||
|
||||
template_t1 = Template.register(cls.apiclient, cls.services["test_templates"][
|
||||
cls.hypervisor.lower() if cls.hypervisor.lower() != 'simulator' else 'xenserver'],
|
||||
zoneid=cls.zone.id, hypervisor=cls.hypervisor.lower())
|
||||
zoneid=cls.zone.id, hypervisor=cls.hypervisor.lower())
|
||||
cls._cleanup.append(template_t1)
|
||||
template_t1.download(cls.apiclient)
|
||||
cls.template_t1 = Template.list(cls.apiclient, templatefilter='all', id=template_t1.id)[0]
|
||||
|
||||
template_t2 = Template.register(cls.apiclient, cls.services["test_templates"][
|
||||
cls.hypervisor.lower() if cls.hypervisor.lower() != 'simulator' else 'xenserver'],
|
||||
zoneid=cls.zone.id, hypervisor=cls.hypervisor.lower())
|
||||
zoneid=cls.zone.id, hypervisor=cls.hypervisor.lower())
|
||||
cls._cleanup.append(template_t2)
|
||||
template_t2.download(cls.apiclient)
|
||||
cls.template_t2 = Template.list(cls.apiclient, templatefilter='all', id=template_t2.id)[0]
|
||||
|
|
@ -74,20 +77,83 @@ class TestRestoreVM(cloudstackTestCase):
|
|||
serviceofferingid=self.service_offering.id)
|
||||
self._cleanup.append(virtual_machine)
|
||||
|
||||
root_vol = Volume.list(self.apiclient, virtualmachineid=virtual_machine.id)[0]
|
||||
self.assertEqual(root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(root_vol.size, self.template_t1.size, "Size of volume and template should match")
|
||||
old_root_vol = Volume.list(self.apiclient, virtualmachineid=virtual_machine.id)[0]
|
||||
self.assertEqual(old_root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(old_root_vol.size, self.template_t1.size, "Size of volume and template should match")
|
||||
|
||||
virtual_machine.restore(self.apiclient, self.template_t2.id, expunge=True)
|
||||
|
||||
virtual_machine.restore(self.apiclient, self.template_t2.id)
|
||||
restored_vm = VirtualMachine.list(self.apiclient, id=virtual_machine.id)[0]
|
||||
self.assertEqual(restored_vm.state, 'Running', "VM should be in a running state")
|
||||
self.assertEqual(restored_vm.templateid, self.template_t2.id, "VM's template after restore is incorrect")
|
||||
|
||||
root_vol = Volume.list(self.apiclient, virtualmachineid=restored_vm.id)[0]
|
||||
self.assertEqual(root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(root_vol.size, self.template_t2.size, "Size of volume and template should match")
|
||||
|
||||
old_root_vol = Volume.list(self.apiclient, id=old_root_vol.id)
|
||||
self.assertEqual(old_root_vol, None, "Old volume should be deleted")
|
||||
|
||||
@attr(tags=["advanced", "basic"], required_hardware="false")
|
||||
def test_02_restore_vm_allocated_root(self):
|
||||
def test_02_restore_vm_with_disk_offering(self):
|
||||
"""Test restore virtual machine
|
||||
"""
|
||||
# create a virtual machine
|
||||
virtual_machine = VirtualMachine.create(self.apiclient, self.services["virtual_machine"], zoneid=self.zone.id,
|
||||
templateid=self.template_t1.id,
|
||||
serviceofferingid=self.service_offering.id)
|
||||
self._cleanup.append(virtual_machine)
|
||||
|
||||
old_root_vol = Volume.list(self.apiclient, virtualmachineid=virtual_machine.id)[0]
|
||||
self.assertEqual(old_root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(old_root_vol.size, self.template_t1.size, "Size of volume and template should match")
|
||||
|
||||
virtual_machine.restore(self.apiclient, self.template_t2.id, self.disk_offering.id, expunge=True)
|
||||
|
||||
restored_vm = VirtualMachine.list(self.apiclient, id=virtual_machine.id)[0]
|
||||
self.assertEqual(restored_vm.state, 'Running', "VM should be in a running state")
|
||||
self.assertEqual(restored_vm.templateid, self.template_t2.id, "VM's template after restore is incorrect")
|
||||
|
||||
root_vol = Volume.list(self.apiclient, virtualmachineid=restored_vm.id)[0]
|
||||
self.assertEqual(root_vol.diskofferingid, self.disk_offering.id, "Disk offering id should match")
|
||||
self.assertEqual(root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(root_vol.size, self.disk_offering.disksize * 1024 * 1024 * 1024,
|
||||
"Size of volume and disk offering should match")
|
||||
|
||||
old_root_vol = Volume.list(self.apiclient, id=old_root_vol.id)
|
||||
self.assertEqual(old_root_vol, None, "Old volume should be deleted")
|
||||
|
||||
@attr(tags=["advanced", "basic"], required_hardware="false")
|
||||
def test_03_restore_vm_with_disk_offering_custom_size(self):
|
||||
"""Test restore virtual machine
|
||||
"""
|
||||
# create a virtual machine
|
||||
virtual_machine = VirtualMachine.create(self.apiclient, self.services["virtual_machine"], zoneid=self.zone.id,
|
||||
templateid=self.template_t1.id,
|
||||
serviceofferingid=self.service_offering.id)
|
||||
self._cleanup.append(virtual_machine)
|
||||
|
||||
old_root_vol = Volume.list(self.apiclient, virtualmachineid=virtual_machine.id)[0]
|
||||
self.assertEqual(old_root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(old_root_vol.size, self.template_t1.size, "Size of volume and template should match")
|
||||
|
||||
virtual_machine.restore(self.apiclient, self.template_t2.id, self.disk_offering.id, rootdisksize=16)
|
||||
|
||||
restored_vm = VirtualMachine.list(self.apiclient, id=virtual_machine.id)[0]
|
||||
self.assertEqual(restored_vm.state, 'Running', "VM should be in a running state")
|
||||
self.assertEqual(restored_vm.templateid, self.template_t2.id, "VM's template after restore is incorrect")
|
||||
|
||||
root_vol = Volume.list(self.apiclient, virtualmachineid=restored_vm.id)[0]
|
||||
self.assertEqual(root_vol.diskofferingid, self.disk_offering.id, "Disk offering id should match")
|
||||
self.assertEqual(root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
self.assertEqual(root_vol.size, 16 * 1024 * 1024 * 1024, "Size of volume and custom disk size should match")
|
||||
|
||||
old_root_vol = Volume.list(self.apiclient, id=old_root_vol.id)[0]
|
||||
self.assertEqual(old_root_vol.state, "Destroy", "Old volume should be in Destroy state")
|
||||
Volume.delete(old_root_vol, self.apiclient)
|
||||
|
||||
@attr(tags=["advanced", "basic"], required_hardware="false")
|
||||
def test_04_restore_vm_allocated_root(self):
|
||||
"""Test restore virtual machine with root disk in allocated state
|
||||
"""
|
||||
# create a virtual machine with allocated root disk by setting startvm=False
|
||||
|
|
@ -96,9 +162,9 @@ class TestRestoreVM(cloudstackTestCase):
|
|||
serviceofferingid=self.service_offering.id,
|
||||
startvm=False)
|
||||
self._cleanup.append(virtual_machine)
|
||||
root_vol = Volume.list(self.apiclient, virtualmachineid=virtual_machine.id)[0]
|
||||
self.assertEqual(root_vol.state, 'Allocated', "Volume should be in Allocated state")
|
||||
self.assertEqual(root_vol.size, self.template_t1.size, "Size of volume and template should match")
|
||||
old_root_vol = Volume.list(self.apiclient, virtualmachineid=virtual_machine.id)[0]
|
||||
self.assertEqual(old_root_vol.state, 'Allocated', "Volume should be in Allocated state")
|
||||
self.assertEqual(old_root_vol.size, self.template_t1.size, "Size of volume and template should match")
|
||||
|
||||
virtual_machine.restore(self.apiclient, self.template_t2.id)
|
||||
restored_vm = VirtualMachine.list(self.apiclient, id=virtual_machine.id)[0]
|
||||
|
|
@ -112,3 +178,6 @@ class TestRestoreVM(cloudstackTestCase):
|
|||
virtual_machine.start(self.apiclient)
|
||||
root_vol = Volume.list(self.apiclient, virtualmachineid=restored_vm.id)[0]
|
||||
self.assertEqual(root_vol.state, 'Ready', "Volume should be in Ready state")
|
||||
|
||||
old_root_vol = Volume.list(self.apiclient, id=old_root_vol.id)
|
||||
self.assertEqual(old_root_vol, None, "Old volume should be deleted")
|
||||
|
|
|
|||
|
|
@ -776,12 +776,25 @@ class VirtualMachine:
|
|||
if response[0] == FAIL:
|
||||
raise Exception(response[1])
|
||||
|
||||
def restore(self, apiclient, templateid=None):
|
||||
def restore(self, apiclient, templateid=None, diskofferingid=None, rootdisksize=None, expunge=None, details=None):
|
||||
"""Restore the instance"""
|
||||
cmd = restoreVirtualMachine.restoreVirtualMachineCmd()
|
||||
cmd.virtualmachineid = self.id
|
||||
if templateid:
|
||||
cmd.templateid = templateid
|
||||
if diskofferingid:
|
||||
cmd.diskofferingid = diskofferingid
|
||||
if rootdisksize:
|
||||
cmd.rootdisksize = rootdisksize
|
||||
if expunge is not None:
|
||||
cmd.expunge = expunge
|
||||
if details:
|
||||
for key, value in list(details.items()):
|
||||
cmd.details.append({
|
||||
'key': key,
|
||||
'value': value
|
||||
})
|
||||
|
||||
return apiclient.restoreVirtualMachine(cmd)
|
||||
|
||||
def get_ssh_client(
|
||||
|
|
@ -1457,7 +1470,7 @@ class Template:
|
|||
@classmethod
|
||||
def register(cls, apiclient, services, zoneid=None,
|
||||
account=None, domainid=None, hypervisor=None,
|
||||
projectid=None, details=None, randomize_name=True):
|
||||
projectid=None, details=None, randomize_name=True, templatetag=None):
|
||||
"""Create template from URL"""
|
||||
|
||||
# Create template from Virtual machine and Volume ID
|
||||
|
|
@ -1522,6 +1535,9 @@ class Template:
|
|||
if details:
|
||||
cmd.details = details
|
||||
|
||||
if templatetag:
|
||||
cmd.templatetag = templatetag
|
||||
|
||||
if "directdownload" in services:
|
||||
cmd.directdownload = services["directdownload"]
|
||||
if "checksum" in services:
|
||||
|
|
|
|||
Loading…
Reference in New Issue