You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/05/09 21:08:58 UTC

[10/50] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index 2b847d0..516b97d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -14,6 +14,34 @@
  */
 package org.apache.geode.management.internal.beans;
 
+import static org.apache.geode.internal.lang.SystemUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.lang.management.MemoryUsage;
+import java.lang.management.OperatingSystemMXBean;
+import java.lang.management.RuntimeMXBean;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import javax.management.JMRuntimeException;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
 import org.apache.geode.cache.CacheClosedException;
@@ -26,11 +54,26 @@ import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.LocatorLauncher;
 import org.apache.geode.distributed.ServerLauncher;
-import org.apache.geode.distributed.internal.*;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.locks.DLockService;
 import org.apache.geode.distributed.internal.locks.DLockStats;
-import org.apache.geode.internal.*;
-import org.apache.geode.internal.cache.*;
+import org.apache.geode.internal.Assert;
+import org.apache.geode.internal.GemFireVersion;
+import org.apache.geode.internal.PureJavaMode;
+import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.cache.DirectoryHolder;
+import org.apache.geode.internal.cache.DiskDirectoryStats;
+import org.apache.geode.internal.cache.DiskRegion;
+import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.DiskStoreStats;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PartitionedRegionStats;
 import org.apache.geode.internal.cache.control.ResourceManagerStats;
 import org.apache.geode.internal.cache.execute.FunctionServiceStats;
 import org.apache.geode.internal.cache.lru.LRUStatistics;
@@ -48,46 +91,45 @@ import org.apache.geode.internal.process.PidUnavailableException;
 import org.apache.geode.internal.process.ProcessUtils;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
 import org.apache.geode.internal.statistics.HostStatHelper;
+import org.apache.geode.internal.statistics.StatSamplerStats;
+import org.apache.geode.internal.statistics.VMStatsContract;
 import org.apache.geode.internal.statistics.platform.LinuxSystemStats;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.internal.statistics.platform.SolarisSystemStats;
-import org.apache.geode.internal.statistics.StatSamplerStats;
-import org.apache.geode.internal.statistics.VMStatsContract;
 import org.apache.geode.internal.statistics.platform.WindowsSystemStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.internal.tcp.ConnectionTable;
-import org.apache.geode.management.*;
+import org.apache.geode.management.DependenciesNotFoundException;
+import org.apache.geode.management.DiskBackupResult;
+import org.apache.geode.management.GemFireProperties;
+import org.apache.geode.management.JVMMetrics;
+import org.apache.geode.management.ManagementException;
+import org.apache.geode.management.OSMetrics;
 import org.apache.geode.management.cli.CommandService;
 import org.apache.geode.management.cli.CommandServiceException;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.ManagementConstants;
 import org.apache.geode.management.internal.ManagementStrings;
 import org.apache.geode.management.internal.SystemManagementService;
-import org.apache.geode.management.internal.beans.stats.*;
+import org.apache.geode.management.internal.beans.stats.AggregateRegionStatsMonitor;
+import org.apache.geode.management.internal.beans.stats.GCStatsMonitor;
+import org.apache.geode.management.internal.beans.stats.MBeanStatsMonitor;
+import org.apache.geode.management.internal.beans.stats.MemberLevelDiskMonitor;
+import org.apache.geode.management.internal.beans.stats.StatType;
+import org.apache.geode.management.internal.beans.stats.StatsAverageLatency;
+import org.apache.geode.management.internal.beans.stats.StatsKey;
+import org.apache.geode.management.internal.beans.stats.StatsLatency;
+import org.apache.geode.management.internal.beans.stats.StatsRate;
+import org.apache.geode.management.internal.beans.stats.VMStatsMonitor;
 import org.apache.geode.management.internal.cli.CommandResponseBuilder;
 import org.apache.geode.management.internal.cli.remote.CommandExecutionContext;
 import org.apache.geode.management.internal.cli.remote.MemberCommandService;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.logging.log4j.Logger;
-
-import javax.management.JMRuntimeException;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import java.io.File;
-import java.io.IOException;
-import java.lang.management.*;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
 
 /**
  * This class acts as an Bridge between MemberMBean and GemFire Cache and Distributed System
- * 
- * 
  */
 public class MemberMBeanBridge {
 
@@ -106,7 +148,7 @@ public class MemberMBeanBridge {
   private static TimeUnit nanoSeconds = TimeUnit.NANOSECONDS;
 
   /** Cache Instance **/
-  private GemFireCacheImpl cache;
+  private InternalCache cache;
 
   /** Distribution Config **/
   private DistributionConfig config;
@@ -128,6 +170,7 @@ public class MemberMBeanBridge {
    * Command Service
    */
   private CommandService commandService;
+
   private String commandServiceInitError;
 
   /**
@@ -148,10 +191,8 @@ public class MemberMBeanBridge {
   /**
    * Reference to JDK bean OperatingSystemMXBean
    */
-
   private OperatingSystemMXBean osBean;
 
-
   /**
    * Host name of the member
    */
@@ -187,7 +228,6 @@ public class MemberMBeanBridge {
 
   private AggregateRegionStatsMonitor regionMonitor;
 
-
   private StatsRate createsRate;
 
   private StatsRate bytesReceivedRate;
@@ -266,8 +306,6 @@ public class MemberMBeanBridge {
 
   private MBeanStatsMonitor systemStatsMonitor;
 
-
-
   private float instCreatesRate = 0;
 
   private float instGetsRate = 0;
@@ -289,7 +327,7 @@ public class MemberMBeanBridge {
 
   private ResourceManagerStats resourceManagerStats;
 
-  public MemberMBeanBridge(GemFireCacheImpl cache, SystemManagementService service) {
+  public MemberMBeanBridge(InternalCache cache, SystemManagementService service) {
     this.cache = cache;
     this.service = service;
 
@@ -303,7 +341,6 @@ public class MemberMBeanBridge {
           .getRedundancyZone(cache.getInternalDistributedSystem().getDistributedMember());
     }
 
-
     this.sampler = system.getStatSampler();
 
     this.config = system.getConfig();
@@ -319,16 +356,9 @@ public class MemberMBeanBridge {
       logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}", e.getMessage());
     } catch (DependenciesNotFoundException e) {
       commandServiceInitError = e.getMessage();
-      if (CacheServerLauncher.isDedicatedCacheServer) {
-        // log as error for dedicated cache server - launched through script
-        // LOG:CONFIG:
-        logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}",
-            e.getMessage());
-      } else {
-        // LOG:CONFIG:
-        logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}",
-            e.getMessage());
-      }
+      // log as error for dedicated cache server - launched through script
+      // LOG:CONFIG:
+      logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}", e.getMessage());
     }
 
     intitGemfireProperties();
@@ -336,7 +366,7 @@ public class MemberMBeanBridge {
     try {
       InetAddress addr = SocketCreator.getLocalHost();
       this.hostname = addr.getHostName();
-    } catch (UnknownHostException ex) {
+    } catch (UnknownHostException ignore) {
       this.hostname = ManagementConstants.DEFAULT_HOST_NAME;
     }
 
@@ -359,7 +389,6 @@ public class MemberMBeanBridge {
     this.runtimeMXBean = ManagementFactory.getRuntimeMXBean();
     this.osBean = ManagementFactory.getOperatingSystemMXBean();
 
-
     // Initialize all the Stats Monitors
     this.monitor =
         new MBeanStatsMonitor(ManagementStrings.MEMBER_CACHE_MONITOR.toLocalizedString());
@@ -387,14 +416,12 @@ public class MemberMBeanBridge {
       }
     }
 
-
     QueryDataFunction qDataFunction = new QueryDataFunction();
     FunctionService.registerFunction(qDataFunction);
 
     this.resourceManagerStats = cache.getInternalResourceManager().getStats();
   }
 
-
   public MemberMBeanBridge() {
     this.monitor =
         new MBeanStatsMonitor(ManagementStrings.MEMBER_CACHE_MONITOR.toLocalizedString());
@@ -412,7 +439,7 @@ public class MemberMBeanBridge {
   }
 
   public MemberMBeanBridge init() {
-    CachePerfStats cachePerfStats = ((GemFireCacheImpl) cache).getCachePerfStats();
+    CachePerfStats cachePerfStats = this.cache.getCachePerfStats();
     addCacheStats(cachePerfStats);
     addFunctionStats(system.getFunctionServiceStats());
 
@@ -440,7 +467,7 @@ public class MemberMBeanBridge {
       }
     }
 
-    MemoryAllocator allocator = ((GemFireCacheImpl) cache).getOffHeapStore();
+    MemoryAllocator allocator = this.cache.getOffHeapStore();
     if ((null != allocator)) {
       OffHeapMemoryStats offHeapStats = allocator.getStats();
 
@@ -546,7 +573,6 @@ public class MemberMBeanBridge {
         removeDirectoryStats(dh.getDiskDirectoryStats());
       }
     }
-
   }
 
   public void removePartionRegionStats(PartitionedRegionStats parStats) {
@@ -561,14 +587,12 @@ public class MemberMBeanBridge {
     regionMonitor.removeDirectoryStatistics(diskDirStats.getStats());
   }
 
-
   public void addLockServiceStats(DLockService lock) {
     if (!lockStatsAdded) {
       DLockStats stats = (DLockStats) lock.getStats();
       addLockServiceStats(stats);
       lockStatsAdded = true;
     }
-
   }
 
   public void addLockServiceStats(DLockStats stats) {
@@ -578,7 +602,6 @@ public class MemberMBeanBridge {
   public void addSystemStats() {
     GemFireStatSampler sampler = system.getStatSampler();
 
-
     ProcessStats processStats = sampler.getProcessStats();
 
     StatSamplerStats samplerStats = sampler.getStatSamplerStats();
@@ -605,8 +628,6 @@ public class MemberMBeanBridge {
         vmStatsMonitor.addStatisticsToMonitor(vmHeapStats);
       }
 
-      // vmStatsMonitor.addStatisticsToMonitor(vm50.getVMNonHeapStats());
-
       StatisticsType gcType = VMStats50.getGCType();
       if (gcType != null) {
         Statistics[] gcStats = system.findStatisticsByType(gcType);
@@ -646,7 +667,6 @@ public class MemberMBeanBridge {
   }
 
   private void initializeStats() {
-
     createsRate = new StatsRate(StatsKey.CREATES, StatType.INT_TYPE, monitor);
     bytesReceivedRate = new StatsRate(StatsKey.RECEIVED_BYTES, StatType.LONG_TYPE, monitor);
     bytesSentRate = new StatsRate(StatsKey.SENT_BYTES, StatType.LONG_TYPE, monitor);
@@ -689,7 +709,6 @@ public class MemberMBeanBridge {
     netLoadsAverageLatency = new StatsAverageLatency(StatsKey.NET_LOADS_COMPLETED,
         StatType.INT_TYPE, StatsKey.NET_LOADS_TIME, monitor);
 
-
     netSearchAverageLatency = new StatsAverageLatency(StatsKey.NET_SEARCH_COMPLETED,
         StatType.INT_TYPE, StatsKey.NET_SEARCH_TIME, monitor);
 
@@ -721,7 +740,6 @@ public class MemberMBeanBridge {
     pdxDeserializationRate =
         new StatsRate(StatsKey.PDX_INSTANCE_DESERIALIZATIONS, StatType.INT_TYPE, monitor);
 
-
     lruDestroyRate = new StatsRate(StatsKey.LRU_DESTROYS, StatType.LONG_TYPE, regionMonitor);
 
     lruEvictionRate = new StatsRate(StatsKey.LRU_EVICTIONS, StatType.LONG_TYPE, regionMonitor);
@@ -731,15 +749,12 @@ public class MemberMBeanBridge {
     if (gemFirePropertyData == null) {
       this.gemFirePropertyData = BeanUtilFuncs.initGemfireProperties(config);
     }
-
   }
 
-
   /**
    * @return Some basic JVM metrics at the particular instance
    */
   public JVMMetrics fetchJVMMetrics() {
-
     long gcCount = getGCStatistic(StatsKey.VM_GC_STATS_COLLECTIONS).longValue();
     long gcTimeMillis = getGCStatistic(StatsKey.VM_GC_STATS_COLLECTION_TIME).longValue();
 
@@ -780,47 +795,45 @@ public class MemberMBeanBridge {
       int availableProcessors = osBean.getAvailableProcessors();
       double systemLoadAverage = osBean.getSystemLoadAverage();
 
-
       openFileDescriptorCount = getVMStatistic(StatsKey.VM_STATS_OPEN_FDS).longValue();
       processCpuTime = getVMStatistic(StatsKey.VM_PROCESS_CPU_TIME).longValue();
 
       try {
         maxFileDescriptorCount =
             (Long) mbeanServer.getAttribute(osObjectName, "MaxFileDescriptorCount");
-      } catch (Exception e) {
+      } catch (Exception ignore) {
         maxFileDescriptorCount = -1;
       }
       try {
         committedVirtualMemorySize =
             (Long) mbeanServer.getAttribute(osObjectName, "CommittedVirtualMemorySize");
-      } catch (Exception e) {
+      } catch (Exception ignore) {
         committedVirtualMemorySize = -1;
       }
 
-
       // If Linux System type exists
       if (PureJavaMode.osStatsAreAvailable() && HostStatHelper.isLinux() && systemStat != null) {
 
         try {
           totalPhysicalMemorySize =
               systemStat.get(StatsKey.LINUX_SYSTEM_PHYSICAL_MEMORY).longValue();
-        } catch (Exception e) {
+        } catch (Exception ignore) {
           totalPhysicalMemorySize = -1;
         }
         try {
           freePhysicalMemorySize = systemStat.get(StatsKey.LINUX_SYSTEM_FREE_MEMORY).longValue();
-        } catch (Exception e) {
+        } catch (Exception ignore) {
           freePhysicalMemorySize = -1;
         }
         try {
           totalSwapSpaceSize = systemStat.get(StatsKey.LINUX_SYSTEM_TOTAL_SWAP_SIZE).longValue();
-        } catch (Exception e) {
+        } catch (Exception ignore) {
           totalSwapSpaceSize = -1;
         }
 
         try {
           freeSwapSpaceSize = systemStat.get(StatsKey.LINUX_SYSTEM_FREE_SWAP_SIZE).longValue();
-        } catch (Exception e) {
+        } catch (Exception ignore) {
           freeSwapSpaceSize = -1;
         }
 
@@ -840,14 +853,11 @@ public class MemberMBeanBridge {
       if (logger.isTraceEnabled()) {
         logger.trace(ex.getMessage(), ex);
       }
-
     }
     return metrics;
-
   }
 
   /**
-   * 
    * @return GemFire Properties
    */
   public GemFireProperties getGemFireProperty() {
@@ -872,15 +882,12 @@ public class MemberMBeanBridge {
    * @return a list of compacted Disk stores
    */
   public String[] compactAllDiskStores() {
-
-    GemFireCacheImpl cacheImpl = (GemFireCacheImpl) cache;
     List<String> compactedStores = new ArrayList<String>();
 
     if (cache != null && !cache.isClosed()) {
-      for (DiskStoreImpl store : cacheImpl.listDiskStoresIncludingRegionOwned()) {
+      for (DiskStore store : this.cache.listDiskStoresIncludingRegionOwned()) {
         if (store.forceCompaction()) {
-          compactedStores.add(store.getPersistentID().getDirectory());
-
+          compactedStores.add(((DiskStoreImpl) store).getPersistentID().getDirectory());
         }
       }
     }
@@ -895,30 +902,26 @@ public class MemberMBeanBridge {
    * @return list all the disk Stores name at cache level
    */
   public String[] listDiskStores(boolean includeRegionOwned) {
-    GemFireCacheImpl cacheImpl = (GemFireCacheImpl) cache;
     String[] retStr = null;
-    Collection<DiskStoreImpl> diskCollection = null;
+    Collection<DiskStore> diskCollection = null;
     if (includeRegionOwned) {
-      diskCollection = cacheImpl.listDiskStoresIncludingRegionOwned();
+      diskCollection = this.cache.listDiskStoresIncludingRegionOwned();
     } else {
-      diskCollection = cacheImpl.listDiskStores();
+      diskCollection = this.cache.listDiskStores();
     }
     if (diskCollection != null && diskCollection.size() > 0) {
       retStr = new String[diskCollection.size()];
-      Iterator<DiskStoreImpl> it = diskCollection.iterator();
+      Iterator<DiskStore> it = diskCollection.iterator();
       int i = 0;
       while (it.hasNext()) {
         retStr[i] = it.next().getName();
         i++;
-
       }
     }
     return retStr;
-
   }
 
   /**
-   * 
    * @return list of disk stores which defaults includeRegionOwned = true;
    */
   public String[] getDiskStores() {
@@ -926,11 +929,9 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return log of the member.
    */
   public String fetchLog(int numLines) {
-
     if (numLines > ManagementConstants.MAX_SHOW_LOG_LINES) {
       numLines = ManagementConstants.MAX_SHOW_LOG_LINES;
     }
@@ -965,13 +966,14 @@ public class MemberMBeanBridge {
       return LocalizedStrings.SystemMemberImpl_NO_LOG_FILE_CONFIGURED_LOG_MESSAGES_WILL_BE_DIRECTED_TO_STDOUT
           .toLocalizedString();
     } else {
-      StringBuffer result = new StringBuffer();
+      StringBuilder result = new StringBuilder();
       if (mainTail != null) {
         result.append(mainTail);
       }
       if (childTail != null) {
-        result.append(
-            "\n" + LocalizedStrings.SystemMemberImpl_TAIL_OF_CHILD_LOG.toLocalizedString() + "\n");
+        result.append(getLineSeparator())
+            .append(LocalizedStrings.SystemMemberImpl_TAIL_OF_CHILD_LOG.toLocalizedString())
+            .append(getLineSeparator());
         result.append(childTail);
       }
       return result.toString();
@@ -985,7 +987,6 @@ public class MemberMBeanBridge {
    * before deciding to shut itself down
    */
   public void shutDownMember() {
-
     final InternalDistributedSystem ids = dm.getSystem();
     if (ids.isConnected()) {
       Thread t = new Thread(new Runnable() {
@@ -993,7 +994,7 @@ public class MemberMBeanBridge {
           try {
             // Allow the Function call to exit
             Thread.sleep(1000);
-          } catch (InterruptedException e) {
+          } catch (InterruptedException ignore) {
           }
           ConnectionTable.threadWantsSharedResources();
           if (ids.isConnected()) {
@@ -1004,25 +1005,20 @@ public class MemberMBeanBridge {
       t.setDaemon(false);
       t.start();
     }
-
   }
 
   /**
    * backs up all the disk to the targeted directory
    * 
    * @param targetDirPath path of the directory where back up is to be taken
-   * @return array of DiskBackup results which might get aggregated at Managing node
-   * 
-   *         Check the validity of this mbean call. When does it make sense to backup a single
-   *         member of a gemfire system in isolation of the other members?
+   * @return array of DiskBackup results which might get aggregated at Managing node Check the
+   *         validity of this mbean call. When does it make sense to backup a single member of a
+   *         gemfire system in isolation of the other members?
    */
   public DiskBackupResult[] backupMember(String targetDirPath) {
-
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-
     if (cache != null) {
-      Collection<DiskStoreImpl> diskStores = cache.listDiskStoresIncludingRegionOwned();
-      for (DiskStoreImpl store : diskStores) {
+      Collection<DiskStore> diskStores = cache.listDiskStoresIncludingRegionOwned();
+      for (DiskStore store : diskStores) {
         store.flush();
       }
     }
@@ -1031,11 +1027,9 @@ public class MemberMBeanBridge {
     File targetDir = new File(targetDirPath);
 
     if (cache == null) {
-
       return null;
 
     } else {
-
       try {
         BackupManager manager =
             cache.startBackup(cache.getInternalDistributedSystem().getDistributedMember());
@@ -1068,7 +1062,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return The name for this member.
    */
   public String getName() {
@@ -1076,7 +1069,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return The ID for this member.
    */
   public String getId() {
@@ -1084,7 +1076,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return The name of the member if it's been set, otherwise the ID of the member
    */
   public String getMember() {
@@ -1102,7 +1093,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return classPath of the VM
    */
   public String getClassPath() {
@@ -1110,7 +1100,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return Connected gateway receivers
    */
   public String[] listConnectedGatewayReceivers() {
@@ -1128,7 +1117,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return Connected gateway senders
    */
   public String[] listConnectedGatewaySenders() {
@@ -1146,7 +1134,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return approximate usage of CPUs
    */
   public float getCpuUsage() {
@@ -1154,7 +1141,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return current time of the system
    */
   public long getCurrentTime() {
@@ -1175,19 +1161,15 @@ public class MemberMBeanBridge {
   /**
    * Gets a String describing the GemFire member's status. A GemFire member includes, but is not
    * limited to: Locators, Managers, Cache Servers and so on.
-   * </p>
-   * 
+   *
    * @return String description of the GemFire member's status.
    * @see #isLocator()
    * @see #isServer()
    */
   public String status() {
-    // if (isLocator()) {
     if (LocatorLauncher.getInstance() != null) {
       return LocatorLauncher.getLocatorState().toJson();
-    }
-    // else if (isServer()) {
-    else if (ServerLauncher.getInstance() != null) {
+    } else if (ServerLauncher.getInstance() != null) {
       return ServerLauncher.getServerState().toJson();
     }
 
@@ -1196,17 +1178,14 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return total heap usage in bytes
    */
   public long getTotalBytesInUse() {
     MemoryUsage memHeap = memoryMXBean.getHeapMemoryUsage();
-    long bytesUsed = memHeap.getUsed();
-    return bytesUsed;
+    return memHeap.getUsed();
   }
 
   /**
-   * 
    * @return Number of availabe CPUs
    */
   public int getAvailableCpus() {
@@ -1215,7 +1194,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return JVM thread list
    */
   public String[] fetchJvmThreads() {
@@ -1235,7 +1213,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return list of regions
    */
   public String[] getListOfRegions() {
@@ -1253,7 +1230,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return configuration data lock lease
    */
   public long getLockLease() {
@@ -1261,14 +1237,12 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return configuration data lock time out
    */
   public long getLockTimeout() {
     return cache.getLockTimeout();
   }
 
-
   /**
    * @return the duration for which the member is up
    */
@@ -1301,7 +1275,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return true if this members has a gateway receiver
    */
   public boolean hasGatewayReceiver() {
@@ -1309,7 +1282,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return true if member has Gateway senders
    */
   public boolean hasGatewaySender() {
@@ -1317,7 +1289,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return true if member contains one locator. From 7.0 only locator can be hosted in a JVM
    */
   public boolean isLocator() {
@@ -1325,18 +1296,15 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return true if the Federating Manager Thread is running
    */
   public boolean isManager() {
-    GemFireCacheImpl existingCache = GemFireCacheImpl.getInstance();
-    if (existingCache == null || existingCache.isClosed()) {
+    if (this.cache == null || this.cache.isClosed()) {
       return false;
     }
     try {
-      boolean isManager = service.isManager();
-      return isManager;
-    } catch (Exception e) {
+      return service.isManager();
+    } catch (Exception ignore) {
       return false;
     }
   }
@@ -1348,28 +1316,23 @@ public class MemberMBeanBridge {
    * @return true if the manager has been created.
    */
   public boolean isManagerCreated() {
-    GemFireCacheImpl existingCache = GemFireCacheImpl.getInstance();
-    if (existingCache == null || existingCache.isClosed()) {
+    if (this.cache == null || this.cache.isClosed()) {
       return false;
     }
     try {
       return service.isManagerCreated();
-    } catch (Exception e) {
+    } catch (Exception ignore) {
       return false;
     }
   }
 
   /**
-   * 
    * @return true if member has a server
    */
   public boolean isServer() {
     return cache.isServer();
   }
 
-  /** Statistics Related Attributes **/
-  /*********************************************************************************************************/
-
   public int getInitialImageKeysReceived() {
     return getMemberLevelStatistic(StatsKey.GET_INITIAL_IMAGE_KEYS_RECEIVED).intValue();
   }
@@ -1496,7 +1459,6 @@ public class MemberMBeanBridge {
 
   public int getTotalHitCount() {
     return getMemberLevelStatistic(StatsKey.GETS).intValue() - getTotalMissCount();
-
   }
 
   public float getLruDestroyRate() {
@@ -1562,7 +1524,6 @@ public class MemberMBeanBridge {
   public int getTotalTransactionsCount() {
     return getMemberLevelStatistic(StatsKey.TRANSACTION_COMMITS).intValue()
         + getMemberLevelStatistic(StatsKey.TRANSACTION_ROLLBACKS).intValue();
-
   }
 
   public long getTransactionCommitsAvgLatency() {
@@ -1660,7 +1621,6 @@ public class MemberMBeanBridge {
     return regionMonitor.getDiskSpace();
   }
 
-
   public float getAverageReads() {
     return instGetsRate;
   }
@@ -1690,7 +1650,6 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return max limit of FD ..Ulimit
    */
   public long getFileDescriptorLimit() {
@@ -1701,14 +1660,13 @@ public class MemberMBeanBridge {
     try {
       maxFileDescriptorCount =
           (Long) mbeanServer.getAttribute(osObjectName, "MaxFileDescriptorCount");
-    } catch (Exception e) {
+    } catch (Exception ignore) {
       maxFileDescriptorCount = -1;
     }
     return maxFileDescriptorCount;
   }
 
   /**
-   * 
    * @return count of currently opened FDs
    */
   public long getTotalFileDescriptorOpen() {
@@ -1729,11 +1687,17 @@ public class MemberMBeanBridge {
     return objects;
   }
 
+  /**
+   * @deprecated Please use {@link #getOffHeapFreeMemory()} instead.
+   */
   @Deprecated
   public long getOffHeapFreeSize() {
     return getOffHeapFreeMemory();
   }
 
+  /**
+   * @deprecated Please use {@link #getOffHeapUsedMemory()} instead.
+   */
   @Deprecated
   public long getOffHeapUsedSize() {
     return getOffHeapUsedMemory();
@@ -1810,7 +1774,6 @@ public class MemberMBeanBridge {
   }
 
   public int getHostCpuUsage() {
-
     if (systemStat != null) {
       return systemStat.get(StatsKey.SYSTEM_CPU_ACTIVE).intValue();
     } else {
@@ -1818,12 +1781,10 @@ public class MemberMBeanBridge {
     }
   }
 
-
   public boolean isCacheServer() {
     return cacheServer;
   }
 
-
   public void setCacheServer(boolean cacheServer) {
     this.cacheServer = cacheServer;
   }
@@ -1836,22 +1797,18 @@ public class MemberMBeanBridge {
     return resourceManagerStats.getRebalancesInProgress();
   }
 
-
   public int getReplyWaitsInProgress() {
     return getMemberLevelStatistic(StatsKey.REPLY_WAITS_IN_PROGRESS).intValue();
   }
 
-
   public int getReplyWaitsCompleted() {
     return getMemberLevelStatistic(StatsKey.REPLY_WAITS_COMPLETED).intValue();
   }
 
-
   public int getVisibleNodes() {
     return getMemberLevelStatistic(StatsKey.NODES).intValue();
   }
 
-
   public long getMaxMemory() {
     Runtime rt = Runtime.getRuntime();
     return rt.maxMemory() / MBFactor;

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
index efd7436..705ba4d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
@@ -30,7 +30,6 @@ import java.util.regex.Pattern;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.Region;
@@ -51,6 +50,7 @@ import org.apache.geode.cache.query.internal.QCompiler;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.InternalEntity;
 import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalDataSet;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegionHelper;
@@ -127,7 +127,7 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
   private QueryDataFunctionResult selectWithType(final FunctionContext context, String queryString,
       final boolean showMember, final String regionName, final int limit,
       final int queryResultSetLimit, final int queryCollectionsDepth) throws Exception {
-    Cache cache = CacheFactory.getAnyInstance();
+    InternalCache cache = getCache();
     Function loclQueryFunc = new LocalQueryFunction("LocalQueryFunction", regionName, showMember)
         .setOptimizeForWrite(true);
     queryString = applyLimitClause(queryString, limit, queryResultSetLimit);
@@ -293,12 +293,11 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
         List<String> decompressedList = new ArrayList<String>();
 
         while (it.hasNext()) {
-
           String decompressedStr = null;
           decompressedStr = BeanUtilFuncs.decompress(it.next().compressedBytes);
           decompressedList.add(decompressedStr);
-
         }
+
         if (zipResult) {
           return BeanUtilFuncs.compress(wrapResult(decompressedList.toString()));
         } else {
@@ -359,7 +358,7 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
       }
     }
 
-    Cache cache = CacheFactory.getAnyInstance();
+    InternalCache cache = (InternalCache) CacheFactory.getAnyInstance();
     try {
 
       SystemManagementService service =
@@ -386,7 +385,6 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
               }
             }
           }
-
         }
       } else {
         return new JsonisedErroMessage(ManagementStrings.QUERY__MSG__INVALID_QUERY
@@ -449,6 +447,10 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
     }
   }
 
+  private InternalCache getCache() {
+    return (InternalCache) CacheFactory.getAnyInstance();
+  }
+
   private static class JsonisedErroMessage {
 
     private static String message = "message";
@@ -467,7 +469,6 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
     public String toString() {
       return gFJsonObject.toString();
     }
-
   }
 
   /**
@@ -479,7 +480,7 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
    *
    * @return a set of regions involved in the query
    */
-  private static Set<String> compileQuery(final Cache cache, final String query)
+  private static Set<String> compileQuery(final InternalCache cache, final String query)
       throws QueryInvalidException {
     QCompiler compiler = new QCompiler();
     Set<String> regionsInQuery = null;
@@ -498,7 +499,7 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
   /**
    * Function to gather data locally. This function is required to execute query with region context
    */
-  private class LocalQueryFunction extends FunctionAdapter {
+  private static class LocalQueryFunction extends FunctionAdapter {
 
     private static final long serialVersionUID = 1L;
 
@@ -537,7 +538,7 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
 
     @Override
     public void execute(final FunctionContext context) {
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getCache();
       QueryService queryService = cache.getQueryService();
       String qstr = (String) context.getArguments();
       Region r = cache.getRegion(regionName);
@@ -557,6 +558,10 @@ public class QueryDataFunction extends FunctionAdapter implements InternalEntity
       }
     }
 
+    private InternalCache getCache() {
+      return (InternalCache) CacheFactory.getAnyInstance();
+    }
+
     @Override
     public String getId() {
       return this.id;

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
index b67b07d..8cfd4f2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
@@ -45,8 +45,7 @@ import org.apache.geode.management.internal.beans.stats.StatsRate;
 /**
  * This class acts as a bridge between a Region and RegionMBean This also listens for statistics
  * changes and update its value accordingly
- * 
- * 
+ *
  * @param <K>
  * @param <V>
  */
@@ -100,10 +99,7 @@ public class RegionMBeanBridge<K, V> {
 
   private StatsRate averageReadsRate;
 
-
-
   public static <K, V> RegionMBeanBridge<K, V> getInstance(Region<K, V> region) {
-
     if (region.getAttributes().getPartitionAttributes() != null) {
       RegionMBeanBridge<K, V> bridge = PartitionedRegionBridge.getInstance(region);
       PartitionedRegion parRegion = ((PartitionedRegion) region);
@@ -119,9 +115,7 @@ public class RegionMBeanBridge<K, V> {
         }
 
         bridge.addDiskRegionBridge(diskRegionBridge);
-
       }
-
       return bridge;
 
     } else {
@@ -139,11 +133,9 @@ public class RegionMBeanBridge<K, V> {
         }
       }
       return bridge;
-
     }
   }
 
-
   protected void addDiskRegionBridge(DiskRegionBridge diskRegionBridge) {
     this.diskRegionBridge = diskRegionBridge;
   }
@@ -167,8 +159,6 @@ public class RegionMBeanBridge<K, V> {
 
     this.persistentEnabled = region.getAttributes().getDataPolicy().withPersistence();
 
-
-
     this.regionStats = ((LocalRegion) region).getRegionPerfStats();
     if (regionStats != null) {
       regionMonitor.addStatisticsToMonitor(regionStats.getStats()); // fixes 46692
@@ -183,7 +173,6 @@ public class RegionMBeanBridge<K, V> {
         if (ea != null && ea.getAlgorithm().isLRUMemory()) {
           this.lruMemoryStats = stats;
         }
-
       }
     }
 
@@ -194,8 +183,6 @@ public class RegionMBeanBridge<K, V> {
     this.member = GemFireCacheImpl.getInstance().getDistributedSystem().getMemberId();
   }
 
-
-
   public String getRegionType() {
     return region.getAttributes().getDataPolicy().toString();
   }
@@ -216,7 +203,6 @@ public class RegionMBeanBridge<K, V> {
     return membershipAttributesData;
   }
 
-
   public RegionAttributesData listRegionAttributes() {
     return regionAttributesData;
   }
@@ -238,11 +224,6 @@ public class RegionMBeanBridge<K, V> {
     return subregionPaths.toArray(new String[subregionPaths.size()]);
   }
 
-  /** Statistic related Methods **/
-
-
-
-  // Dummy constructor for testing purpose only
   public RegionMBeanBridge(CachePerfStats cachePerfStats) {
     this.regionStats = cachePerfStats;
 
@@ -255,8 +236,6 @@ public class RegionMBeanBridge<K, V> {
   // Dummy constructor for testing purpose only
   public RegionMBeanBridge() {}
 
-
-
   public void stopMonitor() {
     regionMonitor.stopListener();
     if (diskRegionBridge != null) {
@@ -264,9 +243,7 @@ public class RegionMBeanBridge<K, V> {
     }
   }
 
-
   private void configureRegionMetrics() {
-
     putAllRate = new StatsRate(StatsKey.PUT_ALLS, StatType.INT_TYPE, regionMonitor);
     getRequestRate = new StatsRate(StatsKey.GETS, StatType.INT_TYPE, regionMonitor);
 
@@ -282,7 +259,6 @@ public class RegionMBeanBridge<K, V> {
     writerCallsAvgLatency = new StatsAverageLatency(StatsKey.CACHE_WRITER_CALLS_COMPLETED,
         StatType.INT_TYPE, StatsKey.CACHE_WRITER_CALL_TIME, regionMonitor);
 
-
     lruDestroyRate = new StatsRate(StatsKey.LRU_DESTROYS, StatType.LONG_TYPE, regionMonitor);
 
     lruEvictionRate = new StatsRate(StatsKey.LRU_EVICTIONS, StatType.LONG_TYPE, regionMonitor);
@@ -290,7 +266,6 @@ public class RegionMBeanBridge<K, V> {
     String[] writesRates = new String[] {StatsKey.PUT_ALLS, StatsKey.PUTS, StatsKey.CREATES};
     averageWritesRate = new StatsRate(writesRates, StatType.INT_TYPE, regionMonitor);
     averageReadsRate = new StatsRate(StatsKey.GETS, StatType.INT_TYPE, regionMonitor);
-
   }
 
   private Number getRegionStatistic(String statName) {
@@ -321,7 +296,6 @@ public class RegionMBeanBridge<K, V> {
     return putAllRate.getRate();
   }
 
-
   public float getPutsRate() {
     return putRequestRate.getRate();
   }
@@ -339,7 +313,6 @@ public class RegionMBeanBridge<K, V> {
       return region.getStatistics().getHitCount();
     }
     return ManagementConstants.NOT_AVAILABLE_LONG;
-
   }
 
   public float getHitRatio() {
@@ -370,7 +343,6 @@ public class RegionMBeanBridge<K, V> {
     return ManagementConstants.NOT_AVAILABLE_LONG;
   }
 
-
   public float getLruDestroyRate() {
     return lruDestroyRate.getRate();
   }
@@ -394,7 +366,6 @@ public class RegionMBeanBridge<K, V> {
     return ManagementConstants.NOT_AVAILABLE_LONG;
   }
 
-
   public boolean isGatewayEnabled() {
     return isGatewayEnabled;
   }
@@ -407,7 +378,6 @@ public class RegionMBeanBridge<K, V> {
     return member;
   }
 
-
   /**
    * Only applicable for PRs
    * 
@@ -520,7 +490,6 @@ public class RegionMBeanBridge<K, V> {
     return null;
   }
 
-
   public long getDiskReadsAverageLatency() {
     if (this.diskRegionBridge != null) {
       return diskRegionBridge.getDiskReadsAverageLatency();

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
index d125e53..bd6d810 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
@@ -14,30 +14,6 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheClosedException;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.Execution;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.cache.execute.AbstractExecution;
-import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.internal.util.IOUtils;
-import org.apache.geode.management.DistributedSystemMXBean;
-import org.apache.geode.management.ManagementService;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.functions.MembersForRegionFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.CommandResultException;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileFilter;
@@ -66,15 +42,40 @@ import java.util.zip.DataFormatException;
 import java.util.zip.Deflater;
 import java.util.zip.Inflater;
 
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Execution;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.execute.AbstractExecution;
+import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.internal.util.IOUtils;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.functions.MembersForRegionFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CommandResultException;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+
 /**
  * This class contains utility methods used by classes used to build the Command Line Interface
  * (CLI).
  *
- *
  * @since GemFire 7.0
  */
 public class CliUtil {
+
   public static final String GFSHVM_IDENTIFIER = "gfsh";
+
   public static boolean isGfshVM = Boolean.getBoolean(GFSHVM_IDENTIFIER);
 
   public static final FileFilter JAR_FILE_FILTER = new CustomFileFilter(".jar");
@@ -123,10 +124,10 @@ public class CliUtil {
     return null;
   }
 
-  public static Cache getCacheIfExists() {
-    Cache cache;
+  public static InternalCache getCacheIfExists() {
+    InternalCache cache;
     try {
-      cache = CacheFactory.getAnyInstance();
+      cache = getInternalCache();
     } catch (CacheClosedException e) {
       // ignore & return null
       cache = null;
@@ -146,8 +147,8 @@ public class CliUtil {
         throw new FileNotFoundException("Could not find " + file.getCanonicalPath());
       }
 
-      if (file.isDirectory()) { // TODO - Abhishek: (1) No recursive search yet. (2) Do we need to
-                                // check/limit size of the files too?
+      if (file.isDirectory()) {
+        // TODO: (1) No recursive search yet. (2) Do we need to check/limit size of the files too?
         File[] childrenFiles = file.listFiles(JAR_FILE_FILTER);
         for (int j = 0; j < childrenFiles.length; j++) {
           // 1. add name of the file as bytes at even index
@@ -221,15 +222,17 @@ public class CliUtil {
     }
   }
 
-
-
   public static boolean isValidFileName(String filePath, String extension) {
     boolean isValid = true;
     return isValid;
   }
 
+  private static InternalCache getInternalCache() {
+    return (InternalCache) CacheFactory.getAnyInstance();
+  }
+
   public static Set<String> getAllRegionNames() {
-    Cache cache = CacheFactory.getAnyInstance();
+    InternalCache cache = getInternalCache();
     Set<String> regionNames = new HashSet<String>();
     Set<Region<?, ?>> rootRegions = cache.rootRegions();
 
@@ -278,8 +281,6 @@ public class CliUtil {
     return sb.toString();
   }
 
-
-
   public static Set<DistributedMember> findMembersOrThrow(final String groups, final String members)
       throws CommandResultException {
 
@@ -307,7 +308,7 @@ public class CliUtil {
    */
   public static Set<DistributedMember> findMembersIncludingLocators(String[] groups,
       String[] members) {
-    Cache cache = CacheFactory.getAnyInstance();
+    InternalCache cache = getInternalCache();
     Set<DistributedMember> allMembers = getAllMembers(cache);
 
     return findMembers(allMembers, groups, members);
@@ -318,7 +319,7 @@ public class CliUtil {
    * locators.
    */
   public static Set<DistributedMember> findMembers(String[] groups, String[] members) {
-    Cache cache = CacheFactory.getAnyInstance();
+    InternalCache cache = getInternalCache();
     Set<DistributedMember> allNormalMembers = getAllNormalMembers(cache);
 
     return findMembers(allNormalMembers, groups, members);
@@ -367,7 +368,7 @@ public class CliUtil {
     DistributedMember memberFound = null;
 
     if (memberNameOrId != null) {
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getInternalCache();
       Set<DistributedMember> memberSet = CliUtil.getAllMembers(cache);
       for (DistributedMember member : memberSet) {
         if (memberNameOrId.equalsIgnoreCase(member.getId())
@@ -442,7 +443,6 @@ public class CliUtil {
     @Override
     public boolean accept(File pathname) {
       String name = pathname.getName();
-
       return name.endsWith(extensionWithDot);
     }
   }
@@ -493,10 +493,6 @@ public class CliUtil {
     private final int dataLength;
     private final byte[] data;
 
-    /**
-     * @param dataLength
-     * @param data
-     */
     public DeflaterInflaterData(int dataLength, byte[] data) {
       this.dataLength = dataLength;
       this.data = data;
@@ -565,20 +561,24 @@ public class CliUtil {
 
   /**
    * Returns a set of all the members of the distributed system excluding locators.
+   * 
+   * @param cache
    */
   @SuppressWarnings("unchecked")
-  public static Set<DistributedMember> getAllNormalMembers(Cache cache) {
-    return new HashSet<DistributedMember>(((InternalDistributedSystem) cache.getDistributedSystem())
+  public static Set<DistributedMember> getAllNormalMembers(InternalCache cache) {
+    return new HashSet<DistributedMember>(cache.getInternalDistributedSystem()
         .getDistributionManager().getNormalDistributionManagerIds());
   }
 
   /**
    * Returns a set of all the members of the distributed system including locators.
+   * 
+   * @param cache
    */
   @SuppressWarnings("unchecked")
-  public static Set<DistributedMember> getAllMembers(Cache cache) {
-    return new HashSet<DistributedMember>(((InternalDistributedSystem) cache.getDistributedSystem())
-        .getDistributionManager().getDistributionManagerIds());
+  public static Set<DistributedMember> getAllMembers(InternalCache cache) {
+    return new HashSet<DistributedMember>(
+        cache.getInternalDistributedSystem().getDistributionManager().getDistributionManagerIds());
   }
 
   @SuppressWarnings("unchecked")
@@ -590,11 +590,12 @@ public class CliUtil {
   /**
    * Returns a set of all the members of the distributed system for the given groups.
    */
-  public static Set<DistributedMember> getDistributedMembersByGroup(Cache cache, String[] groups) {
+  public static Set<DistributedMember> getDistributedMembersByGroup(InternalCache cache,
+      String[] groups) {
     Set<DistributedMember> groupMembers = new HashSet<DistributedMember>();
     for (String group : groups) {
-      groupMembers.addAll(((InternalDistributedSystem) cache.getDistributedSystem())
-          .getDistributionManager().getGroupMembers(group));
+      groupMembers.addAll(
+          cache.getInternalDistributedSystem().getDistributionManager().getGroupMembers(group));
     }
     return groupMembers;
   }
@@ -652,13 +653,13 @@ public class CliUtil {
    * find.
    *
    * @param region region path for which members that have this region are required
-   * @param cache Cache instance to use to find members
+   * @param cache cache instance to use to find members
    * @param returnAll whether to return all members or only the first member we find. Returns all
    *        when <code>true</code>
    * @return a Set of DistributedMember for members that have the specified <code>region</code>.
    */
   public static Set<DistributedMember> getRegionAssociatedMembers(final String region,
-      final Cache cache, boolean returnAll) {
+      final InternalCache cache, boolean returnAll) {
     if (region == null || region.isEmpty()) {
       return null;
     }
@@ -742,20 +743,16 @@ public class CliUtil {
   /**
    * Resolves file system path relative to Gfsh. If the pathname is not specified, then pathname is
    * returned.
-   * <p/>
    *
    * @param pathname a String value specifying the file system pathname to resolve.
    * @return a String specifying a path relative to Gfsh.
    */
-  // Moved form LauncherLifeCycleCommands
   public static String resolvePathname(final String pathname) {
     return (StringUtils.isBlank(pathname) ? pathname
         : IOUtils.tryGetCanonicalPathElseGetAbsolutePath(new File(pathname)));
   }
 
-
   public static void runLessCommandAsExternalViewer(Result commandResult, boolean isError) {
-
     StringBuilder sb = new StringBuilder();
     String NEW_LINE = System.getProperty("line.separator");
 
@@ -784,7 +781,6 @@ public class CliUtil {
       if (file != null)
         file.delete();
     }
-
   }
 
   public static String getClientIdFromCacheClientProxy(CacheClientProxy p) {
@@ -797,7 +793,7 @@ public class CliUtil {
     return buffer.toString();
   }
 
-  public static Set<DistributedMember> getMembersForeRegionViaFunction(Cache cache,
+  public static Set<DistributedMember> getMembersForeRegionViaFunction(InternalCache cache,
       String regionPath, boolean returnAll) {
     try {
       Set<DistributedMember> regionMembers = new HashSet<DistributedMember>();

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AbstractCommandsSupport.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AbstractCommandsSupport.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AbstractCommandsSupport.java
index f077180..ae44e24 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AbstractCommandsSupport.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AbstractCommandsSupport.java
@@ -12,10 +12,15 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal.cli.commands;
 
-import org.apache.geode.cache.Cache;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.springframework.shell.core.CommandMarker;
+
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.Function;
@@ -23,6 +28,7 @@ import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.cli.CliMetaData;
@@ -30,12 +36,6 @@ import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.cli.util.MemberNotFoundException;
-import org.springframework.shell.core.CommandMarker;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.HashSet;
-import java.util.Set;
 
 /**
  * The AbstractCommandsSupport class is an abstract base class encapsulating common functionality
@@ -126,8 +126,8 @@ public abstract class AbstractCommandsSupport implements CommandMarker {
     return (getGfsh() != null);
   }
 
-  protected Cache getCache() {
-    return CacheFactory.getAnyInstance();
+  protected InternalCache getCache() {
+    return (InternalCache) CacheFactory.getAnyInstance();
   }
 
   protected static Gfsh getGfsh() {
@@ -135,7 +135,7 @@ public abstract class AbstractCommandsSupport implements CommandMarker {
   }
 
   @SuppressWarnings("deprecated")
-  protected DistributedMember getMember(final Cache cache, final String memberName) {
+  protected DistributedMember getMember(final InternalCache cache, final String memberName) {
     for (final DistributedMember member : getMembers(cache)) {
       if (memberName.equalsIgnoreCase(member.getName())
           || memberName.equalsIgnoreCase(member.getId())) {
@@ -149,15 +149,14 @@ public abstract class AbstractCommandsSupport implements CommandMarker {
 
   /**
    * Gets all members in the GemFire distributed system/cache.
-   * </p>
-   * 
+   *
    * @param cache the GemFire cache.
    * @return all members in the GemFire distributed system/cache.
-   * @see org.apache.geode.management.internal.cli.CliUtil#getAllMembers(org.apache.geode.cache.Cache)
+   * @see org.apache.geode.management.internal.cli.CliUtil#getAllMembers(org.apache.geode.internal.cache.InternalCache)
    * @deprecated use CliUtil.getAllMembers(org.apache.geode.cache.Cache) instead
    */
   @Deprecated
-  protected Set<DistributedMember> getMembers(final Cache cache) {
+  protected Set<DistributedMember> getMembers(final InternalCache cache) {
     Set<DistributedMember> members = new HashSet<DistributedMember>(cache.getMembers());
     members.add(cache.getDistributedSystem().getDistributedMember());
     return members;

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
index 3d155e5..18940fc 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
@@ -12,13 +12,12 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal.cli.commands;
 
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.CacheServerMXBean;
 import org.apache.geode.management.ClientHealthStatus;
 import org.apache.geode.management.ManagementService;
@@ -53,10 +52,8 @@ import java.util.Set;
 import javax.management.ObjectName;
 
 /**
- * 
  * @since GemFire 8.0
  */
-
 public class ClientCommands implements CommandMarker {
 
   private Gfsh getGfsh() {
@@ -77,7 +74,7 @@ public class ClientCommands implements CommandMarker {
       String headerText = "ClientList";
       resultTable = resultTable.setHeader(headerText);
 
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getCache();
       ManagementService service = ManagementService.getExistingManagementService(cache);
       ObjectName[] cacheServers = service.getDistributedSystemMXBean().listCacheServerObjectNames();
 
@@ -151,6 +148,9 @@ public class ClientCommands implements CommandMarker {
     return result;
   }
 
+  private InternalCache getCache() {
+    return (InternalCache) CacheFactory.getAnyInstance();
+  }
 
   @CliCommand(value = CliStrings.DESCRIBE_CLIENT, help = CliStrings.DESCRIBE_CLIENT__HELP)
   @CliMetaData(relatedTopic = {CliStrings.TOPIC_CLIENT})
@@ -174,7 +174,7 @@ public class ClientCommands implements CommandMarker {
     try {
       CompositeResultData compositeResultData = ResultBuilder.createCompositeResultData();
       SectionResultData sectionResult = compositeResultData.addSection("InfoSection");
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getCache();
 
       ManagementService service = ManagementService.getExistingManagementService(cache);
       ObjectName[] cacheServers = service.getDistributedSystemMXBean().listCacheServerObjectNames();
@@ -204,7 +204,6 @@ public class ClientCommands implements CommandMarker {
             }
           }
         }
-
       }
 
       if (clientHealthStatus == null) {
@@ -225,12 +224,7 @@ public class ClientCommands implements CommandMarker {
         for (int i = 0; i < resultList.size(); i++) {
           try {
             Object object = resultList.get(i);
-            if (object instanceof Exception) {
-              LogWrapper.getInstance().warning(
-                  "Exception in Describe Client " + ((Throwable) object).getMessage(),
-                  ((Throwable) object));
-              continue;
-            } else if (object instanceof Throwable) {
+            if (object instanceof Throwable) {
               LogWrapper.getInstance().warning(
                   "Exception in Describe Client " + ((Throwable) object).getMessage(),
                   ((Throwable) object));
@@ -259,7 +253,6 @@ public class ClientCommands implements CommandMarker {
                   secondaryServers.add(objectResult.secondaryServer);
                 }
               }
-
             }
           } catch (Exception e) {
             LogWrapper.getInstance().info(CliStrings.DESCRIBE_CLIENT_ERROR_FETCHING_STATS_0 + " :: "
@@ -354,7 +347,6 @@ public class ClientCommands implements CommandMarker {
         }
       }
     }
-
   }
 
   @CliAvailabilityIndicator({CliStrings.LIST_CLIENTS, CliStrings.DESCRIBE_CLIENT})

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index c4455ad..3b1fb09 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -14,9 +14,29 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
 import org.apache.geode.LogWriter;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.ExpirationAttributes;
 import org.apache.geode.cache.PartitionResolver;
@@ -30,7 +50,7 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.security.IntegratedSecurityService;
@@ -63,31 +83,12 @@ import org.apache.geode.management.internal.configuration.domain.XmlEntity;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
 
 /**
  * @since GemFire 7.0
  */
 public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
+
   public static final Set<RegionShortcut> PERSISTENT_OVERFLOW_SHORTCUTS =
       new TreeSet<RegionShortcut>();
 
@@ -108,6 +109,9 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     PERSISTENT_OVERFLOW_SHORTCUTS.add(RegionShortcut.LOCAL_PERSISTENT_OVERFLOW);
   }
 
+  /**
+   * TODO: method createRegion is too complex to analyze
+   */
   @CliCommand(value = CliStrings.CREATE_REGION, help = CliStrings.CREATE_REGION__HELP)
   @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
@@ -229,7 +233,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     AtomicReference<XmlEntity> xmlEntity = new AtomicReference<>();
 
     try {
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getCache();
 
       if (regionShortcut != null && useAttributesFrom != null) {
         throw new IllegalArgumentException(
@@ -285,7 +289,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
         String cacheLoaderClass =
             cacheLoader != null ? cacheLoader : regionAttributesResult.getCacheLoaderClass();
         String cacheWriterClass =
-            cacheWriter != null ? cacheWriter : regionAttributesResult.getCacheWriterClass();;
+            cacheWriter != null ? cacheWriter : regionAttributesResult.getCacheWriterClass();
 
         regionFunctionArgs = new RegionFunctionArgs(regionPath, useAttributesFrom, skipIfExists,
             keyConstraint, valueConstraint, statisticsEnabled, entryIdle, entryTTL, regionIdle,
@@ -375,7 +379,6 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     } catch (RuntimeException e) {
       LogWrapper.getInstance().info(e.getMessage(), e);
       result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-
     }
     if (xmlEntity.get() != null) {
       persistClusterConfiguration(result,
@@ -385,11 +388,9 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return result;
   }
 
-
-  public boolean verifyDistributedRegionMbean(Cache cache, String regionName) {
-    GemFireCacheImpl gemfireCache = (GemFireCacheImpl) cache;
+  public boolean verifyDistributedRegionMbean(InternalCache cache, String regionName) {
     int federationInterval =
-        gemfireCache.getInternalDistributedSystem().getConfig().getJmxManagerUpdateRate();
+        cache.getInternalDistributedSystem().getConfig().getJmxManagerUpdateRate();
     long timeEnd = System.currentTimeMillis() + federationInterval + 50;
 
     for (; System.currentTimeMillis() <= timeEnd;) {
@@ -479,7 +480,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     this.securityService.authorizeRegionManage(regionPath);
 
     try {
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getCache();
 
       if (groups != null) {
         validateGroups(cache, groups);
@@ -616,7 +617,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return result;
   }
 
-  private static boolean regionExists(Cache cache, String regionPath) {
+  private static boolean regionExists(InternalCache cache, String regionPath) {
     boolean regionFound = false;
     if (regionPath != null && !Region.SEPARATOR.equals(regionPath)) {
       ManagementService managementService = ManagementService.getExistingManagementService(cache);
@@ -633,7 +634,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return regionFound;
   }
 
-  private void validateRegionPathAndParent(Cache cache, String regionPath) {
+  private void validateRegionPathAndParent(InternalCache cache, String regionPath) {
     if (regionPath == null || "".equals(regionPath)) {
       throw new IllegalArgumentException(CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH);
     }
@@ -649,8 +650,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     }
   }
 
-
-  private void validateGroups(Cache cache, String[] groups) {
+  private void validateGroups(InternalCache cache, String[] groups) {
     if (groups != null && groups.length != 0) {
       Set<String> existingGroups = new HashSet<String>();
       Set<DistributedMember> members = CliUtil.getAllNormalMembers(cache);
@@ -669,7 +669,8 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     }
   }
 
-  private void validateRegionFunctionArgs(Cache cache, RegionFunctionArgs regionFunctionArgs) {
+  private void validateRegionFunctionArgs(InternalCache cache,
+      RegionFunctionArgs regionFunctionArgs) {
     if (regionFunctionArgs.getRegionPath() == null) {
       throw new IllegalArgumentException(CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH);
     }
@@ -751,16 +752,6 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
                 new Object[] {prRedundantCopies}));
         }
       }
-      // Validation for the following is not known yet
-      // if (partitionArgs.isSetPRTotalNumBuckets()) {
-      // int prTotalNumBuckets = partitionArgs.getPrTotalNumBuckets();
-      // }
-      // if (partitionArgs.isSetPRStartupRecoveryDelay()) {
-      // long prStartupRecoveryDelay = partitionArgs.getPrStartupRecoveryDelay();
-      // }
-      // if (partitionArgs.isSetPRRecoveryDelay()) {
-      // long prRecoveryDelay = partitionArgs.getPrRecoveryDelay();
-      // }
     }
 
     String keyConstraint = regionFunctionArgs.getKeyConstraint();
@@ -918,7 +909,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     }
   }
 
-  private boolean diskStoreExists(Cache cache, String diskStoreName) {
+  private boolean diskStoreExists(InternalCache cache, String diskStoreName) {
     ManagementService managementService = ManagementService.getExistingManagementService(cache);
     DistributedSystemMXBean dsMXBean = managementService.getDistributedSystemMXBean();
     Map<String, String[]> diskstore = dsMXBean.listMemberDiskstore();
@@ -935,9 +926,9 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return false;
   }
 
-  private static <K, V> FetchRegionAttributesFunctionResult<K, V> getRegionAttributes(Cache cache,
-      String regionPath) {
-    if (!isClusterwideSameConfig(cache, regionPath)) {
+  private static <K, V> FetchRegionAttributesFunctionResult<K, V> getRegionAttributes(
+      InternalCache cache, String regionPath) {
+    if (!isClusterWideSameConfig(cache, regionPath)) {
       throw new IllegalStateException(CliStrings.format(
           CliStrings.CREATE_REGION__MSG__USE_ATTRIBUTES_FORM_REGIONS_EXISTS_BUT_DIFFERENT_SCOPE_OR_DATAPOLICY_USE_DESCRIBE_REGION_FOR_0,
           regionPath));
@@ -990,7 +981,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return attributes;
   }
 
-  private static boolean isClusterwideSameConfig(Cache cache, String regionPath) {
+  private static boolean isClusterWideSameConfig(InternalCache cache, String regionPath) {
     ManagementService managementService = ManagementService.getExistingManagementService(cache);
 
     DistributedSystemMXBean dsMXBean = managementService.getDistributedSystemMXBean();
@@ -1053,7 +1044,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     AtomicReference<XmlEntity> xmlEntity = new AtomicReference<>();
     try {
       String message = "";
-      Cache cache = CacheFactory.getAnyInstance();
+      InternalCache cache = getCache();
       ManagementService managementService = ManagementService.getExistingManagementService(cache);
       String regionPathToUse = regionPath;
 
@@ -1076,7 +1067,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           CliUtil.executeFunction(RegionDestroyFunction.INSTANCE, regionPath, regionMembersList);
       List<CliFunctionResult> resultsList = (List<CliFunctionResult>) resultCollector.getResult();
       message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGION_0_1_DESTROYED,
-          new Object[] {regionPath, /* subRegionMessage */""});
+          new Object[] {regionPath, ""});
 
       // Only if there is an error is this set to false
       boolean isRegionDestroyed = true;
@@ -1121,7 +1112,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return result;
   }
 
-  private Set<DistributedMember> findMembersForRegion(Cache cache,
+  private Set<DistributedMember> findMembersForRegion(InternalCache cache,
       ManagementService managementService, String regionPath) {
     Set<DistributedMember> membersList = new HashSet<>();
     Set<String> regionMemberIds = new HashSet<>();
@@ -1181,7 +1172,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return membersList;
   }
 
-  private Set<DistributedMember> getMembersByIds(Cache cache, Set<String> memberIds) {
+  private Set<DistributedMember> getMembersByIds(InternalCache cache, Set<String> memberIds) {
     Set<DistributedMember> foundMembers = Collections.emptySet();
     if (memberIds != null && !memberIds.isEmpty()) {
       foundMembers = new HashSet<DistributedMember>();
@@ -1203,7 +1194,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       CliStrings.DESTROY_REGION})
   public boolean isRegionCommandAvailable() {
     boolean isAvailable = true; // always available on server
-    if (CliUtil.isGfshVM()) { // in gfsh check if connected //TODO - Abhishek: make this better
+    if (CliUtil.isGfshVM()) { // in gfsh check if connected //TODO: make this better
       isAvailable = getGfsh() != null && getGfsh().isConnectedAndReady();
     }
     return isAvailable;