mirror of
				https://github.com/apache/cloudstack.git
				synced 2025-10-26 08:42:29 +01:00 
			
		
		
		
	fix bug in create volume from snapshot
This commit is contained in:
		
							parent
							
								
									564d3d6fa0
								
							
						
					
					
						commit
						68ffe1c706
					
				| @ -42,4 +42,5 @@ public interface StorageProcessor { | ||||
|     public Answer createVolume(CreateObjectCommand cmd); | ||||
|     public Answer createSnapshot(CreateObjectCommand cmd); | ||||
|     public Answer deleteVolume(DeleteCommand cmd); | ||||
|     public Answer createVolumeFromSnapshot(CopyCommand cmd); | ||||
| } | ||||
|  | ||||
| @ -82,6 +82,8 @@ public class StorageSubsystemCommandHandlerBase implements StorageSubsystemComma | ||||
|             } | ||||
|         } else if (srcData.getObjectType() == DataObjectType.SNAPSHOT && srcData.getDataStore().getRole() == DataStoreRole.Primary) { | ||||
|             return processor.backupSnasphot(cmd); | ||||
|         } else if (srcData.getObjectType() == DataObjectType.SNAPSHOT && destData.getObjectType() == DataObjectType.VOLUME) { | ||||
|         	return processor.createVolumeFromSnapshot(cmd); | ||||
|         } | ||||
| 
 | ||||
|         return new Answer(cmd, false, "not implemented yet"); | ||||
|  | ||||
| @ -231,12 +231,11 @@ public class AncientDataMotionStrategy implements DataMotionStrategy { | ||||
|                             .getDefaultValue())); | ||||
| 
 | ||||
|             CopyCommand cmd = new CopyCommand(srcData.getTO(), volObj.getTO(), _createVolumeFromSnapshotWait); | ||||
| 
 | ||||
| 
 | ||||
|             Answer answer = storageMgr | ||||
|                     .sendToPool(pool, cmd); | ||||
|             EndPoint ep = selector.select(snapObj, volObj); | ||||
|             Answer answer = ep.sendMessage(cmd); | ||||
|             | ||||
|            return answer; | ||||
|         } catch (StorageUnavailableException e) { | ||||
|         } catch (Exception e) { | ||||
|             s_logger.error(basicErrMsg, e); | ||||
|             throw new CloudRuntimeException(basicErrMsg); | ||||
|         } finally { | ||||
| @ -300,7 +299,7 @@ public class AncientDataMotionStrategy implements DataMotionStrategy { | ||||
|             } else if (srcData.getType() == DataObjectType.SNAPSHOT && | ||||
|             		destData.getType() == DataObjectType.SNAPSHOT) { | ||||
|             	answer = copySnapshot(srcData, destData); | ||||
|             } | ||||
|             }  | ||||
| 
 | ||||
|             if (answer != null && !answer.getResult()) { | ||||
|                 errMsg = answer.getDetails(); | ||||
|  | ||||
| @ -360,7 +360,7 @@ public class SnapshotTest extends CloudStackTestNGBase { | ||||
|         return null; | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     //@Test | ||||
|     public void createSnapshot() { | ||||
|         VolumeInfo vol = createCopyBaseImage(); | ||||
|         SnapshotVO snapshotVO = createSnapshotInDb(vol); | ||||
| @ -394,7 +394,7 @@ public class SnapshotTest extends CloudStackTestNGBase { | ||||
|         return image; | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     //@Test | ||||
|     public void createTemplateFromSnapshot() { | ||||
|         VolumeInfo vol = createCopyBaseImage(); | ||||
|         SnapshotVO snapshotVO = createSnapshotInDb(vol); | ||||
| @ -416,4 +416,24 @@ public class SnapshotTest extends CloudStackTestNGBase { | ||||
|         DataStore imageStore = this.dataStoreMgr.getImageStore(this.dcId); | ||||
|         this.imageService.createTemplateFromSnapshotAsync(snapshot, tmpl, imageStore); | ||||
|     } | ||||
|      | ||||
|     @Test | ||||
|     public void createVolumeFromSnapshot() { | ||||
|         VolumeInfo vol = createCopyBaseImage(); | ||||
|         SnapshotVO snapshotVO = createSnapshotInDb(vol); | ||||
|         SnapshotInfo snapshot = this.snapshotFactory.getSnapshot(snapshotVO.getId(), vol.getDataStore()); | ||||
|         boolean result = false; | ||||
|         for (SnapshotStrategy strategy : this.snapshotStrategies) { | ||||
|             if (strategy.canHandle(snapshot)) { | ||||
|                 snapshot = strategy.takeSnapshot(snapshot); | ||||
|                 result = true; | ||||
|             } | ||||
|         } | ||||
| 
 | ||||
|         AssertJUnit.assertTrue(result); | ||||
| 
 | ||||
|         VolumeVO volVO = createVolume(vol.getTemplateId(), vol.getPoolId()); | ||||
|         VolumeInfo newVol = this.volFactory.getVolume(volVO.getId()); | ||||
|         this.volumeService.createVolumeFromSnapshot(newVol, newVol.getDataStore(), snapshot); | ||||
|     } | ||||
| } | ||||
|  | ||||
| @ -539,7 +539,11 @@ public class VolumeServiceImpl implements VolumeService { | ||||
|     	} | ||||
| 
 | ||||
|     	try { | ||||
|     		volume.processEvent(event); | ||||
|     		if (result.isSuccess()) { | ||||
|     			volume.processEvent(event, result.getAnswer()); | ||||
|     		} else { | ||||
|     			volume.processEvent(event); | ||||
|     		} | ||||
|     		snapshot.processEvent(event); | ||||
|     	} catch (Exception e) { | ||||
|     		s_logger.debug("create volume from snapshot failed", e); | ||||
|  | ||||
| @ -20,7 +20,6 @@ package com.cloud.hypervisor.kvm.storage; | ||||
| 
 | ||||
| import java.io.File; | ||||
| import java.io.FileOutputStream; | ||||
| import java.io.IOException; | ||||
| import java.net.URISyntaxException; | ||||
| import java.text.DateFormat; | ||||
| import java.text.MessageFormat; | ||||
| @ -47,9 +46,9 @@ import org.apache.cloudstack.storage.to.SnapshotObjectTO; | ||||
| import org.apache.cloudstack.storage.to.TemplateObjectTO; | ||||
| import org.apache.cloudstack.storage.to.VolumeObjectTO; | ||||
| import org.apache.cloudstack.utils.qemu.QemuImg; | ||||
| import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat; | ||||
| import org.apache.cloudstack.utils.qemu.QemuImgException; | ||||
| import org.apache.cloudstack.utils.qemu.QemuImgFile; | ||||
| import org.apache.cloudstack.utils.qemu.QemuImg.PhysicalDiskFormat; | ||||
| import org.apache.log4j.Logger; | ||||
| import org.libvirt.Connect; | ||||
| import org.libvirt.Domain; | ||||
| @ -58,37 +57,28 @@ import org.libvirt.DomainSnapshot; | ||||
| import org.libvirt.LibvirtException; | ||||
| 
 | ||||
| import com.cloud.agent.api.Answer; | ||||
| import com.cloud.agent.api.AttachVolumeAnswer; | ||||
| import com.cloud.agent.api.BackupSnapshotAnswer; | ||||
| import com.cloud.agent.api.ManageSnapshotAnswer; | ||||
| import com.cloud.agent.api.ManageSnapshotCommand; | ||||
| import com.cloud.agent.api.storage.CreateAnswer; | ||||
| import com.cloud.agent.api.storage.CreatePrivateTemplateAnswer; | ||||
| import com.cloud.agent.api.storage.PrimaryStorageDownloadAnswer; | ||||
| import com.cloud.agent.api.to.DataStoreTO; | ||||
| import com.cloud.agent.api.to.DataTO; | ||||
| import com.cloud.agent.api.to.DiskTO; | ||||
| import com.cloud.agent.api.to.NfsTO; | ||||
| import com.cloud.agent.api.to.StorageFilerTO; | ||||
| import com.cloud.agent.api.to.VolumeTO; | ||||
| import com.cloud.exception.InternalErrorException; | ||||
| import com.cloud.hypervisor.kvm.resource.LibvirtComputingResource; | ||||
| import com.cloud.hypervisor.kvm.resource.LibvirtConnection; | ||||
| import com.cloud.hypervisor.kvm.resource.LibvirtDomainXMLParser; | ||||
| import com.cloud.hypervisor.kvm.resource.LibvirtVMDef.DiskDef; | ||||
| import com.cloud.storage.JavaStorageLayer; | ||||
| import com.cloud.storage.StorageLayer; | ||||
| import com.cloud.storage.Storage.ImageFormat; | ||||
| import com.cloud.storage.Storage.StoragePoolType; | ||||
| import com.cloud.storage.StorageLayer; | ||||
| import com.cloud.storage.resource.StorageProcessor; | ||||
| import com.cloud.storage.template.Processor; | ||||
| import com.cloud.storage.template.Processor.FormatInfo; | ||||
| import com.cloud.storage.template.QCOW2Processor; | ||||
| import com.cloud.storage.template.TemplateLocation; | ||||
| import com.cloud.storage.template.Processor.FormatInfo; | ||||
| import com.cloud.utils.NumbersUtil; | ||||
| import com.cloud.utils.exception.CloudRuntimeException; | ||||
| import com.cloud.utils.script.Script; | ||||
| import com.cloud.vm.DiskProfile; | ||||
| 
 | ||||
| public class KVMStorageProcessor implements StorageProcessor { | ||||
|     private static final Logger s_logger = Logger.getLogger(KVMStorageProcessor.class); | ||||
| @ -870,4 +860,46 @@ public class KVMStorageProcessor implements StorageProcessor { | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public Answer createVolumeFromSnapshot(CopyCommand cmd) { | ||||
|     	try { | ||||
|     		DataTO srcData = cmd.getSrcTO(); | ||||
|     		SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData; | ||||
|     		DataTO destData = cmd.getDestTO(); | ||||
|     		PrimaryDataStoreTO pool = (PrimaryDataStoreTO)destData.getDataStore(); | ||||
|     		DataStoreTO imageStore = srcData.getDataStore(); | ||||
| 
 | ||||
| 
 | ||||
|     		if (!(imageStore instanceof NfsTO)) { | ||||
|     			return new CopyCmdAnswer("unsupported protocol"); | ||||
|     		} | ||||
| 
 | ||||
|     		NfsTO nfsImageStore = (NfsTO)imageStore; | ||||
| 
 | ||||
|     		String snapshotPath = snapshot.getPath(); | ||||
|     		int index = snapshotPath.lastIndexOf("/"); | ||||
|     		snapshotPath = snapshotPath.substring(0, index); | ||||
|     		String snapshotName = snapshotPath.substring(index + 1); | ||||
|     		KVMStoragePool secondaryPool = storagePoolMgr.getStoragePoolByURI( | ||||
|     				nfsImageStore.getUrl() + File.separator + snapshotPath); | ||||
|     		KVMPhysicalDisk snapshotDisk = secondaryPool.getPhysicalDisk(snapshotName); | ||||
| 
 | ||||
|     		String primaryUuid = pool.getUuid(); | ||||
|     		KVMStoragePool primaryPool = storagePoolMgr | ||||
|     				.getStoragePool(pool.getPoolType(), | ||||
|     						primaryUuid); | ||||
|     		String volUuid = UUID.randomUUID().toString(); | ||||
|     		KVMPhysicalDisk disk = storagePoolMgr.copyPhysicalDisk(snapshotDisk, | ||||
|     				volUuid, primaryPool); | ||||
|     		VolumeObjectTO newVol = new VolumeObjectTO(); | ||||
|     		newVol.setPath(disk.getName()); | ||||
|     		newVol.setSize(disk.getVirtualSize()); | ||||
|     		return new CopyCmdAnswer( | ||||
|     				newVol); | ||||
|     	} catch (CloudRuntimeException e) { | ||||
|     		return new CopyCmdAnswer(e.toString() | ||||
|     				); | ||||
|     	} | ||||
|     } | ||||
| 
 | ||||
| } | ||||
|  | ||||
| @ -44,6 +44,7 @@ import org.apache.log4j.Logger; | ||||
| import com.cloud.agent.api.Answer; | ||||
| import com.cloud.agent.api.BackupSnapshotAnswer; | ||||
| import com.cloud.agent.api.Command; | ||||
| import com.cloud.agent.api.CreateVolumeFromSnapshotAnswer; | ||||
| import com.cloud.agent.api.ManageSnapshotAnswer; | ||||
| import com.cloud.agent.api.ManageSnapshotCommand; | ||||
| import com.cloud.agent.api.storage.CreatePrivateTemplateAnswer; | ||||
| @ -980,6 +981,7 @@ public class VmwareStorageProcessor implements StorageProcessor { | ||||
| 
 | ||||
| 				VolumeObjectTO newVol = new VolumeObjectTO(); | ||||
| 				newVol.setPath(volumeUuid); | ||||
| 				newVol.setSize(volume.getSize() / (1024L * 1024L)); | ||||
| 				return new CreateObjectAnswer(newVol); | ||||
| 			} finally { | ||||
| 				s_logger.info("Destroy dummy VM after volume creation"); | ||||
| @ -1154,4 +1156,120 @@ public class VmwareStorageProcessor implements StorageProcessor { | ||||
| 			return new Answer(cmd, false, msg); | ||||
| 		} | ||||
| 	} | ||||
| 	 | ||||
| 	 private Long restoreVolumeFromSecStorage(VmwareHypervisorHost hyperHost, DatastoreMO primaryDsMo, String newVolumeName, | ||||
| 		        String secStorageUrl, String secStorageDir, String backupName) throws Exception { | ||||
| 
 | ||||
| 		        String secondaryMountPoint = mountService.getMountPoint(secStorageUrl); | ||||
| 		        String srcOVAFileName = secondaryMountPoint + "/" +  secStorageDir + "/" | ||||
| 		            + backupName + "." + ImageFormat.OVA.getFileExtension(); | ||||
| 		        String snapshotDir = ""; | ||||
| 		        if (backupName.contains("/")){ | ||||
| 		            snapshotDir = backupName.split("/")[0]; | ||||
| 		        } | ||||
| 
 | ||||
| 		        File ovafile = new File(srcOVAFileName); | ||||
| 		        String srcOVFFileName = secondaryMountPoint + "/" +  secStorageDir + "/" | ||||
| 		                + backupName + ".ovf"; | ||||
| 		        File ovfFile = new File(srcOVFFileName); | ||||
| 		        // String srcFileName = getOVFFilePath(srcOVAFileName); | ||||
| 		        if (!ovfFile.exists()) { | ||||
| 		            srcOVFFileName = getOVFFilePath(srcOVAFileName); | ||||
| 		            if(srcOVFFileName == null && ovafile.exists() ) {  // volss: ova file exists; o/w can't do tar | ||||
| 		                Script command = new Script("tar", 0, s_logger); | ||||
| 		                command.add("--no-same-owner"); | ||||
| 		                command.add("-xf", srcOVAFileName); | ||||
| 		                command.setWorkDir(secondaryMountPoint + "/" +  secStorageDir + "/" + snapshotDir); | ||||
| 		                s_logger.info("Executing command: " + command.toString()); | ||||
| 		                String result = command.execute(); | ||||
| 		                if(result != null) { | ||||
| 		                        String msg = "Unable to unpack snapshot OVA file at: " + srcOVAFileName; | ||||
| 		                        s_logger.error(msg); | ||||
| 		                        throw new Exception(msg); | ||||
| 		                } | ||||
| 		            } else { | ||||
| 		               String msg = "Unable to find snapshot OVA file at: " + srcOVAFileName; | ||||
| 		               s_logger.error(msg); | ||||
| 		               throw new Exception(msg); | ||||
| 		           } | ||||
| 
 | ||||
| 		           srcOVFFileName = getOVFFilePath(srcOVAFileName); | ||||
| 		        } | ||||
| 		        if(srcOVFFileName == null) { | ||||
| 		            String msg = "Unable to locate OVF file in template package directory: " + srcOVAFileName; | ||||
| 		            s_logger.error(msg); | ||||
| 		            throw new Exception(msg); | ||||
| 		        } | ||||
| 
 | ||||
| 		        VirtualMachineMO clonedVm = null; | ||||
| 		        try { | ||||
| 		            hyperHost.importVmFromOVF(srcOVFFileName, newVolumeName, primaryDsMo, "thin"); | ||||
| 		            clonedVm = hyperHost.findVmOnHyperHost(newVolumeName); | ||||
| 		            if(clonedVm == null) | ||||
| 		                throw new Exception("Unable to create container VM for volume creation"); | ||||
| 
 | ||||
| 		            clonedVm.moveAllVmDiskFiles(primaryDsMo, "", false); | ||||
| 		            clonedVm.detachAllDisks(); | ||||
| 		            return _storage.getSize(srcOVFFileName); | ||||
| 		        } finally { | ||||
| 		            if(clonedVm != null) { | ||||
| 		                clonedVm.detachAllDisks(); | ||||
| 		                clonedVm.destroy(); | ||||
| 		            } | ||||
| 		        } | ||||
| 		    } | ||||
| 
 | ||||
| 	@Override | ||||
| 	public Answer createVolumeFromSnapshot(CopyCommand cmd) { | ||||
| 		DataTO srcData = cmd.getSrcTO(); | ||||
| 		SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData; | ||||
| 		DataTO destData = cmd.getDestTO(); | ||||
| 		PrimaryDataStoreTO pool = (PrimaryDataStoreTO)destData.getDataStore(); | ||||
| 		DataStoreTO imageStore = srcData.getDataStore(); | ||||
| 
 | ||||
| 
 | ||||
| 		if (!(imageStore instanceof NfsTO)) { | ||||
| 			return new CopyCmdAnswer("unsupported protocol"); | ||||
| 		} | ||||
| 
 | ||||
| 		NfsTO nfsImageStore = (NfsTO)imageStore; | ||||
| 		String primaryStorageNameLabel = pool.getUuid(); | ||||
| 	 | ||||
|         String secondaryStorageUrl = nfsImageStore.getUrl(); | ||||
| 		String backedUpSnapshotUuid = snapshot.getPath(); | ||||
| 		int index = backedUpSnapshotUuid.lastIndexOf(File.separator); | ||||
| 		String backupPath = backedUpSnapshotUuid.substring(0, index); | ||||
| 		backedUpSnapshotUuid = backedUpSnapshotUuid.substring(index + 1); | ||||
| 		String details = null; | ||||
| 		String newVolumeName = UUID.randomUUID().toString().replaceAll("-", ""); | ||||
| 
 | ||||
| 		VmwareContext context = hostService.getServiceContext(cmd); | ||||
| 		try { | ||||
| 			VmwareHypervisorHost hyperHost = hostService.getHyperHost(context, cmd); | ||||
| 			ManagedObjectReference morPrimaryDs = HypervisorHostHelper.findDatastoreWithBackwardsCompatibility(hyperHost, | ||||
| 					primaryStorageNameLabel); | ||||
| 			if (morPrimaryDs == null) { | ||||
| 				String msg = "Unable to find datastore: " + primaryStorageNameLabel; | ||||
| 				s_logger.error(msg); | ||||
| 				throw new Exception(msg); | ||||
| 			} | ||||
| 
 | ||||
| 			DatastoreMO primaryDsMo = new DatastoreMO(hyperHost.getContext(), morPrimaryDs); | ||||
| 			Long size = restoreVolumeFromSecStorage(hyperHost, primaryDsMo, | ||||
| 					newVolumeName, secondaryStorageUrl, backupPath, backedUpSnapshotUuid); | ||||
| 
 | ||||
| 			VolumeObjectTO newVol = new VolumeObjectTO(); | ||||
| 			newVol.setPath(newVolumeName); | ||||
| 			newVol.setSize(size); | ||||
| 			return new CopyCmdAnswer(newVol); | ||||
| 		} catch (Throwable e) { | ||||
| 			if (e instanceof RemoteException) { | ||||
| 				hostService.invalidateServiceContext(context); | ||||
| 			} | ||||
| 
 | ||||
| 			s_logger.error("Unexpecpted exception ", e); | ||||
| 			details = "CreateVolumeFromSnapshotCommand exception: " + StringUtils.getExceptionStackInfo(e); | ||||
| 		} | ||||
| 		return new CopyCmdAnswer(details); | ||||
| 	} | ||||
| } | ||||
|  | ||||
| @ -55,6 +55,7 @@ import org.apache.xmlrpc.XmlRpcException; | ||||
| 
 | ||||
| import com.cloud.agent.api.Answer; | ||||
| import com.cloud.agent.api.CreateStoragePoolCommand; | ||||
| import com.cloud.agent.api.CreateVolumeFromSnapshotAnswer; | ||||
| import com.cloud.agent.api.to.DataObjectType; | ||||
| import com.cloud.agent.api.to.DataStoreTO; | ||||
| import com.cloud.agent.api.to.DataTO; | ||||
| @ -1402,4 +1403,71 @@ public class XenServerStorageProcessor implements StorageProcessor { | ||||
|         } | ||||
|         return new CopyCmdAnswer(details); | ||||
|     } | ||||
| 
 | ||||
| 	@Override | ||||
| 	public Answer createVolumeFromSnapshot(CopyCommand cmd) { | ||||
| 		Connection conn = this.hypervisorResource.getConnection(); | ||||
| 		DataTO srcData = cmd.getSrcTO(); | ||||
| 		SnapshotObjectTO snapshot = (SnapshotObjectTO)srcData; | ||||
| 		DataTO destData = cmd.getDestTO(); | ||||
| 		PrimaryDataStoreTO pool = (PrimaryDataStoreTO)destData.getDataStore(); | ||||
| 		DataStoreTO imageStore = srcData.getDataStore(); | ||||
| 
 | ||||
| 		if (!(imageStore instanceof NfsTO)) { | ||||
| 			return new CopyCmdAnswer("unsupported protocol"); | ||||
| 		} | ||||
| 
 | ||||
| 		NfsTO nfsImageStore = (NfsTO)imageStore; | ||||
| 		String primaryStorageNameLabel = pool.getUuid(); | ||||
| 		String secondaryStorageUrl = nfsImageStore.getUrl(); | ||||
| 		int wait = cmd.getWait(); | ||||
| 		boolean result = false; | ||||
| 		// Generic error message. | ||||
| 		String details = null; | ||||
| 		String volumeUUID = null; | ||||
| 
 | ||||
| 		if (secondaryStorageUrl == null) { | ||||
| 			details += " because the URL passed: " + secondaryStorageUrl + " is invalid."; | ||||
| 			return new CopyCmdAnswer(details); | ||||
| 		} | ||||
| 		try { | ||||
| 			SR primaryStorageSR = this.hypervisorResource.getSRByNameLabelandHost(conn, primaryStorageNameLabel); | ||||
| 			if (primaryStorageSR == null) { | ||||
| 				throw new InternalErrorException("Could not create volume from snapshot because the primary Storage SR could not be created from the name label: " | ||||
| 						+ primaryStorageNameLabel); | ||||
| 			} | ||||
| 			// Get the absolute path of the snapshot on the secondary storage. | ||||
| 			String snapshotInstallPath = snapshot.getPath(); | ||||
| 			int index = snapshotInstallPath.lastIndexOf(File.separator); | ||||
| 			String snapshotName = snapshotInstallPath.substring(index + 1); | ||||
| 		 | ||||
| 			if (!snapshotName.startsWith("VHD-") && !snapshotName.endsWith(".vhd")) { | ||||
| 				snapshotInstallPath = snapshotInstallPath + ".vhd"; | ||||
| 			} | ||||
| 			URI snapshotURI = new URI(secondaryStorageUrl + File.separator + snapshotInstallPath); | ||||
| 			String snapshotPath = snapshotURI.getHost() + ":" + snapshotURI.getPath(); | ||||
| 			String srUuid = primaryStorageSR.getUuid(conn); | ||||
| 			volumeUUID = copy_vhd_from_secondarystorage(conn, snapshotPath, srUuid, wait); | ||||
| 			result = true; | ||||
| 			VDI volume = VDI.getByUuid(conn, volumeUUID); | ||||
| 			VDI.Record vdir = volume.getRecord(conn); | ||||
| 			VolumeObjectTO newVol = new VolumeObjectTO(); | ||||
| 			newVol.setPath(volumeUUID); | ||||
| 			newVol.setSize(vdir.virtualSize); | ||||
| 			return new CopyCmdAnswer(newVol); | ||||
| 		} catch (XenAPIException e) { | ||||
| 			details += " due to " + e.toString(); | ||||
| 			s_logger.warn(details, e); | ||||
| 		} catch (Exception e) { | ||||
| 			details += " due to " + e.getMessage(); | ||||
| 			s_logger.warn(details, e); | ||||
| 		} | ||||
| 		if (!result) { | ||||
| 			// Is this logged at a higher level? | ||||
| 			s_logger.error(details); | ||||
| 		} | ||||
| 
 | ||||
| 		// In all cases return something. | ||||
| 		return new CopyCmdAnswer(details); | ||||
| 	} | ||||
| } | ||||
|  | ||||
		Loading…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user