cleanup agent manager

This commit is contained in:
Alex Huang 2011-05-05 17:49:06 -07:00
parent 515981a436
commit 117a81acf6
7 changed files with 209 additions and 345 deletions

View File

@ -41,21 +41,18 @@ import com.cloud.hypervisor.Hypervisor.HypervisorType;
import com.cloud.offering.ServiceOffering;
import com.cloud.resource.ServerResource;
import com.cloud.service.ServiceOfferingVO;
import com.cloud.storage.StoragePoolVO;
import com.cloud.storage.VMTemplateVO;
import com.cloud.template.VirtualMachineTemplate;
import com.cloud.user.User;
import com.cloud.uservm.UserVm;
import com.cloud.utils.Pair;
import com.cloud.utils.component.Manager;
import com.cloud.vm.VMInstanceVO;
/**
* AgentManager manages hosts. It directly coordinates between the DAOs and the connections it manages.
*/
public interface AgentManager extends Manager {
public enum OnError {
Revert, Continue, Stop
Continue, Stop
}
/**
@ -186,13 +183,6 @@ public interface AgentManager extends Manager {
String getHostTags(long hostId);
/**
* Find a host based on the type needed, data center to deploy in, pod to deploy in, service offering, template, and list of
* host to avoid.
*/
Host findHost(Host.Type type, DataCenterVO dc, HostPodVO pod, StoragePoolVO sp, ServiceOfferingVO offering, VMTemplateVO template, VMInstanceVO vm, Host currentHost, Set<Host> avoid);
List<PodCluster> listByDataCenter(long dcId);
List<PodCluster> listByPod(long podId);

View File

@ -87,6 +87,7 @@ import com.cloud.api.commands.UpdateHostPasswordCmd;
import com.cloud.capacity.Capacity;
import com.cloud.capacity.CapacityVO;
import com.cloud.capacity.dao.CapacityDao;
import com.cloud.cluster.ManagementServerNode;
import com.cloud.cluster.StackMaid;
import com.cloud.configuration.dao.ConfigurationDao;
import com.cloud.dc.ClusterDetailsDao;
@ -100,17 +101,11 @@ import com.cloud.dc.dao.ClusterDao;
import com.cloud.dc.dao.DataCenterDao;
import com.cloud.dc.dao.DataCenterIpAddressDao;
import com.cloud.dc.dao.HostPodDao;
import com.cloud.dc.dao.VlanDao;
import com.cloud.deploy.DataCenterDeployment;
import com.cloud.deploy.DeployDestination;
import com.cloud.deploy.DeploymentPlanner;
import com.cloud.deploy.DeploymentPlanner.ExcludeList;
import com.cloud.event.dao.EventDao;
import com.cloud.exception.AgentUnavailableException;
import com.cloud.exception.ConnectionException;
import com.cloud.exception.DiscoveredWithErrorException;
import com.cloud.exception.DiscoveryException;
import com.cloud.exception.InsufficientServerCapacityException;
import com.cloud.exception.InvalidParameterValueException;
import com.cloud.exception.OperationTimedoutException;
import com.cloud.exception.PermissionDeniedException;
@ -131,9 +126,7 @@ import com.cloud.host.dao.HostTagsDao;
import com.cloud.hypervisor.Hypervisor;
import com.cloud.hypervisor.Hypervisor.HypervisorType;
import com.cloud.hypervisor.kvm.resource.KvmDummyResourceBase;
import com.cloud.maint.UpgradeManager;
import com.cloud.network.IPAddressVO;
import com.cloud.network.NetworkManager;
import com.cloud.network.dao.IPAddressDao;
import com.cloud.offering.ServiceOffering;
import com.cloud.org.Cluster;
@ -148,20 +141,16 @@ import com.cloud.storage.StorageManager;
import com.cloud.storage.StoragePoolHostVO;
import com.cloud.storage.StoragePoolVO;
import com.cloud.storage.VMTemplateHostVO;
import com.cloud.storage.VMTemplateVO;
import com.cloud.storage.dao.GuestOSCategoryDao;
import com.cloud.storage.dao.StoragePoolDao;
import com.cloud.storage.dao.StoragePoolHostDao;
import com.cloud.storage.dao.VMTemplateDao;
import com.cloud.storage.dao.VMTemplateHostDao;
import com.cloud.storage.dao.VolumeDao;
import com.cloud.storage.resource.DummySecondaryStorageResource;
import com.cloud.template.VirtualMachineTemplate;
import com.cloud.user.Account;
import com.cloud.user.AccountManager;
import com.cloud.user.User;
import com.cloud.user.UserContext;
import com.cloud.user.dao.UserStatisticsDao;
import com.cloud.uservm.UserVm;
import com.cloud.utils.ActionDelegate;
import com.cloud.utils.NumbersUtil;
@ -178,7 +167,6 @@ import com.cloud.utils.db.SearchCriteria;
import com.cloud.utils.db.Transaction;
import com.cloud.utils.exception.CloudRuntimeException;
import com.cloud.utils.net.Ip;
import com.cloud.utils.net.MacAddress;
import com.cloud.utils.net.NetUtils;
import com.cloud.utils.nio.HandlerFactory;
import com.cloud.utils.nio.Link;
@ -187,7 +175,6 @@ import com.cloud.utils.nio.Task;
import com.cloud.vm.VMInstanceVO;
import com.cloud.vm.VirtualMachine.State;
import com.cloud.vm.VirtualMachineManager;
import com.cloud.vm.VirtualMachineProfileImpl;
import com.cloud.vm.dao.VMInstanceDao;
/**
@ -218,12 +205,8 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
@Inject
protected HostDao _hostDao = null;
@Inject
protected UserStatisticsDao _userStatsDao = null;
@Inject
protected DataCenterDao _dcDao = null;
@Inject
protected VlanDao _vlanDao = null;
@Inject
protected DataCenterIpAddressDao _privateIPAddressDao = null;
@Inject
protected IPAddressDao _publicIPAddressDao = null;
@ -236,8 +219,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
@Inject
protected VMInstanceDao _vmDao = null;
@Inject
protected VolumeDao _volDao = null;
@Inject
protected CapacityDao _capacityDao = null;
@Inject
protected ConfigurationDao _configDao = null;
@ -256,9 +237,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
@Inject
protected HostTagsDao _hostTagsDao = null;
@Inject(adapter = DeploymentPlanner.class)
private Adapters<DeploymentPlanner> _planners;
protected Adapters<Discoverer> _discoverers = null;
protected int _port;
@ -267,12 +245,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
@Inject
protected AlertManager _alertMgr = null;
@Inject
protected NetworkManager _networkMgr = null;
@Inject
protected UpgradeManager _upgradeMgr = null;
@Inject
protected StorageManager _storageMgr = null;
@ -291,8 +263,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
protected int _updateWait;
protected int _alertWait;
protected long _nodeId = -1;
protected float _overProvisioningFactor = 1;
protected float _cpuOverProvisioningFactor = 1;
protected Random _rand = new Random(System.currentTimeMillis());
@ -302,8 +272,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
protected ExecutorService _executor;
@Inject
protected VMTemplateDao _tmpltDao;
@Inject
protected VMTemplateHostDao _vmTemplateHostDao;
@ -358,28 +326,15 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
_discoverers = locator.getAdapters(Discoverer.class);
if (_nodeId == -1) {
// FIXME: We really should not do this like this. It should be done
// at config time and is stored as a config variable.
_nodeId = MacAddress.getMacAddress().toLong();
}
_nodeId = ManagementServerNode.getManagementServerId();
_hostDao.markHostsAsDisconnected(_nodeId, Status.Up, Status.Connecting, Status.Updating, Status.Disconnected, Status.Down);
_monitor = new AgentMonitor(_nodeId, _hostDao, _volDao, _vmDao, _dcDao, _podDao, this, _alertMgr, _pingTimeout);
_monitor = new AgentMonitor(_nodeId, _hostDao, _vmDao, _dcDao, _podDao, this, _alertMgr, _pingTimeout);
registerForHostEvents(_monitor, true, true, false);
_executor = new ThreadPoolExecutor(10, 100, 60l, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory("AgentTaskPool"));
String overProvisioningFactorStr = configs.get("storage.overprovisioning.factor");
_overProvisioningFactor = NumbersUtil.parseFloat(overProvisioningFactorStr, 1);
String cpuOverProvisioningFactorStr = configs.get("cpu.overprovisioning.factor");
_cpuOverProvisioningFactor = NumbersUtil.parseFloat(cpuOverProvisioningFactorStr, 1);
if (_cpuOverProvisioningFactor < 1) {
_cpuOverProvisioningFactor = 1;
}
_connection = new NioServer("AgentManager", _port, workers + 10, this);
s_logger.info("Listening on " + _port + " with " + workers + " workers");
@ -502,32 +457,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
return result;
}
@Override
public Host findHost(final Host.Type type, final DataCenterVO dc, final HostPodVO pod, final StoragePoolVO sp, final ServiceOfferingVO offering, final VMTemplateVO template, VMInstanceVO vm,
Host currentHost, final Set<Host> avoid) {
VirtualMachineProfileImpl<VMInstanceVO> vmProfile = new VirtualMachineProfileImpl<VMInstanceVO>(vm, template, offering, null, null);
DeployDestination dest = null;
DataCenterDeployment plan = new DataCenterDeployment(dc.getId(), pod.getId(), sp.getClusterId(), null, null);
ExcludeList avoids = new ExcludeList();
for (Host h : avoid) {
avoids.addHost(h.getId());
}
for (DeploymentPlanner planner : _planners) {
try {
dest = planner.plan(vmProfile, plan, avoids);
if (dest != null) {
return dest.getHost();
}
} catch (InsufficientServerCapacityException e) {
}
}
s_logger.warn("findHost() could not find a non-null host.");
return null;
}
@Override
public List<PodCluster> listByDataCenter(long dcId) {
List<HostPodVO> pods = _podDao.listByDataCenterId(dcId);
@ -1431,7 +1360,7 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
@Override
public Answer send(Long hostId, Command cmd, int timeout) throws AgentUnavailableException, OperationTimedoutException {
Commands cmds = new Commands(OnError.Revert);
Commands cmds = new Commands(OnError.Stop);
cmds.addCommand(cmd);
send(hostId, cmds, timeout);
Answer[] answers = cmds.getAnswers();
@ -1460,8 +1389,7 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
throw new AgentUnavailableException(-1);
}
// assert noDbTxn() :
// "I know, I know. Why are we so strict as to not allow txn across an agent call? ... Why are we so cruel ... Why are we such a dictator .... Too bad... Sorry...but NO AGENT COMMANDS WRAPPED WITHIN DB TRANSACTIONS!";
assert noDbTxn() : "I know, I know. Why are we so strict as to not allow txn across an agent call? ... Why are we so cruel ... Why are we such a dictator .... Too bad... Sorry...but NO AGENT COMMANDS WRAPPED WITHIN DB TRANSACTIONS!";
Command[] cmds = commands.toCommands();
@ -1477,7 +1405,7 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
}
long seq = _hostDao.getNextSequence(hostId);
Request req = new Request(seq, hostId, _nodeId, cmds, commands.stopOnError(), true, commands.revertOnError());
Request req = new Request(seq, hostId, _nodeId, cmds, commands.stopOnError(), true, false);
Answer[] answers = agent.send(req, timeout);
notifyAnswersToMonitors(hostId, seq, answers);
commands.setAnswers(answers);
@ -1538,7 +1466,7 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
return -1;
}
long seq = _hostDao.getNextSequence(hostId);
Request req = new Request(seq, hostId, _nodeId, cmds, commands.stopOnError(), true, commands.revertOnError());
Request req = new Request(seq, hostId, _nodeId, cmds, commands.stopOnError(), true, false);
agent.send(req, listener);
return seq;
}
@ -2519,7 +2447,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
}
s_logger.info("Old " + server.getType().toString() + " host reconnected w/ id =" + id);
}
createCapacityEntry(startup, server);
return server;
}
@ -2843,85 +2770,6 @@ public class AgentManagerImpl implements AgentManager, HandlerFactory, ResourceS
return _hostDao.findById(hostId);
}
// create capacity entries if none exist for this server
private void createCapacityEntry(final StartupCommand startup, HostVO server) {
SearchCriteria<CapacityVO> capacitySC = _capacityDao.createSearchCriteria();
capacitySC.addAnd("hostOrPoolId", SearchCriteria.Op.EQ, server.getId());
capacitySC.addAnd("dataCenterId", SearchCriteria.Op.EQ, server.getDataCenterId());
capacitySC.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
List<CapacityVO> capacities = _capacityDao.search(capacitySC, null);
// remove old entries, we'll recalculate them anyway
if (startup instanceof StartupStorageCommand) {
if ((capacities != null) && !capacities.isEmpty()) {
for (CapacityVO capacity : capacities) {
_capacityDao.remove(capacity.getId());
}
}
}
if (startup instanceof StartupStorageCommand) {
StartupStorageCommand ssCmd = (StartupStorageCommand) startup;
if (ssCmd.getResourceType() == Storage.StorageResourceType.STORAGE_HOST) {
CapacityVO capacity = new CapacityVO(server.getId(), server.getDataCenterId(), server.getPodId(), server.getClusterId(), 0L, (long) (server.getTotalSize() * _overProvisioningFactor),
CapacityVO.CAPACITY_TYPE_STORAGE_ALLOCATED);
_capacityDao.persist(capacity);
}
} else if (startup instanceof StartupRoutingCommand) {
SearchCriteria<CapacityVO> capacityCPU = _capacityDao.createSearchCriteria();
capacityCPU.addAnd("hostOrPoolId", SearchCriteria.Op.EQ, server.getId());
capacityCPU.addAnd("dataCenterId", SearchCriteria.Op.EQ, server.getDataCenterId());
capacityCPU.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
capacityCPU.addAnd("capacityType", SearchCriteria.Op.EQ, CapacityVO.CAPACITY_TYPE_CPU);
List<CapacityVO> capacityVOCpus = _capacityDao.search(capacityCPU, null);
if (capacityVOCpus != null && !capacityVOCpus.isEmpty()) {
CapacityVO CapacityVOCpu = capacityVOCpus.get(0);
long newTotalCpu = (server.getCpus().longValue() * server.getSpeed().longValue());
if ((CapacityVOCpu.getTotalCapacity() <= newTotalCpu) || ((CapacityVOCpu.getUsedCapacity() + CapacityVOCpu.getReservedCapacity()) <= newTotalCpu)) {
CapacityVOCpu.setTotalCapacity(newTotalCpu);
} else if ((CapacityVOCpu.getUsedCapacity() + CapacityVOCpu.getReservedCapacity() > newTotalCpu) && (CapacityVOCpu.getUsedCapacity() < newTotalCpu)) {
CapacityVOCpu.setReservedCapacity(0);
CapacityVOCpu.setTotalCapacity(newTotalCpu);
} else {
s_logger.debug("What? new cpu is :" + newTotalCpu + ", old one is " + CapacityVOCpu.getUsedCapacity() + "," + CapacityVOCpu.getReservedCapacity() + ","
+ CapacityVOCpu.getTotalCapacity());
}
_capacityDao.update(CapacityVOCpu.getId(), CapacityVOCpu);
} else {
CapacityVO capacity = new CapacityVO(server.getId(), server.getDataCenterId(), server.getPodId(), server.getClusterId(), 0L, (server.getCpus().longValue() * server.getSpeed()
.longValue()), CapacityVO.CAPACITY_TYPE_CPU);
_capacityDao.persist(capacity);
}
SearchCriteria<CapacityVO> capacityMem = _capacityDao.createSearchCriteria();
capacityMem.addAnd("hostOrPoolId", SearchCriteria.Op.EQ, server.getId());
capacityMem.addAnd("dataCenterId", SearchCriteria.Op.EQ, server.getDataCenterId());
capacityMem.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
capacityMem.addAnd("capacityType", SearchCriteria.Op.EQ, CapacityVO.CAPACITY_TYPE_MEMORY);
List<CapacityVO> capacityVOMems = _capacityDao.search(capacityMem, null);
if (capacityVOMems != null && !capacityVOMems.isEmpty()) {
CapacityVO CapacityVOMem = capacityVOMems.get(0);
long newTotalMem = server.getTotalMemory();
if (CapacityVOMem.getTotalCapacity() <= newTotalMem || (CapacityVOMem.getUsedCapacity() + CapacityVOMem.getReservedCapacity() <= newTotalMem)) {
CapacityVOMem.setTotalCapacity(newTotalMem);
} else if (CapacityVOMem.getUsedCapacity() + CapacityVOMem.getReservedCapacity() > newTotalMem && CapacityVOMem.getUsedCapacity() < newTotalMem) {
CapacityVOMem.setReservedCapacity(0);
CapacityVOMem.setTotalCapacity(newTotalMem);
} else {
s_logger.debug("What? new cpu is :" + newTotalMem + ", old one is " + CapacityVOMem.getUsedCapacity() + "," + CapacityVOMem.getReservedCapacity() + ","
+ CapacityVOMem.getTotalCapacity());
}
_capacityDao.update(CapacityVOMem.getId(), CapacityVOMem);
} else {
CapacityVO capacity = new CapacityVO(server.getId(), server.getDataCenterId(), server.getPodId(), server.getClusterId(), 0L, server.getTotalMemory(), CapacityVO.CAPACITY_TYPE_MEMORY);
_capacityDao.persist(capacity);
}
}
}
// protected void upgradeAgent(final Link link, final byte[] request, final
// String reason) {
//

View File

@ -38,7 +38,6 @@ import com.cloud.host.HostVO;
import com.cloud.host.Status;
import com.cloud.host.Status.Event;
import com.cloud.host.dao.HostDao;
import com.cloud.storage.dao.VolumeDao;
import com.cloud.utils.db.GlobalLock;
import com.cloud.vm.VMInstanceVO;
import com.cloud.vm.dao.VMInstanceDao;
@ -50,13 +49,12 @@ public class AgentMonitor extends Thread implements Listener {
private boolean _stop;
private final AgentManagerImpl _agentMgr;
private final VMInstanceDao _vmDao;
private final VolumeDao _volDao;
private DataCenterDao _dcDao = null;
private HostPodDao _podDao = null;
private final AlertManager _alertMgr;
private final long _msId;
public AgentMonitor(long msId, HostDao hostDao, VolumeDao volDao, VMInstanceDao vmDao, DataCenterDao dcDao, HostPodDao podDao, AgentManagerImpl agentMgr, AlertManager alertMgr, long pingTimeout) {
public AgentMonitor(long msId, HostDao hostDao, VMInstanceDao vmDao, DataCenterDao dcDao, HostPodDao podDao, AgentManagerImpl agentMgr, AlertManager alertMgr, long pingTimeout) {
super("AgentMonitor");
_msId = msId;
_pingTimeout = pingTimeout;
@ -64,7 +62,6 @@ public class AgentMonitor extends Thread implements Listener {
_agentMgr = agentMgr;
_stop = false;
_vmDao = vmDao;
_volDao = volDao;
_dcDao = dcDao;
_podDao = podDao;
_alertMgr = alertMgr;

View File

@ -37,7 +37,7 @@ public class Commands implements Iterable<Command> {
}
public Commands(Command cmd) {
this(OnError.Revert);
this(OnError.Stop);
addCommand(cmd);
}
@ -109,11 +109,7 @@ public class Commands implements Iterable<Command> {
}
public boolean stopOnError() {
return _handler == OnError.Revert || _handler == OnError.Stop;
}
public boolean revertOnError() {
return _handler == OnError.Revert;
return _handler == OnError.Stop;
}
public Answer[] getAnswers() {

View File

@ -19,9 +19,9 @@ package com.cloud.agent.manager;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.log4j.Logger;
@ -44,7 +44,7 @@ public class DirectAgentAttache extends AgentAttache {
private final static Logger s_logger = Logger.getLogger(DirectAgentAttache.class);
ServerResource _resource;
static ScheduledExecutorService _executor = Executors.newScheduledThreadPool(100, new NamedThreadFactory("DirectAgent"));
static ScheduledExecutorService _executor = new ScheduledThreadPoolExecutor(100, new NamedThreadFactory("DirectAgent"));
List<ScheduledFuture<?>> _futures = new ArrayList<ScheduledFuture<?>>();
AgentManagerImpl _mgr;
long _seq = 0;
@ -78,7 +78,7 @@ public class DirectAgentAttache extends AgentAttache {
if (!(obj instanceof DirectAgentAttache)) {
return false;
}
return super.equals(obj) && _executor == ((DirectAgentAttache)obj)._executor;
return super.equals(obj);
}
@Override

View File

@ -30,13 +30,20 @@ import javax.naming.ConfigurationException;
import org.apache.log4j.Logger;
import com.cloud.agent.AgentManager;
import com.cloud.agent.Listener;
import com.cloud.agent.api.AgentControlAnswer;
import com.cloud.agent.api.AgentControlCommand;
import com.cloud.agent.api.Answer;
import com.cloud.agent.api.Command;
import com.cloud.agent.api.StartupCommand;
import com.cloud.agent.api.StartupRoutingCommand;
import com.cloud.capacity.dao.CapacityDao;
import com.cloud.configuration.Config;
import com.cloud.configuration.dao.ConfigurationDao;
import com.cloud.exception.ConnectionException;
import com.cloud.host.Host;
import com.cloud.host.HostVO;
import com.cloud.host.Status;
import com.cloud.host.dao.HostDao;
import com.cloud.offering.ServiceOffering;
import com.cloud.service.ServiceOfferingVO;
@ -55,16 +62,22 @@ import com.cloud.vm.VirtualMachine.Event;
import com.cloud.vm.VirtualMachine.State;
import com.cloud.vm.dao.VMInstanceDao;
@Local(value=CapacityManager.class)
public class CapacityManagerImpl implements CapacityManager , StateListener<State, VirtualMachine.Event, VirtualMachine>{
@Local(value = CapacityManager.class)
public class CapacityManagerImpl implements CapacityManager, StateListener<State, VirtualMachine.Event, VirtualMachine>, Listener {
private static final Logger s_logger = Logger.getLogger(CapacityManagerImpl.class);
String _name;
@Inject CapacityDao _capacityDao;
@Inject ConfigurationDao _configDao;
@Inject ServiceOfferingDao _offeringsDao;
@Inject HostDao _hostDao;
@Inject VMInstanceDao _vmDao;
@Inject AgentManager _agentManager;
@Inject
CapacityDao _capacityDao;
@Inject
ConfigurationDao _configDao;
@Inject
ServiceOfferingDao _offeringsDao;
@Inject
HostDao _hostDao;
@Inject
VMInstanceDao _vmDao;
@Inject
AgentManager _agentManager;
private int _hostCapacityCheckerDelay;
private int _hostCapacityCheckerInterval;
@ -74,7 +87,6 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
private float _storageOverProvisioningFactor = 1.0f;
private float _cpuOverProvisioningFactor = 1.0f;
@Override
public boolean configure(String name, Map<String, Object> params) throws ConfigurationException {
_name = name;
@ -97,7 +109,7 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
@Override
public boolean start() {
_executor.schedule(new HostCapacityCollector(), _hostCapacityCheckerDelay, TimeUnit.SECONDS);
_executor.schedule(new HostCapacityCollector(), _hostCapacityCheckerDelay, TimeUnit.SECONDS);
return true;
}
@ -141,14 +153,14 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
long actualTotalCpu = capacityCpu.getTotalCapacity();
String opFactor = _configDao.getValue(Config.CPUOverprovisioningFactor.key());
float cpuOverprovisioningFactor = NumbersUtil.parseFloat(opFactor, 1);
long totalCpu = (long)(actualTotalCpu * cpuOverprovisioningFactor);
long totalCpu = (long) (actualTotalCpu * cpuOverprovisioningFactor);
if (s_logger.isDebugEnabled()) {
s_logger.debug("Hosts's actual total CPU: " + actualTotalCpu + " and CPU after applying overprovisioning: " + totalCpu);
}
long totalMem = capacityMemory.getTotalCapacity();
if (!moveFromReserved) {
/*move resource from used*/
/* move resource from used */
if (usedCpu >= vmCPU) {
capacityCpu.setUsedCapacity(usedCpu - vmCPU);
}
@ -158,28 +170,28 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
if (moveToReservered) {
if (reservedCpu + vmCPU <= totalCpu) {
capacityCpu.setReservedCapacity(reservedCpu + vmCPU);
capacityCpu.setReservedCapacity(reservedCpu + vmCPU);
}
if (reservedMem + vmMem <= totalMem) {
capacityMemory.setReservedCapacity(reservedMem + vmMem);
capacityMemory.setReservedCapacity(reservedMem + vmMem);
}
}
} else {
if (reservedCpu >= vmCPU) {
capacityCpu.setReservedCapacity(reservedCpu - vmCPU);
capacityCpu.setReservedCapacity(reservedCpu - vmCPU);
}
if (reservedMem >= vmMem) {
capacityMemory.setReservedCapacity(reservedMem - vmMem);
}
}
s_logger.debug("release cpu from host: " + hostId + ", old used: " + usedCpu + ",reserved: " + reservedCpu + ", actual total: " + actualTotalCpu + ", total with overprovisioning: " + totalCpu +
"; new used: " + capacityCpu.getUsedCapacity() + ",reserved:" + capacityCpu.getReservedCapacity() +
"; movedfromreserved: " + moveFromReserved + ",moveToReservered" + moveToReservered);
s_logger.debug("release cpu from host: " + hostId + ", old used: " + usedCpu + ",reserved: " + reservedCpu + ", actual total: "
+ actualTotalCpu + ", total with overprovisioning: " + totalCpu + "; new used: " + capacityCpu.getUsedCapacity() + ",reserved:"
+ capacityCpu.getReservedCapacity() + "; movedfromreserved: " + moveFromReserved + ",moveToReservered" + moveToReservered);
s_logger.debug("release mem from host: " + hostId + ", old used: " + usedMem + ",reserved: " + reservedMem + ", total: " + totalMem +
"; new used: " + capacityMemory.getUsedCapacity() + ",reserved:" + capacityMemory.getReservedCapacity() +
"; movedfromreserved: " + moveFromReserved + ",moveToReservered" + moveToReservered);
s_logger.debug("release mem from host: " + hostId + ", old used: " + usedMem + ",reserved: " + reservedMem + ", total: " + totalMem
+ "; new used: " + capacityMemory.getUsedCapacity() + ",reserved:" + capacityMemory.getReservedCapacity()
+ "; movedfromreserved: " + moveFromReserved + ",moveToReservered" + moveToReservered);
_capacityDao.update(capacityCpu.getId(), capacityCpu);
_capacityDao.update(capacityMemory.getId(), capacityMemory);
@ -225,7 +237,7 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
long reservedCpu = capacityCpu.getReservedCapacity();
long reservedMem = capacityMem.getReservedCapacity();
long actualTotalCpu = capacityCpu.getTotalCapacity();
long totalCpu = (long)(actualTotalCpu * cpuOverprovisioningFactor);
long totalCpu = (long) (actualTotalCpu * cpuOverprovisioningFactor);
if (s_logger.isDebugEnabled()) {
s_logger.debug("Hosts's actual total CPU: " + actualTotalCpu + " and CPU after applying overprovisioning: " + totalCpu);
}
@ -235,26 +247,26 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
long freeMem = totalMem - (reservedMem + usedMem);
if (s_logger.isDebugEnabled()) {
s_logger.debug("We are allocating VM, increasing the used capacity of this host:"+ hostId);
s_logger.debug("Current Used CPU: "+usedCpu + " , Free CPU:"+freeCpu+" ,Requested CPU: "+cpu);
s_logger.debug("Current Used RAM: "+usedMem + " , Free RAM:"+freeMem+" ,Requested RAM: "+ram);
}
s_logger.debug("We are allocating VM, increasing the used capacity of this host:" + hostId);
s_logger.debug("Current Used CPU: " + usedCpu + " , Free CPU:" + freeCpu + " ,Requested CPU: " + cpu);
s_logger.debug("Current Used RAM: " + usedMem + " , Free RAM:" + freeMem + " ,Requested RAM: " + ram);
}
capacityCpu.setUsedCapacity(usedCpu + cpu);
capacityMem.setUsedCapacity(usedMem + ram);
if (fromLastHost) {
/*alloc from reserved*/
/* alloc from reserved */
if (s_logger.isDebugEnabled()) {
s_logger.debug("We are allocating VM to the last host again, so adjusting the reserved capacity if it is not less than required");
s_logger.debug("Reserved CPU: "+reservedCpu + " , Requested CPU: "+cpu);
s_logger.debug("Reserved RAM: "+reservedMem + " , Requested RAM: "+ram);
}
s_logger.debug("Reserved CPU: " + reservedCpu + " , Requested CPU: " + cpu);
s_logger.debug("Reserved RAM: " + reservedMem + " , Requested RAM: " + ram);
}
if (reservedCpu >= cpu && reservedMem >= ram) {
capacityCpu.setReservedCapacity(reservedCpu - cpu);
capacityMem.setReservedCapacity(reservedMem - ram);
capacityMem.setReservedCapacity(reservedMem - ram);
}
} else {
/*alloc from free resource*/
/* alloc from free resource */
if (!((reservedCpu + usedCpu + cpu <= totalCpu) && (reservedMem + usedMem + ram <= totalMem))) {
if (s_logger.isDebugEnabled()) {
s_logger.debug("Host doesnt seem to have enough free capacity, but increasing the used capacity anyways, since the VM is already starting on this host ");
@ -262,14 +274,14 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
}
}
s_logger.debug("CPU STATS after allocation: for host: " + hostId + ", old used: " + usedCpu + ", old reserved: " +
reservedCpu + ", actual total: " + actualTotalCpu + ", total with overprovisioning: " + totalCpu +
"; new used:" + capacityCpu.getUsedCapacity() + ", reserved:" + capacityCpu.getReservedCapacity() +
"; requested cpu:" + cpu + ",alloc_from_last:" + fromLastHost);
s_logger.debug("CPU STATS after allocation: for host: " + hostId + ", old used: " + usedCpu + ", old reserved: " + reservedCpu
+ ", actual total: " + actualTotalCpu + ", total with overprovisioning: " + totalCpu + "; new used:"
+ capacityCpu.getUsedCapacity() + ", reserved:" + capacityCpu.getReservedCapacity() + "; requested cpu:" + cpu
+ ",alloc_from_last:" + fromLastHost);
s_logger.debug("RAM STATS after allocation: for host: " + hostId + ", old used: " + usedMem + ", old reserved: " +
reservedMem + ", total: " + totalMem + "; new used: " + capacityMem.getUsedCapacity() + ", reserved: " +
capacityMem.getReservedCapacity() + "; requested mem: " + ram + ",alloc_from_last:" + fromLastHost);
s_logger.debug("RAM STATS after allocation: for host: " + hostId + ", old used: " + usedMem + ", old reserved: " + reservedMem
+ ", total: " + totalMem + "; new used: " + capacityMem.getUsedCapacity() + ", reserved: " + capacityMem.getReservedCapacity()
+ "; requested mem: " + ram + ",alloc_from_last:" + fromLastHost);
_capacityDao.update(capacityCpu.getId(), capacityCpu);
_capacityDao.update(capacityMem.getId(), capacityMem);
@ -277,48 +289,48 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
} catch (Exception e) {
txn.rollback();
return;
}
}
}
@Override
public boolean checkIfHostHasCapacity(long hostId, Integer cpu, long ram, boolean checkFromReservedCapacity, float cpuOverprovisioningFactor){
public boolean checkIfHostHasCapacity(long hostId, Integer cpu, long ram, boolean checkFromReservedCapacity, float cpuOverprovisioningFactor) {
boolean hasCapacity = false;
if (s_logger.isDebugEnabled()) {
s_logger.debug("Checking if host: " + hostId + " has enough capacity for requested CPU: "+ cpu + " and requested RAM: "+ ram + " , cpuOverprovisioningFactor: "+cpuOverprovisioningFactor);
s_logger.debug("Checking if host: " + hostId + " has enough capacity for requested CPU: " + cpu + " and requested RAM: " + ram
+ " , cpuOverprovisioningFactor: " + cpuOverprovisioningFactor);
}
CapacityVO capacityCpu = _capacityDao.findByHostIdType(hostId, CapacityVO.CAPACITY_TYPE_CPU);
CapacityVO capacityMem = _capacityDao.findByHostIdType(hostId, CapacityVO.CAPACITY_TYPE_MEMORY);
if (capacityCpu == null || capacityMem == null) {
if(capacityCpu == null){
if (capacityCpu == null) {
if (s_logger.isDebugEnabled()) {
s_logger.debug("Cannot checkIfHostHasCapacity, Capacity entry for CPU not found in Db, for hostId: "+ hostId);
s_logger.debug("Cannot checkIfHostHasCapacity, Capacity entry for CPU not found in Db, for hostId: " + hostId);
}
}
if(capacityMem == null){
if (capacityMem == null) {
if (s_logger.isDebugEnabled()) {
s_logger.debug("Cannot checkIfHostHasCapacity, Capacity entry for RAM not found in Db, for hostId: "+ hostId);
s_logger.debug("Cannot checkIfHostHasCapacity, Capacity entry for RAM not found in Db, for hostId: " + hostId);
}
}
return false;
}
}
long usedCpu = capacityCpu.getUsedCapacity();
long usedMem = capacityMem.getUsedCapacity();
long reservedCpu = capacityCpu.getReservedCapacity();
long reservedMem = capacityMem.getReservedCapacity();
long actualTotalCpu = capacityCpu.getTotalCapacity();
long totalCpu = (long)(actualTotalCpu * cpuOverprovisioningFactor);
long totalCpu = (long) (actualTotalCpu * cpuOverprovisioningFactor);
if (s_logger.isDebugEnabled()) {
s_logger.debug("Hosts's actual total CPU: " + actualTotalCpu + " and CPU after applying overprovisioning: " + totalCpu);
}
long totalMem = capacityMem.getTotalCapacity();
String failureReason = "";
if (checkFromReservedCapacity) {
long freeCpu = reservedCpu;
@ -326,35 +338,35 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
if (s_logger.isDebugEnabled()) {
s_logger.debug("We need to allocate to the last host again, so checking if there is enough reserved capacity");
s_logger.debug("Reserved CPU: "+freeCpu + " , Requested CPU: "+cpu);
s_logger.debug("Reserved RAM: "+freeMem + " , Requested RAM: "+ram);
}
/*alloc from reserved*/
if (reservedCpu >= cpu){
if(reservedMem >= ram) {
s_logger.debug("Reserved CPU: " + freeCpu + " , Requested CPU: " + cpu);
s_logger.debug("Reserved RAM: " + freeMem + " , Requested RAM: " + ram);
}
/* alloc from reserved */
if (reservedCpu >= cpu) {
if (reservedMem >= ram) {
hasCapacity = true;
}else{
} else {
failureReason = "Host does not have enough reserved RAM available";
}
}else{
} else {
failureReason = "Host does not have enough reserved CPU available";
}
}
} else {
long freeCpu = totalCpu - (reservedCpu + usedCpu);
long freeMem = totalMem - (reservedMem + usedMem);
if (s_logger.isDebugEnabled()) {
s_logger.debug("Free CPU: "+freeCpu + " , Requested CPU: "+cpu);
s_logger.debug("Free RAM: "+freeMem + " , Requested RAM: "+ram);
}
/*alloc from free resource*/
s_logger.debug("Free CPU: " + freeCpu + " , Requested CPU: " + cpu);
s_logger.debug("Free RAM: " + freeMem + " , Requested RAM: " + ram);
}
/* alloc from free resource */
if ((reservedCpu + usedCpu + cpu <= totalCpu)) {
if((reservedMem + usedMem + ram <= totalMem)){
if ((reservedMem + usedMem + ram <= totalMem)) {
hasCapacity = true;
}else{
} else {
failureReason = "Host does not have enough RAM available";
}
}else{
} else {
failureReason = "Host does not have enough CPU available";
}
}
@ -364,21 +376,21 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
s_logger.debug("Host has enough CPU and RAM available");
}
s_logger.debug("STATS: Can alloc CPU from host: " + hostId + ", used: " + usedCpu + ", reserved: " +
reservedCpu + ", actual total: "+actualTotalCpu + ", total with overprovisioning: " + totalCpu +
"; requested cpu:" + cpu + ",alloc_from_last_host?:" + checkFromReservedCapacity);
s_logger.debug("STATS: Can alloc CPU from host: " + hostId + ", used: " + usedCpu + ", reserved: " + reservedCpu + ", actual total: "
+ actualTotalCpu + ", total with overprovisioning: " + totalCpu + "; requested cpu:" + cpu + ",alloc_from_last_host?:"
+ checkFromReservedCapacity);
s_logger.debug("STATS: Can alloc MEM from host: " + hostId + ", used: " + usedMem + ", reserved: " +
reservedMem + ", total: " + totalMem + "; requested mem: " + ram + ",alloc_from_last_host?:" + checkFromReservedCapacity);
s_logger.debug("STATS: Can alloc MEM from host: " + hostId + ", used: " + usedMem + ", reserved: " + reservedMem + ", total: " + totalMem
+ "; requested mem: " + ram + ",alloc_from_last_host?:" + checkFromReservedCapacity);
} else {
if (checkFromReservedCapacity) {
s_logger.debug("STATS: Failed to alloc resource from host: " + hostId + " reservedCpu: " + reservedCpu + ", requested cpu: " + cpu +
", reservedMem: " + reservedMem + ", requested mem: " + ram);
s_logger.debug("STATS: Failed to alloc resource from host: " + hostId + " reservedCpu: " + reservedCpu + ", requested cpu: " + cpu
+ ", reservedMem: " + reservedMem + ", requested mem: " + ram);
} else {
s_logger.debug("STATS: Failed to alloc resource from host: " + hostId + " reservedCpu: " + reservedCpu + ", used cpu: " + usedCpu + ", requested cpu: " + cpu +
", actual total cpu: "+actualTotalCpu + ", total cpu with overprovisioning: " + totalCpu +
", reservedMem: " + reservedMem + ", used Mem: " + usedMem + ", requested mem: " + ram + ", total Mem:" + totalMem);
s_logger.debug("STATS: Failed to alloc resource from host: " + hostId + " reservedCpu: " + reservedCpu + ", used cpu: " + usedCpu
+ ", requested cpu: " + cpu + ", actual total cpu: " + actualTotalCpu + ", total cpu with overprovisioning: " + totalCpu
+ ", reservedMem: " + reservedMem + ", used Mem: " + usedMem + ", requested mem: " + ram + ", total Mem:" + totalMem);
}
if (s_logger.isDebugEnabled()) {
@ -433,10 +445,10 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
s_logger.debug("Found " + vmsByLastHostId.size() + " VM, not running on host " + host.getId());
}
for (VMInstanceVO vm : vmsByLastHostId) {
long secondsSinceLastUpdate = (DateUtil.currentGMTTime().getTime() - vm.getUpdateTime().getTime())/1000;
long secondsSinceLastUpdate = (DateUtil.currentGMTTime().getTime() - vm.getUpdateTime().getTime()) / 1000;
if (secondsSinceLastUpdate < _vmCapacityReleaseInterval) {
ServiceOffering so = offeringsMap.get(vm.getServiceOfferingId());
reservedMemory += so.getRamSize() * 1024L * 1024L;
reservedMemory += so.getRamSize() * 1024L * 1024L;
reservedCpu += so.getCpu() * so.getSpeed();
}
}
@ -445,23 +457,32 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
CapacityVO memCap = _capacityDao.findByHostIdType(host.getId(), CapacityVO.CAPACITY_TYPE_MEMORY);
if (cpuCap.getUsedCapacity() == usedCpu && cpuCap.getReservedCapacity() == reservedCpu) {
s_logger.debug("No need to calibrate cpu capacity, host:" + host.getId() + " usedCpu: " + cpuCap.getUsedCapacity() + " reservedCpu: " + cpuCap.getReservedCapacity());
s_logger.debug("No need to calibrate cpu capacity, host:" + host.getId() + " usedCpu: " + cpuCap.getUsedCapacity()
+ " reservedCpu: " + cpuCap.getReservedCapacity());
} else if (cpuCap.getReservedCapacity() != reservedCpu) {
s_logger.debug("Calibrate reserved cpu for host: " + host.getId() + " old reservedCpu:" + cpuCap.getReservedCapacity() + " new reservedCpu:" + reservedCpu);
s_logger.debug("Calibrate reserved cpu for host: " + host.getId() + " old reservedCpu:" + cpuCap.getReservedCapacity()
+ " new reservedCpu:" + reservedCpu);
cpuCap.setReservedCapacity(reservedCpu);
} else if (cpuCap.getUsedCapacity() != usedCpu) {
s_logger.debug("Calibrate used cpu for host: " + host.getId() + " old usedCpu:" + cpuCap.getUsedCapacity() + " new usedCpu:" + usedCpu);
s_logger.debug("Calibrate used cpu for host: " + host.getId() + " old usedCpu:" + cpuCap.getUsedCapacity() + " new usedCpu:"
+ usedCpu);
cpuCap.setUsedCapacity(usedCpu);
}
if (memCap.getUsedCapacity() == usedMemory && memCap.getReservedCapacity() == reservedMemory) {
s_logger.debug("No need to calibrate memory capacity, host:" + host.getId() + " usedMem: " + memCap.getUsedCapacity() + " reservedMem: " + memCap.getReservedCapacity());
s_logger.debug("No need to calibrate memory capacity, host:" + host.getId() + " usedMem: " + memCap.getUsedCapacity()
+ " reservedMem: " + memCap.getReservedCapacity());
} else if (memCap.getReservedCapacity() != reservedMemory) {
s_logger.debug("Calibrate reserved memory for host: " + host.getId() + " old reservedMem:" + memCap.getReservedCapacity() + " new reservedMem:" + reservedMemory);
s_logger.debug("Calibrate reserved memory for host: " + host.getId() + " old reservedMem:" + memCap.getReservedCapacity()
+ " new reservedMem:" + reservedMemory);
memCap.setReservedCapacity(reservedMemory);
} else if (memCap.getUsedCapacity() != usedMemory) {
/*Didn't calibrate for used memory, because VMs can be in state(starting/migrating) that I don't know on which host they are allocated*/
s_logger.debug("Calibrate used memory for host: " + host.getId() + " old usedMem: " + memCap.getUsedCapacity() + " new usedMem: " + usedMemory);
/*
* Didn't calibrate for used memory, because VMs can be in state(starting/migrating) that I don't know on which host they are
* allocated
*/
s_logger.debug("Calibrate used memory for host: " + host.getId() + " old usedMem: " + memCap.getUsedCapacity()
+ " new usedMem: " + usedMemory);
memCap.setUsedCapacity(usedMemory);
}
@ -471,15 +492,14 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
} catch (Exception e) {
}
}
}
}
}
}
}
@Override
public boolean preStateTransitionEvent(State oldState,
Event event, State newState, VirtualMachine vm, boolean transitionStatus, Long id) {
public boolean preStateTransitionEvent(State oldState, Event event, State newState, VirtualMachine vm, boolean transitionStatus, Long id) {
return true;
}
@ -489,9 +509,8 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
return false;
}
s_logger.debug("VM state transitted from :" + oldState + " to " + newState + " with event: " + event +
"vm's original host id: " + vm.getLastHostId() + " new host id: " + vm.getHostId() + " host id before state transition: " + oldHostId);
s_logger.debug("VM state transitted from :" + oldState + " to " + newState + " with event: " + event + "vm's original host id: "
+ vm.getLastHostId() + " new host id: " + vm.getHostId() + " host id before state transition: " + oldHostId);
if (oldState == State.Starting) {
if (event == Event.OperationFailed) {
@ -507,11 +526,11 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
}
} else if (oldState == State.Migrating) {
if (event == Event.AgentReportStopped) {
/*Release capacity from original host*/
/* Release capacity from original host */
releaseVmCapacity(vm, false, false, vm.getLastHostId());
releaseVmCapacity(vm, false, true, oldHostId);
} else if (event == Event.OperationFailed) {
/*Release from dest host*/
/* Release from dest host */
releaseVmCapacity(vm, false, false, oldHostId);
} else if (event == Event.OperationSucceeded) {
releaseVmCapacity(vm, false, false, vm.getLastHostId());
@ -526,119 +545,133 @@ public class CapacityManagerImpl implements CapacityManager , StateListener<Stat
}
}
if((newState == State.Starting || newState == State.Migrating) && vm.getHostId() != null){
if ((newState == State.Starting || newState == State.Migrating) && vm.getHostId() != null) {
boolean fromLastHost = false;
if(vm.getLastHostId() == vm.getHostId()){
if (vm.getLastHostId() == vm.getHostId()) {
s_logger.debug("VM starting again on the last host it was stopped on");
fromLastHost = true;
}
allocateVmCapacity(vm,fromLastHost);
allocateVmCapacity(vm, fromLastHost);
}
return true;
}
// create capacity entries if none exist for this server
private void createCapacityEntry(final StartupCommand startup, HostVO server) {
SearchCriteria<CapacityVO> capacitySC = _capacityDao
.createSearchCriteria();
// TODO: Get rid of this case once we've determined that the capacity listeners above have all the changes
// create capacity entries if none exist for this server
private void createCapacityEntry(StartupCommand startup, HostVO server) {
SearchCriteria<CapacityVO> capacitySC = _capacityDao.createSearchCriteria();
capacitySC.addAnd("hostOrPoolId", SearchCriteria.Op.EQ, server.getId());
capacitySC.addAnd("dataCenterId", SearchCriteria.Op.EQ,
server.getDataCenterId());
capacitySC.addAnd("dataCenterId", SearchCriteria.Op.EQ, server.getDataCenterId());
capacitySC.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
if (startup instanceof StartupRoutingCommand) {
SearchCriteria<CapacityVO> capacityCPU = _capacityDao
.createSearchCriteria();
capacityCPU.addAnd("hostOrPoolId", SearchCriteria.Op.EQ,
server.getId());
capacityCPU.addAnd("dataCenterId", SearchCriteria.Op.EQ,
server.getDataCenterId());
capacityCPU
.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
capacityCPU.addAnd("capacityType", SearchCriteria.Op.EQ,
CapacityVO.CAPACITY_TYPE_CPU);
List<CapacityVO> capacityVOCpus = _capacityDao.search(capacitySC,
null);
SearchCriteria<CapacityVO> capacityCPU = _capacityDao.createSearchCriteria();
capacityCPU.addAnd("hostOrPoolId", SearchCriteria.Op.EQ, server.getId());
capacityCPU.addAnd("dataCenterId", SearchCriteria.Op.EQ, server.getDataCenterId());
capacityCPU.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
capacityCPU.addAnd("capacityType", SearchCriteria.Op.EQ, CapacityVO.CAPACITY_TYPE_CPU);
List<CapacityVO> capacityVOCpus = _capacityDao.search(capacitySC, null);
if (capacityVOCpus != null && !capacityVOCpus.isEmpty()) {
CapacityVO CapacityVOCpu = capacityVOCpus.get(0);
long newTotalCpu = (long) (server.getCpus().longValue()
* server.getSpeed().longValue() * _cpuOverProvisioningFactor);
long newTotalCpu = (long) (server.getCpus().longValue() * server.getSpeed().longValue() * _cpuOverProvisioningFactor);
if ((CapacityVOCpu.getTotalCapacity() <= newTotalCpu)
|| ((CapacityVOCpu.getUsedCapacity() + CapacityVOCpu
.getReservedCapacity()) <= newTotalCpu)) {
|| ((CapacityVOCpu.getUsedCapacity() + CapacityVOCpu.getReservedCapacity()) <= newTotalCpu)) {
CapacityVOCpu.setTotalCapacity(newTotalCpu);
} else if ((CapacityVOCpu.getUsedCapacity()
+ CapacityVOCpu.getReservedCapacity() > newTotalCpu)
} else if ((CapacityVOCpu.getUsedCapacity() + CapacityVOCpu.getReservedCapacity() > newTotalCpu)
&& (CapacityVOCpu.getUsedCapacity() < newTotalCpu)) {
CapacityVOCpu.setReservedCapacity(0);
CapacityVOCpu.setTotalCapacity(newTotalCpu);
} else {
s_logger.debug("What? new cpu is :" + newTotalCpu
+ ", old one is " + CapacityVOCpu.getUsedCapacity()
+ "," + CapacityVOCpu.getReservedCapacity() + ","
+ CapacityVOCpu.getTotalCapacity());
s_logger.debug("What? new cpu is :" + newTotalCpu + ", old one is " + CapacityVOCpu.getUsedCapacity() + ","
+ CapacityVOCpu.getReservedCapacity() + "," + CapacityVOCpu.getTotalCapacity());
}
_capacityDao.update(CapacityVOCpu.getId(), CapacityVOCpu);
} else {
CapacityVO capacity = new CapacityVO(
server.getId(),
server.getDataCenterId(),
server.getPodId(),
server.getClusterId(),
0L,
(long) (server.getCpus().longValue()
* server.getSpeed().longValue() * _cpuOverProvisioningFactor),
CapacityVO capacity = new CapacityVO(server.getId(), server.getDataCenterId(), server.getPodId(), server.getClusterId(), 0L,
(long) (server.getCpus().longValue() * server.getSpeed().longValue() * _cpuOverProvisioningFactor),
CapacityVO.CAPACITY_TYPE_CPU);
_capacityDao.persist(capacity);
}
SearchCriteria<CapacityVO> capacityMem = _capacityDao
.createSearchCriteria();
capacityMem.addAnd("hostOrPoolId", SearchCriteria.Op.EQ,
server.getId());
capacityMem.addAnd("dataCenterId", SearchCriteria.Op.EQ,
server.getDataCenterId());
capacityMem
.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
capacityMem.addAnd("capacityType", SearchCriteria.Op.EQ,
CapacityVO.CAPACITY_TYPE_MEMORY);
List<CapacityVO> capacityVOMems = _capacityDao.search(capacityMem,
null);
SearchCriteria<CapacityVO> capacityMem = _capacityDao.createSearchCriteria();
capacityMem.addAnd("hostOrPoolId", SearchCriteria.Op.EQ, server.getId());
capacityMem.addAnd("dataCenterId", SearchCriteria.Op.EQ, server.getDataCenterId());
capacityMem.addAnd("podId", SearchCriteria.Op.EQ, server.getPodId());
capacityMem.addAnd("capacityType", SearchCriteria.Op.EQ, CapacityVO.CAPACITY_TYPE_MEMORY);
List<CapacityVO> capacityVOMems = _capacityDao.search(capacityMem, null);
if (capacityVOMems != null && !capacityVOMems.isEmpty()) {
CapacityVO CapacityVOMem = capacityVOMems.get(0);
long newTotalMem = server.getTotalMemory();
if (CapacityVOMem.getTotalCapacity() <= newTotalMem
|| (CapacityVOMem.getUsedCapacity()
+ CapacityVOMem.getReservedCapacity() <= newTotalMem)) {
|| (CapacityVOMem.getUsedCapacity() + CapacityVOMem.getReservedCapacity() <= newTotalMem)) {
CapacityVOMem.setTotalCapacity(newTotalMem);
} else if (CapacityVOMem.getUsedCapacity()
+ CapacityVOMem.getReservedCapacity() > newTotalMem
} else if (CapacityVOMem.getUsedCapacity() + CapacityVOMem.getReservedCapacity() > newTotalMem
&& CapacityVOMem.getUsedCapacity() < newTotalMem) {
CapacityVOMem.setReservedCapacity(0);
CapacityVOMem.setTotalCapacity(newTotalMem);
} else {
s_logger.debug("What? new cpu is :" + newTotalMem
+ ", old one is " + CapacityVOMem.getUsedCapacity()
+ "," + CapacityVOMem.getReservedCapacity() + ","
+ CapacityVOMem.getTotalCapacity());
s_logger.debug("What? new cpu is :" + newTotalMem + ", old one is " + CapacityVOMem.getUsedCapacity() + ","
+ CapacityVOMem.getReservedCapacity() + "," + CapacityVOMem.getTotalCapacity());
}
_capacityDao.update(CapacityVOMem.getId(), CapacityVOMem);
} else {
CapacityVO capacity = new CapacityVO(server.getId(),
server.getDataCenterId(), server.getPodId(), server.getClusterId(), 0L,
server.getTotalMemory(),
CapacityVO.CAPACITY_TYPE_MEMORY);
CapacityVO capacity = new CapacityVO(server.getId(), server.getDataCenterId(), server.getPodId(), server.getClusterId(), 0L,
server.getTotalMemory(), CapacityVO.CAPACITY_TYPE_MEMORY);
_capacityDao.persist(capacity);
}
}
}
@Override
public boolean processAnswers(long agentId, long seq, Answer[] answers) {
// TODO Auto-generated method stub
return false;
}
@Override
public boolean processCommands(long agentId, long seq, Command[] commands) {
// TODO Auto-generated method stub
return false;
}
@Override
public AgentControlAnswer processControlCommand(long agentId, AgentControlCommand cmd) {
// TODO Auto-generated method stub
return null;
}
@Override
public void processConnect(HostVO host, StartupCommand cmd) throws ConnectionException {
// TODO Auto-generated method stub
}
@Override
public boolean processDisconnect(long agentId, Status state) {
// TODO Auto-generated method stub
return false;
}
@Override
public boolean isRecurring() {
// TODO Auto-generated method stub
return false;
}
@Override
public int getTimeout() {
// TODO Auto-generated method stub
return 0;
}
@Override
public boolean processTimeout(long agentId, long seq) {
// TODO Auto-generated method stub
return false;
}
}

View File

@ -657,7 +657,7 @@ public class VirtualMachineManagerImpl implements VirtualMachineManager, Listene
VirtualMachineTO vmTO = hvGuru.implement(vmProfile);
Commands cmds = new Commands(OnError.Revert);
Commands cmds = new Commands(OnError.Stop);
cmds.addCommand(new StartCommand(vmTO));
vmGuru.finalizeDeployment(cmds, vmProfile, dest, ctx);
@ -1291,7 +1291,7 @@ public class VirtualMachineManagerImpl implements VirtualMachineManager, Listene
try {
Commands cmds = new Commands(OnError.Revert);
Commands cmds = new Commands(OnError.Stop);
cmds.addCommand(new RebootCommand(vm.getInstanceName()));
_agentMgr.send(host.getId(), cmds);
@ -1487,7 +1487,7 @@ public class VirtualMachineManagerImpl implements VirtualMachineManager, Listene
profile.addNic(nicProfile);
}
Commands cmds = new Commands(OnError.Revert);
Commands cmds = new Commands(OnError.Stop);
s_logger.debug("Finalizing commands that need to be send to complete Start process for the vm " + vm);
if (vmGuru.finalizeCommandsOnStart(cmds, profile)) {