Extend support of CloudStack-managed storage to KVM

This commit is contained in:
Mike Tutkowski 2013-09-19 15:44:03 -06:00
parent 11441a3672
commit 858ce76665
23 changed files with 1324 additions and 311 deletions

View File

@ -18,21 +18,35 @@
*/ */
package com.cloud.agent.api.to; package com.cloud.agent.api.to;
import java.util.Map;
import com.cloud.storage.Volume; import com.cloud.storage.Volume;
public class DiskTO { public class DiskTO {
public static final String CHAP_INITIATOR_USERNAME = "chapInitiatorUsername";
public static final String CHAP_INITIATOR_SECRET = "chapInitiatorSecret";
public static final String CHAP_TARGET_USERNAME = "chapTargetUsername";
public static final String CHAP_TARGET_SECRET = "chapTargetSecret";
public static final String MANAGED = "managed";
public static final String IQN = "iqn";
public static final String STORAGE_HOST = "storageHost";
public static final String STORAGE_PORT = "storagePort";
public static final String VOLUME_SIZE = "volumeSize";
private DataTO data; private DataTO data;
private Long diskSeq; private Long diskSeq;
private String vdiUuid; private String path;
private Volume.Type type; private Volume.Type type;
private Map<String, String> _details;
public DiskTO() { public DiskTO() {
} }
public DiskTO(DataTO data, Long diskSeq, String vdiUuid, Volume.Type type) { public DiskTO(DataTO data, Long diskSeq, String path, Volume.Type type) {
this.data = data; this.data = data;
this.diskSeq = diskSeq; this.diskSeq = diskSeq;
this.vdiUuid = vdiUuid; this.path = path;
this.type = type; this.type = type;
} }
@ -52,12 +66,12 @@ public class DiskTO {
this.diskSeq = diskSeq; this.diskSeq = diskSeq;
} }
public String getVdiUuid() { public String getPath() {
return vdiUuid; return path;
} }
public void setVdiUuid(String vdiUuid) { public void setPath(String path) {
this.vdiUuid = vdiUuid; this.path = path;
} }
public Volume.Type getType() { public Volume.Type getType() {
@ -67,4 +81,12 @@ public class DiskTO {
public void setType(Volume.Type type) { public void setType(Volume.Type type) {
this.type = type; this.type = type;
} }
public void setDetails(Map<String, String> details) {
_details = details;
}
public Map<String, String> getDetails() {
return _details;
}
} }

View File

@ -16,26 +16,33 @@
// under the License. // under the License.
package com.cloud.agent.api; package com.cloud.agent.api;
import com.cloud.agent.api.to.VirtualMachineTO;
public class MigrateCommand extends Command { public class MigrateCommand extends Command {
String vmName; String vmName;
String destIp; String destIp;
String hostGuid; String hostGuid;
boolean isWindows; boolean isWindows;
VirtualMachineTO vmTO;
protected MigrateCommand() { protected MigrateCommand() {
} }
public MigrateCommand(String vmName, String destIp, boolean isWindows) { public MigrateCommand(String vmName, String destIp, boolean isWindows, VirtualMachineTO vmTO) {
this.vmName = vmName; this.vmName = vmName;
this.destIp = destIp; this.destIp = destIp;
this.isWindows = isWindows; this.isWindows = isWindows;
this.vmTO = vmTO;
} }
public boolean isWindows() { public boolean isWindows() {
return isWindows; return isWindows;
} }
public VirtualMachineTO getVirtualMachine() {
return vmTO;
}
public String getDestinationIp() { public String getDestinationIp() {
return destIp; return destIp;
} }

View File

@ -16,11 +16,14 @@
// under the License. // under the License.
package com.cloud.agent.api; package com.cloud.agent.api;
import java.util.Map;
import com.cloud.agent.api.to.VirtualMachineTO; import com.cloud.agent.api.to.VirtualMachineTO;
public class StartAnswer extends Answer { public class StartAnswer extends Answer {
VirtualMachineTO vm; VirtualMachineTO vm;
String host_guid; String host_guid;
Map<String, String> _iqnToPath;
protected StartAnswer() { protected StartAnswer() {
} }
@ -54,4 +57,12 @@ public class StartAnswer extends Answer {
public String getHost_guid() { public String getHost_guid() {
return host_guid; return host_guid;
} }
public void setIqnToPath(Map<String, String> iqnToPath) {
_iqnToPath = iqnToPath;
}
public Map<String, String> getIqnToPath() {
return _iqnToPath;
}
} }

View File

@ -24,14 +24,6 @@ import com.cloud.agent.api.to.DiskTO;
public final class AttachCommand extends Command implements StorageSubSystemCommand { public final class AttachCommand extends Command implements StorageSubSystemCommand {
private DiskTO disk; private DiskTO disk;
private String vmName; private String vmName;
private String _storageHost;
private int _storagePort;
private boolean _managed;
private String _iScsiName;
private String _chapInitiatorUsername;
private String _chapInitiatorPassword;
private String _chapTargetUsername;
private String _chapTargetPassword;
public AttachCommand(DiskTO disk, String vmName) { public AttachCommand(DiskTO disk, String vmName) {
super(); super();
@ -59,68 +51,4 @@ public final class AttachCommand extends Command implements StorageSubSystemComm
public void setVmName(String vmName) { public void setVmName(String vmName) {
this.vmName = vmName; this.vmName = vmName;
} }
public void setStorageHost(String storageHost) {
_storageHost = storageHost;
}
public String getStorageHost() {
return _storageHost;
}
public void setStoragePort(int storagePort) {
_storagePort = storagePort;
}
public int getStoragePort() {
return _storagePort;
}
public void setManaged(boolean managed) {
_managed = managed;
}
public boolean isManaged() {
return _managed;
}
public void set_iScsiName(String iScsiName) {
this._iScsiName = iScsiName;
}
public String get_iScsiName() {
return _iScsiName;
}
public void setChapInitiatorUsername(String chapInitiatorUsername) {
_chapInitiatorUsername = chapInitiatorUsername;
}
public String getChapInitiatorUsername() {
return _chapInitiatorUsername;
}
public void setChapInitiatorPassword(String chapInitiatorPassword) {
_chapInitiatorPassword = chapInitiatorPassword;
}
public String getChapInitiatorPassword() {
return _chapInitiatorPassword;
}
public void setChapTargetUsername(String chapTargetUsername) {
_chapTargetUsername = chapTargetUsername;
}
public String getChapTargetUsername() {
return _chapTargetUsername;
}
public void setChapTargetPassword(String chapTargetPassword) {
_chapTargetPassword = chapTargetPassword;
}
public String getChapTargetPassword() {
return _chapTargetPassword;
}
} }

View File

@ -51,7 +51,6 @@ import org.apache.cloudstack.storage.datastore.db.PrimaryDataStoreDao;
import org.apache.cloudstack.storage.datastore.db.StoragePoolVO; import org.apache.cloudstack.storage.datastore.db.StoragePoolVO;
import org.apache.cloudstack.storage.to.VolumeObjectTO; import org.apache.cloudstack.storage.to.VolumeObjectTO;
import org.apache.cloudstack.utils.identity.ManagementServerNode; import org.apache.cloudstack.utils.identity.ManagementServerNode;
import org.apache.log4j.Logger;
import com.cloud.agent.AgentManager; import com.cloud.agent.AgentManager;
import com.cloud.agent.Listener; import com.cloud.agent.Listener;
@ -844,6 +843,8 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
VirtualMachineTO vmTO = hvGuru.implement(vmProfile); VirtualMachineTO vmTO = hvGuru.implement(vmProfile);
handlePath(vmTO.getDisks(), vm.getHypervisorType());
cmds = new Commands(Command.OnError.Stop); cmds = new Commands(Command.OnError.Stop);
cmds.addCommand(new StartCommand(vmTO, dest.getHost(), getExecuteInSequence())); cmds.addCommand(new StartCommand(vmTO, dest.getHost(), getExecuteInSequence()));
@ -862,6 +863,7 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
startAnswer = cmds.getAnswer(StartAnswer.class); startAnswer = cmds.getAnswer(StartAnswer.class);
if (startAnswer != null && startAnswer.getResult()) { if (startAnswer != null && startAnswer.getResult()) {
handlePath(vmTO.getDisks(), startAnswer.getIqnToPath());
String host_guid = startAnswer.getHost_guid(); String host_guid = startAnswer.getHost_guid();
if (host_guid != null) { if (host_guid != null) {
HostVO finalHost = _resourceMgr.findHostByGuid(host_guid); HostVO finalHost = _resourceMgr.findHostByGuid(host_guid);
@ -975,14 +977,63 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
} }
} }
// for managed storage on KVM, need to make sure the path field of the volume in question is populated with the IQN
private void handlePath(DiskTO[] disks, HypervisorType hypervisorType) {
if (hypervisorType != HypervisorType.KVM) {
return;
}
if (disks != null) {
for (DiskTO disk : disks) {
Map<String, String> details = disk.getDetails();
boolean isManaged = details != null && Boolean.parseBoolean(details.get(DiskTO.MANAGED));
if (isManaged && disk.getPath() == null) {
Long volumeId = disk.getData().getId();
VolumeVO volume = _volsDao.findById(volumeId);
disk.setPath(volume.get_iScsiName());
volume.setPath(volume.get_iScsiName());
_volsDao.update(volumeId, volume);
}
}
}
}
// for managed storage on XenServer and VMware, need to update the DB with a path if the VDI/VMDK file was newly created
private void handlePath(DiskTO[] disks, Map<String, String> iqnToPath) {
if (disks != null) {
for (DiskTO disk : disks) {
Map<String, String> details = disk.getDetails();
boolean isManaged = details != null && Boolean.parseBoolean(details.get(DiskTO.MANAGED));
if (isManaged && disk.getPath() == null) {
Long volumeId = disk.getData().getId();
VolumeVO volume = _volsDao.findById(volumeId);
String iScsiName = volume.get_iScsiName();
String path = iqnToPath.get(iScsiName);
volume.setPath(path);
_volsDao.update(volumeId, volume);
}
}
}
}
private void syncDiskChainChange(StartAnswer answer) { private void syncDiskChainChange(StartAnswer answer) {
VirtualMachineTO vmSpec = answer.getVirtualMachine(); VirtualMachineTO vmSpec = answer.getVirtualMachine();
for(DiskTO disk : vmSpec.getDisks()) { for(DiskTO disk : vmSpec.getDisks()) {
if(disk.getType() != Volume.Type.ISO) { if(disk.getType() != Volume.Type.ISO) {
VolumeObjectTO vol = (VolumeObjectTO)disk.getData(); VolumeObjectTO vol = (VolumeObjectTO)disk.getData();
VolumeVO volume = _volsDao.findById(vol.getId());
volumeMgr.updateVolumeDiskChain(vol.getId(), vol.getPath(), vol.getChainInfo()); // Use getPath() from VolumeVO to get a fresh copy of what's in the DB.
// Before doing this, in a certain situation, getPath() from VolumeObjectTO
// returned null instead of an actual path (because it was out of date with the DB).
volumeMgr.updateVolumeDiskChain(vol.getId(), volume.getPath(), vol.getChainInfo());
} }
} }
} }
@ -1523,7 +1574,7 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
boolean migrated = false; boolean migrated = false;
try { try {
boolean isWindows = _guestOsCategoryDao.findById(_guestOsDao.findById(vm.getGuestOSId()).getCategoryId()).getName().equalsIgnoreCase("Windows"); boolean isWindows = _guestOsCategoryDao.findById(_guestOsDao.findById(vm.getGuestOSId()).getCategoryId()).getName().equalsIgnoreCase("Windows");
MigrateCommand mc = new MigrateCommand(vm.getInstanceName(), dest.getHost().getPrivateIpAddress(), isWindows); MigrateCommand mc = new MigrateCommand(vm.getInstanceName(), dest.getHost().getPrivateIpAddress(), isWindows, to);
mc.setHostGuid(dest.getHost().getGuid()); mc.setHostGuid(dest.getHost().getGuid());
try { try {
@ -3150,7 +3201,7 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
boolean migrated = false; boolean migrated = false;
try { try {
boolean isWindows = _guestOsCategoryDao.findById(_guestOsDao.findById(vm.getGuestOSId()).getCategoryId()).getName().equalsIgnoreCase("Windows"); boolean isWindows = _guestOsCategoryDao.findById(_guestOsDao.findById(vm.getGuestOSId()).getCategoryId()).getName().equalsIgnoreCase("Windows");
MigrateCommand mc = new MigrateCommand(vm.getInstanceName(), dest.getHost().getPrivateIpAddress(), isWindows); MigrateCommand mc = new MigrateCommand(vm.getInstanceName(), dest.getHost().getPrivateIpAddress(), isWindows, to);
mc.setHostGuid(dest.getHost().getGuid()); mc.setHostGuid(dest.getHost().getGuid());
try { try {

View File

@ -32,6 +32,7 @@ import javax.naming.ConfigurationException;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.cloudstack.engine.orchestration.service.VolumeOrchestrationService; import org.apache.cloudstack.engine.orchestration.service.VolumeOrchestrationService;
import org.apache.cloudstack.engine.subsystem.api.storage.ChapInfo;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStore; import org.apache.cloudstack.engine.subsystem.api.storage.DataStore;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreManager; import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreManager;
import org.apache.cloudstack.engine.subsystem.api.storage.SnapshotDataFactory; import org.apache.cloudstack.engine.subsystem.api.storage.SnapshotDataFactory;
@ -886,7 +887,12 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
for (VolumeVO vol : vols) { for (VolumeVO vol : vols) {
DataTO volTO = volFactory.getVolume(vol.getId()).getTO(); DataTO volTO = volFactory.getVolume(vol.getId()).getTO();
DiskTO disk = new DiskTO(volTO, vol.getDeviceId(), null, vol.getVolumeType()); DiskTO disk = new DiskTO(volTO, vol.getDeviceId(), vol.getPath(), vol.getVolumeType());
VolumeInfo volumeInfo = volFactory.getVolume(vol.getId());
DataStore dataStore = dataStoreMgr.getDataStore(vol.getPoolId(), DataStoreRole.Primary);
disk.setDetails(getDetails(volumeInfo, dataStore));
vm.addDisk(disk); vm.addDisk(disk);
} }
@ -897,6 +903,29 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
} }
} }
private Map<String, String> getDetails(VolumeInfo volumeInfo, DataStore dataStore) {
Map<String, String> details = new HashMap<String, String>();
StoragePoolVO storagePool = _storagePoolDao.findById(dataStore.getId());
details.put(DiskTO.MANAGED, String.valueOf(storagePool.isManaged()));
details.put(DiskTO.STORAGE_HOST, storagePool.getHostAddress());
details.put(DiskTO.STORAGE_PORT, String.valueOf(storagePool.getPort()));
details.put(DiskTO.VOLUME_SIZE, String.valueOf(volumeInfo.getSize()));
details.put(DiskTO.IQN, volumeInfo.get_iScsiName());
ChapInfo chapInfo = volService.getChapInfo(volumeInfo, dataStore);
if (chapInfo != null) {
details.put(DiskTO.CHAP_INITIATOR_USERNAME, chapInfo.getInitiatorUsername());
details.put(DiskTO.CHAP_INITIATOR_SECRET, chapInfo.getInitiatorSecret());
details.put(DiskTO.CHAP_TARGET_USERNAME, chapInfo.getTargetUsername());
details.put(DiskTO.CHAP_TARGET_SECRET, chapInfo.getTargetSecret());
}
return details;
}
private static enum VolumeTaskType { private static enum VolumeTaskType {
RECREATE, NOP, MIGRATE RECREATE, NOP, MIGRATE
} }
@ -1084,7 +1113,12 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
vol = result.first(); vol = result.first();
} }
DataTO volumeTO = volFactory.getVolume(vol.getId()).getTO(); DataTO volumeTO = volFactory.getVolume(vol.getId()).getTO();
DiskTO disk = new DiskTO(volumeTO, vol.getDeviceId(), null, vol.getVolumeType()); DiskTO disk = new DiskTO(volumeTO, vol.getDeviceId(), vol.getPath(), vol.getVolumeType());
VolumeInfo volumeInfo = volFactory.getVolume(vol.getId());
DataStore dataStore = dataStoreMgr.getDataStore(vol.getPoolId(), DataStoreRole.Primary);
disk.setDetails(getDetails(volumeInfo, dataStore));
vm.addDisk(disk); vm.addDisk(disk);
} }
} }

View File

@ -1203,8 +1203,6 @@ ServerResource {
return execute((AttachIsoCommand) cmd); return execute((AttachIsoCommand) cmd);
} else if (cmd instanceof AttachVolumeCommand) { } else if (cmd instanceof AttachVolumeCommand) {
return execute((AttachVolumeCommand) cmd); return execute((AttachVolumeCommand) cmd);
} else if (cmd instanceof StopCommand) {
return execute((StopCommand) cmd);
} else if (cmd instanceof CheckConsoleProxyLoadCommand) { } else if (cmd instanceof CheckConsoleProxyLoadCommand) {
return execute((CheckConsoleProxyLoadCommand) cmd); return execute((CheckConsoleProxyLoadCommand) cmd);
} else if (cmd instanceof WatchConsoleProxyLoadCommand) { } else if (cmd instanceof WatchConsoleProxyLoadCommand) {
@ -2905,6 +2903,8 @@ ServerResource {
*/ */
destDomain = dm.migrate(dconn, (1 << 0) | (1 << 3), xmlDesc, vmName, "tcp:" destDomain = dm.migrate(dconn, (1 << 0) | (1 << 3), xmlDesc, vmName, "tcp:"
+ cmd.getDestinationIp(), _migrateSpeed); + cmd.getDestinationIp(), _migrateSpeed);
_storagePoolMgr.disconnectPhysicalDisksViaVmSpec(cmd.getVirtualMachine());
} catch (LibvirtException e) { } catch (LibvirtException e) {
s_logger.debug("Can't migrate domain: " + e.getMessage()); s_logger.debug("Can't migrate domain: " + e.getMessage());
result = e.getMessage(); result = e.getMessage();
@ -2953,6 +2953,9 @@ ServerResource {
} }
NicTO[] nics = vm.getNics(); NicTO[] nics = vm.getNics();
boolean success = false;
try { try {
Connect conn = LibvirtConnection.getConnectionByVmName(vm.getName()); Connect conn = LibvirtConnection.getConnectionByVmName(vm.getName());
for (NicTO nic : nics) { for (NicTO nic : nics) {
@ -2967,10 +2970,14 @@ ServerResource {
} }
} }
_storagePoolMgr.connectPhysicalDisksViaVmSpec(vm);
synchronized (_vms) { synchronized (_vms) {
_vms.put(vm.getName(), State.Migrating); _vms.put(vm.getName(), State.Migrating);
} }
success = true;
return new PrepareForMigrationAnswer(cmd); return new PrepareForMigrationAnswer(cmd);
} catch (LibvirtException e) { } catch (LibvirtException e) {
return new PrepareForMigrationAnswer(cmd, e.toString()); return new PrepareForMigrationAnswer(cmd, e.toString());
@ -2978,6 +2985,10 @@ ServerResource {
return new PrepareForMigrationAnswer(cmd, e.toString()); return new PrepareForMigrationAnswer(cmd, e.toString());
} catch (URISyntaxException e) { } catch (URISyntaxException e) {
return new PrepareForMigrationAnswer(cmd, e.toString()); return new PrepareForMigrationAnswer(cmd, e.toString());
} finally {
if (!success) {
_storagePoolMgr.disconnectPhysicalDisksViaVmSpec(vm);
}
} }
} }
@ -3241,10 +3252,7 @@ ServerResource {
String result = stopVM(conn, vmName); String result = stopVM(conn, vmName);
if (result == null) { if (result == null) {
for (DiskDef disk : disks) { for (DiskDef disk : disks) {
if (disk.getDeviceType() == DiskDef.deviceType.CDROM cleanupDisk(disk);
&& disk.getDiskPath() != null) {
cleanupDisk(conn, disk);
}
} }
for (InterfaceDef iface: ifaces) { for (InterfaceDef iface: ifaces) {
// We don't know which "traffic type" is associated with // We don't know which "traffic type" is associated with
@ -3517,6 +3525,8 @@ ServerResource {
createVbd(conn, vmSpec, vmName, vm); createVbd(conn, vmSpec, vmName, vm);
_storagePoolMgr.connectPhysicalDisksViaVmSpec(vmSpec);
createVifs(vmSpec, vm); createVifs(vmSpec, vm);
s_logger.debug("starting " + vmName + ": " + vm.toString()); s_logger.debug("starting " + vmName + ": " + vm.toString());
@ -3597,6 +3607,9 @@ ServerResource {
_vms.remove(vmName); _vms.remove(vmName);
} }
} }
if (state != State.Running) {
_storagePoolMgr.disconnectPhysicalDisksViaVmSpec(vmSpec);
}
} }
} }
@ -3678,7 +3691,7 @@ ServerResource {
disk.defNetworkBasedDisk(physicalDisk.getPath().replace("rbd:", ""), pool.getSourceHost(), pool.getSourcePort(), disk.defNetworkBasedDisk(physicalDisk.getPath().replace("rbd:", ""), pool.getSourceHost(), pool.getSourcePort(),
pool.getAuthUserName(), pool.getUuid(), pool.getAuthUserName(), pool.getUuid(),
devId, diskBusType, diskProtocol.RBD); devId, diskBusType, diskProtocol.RBD);
} else if (pool.getType() == StoragePoolType.CLVM) { } else if (pool.getType() == StoragePoolType.CLVM || physicalDisk.getFormat() == PhysicalDiskFormat.RAW) {
disk.defBlockBasedDisk(physicalDisk.getPath(), devId, disk.defBlockBasedDisk(physicalDisk.getPath(), devId,
diskBusType); diskBusType);
} else { } else {
@ -3762,38 +3775,20 @@ ServerResource {
return new CheckSshAnswer(cmd); return new CheckSshAnswer(cmd);
} }
public boolean cleanupDisk(Connect conn, DiskDef disk) { public boolean cleanupDisk(DiskDef disk) {
// need to umount secondary storage
String path = disk.getDiskPath(); String path = disk.getDiskPath();
String poolUuid = null;
if (path.endsWith("systemvm.iso")) {
//Don't need to clean up system vm iso, as it's stored in local
return true;
}
if (path != null) {
String[] token = path.split("/");
if (token.length > 3) {
poolUuid = token[2];
}
}
if (poolUuid == null) { if (path == null) {
return true; s_logger.debug("Unable to clean up disk with null path (perhaps empty cdrom drive):" + disk);
}
try {
// we use libvirt as storage adaptor since we passed a libvirt
// connection to cleanupDisk. We pass a storage type that maps
// to libvirt adaptor.
KVMStoragePool pool = _storagePoolMgr.getStoragePool(
StoragePoolType.Filesystem, poolUuid);
if (pool != null) {
_storagePoolMgr.deleteStoragePool(pool.getType(),pool.getUuid());
}
return true;
} catch (CloudRuntimeException e) {
return false; return false;
} }
if (path.endsWith("systemvm.iso")) {
// don't need to clean up system vm ISO as it's stored in local
return true;
}
return _storagePoolMgr.disconnectPhysicalDiskByPath(path);
} }
protected synchronized String attachOrDetachISO(Connect conn, protected synchronized String attachOrDetachISO(Connect conn,
@ -3823,7 +3818,7 @@ ServerResource {
if (result == null && !isAttach) { if (result == null && !isAttach) {
for (DiskDef disk : disks) { for (DiskDef disk : disks) {
if (disk.getDeviceType() == DiskDef.deviceType.CDROM) { if (disk.getDeviceType() == DiskDef.deviceType.CDROM) {
cleanupDisk(conn, disk); cleanupDisk(disk);
} }
} }

View File

@ -17,19 +17,23 @@
package com.cloud.hypervisor.kvm.storage; package com.cloud.hypervisor.kvm.storage;
import java.util.List; import java.util.List;
import java.util.Map;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat; import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import com.cloud.storage.Storage.StoragePoolType; import com.cloud.storage.Storage.StoragePoolType;
public interface KVMStoragePool { public interface KVMStoragePool {
public KVMPhysicalDisk createPhysicalDisk(String name, public KVMPhysicalDisk createPhysicalDisk(String volumeUuid, PhysicalDiskFormat format, long size);
PhysicalDiskFormat format, long size);
public KVMPhysicalDisk createPhysicalDisk(String name, long size); public KVMPhysicalDisk createPhysicalDisk(String volumeUuid, long size);
public boolean connectPhysicalDisk(String volumeUuid, Map<String, String> details);
public KVMPhysicalDisk getPhysicalDisk(String volumeUuid); public KVMPhysicalDisk getPhysicalDisk(String volumeUuid);
public boolean deletePhysicalDisk(String uuid); public boolean disconnectPhysicalDisk(String volumeUuid);
public boolean deletePhysicalDisk(String volumeUuid);
public List<KVMPhysicalDisk> listPhysicalDisks(); public List<KVMPhysicalDisk> listPhysicalDisks();

View File

@ -18,6 +18,8 @@ package com.cloud.hypervisor.kvm.storage;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import java.util.List;
import java.util.Arrays;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.HashMap; import java.util.HashMap;
@ -25,14 +27,21 @@ import java.util.UUID;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat; import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import com.cloud.agent.api.to.VirtualMachineTO;
import com.cloud.agent.api.to.DiskTO;
import com.cloud.hypervisor.kvm.resource.KVMHABase; import com.cloud.hypervisor.kvm.resource.KVMHABase;
import com.cloud.hypervisor.kvm.resource.KVMHABase.PoolType; import com.cloud.hypervisor.kvm.resource.KVMHABase.PoolType;
import com.cloud.hypervisor.kvm.resource.KVMHAMonitor; import com.cloud.hypervisor.kvm.resource.KVMHAMonitor;
import com.cloud.storage.Storage.StoragePoolType; import com.cloud.storage.Storage.StoragePoolType;
import com.cloud.storage.StorageLayer; import com.cloud.storage.StorageLayer;
import com.cloud.storage.Volume;
import com.cloud.utils.exception.CloudRuntimeException; import com.cloud.utils.exception.CloudRuntimeException;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.cloudstack.storage.to.PrimaryDataStoreTO;
import org.apache.cloudstack.storage.to.VolumeObjectTO;
public class KVMStoragePoolManager { public class KVMStoragePoolManager {
private static final Logger s_logger = Logger private static final Logger s_logger = Logger
.getLogger(KVMStoragePoolManager.class); .getLogger(KVMStoragePoolManager.class);
@ -62,7 +71,6 @@ public class KVMStoragePoolManager {
this.poolType = poolType; this.poolType = poolType;
} }
} }
private StorageAdaptor _storageAdaptor;
private KVMHAMonitor _haMonitor; private KVMHAMonitor _haMonitor;
private final Map<String, StoragePoolInformation> _storagePools = new ConcurrentHashMap<String, StoragePoolInformation>(); private final Map<String, StoragePoolInformation> _storagePools = new ConcurrentHashMap<String, StoragePoolInformation>();
private final Map<String, StorageAdaptor> _storageMapper = new HashMap<String, StorageAdaptor>(); private final Map<String, StorageAdaptor> _storageMapper = new HashMap<String, StorageAdaptor>();
@ -89,11 +97,102 @@ public class KVMStoragePoolManager {
} }
public KVMStoragePoolManager(StorageLayer storagelayer, KVMHAMonitor monitor) { public KVMStoragePoolManager(StorageLayer storagelayer, KVMHAMonitor monitor) {
this._storageAdaptor = new LibvirtStorageAdaptor(storagelayer);
this._haMonitor = monitor; this._haMonitor = monitor;
this._storageMapper.put("libvirt", new LibvirtStorageAdaptor(storagelayer)); this._storageMapper.put("libvirt", new LibvirtStorageAdaptor(storagelayer));
// add other storage adaptors here // add other storage adaptors here
// this._storageMapper.put("newadaptor", new NewStorageAdaptor(storagelayer)); // this._storageMapper.put("newadaptor", new NewStorageAdaptor(storagelayer));
this._storageMapper.put(StoragePoolType.Iscsi.toString(), new iScsiAdmStorageAdaptor());
}
public boolean connectPhysicalDisk(StoragePoolType type, String poolUuid, String volPath, Map<String, String> details) {
StorageAdaptor adaptor = getStorageAdaptor(type);
KVMStoragePool pool = adaptor.getStoragePool(poolUuid);
return adaptor.connectPhysicalDisk(volPath, pool, details);
}
public boolean connectPhysicalDisksViaVmSpec(VirtualMachineTO vmSpec) {
boolean result = false;
final String vmName = vmSpec.getName();
List<DiskTO> disks = Arrays.asList(vmSpec.getDisks());
for (DiskTO disk : disks) {
if (disk.getType() != Volume.Type.ISO) {
VolumeObjectTO vol = (VolumeObjectTO) disk.getData();
PrimaryDataStoreTO store = (PrimaryDataStoreTO) vol.getDataStore();
KVMStoragePool pool = getStoragePool(store.getPoolType(), store.getUuid());
StorageAdaptor adaptor = getStorageAdaptor(pool.getType());
result = adaptor.connectPhysicalDisk(vol.getPath(), pool, disk.getDetails());
if (!result) {
s_logger.error("Failed to connect disks via vm spec for vm: " + vmName + " volume:" + vol.toString());
return result;
}
}
}
return result;
}
public boolean disconnectPhysicalDiskByPath(String path) {
for (Map.Entry<String, StorageAdaptor> set : _storageMapper.entrySet()) {
StorageAdaptor adaptor = set.getValue();
if (adaptor.disconnectPhysicalDiskByPath(path)) {
return true;
}
}
return false;
}
public boolean disconnectPhysicalDisksViaVmSpec(VirtualMachineTO vmSpec) {
if (vmSpec == null) {
/* CloudStack often tries to stop VMs that shouldn't be running, to ensure a known state,
for example if we lose communication with the agent and the VM is brought up elsewhere.
We may not know about these yet. This might mean that we can't use the vmspec map, because
when we restart the agent we lose all of the info about running VMs. */
s_logger.debug("disconnectPhysicalDiskViaVmSpec: Attempted to stop a VM that is not yet in our hash map");
return true;
}
boolean result = true;
final String vmName = vmSpec.getName();
List<DiskTO> disks = Arrays.asList(vmSpec.getDisks());
for (DiskTO disk : disks) {
if (disk.getType() != Volume.Type.ISO) {
s_logger.debug("Disconnecting disk " + disk.getPath());
VolumeObjectTO vol = (VolumeObjectTO) disk.getData();
PrimaryDataStoreTO store = (PrimaryDataStoreTO) vol.getDataStore();
KVMStoragePool pool = getStoragePool(store.getPoolType(), store.getUuid());
StorageAdaptor adaptor = getStorageAdaptor(pool.getType());
// if a disk fails to disconnect, still try to disconnect remaining
boolean subResult = adaptor.disconnectPhysicalDisk(vol.getPath(), pool);
if (!subResult) {
s_logger.error("Failed to disconnect disks via vm spec for vm: " + vmName + " volume:" + vol.toString());
result = false;
}
}
}
return result;
} }
public KVMStoragePool getStoragePool(StoragePoolType type, String uuid) { public KVMStoragePool getStoragePool(StoragePoolType type, String uuid) {
@ -197,6 +296,13 @@ public class KVMStoragePoolManager {
return pool; return pool;
} }
public boolean disconnectPhysicalDisk(StoragePoolType type, String poolUuid, String volPath) {
StorageAdaptor adaptor = getStorageAdaptor(type);
KVMStoragePool pool = adaptor.getStoragePool(poolUuid);
return adaptor.disconnectPhysicalDisk(volPath, pool);
}
public boolean deleteStoragePool(StoragePoolType type, String uuid) { public boolean deleteStoragePool(StoragePoolType type, String uuid) {
StorageAdaptor adaptor = getStorageAdaptor(type); StorageAdaptor adaptor = getStorageAdaptor(type);
_haMonitor.removeStoragePool(uuid); _haMonitor.removeStoragePool(uuid);

View File

@ -828,7 +828,7 @@ public class KVMStorageProcessor implements StorageProcessor {
if (result == null && !isAttach) { if (result == null && !isAttach) {
for (DiskDef disk : disks) { for (DiskDef disk : disks) {
if (disk.getDeviceType() == DiskDef.deviceType.CDROM) { if (disk.getDeviceType() == DiskDef.deviceType.CDROM) {
this.resource.cleanupDisk(conn, disk); this.resource.cleanupDisk(disk);
} }
} }
@ -972,6 +972,9 @@ public class KVMStorageProcessor implements StorageProcessor {
String vmName = cmd.getVmName(); String vmName = cmd.getVmName();
try { try {
Connect conn = LibvirtConnection.getConnectionByVmName(vmName); Connect conn = LibvirtConnection.getConnectionByVmName(vmName);
storagePoolMgr.connectPhysicalDisk(primaryStore.getPoolType(), primaryStore.getUuid(), vol.getPath(), disk.getDetails());
KVMPhysicalDisk phyDisk = storagePoolMgr.getPhysicalDisk(primaryStore.getPoolType(), primaryStore.getUuid(), vol.getPath()); KVMPhysicalDisk phyDisk = storagePoolMgr.getPhysicalDisk(primaryStore.getPoolType(), primaryStore.getUuid(), vol.getPath());
attachOrDetachDisk(conn, true, vmName, phyDisk, disk.getDiskSeq().intValue()); attachOrDetachDisk(conn, true, vmName, phyDisk, disk.getDiskSeq().intValue());
@ -994,10 +997,13 @@ public class KVMStorageProcessor implements StorageProcessor {
String vmName = cmd.getVmName(); String vmName = cmd.getVmName();
try { try {
Connect conn = LibvirtConnection.getConnectionByVmName(vmName); Connect conn = LibvirtConnection.getConnectionByVmName(vmName);
KVMPhysicalDisk phyDisk = storagePoolMgr.getPhysicalDisk(primaryStore.getPoolType(), primaryStore.getUuid(), vol.getPath()); KVMPhysicalDisk phyDisk = storagePoolMgr.getPhysicalDisk(primaryStore.getPoolType(), primaryStore.getUuid(), vol.getPath());
attachOrDetachDisk(conn, false, vmName, phyDisk, disk.getDiskSeq().intValue()); attachOrDetachDisk(conn, false, vmName, phyDisk, disk.getDiskSeq().intValue());
storagePoolMgr.disconnectPhysicalDisk(primaryStore.getPoolType(), primaryStore.getUuid(), vol.getPath());
return new DettachAnswer(disk); return new DettachAnswer(disk);
} catch (LibvirtException e) { } catch (LibvirtException e) {
s_logger.debug("Failed to attach volume: " + vol.getPath() + ", due to " + e.toString()); s_logger.debug("Failed to attach volume: " + vol.getPath() + ", due to " + e.toString());

View File

@ -706,6 +706,53 @@ public class LibvirtStorageAdaptor implements StorageAdaptor {
return disk; return disk;
} }
@Override
public boolean connectPhysicalDisk(String name, KVMStoragePool pool, Map<String, String> details) {
// this is for managed storage that needs to prep disks prior to use
return true;
}
@Override
public boolean disconnectPhysicalDisk(String uuid, KVMStoragePool pool) {
// this is for managed storage that needs to cleanup disks after use
return true;
}
@Override
public boolean disconnectPhysicalDiskByPath(String localPath) {
// we've only ever cleaned up ISOs that are NFS mounted
String poolUuid = null;
if (localPath != null && localPath.startsWith(_mountPoint)) {
String[] token = localPath.split("/");
if (token.length > 3) {
poolUuid = token[2];
}
} else {
return false;
}
if (poolUuid == null) {
return false;
}
try {
Connect conn = LibvirtConnection.getConnection();
StoragePool pool = conn.storagePoolLookupByUUIDString(poolUuid);
deleteStoragePool(poolUuid);
return true;
} catch (LibvirtException ex) {
return false;
} catch (CloudRuntimeException ex) {
return false;
}
}
@Override @Override
public boolean deletePhysicalDisk(String uuid, KVMStoragePool pool) { public boolean deletePhysicalDisk(String uuid, KVMStoragePool pool) {

View File

@ -18,6 +18,7 @@ package com.cloud.hypervisor.kvm.storage;
import java.io.File; import java.io.File;
import java.util.List; import java.util.List;
import java.util.Map;
import com.cloud.utils.exception.CloudRuntimeException; import com.cloud.utils.exception.CloudRuntimeException;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat; import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
@ -153,6 +154,16 @@ public class LibvirtStoragePool implements KVMStoragePool {
return disk; return disk;
} }
@Override
public boolean connectPhysicalDisk(String name, Map<String, String> details) {
return true;
}
@Override
public boolean disconnectPhysicalDisk(String uuid) {
return true;
}
@Override @Override
public boolean deletePhysicalDisk(String uuid) { public boolean deletePhysicalDisk(String uuid) {
return this._storageAdaptor.deletePhysicalDisk(uuid, this); return this._storageAdaptor.deletePhysicalDisk(uuid, this);

View File

@ -17,6 +17,7 @@
package com.cloud.hypervisor.kvm.storage; package com.cloud.hypervisor.kvm.storage;
import java.util.List; import java.util.List;
import java.util.Map;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat; import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import com.cloud.storage.Storage.StoragePoolType; import com.cloud.storage.Storage.StoragePoolType;
@ -25,6 +26,8 @@ public interface StorageAdaptor {
public KVMStoragePool getStoragePool(String uuid); public KVMStoragePool getStoragePool(String uuid);
// given disk path (per database) and pool, create new KVMPhysicalDisk, populate
// it with info from local disk, and return it
public KVMPhysicalDisk getPhysicalDisk(String volumeUuid, public KVMPhysicalDisk getPhysicalDisk(String volumeUuid,
KVMStoragePool pool); KVMStoragePool pool);
@ -36,6 +39,16 @@ public interface StorageAdaptor {
public KVMPhysicalDisk createPhysicalDisk(String name, KVMStoragePool pool, public KVMPhysicalDisk createPhysicalDisk(String name, KVMStoragePool pool,
PhysicalDiskFormat format, long size); PhysicalDiskFormat format, long size);
// given disk path (per database) and pool, prepare disk on host
public boolean connectPhysicalDisk(String volumePath, KVMStoragePool pool, Map<String, String> details);
// given disk path (per database) and pool, clean up disk on host
public boolean disconnectPhysicalDisk(String volumePath, KVMStoragePool pool);
// given local path to file/device (per Libvirt XML), 1) check that device is
// handled by your adaptor, return false if not. 2) clean up device, return true
public boolean disconnectPhysicalDiskByPath(String localPath);
public boolean deletePhysicalDisk(String uuid, KVMStoragePool pool); public boolean deletePhysicalDisk(String uuid, KVMStoragePool pool);
public KVMPhysicalDisk createDiskFromTemplate(KVMPhysicalDisk template, public KVMPhysicalDisk createDiskFromTemplate(KVMPhysicalDisk template,

View File

@ -0,0 +1,373 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.cloud.hypervisor.kvm.storage;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import org.apache.log4j.Logger;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import com.cloud.agent.api.to.DiskTO;
import com.cloud.storage.Storage.StoragePoolType;
import com.cloud.utils.StringUtils;
import com.cloud.utils.exception.CloudRuntimeException;
import com.cloud.utils.script.OutputInterpreter;
import com.cloud.utils.script.Script;
public class iScsiAdmStorageAdaptor implements StorageAdaptor {
private static final Logger s_logger = Logger.getLogger(iScsiAdmStorageAdaptor.class);
private static final Map<String, KVMStoragePool> _mapStorageUuidToStoragePool = new HashMap<String, KVMStoragePool>();
@Override
public KVMStoragePool createStoragePool(String uuid, String host, int port, String path, String userInfo, StoragePoolType storagePoolType) {
iScsiAdmStoragePool storagePool = new iScsiAdmStoragePool(uuid, host, port, storagePoolType, this);
_mapStorageUuidToStoragePool.put(uuid, storagePool);
return storagePool;
}
@Override
public KVMStoragePool getStoragePool(String uuid) {
return _mapStorageUuidToStoragePool.get(uuid);
}
@Override
public boolean deleteStoragePool(String uuid) {
return _mapStorageUuidToStoragePool.remove(uuid) != null;
}
@Override
public boolean deleteStoragePool(KVMStoragePool pool) {
return deleteStoragePool(pool.getUuid());
}
// called from LibvirtComputingResource.execute(CreateCommand)
// does not apply for iScsiAdmStorageAdaptor
@Override
public KVMPhysicalDisk createPhysicalDisk(String volumeUuid, KVMStoragePool pool, PhysicalDiskFormat format, long size) {
throw new UnsupportedOperationException("Creating a physical disk is not supported.");
}
@Override
public boolean connectPhysicalDisk(String volumeUuid, KVMStoragePool pool, Map<String, String> details) {
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10:3260 -o new
Script iScsiAdmCmd = new Script(true, "iscsiadm", 0, s_logger);
iScsiAdmCmd.add("-m", "node");
iScsiAdmCmd.add("-T", getIqn(volumeUuid));
iScsiAdmCmd.add("-p", pool.getSourceHost() + ":" + pool.getSourcePort());
iScsiAdmCmd.add("-o", "new");
String result = iScsiAdmCmd.execute();
if (result != null) {
s_logger.debug("Failed to add iSCSI target " + volumeUuid);
System.out.println("Failed to add iSCSI target " + volumeUuid);
return false;
}
else {
s_logger.debug("Successfully added iSCSI target " + volumeUuid);
System.out.println("Successfully added to iSCSI target " + volumeUuid);
}
String chapInitiatorUsername = details.get(DiskTO.CHAP_INITIATOR_USERNAME);
String chapInitiatorSecret = details.get(DiskTO.CHAP_INITIATOR_SECRET);
if (StringUtils.isNotBlank(chapInitiatorUsername) && StringUtils.isNotBlank(chapInitiatorSecret)) {
try {
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10:3260 --op update -n node.session.auth.authmethod -v CHAP
executeChapCommand(volumeUuid, pool, "node.session.auth.authmethod", "CHAP", null);
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10:3260 --op update -n node.session.auth.username -v username
executeChapCommand(volumeUuid, pool, "node.session.auth.username", chapInitiatorUsername, "username");
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10:3260 --op update -n node.session.auth.password -v password
executeChapCommand(volumeUuid, pool, "node.session.auth.password", chapInitiatorSecret, "password");
}
catch (Exception ex) {
return false;
}
}
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10 --login
iScsiAdmCmd = new Script(true, "iscsiadm", 0, s_logger);
iScsiAdmCmd.add("-m", "node");
iScsiAdmCmd.add("-T", getIqn(volumeUuid));
iScsiAdmCmd.add("-p", pool.getSourceHost() + ":" + pool.getSourcePort());
iScsiAdmCmd.add("--login");
result = iScsiAdmCmd.execute();
if (result != null) {
s_logger.debug("Failed to log in to iSCSI target " + volumeUuid);
System.out.println("Failed to log in to iSCSI target " + volumeUuid);
return false;
}
else {
s_logger.debug("Successfully logged in to iSCSI target " + volumeUuid);
System.out.println("Successfully logged in to iSCSI target " + volumeUuid);
}
// There appears to be a race condition where logging in to the iSCSI volume via iscsiadm
// returns success before the device has been added to the OS.
// What happens is you get logged in and the device shows up, but the device may not
// show up before we invoke Libvirt to attach the device to a VM.
// waitForDiskToBecomeAvailable(String, KVMStoragePool) invokes blockdev
// via getPhysicalDisk(String, KVMStoragePool) and checks if the size came back greater
// than 0.
// After a certain number of tries and a certain waiting period in between tries,
// this method could still return (it should not block indefinitely) (the race condition
// isn't solved here, but made highly unlikely to be a problem).
waitForDiskToBecomeAvailable(volumeUuid, pool);
return true;
}
private void waitForDiskToBecomeAvailable(String volumeUuid, KVMStoragePool pool) {
int numberOfTries = 10;
int timeBetweenTries = 1000;
while (getPhysicalDisk(volumeUuid, pool).getSize() == 0 && numberOfTries > 0) {
numberOfTries--;
try {
Thread.sleep(timeBetweenTries);
}
catch (Exception ex) {
// don't do anything
}
}
}
private void executeChapCommand(String path, KVMStoragePool pool, String nParameter, String vParameter, String detail) throws Exception {
Script iScsiAdmCmd = new Script(true, "iscsiadm", 0, s_logger);
iScsiAdmCmd.add("-m", "node");
iScsiAdmCmd.add("-T", getIqn(path));
iScsiAdmCmd.add("-p", pool.getSourceHost() + ":" + pool.getSourcePort());
iScsiAdmCmd.add("--op", "update");
iScsiAdmCmd.add("-n", nParameter);
iScsiAdmCmd.add("-v", vParameter);
String result = iScsiAdmCmd.execute();
boolean useDetail = detail != null && detail.trim().length() > 0;
detail = useDetail ? detail.trim() + " " : detail;
if (result != null) {
s_logger.debug("Failed to execute CHAP " + (useDetail ? detail : "") + "command for iSCSI target " + path + " : message = " + result);
System.out.println("Failed to execute CHAP " + (useDetail ? detail : "") + "command for iSCSI target " + path + " : message = " + result);
throw new Exception("Failed to execute CHAP " + (useDetail ? detail : "") + "command for iSCSI target " + path + " : message = " + result);
} else {
s_logger.debug("CHAP " + (useDetail ? detail : "") + "command executed successfully for iSCSI target " + path);
System.out.println("CHAP " + (useDetail ? detail : "") + "command executed successfully for iSCSI target " + path);
}
}
// example by-path: /dev/disk/by-path/ip-192.168.233.10:3260-iscsi-iqn.2012-03.com.solidfire:storagepool2-lun-0
private String getByPath(String host, String path) {
return "/dev/disk/by-path/ip-" + host + "-iscsi-" + getIqn(path) + "-lun-" + getLun(path);
}
@Override
public KVMPhysicalDisk getPhysicalDisk(String volumeUuid, KVMStoragePool pool) {
String deviceByPath = getByPath(pool.getSourceHost() + ":" + pool.getSourcePort(), volumeUuid);
KVMPhysicalDisk physicalDisk = new KVMPhysicalDisk(deviceByPath, volumeUuid, pool);
physicalDisk.setFormat(PhysicalDiskFormat.RAW);
long deviceSize = getDeviceSize(deviceByPath);
physicalDisk.setSize(deviceSize);
physicalDisk.setVirtualSize(deviceSize);
return physicalDisk;
}
private long getDeviceSize(String deviceByPath) {
Script iScsiAdmCmd = new Script(true, "blockdev", 0, s_logger);
iScsiAdmCmd.add("--getsize64", deviceByPath);
OutputInterpreter.OneLineParser parser = new OutputInterpreter.OneLineParser();
String result = iScsiAdmCmd.execute(parser);
if (result != null) {
s_logger.warn("Unable to retrieve the size of device " + deviceByPath);
return 0;
}
return Long.parseLong(parser.getLine());
}
private static String getIqn(String path) {
return getComponent(path, 1);
}
private static String getLun(String path) {
return getComponent(path, 2);
}
private static String getComponent(String path, int index) {
String[] tmp = path.split("/");
if (tmp.length != 3) {
String msg = "Wrong format for iScsi path: " + path + ". It should be formatted as '/targetIQN/LUN'.";
s_logger.warn(msg);
throw new CloudRuntimeException(msg);
}
return tmp[index].trim();
}
public boolean disconnectPhysicalDisk(String host, int port, String iqn, String lun) {
// use iscsiadm to log out of the iSCSI target and un-discover it
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10 --logout
Script iScsiAdmCmd = new Script(true, "iscsiadm", 0, s_logger);
iScsiAdmCmd.add("-m", "node");
iScsiAdmCmd.add("-T", iqn);
iScsiAdmCmd.add("-p", host + ":" + port);
iScsiAdmCmd.add("--logout");
String result = iScsiAdmCmd.execute();
if (result != null) {
s_logger.debug("Failed to log out of iSCSI target /" + iqn + "/" + lun + " : message = " + result);
System.out.println("Failed to log out of iSCSI target /" + iqn + "/" + lun + " : message = " + result);
return false;
}
else {
s_logger.debug("Successfully logged out of iSCSI target /" + iqn + "/" + lun);
System.out.println("Successfully logged out of iSCSI target /" + iqn + "/" + lun);
}
// ex. sudo iscsiadm -m node -T iqn.2012-03.com.test:volume1 -p 192.168.233.10:3260 -o delete
iScsiAdmCmd = new Script(true, "iscsiadm", 0, s_logger);
iScsiAdmCmd.add("-m", "node");
iScsiAdmCmd.add("-T", iqn);
iScsiAdmCmd.add("-p", host + ":" + port);
iScsiAdmCmd.add("-o", "delete");
result = iScsiAdmCmd.execute();
if (result != null) {
s_logger.debug("Failed to remove iSCSI target /" + iqn + "/" + lun + " : message = " + result);
System.out.println("Failed to remove iSCSI target /" + iqn + "/" + lun + " : message = " + result);
return false;
} else {
s_logger.debug("Removed iSCSI target /" + iqn + "/" + lun);
System.out.println("Removed iSCSI target /" + iqn + "/" + lun);
}
return true;
}
@Override
public boolean disconnectPhysicalDisk(String volumeUuid, KVMStoragePool pool) {
return disconnectPhysicalDisk(pool.getSourceHost(), pool.getSourcePort(), getIqn(volumeUuid), getLun(volumeUuid));
}
@Override
public boolean disconnectPhysicalDiskByPath(String localPath) {
String search1 = "/dev/disk/by-path/ip-";
String search2 = ":";
String search3 = "-iscsi-";
String search4 = "-lun-";
if (localPath.indexOf(search3) == -1) {
// this volume doesn't below to this adaptor, so just return true
return true;
}
int index = localPath.indexOf(search2);
String host = localPath.substring(search1.length(), index);
int index2 = localPath.indexOf(search3);
String port = localPath.substring(index + search2.length(), index2);
index = localPath.indexOf(search4);
String iqn = localPath.substring(index2 + search3.length(), index);
String lun = localPath.substring(index + search4.length());
return disconnectPhysicalDisk(host, Integer.parseInt(port), iqn, lun);
}
@Override
public boolean deletePhysicalDisk(String volumeUuid, KVMStoragePool pool) {
throw new UnsupportedOperationException("Deleting a physical disk is not supported.");
}
// does not apply for iScsiAdmStorageAdaptor
@Override
public List<KVMPhysicalDisk> listPhysicalDisks(String storagePoolUuid, KVMStoragePool pool) {
throw new UnsupportedOperationException("Listing disks is not supported for this configuration.");
}
@Override
public KVMPhysicalDisk createDiskFromTemplate(KVMPhysicalDisk template, String name, PhysicalDiskFormat format, long size,
KVMStoragePool destPool, int timeout) {
throw new UnsupportedOperationException("Creating a disk from a template is not yet supported for this configuration.");
}
@Override
public KVMPhysicalDisk createTemplateFromDisk(KVMPhysicalDisk disk, String name, PhysicalDiskFormat format,
long size, KVMStoragePool destPool) {
throw new UnsupportedOperationException("Creating a template from a disk is not yet supported for this configuration.");
}
@Override
public KVMPhysicalDisk copyPhysicalDisk(KVMPhysicalDisk disk, String name, KVMStoragePool destPool, int timeout) {
throw new UnsupportedOperationException("Copying a disk is not supported in this configuration.");
}
@Override
public KVMPhysicalDisk createDiskFromSnapshot(KVMPhysicalDisk snapshot, String snapshotName, String name, KVMStoragePool destPool) {
throw new UnsupportedOperationException("Creating a disk from a snapshot is not supported in this configuration.");
}
@Override
public boolean refresh(KVMStoragePool pool) {
return true;
}
@Override
public boolean createFolder(String uuid, String path) {
throw new UnsupportedOperationException("A folder cannot be created in this configuration.");
}
}

View File

@ -0,0 +1,167 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.cloud.hypervisor.kvm.storage;
import java.util.List;
import java.util.Map;
import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat;
import com.cloud.storage.Storage.StoragePoolType;
public class iScsiAdmStoragePool implements KVMStoragePool {
private String _uuid;
private String _sourceHost;
private int _sourcePort;
private StoragePoolType _storagePoolType;
private StorageAdaptor _storageAdaptor;
private String _authUsername;
private String _authSecret;
private String _sourceDir;
private String _localPath;
public iScsiAdmStoragePool(String uuid, String host, int port, StoragePoolType storagePoolType, StorageAdaptor storageAdaptor) {
_uuid = uuid;
_sourceHost = host;
_sourcePort = port;
_storagePoolType = storagePoolType;
_storageAdaptor = storageAdaptor;
}
@Override
public String getUuid() {
return _uuid;
}
@Override
public String getSourceHost() {
return _sourceHost;
}
@Override
public int getSourcePort() {
return _sourcePort;
}
@Override
public long getCapacity() {
return 0;
}
@Override
public long getUsed() {
return 0;
}
@Override
public long getAvailable() {
return 0;
}
@Override
public StoragePoolType getType() {
return _storagePoolType;
}
@Override
public PhysicalDiskFormat getDefaultFormat() {
return PhysicalDiskFormat.RAW;
}
// called from LibvirtComputingResource.copyPhysicalDisk(KVMPhysicalDisk, String, KVMStoragePool) and
// from LibvirtComputingResource.createDiskFromTemplate(KVMPhysicalDisk, String, PhysicalDiskFormat, long, KVMStoragePool)
// does not apply for iScsiAdmStoragePool
@Override
public KVMPhysicalDisk createPhysicalDisk(String name, PhysicalDiskFormat format, long size) {
throw new UnsupportedOperationException("Creating a physical disk is not supported.");
}
// called from LibvirtComputingResource.execute(CreateCommand) and
// from KVMStorageProcessor.createVolume(CreateObjectCommand)
// does not apply for iScsiAdmStoragePool
@Override
public KVMPhysicalDisk createPhysicalDisk(String name, long size) {
throw new UnsupportedOperationException("Creating a physical disk is not supported.");
}
@Override
public boolean connectPhysicalDisk(String name, Map<String, String> details) {
return this._storageAdaptor.connectPhysicalDisk(name, this, details);
}
@Override
public KVMPhysicalDisk getPhysicalDisk(String volumeUuid) {
return this._storageAdaptor.getPhysicalDisk(volumeUuid, this);
}
@Override
public boolean disconnectPhysicalDisk(String volumeUuid) {
return this._storageAdaptor.disconnectPhysicalDisk(volumeUuid, this);
}
@Override
public boolean deletePhysicalDisk(String volumeUuid) {
return this._storageAdaptor.deletePhysicalDisk(volumeUuid, this);
}
// does not apply for iScsiAdmStoragePool
@Override
public List<KVMPhysicalDisk> listPhysicalDisks() {
return this._storageAdaptor.listPhysicalDisks(_uuid, this);
}
// does not apply for iScsiAdmStoragePool
@Override
public boolean refresh() {
return this._storageAdaptor.refresh(this);
}
@Override
public boolean delete() {
return this._storageAdaptor.deleteStoragePool(this);
}
@Override
public boolean createFolder(String path) {
return this._storageAdaptor.createFolder(_uuid, path);
}
@Override
public boolean isExternalSnapshot() {
return false;
}
@Override
public String getAuthUserName() {
return _authUsername;
}
@Override
public String getAuthSecret() {
return _authSecret;
}
@Override
public String getSourceDir() {
return _sourceDir;
}
@Override
public String getLocalPath() {
return _localPath;
}
}

View File

@ -1164,19 +1164,16 @@ public class VmwareStorageProcessor implements StorageProcessor {
@Override @Override
public Answer attachVolume(AttachCommand cmd) { public Answer attachVolume(AttachCommand cmd) {
return this.attachVolume(cmd, cmd.getDisk(), true, cmd.isManaged(), cmd.getVmName(), cmd.get_iScsiName(), Map<String, String> details = cmd.getDisk().getDetails();
cmd.getStorageHost(), cmd.getStoragePort(), cmd.getChapInitiatorUsername(), cmd.getChapInitiatorPassword(), boolean isManaged = Boolean.parseBoolean(details.get(DiskTO.MANAGED));
cmd.getChapTargetUsername(), cmd.getChapTargetPassword()); String iScsiName = details.get(DiskTO.IQN);
String storageHost = details.get(DiskTO.STORAGE_HOST);
int storagePort = Integer.parseInt(details.get(DiskTO.STORAGE_PORT));
return this.attachVolume(cmd, cmd.getDisk(), true, isManaged, cmd.getVmName(), iScsiName, storageHost, storagePort);
} }
private Answer attachVolume(Command cmd, DiskTO disk, boolean isAttach, boolean isManaged, String vmName, String iScsiName, String storageHost, int storagePort) { private Answer attachVolume(Command cmd, DiskTO disk, boolean isAttach, boolean isManaged, String vmName, String iScsiName, String storageHost, int storagePort) {
return attachVolume(cmd, disk, isAttach, isManaged, vmName, iScsiName, storageHost, storagePort, null, null, null, null);
}
private Answer attachVolume(Command cmd, DiskTO disk, boolean isAttach, boolean isManaged, String vmName,
String iScsiName, String storageHost, int storagePort, String initiatorUsername, String initiatorPassword,
String targetUsername, String targetPassword) {
VolumeObjectTO volumeTO = (VolumeObjectTO)disk.getData(); VolumeObjectTO volumeTO = (VolumeObjectTO)disk.getData();
PrimaryDataStoreTO primaryStore = (PrimaryDataStoreTO)volumeTO.getDataStore(); PrimaryDataStoreTO primaryStore = (PrimaryDataStoreTO)volumeTO.getDataStore();
try { try {
@ -1191,8 +1188,11 @@ public class VmwareStorageProcessor implements StorageProcessor {
ManagedObjectReference morDs = null; ManagedObjectReference morDs = null;
if (isAttach && isManaged) { if (isAttach && isManaged) {
Map<String, String> details = disk.getDetails();
morDs = hostService.getVmfsDatastore(hyperHost, VmwareResource.getDatastoreName(iScsiName), storageHost, storagePort, morDs = hostService.getVmfsDatastore(hyperHost, VmwareResource.getDatastoreName(iScsiName), storageHost, storagePort,
VmwareResource.trimIqn(iScsiName), initiatorUsername, initiatorPassword, targetUsername, targetPassword); VmwareResource.trimIqn(iScsiName), details.get(DiskTO.CHAP_INITIATOR_USERNAME), details.get(DiskTO.CHAP_INITIATOR_SECRET),
details.get(DiskTO.CHAP_TARGET_USERNAME), details.get(DiskTO.CHAP_TARGET_SECRET));
DatastoreMO dsMo = new DatastoreMO(hostService.getServiceContext(null), morDs); DatastoreMO dsMo = new DatastoreMO(hostService.getServiceContext(null), morDs);
@ -1236,7 +1236,7 @@ public class VmwareStorageProcessor implements StorageProcessor {
} }
} }
disk.setVdiUuid(datastoreVolumePath); disk.setPath(datastoreVolumePath);
AttachAnswer answer = new AttachAnswer(disk); AttachAnswer answer = new AttachAnswer(disk);

View File

@ -1199,10 +1199,13 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
} }
} }
protected VBD createVbd(Connection conn, DiskTO volume, String vmName, VM vm, BootloaderType bootLoaderType) throws XmlRpcException, XenAPIException { protected VBD createVbd(Connection conn, DiskTO volume, String vmName, VM vm, BootloaderType bootLoaderType, VDI vdi) throws XmlRpcException, XenAPIException {
Volume.Type type = volume.getType(); Volume.Type type = volume.getType();
VDI vdi = mount(conn, vmName, volume); if (vdi == null) {
vdi = mount(conn, vmName, volume);
}
if ( vdi != null ) { if ( vdi != null ) {
Map<String, String> smConfig = vdi.getSmConfig(conn); Map<String, String> smConfig = vdi.getSmConfig(conn);
for (String key : smConfig.keySet()) { for (String key : smConfig.keySet()) {
@ -1608,6 +1611,8 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
String vmName = vmSpec.getName(); String vmName = vmSpec.getName();
State state = State.Stopped; State state = State.Stopped;
VM vm = null; VM vm = null;
// if a VDI is created, record its UUID to send back to the CS MS
Map<String, String> iqnToPath = new HashMap<String, String>();
try { try {
Set<VM> vms = VM.getByNameLabel(conn, vmName); Set<VM> vms = VM.getByNameLabel(conn, vmName);
if ( vms != null ) { if ( vms != null ) {
@ -1636,7 +1641,37 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
vm = createVmFromTemplate(conn, vmSpec, host); vm = createVmFromTemplate(conn, vmSpec, host);
for (DiskTO disk : vmSpec.getDisks()) { for (DiskTO disk : vmSpec.getDisks()) {
createVbd(conn, disk, vmName, vm, vmSpec.getBootloader()); VDI vdi = null;
if (disk.getData() instanceof VolumeObjectTO) {
Map<String, String> details = disk.getDetails();
boolean isManaged = details != null && Boolean.parseBoolean(details.get(DiskTO.MANAGED));
if (isManaged) {
String iScsiName = details.get(DiskTO.IQN);
String storageHost = details.get(DiskTO.STORAGE_HOST);
String chapInitiatorUsername = disk.getDetails().get(DiskTO.CHAP_INITIATOR_USERNAME);
String chapInitiatorSecret = disk.getDetails().get(DiskTO.CHAP_INITIATOR_SECRET);
Long volumeSize = Long.parseLong(details.get(DiskTO.VOLUME_SIZE));
String vdiNameLabel = vmName + "-DATA";
SR sr = getIscsiSR(conn, iScsiName, storageHost, iScsiName,
chapInitiatorUsername, chapInitiatorSecret, true);
vdi = getVDIbyUuid(conn, disk.getPath(), false);
if (vdi == null) {
vdi = createVdi(sr, vdiNameLabel, volumeSize);
iqnToPath.put(iScsiName, vdi.getUuid(conn));
}
else {
vdi.setNameLabel(conn, vdiNameLabel);
}
}
}
createVbd(conn, disk, vmName, vm, vmSpec.getBootloader(), vdi);
} }
if (vmSpec.getType() != VirtualMachine.Type.User) { if (vmSpec.getType() != VirtualMachine.Type.User) {
@ -1718,11 +1753,21 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
} }
state = State.Running; state = State.Running;
return new StartAnswer(cmd);
StartAnswer startAnswer = new StartAnswer(cmd);
startAnswer.setIqnToPath(iqnToPath);
return startAnswer;
} catch (Exception e) { } catch (Exception e) {
s_logger.warn("Catch Exception: " + e.getClass().toString() + " due to " + e.toString(), e); s_logger.warn("Catch Exception: " + e.getClass().toString() + " due to " + e.toString(), e);
String msg = handleVmStartFailure(conn, vmName, vm, "", e); String msg = handleVmStartFailure(conn, vmName, vm, "", e);
return new StartAnswer(cmd, msg);
StartAnswer startAnswer = new StartAnswer(cmd, msg);
startAnswer.setIqnToPath(iqnToPath);
return startAnswer;
} finally { } finally {
synchronized (_cluster.intern()) { synchronized (_cluster.intern()) {
if (state != State.Stopped) { if (state != State.Stopped) {
@ -1733,6 +1778,46 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
s_logger.debug("The VM is in stopped state, detected problem during startup : " + vmName); s_logger.debug("The VM is in stopped state, detected problem during startup : " + vmName);
} }
} }
if (state != State.Running) {
disconnectManagedVolumes(conn, vm);
}
}
}
private void disconnectManagedVolumes(Connection conn, VM vm) {
try {
Set<VBD> vbds = vm.getVBDs(conn);
for (VBD vbd : vbds) {
VDI vdi = vbd.getVDI(conn);
SR sr = null;
try {
sr = vdi.getSR(conn);
}
catch (Exception ex) {
continue;
}
if (sr.getNameLabel(conn).startsWith("/iqn.")) {
VBD.Record vbdr = vbd.getRecord(conn);
if (vbdr.currentlyAttached) {
vbd.unplug(conn);
}
vbd.destroy(conn);
vdi.setNameLabel(conn, "detached");
umount(conn, vdi);
handleSrAndVdiDetach(sr.getNameLabel(conn));
}
}
} catch (Exception ex) {
s_logger.debug(ex.getMessage());
} }
} }
@ -3976,6 +4061,8 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
try { try {
if (vm.getPowerState(conn) == VmPowerState.HALTED) { if (vm.getPowerState(conn) == VmPowerState.HALTED) {
disconnectManagedVolumes(conn, vm);
Map<String, String> platform = vm.getPlatform(conn); Map<String, String> platform = vm.getPlatform(conn);
Integer timeoffset = null; Integer timeoffset = null;
try { try {
@ -5385,7 +5472,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
if (pool.getType() == StoragePoolType.NetworkFilesystem) { if (pool.getType() == StoragePoolType.NetworkFilesystem) {
getNfsSR(conn, pool); getNfsSR(conn, pool);
} else if (pool.getType() == StoragePoolType.IscsiLUN) { } else if (pool.getType() == StoragePoolType.IscsiLUN) {
getIscsiSR(conn, pool.getUuid(), pool.getHost(), pool.getPath(), null, null); getIscsiSR(conn, pool.getUuid(), pool.getHost(), pool.getPath(), null, null, false);
} else if (pool.getType() == StoragePoolType.PreSetup) { } else if (pool.getType() == StoragePoolType.PreSetup) {
} else { } else {
return new Answer(cmd, false, "The pool type: " + pool.getType().name() + " is not supported."); return new Answer(cmd, false, "The pool type: " + pool.getType().name() + " is not supported.");
@ -6224,7 +6311,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
} }
protected SR getIscsiSR(Connection conn, String srNameLabel, String target, String path, protected SR getIscsiSR(Connection conn, String srNameLabel, String target, String path,
String chapInitiatorUsername, String chapInitiatorPassword) { String chapInitiatorUsername, String chapInitiatorPassword, boolean ignoreIntroduceException) {
synchronized (srNameLabel.intern()) { synchronized (srNameLabel.intern()) {
Map<String, String> deviceConfig = new HashMap<String, String>(); Map<String, String> deviceConfig = new HashMap<String, String>();
try { try {
@ -6273,8 +6360,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
deviceConfig.put("target", target); deviceConfig.put("target", target);
deviceConfig.put("targetIQN", targetiqn); deviceConfig.put("targetIQN", targetiqn);
if (StringUtils.isNotBlank(chapInitiatorUsername) && if (StringUtils.isNotBlank(chapInitiatorUsername) && StringUtils.isNotBlank(chapInitiatorPassword)) {
StringUtils.isNotBlank(chapInitiatorPassword)) {
deviceConfig.put("chapuser", chapInitiatorUsername); deviceConfig.put("chapuser", chapInitiatorUsername);
deviceConfig.put("chappassword", chapInitiatorPassword); deviceConfig.put("chappassword", chapInitiatorPassword);
} }
@ -6329,8 +6415,17 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
sr = SR.create(conn, host, deviceConfig, new Long(0), srNameLabel, srNameLabel, type, "user", true, sr = SR.create(conn, host, deviceConfig, new Long(0), srNameLabel, srNameLabel, type, "user", true,
smConfig); smConfig);
} else { } else {
try {
sr = SR.introduce(conn, pooluuid, srNameLabel, srNameLabel, sr = SR.introduce(conn, pooluuid, srNameLabel, srNameLabel,
type, "user", true, smConfig); type, "user", true, smConfig);
}
catch (XenAPIException ex) {
if (ignoreIntroduceException) {
return sr;
}
throw ex;
}
Set<Host> setHosts = Host.getAll(conn); Set<Host> setHosts = Host.getAll(conn);
@ -6557,6 +6652,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
Connection conn = getConnection(); Connection conn = getConnection();
boolean attach = cmd.getAttach(); boolean attach = cmd.getAttach();
String vmName = cmd.getVmName(); String vmName = cmd.getVmName();
String vdiNameLabel = vmName + "-DATA";
Long deviceId = cmd.getDeviceId(); Long deviceId = cmd.getDeviceId();
String errorMsg; String errorMsg;
@ -6571,12 +6667,12 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
if (cmd.getAttach() && cmd.isManaged()) { if (cmd.getAttach() && cmd.isManaged()) {
SR sr = getIscsiSR(conn, cmd.get_iScsiName(), cmd.getStorageHost(), cmd.get_iScsiName(), SR sr = getIscsiSR(conn, cmd.get_iScsiName(), cmd.getStorageHost(), cmd.get_iScsiName(),
cmd.getChapInitiatorUsername(), cmd.getChapInitiatorPassword()); cmd.getChapInitiatorUsername(), cmd.getChapInitiatorPassword(), true);
vdi = getVDIbyUuid(conn, cmd.getVolumePath(), false); vdi = getVDIbyUuid(conn, cmd.getVolumePath(), false);
if (vdi == null) { if (vdi == null) {
vdi = createVdi(sr, cmd.get_iScsiName(), cmd.getVolumeSize()); vdi = createVdi(sr, vdiNameLabel, cmd.getVolumeSize());
} }
} }
else { else {
@ -6632,7 +6728,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
vbd.plug(conn); vbd.plug(conn);
// Update the VDI's label to include the VM name // Update the VDI's label to include the VM name
vdi.setNameLabel(conn, vmName + "-DATA"); vdi.setNameLabel(conn, vdiNameLabel);
return new AttachVolumeAnswer(cmd, Long.parseLong(diskNumber), vdi.getUuid(conn)); return new AttachVolumeAnswer(cmd, Long.parseLong(diskNumber), vdi.getUuid(conn));
} else { } else {

View File

@ -162,6 +162,7 @@ public class XenServerStorageProcessor implements StorageProcessor {
@Override @Override
public AttachAnswer attachVolume(AttachCommand cmd) { public AttachAnswer attachVolume(AttachCommand cmd) {
String vmName = cmd.getVmName(); String vmName = cmd.getVmName();
String vdiNameLabel = vmName + "-DATA";
DiskTO disk = cmd.getDisk(); DiskTO disk = cmd.getDisk();
DataTO data = disk.getData(); DataTO data = disk.getData();
@ -170,16 +171,23 @@ public class XenServerStorageProcessor implements StorageProcessor {
VDI vdi = null; VDI vdi = null;
if (cmd.isManaged()) { Map<String, String> details = cmd.getDisk().getDetails();
SR sr = this.hypervisorResource.getIscsiSR(conn, cmd.get_iScsiName(), cmd.getStorageHost(), cmd.get_iScsiName(), boolean isManaged = Boolean.parseBoolean(details.get(DiskTO.MANAGED));
cmd.getChapInitiatorUsername(), cmd.getChapInitiatorPassword());
if (isManaged) {
String iScsiName = details.get(DiskTO.IQN);
String storageHost = details.get(DiskTO.STORAGE_HOST);
String chapInitiatorUsername = disk.getDetails().get(DiskTO.CHAP_INITIATOR_USERNAME);
String chapInitiatorSecret = disk.getDetails().get(DiskTO.CHAP_INITIATOR_SECRET);
Long volumeSize = Long.parseLong(details.get(DiskTO.VOLUME_SIZE));
SR sr = this.hypervisorResource.getIscsiSR(conn, iScsiName, storageHost, iScsiName,
chapInitiatorUsername, chapInitiatorSecret, true);
vdi = this.hypervisorResource.getVDIbyUuid(conn, data.getPath(), false); vdi = this.hypervisorResource.getVDIbyUuid(conn, data.getPath(), false);
if (vdi == null) { if (vdi == null) {
VolumeObjectTO volume = (VolumeObjectTO)data; vdi = this.hypervisorResource.createVdi(sr, vdiNameLabel, volumeSize);
vdi = this.hypervisorResource.createVdi(sr, cmd.get_iScsiName(), volume.getSize());
} }
} }
else { else {
@ -236,7 +244,7 @@ public class XenServerStorageProcessor implements StorageProcessor {
vbd.plug(conn); vbd.plug(conn);
// Update the VDI's label to include the VM name // Update the VDI's label to include the VM name
vdi.setNameLabel(conn, vmName + "-DATA"); vdi.setNameLabel(conn, vdiNameLabel);
DiskTO newDisk = new DiskTO(disk.getData(), Long.parseLong(diskNumber), vdi.getUuid(conn), disk.getType()); DiskTO newDisk = new DiskTO(disk.getData(), Long.parseLong(diskNumber), vdi.getUuid(conn), disk.getType());
return new AttachAnswer(newDisk); return new AttachAnswer(newDisk);

View File

@ -18,6 +18,8 @@
*/ */
package org.apache.cloudstack.storage.datastore.lifecycle; package org.apache.cloudstack.storage.datastore.lifecycle;
import java.util.ArrayList;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.StringTokenizer; import java.util.StringTokenizer;
@ -33,21 +35,29 @@ import org.apache.cloudstack.storage.datastore.db.PrimaryDataStoreDao;
import org.apache.cloudstack.storage.datastore.db.StoragePoolDetailsDao; import org.apache.cloudstack.storage.datastore.db.StoragePoolDetailsDao;
import org.apache.cloudstack.storage.datastore.util.SolidFireUtil; import org.apache.cloudstack.storage.datastore.util.SolidFireUtil;
import org.apache.cloudstack.storage.volume.datastore.PrimaryDataStoreHelper; import org.apache.cloudstack.storage.volume.datastore.PrimaryDataStoreHelper;
import org.apache.log4j.Logger;
import com.cloud.dc.DataCenterVO; import com.cloud.dc.DataCenterVO;
import com.cloud.dc.dao.DataCenterDao; import com.cloud.dc.dao.DataCenterDao;
import com.cloud.host.HostVO;
import com.cloud.hypervisor.Hypervisor.HypervisorType; import com.cloud.hypervisor.Hypervisor.HypervisorType;
import com.cloud.agent.api.StoragePoolInfo; import com.cloud.agent.api.StoragePoolInfo;
import com.cloud.resource.ResourceManager;
import com.cloud.storage.StorageManager;
import com.cloud.storage.StoragePoolAutomation; import com.cloud.storage.StoragePoolAutomation;
import com.cloud.storage.Storage.StoragePoolType; import com.cloud.storage.Storage.StoragePoolType;
import com.cloud.utils.exception.CloudRuntimeException; import com.cloud.utils.exception.CloudRuntimeException;
public class SolidFirePrimaryDataStoreLifeCycle implements PrimaryDataStoreLifeCycle { public class SolidFirePrimaryDataStoreLifeCycle implements PrimaryDataStoreLifeCycle {
@Inject PrimaryDataStoreDao storagePoolDao; private static final Logger s_logger = Logger.getLogger(SolidFirePrimaryDataStoreLifeCycle.class);
@Inject PrimaryDataStoreHelper dataStoreHelper;
@Inject StoragePoolAutomation storagePoolAutomation; @Inject private DataCenterDao zoneDao;
@Inject StoragePoolDetailsDao storagePoolDetailsDao; @Inject private PrimaryDataStoreDao storagePoolDao;
@Inject DataCenterDao zoneDao; @Inject private PrimaryDataStoreHelper dataStoreHelper;
@Inject private ResourceManager _resourceMgr;
@Inject StorageManager _storageMgr;
@Inject private StoragePoolAutomation storagePoolAutomation;
@Inject private StoragePoolDetailsDao storagePoolDetailsDao;
private static final int DEFAULT_MANAGEMENT_PORT = 443; private static final int DEFAULT_MANAGEMENT_PORT = 443;
private static final int DEFAULT_STORAGE_PORT = 3260; private static final int DEFAULT_STORAGE_PORT = 3260;
@ -305,9 +315,23 @@ public class SolidFirePrimaryDataStoreLifeCycle implements PrimaryDataStoreLifeC
public boolean attachZone(DataStore dataStore, ZoneScope scope, HypervisorType hypervisorType) { public boolean attachZone(DataStore dataStore, ZoneScope scope, HypervisorType hypervisorType) {
dataStoreHelper.attachZone(dataStore); dataStoreHelper.attachZone(dataStore);
return true; List<HostVO> xenServerHosts = _resourceMgr.listAllUpAndEnabledHostsInOneZoneByHypervisor(HypervisorType.XenServer, scope.getScopeId());
List<HostVO> kvmHosts = _resourceMgr.listAllUpAndEnabledHostsInOneZoneByHypervisor(HypervisorType.KVM, scope.getScopeId());
List<HostVO> hosts = new ArrayList<HostVO>();
hosts.addAll(xenServerHosts);
hosts.addAll(kvmHosts);
for (HostVO host : hosts) {
try {
_storageMgr.connectHostToSharedPool(host.getId(), dataStore.getId());
} catch (Exception e) {
s_logger.warn("Unable to establish a connection between " + host + " and " + dataStore, e);
}
} }
return true;
}
@Override @Override
public boolean maintain(DataStore dataStore) { public boolean maintain(DataStore dataStore) {

View File

@ -0,0 +1,102 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.cloudstack.storage.datastore.provider;
import javax.inject.Inject;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreManager;
import org.apache.cloudstack.engine.subsystem.api.storage.HypervisorHostListener;
import org.apache.log4j.Logger;
import com.cloud.agent.AgentManager;
import com.cloud.agent.api.Answer;
import com.cloud.agent.api.ModifyStoragePoolAnswer;
import com.cloud.agent.api.ModifyStoragePoolCommand;
import com.cloud.alert.AlertManager;
import com.cloud.host.HostVO;
import com.cloud.host.dao.HostDao;
import com.cloud.hypervisor.Hypervisor.HypervisorType;
import com.cloud.storage.DataStoreRole;
import com.cloud.storage.StoragePool;
import com.cloud.storage.StoragePoolHostVO;
import com.cloud.storage.dao.StoragePoolHostDao;
import com.cloud.utils.exception.CloudRuntimeException;
public class SolidFireHostListener implements HypervisorHostListener {
private static final Logger s_logger = Logger.getLogger(SolidFireHostListener.class);
@Inject private AgentManager _agentMgr;
@Inject private AlertManager _alertMgr;
@Inject private DataStoreManager _dataStoreMgr;
@Inject private HostDao _hostDao;
@Inject private StoragePoolHostDao storagePoolHostDao;
@Override
public boolean hostConnect(long hostId, long storagePoolId) {
HostVO host = _hostDao.findById(hostId);
StoragePoolHostVO storagePoolHost = storagePoolHostDao.findByPoolHost(storagePoolId, hostId);
if (storagePoolHost == null) {
storagePoolHost = new StoragePoolHostVO(storagePoolId, hostId, "");
storagePoolHostDao.persist(storagePoolHost);
}
// just want to send the ModifyStoragePoolCommand for KVM
if (host.getHypervisorType() != HypervisorType.KVM) {
return true;
}
StoragePool storagePool = (StoragePool)_dataStoreMgr.getDataStore(storagePoolId, DataStoreRole.Primary);
ModifyStoragePoolCommand cmd = new ModifyStoragePoolCommand(true, storagePool);
Answer answer = _agentMgr.easySend(hostId, cmd);
if (answer == null) {
throw new CloudRuntimeException("Unable to get an answer to the modify storage pool command (" + storagePool.getId() + ")");
}
if (!answer.getResult()) {
String msg = "Unable to attach storage pool " + storagePoolId + " to host " + hostId;
_alertMgr.sendAlert(AlertManager.ALERT_TYPE_HOST, storagePool.getDataCenterId(), storagePool.getPodId(), msg, msg);
throw new CloudRuntimeException("Unable to establish a connection from agent to storage pool " + storagePool.getId() +
" due to " + answer.getDetails() + " (" + storagePool.getId() + ")");
}
assert (answer instanceof ModifyStoragePoolAnswer) : "ModifyStoragePoolAnswer expected ; Pool = " + storagePool.getId() + " Host = " + hostId;
s_logger.info("Connection established between storage pool " + storagePool + " and host + " + hostId);
return true;
}
@Override
public boolean hostDisconnected(long hostId, long storagePoolId) {
StoragePoolHostVO storagePoolHost = storagePoolHostDao.findByPoolHost(storagePoolId, hostId);
if (storagePoolHost != null) {
storagePoolHostDao.deleteStoragePoolHostDetails(hostId, storagePoolId);
}
return true;
}
}

View File

@ -66,15 +66,7 @@ public class SolidfirePrimaryDataStoreProvider implements PrimaryDataStoreProvid
public boolean configure(Map<String, Object> params) { public boolean configure(Map<String, Object> params) {
lifecycle = ComponentContext.inject(SolidFirePrimaryDataStoreLifeCycle.class); lifecycle = ComponentContext.inject(SolidFirePrimaryDataStoreLifeCycle.class);
driver = ComponentContext.inject(SolidfirePrimaryDataStoreDriver.class); driver = ComponentContext.inject(SolidfirePrimaryDataStoreDriver.class);
listener = ComponentContext.inject(new HypervisorHostListener() { listener = ComponentContext.inject(SolidFireHostListener.class);
public boolean hostConnect(long hostId, long poolId) {
return true;
}
public boolean hostDisconnected(long hostId, long poolId) {
return true;
}
});
return true; return true;
} }

View File

@ -20,6 +20,7 @@ import java.util.ArrayList;
import java.util.Date; import java.util.Date;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.HashMap;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -1231,7 +1232,7 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
StoragePoolVO volumePool = _storagePoolDao.findById(volume.getPoolId()); StoragePoolVO volumePool = _storagePoolDao.findById(volume.getPoolId());
DataTO volTO = volFactory.getVolume(volume.getId()).getTO(); DataTO volTO = volFactory.getVolume(volume.getId()).getTO();
DiskTO disk = new DiskTO(volTO, volume.getDeviceId(), null, volume.getVolumeType()); DiskTO disk = new DiskTO(volTO, volume.getDeviceId(), volume.getPath(), volume.getVolumeType());
DettachCommand cmd = new DettachCommand(disk, vm.getInstanceName()); DettachCommand cmd = new DettachCommand(disk, vm.getInstanceName());
@ -1605,29 +1606,41 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
if (sendCommand) { if (sendCommand) {
volumeToAttachStoragePool = _storagePoolDao.findById(volumeToAttach.getPoolId()); volumeToAttachStoragePool = _storagePoolDao.findById(volumeToAttach.getPoolId());
long storagePoolId = volumeToAttachStoragePool.getId();
HostVO host = _hostDao.findById(hostId);
if (host.getHypervisorType() == HypervisorType.KVM &&
volumeToAttachStoragePool.isManaged() &&
volumeToAttach.getPath() == null) {
volumeToAttach.setPath(volumeToAttach.get_iScsiName());
_volsDao.update(volumeToAttach.getId(), volumeToAttach);
}
DataTO volTO = volFactory.getVolume(volumeToAttach.getId()).getTO(); DataTO volTO = volFactory.getVolume(volumeToAttach.getId()).getTO();
DiskTO disk = new DiskTO(volTO, deviceId, null, volumeToAttach.getVolumeType()); DiskTO disk = new DiskTO(volTO, deviceId, volumeToAttach.getPath(), volumeToAttach.getVolumeType());
AttachCommand cmd = new AttachCommand(disk, vm.getInstanceName()); AttachCommand cmd = new AttachCommand(disk, vm.getInstanceName());
cmd.setManaged(volumeToAttachStoragePool.isManaged());
cmd.setStorageHost(volumeToAttachStoragePool.getHostAddress());
cmd.setStoragePort(volumeToAttachStoragePool.getPort());
cmd.set_iScsiName(volumeToAttach.get_iScsiName());
VolumeInfo volumeInfo = volFactory.getVolume(volumeToAttach.getId()); VolumeInfo volumeInfo = volFactory.getVolume(volumeToAttach.getId());
DataStore dataStore = dataStoreMgr.getDataStore(storagePoolId, DataStoreRole.Primary); DataStore dataStore = dataStoreMgr.getDataStore(volumeToAttachStoragePool.getId(), DataStoreRole.Primary);
ChapInfo chapInfo = volService.getChapInfo(volumeInfo, dataStore); ChapInfo chapInfo = volService.getChapInfo(volumeInfo, dataStore);
Map<String, String> details = new HashMap<String, String>();
disk.setDetails(details);
details.put(DiskTO.MANAGED, String.valueOf(volumeToAttachStoragePool.isManaged()));
details.put(DiskTO.STORAGE_HOST, volumeToAttachStoragePool.getHostAddress());
details.put(DiskTO.STORAGE_PORT, String.valueOf(volumeToAttachStoragePool.getPort()));
details.put(DiskTO.VOLUME_SIZE, String.valueOf(volumeToAttach.getSize()));
details.put(DiskTO.IQN, volumeToAttach.get_iScsiName());
if (chapInfo != null) { if (chapInfo != null) {
cmd.setChapInitiatorUsername(chapInfo.getInitiatorUsername()); details.put(DiskTO.CHAP_INITIATOR_USERNAME, chapInfo.getInitiatorUsername());
cmd.setChapInitiatorPassword(chapInfo.getInitiatorSecret()); details.put(DiskTO.CHAP_INITIATOR_SECRET, chapInfo.getInitiatorSecret());
cmd.setChapTargetUsername(chapInfo.getTargetUsername()); details.put(DiskTO.CHAP_TARGET_USERNAME, chapInfo.getTargetUsername());
cmd.setChapTargetPassword(chapInfo.getTargetSecret()); details.put(DiskTO.CHAP_TARGET_SECRET, chapInfo.getTargetSecret());
} }
try { try {
@ -1646,7 +1659,7 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
volumeToAttach = _volsDao.findById(volumeToAttach.getId()); volumeToAttach = _volsDao.findById(volumeToAttach.getId());
if (volumeToAttachStoragePool.isManaged() && volumeToAttach.getPath() == null) { if (volumeToAttachStoragePool.isManaged() && volumeToAttach.getPath() == null) {
volumeToAttach.setPath(answer.getDisk().getVdiUuid()); volumeToAttach.setPath(answer.getDisk().getPath());
_volsDao.update(volumeToAttach.getId(), volumeToAttach); _volsDao.update(volumeToAttach.getId(), volumeToAttach);
} }

View File

@ -81,6 +81,9 @@ public class StoragePoolMonitor implements Listener {
List<StoragePoolVO> zoneStoragePoolsByHypervisor = _poolDao.findZoneWideStoragePoolsByHypervisor(host.getDataCenterId(), scCmd.getHypervisorType()); List<StoragePoolVO> zoneStoragePoolsByHypervisor = _poolDao.findZoneWideStoragePoolsByHypervisor(host.getDataCenterId(), scCmd.getHypervisorType());
zoneStoragePoolsByTags.retainAll(zoneStoragePoolsByHypervisor); zoneStoragePoolsByTags.retainAll(zoneStoragePoolsByHypervisor);
pools.addAll(zoneStoragePoolsByTags); pools.addAll(zoneStoragePoolsByTags);
List<StoragePoolVO> zoneStoragePoolsByAnyHypervisor = _poolDao.findZoneWideStoragePoolsByHypervisor(host.getDataCenterId(), HypervisorType.Any);
pools.addAll(zoneStoragePoolsByAnyHypervisor);
for (StoragePoolVO pool : pools) { for (StoragePoolVO pool : pools) {
if (pool.getStatus() != StoragePoolStatus.Up) { if (pool.getStatus() != StoragePoolStatus.Up) {
continue; continue;