You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cloudstack.apache.org by ke...@apache.org on 2014/03/01 00:37:35 UTC

[15/33] git commit: updated refs/heads/master to 90262a8

VirtualMachineManager.migrateAway() need to put under management of new sync model


Project: http://git-wip-us.apache.org/repos/asf/cloudstack/repo
Commit: http://git-wip-us.apache.org/repos/asf/cloudstack/commit/fa7d9ad1
Tree: http://git-wip-us.apache.org/repos/asf/cloudstack/tree/fa7d9ad1
Diff: http://git-wip-us.apache.org/repos/asf/cloudstack/diff/fa7d9ad1

Branch: refs/heads/master
Commit: fa7d9ad1a86c3b4274a081fa3a8c7813212b6903
Parents: 434f8da
Author: Kelven Yang <ke...@gmail.com>
Authored: Thu Feb 6 16:44:34 2014 -0800
Committer: Kelven Yang <ke...@gmail.com>
Committed: Fri Feb 28 15:35:58 2014 -0800

----------------------------------------------------------------------
 .../src/com/cloud/vm/VirtualMachineManager.java |   2 +-
 .../com/cloud/ha/HighAvailabilityManager.java   |   3 +
 .../com/cloud/vm/VirtualMachineManagerImpl.java | 118 ++++++++++++++++++-
 .../src/com/cloud/vm/VmWorkMigrateAway.java     |  36 ++++++
 .../cloud/ha/HighAvailabilityManagerImpl.java   |  17 ++-
 5 files changed, 164 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cloudstack/blob/fa7d9ad1/engine/api/src/com/cloud/vm/VirtualMachineManager.java
----------------------------------------------------------------------
diff --git a/engine/api/src/com/cloud/vm/VirtualMachineManager.java b/engine/api/src/com/cloud/vm/VirtualMachineManager.java
index ac79860..fcfa3f1 100644
--- a/engine/api/src/com/cloud/vm/VirtualMachineManager.java
+++ b/engine/api/src/com/cloud/vm/VirtualMachineManager.java
@@ -109,7 +109,7 @@ public interface VirtualMachineManager extends Manager {
 
     void destroy(String vmUuid) throws AgentUnavailableException, OperationTimedoutException, ConcurrentOperationException;
 
-    void migrateAway(String vmUuid, long hostId, DeploymentPlanner planner) throws InsufficientServerCapacityException;
+    void migrateAway(String vmUuid, long hostId) throws InsufficientServerCapacityException;
 
     void migrate(String vmUuid, long srcHostId, DeployDestination dest) throws ResourceUnavailableException, ConcurrentOperationException;
 

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/fa7d9ad1/engine/components-api/src/com/cloud/ha/HighAvailabilityManager.java
----------------------------------------------------------------------
diff --git a/engine/components-api/src/com/cloud/ha/HighAvailabilityManager.java b/engine/components-api/src/com/cloud/ha/HighAvailabilityManager.java
index e1b308a..eb60f3e 100644
--- a/engine/components-api/src/com/cloud/ha/HighAvailabilityManager.java
+++ b/engine/components-api/src/com/cloud/ha/HighAvailabilityManager.java
@@ -18,6 +18,7 @@ package com.cloud.ha;
 
 import java.util.List;
 
+import com.cloud.deploy.DeploymentPlanner;
 import com.cloud.host.HostVO;
 import com.cloud.host.Status;
 import com.cloud.utils.component.Manager;
@@ -103,4 +104,6 @@ public interface HighAvailabilityManager extends Manager {
      * @return
      */
     String getHaTag();
+
+    DeploymentPlanner getHAPlanner();
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/fa7d9ad1/engine/orchestration/src/com/cloud/vm/VirtualMachineManagerImpl.java
----------------------------------------------------------------------
diff --git a/engine/orchestration/src/com/cloud/vm/VirtualMachineManagerImpl.java b/engine/orchestration/src/com/cloud/vm/VirtualMachineManagerImpl.java
index 09a9980..fa0540e 100755
--- a/engine/orchestration/src/com/cloud/vm/VirtualMachineManagerImpl.java
+++ b/engine/orchestration/src/com/cloud/vm/VirtualMachineManagerImpl.java
@@ -1546,7 +1546,7 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
 
         // Disable this hacking thing, VM snapshot task need to be managed by its orchestartion flow istelf instead of
         // hacking it here at general VM manager
-        /*        
+        /*
                 if (_vmSnapshotMgr.hasActiveVMSnapshotTasks(vm.getId())) {
                     s_logger.error("State transit with event: " + e + " failed due to: " + vm.getInstanceName() + " has active VM snapshots tasks");
                     return false;
@@ -2183,7 +2183,53 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
     }
 
     @Override
-    public void migrateAway(String vmUuid, long srcHostId, DeploymentPlanner planner) throws InsufficientServerCapacityException {
+    public void migrateAway(String vmUuid, long srcHostId) throws InsufficientServerCapacityException {
+        AsyncJobExecutionContext jobContext = AsyncJobExecutionContext.getCurrentExecutionContext();
+        if (!VmJobEnabled.value() || jobContext.isJobDispatchedBy(VmWorkConstants.VM_WORK_JOB_DISPATCHER)) {
+            // avoid re-entrance
+
+            VmWorkJobVO placeHolder = null;
+            if (VmJobEnabled.value()) {
+                VirtualMachine vm = _vmDao.findByUuid(vmUuid);
+                placeHolder = createPlaceHolderWork(vm.getId());
+            }
+            try {
+                try {
+                    orchestrateMigrateAway(vmUuid, srcHostId, null);
+                } catch (InsufficientServerCapacityException e) {
+                    s_logger.warn("Failed to deploy vm " + vmUuid + " with original planner, sending HAPlanner");
+                    orchestrateMigrateAway(vmUuid, srcHostId, _haMgr.getHAPlanner());
+                }
+            } finally {
+                if (VmJobEnabled.value())
+                    _workJobDao.expunge(placeHolder.getId());
+            }
+        } else {
+            Outcome<VirtualMachine> outcome = migrateVmAwayThroughJobQueue(vmUuid, srcHostId);
+
+            try {
+                VirtualMachine vm = outcome.get();
+            } catch (InterruptedException e) {
+                throw new RuntimeException("Operation is interrupted", e);
+            } catch (java.util.concurrent.ExecutionException e) {
+                throw new RuntimeException("Execution excetion", e);
+            }
+
+            Object jobException = _jobMgr.unmarshallResultObject(outcome.getJob());
+            if (jobException != null) {
+                if (jobException instanceof InsufficientServerCapacityException)
+                    throw (InsufficientServerCapacityException)jobException;
+                else if (jobException instanceof ConcurrentOperationException)
+                    throw (ConcurrentOperationException)jobException;
+                else if (jobException instanceof RuntimeException)
+                    throw (RuntimeException)jobException;
+                else if (jobException instanceof Throwable)
+                    throw new RuntimeException("Unexpected exception", (Throwable)jobException);
+            }
+        }
+    }
+
+    private void orchestrateMigrateAway(String vmUuid, long srcHostId, DeploymentPlanner planner) throws InsufficientServerCapacityException {
         VMInstanceVO vm = _vmDao.findByUuid(vmUuid);
         if (vm == null) {
             s_logger.debug("Unable to find a VM for " + vmUuid);
@@ -4562,6 +4608,57 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
                 VirtualMachine.PowerState.PowerOn, vm.getId(), vm.getPowerHostId());
     }
 
+    public Outcome<VirtualMachine> migrateVmAwayThroughJobQueue(final String vmUuid, final long srcHostId) {
+        final CallContext context = CallContext.current();
+        final User user = context.getCallingUser();
+        final Account account = context.getCallingAccount();
+
+        final VMInstanceVO vm = _vmDao.findByUuid(vmUuid);
+
+        Object[] result = Transaction.execute(new TransactionCallback<Object[]>() {
+            @Override
+            public Object[] doInTransaction(TransactionStatus status) {
+
+                _vmDao.lockRow(vm.getId(), true);
+
+                List<VmWorkJobVO> pendingWorkJobs = _workJobDao.listPendingWorkJobs(
+                        VirtualMachine.Type.Instance, vm.getId(),
+                        VmWorkMigrateAway.class.getName());
+
+                VmWorkJobVO workJob = null;
+                if (pendingWorkJobs != null && pendingWorkJobs.size() > 0) {
+                    assert (pendingWorkJobs.size() == 1);
+                    workJob = pendingWorkJobs.get(0);
+                } else {
+
+                    workJob = new VmWorkJobVO(context.getContextId());
+
+                    workJob.setDispatcher(VmWorkConstants.VM_WORK_JOB_DISPATCHER);
+                    workJob.setCmd(VmWorkMigrateAway.class.getName());
+
+                    workJob.setAccountId(account.getId());
+                    workJob.setUserId(user.getId());
+                    workJob.setVmType(VirtualMachine.Type.Instance);
+                    workJob.setVmInstanceId(vm.getId());
+                    workJob.setRelated(AsyncJobExecutionContext.getOriginJobContextId());
+
+                    // save work context info (there are some duplications)
+                    VmWorkMigrateAway workInfo = new VmWorkMigrateAway(user.getId(), account.getId(), vm.getId(), VirtualMachineManagerImpl.VM_WORK_JOB_HANDLER, srcHostId);
+                    workJob.setCmdInfo(VmWorkSerializer.serialize(workInfo));
+
+                    _jobMgr.submitAsyncJob(workJob, VmWorkConstants.VM_WORK_QUEUE, vm.getId());
+                }
+                return new Object[] {workJob, new Long(workJob.getId())};
+            }
+        });
+
+        final long jobId = (Long)result[1];
+        AsyncJobExecutionContext.getCurrentExecutionContext().joinJob(jobId);
+
+        return new VmStateSyncOutcome((VmWorkJobVO)result[0],
+                VirtualMachine.PowerState.PowerOn, vm.getId(), vm.getPowerHostId());
+    }
+
     public Outcome<VirtualMachine> migrateVmWithStorageThroughJobQueue(
             final String vmUuid, final long srcHostId, final long destHostId,
             final Map<Volume, StoragePool> volumeToPool) {
@@ -4964,6 +5061,23 @@ public class VirtualMachineManagerImpl extends ManagerBase implements VirtualMac
         return new Pair<JobInfo.Status, String>(JobInfo.Status.SUCCEEDED, null);
     }
 
+    private Pair<JobInfo.Status, String> orchestrateMigrateAway(VmWorkMigrateAway work) throws Exception {
+        VMInstanceVO vm = _entityMgr.findById(VMInstanceVO.class, work.getVmId());
+        if (vm == null) {
+            s_logger.info("Unable to find vm " + work.getVmId());
+        }
+        assert (vm != null);
+
+        try {
+            orchestrateMigrateAway(vm.getUuid(), work.getSrcHostId(), null);
+        } catch (InsufficientServerCapacityException e) {
+            s_logger.warn("Failed to deploy vm " + vm.getId() + " with original planner, sending HAPlanner");
+            orchestrateMigrateAway(vm.getUuid(), work.getSrcHostId(), _haMgr.getHAPlanner());
+        }
+
+        return new Pair<JobInfo.Status, String>(JobInfo.Status.SUCCEEDED, null);
+    }
+
     private Pair<JobInfo.Status, String> orchestrateMigrateWithStorage(VmWorkMigrateWithStorage work) throws Exception {
         VMInstanceVO vm = _entityMgr.findById(VMInstanceVO.class, work.getVmId());
         if (vm == null) {

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/fa7d9ad1/engine/orchestration/src/com/cloud/vm/VmWorkMigrateAway.java
----------------------------------------------------------------------
diff --git a/engine/orchestration/src/com/cloud/vm/VmWorkMigrateAway.java b/engine/orchestration/src/com/cloud/vm/VmWorkMigrateAway.java
new file mode 100644
index 0000000..92189ed
--- /dev/null
+++ b/engine/orchestration/src/com/cloud/vm/VmWorkMigrateAway.java
@@ -0,0 +1,36 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package com.cloud.vm;
+
+public class VmWorkMigrateAway extends VmWork {
+
+    private static final long serialVersionUID = -5917512239025814373L;
+
+    private long srcHostId;
+
+    public VmWorkMigrateAway(long userId, long accountId, long vmId, String handlerName,
+            long srcHostId) {
+        super(userId, accountId, vmId, handlerName);
+
+        this.srcHostId = srcHostId;
+    }
+
+    public long getSrcHostId() {
+        return srcHostId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/fa7d9ad1/server/src/com/cloud/ha/HighAvailabilityManagerImpl.java
----------------------------------------------------------------------
diff --git a/server/src/com/cloud/ha/HighAvailabilityManagerImpl.java b/server/src/com/cloud/ha/HighAvailabilityManagerImpl.java
index 3963f29..24a3498 100755
--- a/server/src/com/cloud/ha/HighAvailabilityManagerImpl.java
+++ b/server/src/com/cloud/ha/HighAvailabilityManagerImpl.java
@@ -32,7 +32,6 @@ import javax.naming.ConfigurationException;
 import org.apache.log4j.Logger;
 import org.apache.log4j.NDC;
 
-import com.cloud.deploy.HAPlanner;
 import org.apache.cloudstack.engine.orchestration.service.VolumeOrchestrationService;
 import org.apache.cloudstack.framework.config.dao.ConfigurationDao;
 import org.apache.cloudstack.managed.context.ManagedContext;
@@ -48,6 +47,8 @@ import com.cloud.dc.DataCenterVO;
 import com.cloud.dc.HostPodVO;
 import com.cloud.dc.dao.DataCenterDao;
 import com.cloud.dc.dao.HostPodDao;
+import com.cloud.deploy.DeploymentPlanner;
+import com.cloud.deploy.HAPlanner;
 import com.cloud.exception.AgentUnavailableException;
 import com.cloud.exception.ConcurrentOperationException;
 import com.cloud.exception.InsufficientCapacityException;
@@ -148,7 +149,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements HighAvai
     }
 
     public void setHaPlanners(List<HAPlanner> haPlanners) {
-        this._haPlanners = haPlanners;
+        _haPlanners = haPlanners;
     }
 
 
@@ -611,13 +612,7 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements HighAvai
 
             VMInstanceVO vm = _instanceDao.findById(vmId);
             // First try starting the vm with its original planner, if it doesn't succeed send HAPlanner as its an emergency.
-            boolean result = false;
-            try {
-                _itMgr.migrateAway(vm.getUuid(), srcHostId, null);
-            }catch (InsufficientServerCapacityException e) {
-                s_logger.warn("Failed to deploy vm " + vmId + " with original planner, sending HAPlanner");
-                _itMgr.migrateAway(vm.getUuid(), srcHostId, _haPlanners.get(0));
-            }
+            _itMgr.migrateAway(vm.getUuid(), srcHostId);
             return null;
         } catch (InsufficientServerCapacityException e) {
             s_logger.warn("Insufficient capacity for migrating a VM.");
@@ -935,4 +930,8 @@ public class HighAvailabilityManagerImpl extends ManagerBase implements HighAvai
         return _haTag;
     }
 
+    @Override
+    public DeploymentPlanner getHAPlanner() {
+        return _haPlanners.get(0);
+    }
 }