Commit 059beb41 authored by rodrigo.calheiros's avatar rodrigo.calheiros

NetworkCloudSim merged with CloudSim

parent 09118073
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import org.cloudbus.cloudsim.core.CloudSim;
import org.cloudbus.cloudsim.core.CloudSimTags;
import org.cloudbus.cloudsim.core.SimEvent;
import org.cloudbus.cloudsim.core.predicates.PredicateType;
public class AggregateSwitch extends Switch{
public AggregateSwitch(String name, int level, NetworkDatacenter dc) {
super(name, level, dc);
// TODO Auto-generated constructor stub
downlinkswitchpktlist=new HashMap<Integer,List<HostPacket>>();
uplinkswitchpktlist=new HashMap<Integer,List<HostPacket>>();
uplinkbandwidth=Constants.BandWidthAggRoot;
downlinkbandwidth=Constants.BandWidthEdgeAgg;
latency=Constants.SwitchingDelayAgg;
numport=Constants.AggSwitchPort;
uplinkswitches=new ArrayList<Switch>();
downlinkswitches=new ArrayList<Switch>();
}
private void processpacket_down(SimEvent ev) {
// TODO Auto-generated method stub
//packet coming from up level router.
//has to send downward
//check which switch to forward to
//add packet in the switch list
//add packet in the host list
//int src=ev.getSource();
HostPacket hspkt=(HostPacket) ev.getData();
int recvVMid=hspkt.pkt.reciever;
CloudSim.cancelAll(getId(), new PredicateType(CloudSimTags.Network_Event_send));
schedule(getId(),this.latency, CloudSimTags.Network_Event_send);
if(this.level==Constants.Agg_LEVEL)
{
//packet is coming from root so need to be sent to edgelevel swich
//find the id for edgelevel switch
int switchid=dc.VmToSwitchid.get(recvVMid);
List<HostPacket> pktlist=this.downlinkswitchpktlist.get(switchid);
if(pktlist==null){
pktlist=new ArrayList<HostPacket>();
this.downlinkswitchpktlist.put(switchid, pktlist);
}
pktlist.add(hspkt);
return;
}
}
private void processpacket_up(SimEvent ev) {
// TODO Auto-generated method stub
//packet coming from down level router.
//has to send up
//check which switch to forward to
//add packet in the switch list
//
//int src=ev.getSource();
HostPacket hspkt=(HostPacket) ev.getData();
int recvVMid=hspkt.pkt.reciever;
CloudSim.cancelAll(getId(), new PredicateType(CloudSimTags.Network_Event_send));
schedule(getId(),this.switching_delay, CloudSimTags.Network_Event_send);
if(this.level==Constants.Agg_LEVEL)
{
//packet is coming from edge level router so need to be sent to either root or another edge level swich
//find the id for edgelevel switch
int switchid=dc.VmToSwitchid.get(recvVMid);
boolean flagtoswtich=false;
for(Switch sw:this.downlinkswitches)
{
if(switchid==sw.getId()) flagtoswtich=true;
}
if(flagtoswtich)
{
List<HostPacket> pktlist=this.downlinkswitchpktlist.get(switchid);
if(pktlist==null){
pktlist=new ArrayList<HostPacket>();
this.downlinkswitchpktlist.put(switchid, pktlist);
}
pktlist.add(hspkt);
}
else//send to up
{
Switch sw=this.uplinkswitches.get(0);
List<HostPacket> pktlist=this.uplinkswitchpktlist.get(sw.getId());
if(pktlist==null){
pktlist=new ArrayList<HostPacket>();
this.uplinkswitchpktlist.put(sw.getId(), pktlist);
}
pktlist.add(hspkt);
}
}
}
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.List;
import org.cloudbus.cloudsim.UtilizationModel;
import org.cloudbus.cloudsim.UtilizationModelFull;
import org.cloudbus.cloudsim.core.CloudSim;
public class AppCloudlet {
public static final int APP_MC = 1;
public static final int APP_Workflow = 3;
public AppCloudlet(int type, int appID,
double deadline, int numbervm, int userId) {
super();
this.type = type;
this.appID = appID;
this.deadline = deadline;
this.numbervm = numbervm;
this.userId = userId;
clist=new ArrayList<NetworkCloudlet>();
}
public int type; //fft,fem
public int appID;
public ArrayList<NetworkCloudlet> clist;
public double deadline;
public double accuracy;
public int numbervm;
public int userId;
public double exeTime;
public int requestclass;
public void createCloudletList(List<Integer> vmIdList)
{
for(int i=0;i<numbervm;i++){
long length = 4;
long fileSize = 300;
long outputSize = 300;
long memory = 256;
int pesNumber = 4;
UtilizationModel utilizationModel = new UtilizationModelFull();
//HPCCloudlet cl=new HPCCloudlet();
NetworkCloudlet cl = new NetworkCloudlet(Constants.currentCloudletId, length, pesNumber, fileSize, outputSize, memory, utilizationModel, utilizationModel, utilizationModel);
// setting the owner of these Cloudlets
Constants.currentCloudletId++;
cl.setUserId(userId);
cl.submittime=CloudSim.clock();
// TaskStage ts=new TaskStage(Constants.EXECUTION, 0, length, 0,memory, (i+1)%numbervm);
// TaskStage ts1=new TaskStage(Constants.WAIT_SEND, 100, 0, 0, memory, (i+1)%numbervm);
// TaskStage ts2=new TaskStage(Constants.WAIT_RECV, 100, 0, 0, memory, (i+1)%numbervm);
// TaskStage ts3=new TaskStage(Constants.EXECUTION, 0, length, 0, memory, (i+1)%numbervm);
// cl.stages.add(ts);
// cl.stages.add(ts1);
// cl.stages.add(ts2);
// cl.stages.add(ts3);
// cl.submittime=CloudSim.clock();
cl.currStagenum=-1;
clist.add(cl);
}
//based on type
}
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class Constants {
public static final int FFT = 0;
public static int roundrobinRack=0;
public static int maxhostVM=2;
public static int HOST_PEs=8;
public static double maxMemperVM=1024*1024;//kb
public static final int BIN_NODE_TIME=10;
public static int currentCloudletId=0;
public static int currentAppId=0;
public static final int EXECUTION=0;
public static final int WAIT_SEND=1;
public static final int WAIT_RECV=2;
public static final int FINISH=-2;
public static final int ROOT_LEVEL=0;
public static final int Agg_LEVEL=1;
public static final int EDGE_LEVEL=2;
public static final int PES_NUMBER=4;
public static final int FILE_SIZE=300;
public static final int OUTPUT_SIZE=300;
public static int REQUESTS_PER_SECOND=200;
public static final int COMMUNICATION_LENGTH=1;
public static int Requestspersecond = 2;
public static int REQUEST_CLASSES=10;
public static double LOW_ACCURACY_RATE=0.2;
public static double MEDIUM_ACCURACY_RATE=.3;
public static double HIGH_ACCURACY_RATE=0.9;
public static boolean schedulerOverlap=true;
public static boolean BASE = true;
public static long BandWidthEdgeAgg=100*1024*1024;//100 Megabits
public static long BandWidthEdgeHost=100*1024*1024;//
public static long BandWidthAggRoot=20*1024*1024*2;//40gb
public static double SwitchingDelayRoot=.00285;
public static double SwitchingDelayAgg=.00245;//.00245
public static double SwitchingDelayEdge=.00157;//ms
public static double EdgeSwitchPort=4;//number of host
public static double AggSwitchPort=1;//number of Edge
public static double RootSwitchPort=1;//number of Agg
public static double seed=199;
public static boolean logflag=false;
public static Map<Integer,Map<Integer,Boolean>> Cache=new HashMap<Integer,Map<Integer,Boolean>>();
public static int iteration=10;
public static int nexttime=1000;
public static int totaldatatransfer=0;
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import org.cloudbus.cloudsim.core.CloudSim;
import org.cloudbus.cloudsim.core.CloudSimTags;
import org.cloudbus.cloudsim.core.SimEvent;
import org.cloudbus.cloudsim.core.predicates.PredicateType;
public class EdgeSwitch extends Switch{
public EdgeSwitch(String name, int dcid, NetworkDatacenter dc) {
super(name, dcid, dc);
hostlist=new HashMap<Integer,NetworkHost>();
uplinkswitchpktlist=new HashMap<Integer,List<HostPacket>>();
packetTohost=new HashMap<Integer,List<HostPacket>>();
uplinkbandwidth=Constants.BandWidthEdgeAgg;
downlinkbandwidth=Constants.BandWidthEdgeHost;
this.switching_delay=Constants.SwitchingDelayEdge;
numport=Constants.EdgeSwitchPort;
uplinkswitches=new ArrayList<Switch>();
// TODO Auto-generated constructor stub
}
private void registerHost(SimEvent ev) {
// TODO Auto-generated method stub
NetworkHost hs=(NetworkHost)ev.getData();
hostlist.put(hs.getId(),(NetworkHost)ev.getData());
}
private void processpacket_up(SimEvent ev) {
// TODO Auto-generated method stub
//packet coming from down level router/host.
//has to send up
//check which switch to forward to
//add packet in the switch list
//
//int src=ev.getSource();
HostPacket hspkt=(HostPacket) ev.getData();
int recvVMid=hspkt.pkt.reciever;
CloudSim.cancelAll(getId(), new PredicateType(CloudSimTags.Network_Event_send));
schedule(getId(),this.switching_delay, CloudSimTags.Network_Event_send);
// packet is recieved from host
//packet is to be sent to aggregate level or to another host in the same level
int hostid=dc.VmtoHostlist.get(recvVMid);
NetworkHost hs=this.hostlist.get(hostid);
hspkt.recieverhostid=hostid;
//packet needs to go to a host which is connected directly to switch
if(hs!=null)
{
//packet to be sent to host connected to the switch
List<HostPacket> pktlist=this.packetTohost.get(hostid);
if(pktlist==null){
pktlist=new ArrayList<HostPacket>();
this.packetTohost.put(hostid, pktlist);
}
pktlist.add(hspkt);
return;
}
//otherwise
//packet is to be sent to upper switch
//ASSUMPTION EACH EDGE is Connected to one aggregate level switch
//if there are more than one Aggregate level switch one need to modify following code
Switch sw=this.uplinkswitches.get(0);
List<HostPacket> pktlist=this.uplinkswitchpktlist.get(sw.getId());
if(pktlist==null){
pktlist=new ArrayList<HostPacket>();
this.uplinkswitchpktlist.put(sw.getId(), pktlist);
}
pktlist.add(hspkt);
return;
}
private void processpacketforward(SimEvent ev) {
// TODO Auto-generated method stub
//search for the host and packets..send to them
if(this.uplinkswitchpktlist!=null)
{
for(Entry<Integer, List<HostPacket>> es:uplinkswitchpktlist.entrySet())
{
int tosend=es.getKey();
List<HostPacket> hspktlist=es.getValue();
if(!hspktlist.isEmpty()){
//sharing bandwidth between packets
double avband=this.uplinkbandwidth/hspktlist.size();
Iterator<HostPacket> it=hspktlist.iterator();
while(it.hasNext())
{
HostPacket hspkt=it.next();
double delay=1000*hspkt.pkt.data/avband;
this.send(tosend,delay,CloudSimTags.Network_Event_UP, hspkt);
}
hspktlist.clear();
}
}
}
if(this.packetTohost!=null)
{
for(Entry<Integer, List<HostPacket>> es:packetTohost.entrySet())
{
int tosend=es.getKey();
NetworkHost hs=this.hostlist.get(tosend);
List<HostPacket> hspktlist=es.getValue();
if(!hspktlist.isEmpty()){
double avband=this.downlinkbandwidth/hspktlist.size();
Iterator<HostPacket> it=hspktlist.iterator();
while(it.hasNext()){
HostPacket hspkt=it.next();
//hspkt.recieverhostid=tosend;
//hs.packetrecieved.add(hspkt);
this.send(this.getId(),hspkt.pkt.data/avband,CloudSimTags.Network_Event_Host, hspkt);
}
hspktlist.clear();
}
}
}
//or to switch at next level.
//clear the list
}
}
package org.cloudbus.cloudsim.network.datacenter;
public class HostPacket {
public HostPacket(int id, NetPacket pkt2, int vmid, int cloudletid) {
// TODO Auto-generated constructor stub
this.pkt=pkt2;
this.sendervmid=vmid;
this.cloudletid=cloudletid;
this.senderhostid=id;
this.stime=pkt.sendtime;
this.recievervmid=pkt2.reciever;
}
NetPacket pkt;
int senderhostid;
int recieverhostid;
int sendervmid;
int recievervmid;
int cloudletid;
double stime;//time when sent
double rtime;//time when received
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.List;
import org.cloudbus.cloudsim.UtilizationModel;
import org.cloudbus.cloudsim.UtilizationModelFull;
import org.cloudbus.cloudsim.core.CloudSim;
public class MonteCarloApp extends AppCloudlet {
public MonteCarloApp(int type, int appID, double deadline, int numbervm, int userId) {
super(type, appID, deadline, numbervm,userId);
this.numbervm=this.getnumvm();
this.exeTime=getExecTime()/this.numbervm;
}
@Override
public void createCloudletList(List<Integer> vmIdList){
//basically, each task runs the simulation and then data is consolidated in one task
int executionTime = getExecTime();
long memory = accuracyToMemory();
long fileSize = Constants.FILE_SIZE;
long outputSize = Constants.OUTPUT_SIZE;
int pesNumber = Constants.PES_NUMBER;
int stgId=0;
int t=Constants.currentCloudletId;
for(int i=0;i<numbervm;i++){
UtilizationModel utilizationModel = new UtilizationModelFull();
NetworkCloudlet cl = new NetworkCloudlet(Constants.currentCloudletId, executionTime/numbervm, pesNumber, fileSize, outputSize, memory, utilizationModel, utilizationModel, utilizationModel);
Constants.currentCloudletId++;
cl.setUserId(userId);
cl.submittime=CloudSim.clock();
cl.currStagenum=-1;
cl.setVmId(vmIdList.get(i));
//compute and send data to node 0
cl.stages.add(new TaskStage(Constants.EXECUTION, Constants.COMMUNICATION_LENGTH, executionTime/numbervm, stgId++, memory, vmIdList.get(0),cl.getCloudletId()));
//0 has an extra stage of waiting for results; others send
if (i==0){
for(int j=1;j<numbervm;j++)
cl.stages.add(new TaskStage(Constants.WAIT_RECV, Constants.COMMUNICATION_LENGTH, 0, stgId++, memory, vmIdList.get(j),cl.getCloudletId()+j));
} else {
cl.stages.add(new TaskStage(Constants.WAIT_SEND, Constants.COMMUNICATION_LENGTH, 0, stgId++, memory, vmIdList.get(0),t));
}
clist.add(cl);
}
}
public int getnumvm(){
double exetime=getExecTime()/2;//for two vms
if(this.deadline>exetime)
return 2;
else if(this.deadline>(exetime/4)) return 4;
return 4;
}
private int getExecTime() {
//use exec constraints as Binomial
return 100;
}
private long accuracyToMemory() {
//use same memory constraints as Binomial
return 240076;
}
}
package org.cloudbus.cloudsim.network.datacenter;
public class NetPacket {
public NetPacket(int sender, int reciever, double data, double sendtime,
double recievetime,int vsnd,int vrvd) {
super();
this.sender = sender;
this.reciever = reciever;
this.data = data;
this.sendtime = sendtime;
this.recievetime = recievetime;
this.virtualrecvid=vrvd;
this.virtualsendid=vsnd;
}
int sender;
int virtualrecvid;
int virtualsendid;
int reciever;
double data;
double sendtime;
double recievetime;
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.Map;
import org.cloudbus.cloudsim.Cloudlet;
import org.cloudbus.cloudsim.UtilizationModel;
public class NetworkCloudlet extends Cloudlet implements Comparable{
long memory;
public NetworkCloudlet(int cloudletId, long cloudletLength,
int pesNumber, long cloudletFileSize, long cloudletOutputSize, long memory,
UtilizationModel utilizationModelCpu,
UtilizationModel utilizationModelRam,
UtilizationModel utilizationModelBw) {
super(cloudletId, cloudletLength, pesNumber, cloudletFileSize,
cloudletOutputSize, utilizationModelCpu, utilizationModelRam,
utilizationModelBw);
// TODO Auto-generated constructor stub
currStagenum=-1;
this.memory=memory;
stages=new ArrayList<TaskStage>();
}
public double submittime;
public double finishtime;
public double exetime;
public double numStage;
public int currStagenum;
public double timetostartStage;
public double timespentInStage;
public Map<Double, NetPacket> timeCommunicate;
public ArrayList<TaskStage> stages;
public double starttime;
@Override
public int compareTo(Object arg0) {
// TODO Auto-generated method stub
NetworkCloudlet s1=(NetworkCloudlet)arg0;
int alpha=0;
return 0;
}
public double getSubmittime() {
// TODO Auto-generated method stub
return submittime;
};
}
package org.cloudbus.cloudsim.network.datacenter;
/*
* Title: CloudSim Toolkit
* Description: CloudSim (Cloud Simulation) Toolkit for Modeling and Simulation of Clouds
* Licence: GPL - http://www.gnu.org/copyleft/gpl.html
*
* Copyright (c) 2009-2010, The University of Melbourne, Australia
*/
import java.text.DecimalFormat;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.cloudbus.cloudsim.Cloudlet;
import org.cloudbus.cloudsim.CloudletScheduler;
import org.cloudbus.cloudsim.DataCloudTags;
import org.cloudbus.cloudsim.Datacenter;
import org.cloudbus.cloudsim.DatacenterCharacteristics;
import org.cloudbus.cloudsim.File;
import org.cloudbus.cloudsim.Host;
import org.cloudbus.cloudsim.InfoPacket;
import org.cloudbus.cloudsim.Log;
import org.cloudbus.cloudsim.Storage;
import org.cloudbus.cloudsim.Vm;
import org.cloudbus.cloudsim.VmAllocationPolicy;
import org.cloudbus.cloudsim.core.CloudSim;
import org.cloudbus.cloudsim.core.CloudSimTags;
import org.cloudbus.cloudsim.core.SimEntity;
import org.cloudbus.cloudsim.core.SimEvent;
/**
* Datacenter class is a CloudResource whose hostList
* are virtualized. It deals with processing of VM queries (i.e., handling
* of VMs) instead of processing Cloudlet-related queries. So, even though an
* AllocPolicy will be instantiated (in the init() method of the superclass,
* it will not be used, as processing of cloudlets are handled by the CloudletScheduler
* and processing of VirtualMachines are handled by the VmAllocationPolicy.
*
* @author Rodrigo N. Calheiros
* @author Anton Beloglazov
* @since CloudSim Toolkit 1.0
*/
public class NetworkDatacenter extends Datacenter {
public NetworkDatacenter(String name,
DatacenterCharacteristics characteristics,
VmAllocationPolicy vmAllocationPolicy, List<Storage> storageList,
double schedulingInterval) throws Exception {
super(name, characteristics, vmAllocationPolicy, storageList,
schedulingInterval);
// TODO Auto-generated constructor stub
VmToSwitchid=new HashMap<Integer,Integer>();
HostToSwitchid=new HashMap<Integer,Integer>();
VmtoHostlist=new HashMap<Integer,Integer>();
Switchlist=new HashMap<Integer,Switch>();
}
public Map<Integer,Integer> VmToSwitchid;
public Map<Integer,Integer> HostToSwitchid;
public Map<Integer,Switch> Switchlist;
public Map<Integer,Integer> VmtoHostlist;
public Map<Integer, Switch> getEdgeSwitch(){
Map<Integer,Switch> edgeswitch=new HashMap<Integer,Switch>();
for(Entry<Integer, Switch> es:Switchlist.entrySet()){
if(es.getValue().level==Constants.EDGE_LEVEL)
{
edgeswitch.put(es.getKey(), es.getValue());
}
}
return edgeswitch;
}
public boolean processVmCreateHPC(Vm vm) {
boolean result = getVmAllocationPolicy().allocateHostForVm(vm);
if (result) {
this.VmToSwitchid.put(vm.getId(), ((NetworkHost)vm.getHost()).sw.getId());
this.VmtoHostlist.put(vm.getId(),vm.getHost().getId());
System.out.println(vm.getId()+" VM is created on "+vm.getHost().getId());
double amount = 0.0;
if (getDebts().containsKey(vm.getUserId())) {
amount = getDebts().get(vm.getUserId());
}
amount += getCharacteristics().getCostPerMem() * vm.getRam();
amount += getCharacteristics().getCostPerStorage() * vm.getSize();
getDebts().put(vm.getUserId(), amount);
getVmList().add(vm);
vm.updateVmProcessing(CloudSim.clock(), getVmAllocationPolicy().getHost(vm).getVmScheduler().getAllocatedMipsForVm(vm));
}
return result;
}
protected void processCloudletSubmit(SimEvent ev, boolean ack) {
updateCloudletProcessing();
try {
// gets the Cloudlet object
Cloudlet cl = (Cloudlet) ev.getData();
// checks whether this Cloudlet has finished or not
if (cl.isFinished()){
String name = CloudSim.getEntityName(cl.getUserId());
Log.printLine(getName()+": Warning - Cloudlet #"+cl.getCloudletId()+" owned by "+name+" is already completed/finished.");
Log.printLine("Therefore, it is not being executed again");
Log.printLine();
// NOTE: If a Cloudlet has finished, then it won't be processed.
// So, if ack is required, this method sends back a result.
// If ack is not required, this method don't send back a result.
// Hence, this might cause CloudSim to be hanged since waiting
// for this Cloudlet back.
if (ack) {
int[] data = new int[3];
data[0] = getId();
data[1] = cl.getCloudletId();
data[2] = CloudSimTags.FALSE;
// unique tag = operation tag
int tag = CloudSimTags.CLOUDLET_SUBMIT_ACK;
sendNow(cl.getUserId(), tag, data);
}
sendNow(cl.getUserId(), CloudSimTags.CLOUDLET_RETURN, cl);
return;
}
// process this Cloudlet to this CloudResource
cl.setResourceParameter(getId(), getCharacteristics().getCostPerSecond(), getCharacteristics().getCostPerBw());
int userId = cl.getUserId();
int vmId = cl.getVmId();
double fileTransferTime = predictFileTransferTime(cl.getRequiredFiles()); //time to transfer the files
Host host = getVmAllocationPolicy().getHost(vmId, userId);
Vm vm = host.getVm(vmId, userId);
CloudletScheduler scheduler = vm.getCloudletScheduler();
//System.out.println("cloudlet recieved by VM"+vmId);
double estimatedFinishTime = scheduler.cloudletSubmit(cl,fileTransferTime);
//if (estimatedFinishTime > 0.0 && estimatedFinishTime < getSchedulingInterval()) { //if this cloudlet is in the exec queue
if (estimatedFinishTime > 0.0) { //if this cloudlet is in the exec queue
//double estimatedFinishTime = (cl.getCloudletTotalLength()/(capacity*cl.getPesNumber())); //time to process the cloudlet
//Log.printLine(estimatedFinishTime+"="+gl.getCloudletLength()+"/("+capacity+"*"+gl.getNumPE()+")");
estimatedFinishTime += fileTransferTime;
//estimatedFinishTime += CloudSim.clock();
//Log.printLine(CloudSim.clock()+": Next event scheduled to +"+estimatedFinishTime);
send(getId(), estimatedFinishTime, CloudSimTags.VM_DATACENTER_EVENT);
//event to update the stages
send(getId(), 0.0001, CloudSimTags.VM_DATACENTER_EVENT);
}
if (ack) {
int[] data = new int[3];
data[0] = getId();
data[1] = cl.getCloudletId();
data[2] = CloudSimTags.TRUE;
// unique tag = operation tag
int tag = CloudSimTags.CLOUDLET_SUBMIT_ACK;
sendNow(cl.getUserId(), tag, data);
}
}
catch (ClassCastException c) {
Log.printLine(getName() + ".processCloudletSubmit(): " + "ClassCastException error.");
c.printStackTrace();
}
catch (Exception e) {
Log.printLine(getName() + ".processCloudletSubmit(): " + "Exception error.");
e.printStackTrace();
}
checkCloudletCompletion();
}
}
/*
* Title: CloudSim Toolkit
* Description: CloudSim (Cloud Simulation) Toolkit for Modeling and Simulation of Clouds
* Licence: GPL - http://www.gnu.org/copyleft/gpl.html
*
* Copyright (c) 2009-2010, The University of Melbourne, Australia
*/
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import org.cloudbus.cloudsim.Datacenter;
import org.cloudbus.cloudsim.Host;
import org.cloudbus.cloudsim.Log;
import org.cloudbus.cloudsim.Pe;
import org.cloudbus.cloudsim.Vm;
import org.cloudbus.cloudsim.VmScheduler;
import org.cloudbus.cloudsim.core.CloudSim;
import org.cloudbus.cloudsim.core.CloudSimTags;
import org.cloudbus.cloudsim.lists.PeList;
import org.cloudbus.cloudsim.lists.VmList;
import org.cloudbus.cloudsim.provisioners.BwProvisioner;
import org.cloudbus.cloudsim.provisioners.RamProvisioner;
/**
* Host class extends a Machine to include other hostList beside PEs
* to support simulation of virtualized grids. It executes actions related
* to management of virtual machines (e.g., creation and destruction). A host has
* a defined policy for provisioning memory and bw, as well as an allocation policy
* for Pe's to virtual machines.
*
* A host is associated to a datacenter. It can host virtual machines.
*
* @author Rodrigo N. Calheiros
* @author Anton Beloglazov
* @since CloudSim Toolkit 1.0
*/
public class NetworkHost extends Host {
public List <HostPacket> packetTosendLocal;
public List <HostPacket> packetTosendGlobal;
public List <HostPacket> packetrecieved;
public double memory;
public Switch sw;
public double bandwidth;//latency
public List<Double> CPUfinTimeCPU=new ArrayList<Double>();//time when last job will finish on CPU1
public double fintime=0;
public NetworkHost(int id, RamProvisioner ramProvisioner,
BwProvisioner bwProvisioner, long storage,
List<? extends Pe> peList, VmScheduler vmScheduler) {
super(id, ramProvisioner, bwProvisioner, storage, peList, vmScheduler);
// TODO Auto-generated constructor stub
this.packetrecieved=new ArrayList<HostPacket>();
this.packetTosendGlobal=new ArrayList<HostPacket>();
this.packetTosendLocal=new ArrayList<HostPacket>();
}
/**
* Requests updating of processing of cloudlets in the VMs running in this host.
*
* @param currentTime the current time
*
* @return expected time of completion of the next cloudlet in all VMs in this host. Double.MAX_VALUE
* if there is no future events expected in this host
*
* @pre currentTime >= 0.0
* @post $none
*/
public double updateVmsProcessing(double currentTime) {
double smallerTime = Double.MAX_VALUE;
//insert in each vm packet recieved
recvpackets();
for (Vm vm : super.getVmList()) {
// if (vm.isInMigration()) {
// continue;
// }
double time = ((NetworkVm)vm).updateVmProcessing(currentTime, getVmScheduler().getAllocatedMipsForVm(vm));
if (time > 0.0 && time < smallerTime) {
smallerTime = time;
}
}
sendpackets();
return smallerTime;
}
private void recvpackets() {
// TODO Auto-generated method stub
for(HostPacket hs:packetrecieved)
{
//hs.stime=hs.rtime;
hs.pkt.recievetime=CloudSim.clock();
//insertthe packet in recievedlist
Vm vm=VmList.getById(getVmList(), hs.pkt.reciever);
List<NetPacket> pktlist=((CloudletHPCSpaceShared)vm.getCloudletScheduler()).pktrecv.get(hs.pkt.sender);
//List<NetPacket> pktlist=((CloudletHPCSpaceSharedO)vm.getCloudletScheduler()).pktrecv.get(hs.pkt.virtualsendid);
if(pktlist==null){
pktlist=new ArrayList<NetPacket>();
((CloudletHPCSpaceShared)vm.getCloudletScheduler()).pktrecv.put(hs.pkt.sender, pktlist);
// ((CloudletHPCSpaceSharedO)vm.getCloudletScheduler()).pktrecv.put(hs.pkt.virtualsendid, pktlist);
}
pktlist.add(hs.pkt);
}
packetrecieved.clear();
}
private void sendpackets() {
// TODO Auto-generated method stub
for(Vm vm:super.getVmList())
{
for(Entry<Integer, List<NetPacket>> es:((CloudletHPCSpaceShared)vm.getCloudletScheduler()).pkttosend.entrySet())
//for(Entry<Integer, List<NetPacket>> es:((CloudletHPCSpaceSharedO)vm.getCloudletScheduler()).pkttosend.entrySet())
{
List<NetPacket> pktlist=es.getValue();
for(NetPacket pkt:pktlist)
{
HostPacket hpkt=new HostPacket(this.getId(),pkt,vm.getId(),pkt.sender);
Vm vm2=VmList.getById(this.getVmList(), hpkt.recievervmid);
if(vm2!=null) this.packetTosendLocal.add(hpkt);
else this.packetTosendGlobal.add(hpkt);
}
pktlist.clear();
}
}
boolean flag=false;
for(HostPacket hs:packetTosendLocal)
{
flag=true;
hs.stime=hs.rtime;
hs.pkt.recievetime=CloudSim.clock();
//insertthe packet in recievedlist
Vm vm=VmList.getById(getVmList(), hs.pkt.reciever);
//Vm vm=getVmList().get(hs.pkt.reciever);
List<NetPacket> pktlist=((CloudletHPCSpaceShared)vm.getCloudletScheduler()).pktrecv.get(hs.pkt.sender);
//List<NetPacket> pktlist=((CloudletHPCSpaceSharedO)vm.getCloudletScheduler()).pktrecv.get(hs.pkt.virtualsendid);
if(pktlist==null){
pktlist=new ArrayList<NetPacket>();
((CloudletHPCSpaceShared)vm.getCloudletScheduler()).pktrecv.put(hs.pkt.sender, pktlist);
// ((CloudletHPCSpaceSharedO)vm.getCloudletScheduler()).pktrecv.put(hs.pkt.virtualsendid, pktlist);
}
pktlist.add(hs.pkt);
}
if(flag){
for (Vm vm : super.getVmList()) {
// if (vm.isInMigration()) {
// continue;
// }
double time = ((NetworkVm)vm).updateVmProcessing(CloudSim.clock(), getVmScheduler().getAllocatedMipsForVm(vm));
}
}
this.packetTosendLocal.clear();
double avband=this.bandwidth/packetTosendGlobal.size();
for(HostPacket hs:packetTosendGlobal)
{
double delay=(1000*hs.pkt.data)/avband;
Constants.totaldatatransfer+=hs.pkt.data;
//System.out.println(hs.pkt.virtualsendid+" "+hs.pkt.virtualrecvid+" "+hs.pkt.data);
CloudSim.send(this.getDatacenter().getId(), this.sw.getId(), delay, CloudSimTags.Network_Event_UP, hs);
//send to switch with delay
}
this.packetTosendGlobal.clear();
}
@SuppressWarnings("unchecked")
public double getMaxUtilizationAmongVmsPes(Vm vm) {
return PeList.getMaxUtilizationAmongVmsPes((List<Pe>) getPeList(), vm);
}
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.List;
import org.cloudbus.cloudsim.CloudletScheduler;
import org.cloudbus.cloudsim.Vm;
public class NetworkVm extends Vm implements Comparable{
public NetworkVm(int id, int userId, double mips, int pesNumber, int ram,
long bw, long size, String vmm, CloudletScheduler cloudletScheduler) {
super(id, userId, mips, pesNumber, ram, bw, size, vmm, cloudletScheduler);
// TODO Auto-generated constructor stub
cloudletlist=new ArrayList<NetworkCloudlet>();
}
public ArrayList<NetworkCloudlet> cloudletlist;
int type;
public ArrayList<NetPacket> recvPktlist;
public double memory;
public boolean flagfree;//if true it is free
public double finishtime;
public boolean isFree()
{
return flagfree;
}
public int compareTo(Object arg0) {
NetworkVm hs=(NetworkVm)arg0;
if(hs.finishtime>this.finishtime) return -1;
if(hs.finishtime<this.finishtime) return 1;
return 0;
}
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import org.cloudbus.cloudsim.core.CloudSim;
import org.cloudbus.cloudsim.core.CloudSimTags;
import org.cloudbus.cloudsim.core.SimEvent;
import org.cloudbus.cloudsim.core.predicates.PredicateType;
public class RootSwitch extends Switch{
public RootSwitch(String name, int level, NetworkDatacenter dc) {
super(name, level, dc);
downlinkswitchpktlist=new HashMap<Integer,List<HostPacket>>();
downlinkswitches=new ArrayList<Switch>();
downlinkbandwidth=Constants.BandWidthAggRoot;
latency=Constants.SwitchingDelayRoot;
numport=Constants.RootSwitchPort;
// TODO Auto-generated constructor stub
}
private void processpacket_up(SimEvent ev) {
// TODO Auto-generated method stub
//packet coming from down level router.
//has to send up
//check which switch to forward to
//add packet in the switch list
//
//int src=ev.getSource();
HostPacket hspkt=(HostPacket) ev.getData();
int recvVMid=hspkt.pkt.reciever;
CloudSim.cancelAll(getId(), new PredicateType(CloudSimTags.Network_Event_send));
schedule(getId(),this.switching_delay, CloudSimTags.Network_Event_send);
if(this.level==Constants.ROOT_LEVEL)
{
//get id of edge router
int edgeswitchid=dc.VmToSwitchid.get(recvVMid);
//search which aggregate switch has it
int aggSwtichid=-1;;
for(Switch sw:this.downlinkswitches)
{
for(Switch edge:sw.downlinkswitches)
{
if(edge.getId()==edgeswitchid){
aggSwtichid=sw.getId();
break;
}
}
}
if(aggSwtichid<0) System.out.println(" No destination for this packet");
else{
List<HostPacket> pktlist=this.downlinkswitchpktlist.get(aggSwtichid);
if(pktlist==null){
pktlist=new ArrayList<HostPacket>();
this.downlinkswitchpktlist.put(aggSwtichid, pktlist);
}
pktlist.add(hspkt);
}
}
}
}
package org.cloudbus.cloudsim.network.datacenter;
public class TaskStage {
public TaskStage(int type, double data, double time, double stageid,long memory,
int peer,int vpeer) {
super();
this.type = type;
this.data = data;
this.time = time;
this.stageid = stageid;
this.memory = memory;
this.peer = peer;
this.vpeer=vpeer;
}
int vpeer;
int type;//execution, recv, send,
double data;//data generated or send or recv
double time;//execution time for this stage
double stageid;
long memory;
int peer;//from whom data needed to be recieved or send
}
/*
* Title: CloudSim Toolkit
* Description: CloudSim (Cloud Simulation) Toolkit for Modeling and Simulation of Clouds
* Licence: GPL - http://www.gnu.org/copyleft/gpl.html
*
* Copyright (c) 2009-2010, The University of Melbourne, Australia
*/
package org.cloudbus.cloudsim.network.datacenter;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.cloudbus.cloudsim.Host;
import org.cloudbus.cloudsim.Log;
import org.cloudbus.cloudsim.Vm;
import org.cloudbus.cloudsim.VmAllocationPolicy;
import org.cloudbus.cloudsim.core.CloudSim;
import org.cloudbus.cloudsim.power.PowerHost;
/**
* VmAllocationPolicySimple is an VmAllocationPolicy that
* chooses, as the host for a VM, the host with
* less PEs in use.
*
* @author Rodrigo N. Calheiros
* @author Anton Beloglazov
* @since CloudSim Toolkit 1.0
*/
public class VmHPCAllocationPolicySimple extends VmAllocationPolicy {
/** The vm table. */
private Map<String, Host> vmTable;
/** The used pes. */
private Map<String, Integer> usedPes;
/** The free pes. */
private List<Integer> freePes;
/**
* Creates the new VmAllocationPolicySimple object.
*
* @param list the list
*
* @pre $none
* @post $none
*/
public VmHPCAllocationPolicySimple(List<? extends Host> list) {
super(list);
setFreePes(new ArrayList<Integer>());
for (Host host : getHostList()) {
getFreePes().add(host.getPesNumber());
}
setVmTable(new HashMap<String, Host>());
setUsedPes(new HashMap<String, Integer>());
}
/**
* Allocates a host for a given VM.
*
* @param vm VM specification
*
* @return $true if the host could be allocated; $false otherwise
*
* @pre $none
* @post $none
*/
@Override
public boolean allocateHostForVm(Vm vm) {
NetworkHost allocatedHost = findHostForVm(vm);
if (allocatedHost != null && allocatedHost.vmCreate(vm)) { //if vm has been succesfully created in the host
getVmTable().put(vm.getUid(), allocatedHost);
if (!Log.isDisabled()) {
Log.print(String.format("%.2f: VM #" + vm.getId() + " has been allocated to the host #" + allocatedHost.getId() + "\n", CloudSim.clock()));
}
return true;
}
return false;
}
public NetworkHost findHostForVm(Vm vm) {
double minPower = Double.MAX_VALUE;
NetworkHost allocatedHost = null;
for (NetworkHost host : this.<NetworkHost>getHostList()) {
if (host.isSuitableForVm(vm)) {
double maxUtilization = getMaxUtilizationAfterAllocation(host, vm);
if ((!vm.isRecentlyCreated() && maxUtilization > 1) || (vm.isRecentlyCreated() && maxUtilization > 1.0)) {
continue;
}
allocatedHost = host;
}
}
return allocatedHost;
}
protected double getMaxUtilizationAfterAllocation(NetworkHost host, Vm vm) {
List<Double> allocatedMipsForVm = null;
NetworkHost allocatedHost = (NetworkHost) vm.getHost();
if (allocatedHost != null) {
allocatedMipsForVm = vm.getHost().getAllocatedMipsForVm(vm);
}
if (!host.allocatePesForVm(vm, vm.getCurrentRequestedMips())) {
return -1;
}
double maxUtilization = host.getMaxUtilizationAmongVmsPes(vm);
host.deallocatePesForVm(vm);
if (allocatedHost != null && allocatedMipsForVm != null) {
vm.getHost().allocatePesForVm(vm, allocatedMipsForVm);
}
return maxUtilization;
}
/**
* Releases the host used by a VM.
*
* @param vm the vm
*
* @pre $none
* @post none
*/
@Override
public void deallocateHostForVm(Vm vm) {
Host host = getVmTable().remove(vm.getUid());
int idx = getHostList().indexOf(host);
int pes = getUsedPes().remove(vm.getUid());
if (host != null) {
host.vmDestroy(vm);
getFreePes().set(idx, getFreePes().get(idx) + pes);
}
}
/**
* Gets the host that is executing the given VM belonging to the
* given user.
*
* @param vm the vm
*
* @return the Host with the given vmID and userID; $null if not found
*
* @pre $none
* @post $none
*/
@Override
public Host getHost(Vm vm) {
return getVmTable().get(vm.getUid());
}
/**
* Gets the host that is executing the given VM belonging to the
* given user.
*
* @param vmId the vm id
* @param userId the user id
*
* @return the Host with the given vmID and userID; $null if not found
*
* @pre $none
* @post $none
*/
@Override
public Host getHost(int vmId, int userId) {
return getVmTable().get(Vm.getUid(userId, vmId));
}
/**
* Gets the vm table.
*
* @return the vm table
*/
public Map<String, Host> getVmTable() {
return vmTable;
}
/**
* Sets the vm table.
*
* @param vmTable the vm table
*/
protected void setVmTable(Map<String, Host> vmTable) {
this.vmTable = vmTable;
}
/**
* Gets the used pes.
*
* @return the used pes
*/
protected Map<String, Integer> getUsedPes() {
return usedPes;
}
/**
* Sets the used pes.
*
* @param usedPes the used pes
*/
protected void setUsedPes(Map<String, Integer> usedPes) {
this.usedPes = usedPes;
}
/**
* Gets the free pes.
*
* @return the free pes
*/
protected List<Integer> getFreePes() {
return freePes;
}
/**
* Sets the free pes.
*
* @param freePes the new free pes
*/
protected void setFreePes(List<Integer> freePes) {
this.freePes = freePes;
}
/* (non-Javadoc)
* @see cloudsim.VmAllocationPolicy#optimizeAllocation(double, cloudsim.VmList, double)
*/
@Override
public List<Map<String, Object>> optimizeAllocation(List<? extends Vm> vmList) {
// TODO Auto-generated method stub
return null;
}
/* (non-Javadoc)
* @see org.cloudbus.cloudsim.VmAllocationPolicy#allocateHostForVm(org.cloudbus.cloudsim.Vm, org.cloudbus.cloudsim.Host)
*/
@Override
public boolean allocateHostForVm(Vm vm, Host host) {
if (host.vmCreate(vm)) { //if vm has been succesfully created in the host
getVmTable().put(vm.getUid(), host);
Log.formatLine("%.2f: VM #" + vm.getId() + " has been allocated to the host #" + host.getId(), CloudSim.clock());
return true;
}
return false;
}
}
package org.cloudbus.cloudsim.network.datacenter;
import java.util.List;
import org.cloudbus.cloudsim.UtilizationModel;
import org.cloudbus.cloudsim.UtilizationModelFull;
import org.cloudbus.cloudsim.core.CloudSim;
public class WorkflowApp extends AppCloudlet{
public WorkflowApp(int type, int appID, double deadline,
int numbervm, int userId) {
super(type, appID, deadline, numbervm, userId);
// TODO Auto-generated constructor stub
this.exeTime=100;
this.numbervm=3;
}
public void createCloudletList(List<Integer> vmIdList){
long fileSize = Constants.FILE_SIZE;
long outputSize = Constants.OUTPUT_SIZE;
int pesNumber = Constants.PES_NUMBER;
int memory=100;
UtilizationModel utilizationModel = new UtilizationModelFull();
int i=0;
//Task A
NetworkCloudlet cl = new NetworkCloudlet(Constants.currentCloudletId, 0, 1, fileSize, outputSize, memory, utilizationModel, utilizationModel, utilizationModel);
cl.numStage=2;
Constants.currentCloudletId++;
cl.setUserId(userId);
cl.submittime=CloudSim.clock();
cl.currStagenum=-1;
cl.setVmId(vmIdList.get(i));
//first stage: big computation
cl.stages.add(new TaskStage(Constants.EXECUTION, 0, 1000*0.8, 0, memory, vmIdList.get(0),cl.getCloudletId()));
cl.stages.add(new TaskStage(Constants.WAIT_SEND, 1000, 0, 1, memory, vmIdList.get(2),cl.getCloudletId()+2));
clist.add(cl);
i++;
//Task B
NetworkCloudlet clb = new NetworkCloudlet(Constants.currentCloudletId, 0, 1, fileSize, outputSize, memory, utilizationModel, utilizationModel, utilizationModel);
clb.numStage=2;
Constants.currentCloudletId++;
clb.setUserId(userId);
clb.submittime=CloudSim.clock();
clb.currStagenum=-1;
clb.setVmId(vmIdList.get(i));
//first stage: big computation
clb.stages.add(new TaskStage(Constants.EXECUTION, 0, 1000*0.8, 0, memory, vmIdList.get(1),clb.getCloudletId()));
clb.stages.add(new TaskStage(Constants.WAIT_SEND, 1000, 0, 1, memory, vmIdList.get(2),clb.getCloudletId()+1));
clist.add(clb);
i++;
//Task C
NetworkCloudlet clc = new NetworkCloudlet(Constants.currentCloudletId, 0, 1, fileSize, outputSize, memory, utilizationModel, utilizationModel, utilizationModel);
clc.numStage=2;
Constants.currentCloudletId++;
clc.setUserId(userId);
clc.submittime=CloudSim.clock();
clc.currStagenum=-1;
clc.setVmId(vmIdList.get(i));
//first stage: big computation
clc.stages.add(new TaskStage(Constants.WAIT_RECV, 1000, 0, 0, memory, vmIdList.get(0),cl.getCloudletId()));
clc.stages.add(new TaskStage(Constants.WAIT_RECV, 1000, 0, 1, memory, vmIdList.get(1),cl.getCloudletId()+1));
clc.stages.add(new TaskStage(Constants.EXECUTION, 0, 1000*0.8, 1, memory, vmIdList.get(0),clc.getCloudletId()));
clist.add(clc);
}
}
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment