mirror of
				https://github.com/apache/cloudstack.git
				synced 2025-10-26 08:42:29 +01:00 
			
		
		
		
	Merge release branch 4.20 to main
* 4.20: server: investigate pending HA work when executing in new MS session (#10167) extra null guard (#10264)
This commit is contained in:
		
						commit
						048649d351
					
				| @ -84,6 +84,13 @@ public interface HighAvailabilityManager extends Manager { | ||||
|         HA;         // Restart a VM. | ||||
|     } | ||||
| 
 | ||||
|     enum ReasonType { | ||||
|         Unknown, | ||||
|         HostMaintenance, | ||||
|         HostDown, | ||||
|         HostDegraded; | ||||
|     } | ||||
| 
 | ||||
|     enum Step { | ||||
|         Scheduled, Investigating, Fencing, Stopping, Restarting, Migrating, Cancelled, Done, Error, | ||||
|     } | ||||
| @ -92,7 +99,7 @@ public interface HighAvailabilityManager extends Manager { | ||||
|      * Investigate why a host has disconnected and migrate the VMs on it | ||||
|      * if necessary. | ||||
|      * | ||||
|      * @param host - the host that has disconnected. | ||||
|      * @param hostId - the id of the host that has disconnected. | ||||
|      */ | ||||
|     Status investigate(long hostId); | ||||
| 
 | ||||
| @ -109,17 +116,19 @@ public interface HighAvailabilityManager extends Manager { | ||||
|      * @param investigate must be investigated before we do anything with this vm. | ||||
|      */ | ||||
|     void scheduleRestart(VMInstanceVO vm, boolean investigate); | ||||
|     void scheduleRestart(VMInstanceVO vm, boolean investigate, ReasonType reasonType); | ||||
| 
 | ||||
|     void cancelDestroy(VMInstanceVO vm, Long hostId); | ||||
| 
 | ||||
|     boolean scheduleDestroy(VMInstanceVO vm, long hostId); | ||||
|     boolean scheduleDestroy(VMInstanceVO vm, long hostId, ReasonType reasonType); | ||||
| 
 | ||||
|     /** | ||||
|      * Schedule restarts for all vms running on the host. | ||||
|      * @param host host. | ||||
|      * @param investigate TODO | ||||
|      * @param investigate whether to investigate | ||||
|      * @param reasonType reason for HA work | ||||
|      */ | ||||
|     void scheduleRestartForVmsOnHost(HostVO host, boolean investigate); | ||||
|     void scheduleRestartForVmsOnHost(HostVO host, boolean investigate, ReasonType reasonType); | ||||
| 
 | ||||
|     /** | ||||
|      * Schedule the vm for migration. | ||||
| @ -128,6 +137,7 @@ public interface HighAvailabilityManager extends Manager { | ||||
|      * @return true if schedule worked. | ||||
|      */ | ||||
|     boolean scheduleMigration(VMInstanceVO vm); | ||||
|     boolean scheduleMigration(VMInstanceVO vm, ReasonType reasonType); | ||||
| 
 | ||||
|     List<VMInstanceVO> findTakenMigrationWork(); | ||||
| 
 | ||||
| @ -140,10 +150,11 @@ public interface HighAvailabilityManager extends Manager { | ||||
|      * 3. Check if a VM has been stopped: WorkType.CheckStop | ||||
|      * | ||||
|      * @param vm virtual machine to stop. | ||||
|      * @param host host the virtual machine is on. | ||||
|      * @param hostId the id of the host the virtual machine is on. | ||||
|      * @param type which type of stop is requested. | ||||
|      */ | ||||
|     boolean scheduleStop(VMInstanceVO vm, long hostId, WorkType type); | ||||
|     boolean scheduleStop(VMInstanceVO vm, long hostId, WorkType type, ReasonType reasonType); | ||||
| 
 | ||||
|     void cancelScheduledMigrations(HostVO host); | ||||
| 
 | ||||
|  | ||||
| @ -633,11 +633,11 @@ public class AgentManagerImpl extends ManagerBase implements AgentManager, Handl | ||||
|                     } | ||||
|                 } catch (final HypervisorVersionChangedException hvce) { | ||||
|                     handleDisconnectWithoutInvestigation(attache, Event.ShutdownRequested, true, true); | ||||
|                     throw new CloudRuntimeException("Unable to connect " + attache.getId(), hvce); | ||||
|                     throw new CloudRuntimeException("Unable to connect " + (attache == null ? "<unknown agent>" : attache.getId()), hvce); | ||||
|                 } catch (final Exception e) { | ||||
|                     logger.error("Monitor {} says there is an error in the connect process for {} due to {}", monitor.second().getClass().getSimpleName(), hostId, e.getMessage(), e); | ||||
|                     handleDisconnectWithoutInvestigation(attache, Event.AgentDisconnected, true, true); | ||||
|                     throw new CloudRuntimeException("Unable to connect " + attache.getId(), e); | ||||
|                     throw new CloudRuntimeException("Unable to connect " + (attache == null ? "<unknown agent>" : attache.getId()), e); | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
| @ -989,7 +989,7 @@ public class AgentManagerImpl extends ManagerBase implements AgentManager, Handl | ||||
|         handleDisconnectWithoutInvestigation(attache, event, true, true); | ||||
|         host = _hostDao.findById(hostId); // Maybe the host magically reappeared? | ||||
|         if (host != null && host.getStatus() == Status.Down) { | ||||
|             _haMgr.scheduleRestartForVmsOnHost(host, true); | ||||
|             _haMgr.scheduleRestartForVmsOnHost(host, true, HighAvailabilityManager.ReasonType.HostDown); | ||||
|         } | ||||
|         return true; | ||||
|     } | ||||
|  | ||||
| @ -35,3 +35,6 @@ CALL `cloud`.`IDEMPOTENT_ADD_COLUMN`('cloud.volumes', 'last_id', 'bigint(20) uns | ||||
| 
 | ||||
| -- Add used_iops column to support IOPS data in storage stats | ||||
| CALL `cloud`.`IDEMPOTENT_ADD_COLUMN`('cloud.storage_pool', 'used_iops', 'bigint unsigned DEFAULT NULL COMMENT "IOPS currently in use for this storage pool" '); | ||||
| 
 | ||||
| -- Add reason column for op_ha_work | ||||
| CALL `cloud`.`IDEMPOTENT_ADD_COLUMN`('cloud.op_ha_work', 'reason', 'varchar(32) DEFAULT NULL COMMENT "Reason for the HA work"'); | ||||
|  | ||||
| @ -86,6 +86,10 @@ public class HaWorkVO implements InternalIdentity { | ||||
|     @Column(name = "tried") | ||||
|     int timesTried; | ||||
| 
 | ||||
|     @Column(name = "reason") | ||||
|     @Enumerated(value = EnumType.STRING) | ||||
|     private HighAvailabilityManager.ReasonType reasonType; | ||||
| 
 | ||||
|     protected HaWorkVO() { | ||||
|     } | ||||
| 
 | ||||
| @ -179,7 +183,7 @@ public class HaWorkVO implements InternalIdentity { | ||||
|     } | ||||
| 
 | ||||
|     public HaWorkVO(final long instanceId, final VirtualMachine.Type type, final WorkType workType, final Step step, final long hostId, final State previousState, | ||||
|             final int timesTried, final long updated) { | ||||
|             final int timesTried, final long updated, HighAvailabilityManager.ReasonType reasonType) { | ||||
|         this.workType = workType; | ||||
|         this.type = type; | ||||
|         this.instanceId = instanceId; | ||||
| @ -191,6 +195,7 @@ public class HaWorkVO implements InternalIdentity { | ||||
|         this.step = step; | ||||
|         this.timeToTry = System.currentTimeMillis() >> 10; | ||||
|         this.updateTime = updated; | ||||
|         this.reasonType = reasonType; | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
| @ -207,4 +212,12 @@ public class HaWorkVO implements InternalIdentity { | ||||
|             .append("]") | ||||
|             .toString(); | ||||
|     } | ||||
| 
 | ||||
|     public HighAvailabilityManager.ReasonType getReasonType() { | ||||
|         return reasonType; | ||||
|     } | ||||
| 
 | ||||
|     public void setReasonType(HighAvailabilityManager.ReasonType reasonType) { | ||||
|         this.reasonType = reasonType; | ||||
|     } | ||||
| } | ||||
|  | ||||
| @ -19,6 +19,7 @@ package com.cloud.ha; | ||||
| import static org.apache.cloudstack.framework.config.ConfigKey.Scope.Zone; | ||||
| 
 | ||||
| import java.util.ArrayList; | ||||
| import java.util.Arrays; | ||||
| import java.util.Date; | ||||
| import java.util.HashMap; | ||||
| import java.util.List; | ||||
| @ -43,6 +44,7 @@ import org.apache.cloudstack.framework.config.dao.ConfigurationDao; | ||||
| import org.apache.cloudstack.managed.context.ManagedContext; | ||||
| import org.apache.cloudstack.managed.context.ManagedContextRunnable; | ||||
| import org.apache.cloudstack.management.ManagementServerHost; | ||||
| import org.apache.logging.log4j.ThreadContext; | ||||
| 
 | ||||
| import com.cloud.agent.AgentManager; | ||||
| import com.cloud.alert.AlertManager; | ||||
| @ -90,7 +92,6 @@ import com.cloud.vm.VirtualMachine; | ||||
| import com.cloud.vm.VirtualMachineManager; | ||||
| import com.cloud.vm.VirtualMachineProfile; | ||||
| import com.cloud.vm.dao.VMInstanceDao; | ||||
| import org.apache.logging.log4j.ThreadContext; | ||||
| 
 | ||||
| /** | ||||
|  * HighAvailabilityManagerImpl coordinates the HA process. VMs are registered with the HA Manager for HA. The request is stored | ||||
| @ -133,6 +134,9 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|     protected static ConfigKey<Boolean> VmHaAlertsEnabled = new ConfigKey<>("Advanced", Boolean.class, "vm.ha.alerts.enabled", "true", | ||||
|             "Enable/Disable alerts for the VM HA operations, it is enabled by default.", true, Zone); | ||||
| 
 | ||||
|     protected static final List<ReasonType> CancellableWorkReasonTypes = | ||||
|             Arrays.asList(ReasonType.HostMaintenance, ReasonType.HostDown, ReasonType.HostDegraded); | ||||
| 
 | ||||
|     WorkerThread[] _workers; | ||||
|     boolean _stopped; | ||||
|     long _timeToSleep; | ||||
| @ -269,8 +273,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void scheduleRestartForVmsOnHost(final HostVO host, boolean investigate) { | ||||
| 
 | ||||
|     public void scheduleRestartForVmsOnHost(final HostVO host, boolean investigate, ReasonType reasonType) { | ||||
|         if (host.getType() != Host.Type.Routing) { | ||||
|             return; | ||||
|         } | ||||
| @ -337,12 +340,12 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|                 logger.debug("VM {} is not on down host {} it is on other host {} VM HA is done", vm, host, hostId); | ||||
|                 continue; | ||||
|             } | ||||
|             scheduleRestart(vm, investigate); | ||||
|             scheduleRestart(vm, investigate, reasonType); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public boolean scheduleStop(VMInstanceVO vm, long hostId, WorkType type) { | ||||
|     public boolean scheduleStop(VMInstanceVO vm, long hostId, WorkType type, ReasonType reasonType) { | ||||
|         assert (type == WorkType.CheckStop || type == WorkType.ForceStop || type == WorkType.Stop); | ||||
| 
 | ||||
|         if (_haDao.hasBeenScheduled(vm.getId(), type)) { | ||||
| @ -359,7 +362,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             return false; | ||||
|         } | ||||
| 
 | ||||
|         HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), type, Step.Scheduled, hostId, vm.getState(), 0, vm.getUpdated()); | ||||
|         HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), type, Step.Scheduled, hostId, vm.getState(), 0, vm.getUpdated(), reasonType); | ||||
|         _haDao.persist(work); | ||||
|         if (logger.isDebugEnabled()) { | ||||
|             logger.debug("Scheduled " + work); | ||||
| @ -368,6 +371,11 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|         return true; | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public boolean scheduleStop(VMInstanceVO vm, long hostId, WorkType type) { | ||||
|         return scheduleStop(vm, hostId, type, null); | ||||
|     } | ||||
| 
 | ||||
|     protected void wakeupWorkers() { | ||||
|         logger.debug("Wakeup workers HA"); | ||||
|         for (WorkerThread worker : _workers) { | ||||
| @ -376,7 +384,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public boolean scheduleMigration(final VMInstanceVO vm) { | ||||
|     public boolean scheduleMigration(final VMInstanceVO vm, ReasonType reasonType) { | ||||
|         if (vm.getHostId() == null) { | ||||
|             return false; | ||||
|         } | ||||
| @ -390,7 +398,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             return false; | ||||
|         } | ||||
| 
 | ||||
|         final HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), WorkType.Migration, Step.Scheduled, vm.getHostId(), vm.getState(), 0, vm.getUpdated()); | ||||
|         final HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), WorkType.Migration, Step.Scheduled, vm.getHostId(), vm.getState(), 0, vm.getUpdated(), reasonType); | ||||
|         _haDao.persist(work); | ||||
|         logger.info("Scheduled migration work of VM {} from host {} with HAWork {}", vm, _hostDao.findById(vm.getHostId()), work); | ||||
|         wakeupWorkers(); | ||||
| @ -398,7 +406,12 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void scheduleRestart(VMInstanceVO vm, boolean investigate) { | ||||
|     public boolean scheduleMigration(final VMInstanceVO vm) { | ||||
|         return scheduleMigration(vm, null); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void scheduleRestart(VMInstanceVO vm, boolean investigate, ReasonType reasonType) { | ||||
|         if (!VmHaEnabled.valueIn(vm.getDataCenterId())) { | ||||
|             String message = String.format("Unable to schedule restart for the VM %s (%d), VM high availability manager is disabled.", vm.getName(), vm.getId()); | ||||
|             if (logger.isDebugEnabled()) { | ||||
| @ -490,7 +503,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|         } | ||||
| 
 | ||||
|         HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), WorkType.HA, investigate ? Step.Investigating : Step.Scheduled, | ||||
|                 hostId != null ? hostId : 0L, vm.getState(), timesTried, vm.getUpdated()); | ||||
|                 hostId != null ? hostId : 0L, vm.getState(), timesTried, vm.getUpdated(), reasonType); | ||||
|         _haDao.persist(work); | ||||
| 
 | ||||
|         if (logger.isInfoEnabled()) { | ||||
| @ -500,6 +513,11 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|         wakeupWorkers(); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void scheduleRestart(VMInstanceVO vm, boolean investigate) { | ||||
|         scheduleRestart(vm, investigate, null); | ||||
|     } | ||||
| 
 | ||||
|     private void startVm(VirtualMachine vm, Map<VirtualMachineProfile.Param, Object> params, | ||||
|            DeploymentPlanner planner) throws InsufficientCapacityException, ResourceUnavailableException, | ||||
|             ConcurrentOperationException, OperationTimedoutException { | ||||
| @ -561,6 +579,9 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             logger.info("Unable to find vm: " + vmId); | ||||
|             return null; | ||||
|         } | ||||
|         if (checkAndCancelWorkIfNeeded(work)) { | ||||
|             return null; | ||||
|         } | ||||
| 
 | ||||
|         logger.info("HA on " + vm); | ||||
|         if (vm.getState() != work.getPreviousState() || vm.getUpdated() != work.getUpdateTime()) { | ||||
| @ -762,6 +783,22 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|         return (System.currentTimeMillis() >> 10) + _restartRetryInterval; | ||||
|     } | ||||
| 
 | ||||
|     protected boolean checkAndCancelWorkIfNeeded(final HaWorkVO work) { | ||||
|         if (!Step.Investigating.equals(work.getStep())) { | ||||
|             return false; | ||||
|         } | ||||
|         if (!CancellableWorkReasonTypes.contains(work.getReasonType())) { | ||||
|             return false; | ||||
|         } | ||||
|         Status hostStatus = investigate(work.getHostId()); | ||||
|         if (!Status.Up.equals(hostStatus)) { | ||||
|             return false; | ||||
|         } | ||||
|         logger.debug("Cancelling {} as it is not needed anymore", () -> work); | ||||
|         work.setStep(Step.Cancelled); | ||||
|         return true; | ||||
|     } | ||||
| 
 | ||||
|     public Long migrate(final HaWorkVO work) { | ||||
|         long vmId = work.getInstanceId(); | ||||
|         long srcHostId = work.getHostId(); | ||||
| @ -772,6 +809,9 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             logger.info("Unable to find vm: " + vmId + ", skipping migrate."); | ||||
|             return null; | ||||
|         } | ||||
|         if (checkAndCancelWorkIfNeeded(work)) { | ||||
|             return null; | ||||
|         } | ||||
|         logger.info("Migration attempt: for VM {}from host {}. Starting attempt: {}/{} times.", vm, srcHost, 1 + work.getTimesTried(), _maxRetries); | ||||
|         try { | ||||
|             work.setStep(Step.Migrating); | ||||
| @ -791,7 +831,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public boolean scheduleDestroy(VMInstanceVO vm, long hostId) { | ||||
|     public boolean scheduleDestroy(VMInstanceVO vm, long hostId, ReasonType reasonType) { | ||||
|         if (!VmHaEnabled.valueIn(vm.getDataCenterId())) { | ||||
|             String message = String.format("Unable to schedule destroy for the VM %s (%d) on host %d, VM high availability manager is disabled.", vm.getName(), vm.getId(), hostId); | ||||
|             if (logger.isDebugEnabled()) { | ||||
| @ -801,7 +841,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             return false; | ||||
|         } | ||||
| 
 | ||||
|         final HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), WorkType.Destroy, Step.Scheduled, hostId, vm.getState(), 0, vm.getUpdated()); | ||||
|         final HaWorkVO work = new HaWorkVO(vm.getId(), vm.getType(), WorkType.Destroy, Step.Scheduled, hostId, vm.getState(), 0, vm.getUpdated(), reasonType); | ||||
|         _haDao.persist(work); | ||||
|         if (logger.isDebugEnabled()) { | ||||
|             logger.debug("Scheduled " + work.toString()); | ||||
| @ -838,6 +878,9 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             logger.info("No longer can find VM " + work.getInstanceId() + ". Throwing away " + work); | ||||
|             return null; | ||||
|         } | ||||
|         if (checkAndCancelWorkIfNeeded(work)) { | ||||
|             return null; | ||||
|         } | ||||
|         boolean expunge = VirtualMachine.Type.SecondaryStorageVm.equals(vm.getType()) | ||||
|                 || VirtualMachine.Type.ConsoleProxy.equals(vm.getType()); | ||||
|         if (!expunge && VirtualMachine.State.Destroyed.equals(work.getPreviousState())) { | ||||
| @ -872,6 +915,9 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|             work.setStep(Step.Done); | ||||
|             return null; | ||||
|         } | ||||
|         if (checkAndCancelWorkIfNeeded(work)) { | ||||
|             return null; | ||||
|         } | ||||
|         logger.info("Stopping " + vm); | ||||
|         try { | ||||
|             if (work.getWorkType() == WorkType.Stop) { | ||||
| @ -1057,6 +1103,8 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
|     public boolean start() { | ||||
|         _stopped = false; | ||||
| 
 | ||||
|         _haDao.markPendingWorksAsInvestigating(); | ||||
| 
 | ||||
|         for (final WorkerThread thread : _workers) { | ||||
|             thread.start(); | ||||
|         } | ||||
| @ -1074,6 +1122,8 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements Configur | ||||
| 
 | ||||
|         _executor.shutdown(); | ||||
| 
 | ||||
|         _haDao.markServerPendingWorksAsInvestigating(_msServer.getId()); | ||||
| 
 | ||||
|         return true; | ||||
|     } | ||||
| 
 | ||||
|  | ||||
| @ -86,4 +86,6 @@ public interface HighAvailabilityDao extends GenericDao<HaWorkVO, Long> { | ||||
| 
 | ||||
|     List<HaWorkVO> listPendingMigrationsForVm(long vmId); | ||||
|     int expungeByVmList(List<Long> vmIds, Long batchSize); | ||||
|     void markPendingWorksAsInvestigating(); | ||||
|     void markServerPendingWorksAsInvestigating(long managementServerId); | ||||
| } | ||||
|  | ||||
| @ -31,12 +31,13 @@ import com.cloud.utils.db.SearchBuilder; | ||||
| import com.cloud.utils.db.SearchCriteria; | ||||
| import com.cloud.utils.db.SearchCriteria.Op; | ||||
| import com.cloud.utils.db.TransactionLegacy; | ||||
| import com.cloud.utils.db.UpdateBuilder; | ||||
| import com.cloud.utils.exception.CloudRuntimeException; | ||||
| 
 | ||||
| @Component | ||||
| public class HighAvailabilityDaoImpl extends GenericDaoBase<HaWorkVO, Long> implements HighAvailabilityDao { | ||||
| 
 | ||||
|     private final SearchBuilder<HaWorkVO> TBASearch; | ||||
|     protected SearchBuilder<HaWorkVO> TBASearch; | ||||
|     private final SearchBuilder<HaWorkVO> PreviousInstanceSearch; | ||||
|     private final SearchBuilder<HaWorkVO> UntakenMigrationSearch; | ||||
|     private final SearchBuilder<HaWorkVO> CleanupSearch; | ||||
| @ -270,4 +271,31 @@ public class HighAvailabilityDaoImpl extends GenericDaoBase<HaWorkVO, Long> impl | ||||
|         sc.setParameters("vmIds", vmIds.toArray()); | ||||
|         return batchExpunge(sc, batchSize); | ||||
|     } | ||||
| 
 | ||||
|     protected void updatePendingWorkToInvestigating(SearchCriteria<HaWorkVO> sc) { | ||||
|         HaWorkVO haWorkVO = createForUpdate(); | ||||
|         haWorkVO.setStep(Step.Investigating); | ||||
|         UpdateBuilder updateBuilder = getUpdateBuilder(haWorkVO); | ||||
|         update(updateBuilder, sc, null); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void markPendingWorksAsInvestigating() { | ||||
|         final SearchCriteria<HaWorkVO> sc = TBASearch.create(); | ||||
|         sc.setParameters("time", System.currentTimeMillis() >> 10); | ||||
|         sc.setParameters("step", Step.Done, Step.Cancelled); | ||||
|         updatePendingWorkToInvestigating(sc); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void markServerPendingWorksAsInvestigating(long managementServerId) { | ||||
|         SearchBuilder<HaWorkVO> sb = createSearchBuilder(); | ||||
|         sb.and("server", sb.entity().getServerId(), Op.EQ); | ||||
|         sb.and("step", sb.entity().getStep(), Op.NIN); | ||||
|         sb.done(); | ||||
|         SearchCriteria<HaWorkVO> sc = sb.create(); | ||||
|         sc.setParameters("server", managementServerId); | ||||
|         sc.setParameters("step", Step.Done, Step.Cancelled); | ||||
|         updatePendingWorkToInvestigating(sc); | ||||
|     } | ||||
| } | ||||
|  | ||||
| @ -37,16 +37,6 @@ import java.util.stream.Collectors; | ||||
| import javax.inject.Inject; | ||||
| import javax.naming.ConfigurationException; | ||||
| 
 | ||||
| import com.cloud.alert.AlertManager; | ||||
| import com.cloud.cpu.CPU; | ||||
| import com.cloud.exception.StorageConflictException; | ||||
| import com.cloud.exception.StorageUnavailableException; | ||||
| import com.cloud.ha.HighAvailabilityManagerImpl; | ||||
| import com.cloud.host.HostTagVO; | ||||
| import com.cloud.storage.Volume; | ||||
| import com.cloud.storage.VolumeVO; | ||||
| import com.cloud.storage.dao.VolumeDao; | ||||
| import com.cloud.hypervisor.HypervisorGuru; | ||||
| import org.apache.cloudstack.alert.AlertService; | ||||
| import org.apache.cloudstack.annotation.AnnotationService; | ||||
| import org.apache.cloudstack.annotation.dao.AnnotationDao; | ||||
| @ -93,6 +83,7 @@ import com.cloud.agent.api.UpdateHostPasswordCommand; | ||||
| import com.cloud.agent.api.VgpuTypesInfo; | ||||
| import com.cloud.agent.api.to.GPUDeviceTO; | ||||
| import com.cloud.agent.transport.Request; | ||||
| import com.cloud.alert.AlertManager; | ||||
| import com.cloud.capacity.Capacity; | ||||
| import com.cloud.capacity.CapacityManager; | ||||
| import com.cloud.capacity.CapacityState; | ||||
| @ -101,6 +92,7 @@ import com.cloud.capacity.dao.CapacityDao; | ||||
| import com.cloud.cluster.ClusterManager; | ||||
| import com.cloud.configuration.Config; | ||||
| import com.cloud.configuration.ConfigurationManager; | ||||
| import com.cloud.cpu.CPU; | ||||
| import com.cloud.dc.ClusterDetailsDao; | ||||
| import com.cloud.dc.ClusterDetailsVO; | ||||
| import com.cloud.dc.ClusterVO; | ||||
| @ -134,6 +126,8 @@ import com.cloud.exception.InvalidParameterValueException; | ||||
| import com.cloud.exception.PermissionDeniedException; | ||||
| import com.cloud.exception.ResourceInUseException; | ||||
| import com.cloud.exception.ResourceUnavailableException; | ||||
| import com.cloud.exception.StorageConflictException; | ||||
| import com.cloud.exception.StorageUnavailableException; | ||||
| import com.cloud.gpu.GPU; | ||||
| import com.cloud.gpu.HostGpuGroupsVO; | ||||
| import com.cloud.gpu.VGPUTypesVO; | ||||
| @ -141,10 +135,12 @@ import com.cloud.gpu.dao.HostGpuGroupsDao; | ||||
| import com.cloud.gpu.dao.VGPUTypesDao; | ||||
| import com.cloud.ha.HighAvailabilityManager; | ||||
| import com.cloud.ha.HighAvailabilityManager.WorkType; | ||||
| import com.cloud.ha.HighAvailabilityManagerImpl; | ||||
| import com.cloud.host.DetailVO; | ||||
| import com.cloud.host.Host; | ||||
| import com.cloud.host.Host.Type; | ||||
| import com.cloud.host.HostStats; | ||||
| import com.cloud.host.HostTagVO; | ||||
| import com.cloud.host.HostVO; | ||||
| import com.cloud.host.Status; | ||||
| import com.cloud.host.Status.Event; | ||||
| @ -153,6 +149,7 @@ import com.cloud.host.dao.HostDetailsDao; | ||||
| import com.cloud.host.dao.HostTagsDao; | ||||
| import com.cloud.hypervisor.Hypervisor; | ||||
| import com.cloud.hypervisor.Hypervisor.HypervisorType; | ||||
| import com.cloud.hypervisor.HypervisorGuru; | ||||
| import com.cloud.hypervisor.kvm.discoverer.KvmDummyResourceBase; | ||||
| import com.cloud.network.dao.IPAddressDao; | ||||
| import com.cloud.network.dao.IPAddressVO; | ||||
| @ -170,10 +167,13 @@ import com.cloud.storage.StoragePoolHostVO; | ||||
| import com.cloud.storage.StoragePoolStatus; | ||||
| import com.cloud.storage.StorageService; | ||||
| import com.cloud.storage.VMTemplateVO; | ||||
| import com.cloud.storage.Volume; | ||||
| import com.cloud.storage.VolumeVO; | ||||
| import com.cloud.storage.dao.DiskOfferingDao; | ||||
| import com.cloud.storage.dao.GuestOSCategoryDao; | ||||
| import com.cloud.storage.dao.StoragePoolHostDao; | ||||
| import com.cloud.storage.dao.VMTemplateDao; | ||||
| import com.cloud.storage.dao.VolumeDao; | ||||
| import com.cloud.user.Account; | ||||
| import com.cloud.user.AccountManager; | ||||
| import com.cloud.utils.Ternary; | ||||
| @ -1348,7 +1348,7 @@ public class ResourceManagerImpl extends ManagerBase implements ResourceManager, | ||||
|         if (VirtualMachine.Type.SecondaryStorageVm.equals(vm.getType()) | ||||
|                 || VirtualMachine.Type.ConsoleProxy.equals(vm.getType())) { | ||||
|             logger.error("Maintenance: VM is of type {}. Destroying VM {} immediately instead of migration.", vm.getType(), vm); | ||||
|             _haMgr.scheduleDestroy(vm, host.getId()); | ||||
|             _haMgr.scheduleDestroy(vm, host.getId(), HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|             return; | ||||
|         } | ||||
|         logger.error("Maintenance: No hosts available for migrations. Scheduling shutdown for VM {} instead of migration.", vm); | ||||
| @ -1405,10 +1405,10 @@ public class ResourceManagerImpl extends ManagerBase implements ResourceManager, | ||||
|                     handleVmForLastHostOrWithVGpu(host, vm); | ||||
|                 } else if (HypervisorType.LXC.equals(host.getHypervisorType()) && VirtualMachine.Type.User.equals(vm.getType())){ | ||||
|                     //Migration is not supported for LXC Vms. Schedule restart instead. | ||||
|                     _haMgr.scheduleRestart(vm, false); | ||||
|                     _haMgr.scheduleRestart(vm, false, HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|                 } else if (userVmManager.isVMUsingLocalStorage(vm)) { | ||||
|                     if (isMaintenanceLocalStrategyForceStop()) { | ||||
|                         _haMgr.scheduleStop(vm, hostId, WorkType.ForceStop); | ||||
|                         _haMgr.scheduleStop(vm, hostId, WorkType.ForceStop, HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|                     } else if (isMaintenanceLocalStrategyMigrate()) { | ||||
|                         migrateAwayVmWithVolumes(host, vm); | ||||
|                     } else if (!isMaintenanceLocalStrategyDefault()){ | ||||
| @ -1421,7 +1421,7 @@ public class ResourceManagerImpl extends ManagerBase implements ResourceManager, | ||||
|                     } | ||||
|                 } else { | ||||
|                     logger.info("Maintenance: scheduling migration of VM {} from host {}", vm, host); | ||||
|                     _haMgr.scheduleMigration(vm); | ||||
|                     _haMgr.scheduleMigration(vm, HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
| @ -1637,7 +1637,7 @@ public class ResourceManagerImpl extends ManagerBase implements ResourceManager, | ||||
|         for (VMInstanceVO vm : allVmsOnHost) { | ||||
|             State vmState = vm.getState(); | ||||
|             if (vmState == State.Starting || vmState == State.Running || vmState == State.Stopping) { | ||||
|                 _haMgr.scheduleRestart(vm, false); | ||||
|                 _haMgr.scheduleRestart(vm, false, HighAvailabilityManager.ReasonType.HostDegraded); | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
| @ -74,7 +74,7 @@ public abstract class HAAbstractHostProvider extends AdapterBase implements HAPr | ||||
|             try { | ||||
|                 logger.debug("Trying to disconnect the host without investigation and scheduling HA for the VMs on host {}", host); | ||||
|                 agentManager.disconnectWithoutInvestigation(host.getId(), Event.HostDown); | ||||
|                 oldHighAvailabilityManager.scheduleRestartForVmsOnHost((HostVO)host, true); | ||||
|                 oldHighAvailabilityManager.scheduleRestartForVmsOnHost((HostVO)host, true, HighAvailabilityManager.ReasonType.HostDown); | ||||
|             } catch (Exception e) { | ||||
|                 logger.error("Failed to disconnect host and schedule HA restart of VMs after fencing the host: ", e); | ||||
|             } | ||||
|  | ||||
| @ -135,6 +135,9 @@ public class HighAvailabilityManagerImplTest { | ||||
|     @Mock | ||||
|     UserVmManager userVmManager; | ||||
| 
 | ||||
|     @Mock | ||||
|     private HaWorkVO mockWork; | ||||
| 
 | ||||
|     HighAvailabilityManagerImpl highAvailabilityManager; | ||||
|     HighAvailabilityManagerImpl highAvailabilityManagerSpy; | ||||
|     static Method processWorkMethod = null; | ||||
| @ -185,7 +188,7 @@ public class HighAvailabilityManagerImplTest { | ||||
|         highAvailabilityManager.VmHaEnabled = haEnabled; | ||||
|         Mockito.when(highAvailabilityManager.VmHaEnabled.valueIn(1L)).thenReturn(true); | ||||
| 
 | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true); | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true, HighAvailabilityManager.ReasonType.HostDown); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
| @ -193,7 +196,7 @@ public class HighAvailabilityManagerImplTest { | ||||
|         Mockito.when(hostVO.getType()).thenReturn(Host.Type.Routing); | ||||
|         Mockito.when(hostVO.getHypervisorType()).thenReturn(HypervisorType.VMware); | ||||
| 
 | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true); | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true, HighAvailabilityManager.ReasonType.HostDown); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
| @ -206,7 +209,7 @@ public class HighAvailabilityManagerImplTest { | ||||
|         highAvailabilityManager.VmHaEnabled = haEnabled; | ||||
|         Mockito.when(highAvailabilityManager.VmHaEnabled.valueIn(1L)).thenReturn(false); | ||||
| 
 | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true); | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true, HighAvailabilityManager.ReasonType.HostDown); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
| @ -240,7 +243,7 @@ public class HighAvailabilityManagerImplTest { | ||||
|         highAvailabilityManager.VmHaEnabled = haEnabled; | ||||
|         Mockito.when(highAvailabilityManager.VmHaEnabled.valueIn(1L)).thenReturn(true); | ||||
| 
 | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true); | ||||
|         highAvailabilityManager.scheduleRestartForVmsOnHost(hostVO, true, HighAvailabilityManager.ReasonType.HostDown); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
| @ -336,7 +339,7 @@ public class HighAvailabilityManagerImplTest { | ||||
|         Mockito.when(vm.getState()).thenReturn(VirtualMachine.State.Running); | ||||
|         Mockito.when(_haDao.persist((HaWorkVO)Mockito.any())).thenReturn(Mockito.mock(HaWorkVO.class)); | ||||
| 
 | ||||
|         assertTrue(highAvailabilityManager.scheduleDestroy(vm, 1L)); | ||||
|         assertTrue(highAvailabilityManager.scheduleDestroy(vm, 1L, HighAvailabilityManager.ReasonType.HostMaintenance)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
| @ -348,7 +351,7 @@ public class HighAvailabilityManagerImplTest { | ||||
|         highAvailabilityManager.VmHaEnabled = haEnabled; | ||||
|         Mockito.when(highAvailabilityManager.VmHaEnabled.valueIn(1L)).thenReturn(false); | ||||
| 
 | ||||
|         assertFalse(highAvailabilityManager.scheduleDestroy(vm, 1L)); | ||||
|         assertFalse(highAvailabilityManager.scheduleDestroy(vm, 1L, HighAvailabilityManager.ReasonType.HostMaintenance)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
| @ -402,7 +405,7 @@ public class HighAvailabilityManagerImplTest { | ||||
| 
 | ||||
|     private void processWorkWithRetryCount(int count, Step expectedStep) { | ||||
|         assertNotNull(processWorkMethod); | ||||
|         HaWorkVO work = new HaWorkVO(1l, VirtualMachine.Type.User, WorkType.Migration, Step.Scheduled, 1l, VirtualMachine.State.Running, count, 12345678l); | ||||
|         HaWorkVO work = new HaWorkVO(1l, VirtualMachine.Type.User, WorkType.Migration, Step.Scheduled, 1l, VirtualMachine.State.Running, count, 12345678l, null); | ||||
|         Mockito.doReturn(12345678l).when(highAvailabilityManagerSpy).migrate(work); | ||||
|         try { | ||||
|             processWorkMethod.invoke(highAvailabilityManagerSpy, work); | ||||
| @ -425,4 +428,53 @@ public class HighAvailabilityManagerImplTest { | ||||
|     public void processWorkWithRetryCountNotExceeded() { | ||||
|         processWorkWithRetryCount(3, Step.Scheduled); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testCheckAndCancelWorkIfNeeded_Success() { | ||||
|         Mockito.when(mockWork.getStep()).thenReturn(Step.Investigating); | ||||
|         Mockito.when(mockWork.getReasonType()).thenReturn(HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|         Mockito.when(mockWork.getHostId()).thenReturn(1L); | ||||
|         Mockito.doReturn(Status.Up).when(highAvailabilityManagerSpy).investigate(1L); | ||||
|         Mockito.doNothing().when(mockWork).setStep(Step.Cancelled); | ||||
|         boolean result = highAvailabilityManagerSpy.checkAndCancelWorkIfNeeded(mockWork); | ||||
|         assertTrue(result); | ||||
|         Mockito.verify(mockWork).setStep(Step.Cancelled); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testCheckAndCancelWorkIfNeeded_StepNotInvestigating() { | ||||
|         Mockito.when(mockWork.getStep()).thenReturn(Step.Cancelled); | ||||
|         boolean result = highAvailabilityManagerSpy.checkAndCancelWorkIfNeeded(mockWork); | ||||
|         assertFalse(result); | ||||
|         Mockito.verify(mockWork, Mockito.never()).setStep(Mockito.any()); | ||||
|     } | ||||
| 
 | ||||
|     private void runInvalidReasonCheckAndCancelWorkIfNeeded(HighAvailabilityManager.ReasonType reasonType) { | ||||
|         Mockito.when(mockWork.getStep()).thenReturn(Step.Investigating); | ||||
|         Mockito.when(mockWork.getReasonType()).thenReturn(reasonType); | ||||
|         boolean result = highAvailabilityManagerSpy.checkAndCancelWorkIfNeeded(mockWork); | ||||
|         assertFalse(result); | ||||
|         Mockito.verify(mockWork, Mockito.never()).setStep(Mockito.any()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testCheckAndCancelWorkIfNeeded_InvalidReasonType() { | ||||
|         runInvalidReasonCheckAndCancelWorkIfNeeded(HighAvailabilityManager.ReasonType.Unknown); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testCheckAndCancelWorkIfNeeded_NullReasonType() { | ||||
|         runInvalidReasonCheckAndCancelWorkIfNeeded(null); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testCheckAndCancelWorkIfNeeded_HostStatusNotUp() { | ||||
|         Mockito.when(mockWork.getStep()).thenReturn(Step.Investigating); | ||||
|         Mockito.when(mockWork.getReasonType()).thenReturn(HighAvailabilityManager.ReasonType.HostDown); | ||||
|         Mockito.when(mockWork.getHostId()).thenReturn(1L); | ||||
|         Mockito.doReturn(Status.Down).when(highAvailabilityManagerSpy).investigate(1L); | ||||
|         boolean result = highAvailabilityManagerSpy.checkAndCancelWorkIfNeeded(mockWork); | ||||
|         assertFalse(result); | ||||
|         Mockito.verify(mockWork, Mockito.never()).setStep(Mockito.any()); | ||||
|     } | ||||
| } | ||||
|  | ||||
| @ -22,20 +22,25 @@ import java.util.List; | ||||
| import org.junit.Assert; | ||||
| import org.junit.Test; | ||||
| import org.junit.runner.RunWith; | ||||
| import org.mockito.MockedStatic; | ||||
| import org.mockito.Mockito; | ||||
| import org.mockito.Spy; | ||||
| import org.mockito.junit.MockitoJUnitRunner; | ||||
| import org.mockito.stubbing.Answer; | ||||
| 
 | ||||
| import com.cloud.ha.HaWorkVO; | ||||
| import com.cloud.ha.HighAvailabilityManager; | ||||
| import com.cloud.utils.db.GenericDaoBase; | ||||
| import com.cloud.utils.db.SearchBuilder; | ||||
| import com.cloud.utils.db.SearchCriteria; | ||||
| import com.cloud.utils.db.UpdateBuilder; | ||||
| import com.cloud.vm.VirtualMachine; | ||||
| 
 | ||||
| @RunWith(MockitoJUnitRunner.class) | ||||
| public class HighAvailabilityDaoImplTest { | ||||
| 
 | ||||
|     @Spy | ||||
|     HighAvailabilityDaoImpl highAvailabilityDaoImpl; | ||||
|     HighAvailabilityDaoImpl highAvailabilityDaoImpl = new HighAvailabilityDaoImpl(); | ||||
| 
 | ||||
|     @Test | ||||
|     public void testExpungeByVmListNoVms() { | ||||
| @ -65,4 +70,56 @@ public class HighAvailabilityDaoImplTest { | ||||
|         Mockito.verify(highAvailabilityDaoImpl, Mockito.times(1)) | ||||
|                 .batchExpunge(sc, batchSize); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testMarkPendingWorksAsInvestigating() throws Exception { | ||||
|         SearchBuilder<HaWorkVO> mockTBASearch = Mockito.mock(SearchBuilder.class); | ||||
|         highAvailabilityDaoImpl.TBASearch = mockTBASearch; | ||||
|         SearchCriteria<HaWorkVO> mockSearchCriteria = Mockito.mock(SearchCriteria.class); | ||||
|         UpdateBuilder mockUpdateBuilder = Mockito.mock(UpdateBuilder.class); | ||||
|         Mockito.when(mockTBASearch.create()).thenReturn(mockSearchCriteria); | ||||
|         Mockito.doNothing().when(mockSearchCriteria).setParameters(Mockito.eq("time"), Mockito.anyLong()); | ||||
|         Mockito.doNothing().when(mockSearchCriteria).setParameters(Mockito.eq("step"), Mockito.eq(HighAvailabilityManager.Step.Done), Mockito.eq(HighAvailabilityManager.Step.Cancelled)); | ||||
|         HaWorkVO haWorkVO = new HaWorkVO(1L, VirtualMachine.Type.User, null, | ||||
|                 null, 1L, null, 0, 0, | ||||
|                 HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|         Mockito.when(highAvailabilityDaoImpl.createForUpdate()).thenReturn(haWorkVO); | ||||
|         try(MockedStatic<GenericDaoBase> genericDaoBaseMockedStatic = Mockito.mockStatic(GenericDaoBase.class)) { | ||||
|             genericDaoBaseMockedStatic.when(() -> GenericDaoBase.getUpdateBuilder(Mockito.any())).thenReturn(mockUpdateBuilder); | ||||
|             Mockito.doReturn(5).when(highAvailabilityDaoImpl).update(Mockito.any(UpdateBuilder.class), Mockito.any(), Mockito.nullable(Integer.class)); | ||||
|             highAvailabilityDaoImpl.markPendingWorksAsInvestigating(); | ||||
|             Mockito.verify(mockTBASearch).create(); | ||||
|             Mockito.verify(mockSearchCriteria).setParameters(Mockito.eq("time"), Mockito.anyLong()); | ||||
|             Mockito.verify(mockSearchCriteria).setParameters(Mockito.eq("step"), Mockito.eq(HighAvailabilityManager.Step.Done), Mockito.eq(HighAvailabilityManager.Step.Cancelled)); | ||||
|             Assert.assertEquals(HighAvailabilityManager.Step.Investigating, haWorkVO.getStep()); // Ensure the step is set correctly | ||||
|             Mockito.verify(highAvailabilityDaoImpl).update(Mockito.eq(mockUpdateBuilder), Mockito.eq(mockSearchCriteria), Mockito.isNull()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testMarkServerPendingWorksAsInvestigating() { | ||||
|         SearchBuilder<HaWorkVO> mockSearch = Mockito.mock(SearchBuilder.class); | ||||
|         Mockito.doReturn(Mockito.mock(HaWorkVO.class)).when(mockSearch).entity(); | ||||
|         Mockito.doReturn(mockSearch).when(highAvailabilityDaoImpl).createSearchBuilder(); | ||||
|         SearchCriteria<HaWorkVO> mockSearchCriteria = Mockito.mock(SearchCriteria.class); | ||||
|         UpdateBuilder mockUpdateBuilder = Mockito.mock(UpdateBuilder.class); | ||||
|         Mockito.when(mockSearch.create()).thenReturn(mockSearchCriteria); | ||||
|         Mockito.doNothing().when(mockSearchCriteria).setParameters(Mockito.eq("server"), Mockito.eq(1L)); | ||||
|         Mockito.doNothing().when(mockSearchCriteria).setParameters(Mockito.eq("step"), Mockito.eq(HighAvailabilityManager.Step.Done), Mockito.eq(HighAvailabilityManager.Step.Cancelled)); | ||||
|         HaWorkVO haWorkVO = new HaWorkVO(1L, VirtualMachine.Type.User, null, | ||||
|                 null, 1L, null, 0, 0, | ||||
|                 HighAvailabilityManager.ReasonType.HostMaintenance); | ||||
|         Mockito.when(highAvailabilityDaoImpl.createForUpdate()).thenReturn(haWorkVO); | ||||
|         Mockito.when(highAvailabilityDaoImpl.createForUpdate()).thenReturn(haWorkVO); | ||||
|         try(MockedStatic<GenericDaoBase> genericDaoBaseMockedStatic = Mockito.mockStatic(GenericDaoBase.class)) { | ||||
|             genericDaoBaseMockedStatic.when(() -> GenericDaoBase.getUpdateBuilder(Mockito.any())).thenReturn(mockUpdateBuilder); | ||||
|             Mockito.doReturn(5).when(highAvailabilityDaoImpl).update(Mockito.any(UpdateBuilder.class), Mockito.any(), Mockito.nullable(Integer.class)); | ||||
|             highAvailabilityDaoImpl.markServerPendingWorksAsInvestigating(1L); | ||||
|             Mockito.verify(mockSearch).create(); | ||||
|             Mockito.verify(mockSearchCriteria).setParameters(Mockito.eq("server"), Mockito.eq(1L)); | ||||
|             Mockito.verify(mockSearchCriteria).setParameters(Mockito.eq("step"), Mockito.eq(HighAvailabilityManager.Step.Done), Mockito.eq(HighAvailabilityManager.Step.Cancelled)); | ||||
|             Assert.assertEquals(HighAvailabilityManager.Step.Investigating, haWorkVO.getStep()); // Ensure the step is set correctly | ||||
|             Mockito.verify(highAvailabilityDaoImpl).update(Mockito.eq(mockUpdateBuilder), Mockito.eq(mockSearchCriteria), Mockito.isNull()); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | ||||
		Loading…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user