You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cloudstack.apache.org by bh...@apache.org on 2015/05/19 12:19:21 UTC

[2/2] git commit: updated refs/heads/master to a99c9d0

Implementation for the ability to disable a storage pool for provisioning

... of new volumes. Following changes are implemented 1. Disable or enable a pool with the
updateStoragePool api. A new 'enabled' parameter added for the same. 2. When a
pool is disabled the state of the pool is updated to 'Disabled' in the db. On
enabling it is updated back to 'Up'. Alert is raised when a pool is disabled or
enabled. 3. Updated other storage providers to also honour the disabled state.
4. A disabled pool is skipped by allocators for provisioing of new volumes. 5.
Since the allocators skip a disabled pool for provisioning of volumes, the
volumes are also not listed as a destination for volume migration.

FS: https://cwiki.apache.org/confluence/display/CLOUDSTACK/Disabling+Storage+Pool+for+Provisioning

This closes #257

Signed-off-by: Rohit Yadav <ro...@shapeblue.com>


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

Branch: refs/heads/master
Commit: a99c9d0e68b42725a9f174d94f8d91c8a05398f3
Parents: bab4e3a
Author: Devdeep Singh <de...@gmail.com>
Authored: Tue May 5 18:24:04 2015 +0530
Committer: Rohit Yadav <ro...@shapeblue.com>
Committed: Tue May 19 11:16:49 2015 +0100

----------------------------------------------------------------------
 api/src/com/cloud/event/EventTypes.java         |  9 +++++
 .../com/cloud/storage/StoragePoolStatus.java    |  2 +-
 .../admin/storage/UpdateStoragePoolCmd.java     |  8 +++++
 .../api/storage/PrimaryDataStoreLifeCycle.java  |  2 ++
 .../datastore/db/PrimaryDataStoreDao.java       |  2 ++
 .../datastore/db/PrimaryDataStoreDaoImpl.java   | 21 +++++++++++
 .../ClusterScopeStoragePoolAllocator.java       | 16 +++++++--
 .../allocator/LocalStoragePoolAllocator.java    | 11 ++++++
 .../allocator/ZoneWideStoragePoolAllocator.java | 11 ++++++
 .../datastore/PrimaryDataStoreHelper.java       | 14 ++++++++
 .../ElastistorPrimaryDataStoreLifeCycle.java    | 10 ++++++
 ...CloudStackPrimaryDataStoreLifeCycleImpl.java | 10 ++++++
 .../NexentaPrimaryDataStoreLifeCycle.java       | 10 ++++++
 .../SamplePrimaryDataStoreLifeCycleImpl.java    |  8 +++++
 .../SolidFirePrimaryDataStoreLifeCycle.java     | 10 ++++++
 ...olidFireSharedPrimaryDataStoreLifeCycle.java | 10 ++++++
 .../com/cloud/storage/StorageManagerImpl.java   | 37 +++++++++++++++++++-
 17 files changed, 187 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/api/src/com/cloud/event/EventTypes.java
----------------------------------------------------------------------
diff --git a/api/src/com/cloud/event/EventTypes.java b/api/src/com/cloud/event/EventTypes.java
index 20f287a..78236fa 100644
--- a/api/src/com/cloud/event/EventTypes.java
+++ b/api/src/com/cloud/event/EventTypes.java
@@ -54,6 +54,7 @@ import com.cloud.offering.NetworkOffering;
 import com.cloud.offering.ServiceOffering;
 import com.cloud.projects.Project;
 import com.cloud.server.ResourceTag;
+import com.cloud.storage.StoragePool;
 import com.cloud.storage.GuestOS;
 import com.cloud.storage.GuestOSHypervisor;
 import com.cloud.storage.Snapshot;
@@ -306,6 +307,10 @@ public class EventTypes {
     public static final String EVENT_MAINTENANCE_PREPARE = "MAINT.PREPARE";
     public static final String EVENT_MAINTENANCE_PREPARE_PRIMARY_STORAGE = "MAINT.PREPARE.PS";
 
+    // Primary storage pool
+    public static final String EVENT_ENABLE_PRIMARY_STORAGE = "ENABLE.PS";
+    public static final String EVENT_DISABLE_PRIMARY_STORAGE = "DISABLE.PS";
+
     // VPN
     public static final String EVENT_REMOTE_ACCESS_VPN_CREATE = "VPN.REMOTE.ACCESS.CREATE";
     public static final String EVENT_REMOTE_ACCESS_VPN_DESTROY = "VPN.REMOTE.ACCESS.DESTROY";
@@ -728,6 +733,10 @@ public class EventTypes {
         entityEventDetails.put(EVENT_MAINTENANCE_PREPARE, Host.class);
         entityEventDetails.put(EVENT_MAINTENANCE_PREPARE_PRIMARY_STORAGE, Host.class);
 
+        // Primary storage pool
+        entityEventDetails.put(EVENT_ENABLE_PRIMARY_STORAGE, StoragePool.class);
+        entityEventDetails.put(EVENT_DISABLE_PRIMARY_STORAGE, StoragePool.class);
+
         // VPN
         entityEventDetails.put(EVENT_REMOTE_ACCESS_VPN_CREATE, RemoteAccessVpn.class);
         entityEventDetails.put(EVENT_REMOTE_ACCESS_VPN_DESTROY, RemoteAccessVpn.class);

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/api/src/com/cloud/storage/StoragePoolStatus.java
----------------------------------------------------------------------
diff --git a/api/src/com/cloud/storage/StoragePoolStatus.java b/api/src/com/cloud/storage/StoragePoolStatus.java
index c7ff0ff..778d388 100644
--- a/api/src/com/cloud/storage/StoragePoolStatus.java
+++ b/api/src/com/cloud/storage/StoragePoolStatus.java
@@ -17,5 +17,5 @@
 package com.cloud.storage;
 
 public enum StoragePoolStatus {
-    Initial, Initialized, Creating, Attaching, Up, PrepareForMaintenance, ErrorInMaintenance, CancelMaintenance, Maintenance, Removed;
+    Initial, Initialized, Creating, Attaching, Up, PrepareForMaintenance, ErrorInMaintenance, CancelMaintenance, Maintenance, Disabled, Removed;
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/api/src/org/apache/cloudstack/api/command/admin/storage/UpdateStoragePoolCmd.java
----------------------------------------------------------------------
diff --git a/api/src/org/apache/cloudstack/api/command/admin/storage/UpdateStoragePoolCmd.java b/api/src/org/apache/cloudstack/api/command/admin/storage/UpdateStoragePoolCmd.java
index 3d1a773..6bf6228 100644
--- a/api/src/org/apache/cloudstack/api/command/admin/storage/UpdateStoragePoolCmd.java
+++ b/api/src/org/apache/cloudstack/api/command/admin/storage/UpdateStoragePoolCmd.java
@@ -54,6 +54,10 @@ public class UpdateStoragePoolCmd extends BaseCmd {
     @Parameter(name = ApiConstants.CAPACITY_BYTES, type = CommandType.LONG, required = false, description = "bytes CloudStack can provision from this storage pool")
     private Long capacityBytes;
 
+    @Parameter(name = ApiConstants.ENABLED, type = CommandType.BOOLEAN, required = false, description = "false to disable the pool for allocation of new volumes, true to" +
+            " enable it back.")
+    private Boolean enabled;
+
     /////////////////////////////////////////////////////
     /////////////////// Accessors ///////////////////////
     /////////////////////////////////////////////////////
@@ -74,6 +78,10 @@ public class UpdateStoragePoolCmd extends BaseCmd {
         return capacityBytes;
     }
 
+    public Boolean getEnabled() {
+        return enabled;
+    }
+
     /////////////////////////////////////////////////////
     /////////////// API Implementation///////////////////
     /////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/api/src/org/apache/cloudstack/engine/subsystem/api/storage/PrimaryDataStoreLifeCycle.java
----------------------------------------------------------------------
diff --git a/engine/api/src/org/apache/cloudstack/engine/subsystem/api/storage/PrimaryDataStoreLifeCycle.java b/engine/api/src/org/apache/cloudstack/engine/subsystem/api/storage/PrimaryDataStoreLifeCycle.java
index 7640cf3..fcbc19c 100644
--- a/engine/api/src/org/apache/cloudstack/engine/subsystem/api/storage/PrimaryDataStoreLifeCycle.java
+++ b/engine/api/src/org/apache/cloudstack/engine/subsystem/api/storage/PrimaryDataStoreLifeCycle.java
@@ -27,4 +27,6 @@ public interface PrimaryDataStoreLifeCycle extends DataStoreLifeCycle {
     public static final String CAPACITY_IOPS = "capacityIops";
 
     void updateStoragePool(StoragePool storagePool, Map<String, String> details);
+    void enableStoragePool(DataStore store);
+    void disableStoragePool(DataStore store);
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDao.java
----------------------------------------------------------------------
diff --git a/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDao.java b/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDao.java
index a976bfb..3939545 100644
--- a/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDao.java
+++ b/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDao.java
@@ -77,6 +77,8 @@ public interface PrimaryDataStoreDao extends GenericDao<StoragePoolVO, Long> {
 
     List<StoragePoolVO> findPoolsByTags(long dcId, long podId, Long clusterId, String[] tags);
 
+    List<StoragePoolVO> findDisabledPoolsByScope(long dcId, Long podId, Long clusterId, ScopeType scope);
+
     /**
      * Find pool by UUID.
      *

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDaoImpl.java
----------------------------------------------------------------------
diff --git a/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDaoImpl.java b/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDaoImpl.java
index ae2287e..faf5291 100644
--- a/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDaoImpl.java
+++ b/engine/schema/src/org/apache/cloudstack/storage/datastore/db/PrimaryDataStoreDaoImpl.java
@@ -76,6 +76,7 @@ public class PrimaryDataStoreDaoImpl extends GenericDaoBase<StoragePoolVO, Long>
         AllFieldSearch.and("datacenterId", AllFieldSearch.entity().getDataCenterId(), SearchCriteria.Op.EQ);
         AllFieldSearch.and("hostAddress", AllFieldSearch.entity().getHostAddress(), SearchCriteria.Op.EQ);
         AllFieldSearch.and("status", AllFieldSearch.entity().getStatus(), SearchCriteria.Op.EQ);
+        AllFieldSearch.and("scope", AllFieldSearch.entity().getScope(), SearchCriteria.Op.EQ);
         AllFieldSearch.and("path", AllFieldSearch.entity().getPath(), SearchCriteria.Op.EQ);
         AllFieldSearch.and("podId", AllFieldSearch.entity().getPodId(), Op.EQ);
         AllFieldSearch.and("clusterId", AllFieldSearch.entity().getClusterId(), Op.EQ);
@@ -310,6 +311,26 @@ public class PrimaryDataStoreDaoImpl extends GenericDaoBase<StoragePoolVO, Long>
     }
 
     @Override
+    public List<StoragePoolVO> findDisabledPoolsByScope(long dcId, Long podId, Long clusterId, ScopeType scope) {
+        List<StoragePoolVO> storagePools = null;
+        SearchCriteria<StoragePoolVO> sc = AllFieldSearch.create();
+        sc.setParameters("status", StoragePoolStatus.Disabled);
+        sc.setParameters("scope", scope);
+
+        if (scope == ScopeType.ZONE) {
+            sc.setParameters("datacenterId", dcId);
+            storagePools = listBy(sc);
+        } else if ((scope == ScopeType.CLUSTER || scope == ScopeType.HOST) && podId != null && clusterId != null) {
+            sc.setParameters("datacenterId", dcId);
+            sc.setParameters("podId", podId);
+            sc.setParameters("clusterId", clusterId);
+            storagePools = listBy(sc);
+        }
+
+        return storagePools;
+    }
+
+    @Override
     public List<StoragePoolVO> findLocalStoragePoolsByTags(long dcId, long podId, Long clusterId, String[] tags) {
         List<StoragePoolVO> storagePools = null;
         if (tags == null || tags.length == 0) {

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/storage/src/org/apache/cloudstack/storage/allocator/ClusterScopeStoragePoolAllocator.java
----------------------------------------------------------------------
diff --git a/engine/storage/src/org/apache/cloudstack/storage/allocator/ClusterScopeStoragePoolAllocator.java b/engine/storage/src/org/apache/cloudstack/storage/allocator/ClusterScopeStoragePoolAllocator.java
index 0f4df4f..d78bd09 100644
--- a/engine/storage/src/org/apache/cloudstack/storage/allocator/ClusterScopeStoragePoolAllocator.java
+++ b/engine/storage/src/org/apache/cloudstack/storage/allocator/ClusterScopeStoragePoolAllocator.java
@@ -33,6 +33,7 @@ import org.springframework.stereotype.Component;
 import com.cloud.deploy.DeploymentPlan;
 import com.cloud.deploy.DeploymentPlanner.ExcludeList;
 import com.cloud.offering.ServiceOffering;
+import com.cloud.storage.ScopeType;
 import com.cloud.storage.StoragePool;
 import com.cloud.storage.dao.DiskOfferingDao;
 import com.cloud.vm.DiskProfile;
@@ -70,9 +71,20 @@ public class ClusterScopeStoragePoolAllocator extends AbstractStoragePoolAllocat
             return null;
         }
         if (dskCh.getTags() != null && dskCh.getTags().length != 0) {
-            s_logger.debug("Looking for pools in dc: " + dcId + "  pod:" + podId + "  cluster:" + clusterId + " having tags:" + Arrays.toString(dskCh.getTags()));
+            s_logger.debug("Looking for pools in dc: " + dcId + "  pod:" + podId + "  cluster:" + clusterId + " having tags:" + Arrays.toString(dskCh.getTags()) +
+                    ". Disabled pools will be ignored.");
         } else {
-            s_logger.debug("Looking for pools in dc: " + dcId + "  pod:" + podId + "  cluster:" + clusterId);
+            s_logger.debug("Looking for pools in dc: " + dcId + "  pod:" + podId + "  cluster:" + clusterId + ". Disabled pools will be ignored.");
+        }
+
+        if (s_logger.isTraceEnabled()) {
+            // Log the pools details that are ignored because they are in disabled state
+            List<StoragePoolVO> disabledPools = _storagePoolDao.findDisabledPoolsByScope(dcId, podId, clusterId, ScopeType.CLUSTER);
+            if (disabledPools != null && !disabledPools.isEmpty()) {
+                for (StoragePoolVO pool : disabledPools) {
+                    s_logger.trace("Ignoring pool " + pool + " as it is in disabled state.");
+                }
+            }
         }
 
         List<StoragePoolVO> pools = _storagePoolDao.findPoolsByTags(dcId, podId, clusterId, dskCh.getTags());

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/storage/src/org/apache/cloudstack/storage/allocator/LocalStoragePoolAllocator.java
----------------------------------------------------------------------
diff --git a/engine/storage/src/org/apache/cloudstack/storage/allocator/LocalStoragePoolAllocator.java b/engine/storage/src/org/apache/cloudstack/storage/allocator/LocalStoragePoolAllocator.java
index 446e101..a4edf76 100644
--- a/engine/storage/src/org/apache/cloudstack/storage/allocator/LocalStoragePoolAllocator.java
+++ b/engine/storage/src/org/apache/cloudstack/storage/allocator/LocalStoragePoolAllocator.java
@@ -35,6 +35,7 @@ import com.cloud.capacity.dao.CapacityDao;
 import com.cloud.deploy.DeploymentPlan;
 import com.cloud.deploy.DeploymentPlanner.ExcludeList;
 import com.cloud.service.dao.ServiceOfferingDao;
+import com.cloud.storage.ScopeType;
 import com.cloud.storage.StoragePool;
 import com.cloud.storage.dao.StoragePoolHostDao;
 import com.cloud.utils.NumbersUtil;
@@ -69,6 +70,16 @@ public class LocalStoragePoolAllocator extends AbstractStoragePoolAllocator {
             return null;
         }
 
+        if (s_logger.isTraceEnabled()) {
+            // Log the pools details that are ignored because they are in disabled state
+            List<StoragePoolVO> disabledPools = _storagePoolDao.findDisabledPoolsByScope(plan.getDataCenterId(), plan.getPodId(), plan.getClusterId(), ScopeType.HOST);
+            if (disabledPools != null && !disabledPools.isEmpty()) {
+                for (StoragePoolVO pool : disabledPools) {
+                    s_logger.trace("Ignoring pool " + pool + " as it is in disabled state.");
+                }
+            }
+        }
+
         List<StoragePool> suitablePools = new ArrayList<StoragePool>();
 
         // data disk and host identified from deploying vm (attach volume case)

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/storage/src/org/apache/cloudstack/storage/allocator/ZoneWideStoragePoolAllocator.java
----------------------------------------------------------------------
diff --git a/engine/storage/src/org/apache/cloudstack/storage/allocator/ZoneWideStoragePoolAllocator.java b/engine/storage/src/org/apache/cloudstack/storage/allocator/ZoneWideStoragePoolAllocator.java
index b38b76f..7a10966 100644
--- a/engine/storage/src/org/apache/cloudstack/storage/allocator/ZoneWideStoragePoolAllocator.java
+++ b/engine/storage/src/org/apache/cloudstack/storage/allocator/ZoneWideStoragePoolAllocator.java
@@ -33,6 +33,7 @@ import org.apache.cloudstack.storage.datastore.db.StoragePoolVO;
 import com.cloud.deploy.DeploymentPlan;
 import com.cloud.deploy.DeploymentPlanner.ExcludeList;
 import com.cloud.hypervisor.Hypervisor.HypervisorType;
+import com.cloud.storage.ScopeType;
 import com.cloud.storage.StoragePool;
 import com.cloud.user.Account;
 import com.cloud.vm.DiskProfile;
@@ -55,6 +56,16 @@ public class ZoneWideStoragePoolAllocator extends AbstractStoragePoolAllocator {
             return null;
         }
 
+        if (s_logger.isTraceEnabled()) {
+            // Log the pools details that are ignored because they are in disabled state
+            List<StoragePoolVO> disabledPools = _storagePoolDao.findDisabledPoolsByScope(plan.getDataCenterId(), null, null, ScopeType.ZONE);
+            if (disabledPools != null && !disabledPools.isEmpty()) {
+                for (StoragePoolVO pool : disabledPools) {
+                    s_logger.trace("Ignoring pool " + pool + " as it is in disabled state.");
+                }
+            }
+        }
+
         List<StoragePool> suitablePools = new ArrayList<StoragePool>();
 
         List<StoragePoolVO> storagePools = _storagePoolDao.findZoneWideStoragePoolsByTags(plan.getDataCenterId(), dskCh.getTags());

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/engine/storage/src/org/apache/cloudstack/storage/volume/datastore/PrimaryDataStoreHelper.java
----------------------------------------------------------------------
diff --git a/engine/storage/src/org/apache/cloudstack/storage/volume/datastore/PrimaryDataStoreHelper.java b/engine/storage/src/org/apache/cloudstack/storage/volume/datastore/PrimaryDataStoreHelper.java
index ef0db85..8752c19 100644
--- a/engine/storage/src/org/apache/cloudstack/storage/volume/datastore/PrimaryDataStoreHelper.java
+++ b/engine/storage/src/org/apache/cloudstack/storage/volume/datastore/PrimaryDataStoreHelper.java
@@ -192,6 +192,20 @@ public class PrimaryDataStoreHelper {
         return true;
     }
 
+    public boolean disable(DataStore store) {
+        StoragePoolVO pool = this.dataStoreDao.findById(store.getId());
+        pool.setStatus(StoragePoolStatus.Disabled);
+        this.dataStoreDao.update(pool.getId(), pool);
+        return true;
+    }
+
+    public boolean enable(DataStore store) {
+        StoragePoolVO pool = this.dataStoreDao.findById(store.getId());
+        pool.setStatus(StoragePoolStatus.Up);
+        dataStoreDao.update(pool.getId(), pool);
+        return true;
+    }
+
     protected boolean deletePoolStats(Long poolId) {
         CapacityVO capacity1 = _capacityDao.findByHostIdType(poolId, Capacity.CAPACITY_TYPE_STORAGE);
         CapacityVO capacity2 = _capacityDao.findByHostIdType(poolId, Capacity.CAPACITY_TYPE_STORAGE_ALLOCATED);

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/plugins/storage/volume/cloudbyte/src/org/apache/cloudstack/storage/datastore/lifecycle/ElastistorPrimaryDataStoreLifeCycle.java
----------------------------------------------------------------------
diff --git a/plugins/storage/volume/cloudbyte/src/org/apache/cloudstack/storage/datastore/lifecycle/ElastistorPrimaryDataStoreLifeCycle.java b/plugins/storage/volume/cloudbyte/src/org/apache/cloudstack/storage/datastore/lifecycle/ElastistorPrimaryDataStoreLifeCycle.java
index 1778d43..5a8ccf1 100644
--- a/plugins/storage/volume/cloudbyte/src/org/apache/cloudstack/storage/datastore/lifecycle/ElastistorPrimaryDataStoreLifeCycle.java
+++ b/plugins/storage/volume/cloudbyte/src/org/apache/cloudstack/storage/datastore/lifecycle/ElastistorPrimaryDataStoreLifeCycle.java
@@ -464,6 +464,16 @@ public class ElastistorPrimaryDataStoreLifeCycle implements PrimaryDataStoreLife
         return true;
     }
 
+    @Override
+    public void enableStoragePool(DataStore dataStore) {
+        _dataStoreHelper.enable(dataStore);
+    }
+
+    @Override
+    public void disableStoragePool(DataStore dataStore) {
+        _dataStoreHelper.disable(dataStore);
+    }
+
     @SuppressWarnings("finally")
     @Override
     public boolean deleteDataStore(DataStore store) {

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/plugins/storage/volume/default/src/org/apache/cloudstack/storage/datastore/lifecycle/CloudStackPrimaryDataStoreLifeCycleImpl.java
----------------------------------------------------------------------
diff --git a/plugins/storage/volume/default/src/org/apache/cloudstack/storage/datastore/lifecycle/CloudStackPrimaryDataStoreLifeCycleImpl.java b/plugins/storage/volume/default/src/org/apache/cloudstack/storage/datastore/lifecycle/CloudStackPrimaryDataStoreLifeCycleImpl.java
index d11342c..38e9d9c 100644
--- a/plugins/storage/volume/default/src/org/apache/cloudstack/storage/datastore/lifecycle/CloudStackPrimaryDataStoreLifeCycleImpl.java
+++ b/plugins/storage/volume/default/src/org/apache/cloudstack/storage/datastore/lifecycle/CloudStackPrimaryDataStoreLifeCycleImpl.java
@@ -522,4 +522,14 @@ public class CloudStackPrimaryDataStoreLifeCycleImpl implements PrimaryDataStore
     @Override
     public void updateStoragePool(StoragePool storagePool, Map<String, String> details) {
     }
+
+    @Override
+    public void enableStoragePool(DataStore dataStore) {
+        dataStoreHelper.enable(dataStore);
+    }
+
+    @Override
+    public void disableStoragePool(DataStore dataStore) {
+        dataStoreHelper.disable(dataStore);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/plugins/storage/volume/nexenta/src/org/apache/cloudstack/storage/datastore/lifecylce/NexentaPrimaryDataStoreLifeCycle.java
----------------------------------------------------------------------
diff --git a/plugins/storage/volume/nexenta/src/org/apache/cloudstack/storage/datastore/lifecylce/NexentaPrimaryDataStoreLifeCycle.java b/plugins/storage/volume/nexenta/src/org/apache/cloudstack/storage/datastore/lifecylce/NexentaPrimaryDataStoreLifeCycle.java
index 493f274..4fffb34 100644
--- a/plugins/storage/volume/nexenta/src/org/apache/cloudstack/storage/datastore/lifecylce/NexentaPrimaryDataStoreLifeCycle.java
+++ b/plugins/storage/volume/nexenta/src/org/apache/cloudstack/storage/datastore/lifecylce/NexentaPrimaryDataStoreLifeCycle.java
@@ -166,6 +166,16 @@ public class NexentaPrimaryDataStoreLifeCycle
     }
 
     @Override
+    public void enableStoragePool(DataStore dataStore) {
+        dataStoreHelper.enable(dataStore);
+    }
+
+    @Override
+    public void disableStoragePool(DataStore dataStore) {
+        dataStoreHelper.disable(dataStore);
+    }
+
+    @Override
     public boolean deleteDataStore(DataStore store) {
         return dataStoreHelper.deletePrimaryDataStore(store);
     }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/plugins/storage/volume/sample/src/org/apache/cloudstack/storage/datastore/lifecycle/SamplePrimaryDataStoreLifeCycleImpl.java
----------------------------------------------------------------------
diff --git a/plugins/storage/volume/sample/src/org/apache/cloudstack/storage/datastore/lifecycle/SamplePrimaryDataStoreLifeCycleImpl.java b/plugins/storage/volume/sample/src/org/apache/cloudstack/storage/datastore/lifecycle/SamplePrimaryDataStoreLifeCycleImpl.java
index ee38072..3a0ce83 100644
--- a/plugins/storage/volume/sample/src/org/apache/cloudstack/storage/datastore/lifecycle/SamplePrimaryDataStoreLifeCycleImpl.java
+++ b/plugins/storage/volume/sample/src/org/apache/cloudstack/storage/datastore/lifecycle/SamplePrimaryDataStoreLifeCycleImpl.java
@@ -138,4 +138,12 @@ public class SamplePrimaryDataStoreLifeCycleImpl implements PrimaryDataStoreLife
     @Override
     public void updateStoragePool(StoragePool storagePool, Map<String, String> details) {
     }
+
+    @Override
+    public void enableStoragePool(DataStore store) {
+    }
+
+    @Override
+    public void disableStoragePool(DataStore store) {
+    }
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFirePrimaryDataStoreLifeCycle.java
----------------------------------------------------------------------
diff --git a/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFirePrimaryDataStoreLifeCycle.java b/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFirePrimaryDataStoreLifeCycle.java
index 7c36416..4b38f22 100644
--- a/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFirePrimaryDataStoreLifeCycle.java
+++ b/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFirePrimaryDataStoreLifeCycle.java
@@ -287,4 +287,14 @@ public class SolidFirePrimaryDataStoreLifeCycle implements PrimaryDataStoreLifeC
             }
         }
     }
+
+    @Override
+    public void enableStoragePool(DataStore dataStore) {
+        dataStoreHelper.enable(dataStore);
+    }
+
+    @Override
+    public void disableStoragePool(DataStore dataStore) {
+        dataStoreHelper.disable(dataStore);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFireSharedPrimaryDataStoreLifeCycle.java
----------------------------------------------------------------------
diff --git a/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFireSharedPrimaryDataStoreLifeCycle.java b/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFireSharedPrimaryDataStoreLifeCycle.java
index 4fa49ef..bc22ac7 100644
--- a/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFireSharedPrimaryDataStoreLifeCycle.java
+++ b/plugins/storage/volume/solidfire/src/org/apache/cloudstack/storage/datastore/lifecycle/SolidFireSharedPrimaryDataStoreLifeCycle.java
@@ -659,4 +659,14 @@ public class SolidFireSharedPrimaryDataStoreLifeCycle implements PrimaryDataStor
 
         SolidFireUtil.updateCsDbWithSolidFireIopsInfo(storagePool.getId(), _primaryDataStoreDao, _storagePoolDetailsDao, minIops, maxIops, burstIops);
     }
+
+    @Override
+    public void enableStoragePool(DataStore dataStore) {
+        _primaryDataStoreHelper.enable(dataStore);
+    }
+
+    @Override
+    public void disableStoragePool(DataStore dataStore) {
+        _primaryDataStoreHelper.disable(dataStore);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cloudstack/blob/a99c9d0e/server/src/com/cloud/storage/StorageManagerImpl.java
----------------------------------------------------------------------
diff --git a/server/src/com/cloud/storage/StorageManagerImpl.java b/server/src/com/cloud/storage/StorageManagerImpl.java
index f31095b..e725ce3 100644
--- a/server/src/com/cloud/storage/StorageManagerImpl.java
+++ b/server/src/com/cloud/storage/StorageManagerImpl.java
@@ -116,6 +116,8 @@ import com.cloud.dc.ClusterVO;
 import com.cloud.dc.DataCenterVO;
 import com.cloud.dc.dao.ClusterDao;
 import com.cloud.dc.dao.DataCenterDao;
+import com.cloud.event.ActionEvent;
+import com.cloud.event.EventTypes;
 import com.cloud.exception.AgentUnavailableException;
 import com.cloud.exception.ConnectionException;
 import com.cloud.exception.DiscoveryException;
@@ -732,6 +734,32 @@ public class StorageManagerImpl extends ManagerBase implements StorageManager, C
         return details;
     }
 
+    @ActionEvent(eventType = EventTypes.EVENT_DISABLE_PRIMARY_STORAGE, eventDescription = "disable storage pool")
+    private void disablePrimaryStoragePool(StoragePoolVO primaryStorage) {
+        if (!primaryStorage.getStatus().equals(StoragePoolStatus.Up)) {
+            throw new InvalidParameterValueException("Primary storage with id " + primaryStorage.getId() + " cannot be disabled. Storage pool state : " +
+                    primaryStorage.getStatus().toString());
+        }
+
+        DataStoreProvider provider = _dataStoreProviderMgr.getDataStoreProvider(primaryStorage.getStorageProviderName());
+        DataStoreLifeCycle dataStoreLifeCycle = provider.getDataStoreLifeCycle();
+        DataStore store = _dataStoreMgr.getDataStore(primaryStorage.getId(), DataStoreRole.Primary);
+        ((PrimaryDataStoreLifeCycle)dataStoreLifeCycle).disableStoragePool(store);
+    }
+
+    @ActionEvent(eventType = EventTypes.EVENT_ENABLE_PRIMARY_STORAGE, eventDescription = "enable storage pool")
+    private void enablePrimaryStoragePool(StoragePoolVO primaryStorage) {
+        if (!primaryStorage.getStatus().equals(StoragePoolStatus.Disabled)) {
+            throw new InvalidParameterValueException("Primary storage with id " + primaryStorage.getId() + " cannot be enabled. Storage pool state : " +
+                    primaryStorage.getStatus().toString());
+        }
+
+        DataStoreProvider provider = _dataStoreProviderMgr.getDataStoreProvider(primaryStorage.getStorageProviderName());
+        DataStoreLifeCycle dataStoreLifeCycle = provider.getDataStoreLifeCycle();
+        DataStore store = _dataStoreMgr.getDataStore(primaryStorage.getId(), DataStoreRole.Primary);
+        ((PrimaryDataStoreLifeCycle)dataStoreLifeCycle).enableStoragePool(store);
+    }
+
     @Override
     public PrimaryDataStoreInfo updateStoragePool(UpdateStoragePoolCmd cmd) throws IllegalArgumentException {
         // Input validation
@@ -816,7 +844,14 @@ public class StorageManagerImpl extends ManagerBase implements StorageManager, C
             }
         }
 
-        if (updatedDetails.size() >= 0) {
+        Boolean enabled = cmd.getEnabled();
+        if (enabled != null) {
+            if (enabled) {
+                enablePrimaryStoragePool(pool);
+            } else {
+                disablePrimaryStoragePool(pool);
+            }
+        } else if (updatedDetails.size() >= 0) {
             _storagePoolDao.updateDetails(id, updatedDetails);
         }