You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cloudstack.apache.org by ed...@apache.org on 2012/07/26 19:15:28 UTC

git commit: ifix CS-15609 Volumes can be created as a part of VM creation when un-allocated space is insufficient on primary storage

Updated Branches:
  refs/heads/master 58a9b94df -> 50ffa95f6


ifix CS-15609 Volumes can be created as a part of VM creation when
un-allocated space is insufficient on primary storage

check the availability of un-allocated primary storage space during
planning stage, for multiple-volume VM creation scenario
modification in StorageManagerImpl.java and StorageManager.java:
add a new method storagePoolHasEnoughSpace(List<Volumes>, StoragePool),
check if storagePool has enough space for all requested volumes
modification in FirstfitPlanner.findPotentialDeploymentResources:
handle multiple volume case, keep track of allocated volumes for pools
and call storagePoolHasEnoughSpace to check space availability
modification in AbstractStoragePoolAllocator.java:
extract capacity computation logic and make a new method in
StorageManagerImpl

RB: https://reviews.apache.org/r/6028/
Send-by: mice_xia@tcloudcomputing.com


Project: http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/commit/50ffa95f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/tree/50ffa95f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/diff/50ffa95f

Branch: refs/heads/master
Commit: 50ffa95f63c100271e40fa11add3e440545db918
Parents: 58a9b94
Author: Edison Su <su...@gmail.com>
Authored: Thu Jul 26 10:10:18 2012 -0700
Committer: Edison Su <su...@gmail.com>
Committed: Thu Jul 26 10:10:18 2012 -0700

----------------------------------------------------------------------
 server/src/com/cloud/deploy/FirstFitPlanner.java   |   43 ++++++-
 server/src/com/cloud/storage/StorageManager.java   |    4 +-
 .../src/com/cloud/storage/StorageManagerImpl.java  |   96 +++++++++++++++
 .../allocator/AbstractStoragePoolAllocator.java    |   77 ++-----------
 4 files changed, 146 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/blob/50ffa95f/server/src/com/cloud/deploy/FirstFitPlanner.java
----------------------------------------------------------------------
diff --git a/server/src/com/cloud/deploy/FirstFitPlanner.java b/server/src/com/cloud/deploy/FirstFitPlanner.java
index eb82c75..e70ea4d 100755
--- a/server/src/com/cloud/deploy/FirstFitPlanner.java
+++ b/server/src/com/cloud/deploy/FirstFitPlanner.java
@@ -17,10 +17,12 @@
 package com.cloud.deploy;
 
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeSet;
 
 import javax.ejb.Local;
 import javax.naming.ConfigurationException;
@@ -55,6 +57,7 @@ import com.cloud.org.Cluster;
 import com.cloud.org.Grouping;
 import com.cloud.resource.ResourceState;
 import com.cloud.storage.DiskOfferingVO;
+import com.cloud.storage.StorageManager;
 import com.cloud.storage.StoragePool;
 import com.cloud.storage.StoragePoolHostVO;
 import com.cloud.storage.StoragePoolVO;
@@ -98,6 +101,7 @@ public class FirstFitPlanner extends PlannerBase implements DeploymentPlanner {
     @Inject protected StoragePoolDao _storagePoolDao;
     @Inject protected CapacityDao _capacityDao;
     @Inject protected AccountManager _accountMgr;
+    @Inject protected StorageManager _storageMgr;
 
     @Inject(adapter=StoragePoolAllocator.class)
     protected Adapters<StoragePoolAllocator> _storagePoolAllocators;
@@ -638,25 +642,56 @@ public class FirstFitPlanner extends PlannerBase implements DeploymentPlanner {
         s_logger.debug("Trying to find a potenial host and associated storage pools from the suitable host/pool lists for this VM");
 
         boolean hostCanAccessPool = false;
+        boolean haveEnoughSpace = false;
         Map<Volume, StoragePool> storage = new HashMap<Volume, StoragePool>();
+        TreeSet<Volume> volumesOrderBySizeDesc = new TreeSet<Volume>(new Comparator<Volume>() {
+            @Override
+            public int compare(Volume v1, Volume v2) {
+                if(v1.getSize() < v2.getSize())
+                    return 1;
+                else 
+                    return -1;
+            }
+        });
+        volumesOrderBySizeDesc.addAll(suitableVolumeStoragePools.keySet());
+        boolean multipleVolume = volumesOrderBySizeDesc.size() > 1;
         for(Host potentialHost : suitableHosts){
-            for(Volume vol : suitableVolumeStoragePools.keySet()){
+            Map<StoragePool,List<Volume>> volumeAllocationMap = new HashMap<StoragePool,List<Volume>>();
+            for(Volume vol : volumesOrderBySizeDesc){
+                haveEnoughSpace = false;
                 s_logger.debug("Checking if host: "+potentialHost.getId() +" can access any suitable storage pool for volume: "+ vol.getVolumeType());
                 List<StoragePool> volumePoolList = suitableVolumeStoragePools.get(vol);
                 hostCanAccessPool = false;
                 for(StoragePool potentialSPool : volumePoolList){
                     if(hostCanAccessSPool(potentialHost, potentialSPool)){
-                        storage.put(vol, potentialSPool);
                         hostCanAccessPool = true;
+                        if(multipleVolume){
+                            List<Volume> requestVolumes  = null;
+                            if(volumeAllocationMap.containsKey(potentialSPool))
+                                requestVolumes = volumeAllocationMap.get(potentialSPool);
+                            else
+                                requestVolumes = new ArrayList<Volume>();
+                            requestVolumes.add(vol);
+
+                            if(!_storageMgr.storagePoolHasEnoughSpace(requestVolumes, potentialSPool))
+                                continue;
+                            volumeAllocationMap.put(potentialSPool,requestVolumes);
+                        }
+                        storage.put(vol, potentialSPool);
+                        haveEnoughSpace = true;
                         break;
                     }
                 }
                 if(!hostCanAccessPool){
                     break;
                 }
+                if(!haveEnoughSpace) {
+                    s_logger.warn("insufficient capacity to allocate all volumes");
+                    break;
+                }
             }
-            if(hostCanAccessPool){
-                s_logger.debug("Found a potential host " + "id: "+potentialHost.getId() + " name: " +potentialHost.getName()+ " and associated storage pools for this VM");
+            if(hostCanAccessPool && haveEnoughSpace){
+                s_logger.debug("Found a potential host " + "id: "+potentialHost.getId() + " name: " +potentialHost.getName() + " and associated storage pools for this VM");
                 return new Pair<Host, Map<Volume, StoragePool>>(potentialHost, storage);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/blob/50ffa95f/server/src/com/cloud/storage/StorageManager.java
----------------------------------------------------------------------
diff --git a/server/src/com/cloud/storage/StorageManager.java b/server/src/com/cloud/storage/StorageManager.java
index 67ad97c..59a0221 100755
--- a/server/src/com/cloud/storage/StorageManager.java
+++ b/server/src/com/cloud/storage/StorageManager.java
@@ -235,4 +235,6 @@ public interface StorageManager extends StorageService, Manager {
 	String getSupportedImageFormatForCluster(Long clusterId);
 
 	HypervisorType getHypervisorTypeFromFormat(ImageFormat format);
-}
+
+        boolean storagePoolHasEnoughSpace(List<Volume> volume, StoragePool pool);
+}

http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/blob/50ffa95f/server/src/com/cloud/storage/StorageManagerImpl.java
----------------------------------------------------------------------
diff --git a/server/src/com/cloud/storage/StorageManagerImpl.java b/server/src/com/cloud/storage/StorageManagerImpl.java
index d94bada..ed9df30 100755
--- a/server/src/com/cloud/storage/StorageManagerImpl.java
+++ b/server/src/com/cloud/storage/StorageManagerImpl.java
@@ -80,6 +80,7 @@ import com.cloud.api.commands.UpdateStoragePoolCmd;
 import com.cloud.api.commands.UploadVolumeCmd;
 import com.cloud.async.AsyncJobManager;
 import com.cloud.capacity.Capacity;
+import com.cloud.capacity.CapacityManager;
 import com.cloud.capacity.CapacityState;
 import com.cloud.capacity.CapacityVO;
 import com.cloud.capacity.dao.CapacityDao;
@@ -133,6 +134,7 @@ import com.cloud.resource.ResourceManager;
 import com.cloud.resource.ResourceState;
 import com.cloud.server.ManagementServer;
 import com.cloud.server.ResourceTag.TaggedResourceType;
+import com.cloud.server.StatsCollector;
 import com.cloud.service.ServiceOfferingVO;
 import com.cloud.service.dao.ServiceOfferingDao;
 import com.cloud.storage.Storage.ImageFormat;
@@ -274,6 +276,8 @@ public class StorageManagerImpl implements StorageManager, Manager, ClusterManag
     @Inject
     protected CapacityDao _capacityDao;
     @Inject
+    protected CapacityManager _capacityMgr;
+    @Inject
     protected DiskOfferingDao _diskOfferingDao;
     @Inject
     protected AccountDao _accountDao;
@@ -352,6 +356,9 @@ public class StorageManagerImpl implements StorageManager, Manager, ClusterManag
     private StateMachine2<Volume.State, Volume.Event, Volume> _volStateMachine;
     private int _customDiskOfferingMinSize = 1;
     private int _customDiskOfferingMaxSize = 1024;
+    private double _storageUsedThreshold = 1.0d;
+    private double _storageAllocatedThreshold = 1.0d;
+    protected BigDecimal _storageOverprovisioningFactor = new BigDecimal(1);
 
     public boolean share(VMInstanceVO vm, List<VolumeVO> vols, HostVO host, boolean cancelPreviousShare) throws StorageUnavailableException {
 
@@ -955,6 +962,19 @@ public class StorageManagerImpl implements StorageManager, Manager, ClusterManag
         String time = configs.get("storage.cleanup.interval");
         _storageCleanupInterval = NumbersUtil.parseInt(time, 86400);
 
+        String storageUsedThreshold = configDao.getValue(Config.StorageCapacityDisableThreshold.key());
+        if (storageUsedThreshold != null) {
+            _storageUsedThreshold = Double.parseDouble(storageUsedThreshold);
+        }
+
+        String storageAllocatedThreshold = configDao.getValue(Config.StorageAllocatedCapacityDisableThreshold.key());
+        if (storageAllocatedThreshold != null) {
+            _storageAllocatedThreshold = Double.parseDouble(storageAllocatedThreshold);
+        }
+
+        String globalStorageOverprovisioningFactor = configs.get("storage.overprovisioning.factor");
+        _storageOverprovisioningFactor = new BigDecimal(NumbersUtil.parseFloat(globalStorageOverprovisioningFactor, 2.0f));
+
         s_logger.info("Storage cleanup enabled: " + _storageCleanupEnabled + ", interval: " + _storageCleanupInterval + ", template cleanup enabled: " + _templateCleanupEnabled);
 
         String workers = configs.get("expunge.workers");
@@ -3906,5 +3926,81 @@ public class StorageManagerImpl implements StorageManager, Manager, ClusterManag
             return HypervisorType.None;
         }
     }
+
+    private boolean checkUsagedSpace(StoragePool pool){
+        StatsCollector sc = StatsCollector.getInstance();
+        if (sc != null) {
+            long totalSize = pool.getCapacityBytes();
+            StorageStats stats = sc.getStoragePoolStats(pool.getId());
+            if(stats == null){
+                stats = sc.getStorageStats(pool.getId());
+            }
+            if (stats != null) {
+                double usedPercentage = ((double)stats.getByteUsed() / (double)totalSize);
+                if (s_logger.isDebugEnabled()) {
+                    s_logger.debug("Checking pool " + pool.getId() + " for storage, totalSize: " + pool.getCapacityBytes() + ", usedBytes: " + stats.getByteUsed() + ", usedPct: " + usedPercentage + ", disable threshold: " + _storageUsedThreshold);
+                }
+                if (usedPercentage >= _storageUsedThreshold) {
+                    if (s_logger.isDebugEnabled()) {
+                        s_logger.debug("Insufficient space on pool: " + pool.getId() + " since its usage percentage: " +usedPercentage + " has crossed the pool.storage.capacity.disablethreshold: " + _storageUsedThreshold);
+                    }
+                    return false;
+                }
+            }
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean storagePoolHasEnoughSpace(List<Volume> volumes, StoragePool pool) {
+        if(volumes == null || volumes.isEmpty())
+            return false;
+
+        if(!checkUsagedSpace(pool))
+            return false;
+
+        // allocated space includes template of specified volume
+        StoragePoolVO poolVO = _storagePoolDao.findById(pool.getId());
+        long allocatedSizeWithtemplate = _capacityMgr.getAllocatedPoolCapacity(poolVO, null);
+        long totalAskingSize = 0;
+        for (Volume volume : volumes) {
+            if(volume.getTemplateId()!=null){
+                VMTemplateVO tmpl = _templateDao.findById(volume.getTemplateId());
+                if (tmpl.getFormat() != ImageFormat.ISO){
+                    allocatedSizeWithtemplate = _capacityMgr.getAllocatedPoolCapacity(poolVO, tmpl);
+                }
+            }
+            if(volume.getState() != Volume.State.Ready)
+                totalAskingSize = totalAskingSize + volume.getSize();
+        }
+
+        long totalOverProvCapacity;
+        if (pool.getPoolType() == StoragePoolType.NetworkFilesystem) {
+            totalOverProvCapacity = _storageOverprovisioningFactor.multiply(new BigDecimal(pool.getCapacityBytes())).longValue();// All this for the inaccuracy of floats for big number multiplication.
+        }else {
+            totalOverProvCapacity = pool.getCapacityBytes();
+        }
+
+        if (s_logger.isDebugEnabled()) {
+            s_logger.debug("Checking pool: " + pool.getId() + " for volume allocation " + volumes.toString() + ", maxSize : " + totalOverProvCapacity + ", totalAllocatedSize : " + allocatedSizeWithtemplate + ", askingSize : " + totalAskingSize + ", allocated disable threshold: " + _storageAllocatedThreshold);
+        }
+
+        double usedPercentage = (allocatedSizeWithtemplate + totalAskingSize) / (double)(totalOverProvCapacity);
+        if (usedPercentage > _storageAllocatedThreshold){
+            if (s_logger.isDebugEnabled()) {
+                s_logger.debug("Insufficient un-allocated capacity on: " + pool.getId() + " for volume allocation: " + volumes.toString() + " since its allocated percentage: " +usedPercentage + " has crossed the allocated pool.storage.allocated.capacity.disablethreshold: " + _storageAllocatedThreshold + ", skipping this pool");
+            }
+            return false;
+        }
+
+        if (totalOverProvCapacity < (allocatedSizeWithtemplate + totalAskingSize)) {
+            if (s_logger.isDebugEnabled()) {
+                s_logger.debug("Insufficient un-allocated capacity on: " + pool.getId() + " for volume allocation: " + volumes.toString() + ", not enough storage, maxSize : " + totalOverProvCapacity + ", totalAllocatedSize : " + allocatedSizeWithtemplate + ", askingSize : " + totalAskingSize);
+            }
+            return false;
+        }
+        return true;
+    }
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-cloudstack/blob/50ffa95f/server/src/com/cloud/storage/allocator/AbstractStoragePoolAllocator.java
----------------------------------------------------------------------
diff --git a/server/src/com/cloud/storage/allocator/AbstractStoragePoolAllocator.java b/server/src/com/cloud/storage/allocator/AbstractStoragePoolAllocator.java
index 0a0f66e..87cb065 100755
--- a/server/src/com/cloud/storage/allocator/AbstractStoragePoolAllocator.java
+++ b/server/src/com/cloud/storage/allocator/AbstractStoragePoolAllocator.java
@@ -17,6 +17,7 @@
 package com.cloud.storage.allocator;
 
 import java.math.BigDecimal;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -27,7 +28,6 @@ import javax.naming.ConfigurationException;
 import org.apache.log4j.Logger;
 
 import com.cloud.capacity.CapacityManager;
-import com.cloud.configuration.Config;
 import com.cloud.configuration.dao.ConfigurationDao;
 import com.cloud.dc.ClusterVO;
 import com.cloud.dc.dao.ClusterDao;
@@ -41,13 +41,11 @@ import com.cloud.storage.StorageManager;
 import com.cloud.storage.StoragePool;
 import com.cloud.storage.StoragePoolStatus;
 import com.cloud.storage.StoragePoolVO;
-import com.cloud.storage.StorageStats;
-import com.cloud.storage.VMTemplateHostVO;
 import com.cloud.storage.VMTemplateStoragePoolVO;
 import com.cloud.storage.VMTemplateStorageResourceAssoc;
 import com.cloud.storage.VMTemplateStorageResourceAssoc.Status;
-import com.cloud.storage.VMTemplateSwiftVO;
 import com.cloud.storage.VMTemplateVO;
+import com.cloud.storage.Volume;
 import com.cloud.storage.Volume.Type;
 import com.cloud.storage.dao.StoragePoolDao;
 import com.cloud.storage.dao.StoragePoolHostDao;
@@ -58,7 +56,6 @@ import com.cloud.storage.dao.VolumeDao;
 import com.cloud.storage.swift.SwiftManager;
 import com.cloud.template.TemplateManager;
 import com.cloud.utils.NumbersUtil;
-import com.cloud.utils.Pair;
 import com.cloud.utils.component.AdapterBase;
 import com.cloud.utils.component.Inject;
 import com.cloud.vm.DiskProfile;
@@ -83,8 +80,6 @@ public abstract class AbstractStoragePoolAllocator extends AdapterBase implement
     long _extraBytesPerVolume = 0;
     Random _rand;
     boolean _dontMatter;
-    double _storageUsedThreshold = 1.0d;
-    double _storageAllocatedThreshold = 1.0d;
     
     @Override
     public boolean configure(String name, Map<String, Object> params) throws ConfigurationException {
@@ -97,17 +92,6 @@ public abstract class AbstractStoragePoolAllocator extends AdapterBase implement
         
         _extraBytesPerVolume = 0;
         
-        
-        String storageUsedThreshold = _configDao.getValue(Config.StorageCapacityDisableThreshold.key());        
-        if (storageUsedThreshold != null) {
-            _storageUsedThreshold = Double.parseDouble(storageUsedThreshold);
-        }
-
-        String storageAllocatedThreshold = _configDao.getValue(Config.StorageAllocatedCapacityDisableThreshold.key());
-        if (storageAllocatedThreshold != null) {
-            _storageAllocatedThreshold = Double.parseDouble(storageAllocatedThreshold);
-        }
-        
         _rand = new Random(System.currentTimeMillis());
         
         _dontMatter = Boolean.parseBoolean(configs.get("storage.overwrite.provisioning"));
@@ -192,60 +176,15 @@ public abstract class AbstractStoragePoolAllocator extends AdapterBase implement
 			return false;
 		}
 
-		// check the used size against the total size, skip this host if it's greater than the configured
-		// capacity check "storage.capacity.threshold"
-		if (sc != null) {
-			long totalSize = pool.getCapacityBytes();
-			StorageStats stats = sc.getStoragePoolStats(pool.getId());
-			if(stats == null){
-				stats = sc.getStorageStats(pool.getId());
-			}
-			if (stats != null) {
-				double usedPercentage = ((double)stats.getByteUsed() / (double)totalSize);
-				if (s_logger.isDebugEnabled()) {
-					s_logger.debug("Attempting to look for pool " + pool.getId() + " for storage, totalSize: " + pool.getCapacityBytes() + ", usedBytes: " + stats.getByteUsed() + ", usedPct: " + usedPercentage + ", disable threshold: " + _storageUsedThreshold);
-				}
-				if (usedPercentage >= _storageUsedThreshold) {
-					if (s_logger.isDebugEnabled()) {
-						s_logger.debug("Cannot allocate this pool " + pool.getId() + " for storage since its usage percentage: " +usedPercentage + " has crossed the pool.storage.capacity.disablethreshold: " + _storageUsedThreshold + ", skipping this pool");
-					}
-					return false;
-				}
-			}
-		}
-		
-		long totalAllocatedSize  = _capacityMgr.getAllocatedPoolCapacity(pool, null);
-		long askingSize = dskCh.getSize();
-		
-		long totalOverProvCapacity;
-        if (pool.getPoolType() == StoragePoolType.NetworkFilesystem) {
-            totalOverProvCapacity = _storageOverprovisioningFactor.multiply(new BigDecimal(pool.getCapacityBytes())).longValue();// All this for the inaccuracy of floats for big number multiplication.
-        }else {
-            totalOverProvCapacity = pool.getCapacityBytes();
-        }
 
-		if (s_logger.isDebugEnabled()) {
-			s_logger.debug("Attempting to look for pool " + pool.getId() + " for storage, maxSize : " + totalOverProvCapacity + ", totalAllocatedSize : " + totalAllocatedSize + ", askingSize : " + askingSize + ", allocated disable threshold: " + _storageAllocatedThreshold);
-		}
-
-		double usedPercentage = (totalAllocatedSize + askingSize) / (double)(totalOverProvCapacity);
-		if (usedPercentage > _storageAllocatedThreshold){
-			if (s_logger.isDebugEnabled()) {
-				s_logger.debug("Cannot allocate this pool " + pool.getId() + " for storage since its allocated percentage: " +usedPercentage + " has crossed the allocated pool.storage.allocated.capacity.disablethreshold: " + _storageAllocatedThreshold + ", skipping this pool");
-			}
-			return false;
-		}
-
-		if (totalOverProvCapacity < (totalAllocatedSize + askingSize)) {
-			if (s_logger.isDebugEnabled()) {
-				s_logger.debug("Cannot allocate this pool " + pool.getId() + " for storage, not enough storage, maxSize : " + totalOverProvCapacity + ", totalAllocatedSize : " + totalAllocatedSize + ", askingSize : " + askingSize);
-			}
+        // check capacity  
+        Volume volume =  _volumeDao.findById(dskCh.getVolumeId());
+        List<Volume> requestVolumes = new ArrayList<Volume>();
+        requestVolumes.add(volume);
+        return _storageMgr.storagePoolHasEnoughSpace(requestVolumes, pool);
+	}
 
-			return false;
-		}
 
-		return true;
-	}
 	
 	@Override
 	public String chooseStorageIp(VirtualMachine vm, Host host, Host storage) {