Commits related to taking snapshots that live on primary storage

This commit is contained in:
Mike Tutkowski 2014-10-22 12:42:23 -06:00
commit 97aa02ceb0
30 changed files with 748 additions and 233 deletions

View File

@ -34,6 +34,7 @@ public class DiskTO {
public static final String VOLUME_SIZE = "volumeSize";
public static final String MOUNT_POINT = "mountpoint";
public static final String PROTOCOL_TYPE = "protocoltype";
public static final String PATH = "path";
private DataTO data;
private Long diskSeq;

View File

@ -38,7 +38,6 @@ public interface Volume extends ControlledEntity, Identity, InternalIdentity, Ba
Ready("The volume is ready to be used."),
Migrating("The volume is migrating to other storage pool"),
Snapshotting("There is a snapshot created on this volume, not backed up to secondary storage yet"),
Reverting("Replace the existing volume on a storage system with a snapshot of it"),
Resizing("The volume is being resized"),
Expunging("The volume is being expunging"),
Expunged("The volume is being expunging"),
@ -86,11 +85,8 @@ public interface Volume extends ControlledEntity, Identity, InternalIdentity, Ba
s_fsm.addTransition(Expunging, Event.OperationSucceeded, Expunged);
s_fsm.addTransition(Expunging, Event.OperationFailed, Destroy);
s_fsm.addTransition(Ready, Event.SnapshotRequested, Snapshotting);
s_fsm.addTransition(Ready, Event.RevertRequested, Reverting);
s_fsm.addTransition(Snapshotting, Event.OperationSucceeded, Ready);
s_fsm.addTransition(Snapshotting, Event.OperationFailed, Ready);
s_fsm.addTransition(Reverting, Event.OperationSucceeded, Ready);
s_fsm.addTransition(Reverting, Event.OperationFailed, Ready);
s_fsm.addTransition(Ready, Event.MigrationRequested, Migrating);
s_fsm.addTransition(Migrating, Event.OperationSucceeded, Ready);
s_fsm.addTransition(Migrating, Event.OperationFailed, Ready);
@ -115,7 +111,6 @@ public interface Volume extends ControlledEntity, Identity, InternalIdentity, Ba
UploadRequested,
MigrationRequested,
SnapshotRequested,
RevertRequested,
DestroyRequested,
ExpungingRequested,
ResizeRequested;

View File

@ -26,6 +26,7 @@ import org.apache.cloudstack.storage.command.DeleteCommand;
import org.apache.cloudstack.storage.command.DettachCommand;
import org.apache.cloudstack.storage.command.ForgetObjectCmd;
import org.apache.cloudstack.storage.command.IntroduceObjectCmd;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import com.cloud.agent.api.Answer;
@ -62,7 +63,9 @@ public interface StorageProcessor {
public Answer deleteSnapshot(DeleteCommand cmd);
Answer introduceObject(IntroduceObjectCmd cmd);
public Answer introduceObject(IntroduceObjectCmd cmd);
Answer forgetObject(ForgetObjectCmd cmd);
public Answer forgetObject(ForgetObjectCmd cmd);
public Answer snapshotAndCopy(SnapshotAndCopyCommand cmd);
}

View File

@ -28,6 +28,7 @@ import org.apache.cloudstack.storage.command.CreateObjectCommand;
import org.apache.cloudstack.storage.command.DeleteCommand;
import org.apache.cloudstack.storage.command.DettachCommand;
import org.apache.cloudstack.storage.command.IntroduceObjectCmd;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import org.apache.cloudstack.storage.command.StorageSubSystemCommand;
import com.cloud.agent.api.Answer;
@ -61,7 +62,10 @@ public class StorageSubsystemCommandHandlerBase implements StorageSubsystemComma
return execute((DettachCommand)command);
} else if (command instanceof IntroduceObjectCmd) {
return processor.introduceObject((IntroduceObjectCmd)command);
} else if (command instanceof SnapshotAndCopyCommand) {
return processor.snapshotAndCopy((SnapshotAndCopyCommand)command);
}
return new Answer((Command)command, false, "not implemented yet");
}

View File

@ -0,0 +1,41 @@
//
// 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.command;
import com.cloud.agent.api.Answer;
public class SnapshotAndCopyAnswer extends Answer {
private String _path;
public SnapshotAndCopyAnswer() {
}
public SnapshotAndCopyAnswer(String errMsg) {
super(null, false, errMsg);
}
public void setPath(String path) {
_path = path;
}
public String getPath() {
return _path;
}
}

View File

@ -0,0 +1,60 @@
//
// 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.command;
import com.cloud.agent.api.Command;
import java.util.Map;
public final class SnapshotAndCopyCommand extends Command implements StorageSubSystemCommand {
private String _uuidOfSourceVdi;
private Map<String, String> _sourceDetails;
private Map<String, String> _destDetails;
private boolean _executeInSequence = true;
public SnapshotAndCopyCommand(String uuidOfSourceVdi, Map<String, String> sourceDetails, Map<String, String> destDetails) {
_uuidOfSourceVdi = uuidOfSourceVdi;
_sourceDetails = sourceDetails;
_destDetails = destDetails;
}
public String getUuidOfSourceVdi() {
return _uuidOfSourceVdi;
}
public Map<String, String> getSourceDetails() {
return _sourceDetails;
}
public Map<String, String> getDestDetails() {
return _destDetails;
}
@Override
public void setExecuteInSequence(boolean executeInSequence) {
_executeInSequence = executeInSequence;
}
@Override
public boolean executeInSequence() {
return _executeInSequence;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.cloudstack.engine.orchestration.service;
import java.util.Map;
import java.util.Set;
import org.apache.cloudstack.engine.subsystem.api.storage.DataObject;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStore;
import org.apache.cloudstack.engine.subsystem.api.storage.VolumeInfo;
@ -95,9 +96,9 @@ public interface VolumeOrchestrationService {
void cleanupVolumes(long vmId) throws ConcurrentOperationException;
void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore);
void revokeAccess(DataObject dataObject, Host host, DataStore dataStore);
void disconnectVolumesFromHost(long vmId, long hostId);
void revokeAccess(long vmId, long hostId);
void migrateVolumes(VirtualMachine vm, VirtualMachineTO vmTo, Host srcHost, Host destHost, Map<Volume, StoragePool> volumeToPool);

View File

@ -28,9 +28,9 @@ import com.cloud.storage.Volume;
public interface PrimaryDataStoreDriver extends DataStoreDriver {
public ChapInfo getChapInfo(VolumeInfo volumeInfo);
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore);
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore);
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore);
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore);
// intended for managed storage (cloud.storage_pool.managed = true)
// if not managed, return volume.getSize()

View File

@ -44,9 +44,9 @@ public interface VolumeService {
ChapInfo getChapInfo(VolumeInfo volumeInfo, DataStore dataStore);
boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore);
boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore);
void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore);
void revokeAccess(DataObject dataObject, Host host, DataStore dataStore);
/**
* Creates the volume based on the given criteria

View File

@ -517,7 +517,7 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
}
if (hostId != null) {
volumeMgr.disconnectVolumesFromHost(vm.getId(), hostId);
volumeMgr.revokeAccess(vm.getId(), hostId);
}
// Clean up volumes based on the vm's instance id

View File

@ -34,6 +34,7 @@ import org.apache.log4j.Logger;
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.DataObject;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStore;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreDriver;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreManager;
@ -871,16 +872,16 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
}
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {
DataStoreDriver dataStoreDriver = dataStore != null ? dataStore.getDriver() : null;
if (dataStoreDriver instanceof PrimaryDataStoreDriver) {
((PrimaryDataStoreDriver)dataStoreDriver).disconnectVolumeFromHost(volumeInfo, host, dataStore);
((PrimaryDataStoreDriver)dataStoreDriver).revokeAccess(dataObject, host, dataStore);
}
}
@Override
public void disconnectVolumesFromHost(long vmId, long hostId) {
public void revokeAccess(long vmId, long hostId) {
HostVO host = _hostDao.findById(hostId);
List<VolumeVO> volumesForVm = _volsDao.findByInstance(vmId);
@ -888,10 +889,12 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
if (volumesForVm != null) {
for (VolumeVO volumeForVm : volumesForVm) {
VolumeInfo volumeInfo = volFactory.getVolume(volumeForVm.getId());
// pool id can be null for the VM's volumes in Allocated state
if (volumeForVm.getPoolId() != null) {
DataStore dataStore = dataStoreMgr.getDataStore(volumeForVm.getPoolId(), DataStoreRole.Primary);
volService.disconnectVolumeFromHost(volumeInfo, host, dataStore);
volService.revokeAccess(volumeInfo, host, dataStore);
}
}
}
@ -1246,7 +1249,7 @@ public class VolumeOrchestrator extends ManagerBase implements VolumeOrchestrati
long hostId = vm.getVirtualMachine().getHostId();
Host host = _hostDao.findById(hostId);
volService.connectVolumeToHost(volFactory.getVolume(newVol.getId()), host, destPool);
volService.grantAccess(volFactory.getVolume(newVol.getId()), host, destPool);
}
newVol = _volsDao.findById(newVol.getId());

View File

@ -0,0 +1,72 @@
/*
* 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.motion;
import java.util.Map;
import org.apache.log4j.Logger;
import org.springframework.stereotype.Component;
import org.apache.cloudstack.engine.subsystem.api.storage.CopyCommandResult;
import org.apache.cloudstack.engine.subsystem.api.storage.DataMotionStrategy;
import org.apache.cloudstack.engine.subsystem.api.storage.DataObject;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStore;
import org.apache.cloudstack.engine.subsystem.api.storage.StrategyPriority;
import org.apache.cloudstack.engine.subsystem.api.storage.VolumeInfo;
import org.apache.cloudstack.framework.async.AsyncCompletionCallback;
import com.cloud.agent.api.to.VirtualMachineTO;
import com.cloud.host.Host;
@Component
public class StorageSystemDataMotionStrategy implements DataMotionStrategy {
private static final Logger s_logger = Logger.getLogger(StorageSystemDataMotionStrategy.class);
@Override
public StrategyPriority canHandle(DataObject srcData, DataObject destData) {
return StrategyPriority.DEFAULT;
}
@Override
public StrategyPriority canHandle(Map<VolumeInfo, DataStore> volumeMap, Host srcHost, Host destHost) {
return StrategyPriority.CANT_HANDLE;
}
@Override
public Void copyAsync(DataObject srcData, DataObject destData, Host destHost, AsyncCompletionCallback<CopyCommandResult> callback) {
return null;
}
@Override
public Void copyAsync(DataObject srcData, DataObject destData, AsyncCompletionCallback<CopyCommandResult> callback) {
return copyAsync(srcData, destData, null, callback);
}
@Override
public Void copyAsync(Map<VolumeInfo, DataStore> volumeMap, VirtualMachineTO vmTo, Host srcHost, Host destHost, AsyncCompletionCallback<CopyCommandResult> callback) {
CopyCommandResult result = new CopyCommandResult(null, null);
result.setResult("Unsupported operation requested for copying data.");
callback.complete(result);
return null;
}
}

View File

@ -49,10 +49,10 @@ public class FakePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
}
@Override
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) { return false; }
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore) { return false; }
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {}
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {}
@Override
public long getUsedBytes(StoragePool storagePool) {

View File

@ -42,7 +42,7 @@ public class SnapshotStateMachineManagerImpl implements SnapshotStateMachineMana
stateMachine.addTransition(Snapshot.State.Creating, Event.OperationNotPerformed, Snapshot.State.BackedUp);
stateMachine.addTransition(Snapshot.State.Creating, Event.OperationFailed, Snapshot.State.Error);
stateMachine.addTransition(Snapshot.State.CreatedOnPrimary, Event.BackupToSecondary, Snapshot.State.BackingUp);
stateMachine.addTransition(State.CreatedOnPrimary, Event.OperationNotPerformed, State.BackedUp);
stateMachine.addTransition(Snapshot.State.CreatedOnPrimary, Event.OperationNotPerformed, Snapshot.State.BackedUp);
stateMachine.addTransition(Snapshot.State.BackingUp, Event.OperationSucceeded, Snapshot.State.BackedUp);
stateMachine.addTransition(Snapshot.State.BackingUp, Event.OperationFailed, Snapshot.State.Error);
stateMachine.addTransition(Snapshot.State.BackedUp, Event.DestroyRequested, Snapshot.State.Destroying);

View File

@ -16,12 +16,15 @@
// under the License.
package org.apache.cloudstack.storage.snapshot;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.inject.Inject;
import org.apache.log4j.Logger;
import org.springframework.stereotype.Component;
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.DataStoreCapabilities;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreManager;
@ -30,43 +33,53 @@ import org.apache.cloudstack.engine.subsystem.api.storage.SnapshotInfo;
import org.apache.cloudstack.engine.subsystem.api.storage.SnapshotResult;
import org.apache.cloudstack.engine.subsystem.api.storage.StrategyPriority;
import org.apache.cloudstack.engine.subsystem.api.storage.VolumeInfo;
import org.apache.cloudstack.engine.subsystem.api.storage.VolumeService;
import org.apache.cloudstack.storage.command.SnapshotAndCopyAnswer;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import org.apache.cloudstack.storage.datastore.db.PrimaryDataStoreDao;
import org.apache.cloudstack.storage.datastore.db.SnapshotDataStoreDao;
import org.apache.cloudstack.storage.datastore.db.StoragePoolVO;
import com.cloud.dc.ClusterVO;
import com.cloud.dc.dao.ClusterDao;
import com.cloud.agent.AgentManager;
import com.cloud.agent.api.to.DiskTO;
import com.cloud.exception.InvalidParameterValueException;
import com.cloud.host.HostVO;
import com.cloud.host.dao.HostDao;
import com.cloud.hypervisor.Hypervisor.HypervisorType;
import com.cloud.org.Cluster;
import com.cloud.org.Grouping.AllocationState;
import com.cloud.resource.ResourceState;
import com.cloud.server.ManagementService;
import com.cloud.storage.DataStoreRole;
import com.cloud.storage.Snapshot;
import com.cloud.storage.SnapshotVO;
import com.cloud.storage.Storage.ImageFormat;
import com.cloud.storage.Volume;
import com.cloud.storage.Volume.Type;
import com.cloud.storage.VolumeVO;
import com.cloud.storage.dao.SnapshotDao;
import com.cloud.storage.dao.SnapshotDetailsDao;
import com.cloud.storage.dao.SnapshotDetailsVO;
import com.cloud.storage.dao.VolumeDao;
import com.cloud.uservm.UserVm;
import com.cloud.utils.db.DB;
import com.cloud.utils.db.EntityManager;
import com.cloud.utils.exception.CloudRuntimeException;
import com.cloud.utils.fsm.NoTransitionException;
import com.cloud.vm.VirtualMachine.State;
import com.cloud.vm.VMInstanceVO;
import com.cloud.vm.dao.VMInstanceDao;
@Component
public class StorageSystemSnapshotStrategy extends SnapshotStrategyBase {
private static final Logger s_logger = Logger.getLogger(StorageSystemSnapshotStrategy.class);
@Inject private ClusterDao _clusterDao;
@Inject private AgentManager _agentMgr;
@Inject private DataStoreManager _dataStoreMgr;
@Inject private EntityManager _entityMgr;
@Inject private HostDao _hostDao;
@Inject private ManagementService _mgr;
@Inject private PrimaryDataStoreDao _storagePoolDao;
@Inject private SnapshotDao _snapshotDao;
@Inject private SnapshotDataFactory _snapshotDataFactory;
@Inject private SnapshotDataStoreDao _snapshotStoreDao;
@Inject private PrimaryDataStoreDao _storagePoolDao;
@Inject private SnapshotDetailsDao _snapshotDetailsDao;
@Inject private VMInstanceDao _vmInstanceDao;
@Inject private VolumeDao _volumeDao;
@Inject private VolumeService _volService;
@Override
public SnapshotInfo backupSnapshot(SnapshotInfo snapshotInfo) {
@ -133,115 +146,52 @@ public class StorageSystemSnapshotStrategy extends SnapshotStrategyBase {
@Override
public boolean revertSnapshot(Long snapshotId) {
// verify the following:
// if root disk, the VM cannot be running (don't allow this at all for ESX)
// if data disk, the disk cannot be in the attached state
SnapshotInfo snapshotInfo = _snapshotDataFactory.getSnapshot(snapshotId, DataStoreRole.Primary);
VolumeInfo volumeInfo = snapshotInfo.getBaseVolume();
VolumeVO volume = _volumeDao.findById(volumeInfo.getId());
if (volume.getVolumeType() == Type.DATADISK) {
if (volume.getAttached() != null) {
throw new CloudRuntimeException("A data disk must be in the detached state in order to perform a revert.");
}
}
else if (volume.getVolumeType() == Type.ROOT) {
Long instanceId = volume.getInstanceId();
UserVm vm = _entityMgr.findById(UserVm.class, instanceId);
Long hostId = vm.getHostId();
HostVO hostVO = _hostDao.findById(hostId);
Long clusterId = hostVO.getClusterId();
ClusterVO clusterVO = _clusterDao.findById(clusterId);
if (clusterVO.getHypervisorType() != HypervisorType.XenServer && clusterVO.getHypervisorType() != HypervisorType.KVM) {
throw new CloudRuntimeException("Unsupported hypervisor type for root disk revert. Create a template from this disk and use it instead.");
}
if (vm.getState() != State.Stopped) {
throw new CloudRuntimeException("A root disk cannot be reverted unless the VM it's attached to is in the stopped state.");
}
}
else {
throw new CloudRuntimeException("Unsupported volume type");
}
SnapshotVO snapshotVO = _snapshotDao.acquireInLockTable(snapshotId);
if (snapshotVO == null) {
throw new CloudRuntimeException("Failed to acquire lock on the following snapshot: " + snapshotId);
}
try {
volumeInfo.stateTransit(Volume.Event.RevertRequested);
boolean result = false;
try {
result = snapshotSvr.revertSnapshot(snapshotId);
}
finally {
if (result) {
volumeInfo.stateTransit(Volume.Event.OperationSucceeded);
}
else {
String msg = "Failed to revert the volume to a snapshot";
s_logger.debug(msg);
volumeInfo.stateTransit(Volume.Event.OperationFailed);
throw new CloudRuntimeException("Failed to revert the volume to a snapshot");
}
}
}
finally {
if (snapshotVO != null) {
_snapshotDao.releaseFromLockTable(snapshotId);
}
}
return true;
throw new UnsupportedOperationException("Reverting not supported. Create a template or volume based on the snapshot instead.");
}
@Override
@DB
public SnapshotInfo takeSnapshot(SnapshotInfo snapshotInfo) {
VolumeInfo volumeInfo = snapshotInfo.getBaseVolume();
if (volumeInfo.getFormat() != ImageFormat.VHD) {
throw new CloudRuntimeException("Only the " + ImageFormat.VHD.toString() + " image type is currently supported.");
}
SnapshotVO snapshotVO = _snapshotDao.acquireInLockTable(snapshotInfo.getId());
if (snapshotVO == null) {
throw new CloudRuntimeException("Failed to acquire lock on the following snapshot: " + snapshotInfo.getId());
}
try {
VolumeInfo volumeInfo = snapshotInfo.getBaseVolume();
SnapshotResult result = null;
try {
volumeInfo.stateTransit(Volume.Event.SnapshotRequested);
SnapshotResult result = null;
// tell the storage driver to create a back-end volume (eventually used to create a new SR on and to copy the VM snapshot VDI to)
result = snapshotSvr.takeSnapshot(snapshotInfo);
try {
result = snapshotSvr.takeSnapshot(snapshotInfo);
if (result.isFailed()) {
s_logger.debug("Failed to take a snapshot: " + result.getResult());
if (result.isFailed()) {
s_logger.debug("Failed to take the following snapshot: " + result.getResult());
throw new CloudRuntimeException(result.getResult());
}
markAsBackedUp((SnapshotObject)result.getSnashot());
}
finally {
if (result != null && result.isSuccess()) {
volumeInfo.stateTransit(Volume.Event.OperationSucceeded);
}
else {
volumeInfo.stateTransit(Volume.Event.OperationFailed);
}
throw new CloudRuntimeException(result.getResult());
}
// send a command to XenServer to create a VM snapshot on the applicable SR (get back the VDI UUID of the VM snapshot)
performSnapshotAndCopyOnHostSide(volumeInfo, snapshotInfo);
markAsBackedUp((SnapshotObject)result.getSnashot());
}
finally {
if (result != null && result.isSuccess()) {
volumeInfo.stateTransit(Volume.Event.OperationSucceeded);
}
else {
volumeInfo.stateTransit(Volume.Event.OperationFailed);
}
if (snapshotVO != null) {
_snapshotDao.releaseFromLockTable(snapshotInfo.getId());
}
@ -250,6 +200,180 @@ public class StorageSystemSnapshotStrategy extends SnapshotStrategyBase {
return snapshotInfo;
}
private void performSnapshotAndCopyOnHostSide(VolumeInfo volumeInfo, SnapshotInfo snapshotInfo) {
Map<String, String> sourceDetails = null;
VolumeVO volumeVO = _volumeDao.findById(volumeInfo.getId());
Long vmInstanceId = volumeVO.getInstanceId();
VMInstanceVO vmInstanceVO = _vmInstanceDao.findById(vmInstanceId);
Long hostId = null;
// if the volume to snapshot is associated with a VM
if (vmInstanceVO != null) {
hostId = vmInstanceVO.getHostId();
// if the VM is not associated with a host
if (hostId == null) {
sourceDetails = getSourceDetails(volumeInfo);
hostId = vmInstanceVO.getLastHostId();
}
}
// volume to snapshot is not associated with a VM (could be a data disk in the detached state)
else {
sourceDetails = getSourceDetails(volumeInfo);
}
HostVO hostVO = getHostId(hostId, volumeVO);
long storagePoolId = volumeVO.getPoolId();
StoragePoolVO storagePoolVO = _storagePoolDao.findById(storagePoolId);
DataStore dataStore = _dataStoreMgr.getDataStore(storagePoolId, DataStoreRole.Primary);
Map<String, String> destDetails = getDestDetails(storagePoolVO, snapshotInfo);
SnapshotAndCopyCommand snapshotAndCopyCommand = new SnapshotAndCopyCommand(volumeInfo.getPath(), sourceDetails, destDetails);
SnapshotAndCopyAnswer snapshotAndCopyAnswer = null;
try {
// if sourceDetails != null, we need to connect the host(s) to the volume
if (sourceDetails != null) {
_volService.grantAccess(volumeInfo, hostVO, dataStore);
}
_volService.grantAccess(snapshotInfo, hostVO, dataStore);
snapshotAndCopyAnswer = (SnapshotAndCopyAnswer)_agentMgr.send(hostVO.getId(), snapshotAndCopyCommand);
}
catch (Exception ex) {
throw new CloudRuntimeException(ex.getMessage());
}
finally {
try {
_volService.revokeAccess(snapshotInfo, hostVO, dataStore);
// if sourceDetails != null, we need to disconnect the host(s) from the volume
if (sourceDetails != null) {
_volService.revokeAccess(volumeInfo, hostVO, dataStore);
}
}
catch (Exception ex) {
s_logger.debug(ex.getMessage(), ex);
}
}
if (snapshotAndCopyAnswer == null || !snapshotAndCopyAnswer.getResult()) {
final String errMsg;
if (snapshotAndCopyAnswer != null && snapshotAndCopyAnswer.getDetails() != null && !snapshotAndCopyAnswer.getDetails().isEmpty()) {
errMsg = snapshotAndCopyAnswer.getDetails();
}
else {
errMsg = "Unable to perform host-side operation";
}
throw new CloudRuntimeException(errMsg);
}
String path = snapshotAndCopyAnswer.getPath(); // for XenServer, this is the VDI's UUID
SnapshotDetailsVO snapshotDetail = new SnapshotDetailsVO(snapshotInfo.getId(),
DiskTO.PATH,
path,
false);
_snapshotDetailsDao.persist(snapshotDetail);
}
private Map<String, String> getSourceDetails(VolumeInfo volumeInfo) {
Map<String, String> sourceDetails = new HashMap<String, String>();
VolumeVO volumeVO = _volumeDao.findById(volumeInfo.getId());
long storagePoolId = volumeVO.getPoolId();
StoragePoolVO storagePoolVO = _storagePoolDao.findById(storagePoolId);
sourceDetails.put(DiskTO.STORAGE_HOST, storagePoolVO.getHostAddress());
sourceDetails.put(DiskTO.STORAGE_PORT, String.valueOf(storagePoolVO.getPort()));
sourceDetails.put(DiskTO.IQN, volumeVO.get_iScsiName());
ChapInfo chapInfo = _volService.getChapInfo(volumeInfo, volumeInfo.getDataStore());
if (chapInfo != null) {
sourceDetails.put(DiskTO.CHAP_INITIATOR_USERNAME, chapInfo.getInitiatorUsername());
sourceDetails.put(DiskTO.CHAP_INITIATOR_SECRET, chapInfo.getInitiatorSecret());
sourceDetails.put(DiskTO.CHAP_TARGET_USERNAME, chapInfo.getTargetUsername());
sourceDetails.put(DiskTO.CHAP_TARGET_SECRET, chapInfo.getTargetSecret());
}
return sourceDetails;
}
private Map<String, String> getDestDetails(StoragePoolVO storagePoolVO, SnapshotInfo snapshotInfo) {
Map<String, String> destDetails = new HashMap<String, String>();
destDetails.put(DiskTO.STORAGE_HOST, storagePoolVO.getHostAddress());
destDetails.put(DiskTO.STORAGE_PORT, String.valueOf(storagePoolVO.getPort()));
long snapshotId = snapshotInfo.getId();
destDetails.put(DiskTO.IQN, getProperty(snapshotId, DiskTO.IQN));
destDetails.put(DiskTO.CHAP_INITIATOR_USERNAME, getProperty(snapshotId, DiskTO.CHAP_INITIATOR_USERNAME));
destDetails.put(DiskTO.CHAP_INITIATOR_SECRET, getProperty(snapshotId, DiskTO.CHAP_INITIATOR_SECRET));
destDetails.put(DiskTO.CHAP_TARGET_USERNAME, getProperty(snapshotId, DiskTO.CHAP_TARGET_USERNAME));
destDetails.put(DiskTO.CHAP_TARGET_SECRET, getProperty(snapshotId, DiskTO.CHAP_TARGET_SECRET));
return destDetails;
}
private String getProperty(long snapshotId, String property) {
SnapshotDetailsVO snapshotDetails = _snapshotDetailsDao.findDetail(snapshotId, property);
if (snapshotDetails != null) {
return snapshotDetails.getValue();
}
return null;
}
private HostVO getHostId(Long hostId, VolumeVO volumeVO) {
HostVO hostVO = _hostDao.findById(hostId);
if (hostVO != null) {
return hostVO;
}
// pick a host in any XenServer cluster that's in the applicable zone
long zoneId = volumeVO.getDataCenterId();
List<? extends Cluster> clusters = _mgr.searchForClusters(zoneId, new Long(0), Long.MAX_VALUE, HypervisorType.XenServer.toString());
if (clusters == null) {
throw new CloudRuntimeException("Unable to locate an applicable cluster");
}
for (Cluster cluster : clusters) {
if (cluster.getAllocationState() == AllocationState.Enabled) {
List<HostVO> hosts = _hostDao.findByClusterId(cluster.getId());
if (hosts != null) {
for (HostVO host : hosts) {
if (host.getResourceState() == ResourceState.Enabled) {
return host;
}
}
}
}
}
throw new CloudRuntimeException("Unable to locate an applicable cluster");
}
private void markAsBackedUp(SnapshotObject snapshotObj) {
try {
snapshotObj.processEvent(Snapshot.Event.BackupToSecondary);
@ -269,6 +393,10 @@ public class StorageSystemSnapshotStrategy extends SnapshotStrategyBase {
@Override
public StrategyPriority canHandle(Snapshot snapshot, SnapshotOperation op) {
if (SnapshotOperation.REVERT.equals(op)) {
return StrategyPriority.CANT_HANDLE;
}
long volumeId = snapshot.getVolumeId();
VolumeVO volumeVO = _volumeDao.findById(volumeId);

View File

@ -167,22 +167,22 @@ public class VolumeServiceImpl implements VolumeService {
}
@Override
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore) {
DataStoreDriver dataStoreDriver = dataStore != null ? dataStore.getDriver() : null;
if (dataStoreDriver instanceof PrimaryDataStoreDriver) {
return ((PrimaryDataStoreDriver)dataStoreDriver).connectVolumeToHost(volumeInfo, host, dataStore);
return ((PrimaryDataStoreDriver)dataStoreDriver).grantAccess(dataObject, host, dataStore);
}
return false;
}
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {
DataStoreDriver dataStoreDriver = dataStore != null ? dataStore.getDriver() : null;
if (dataStoreDriver instanceof PrimaryDataStoreDriver) {
((PrimaryDataStoreDriver)dataStoreDriver).disconnectVolumeFromHost(volumeInfo, host, dataStore);
((PrimaryDataStoreDriver)dataStoreDriver).revokeAccess(dataObject, host, dataStore);
}
}
@ -677,7 +677,7 @@ public class VolumeServiceImpl implements VolumeService {
// refresh the volume from the DB
volumeInfo = volFactory.getVolume(volumeInfo.getId(), destPrimaryDataStore);
connectVolumeToHost(volumeInfo, destHost, destPrimaryDataStore);
grantAccess(volumeInfo, destHost, destPrimaryDataStore);
ManagedCreateBaseImageContext<CreateCmdResult> context = new ManagedCreateBaseImageContext<CreateCmdResult>(null, volumeInfo,
destPrimaryDataStore, srcTemplateInfo, future);
@ -712,7 +712,7 @@ public class VolumeServiceImpl implements VolumeService {
volumeInfo.processEvent(Event.DestroyRequested);
disconnectVolumeFromHost(volumeInfo, destHost, destPrimaryDataStore);
revokeAccess(volumeInfo, destHost, destPrimaryDataStore);
try {
AsyncCallFuture<VolumeApiResult> expungeVolumeFuture = expungeVolumeAsync(volumeInfo);

View File

@ -62,6 +62,8 @@ import org.apache.cloudstack.storage.command.DettachAnswer;
import org.apache.cloudstack.storage.command.DettachCommand;
import org.apache.cloudstack.storage.command.ForgetObjectCmd;
import org.apache.cloudstack.storage.command.IntroduceObjectCmd;
import org.apache.cloudstack.storage.command.SnapshotAndCopyAnswer;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import org.apache.cloudstack.storage.to.PrimaryDataStoreTO;
import org.apache.cloudstack.storage.to.SnapshotObjectTO;
import org.apache.cloudstack.storage.to.TemplateObjectTO;
@ -141,6 +143,13 @@ public class KVMStorageProcessor implements StorageProcessor {
return true;
}
@Override
public SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand cmd) {
s_logger.info("'SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand)' not currently used for KVMStorageProcessor");
return new SnapshotAndCopyAnswer();
}
@Override
public Answer copyTemplateToPrimaryStorage(CopyCommand cmd) {
DataTO srcData = cmd.getSrcTO();

View File

@ -35,6 +35,8 @@ import org.apache.cloudstack.storage.command.DettachAnswer;
import org.apache.cloudstack.storage.command.DettachCommand;
import org.apache.cloudstack.storage.command.ForgetObjectCmd;
import org.apache.cloudstack.storage.command.IntroduceObjectCmd;
import org.apache.cloudstack.storage.command.SnapshotAndCopyAnswer;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import org.apache.cloudstack.storage.to.SnapshotObjectTO;
import org.apache.cloudstack.storage.to.TemplateObjectTO;
import org.apache.cloudstack.storage.to.VolumeObjectTO;
@ -57,6 +59,13 @@ public class SimulatorStorageProcessor implements StorageProcessor {
this.hypervisorResource = resource;
}
@Override
public SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand cmd) {
s_logger.info("'SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand)' not currently used for SimulatorStorageProcessor");
return new SnapshotAndCopyAnswer();
}
@Override
public Answer copyTemplateToPrimaryStorage(CopyCommand cmd) {
TemplateObjectTO template = new TemplateObjectTO();

View File

@ -59,6 +59,8 @@ import org.apache.cloudstack.storage.command.DeleteCommand;
import org.apache.cloudstack.storage.command.DettachCommand;
import org.apache.cloudstack.storage.command.ForgetObjectCmd;
import org.apache.cloudstack.storage.command.IntroduceObjectCmd;
import org.apache.cloudstack.storage.command.SnapshotAndCopyAnswer;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import org.apache.cloudstack.storage.to.PrimaryDataStoreTO;
import org.apache.cloudstack.storage.to.SnapshotObjectTO;
import org.apache.cloudstack.storage.to.TemplateObjectTO;
@ -124,6 +126,13 @@ public class VmwareStorageProcessor implements StorageProcessor {
_gson = GsonHelper.getGsonLogger();
}
@Override
public SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand cmd) {
s_logger.info("'SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand)' not currently used for VmwareStorageProcessor");
return new SnapshotAndCopyAnswer();
}
private String getOVFFilePath(String srcOVAFileName) {
File file = new File(srcOVAFileName);
assert (_storage != null);

View File

@ -1911,7 +1911,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
String chapInitiatorUsername = details.get(DiskTO.CHAP_INITIATOR_USERNAME);
String chapInitiatorSecret = details.get(DiskTO.CHAP_INITIATOR_SECRET);
String mountpoint = details.get(DiskTO.MOUNT_POINT);
String protocoltype=details.get(DiskTO.PROTOCOL_TYPE);
String protocoltype = details.get(DiskTO.PROTOCOL_TYPE);
if (StoragePoolType.NetworkFilesystem.toString().equalsIgnoreCase(protocoltype)) {
String poolid = storageHost + ":" + mountpoint;
@ -6124,8 +6124,7 @@ public abstract class CitrixResourceBase implements ServerResource, HypervisorRe
VDI vdi = null;
if (cmd.getAttach() && cmd.isManaged()) {
SR sr =
getIscsiSR(conn, cmd.get_iScsiName(), cmd.getStorageHost(), cmd.get_iScsiName(), cmd.getChapInitiatorUsername(), cmd.getChapInitiatorPassword(), true);
SR sr = getIscsiSR(conn, cmd.get_iScsiName(), cmd.getStorageHost(), cmd.get_iScsiName(), cmd.getChapInitiatorUsername(), cmd.getChapInitiatorPassword(), true);
vdi = getVDIbyUuid(conn, cmd.getVolumePath(), false);

View File

@ -44,6 +44,8 @@ import org.apache.cloudstack.storage.command.DettachCommand;
import org.apache.cloudstack.storage.command.ForgetObjectCmd;
import org.apache.cloudstack.storage.command.IntroduceObjectAnswer;
import org.apache.cloudstack.storage.command.IntroduceObjectCmd;
import org.apache.cloudstack.storage.command.SnapshotAndCopyAnswer;
import org.apache.cloudstack.storage.command.SnapshotAndCopyCommand;
import org.apache.cloudstack.storage.datastore.protocol.DataStoreProtocol;
import org.apache.cloudstack.storage.to.PrimaryDataStoreTO;
import org.apache.cloudstack.storage.to.SnapshotObjectTO;
@ -93,6 +95,69 @@ public class XenServerStorageProcessor implements StorageProcessor {
hypervisorResource = resource;
}
// if the source SR needs to be attached to, do so
// take a snapshot of the source VDI (on the source SR)
// create an iSCSI SR based on the new back-end volume
// copy the snapshot to the new SR
// delete the snapshot
// detach the new SR
// if we needed to perform an attach to the source SR, detach from it
@Override
public SnapshotAndCopyAnswer snapshotAndCopy(SnapshotAndCopyCommand cmd) {
Connection conn = hypervisorResource.getConnection();
try {
SR sourceSr = null;
Map<String, String> sourceDetails = cmd.getSourceDetails();
if (sourceDetails != null && sourceDetails.keySet().size() > 0) {
String iScsiName = sourceDetails.get(DiskTO.IQN);
String storageHost = sourceDetails.get(DiskTO.STORAGE_HOST);
String chapInitiatorUsername = sourceDetails.get(DiskTO.CHAP_INITIATOR_USERNAME);
String chapInitiatorSecret = sourceDetails.get(DiskTO.CHAP_INITIATOR_SECRET);
sourceSr = hypervisorResource.getIscsiSR(conn, iScsiName, storageHost, iScsiName, chapInitiatorUsername, chapInitiatorSecret, false);
}
VDI vdiToSnapshot = VDI.getByUuid(conn, cmd.getUuidOfSourceVdi());
VDI vdiSnapshot = vdiToSnapshot.snapshot(conn, new HashMap<String, String>());
Map<String, String> destDetails = cmd.getDestDetails();
String iScsiName = destDetails.get(DiskTO.IQN);
String storageHost = destDetails.get(DiskTO.STORAGE_HOST);
String chapInitiatorUsername = destDetails.get(DiskTO.CHAP_INITIATOR_USERNAME);
String chapInitiatorSecret = destDetails.get(DiskTO.CHAP_INITIATOR_SECRET);
SR newSr = hypervisorResource.getIscsiSR(conn, iScsiName, storageHost, iScsiName, chapInitiatorUsername, chapInitiatorSecret, false);
VDI vdiCopy = vdiSnapshot.copy(conn, newSr);
String vdiUuid = vdiCopy.getUuid(conn);
vdiSnapshot.destroy(conn);
if (sourceSr != null) {
hypervisorResource.removeSR(conn, sourceSr);
}
hypervisorResource.removeSR(conn, newSr);
SnapshotAndCopyAnswer snapshotAndCopyAnswer = new SnapshotAndCopyAnswer();
snapshotAndCopyAnswer.setPath(vdiUuid);
return snapshotAndCopyAnswer;
}
catch (Exception ex) {
s_logger.warn("Failed to take and copy snapshot: " + ex.toString(), ex);
return new SnapshotAndCopyAnswer(ex.getMessage());
}
}
@Override
public AttachAnswer attachIso(AttachCommand cmd) {
DiskTO disk = cmd.getDisk();

View File

@ -401,13 +401,13 @@ public class ElastistorPrimaryDataStoreDriver extends CloudStackPrimaryDataStore
}
@Override
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore) {
// TODO Auto-generated method stub
return false;
}
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {
// TODO Auto-generated method stub
}

View File

@ -149,12 +149,12 @@ public class CloudStackPrimaryDataStoreDriverImpl implements PrimaryDataStoreDri
}
@Override
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore) {
return false;
}
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {
}
@Override

View File

@ -58,12 +58,12 @@ public class NexentaPrimaryDataStoreDriver implements PrimaryDataStoreDriver {
private static final Logger logger = Logger.getLogger(NexentaPrimaryDataStoreDriver.class);
@Override
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore) {
return false; //To change body of implemented methods use File | Settings | File Templates.
}
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {
//To change body of implemented methods use File | Settings | File Templates.
}

View File

@ -82,10 +82,10 @@ public class SamplePrimaryDataStoreDriverImpl implements PrimaryDataStoreDriver
}
@Override
public boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) { return false; }
public boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore) { return false; }
@Override
public void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore) {}
public void revokeAccess(DataObject dataObject, Host host, DataStore dataStore) {}
@Override
public long getUsedBytes(StoragePool storagePool) {

View File

@ -47,6 +47,7 @@ import com.cloud.agent.api.Answer;
import com.cloud.agent.api.to.DataObjectType;
import com.cloud.agent.api.to.DataStoreTO;
import com.cloud.agent.api.to.DataTO;
import com.cloud.agent.api.to.DiskTO;
import com.cloud.capacity.CapacityManager;
import com.cloud.dc.ClusterVO;
import com.cloud.dc.ClusterDetailsVO;
@ -125,13 +126,13 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
// if the ID of volumeInfo in not in the VAG, add it (ModifyVolumeAccessGroup)
// if the VAG doesn't exist, create it with the IQNs of the hosts and the ID of volumeInfo (CreateVolumeAccessGroup)
@Override
public synchronized boolean connectVolumeToHost(VolumeInfo volumeInfo, Host host, DataStore dataStore)
public synchronized boolean grantAccess(DataObject dataObject, Host host, DataStore dataStore)
{
if (volumeInfo == null || host == null || dataStore == null) {
if (dataObject == null || host == null || dataStore == null) {
return false;
}
long sfVolumeId = Long.parseLong(volumeInfo.getFolder());
long sfVolumeId = getSolidFireVolumeId(dataObject);
long clusterId = host.getClusterId();
long storagePoolId = dataStore.getId();
@ -168,13 +169,13 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
// if the VAG exists
// remove the ID of volumeInfo from the VAG (ModifyVolumeAccessGroup)
@Override
public synchronized void disconnectVolumeFromHost(VolumeInfo volumeInfo, Host host, DataStore dataStore)
public synchronized void revokeAccess(DataObject dataObject, Host host, DataStore dataStore)
{
if (volumeInfo == null || host == null || dataStore == null) {
if (dataObject == null || host == null || dataStore == null) {
return;
}
long sfVolumeId = Long.parseLong(volumeInfo.getFolder());
long sfVolumeId = getSolidFireVolumeId(dataObject);
long clusterId = host.getClusterId();
long storagePoolId = dataStore.getId();
@ -196,6 +197,24 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
}
}
private long getSolidFireVolumeId(DataObject dataObject) {
if (dataObject.getType() == DataObjectType.VOLUME) {
return Long.parseLong(((VolumeInfo)dataObject).getFolder());
}
if (dataObject.getType() == DataObjectType.SNAPSHOT) {
SnapshotDetailsVO snapshotDetails = _snapshotDetailsDao.findDetail(dataObject.getId(), SolidFireUtil.VOLUME_ID);
if (snapshotDetails == null || snapshotDetails.getValue() == null) {
throw new CloudRuntimeException("Unable to locate the volume ID associated with the following snapshot ID: " + dataObject.getId());
}
return Long.parseLong(snapshotDetails.getValue());
}
throw new CloudRuntimeException("Invalid DataObjectType (" + dataObject.getType() + ") passed to getSolidFireVolumeId(DataObject)");
}
private long getDefaultMinIops(long storagePoolId) {
StoragePoolDetailVO storagePoolDetail = _storagePoolDetailsDao.findDetail(storagePoolId, SolidFireUtil.CLUSTER_DEFAULT_MIN_IOPS);
@ -268,11 +287,11 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
if (lstSnapshots != null) {
for (SnapshotVO snapshot : lstSnapshots) {
SnapshotDetailsVO snapshotDetails = _snapshotDetailsDao.findDetail(snapshot.getId(), SolidFireUtil.SNAPSHOT_STORAGE_POOL_ID);
SnapshotDetailsVO snapshotDetails = _snapshotDetailsDao.findDetail(snapshot.getId(), SolidFireUtil.STORAGE_POOL_ID);
// if this snapshot belong to the storagePool that was passed in
// if this snapshot belongs to the storagePool that was passed in
if (snapshotDetails != null && snapshotDetails.getValue() != null && Long.parseLong(snapshotDetails.getValue()) == storagePool.getId()) {
snapshotDetails = _snapshotDetailsDao.findDetail(snapshot.getId(), SolidFireUtil.SNAPSHOT_SIZE);
snapshotDetails = _snapshotDetailsDao.findDetail(snapshot.getId(), SolidFireUtil.VOLUME_SIZE);
if (snapshotDetails != null && snapshotDetails.getValue() != null) {
long snapshotSize = Long.parseLong(snapshotDetails.getValue());
@ -492,10 +511,10 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
try {
VolumeInfo volumeInfo = snapshotInfo.getBaseVolume();
VolumeVO volume = _volumeDao.findById(volumeInfo.getId());
VolumeVO volumeVO = _volumeDao.findById(volumeInfo.getId());
long sfVolumeId = Long.parseLong(volume.getFolder());
long storagePoolId = volume.getPoolId();
long sfVolumeId = Long.parseLong(volumeVO.getFolder());
long storagePoolId = volumeVO.getPoolId();
SolidFireUtil.SolidFireConnection sfConnection = SolidFireUtil.getSolidFireConnection(storagePoolId, _storagePoolDetailsDao);
@ -504,34 +523,36 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
StoragePoolVO storagePool = _storagePoolDao.findById(storagePoolId);
long capacityBytes = storagePool.getCapacityBytes();
// getUsedBytes(StoragePool) will not include the bytes of the proposed snapshot because
// updateSnapshotDetails(long, long, long, long) has not yet been called for this snapshot
// getUsedBytes(StoragePool) will not include the bytes of the proposed new volume because
// updateSnapshotDetails(long, long, long, long, String) has not yet been called for this new volume
long usedBytes = getUsedBytes(storagePool);
long sfVolumeSize = sfVolume.getTotalSize();
usedBytes += sfVolumeSize;
// For taking a snapshot, we need to check to make sure a sufficient amount of space remains in the primary storage.
// For creating a volume, we need to check to make sure a sufficient amount of space remains in the primary storage.
// For the purpose of "charging" these bytes against storage_pool.capacityBytes, we take the full size of the SolidFire volume.
// Generally snapshots take up much less space than the size of the volume, but the easiest way to track this space usage
// is to take the full size of the volume (you can always increase the amount of bytes you give to the primary storage).
if (usedBytes > capacityBytes) {
throw new CloudRuntimeException("Insufficient amount of space remains in this primary storage to take a snapshot");
}
storagePool.setUsedBytes(usedBytes);
long sfSnapshotId = SolidFireUtil.createSolidFireSnapshot(sfConnection, sfVolumeId, snapshotInfo.getUuid());
/** @todo Mike T. fill in the CloudStackVolumeSize */
long sfNewVolumeId = SolidFireUtil.createSolidFireVolume(sfConnection, snapshotInfo.getUuid(), sfVolume.getAccountId(), sfVolumeSize,
sfVolume.isEnable512e(), "", sfVolume.getMinIops(), sfVolume.getMaxIops(), sfVolume.getBurstIops());
// Now that we have successfully taken a snapshot, update the space usage in the storage_pool table (even
// though storage_pool.used_bytes is likely no longer in use).
// Now that we have successfully created a volume, update the space usage in the storage_pool table
// (even though storage_pool.used_bytes is likely no longer in use).
_storagePoolDao.update(storagePoolId, storagePool);
updateSnapshotDetails(snapshotInfo.getId(), sfSnapshotId, storagePoolId, sfVolumeSize);
SolidFireUtil.SolidFireVolume sfNewVolume = SolidFireUtil.getSolidFireVolume(sfConnection, sfNewVolumeId);
updateSnapshotDetails(snapshotInfo.getId(), sfNewVolumeId, storagePoolId, sfVolumeSize, sfNewVolume.getIqn());
SnapshotObjectTO snapshotObjectTo = (SnapshotObjectTO)snapshotInfo.getTO();
snapshotObjectTo.setPath(String.valueOf(sfSnapshotId));
snapshotObjectTo.setPath(String.valueOf(sfNewVolumeId));
CreateObjectAnswer createObjectAnswer = new CreateObjectAnswer(snapshotObjectTo);
@ -550,39 +571,52 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
callback.complete(result);
}
private void updateSnapshotDetails(long csSnapshotId, long sfSnapshotId, long storagePoolId, long sfSnapshotSize) {
SnapshotDetailsVO accountDetail = new SnapshotDetailsVO(csSnapshotId,
SolidFireUtil.SNAPSHOT_ID,
String.valueOf(sfSnapshotId),
private void updateSnapshotDetails(long csSnapshotId, long sfNewVolumeId, long storagePoolId, long sfNewVolumeSize, String sfNewVolumeIqn) {
SnapshotDetailsVO snapshotDetail = new SnapshotDetailsVO(csSnapshotId,
SolidFireUtil.VOLUME_ID,
String.valueOf(sfNewVolumeId),
false);
_snapshotDetailsDao.persist(accountDetail);
_snapshotDetailsDao.persist(snapshotDetail);
accountDetail = new SnapshotDetailsVO(csSnapshotId,
SolidFireUtil.SNAPSHOT_STORAGE_POOL_ID,
String.valueOf(storagePoolId),
false);
snapshotDetail = new SnapshotDetailsVO(csSnapshotId,
SolidFireUtil.STORAGE_POOL_ID,
String.valueOf(storagePoolId),
false);
_snapshotDetailsDao.persist(accountDetail);
_snapshotDetailsDao.persist(snapshotDetail);
accountDetail = new SnapshotDetailsVO(csSnapshotId,
SolidFireUtil.SNAPSHOT_SIZE,
String.valueOf(sfSnapshotSize),
false);
snapshotDetail = new SnapshotDetailsVO(csSnapshotId,
SolidFireUtil.VOLUME_SIZE,
String.valueOf(sfNewVolumeSize),
false);
_snapshotDetailsDao.persist(accountDetail);
_snapshotDetailsDao.persist(snapshotDetail);
snapshotDetail = new SnapshotDetailsVO(csSnapshotId,
DiskTO.IQN,
sfNewVolumeIqn,
false);
_snapshotDetailsDao.persist(snapshotDetail);
}
// return null for no error message
private String deleteSnapshot(SnapshotInfo snapshotInfo, long storagePoolId) {
String errMsg = null;
long snapshotId = snapshotInfo.getId();
try {
SolidFireUtil.SolidFireConnection sfConnection = SolidFireUtil.getSolidFireConnection(storagePoolId, _storagePoolDetailsDao);
SolidFireUtil.deleteSolidFireSnapshot(sfConnection, getSolidFireSnapshotId(snapshotInfo.getId()));
SnapshotDetailsVO snapshotDetails = _snapshotDetailsDao.findDetail(snapshotId, SolidFireUtil.VOLUME_ID);
_snapshotDetailsDao.removeDetails(snapshotInfo.getId());
long volumeId = Long.parseLong(snapshotDetails.getValue());
SolidFireUtil.deleteSolidFireVolume(sfConnection, volumeId);
_snapshotDetailsDao.removeDetails(snapshotId);
StoragePoolVO storagePool = _storagePoolDao.findById(storagePoolId);
@ -594,7 +628,7 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
_storagePoolDao.update(storagePoolId, storagePool);
}
catch (Exception ex) {
s_logger.debug(SolidFireUtil.LOG_PREFIX + "Failed to delete SolidFire snapshot: " + snapshotInfo.getId(), ex);
s_logger.debug(SolidFireUtil.LOG_PREFIX + "Failed to delete SolidFire volume. CloudStack snapshot ID: " + snapshotId, ex);
errMsg = ex.getMessage();
}
@ -602,38 +636,9 @@ public class SolidFirePrimaryDataStoreDriver implements PrimaryDataStoreDriver {
return errMsg;
}
private long getSolidFireSnapshotId(long csSnapshotId) {
SnapshotDetailsVO snapshotDetails = _snapshotDetailsDao.findDetail(csSnapshotId, SolidFireUtil.SNAPSHOT_ID);
return Long.parseLong(snapshotDetails.getValue());
}
@Override
public void revertSnapshot(SnapshotInfo snapshotInfo, AsyncCompletionCallback<CommandResult> callback) {
String errMsg = null;
try {
VolumeInfo volumeInfo = snapshotInfo.getBaseVolume();
long storagePoolId = volumeInfo.getPoolId();
long sfVolumeId = Long.parseLong(volumeInfo.getFolder());
long sfSnapshotId = getSolidFireSnapshotId(snapshotInfo.getId());
SolidFireUtil.SolidFireConnection sfConnection = SolidFireUtil.getSolidFireConnection(storagePoolId, _storagePoolDetailsDao);
SolidFireUtil.rollBackVolumeToSnapshot(sfConnection, sfVolumeId, sfSnapshotId);
}
catch (Exception ex) {
s_logger.debug(SolidFireUtil.LOG_PREFIX + "Failed to take CloudStack snapshot: " + snapshotInfo.getId(), ex);
errMsg = ex.getMessage();
}
CommandResult result = new CommandResult();
result.setResult(errMsg);
callback.complete(result);
throw new UnsupportedOperationException("Reverting not supported. Create a template or volume based on the snapshot instead.");
}
@Override

View File

@ -92,13 +92,10 @@ public class SolidFireUtil {
public static final String ACCOUNT_ID = "accountId";
public static final String VOLUME_ID = "volumeId";
public static final String SNAPSHOT_ID = "snapshotId";
public static final String CLONE_ID = "cloneId";
public static final String VOLUME_SIZE = "sfVolumeSize";
public static final String SNAPSHOT_SIZE = "sfSnapshotSize";
public static final String SNAPSHOT_STORAGE_POOL_ID = "sfSnapshotStoragePoolId";
public static final String STORAGE_POOL_ID = "sfStoragePoolId";
public static final String CHAP_INITIATOR_USERNAME = "chapInitiatorUsername";
public static final String CHAP_INITIATOR_SECRET = "chapInitiatorSecret";
@ -513,9 +510,14 @@ public class SolidFireUtil {
String strVolumeIqn = getVolumeIqn(volumeGetResult, lVolumeId);
long lAccountId = getVolumeAccountId(volumeGetResult, lVolumeId);
String strVolumeStatus = getVolumeStatus(volumeGetResult, lVolumeId);
boolean enable512e = getVolumeEnable512e(volumeGetResult, lVolumeId);
long lMinIops = getVolumeMinIops(volumeGetResult, lVolumeId);
long lMaxIops = getVolumeMaxIops(volumeGetResult, lVolumeId);
long lBurstIops = getVolumeBurstIops(volumeGetResult, lVolumeId);
long lTotalSize = getVolumeTotalSize(volumeGetResult, lVolumeId);
return new SolidFireVolume(lVolumeId, strVolumeName, strVolumeIqn, lAccountId, strVolumeStatus, lTotalSize);
return new SolidFireVolume(lVolumeId, strVolumeName, strVolumeIqn, lAccountId, strVolumeStatus, enable512e,
lMinIops, lMaxIops, lBurstIops, lTotalSize);
}
public static List<SolidFireVolume> getSolidFireVolumesForAccountId(SolidFireConnection sfConnection, long lAccountId) {
@ -534,7 +536,8 @@ public class SolidFireUtil {
List<SolidFireVolume> sfVolumes = new ArrayList<SolidFireVolume>();
for (VolumeGetResult.Result.Volume volume : volumeGetResult.result.volumes) {
sfVolumes.add(new SolidFireVolume(volume.volumeID, volume.name, volume.iqn, volume.accountID, volume.status, volume.totalSize));
sfVolumes.add(new SolidFireVolume(volume.volumeID, volume.name, volume.iqn, volume.accountID, volume.status, volume.enable512e,
volume.qos.minIOPS, volume.qos.maxIOPS, volume.qos.burstIOPS, volume.totalSize));
}
return sfVolumes;
@ -557,7 +560,8 @@ public class SolidFireUtil {
List<SolidFireVolume> deletedVolumes = new ArrayList<SolidFireVolume> ();
for (VolumeGetResult.Result.Volume volume : volumeGetResult.result.volumes) {
deletedVolumes.add(new SolidFireVolume(volume.volumeID, volume.name, volume.iqn, volume.accountID, volume.status, volume.totalSize));
deletedVolumes.add(new SolidFireVolume(volume.volumeID, volume.name, volume.iqn, volume.accountID, volume.status, volume.enable512e,
volume.qos.minIOPS, volume.qos.maxIOPS, volume.qos.burstIOPS, volume.totalSize));
}
return deletedVolumes;
@ -593,16 +597,25 @@ public class SolidFireUtil {
private final String _iqn;
private final long _accountId;
private final String _status;
private final boolean _enable512e;
private final long _minIops;
private final long _maxIops;
private final long _burstIops;
private final long _totalSize;
public SolidFireVolume(long id, String name, String iqn,
long accountId, String status, long totalSize)
long accountId, String status, boolean enable512e,
long minIops, long maxIops, long burstIops, long totalSize)
{
_id = id;
_name = name;
_iqn = "/" + iqn + "/0";
_accountId = accountId;
_status = status;
_enable512e = enable512e;
_minIops = minIops;
_maxIops = maxIops;
_burstIops = burstIops;
_totalSize = totalSize;
}
@ -626,6 +639,22 @@ public class SolidFireUtil {
return ACTIVE.equalsIgnoreCase(_status);
}
public boolean isEnable512e() {
return _enable512e;
}
public long getMinIops() {
return _minIops;
}
public long getMaxIops() {
return _maxIops;
}
public long getBurstIops() {
return _burstIops;
}
public long getTotalSize() {
return _totalSize;
}
@ -703,10 +732,10 @@ public class SolidFireUtil {
verifyResult(rollbackInitiatedResult.result, strRollbackInitiatedResultJson, gson);
}
public static long createSolidFireClone(SolidFireConnection sfConnection, long lVolumeId, String cloneName) {
public static long createSolidFireClone(SolidFireConnection sfConnection, long lVolumeId, long lSnapshotId, String cloneName) {
final Gson gson = new GsonBuilder().create();
CloneToCreate cloneToCreate = new CloneToCreate(lVolumeId, cloneName);
CloneToCreate cloneToCreate = new CloneToCreate(lVolumeId, lSnapshotId, cloneName);
String strCloneToCreateJson = gson.toJson(cloneToCreate);
@ -1332,16 +1361,18 @@ public class SolidFireUtil {
private final String method = "CloneVolume";
private final CloneToCreateParams params;
private CloneToCreate(final long lVolumeId, final String cloneName) {
params = new CloneToCreateParams(lVolumeId, cloneName);
private CloneToCreate(final long lVolumeId, final long lSnapshotId, final String cloneName) {
params = new CloneToCreateParams(lVolumeId, lSnapshotId, cloneName);
}
private static final class CloneToCreateParams {
private long volumeID;
private long snapshotID;
private String name;
private CloneToCreateParams(final long lVolumeId, final String cloneName) {
private CloneToCreateParams(final long lVolumeId, final long lSnapshotId, final String cloneName) {
volumeID = lVolumeId;
snapshotID = lSnapshotId;
name = cloneName;
}
}
@ -1560,7 +1591,15 @@ public class SolidFireUtil {
private String iqn;
private long accountID;
private String status;
private boolean enable512e;
private Qos qos;
private long totalSize;
private static final class Qos {
private long minIOPS;
private long maxIOPS;
private long burstIOPS;
}
}
}
}
@ -1788,6 +1827,50 @@ public class SolidFireUtil {
throw new CloudRuntimeException("Could not determine the status of the volume for volume ID of " + lVolumeId + ".");
}
private static boolean getVolumeEnable512e(VolumeGetResult volumeGetResult, long lVolumeId)
{
if (volumeGetResult.result.volumes != null && volumeGetResult.result.volumes.length == 1 &&
volumeGetResult.result.volumes[0].volumeID == lVolumeId)
{
return volumeGetResult.result.volumes[0].enable512e;
}
throw new CloudRuntimeException("Could not determine the enable 512 emulation of the volume for volume ID of " + lVolumeId + ".");
}
private static long getVolumeMinIops(VolumeGetResult volumeGetResult, long lVolumeId)
{
if (volumeGetResult.result.volumes != null && volumeGetResult.result.volumes.length == 1 &&
volumeGetResult.result.volumes[0].volumeID == lVolumeId && volumeGetResult.result.volumes[0].qos != null)
{
return volumeGetResult.result.volumes[0].qos.minIOPS;
}
throw new CloudRuntimeException("Could not determine the min IOPS of the volume for volume ID of " + lVolumeId + ".");
}
private static long getVolumeMaxIops(VolumeGetResult volumeGetResult, long lVolumeId)
{
if (volumeGetResult.result.volumes != null && volumeGetResult.result.volumes.length == 1 &&
volumeGetResult.result.volumes[0].volumeID == lVolumeId && volumeGetResult.result.volumes[0].qos != null)
{
return volumeGetResult.result.volumes[0].qos.maxIOPS;
}
throw new CloudRuntimeException("Could not determine the max IOPS of the volume for volume ID of " + lVolumeId + ".");
}
private static long getVolumeBurstIops(VolumeGetResult volumeGetResult, long lVolumeId)
{
if (volumeGetResult.result.volumes != null && volumeGetResult.result.volumes.length == 1 &&
volumeGetResult.result.volumes[0].volumeID == lVolumeId && volumeGetResult.result.volumes[0].qos != null)
{
return volumeGetResult.result.volumes[0].qos.burstIOPS;
}
throw new CloudRuntimeException("Could not determine the burst IOPS of the volume for volume ID of " + lVolumeId + ".");
}
private static long getVolumeTotalSize(VolumeGetResult volumeGetResult, long lVolumeId)
{
if (volumeGetResult.result.volumes != null && volumeGetResult.result.volumes.length == 1 &&

View File

@ -1654,7 +1654,7 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
// volume.getPoolId() should be null if the VM we are detaching the disk from has never been started before
DataStore dataStore = volume.getPoolId() != null ? dataStoreMgr.getDataStore(volume.getPoolId(), DataStoreRole.Primary) : null;
volService.disconnectVolumeFromHost(volFactory.getVolume(volume.getId()), host, dataStore);
volService.revokeAccess(volFactory.getVolume(volume.getId()), host, dataStore);
return _volsDao.findById(volumeId);
} else {
@ -2163,10 +2163,10 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
// if we don't have a host, the VM we are attaching the disk to has never been started before
if (host != null) {
try {
volService.connectVolumeToHost(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
volService.grantAccess(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
}
catch (Exception e) {
volService.disconnectVolumeFromHost(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
volService.revokeAccess(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
throw new CloudRuntimeException(e.getMessage());
}
@ -2211,7 +2211,7 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
answer = (AttachAnswer)_agentMgr.send(hostId, cmd);
} catch (Exception e) {
if(host!=null) {
volService.disconnectVolumeFromHost(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
volService.revokeAccess(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
}
throw new CloudRuntimeException(errorMsg + " due to: " + e.getMessage());
}
@ -2250,7 +2250,7 @@ public class VolumeApiServiceImpl extends ManagerBase implements VolumeApiServic
}
}
if(host!= null) {
volService.disconnectVolumeFromHost(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
volService.revokeAccess(volFactory.getVolume(volumeToAttach.getId()), host, dataStore);
}
throw new CloudRuntimeException(errorMsg);
}

View File

@ -54,6 +54,7 @@ import org.apache.cloudstack.api.command.user.template.UpdateTemplatePermissions
import org.apache.cloudstack.context.CallContext;
import org.apache.cloudstack.engine.orchestration.service.VolumeOrchestrationService;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStore;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreCapabilities;
import org.apache.cloudstack.engine.subsystem.api.storage.DataStoreManager;
import org.apache.cloudstack.engine.subsystem.api.storage.EndPoint;
import org.apache.cloudstack.engine.subsystem.api.storage.EndPointSelector;
@ -1376,12 +1377,20 @@ public class TemplateManagerImpl extends ManagerBase implements TemplateManager,
throw new CloudRuntimeException("cannot find an image store for zone " + zoneId);
}
AsyncCallFuture<TemplateApiResult> future = null;
if (snapshotId != null) {
SnapshotInfo snapInfo = _snapshotFactory.getSnapshot(snapshotId, DataStoreRole.Image);
DataStore snapStore = snapInfo.getDataStore();
if (snapStore != null) {
store = snapStore; // pick snapshot image store to create template
DataStoreRole dataStoreRole = getDataStoreRole(snapshot);
SnapshotInfo snapInfo = _snapshotFactory.getSnapshot(snapshotId, dataStoreRole);
if (dataStoreRole == DataStoreRole.Image) {
DataStore snapStore = snapInfo.getDataStore();
if (snapStore != null) {
store = snapStore; // pick snapshot image store to create template
}
}
future = _tmpltSvr.createTemplateFromSnapshotAsync(snapInfo, tmplInfo, store);
} else if (volumeId != null) {
VolumeInfo volInfo = _volFactory.getVolume(volumeId);
@ -1465,6 +1474,25 @@ public class TemplateManagerImpl extends ManagerBase implements TemplateManager,
}
}
private DataStoreRole getDataStoreRole(Snapshot snapshot) {
long volumeId = snapshot.getVolumeId();
VolumeVO volumeVO = _volumeDao.findById(volumeId);
long storagePoolId = volumeVO.getPoolId();
DataStore dataStore = _dataStoreMgr.getDataStore(storagePoolId, DataStoreRole.Primary);
Map<String, String> mapCapabilities = dataStore.getDriver().getCapabilities();
String value = mapCapabilities.get(DataStoreCapabilities.STORAGE_SYSTEM_SNAPSHOT.toString());
Boolean supportsStorageSystemSnapshots = new Boolean(value);
if (supportsStorageSystemSnapshots) {
return DataStoreRole.Primary;
}
return DataStoreRole.Image;
}
@Override
@ActionEvent(eventType = EventTypes.EVENT_TEMPLATE_CREATE, eventDescription = "creating template", create = true)
public VMTemplateVO createPrivateTemplateRecord(CreateTemplateCmd cmd, Account templateOwner) throws ResourceAllocationException {

View File

@ -4905,7 +4905,7 @@ public class UserVmManagerImpl extends ManagerBase implements UserVmManager, Vir
// root.getPoolId() should be null if the VM we are detaching the disk from has never been started before
DataStore dataStore = root.getPoolId() != null ? _dataStoreMgr.getDataStore(root.getPoolId(), DataStoreRole.Primary) : null;
volumeMgr.disconnectVolumeFromHost(volFactory.getVolume(root.getId()), host, dataStore);
volumeMgr.revokeAccess(volFactory.getVolume(root.getId()), host, dataStore);
}
}
}