You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2017/06/27 05:00:43 UTC

[28/46] geode git commit: GEODE-2919: Provide finer grained security

GEODE-2919: Provide finer grained security

 - For full details, see https://cwiki.apache.org/confluence/display/GEODE/Finer+grained+security


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

Branch: refs/heads/feature/GEM-1483
Commit: 451d12e83c2457c22535ae684995e973ba8d3dd2
Parents: 3ee585c
Author: Patrick Rhomberg <pr...@pivotal.io>
Authored: Wed Jun 21 10:51:32 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Fri Jun 23 15:46:01 2017 -0700

----------------------------------------------------------------------
 .../geode/management/CacheServerMXBean.java     | 122 ++---
 .../geode/management/DiskStoreMXBean.java       |  68 +--
 .../management/DistributedSystemMXBean.java     | 174 +++----
 .../geode/management/GatewayReceiverMXBean.java |  75 +--
 .../geode/management/GatewaySenderMXBean.java   |  88 ++--
 .../geode/management/LockServiceMXBean.java     |  20 +-
 .../apache/geode/management/MemberMXBean.java   | 263 +++++-----
 .../internal/beans/DistributedSystemMBean.java  |  21 +-
 .../management/internal/beans/MemberMBean.java  |   7 +-
 .../internal/beans/MemberMBeanBridge.java       |   4 +-
 .../internal/cli/commands/ClientCommands.java   |  90 ++--
 .../CreateAlterDestroyRegionCommands.java       |  43 +-
 .../internal/cli/commands/DataCommands.java     |  64 ++-
 .../internal/cli/commands/DeployCommands.java   |  34 +-
 .../cli/commands/DiskStoreCommands.java         | 222 ++++-----
 .../cli/commands/DurableClientCommands.java     |  90 ++--
 .../internal/cli/commands/FunctionCommands.java | 136 ++----
 .../internal/cli/commands/GfshCommand.java      |  20 +-
 .../internal/cli/commands/IndexCommands.java    | 138 +++---
 .../cli/commands/MiscellaneousCommands.java     | 258 +++++-----
 .../internal/cli/commands/PDXCommands.java      |  39 +-
 .../internal/cli/commands/QueueCommands.java    |  55 +--
 .../internal/cli/commands/RegionCommands.java   |  76 ++-
 .../internal/cli/commands/WanCommands.java      | 219 +++++----
 .../commands/lifecycle/StartPulseCommand.java   |  18 +-
 .../cli/functions/ContinuousQueryFunction.java  | 148 ++++++
 .../cli/functions/ContunuousQueryFunction.java  | 150 ------
 .../cli/functions/RegionFunctionArgs.java       |  26 +-
 .../cli/functions/UserFunctionExecution.java    |  42 +-
 .../internal/cli/i18n/CliStrings.java           |  44 +-
 .../internal/cli/remote/CommandProcessor.java   |  17 +-
 .../internal/security/MBeanServerWrapper.java   |  38 +-
 .../MemberMBeanAttributesDUnitTest.java         |  35 +-
 .../bean/stats/MemberLevelStatsJUnitTest.java   |   8 +-
 .../CreateAlterDestroyRegionCommandsTest.java   |  17 +-
 ...CacheServerMBeanAuthenticationJUnitTest.java |  20 +-
 .../CacheServerMBeanAuthorizationJUnitTest.java |  42 +-
 ...eServerMBeanWithShiroIniIntegrationTest.java |  15 +-
 .../security/CliCommandsSecurityTest.java       |  34 +-
 .../security/DataCommandsSecurityTest.java      |  13 +-
 .../security/DeployCommandsSecurityTest.java    |  28 +-
 .../DiskStoreMXBeanSecurityJUnitTest.java       |  98 ++--
 .../DistributedSystemMXBeanSecurityTest.java    |  77 +++
 .../GatewayReceiverMBeanSecurityTest.java       |  42 +-
 .../GatewaySenderMBeanSecurityTest.java         |  68 +--
 .../security/GfshCommandsPostProcessorTest.java |  28 +-
 .../security/GfshCommandsSecurityTest.java      |  76 +--
 .../LockServiceMBeanAuthorizationJUnitTest.java |  73 ++-
 .../security/MBeanSecurityJUnitTest.java        |  30 +-
 .../ManagerMBeanAuthorizationJUnitTest.java     |  30 +-
 .../security/MemberMBeanSecurityJUnitTest.java  |  51 +-
 .../internal/security/MultiUserDUnitTest.java   | 102 ++--
 .../security/ResourcePermissionTest.java        |  13 +-
 .../internal/security/TestCommand.java          | 157 +++---
 .../dunit/rules/LocatorServerStartupRule.java   |  19 +-
 .../test/dunit/rules/MemberStarterRule.java     |   6 +
 .../codeAnalysis/sanctionedSerializables.txt    |   4 +-
 .../cli/commands/ClientCommandsDUnitTest.java   | 478 ++++++++++---------
 .../internal/cli/LuceneIndexCommands.java       |  58 ++-
 .../lucene/internal/cli/LuceneIndexInfo.java    |  10 -
 .../functions/LuceneCreateIndexFunction.java    |  14 +-
 .../lucene/LuceneCommandsSecurityDUnitTest.java | 126 +++--
 .../GfshCommandsOverHttpSecurityTest.java       |   4 +-
 63 files changed, 2342 insertions(+), 2243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/CacheServerMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/CacheServerMXBean.java b/geode-core/src/main/java/org/apache/geode/management/CacheServerMXBean.java
index 3adbf01..d0159f0 100644
--- a/geode-core/src/main/java/org/apache/geode/management/CacheServerMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/CacheServerMXBean.java
@@ -18,12 +18,13 @@ import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
 
 /**
  * MBean that provides access to information and management functionality for a {@link CacheServer}.
  * 
  * <p>
- * The will be one CacheServermBean per {@link CacheServer} started in GemFire node.
+ * The will be one CacheServerMBean per {@link CacheServer} started in GemFire node.
  * 
  * <p>
  * ObjectName for this MBean is GemFire:service=CacheServer,port={0},type=Member,member={1}
@@ -60,48 +61,48 @@ public interface CacheServerMXBean {
   /**
    * Returns the port on which this CacheServer listens for clients.
    */
-  public int getPort();
+  int getPort();
 
   /**
    * Returns a string representing the IP address or host name that this CacheServer will listen on.
    */
-  public String getBindAddress();
+  String getBindAddress();
 
   /**
    * Returns the configured buffer size of the socket connection for this CacheServer.
    */
-  public int getSocketBufferSize();
+  int getSocketBufferSize();
 
   /**
    * Returns the maximum amount of time between client pings. This value is used to determine the
    * health of client's attached to the server.
    */
-  public int getMaximumTimeBetweenPings();
+  int getMaximumTimeBetweenPings();
 
   /**
    * Returns the maximum allowed client connections.
    */
-  public int getMaxConnections();
+  int getMaxConnections();
 
   /**
-   * Returns the maxium number of threads allowed in this CacheServer to service client requests.
+   * Returns the maximum number of threads allowed in this CacheServer to service client requests.
    */
-  public int getMaxThreads();
+  int getMaxThreads();
 
   /**
    * Returns the maximum number of messages that can be enqueued in a client-queue.
    */
-  public int getMaximumMessageCount();
+  int getMaximumMessageCount();
 
   /**
    * Returns the time (in seconds) after which a message in the client queue will expire.
    */
-  public int getMessageTimeToLive();
+  int getMessageTimeToLive();
 
   /**
    * Returns the frequency (in milliseconds) to poll the load probe on this CacheServer.
    */
-  public long getLoadPollInterval();
+  long getLoadPollInterval();
 
   /**
    * Returns the name or IP address to pass to the client as the location where the server is
@@ -111,164 +112,164 @@ public interface CacheServerMXBean {
    * and won’t be able to find the server. For this situation, you must supply the server’s
    * alternate hostname for the locator to pass to the client.
    */
-  public String getHostNameForClients();
+  String getHostNameForClients();
 
   /**
    * Returns the load probe for this CacheServer.
    */
-  public ServerLoadData fetchLoadProbe();
+  ServerLoadData fetchLoadProbe();
 
   /**
    * Returns whether or not this CacheServer is running.
    * 
    * @return True of the server is running, false otherwise.
    */
-  public boolean isRunning();
+  boolean isRunning();
 
   /**
    * Returns the capacity (in megabytes) of the client queue.
    */
-  public int getCapacity();
+  int getCapacity();
 
   /**
    * Returns the eviction policy that is executed when the capacity of the client queue is reached.
    */
-  public String getEvictionPolicy();
+  String getEvictionPolicy();
 
   /**
    * Returns the name of the disk store that is used for persistence.
    */
-  public String getDiskStoreName();
+  String getDiskStoreName();
 
   /**
    * Returns the number of sockets accepted and used for client to server messaging.
    */
-  public int getClientConnectionCount();
+  int getClientConnectionCount();
 
   /**
    * Returns the number of client virtual machines connected.
    */
-  public int getCurrentClients();
+  int getCurrentClients();
 
   /**
    * Returns the average get request latency.
    */
-  public long getGetRequestAvgLatency();
+  long getGetRequestAvgLatency();
 
   /**
    * Returns the average put request latency.
    */
-  public long getPutRequestAvgLatency();
+  long getPutRequestAvgLatency();
 
   /**
    * Returns the total number of client connections that timed out and were closed.
    */
-  public int getTotalConnectionsTimedOut();
+  int getTotalConnectionsTimedOut();
 
   /**
    * Returns the total number of client connection requests that failed.
    */
-  public int getTotalFailedConnectionAttempts();
+  int getTotalFailedConnectionAttempts();
 
   /**
    * Returns the current number of connections waiting for a thread to start processing their
    * message.
    */
-  public int getThreadQueueSize();
+  int getThreadQueueSize();
 
   /**
    * Returns the current number of threads handling a client connection.
    */
-  public int getConnectionThreads();
+  int getConnectionThreads();
 
   /**
    * Returns the load from client to server connections as reported by the load probe installed in
    * this server.
    */
-  public double getConnectionLoad();
+  double getConnectionLoad();
 
   /**
    * Returns the estimate of how much load is added for each new connection as reported by the load
    * probe installed in this server.
    */
-  public double getLoadPerConnection();
+  double getLoadPerConnection();
 
   /**
    * Returns the load from queues as reported by the load probe installed in this server.
    */
-  public double getQueueLoad();
+  double getQueueLoad();
 
   /**
    * Returns the estimate of how much load is added for each new queue as reported by the load probe
    * installed in this server.
    */
-  public double getLoadPerQueue();
+  double getLoadPerQueue();
 
 
   /**
    * Returns the rate of get requests.
    */
-  public float getGetRequestRate();
+  float getGetRequestRate();
 
   /**
    * Returns the rate of put requests.
    */
-  public float getPutRequestRate();
+  float getPutRequestRate();
 
   /**
    * Returns the total number of bytes sent to clients.
    */
-  public long getTotalSentBytes();
+  long getTotalSentBytes();
 
   /**
    * Returns the total number of bytes received from clients.
    */
-  public long getTotalReceivedBytes();
+  long getTotalReceivedBytes();
 
   /**
    * Returns the number of cache client notification requests.
    */
-  public int getNumClientNotificationRequests();
+  int getNumClientNotificationRequests();
 
   /**
    * Returns the average latency for processing client notifications.
    */
-  public long getClientNotificationAvgLatency();
+  long getClientNotificationAvgLatency();
 
   /**
    * Returns the rate of client notifications.
    */
-  public float getClientNotificationRate();
+  float getClientNotificationRate();
 
   /**
    * Returns the number of registered CQs.
    */
-  public long getRegisteredCQCount();
+  long getRegisteredCQCount();
 
   /**
    * Returns the number of active (currently executing) CQs.
    */
-  public long getActiveCQCount();
+  long getActiveCQCount();
 
   /**
    * Returns the rate of queries.
    */
-  public float getQueryRequestRate();
+  float getQueryRequestRate();
 
   /**
    * Returns the total number of indexes in use by the member.
    */
-  public int getIndexCount();
+  int getIndexCount();
 
   /**
    * Returns a list of names for all indexes.
    */
-  public String[] getIndexList();
+  String[] getIndexList();
 
   /**
    * Returns the total time spent updating indexes due to changes in the data.
    */
-  public long getTotalIndexMaintenanceTime();
+  long getTotalIndexMaintenanceTime();
 
   /**
    * Remove an index.
@@ -276,12 +277,12 @@ public interface CacheServerMXBean {
    * @param indexName Name of the index to be removed.
    */
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void removeIndex(String indexName) throws Exception;
+  void removeIndex(String indexName) throws Exception;
 
   /**
    * Returns a list of names for all registered CQs.
    */
-  public String[] getContinuousQueryList();
+  String[] getContinuousQueryList();
 
   /**
    * Execute an ad-hoc CQ on the server
@@ -293,7 +294,7 @@ public interface CacheServerMXBean {
    */
   @Deprecated
   @ResourceOperation(resource = Resource.DATA, operation = Operation.READ)
-  public void executeContinuousQuery(String queryName) throws Exception;
+  void executeContinuousQuery(String queryName) throws Exception;
 
   /**
    * Stop (pause) a CQ from executing
@@ -305,8 +306,9 @@ public interface CacheServerMXBean {
    *             methods to modify a CQ.
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void stopContinuousQuery(String queryName) throws Exception;
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.QUERY)
+  void stopContinuousQuery(String queryName) throws Exception;
 
   /**
    * Unregister all CQs from a region
@@ -317,8 +319,9 @@ public interface CacheServerMXBean {
    *             methods to modify a CQ.
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void closeAllContinuousQuery(String regionName) throws Exception;
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.QUERY)
+  void closeAllContinuousQuery(String regionName) throws Exception;
 
 
   /**
@@ -330,8 +333,9 @@ public interface CacheServerMXBean {
    *             methods to modify a CQ.
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void closeContinuousQuery(String queryName) throws Exception;
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.QUERY)
+  void closeContinuousQuery(String queryName) throws Exception;
 
 
   /**
@@ -339,7 +343,7 @@ public interface CacheServerMXBean {
    * 
    * @return A list of IDs or a length 0 array if no clients are registered.
    */
-  public String[] getClientIds() throws Exception;
+  String[] getClientIds() throws Exception;
 
   /**
    * Returns health and statistic information for the give client id. Some of the information (CPUs,
@@ -348,44 +352,44 @@ public interface CacheServerMXBean {
    * 
    * @param clientId ID of the client for which to retrieve information.
    */
-  public ClientHealthStatus showClientStats(String clientId) throws Exception;
+  ClientHealthStatus showClientStats(String clientId) throws Exception;
 
   /**
    * Returns the number of clients who have existing subscriptions.
    */
-  public int getNumSubscriptions();
+  int getNumSubscriptions();
 
   /**
    * Returns health and statistic information for all clients. Some of the information (CPUs,
    * NumOfCacheListenerCalls, NumOfGets,NumOfMisses, NumOfPuts,NumOfThreads, ProcessCpuTime) only
    * available for clients which have set a "StatisticsInterval".
    */
-  public ClientHealthStatus[] showAllClientStats() throws Exception;
+  ClientHealthStatus[] showAllClientStats() throws Exception;
 
   /**
    * Shows a list of client with their queue statistics. Client queue statistics shown in this
    * method are the following
    * 
-   * eventsEnqued,eventsRemoved , eventsConflated ,markerEventsConflated , eventsExpired,
+   * eventsEnqueued,eventsRemoved , eventsConflated ,markerEventsConflated , eventsExpired,
    * eventsRemovedByQrm , eventsTaken , numVoidRemovals
    * 
    * @return an array of ClientQueueDetail
    * @throws Exception
    */
-  public ClientQueueDetail[] showClientQueueDetails() throws Exception;
+  ClientQueueDetail[] showClientQueueDetails() throws Exception;
 
   /**
    * 
    * Shows queue statistics of the given client. Client queue statistics shown in this method are
    * the following
    * 
-   * eventsEnqued,eventsRemoved , eventsConflated ,markerEventsConflated , eventsExpired,
+   * eventsEnqueued,eventsRemoved , eventsConflated ,markerEventsConflated , eventsExpired,
    * eventsRemovedByQrm , eventsTaken , numVoidRemovals
    * 
    * @param clientId the ID of client which is returned by the attribute ClientIds
    * @return ClientQueueDetail
    * @throws Exception
    */
-  public ClientQueueDetail showClientQueueDetails(String clientId) throws Exception;
+  ClientQueueDetail showClientQueueDetails(String clientId) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/DiskStoreMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/DiskStoreMXBean.java b/geode-core/src/main/java/org/apache/geode/management/DiskStoreMXBean.java
index cca6272..6275df5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/DiskStoreMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/DiskStoreMXBean.java
@@ -18,6 +18,7 @@ import org.apache.geode.cache.DiskStore;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
 
 /**
  * MBean that provides access to information and management functionality for a {@link DiskStore}.
@@ -31,58 +32,58 @@ public interface DiskStoreMXBean {
   /**
    * Returns the name of the DiskStore.
    */
-  public String getName();
+  String getName();
 
   /**
    * Returns whether disk files are to be automatically compacted.
    * 
    * @return True if disk files are automatically compacted, false otherwise
    */
-  public boolean isAutoCompact();
+  boolean isAutoCompact();
 
   /**
    * Returns the threshold at which an op-log may be compacted. Until it reaches this threshold the
    * op-log will not be compacted. The threshold is a percentage in the range 0..100.
    */
-  public int getCompactionThreshold();
+  int getCompactionThreshold();
 
   /**
    * Returns whether manual compaction of disk files is allowed.
    * 
    * @return True if manual compaction is allowed, false otherwise.
    */
-  public boolean isForceCompactionAllowed();
+  boolean isForceCompactionAllowed();
 
   /**
    * Returns the maximum size (in megabytes) that a single op-log can grow to.
    */
-  public long getMaxOpLogSize();
+  long getMaxOpLogSize();
 
   /**
    * Returns the time (in milliseconds) that can elapse before unwritten data is saved to disk.
    */
-  public long getTimeInterval();
+  long getTimeInterval();
 
   /**
    * Returns the size of the write buffer that this DiskStore will use when writing data to disk.
    */
-  public int getWriteBufferSize();
+  int getWriteBufferSize();
 
   /**
    * Returns the path of the directories to which the region's data will be written.
    */
-  public String[] getDiskDirectories();
+  String[] getDiskDirectories();
 
   /**
    * Returns the maximum number of operations that can be asynchronously queued for saving to disk.
    * When this limit is reached operations will block until they can be put in the queue.
    */
-  public int getQueueSize();
+  int getQueueSize();
 
   /**
    * Returns the total number of bytes of space this DiskStore has used.
    */
-  public long getTotalBytesOnDisk();
+  long getTotalBytesOnDisk();
 
   /**
    * Returns the average latency of disk reads in nanoseconds Its the average latency required to
@@ -92,7 +93,7 @@ public interface DiskStoreMXBean {
    * So this rate won't match the number of bytes put in all regions.This is rate of actual bytes
    * system is persisting.
    */
-  public float getDiskReadsRate();
+  float getDiskReadsRate();
 
   /**
    * Returns the average latency of disk writes in nanoseconds. Its the average latency required to
@@ -102,44 +103,44 @@ public interface DiskStoreMXBean {
    * So this rate won't match the number of bytes put in all regions. This is rate of actual bytes
    * system is persisting.
    */
-  public float getDiskWritesRate();
+  float getDiskWritesRate();
 
   /**
    * Returns the disk reads average latency in nanoseconds. It depicts average time needed to read
    * one byte of data from disk.
    */
-  public long getDiskReadsAvgLatency();
+  long getDiskReadsAvgLatency();
 
   /**
    * Returns the disk writes average latency in nanoseconds. It depicts average time needed to write
    * one byte of data to disk.
    */
-  public long getDiskWritesAvgLatency();
+  long getDiskWritesAvgLatency();
 
   /**
    * Returns the flush time average latency.
    */
-  public long getFlushTimeAvgLatency();
+  long getFlushTimeAvgLatency();
 
   /**
    * Returns the number of entries in the asynchronous queue waiting to be written to disk.
    */
-  public int getTotalQueueSize();
+  int getTotalQueueSize();
 
   /**
    * Returns the number of backups currently in progress on this DiskStore.
    */
-  public int getTotalBackupInProgress();
+  int getTotalBackupInProgress();
 
   /**
    * Returns the number of backups of this DiskStore that have been completed.
    */
-  public int getTotalBackupCompleted();
+  int getTotalBackupCompleted();
 
   /**
    * Returns the number of persistent regions currently being recovered from disk.
    */
-  public int getTotalRecoveriesInProgress();
+  int getTotalRecoveriesInProgress();
 
   /**
    * Requests the DiskStore to start writing to a new op-log. The old oplog will be asynchronously
@@ -149,8 +150,9 @@ public interface DiskStoreMXBean {
    * compaction is true then the application will wait for the other op-logs to be compacted and
    * additional space is available.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void forceRoll();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  void forceRoll();
 
   /**
    * Requests the DiskStore to start compacting. The compaction is done even if automatic compaction
@@ -161,15 +163,17 @@ public interface DiskStoreMXBean {
    * @return True if one or more op-logs were compacted or false to indicate that no op-logs were
    *         ready to be compacted or that a compaction was already in progress.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public boolean forceCompaction();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  boolean forceCompaction();
 
   /**
    * Causes any data that is currently in the asynchronous queue to be written to disk. Does not
    * return until the flush is complete.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void flush();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  void flush();
 
   /**
    * Returns the warning threshold for disk usage as a percentage of the total disk volume.
@@ -177,7 +181,7 @@ public interface DiskStoreMXBean {
    * @return the warning percent
    * @since GemFire 8.0
    */
-  public float getDiskUsageWarningPercentage();
+  float getDiskUsageWarningPercentage();
 
   /**
    * Returns the critical threshold for disk usage as a percentage of the total disk volume.
@@ -185,21 +189,23 @@ public interface DiskStoreMXBean {
    * @return the critical percent
    * @since GemFire 8.0
    */
-  public float getDiskUsageCriticalPercentage();
+  float getDiskUsageCriticalPercentage();
 
   /**
    * Sets the value of the disk usage warning percentage.
    * 
    * @param warningPercent the warning percent
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void setDiskUsageWarningPercentage(float warningPercent);
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  void setDiskUsageWarningPercentage(float warningPercent);
 
   /**
    * Sets the value of the disk usage critical percentage.
    * 
    * @param criticalPercent the critical percent
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void setDiskUsageCriticalPercentage(float criticalPercent);
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  void setDiskUsageCriticalPercentage(float criticalPercent);
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/DistributedSystemMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/DistributedSystemMXBean.java b/geode-core/src/main/java/org/apache/geode/management/DistributedSystemMXBean.java
index f6f701e..3d52115 100644
--- a/geode-core/src/main/java/org/apache/geode/management/DistributedSystemMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/DistributedSystemMXBean.java
@@ -15,6 +15,7 @@
 package org.apache.geode.management;
 
 import java.util.Map;
+
 import javax.management.ObjectName;
 
 import org.apache.geode.cache.DiskStore;
@@ -24,6 +25,7 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
 
 /**
  * MBean that provides access to information and management operations for a
@@ -76,22 +78,22 @@ import org.apache.geode.security.ResourcePermission.Resource;
 public interface DistributedSystemMXBean {
 
   /**
-   * Returns the ID of thie DistributedSystem. allow anyone to access this method
+   * Returns the ID of the DistributedSystem. allow anyone to access this method
    *
    * @return The DistributedSystem ID or -1 if not set.
    */
   @ResourceOperation()
-  public int getDistributedSystemId();
+  int getDistributedSystemId();
 
   /**
    * Returns the number of members in the distributed system.
    */
-  public int getMemberCount();
+  int getMemberCount();
 
   /**
    * Returns a list of names for all members.
    */
-  public String[] listMembers();
+  String[] listMembers();
 
   /**
    * Returns a list of names for locator members.
@@ -101,49 +103,49 @@ public interface DistributedSystemMXBean {
    * @return a list of names for locator members.
    */
 
-  public String[] listLocatorMembers(boolean onlyStandAloneLocators);
+  String[] listLocatorMembers(boolean onlyStandAloneLocators);
 
   /**
    * Returns a list of names for all groups.
    */
-  public String[] listGroups();
+  String[] listGroups();
 
   /**
    * Returns the number of locators in the distributed system.
    */
-  public int getLocatorCount();
+  int getLocatorCount();
 
   /**
    * Returns a list of IDs for all locators.
    */
-  public String[] listLocators(); // TODO - Abhishek Should be renamed to
-                                  // listLocatorDiscoveryConfigurations? Do we need something for
-                                  // mcast too?
+  String[] listLocators(); // TODO - Abhishek Should be renamed to
+                           // listLocatorDiscoveryConfigurations? Do we need something for
+                           // mcast too?
 
   /**
    * Returns the number of disks stores in the distributed system.
    */
-  public int getSystemDiskStoreCount();
+  int getSystemDiskStoreCount();
 
   /**
    * Returns a map of all {@link DistributedMember}s and their {@link DiskStore}s.
    */
-  public Map<String, String[]> listMemberDiskstore();
+  Map<String, String[]> listMemberDiskstore();
 
   /**
    * Returns a list of IDs for all gateway senders.
    */
-  public String[] listGatewaySenders();
+  String[] listGatewaySenders();
 
   /**
    * Returns a list of IDs for all gateway receivers.
    */
-  public String[] listGatewayReceivers();
+  String[] listGatewayReceivers();
 
   /**
    * Returns the minimum level set for alerts to be delivered to listeners.
    */
-  public String getAlertLevel();
+  String getAlertLevel();
 
   /**
    * Sets the minimum level for alerts to be delivered to listeners.
@@ -152,74 +154,74 @@ public interface DistributedSystemMXBean {
    *        SEVERE or NONE.
    */
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.WRITE)
-  public void changeAlertLevel(String alertLevel) throws Exception;
+  void changeAlertLevel(String alertLevel) throws Exception;
 
   /**
    * Returns the total available heap (in megabytes) across all distributed members.
    */
-  public long getTotalHeapSize();
+  long getTotalHeapSize();
 
   /**
    * Returns the total number of entries in all regions.
    */
-  public long getTotalRegionEntryCount();
+  long getTotalRegionEntryCount();
 
   /**
    * Returns the number of {@link Region}s.
    */
 
-  public int getTotalRegionCount();
+  int getTotalRegionCount();
 
   /**
    * Returns the number of times that a cache miss occurred for all regions.
    */
-  public int getTotalMissCount();
+  int getTotalMissCount();
 
   /**
    * Returns the number of times that a hit occurred for all regions.
    */
-  public int getTotalHitCount();
+  int getTotalHitCount();
 
   /**
    * Returns the number of connected clients.
    */
-  public int getNumClients();
+  int getNumClients();
 
 
   /**
    * Returns the average number of disk reads per second across all distributed members.
    */
-  public float getDiskReadsRate();
+  float getDiskReadsRate();
 
   /**
    * Returns the average number of disk writes per second across all distributed members.
    */
-  public float getDiskWritesRate();
+  float getDiskWritesRate();
 
   /**
    * Returns the average disk flush latency time.
    */
-  public long getDiskFlushAvgLatency();
+  long getDiskFlushAvgLatency();
 
   /**
    * Returns the number of backups currently in progress for all disk stores.
    */
-  public int getTotalBackupInProgress();
+  int getTotalBackupInProgress();
 
   /**
    * Returns the number of initial images in progress.
    */
-  public int getNumInitialImagesInProgress();
+  int getNumInitialImagesInProgress();
 
   /**
    * Returns the number of active (currently executing) CQs for all cache servers.
    */
-  public long getActiveCQCount();
+  long getActiveCQCount();
 
   /**
    * Returns the average number of queries per second across all distributed members.
    */
-  public float getQueryRequestRate();
+  float getQueryRequestRate();
 
   /**
    * Performs a backup on all members.
@@ -229,8 +231,7 @@ public interface DistributedSystemMXBean {
    * @return The results of the backup request.
    */
   @ResourceOperation(resource = Resource.DATA, operation = Operation.READ)
-  public DiskBackupStatus backupAllMembers(String targetDirPath, String baselineDirPath)
-      throws Exception;
+  DiskBackupStatus backupAllMembers(String targetDirPath, String baselineDirPath) throws Exception;
 
   /**
    * Returns the configuration information for a distributed member.
@@ -239,7 +240,7 @@ public interface DistributedSystemMXBean {
    * @return The configuration information for a member.
    * @throws Exception for an invalid member ID.
    */
-  public GemFireProperties fetchMemberConfiguration(String member) throws Exception;
+  GemFireProperties fetchMemberConfiguration(String member) throws Exception;
 
   /**
    * Returns the total time (in seconds) since a distributed member was started.
@@ -248,21 +249,21 @@ public interface DistributedSystemMXBean {
    * @return The total time (in seconds) since a member was started.
    * @throws Exception for an invalid member ID.
    */
-  public long fetchMemberUpTime(String member) throws Exception;
+  long fetchMemberUpTime(String member) throws Exception;
 
   /**
    * Returns a list of names for all cache servers which are able to serve requests from GemFire
    * clients.
    * 
    */
-  public String[] listCacheServers();
+  String[] listCacheServers();
 
 
   /**
    * Returns a list of names for all servers where server means any long-running GemFire process
    * that was started with "start server" command from GFSH.
    */
-  public String[] listServers();
+  String[] listServers();
 
   /**
    * Returns JVM metrics for a distributed member.
@@ -270,7 +271,7 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @throws Exception for an invalid member ID.
    */
-  public JVMMetrics showJVMMetrics(String member) throws Exception;
+  JVMMetrics showJVMMetrics(String member) throws Exception;
 
   /**
    * Returns operating system metrics for a distributed member.
@@ -278,7 +279,7 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @throws Exception for an invalid member ID.
    */
-  public OSMetrics showOSMetrics(String member) throws Exception;
+  OSMetrics showOSMetrics(String member) throws Exception;
 
   /**
    * Returns network metrics for a distributed member.
@@ -286,7 +287,7 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @throws Exception for an invalid member ID.
    */
-  public NetworkMetrics showNetworkMetric(String member) throws Exception;
+  NetworkMetrics showNetworkMetric(String member) throws Exception;
 
   /**
    * Returns disk metrics for a distributed member.
@@ -294,7 +295,7 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @throws Exception for an invalid member ID.
    */
-  public DiskMetrics showDiskMetrics(String member) throws Exception;
+  DiskMetrics showDiskMetrics(String member) throws Exception;
 
   /**
    * Shuts down all members of a distributed system except for the managing member.
@@ -302,17 +303,17 @@ public interface DistributedSystemMXBean {
    * @return List of names of all distributed members that were shutdown.
    */
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
-  public String[] shutDownAllMembers() throws Exception;
+  String[] shutDownAllMembers() throws Exception;
 
   /**
    * Returns a list of names for all regions.
    */
-  public String[] listRegions();
+  String[] listRegions();
 
   /**
    * Returns a list of full paths for all regions.
    */
-  public String[] listAllRegionPaths();
+  String[] listAllRegionPaths();
 
   /**
    * Removes a disk store from the distributed system.
@@ -320,40 +321,41 @@ public interface DistributedSystemMXBean {
    * @param diskStoreId UUID of the disk store to remove
    * @return True if the request is successful, false otherwise.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public boolean revokeMissingDiskStores(String diskStoreId) throws Exception;
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  boolean revokeMissingDiskStores(String diskStoreId);
 
   /**
    * Returns a list of details for disk stores which have been determined to be unavailable during
    * the recovery of region.
    */
-  public PersistentMemberDetails[] listMissingDiskStores();
+  PersistentMemberDetails[] listMissingDiskStores();
 
   /**
    * Returns the object name for a {@link MemberMXBean} used to access this distributed member.
    * allow anyone to access this method
    */
   @ResourceOperation()
-  public ObjectName getMemberObjectName();
+  ObjectName getMemberObjectName();
 
   /**
    * Returns the object name for a {@link ManagerMXBean} used to access the management service
    * running on this distributed member.
    */
-  public ObjectName getManagerObjectName();
+  ObjectName getManagerObjectName();
 
   /**
    * Returns a list of object names for the {@link MemberMXBean}s used to access all distributed
    * members.
    */
-  public ObjectName[] listMemberObjectNames();
+  ObjectName[] listMemberObjectNames();
 
   /**
    * Returns the object name for a {@link MemberMXBean} used to access a distributed member..
    *
    * @param member Name or ID of the member.
    */
-  public ObjectName fetchMemberObjectName(String member) throws Exception;
+  ObjectName fetchMemberObjectName(String member) throws Exception;
 
   /**
    * Returns a list of object names for the {@link RegionMXBean}s used to access all regions on a
@@ -361,7 +363,7 @@ public interface DistributedSystemMXBean {
    *
    * @param memberMBeanName ObjectName of the member.
    */
-  public ObjectName[] fetchRegionObjectNames(ObjectName memberMBeanName) throws Exception;
+  ObjectName[] fetchRegionObjectNames(ObjectName memberMBeanName) throws Exception;
 
   /**
    * Returns a list of object names for the {@link DistributedRegionMXBean}s used to access all
@@ -369,7 +371,7 @@ public interface DistributedSystemMXBean {
    *
    * @return An array of object names or an empty array if no distributed regions are found.
    */
-  public ObjectName[] listDistributedRegionObjectNames();
+  ObjectName[] listDistributedRegionObjectNames();
 
   /**
    * Returns the object name for a {@link DistributedRegionMXBean} used to access a distributed
@@ -377,7 +379,7 @@ public interface DistributedSystemMXBean {
    *
    * @param regionPath Full path of the region.
    */
-  public ObjectName fetchDistributedRegionObjectName(String regionPath) throws Exception;
+  ObjectName fetchDistributedRegionObjectName(String regionPath) throws Exception;
 
   /**
    * Returns the object name for a {@link RegionMXBean} used to access a region.
@@ -385,7 +387,7 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @param regionPath Full path of the region.
    */
-  public ObjectName fetchRegionObjectName(String member, String regionPath) throws Exception;
+  ObjectName fetchRegionObjectName(String member, String regionPath) throws Exception;
 
   /**
    * Returns the object name for a {@link GatewaySenderMXBean} used to access a gateway sender.
@@ -393,14 +395,14 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @param senderId ID of a gateway sender.
    */
-  public ObjectName fetchGatewaySenderObjectName(String member, String senderId) throws Exception;
+  ObjectName fetchGatewaySenderObjectName(String member, String senderId) throws Exception;
 
   /**
    * Returns the object name for a {@link GatewayReceiverMXBean} used to access a gateway receiver.
    *
    * @param member Name or ID of the member.
    */
-  public ObjectName fetchGatewayReceiverObjectName(String member) throws Exception;
+  ObjectName fetchGatewayReceiverObjectName(String member) throws Exception;
 
   /**
    * Returns a list of object names for the {@link GatewaySenderMXBean}s used to access all gateway
@@ -408,7 +410,7 @@ public interface DistributedSystemMXBean {
    *
    * @return An array of object names or an empty array if no gateway senders are found.
    */
-  public ObjectName[] listGatewaySenderObjectNames();
+  ObjectName[] listGatewaySenderObjectNames();
 
   /**
    * Returns a list of object names for the {@link GatewaySenderMXBean}s used to access all gateway
@@ -416,7 +418,7 @@ public interface DistributedSystemMXBean {
    *
    * @param member Name or ID of the member.
    */
-  public ObjectName[] listGatewaySenderObjectNames(String member) throws Exception;
+  ObjectName[] listGatewaySenderObjectNames(String member) throws Exception;
 
   /**
    * Returns a list of object names for the {@link GatewayReceiverMXBean}s used to access all
@@ -424,7 +426,7 @@ public interface DistributedSystemMXBean {
    *
    * @return An array of object names or an empty array if no gateway receivers are found.
    */
-  public ObjectName[] listGatewayReceiverObjectNames();
+  ObjectName[] listGatewayReceiverObjectNames();
 
   /**
    * Returns the object name for a {@link DistributedLockServiceMXBean} used to access a distributed
@@ -432,7 +434,7 @@ public interface DistributedSystemMXBean {
    *
    * @param lockServiceName Name of the lock service.
    */
-  public ObjectName fetchDistributedLockServiceObjectName(String lockServiceName) throws Exception;
+  ObjectName fetchDistributedLockServiceObjectName(String lockServiceName) throws Exception;
 
   /**
    * Returns the object name for a {@link LockServiceMXBean} used to access a lock service.
@@ -440,7 +442,7 @@ public interface DistributedSystemMXBean {
    * @param member Name or Id of the member.
    * @param lockService Name of the lock service.
    */
-  public ObjectName fetchLockServiceObjectName(String member, String lockService) throws Exception;
+  ObjectName fetchLockServiceObjectName(String member, String lockService) throws Exception;
 
   /**
    * Returns object name of a {@link DiskStoreMXBean} for a given name and member
@@ -450,7 +452,7 @@ public interface DistributedSystemMXBean {
    * @return a ObjectName
    * @throws Exception
    */
-  public ObjectName fetchDiskStoreObjectName(String member, String diskStoreName) throws Exception;
+  ObjectName fetchDiskStoreObjectName(String member, String diskStoreName) throws Exception;
 
   /**
    * Returns the object name for a {@link CacheServerMXBean} used to access a cache server.
@@ -458,70 +460,70 @@ public interface DistributedSystemMXBean {
    * @param member Name or ID of the member.
    * @param port Port of the server.
    */
-  public ObjectName fetchCacheServerObjectName(String member, int port) throws Exception;
+  ObjectName fetchCacheServerObjectName(String member, int port) throws Exception;
 
   /**
    * Returns a list of object names for the {@link CacheServerMXBean}s used to access all cache
    * servers.
    */
-  public ObjectName[] listCacheServerObjectNames();
+  ObjectName[] listCacheServerObjectNames();
 
   /**
    * Returns the number of map-reduce jobs currently running on all members in the distributed
    * system.
    */
-  public int getNumRunningFunctions();
+  int getNumRunningFunctions();
 
   /**
    * Returns the number of CQs registers on all members.
    */
-  public long getRegisteredCQCount();
+  long getRegisteredCQCount();
 
   /**
    * Returns the number of bytes used on all disks.
    */
-  public long getTotalDiskUsage();
+  long getTotalDiskUsage();
 
   /**
    * Returns the total heap used on all members.
    */
-  public long getUsedHeapSize();
+  long getUsedHeapSize();
 
   /**
    * Returns the average number of reads per second for all members.
    */
-  public float getAverageReads();
+  float getAverageReads();
 
   /**
    * Returns the average writes per second, including both put and putAll operations, for all
    * members.
    */
-  public float getAverageWrites();
+  float getAverageWrites();
 
   /**
    * Returns the number of subscriptions for all members.
    */
-  public int getNumSubscriptions();
+  int getNumSubscriptions();
 
 
   /**
    * Returns the number of garbage collection operations for all members.
    */
-  public long getGarbageCollectionCount();
+  long getGarbageCollectionCount();
 
   /**
    * Returns a map of remote distributed system IDs and the current connection status for each.
    */
-  public Map<String, Boolean> viewRemoteClusterStatus();
+  Map<String, Boolean> viewRemoteClusterStatus();
 
   /**
    * Returns the number JVM pauses (which may or may not include full garbage collection pauses)
    * detected by GemFire.
    */
-  public long getJVMPauses();
+  long getJVMPauses();
 
   /**
-   * This API is used to query data from GemFire system. This returns a JSON formated String having
+   * This API is used to query data from GemFire system. This returns a JSON formatted String having
    * data and it's type. Type and value of data makes an array , type preceding the value.
    * 
    * e.g. {"result":[["java.lang.String","v"],["java.lang.String","b"]]}
@@ -548,10 +550,10 @@ public interface DistributedSystemMXBean {
    *        mandatory to give this input barring join queries on PR. If member list is not provided
    *        query will be for the whole cluster.
    * @param limit result set limit. If not set or 0 is passed default limit of 1000 will be set.
-   * @return a JSON formated string containing data and its type
+   * @return a JSON formatted string containing data and its type
    */
   @ResourceOperation(resource = Resource.DATA, operation = Operation.READ)
-  public String queryData(String queryString, String members, int limit) throws Exception;
+  String queryData(String queryString, String members, int limit) throws Exception;
 
   /**
    * 
@@ -575,7 +577,7 @@ public interface DistributedSystemMXBean {
    * @return a byte[] which is a compressed JSON string.
    */
   @ResourceOperation(resource = Resource.DATA, operation = Operation.READ)
-  public byte[] queryDataForCompressedResult(String queryString, String members, int limit)
+  byte[] queryDataForCompressedResult(String queryString, String members, int limit)
       throws Exception;
 
 
@@ -583,31 +585,33 @@ public interface DistributedSystemMXBean {
    * Returns the number of committed transactions across all members. It gives point in time value
    * i.e. Number of tx committed at the time of reading this value
    */
-  public int getTransactionCommitted();
+  int getTransactionCommitted();
 
   /**
    * Returns the number of transactions that were rolled back across all members. It gives point in
    * time value i.e. Number of tx rolled back at the time of reading this value
    */
-  public int getTransactionRolledBack();
+  int getTransactionRolledBack();
 
   /**
    * Number of rows DistributedSystemMXBean.queryData() operation will return. By default it will be
    * 1000. User can modify this to control number of rows to be shown on Pulse, as Pulse DataBrowser
    * internally uses DistributedSystemMXBean.queryData()
    */
-  public int getQueryResultSetLimit();
+  int getQueryResultSetLimit();
 
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void setQueryResultSetLimit(int queryResultSetLimit);
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.QUERY)
+  void setQueryResultSetLimit(int queryResultSetLimit);
 
   /**
    * Number of elements in a collection to be shown in queryData operation if query results contain
    * collections like Map, List etc.
    * 
    */
-  public int getQueryCollectionsDepth();
+  int getQueryCollectionsDepth();
 
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void setQueryCollectionsDepth(int queryCollectionsDepth);
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.QUERY)
+  void setQueryCollectionsDepth(int queryCollectionsDepth);
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/GatewayReceiverMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/GatewayReceiverMXBean.java b/geode-core/src/main/java/org/apache/geode/management/GatewayReceiverMXBean.java
index 749a743..556bd6f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/GatewayReceiverMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/GatewayReceiverMXBean.java
@@ -18,6 +18,7 @@ import org.apache.geode.cache.wan.GatewayReceiver;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
 
 /**
  * MBean that provides access to information and management functionality for a
@@ -32,180 +33,182 @@ public interface GatewayReceiverMXBean {
   /**
    * Returns the port the receiver is listening on.
    */
-  public int getPort();
+  int getPort();
 
   /**
    * Returns the configured buffer size of the socket connection.
    */
-  public int getSocketBufferSize();
+  int getSocketBufferSize();
 
   /**
    * Returns the bind address on the host.
    */
-  public String getBindAddress();
+  String getBindAddress();
 
   /**
    * Returns the maximum amount of time between client pings.
    */
-  public int getMaximumTimeBetweenPings();
+  int getMaximumTimeBetweenPings();
 
   /**
    * Returns whether the receiver is in running state.
    * 
    * @return True if the receiver is in a running state, false otherwise.
    */
-  public boolean isRunning();
+  boolean isRunning();
 
   /**
    * Returns the instantaneous rate of events received.
    */
-  public float getEventsReceivedRate();
+  float getEventsReceivedRate();
 
   /**
    * Returns the rate of create requests received.
    */
-  public float getCreateRequestsRate();
+  float getCreateRequestsRate();
 
   /**
    * Returns the rate of update requests received.
    */
-  public float getUpdateRequestsRate();
+  float getUpdateRequestsRate();
 
   /**
    * Returns the rate of destroy requests received.
    */
-  public float getDestroyRequestsRate();
+  float getDestroyRequestsRate();
 
   /**
    * Returns the number of duplicate batches which have been received.
    */
-  public int getDuplicateBatchesReceived();
+  int getDuplicateBatchesReceived();
 
   /**
    * Returns the number of batches which have been received out of order.
    */
-  public int getOutoforderBatchesReceived();
+  int getOutoforderBatchesReceived();
 
   /**
    * Starts the gateway receiver.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void start() throws Exception;
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void start() throws Exception;
 
   /**
    * Stops the gateway receiver.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void stop() throws Exception;
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void stop() throws Exception;
 
   /**
    * Returns the configured start port.
    */
-  public int getStartPort();
+  int getStartPort();
 
   /**
    * Returns the configured end port.
    */
-  public int getEndPort();
+  int getEndPort();
 
   /**
    * Returns a list of names for the transport filters in use.
    */
-  public String[] getGatewayTransportFilters();
+  String[] getGatewayTransportFilters();
 
   /**
    * Returns the number of sockets accepted and used for client to server messaging.
    */
-  public int getClientConnectionCount();
+  int getClientConnectionCount();
 
   /**
    * Returns the number of client virtual machines connected and acting as a gateway.
    */
-  public int getNumGateways();
+  int getNumGateways();
 
   /**
    * Returns the average get request latency.
    */
-  public long getGetRequestAvgLatency();
+  long getGetRequestAvgLatency();
 
   /**
    * Returns the average put request latency.
    */
-  public long getPutRequestAvgLatency();
+  long getPutRequestAvgLatency();
 
   /**
    * Returns the total number of client connections that timed out and were closed.
    */
-  public int getTotalConnectionsTimedOut();
+  int getTotalConnectionsTimedOut();
 
   /**
    * Returns the total number of client connection requests that failed.
    */
-  public int getTotalFailedConnectionAttempts();
+  int getTotalFailedConnectionAttempts();
 
   /**
    * Returns the current number of connections waiting for a thread to start processing their
    * message.
    */
-  public int getThreadQueueSize();
+  int getThreadQueueSize();
 
   /**
    * Returns the current number of threads handling a client connection.
    */
-  public int getConnectionThreads();
+  int getConnectionThreads();
 
   /**
    * Returns the load from client to server connections as reported by the load probe installed in
    * this server.
    */
-  public double getConnectionLoad();
+  double getConnectionLoad();
 
   /**
    * Returns the estimate of how much load is added for each new connection as reported by the load
    * probe installed in this server.
    */
-  public double getLoadPerConnection();
+  double getLoadPerConnection();
 
   /**
    * Returns the load from queues as reported by the load probe installed in this server.
    */
-  public double getQueueLoad();
+  double getQueueLoad();
 
   /**
    * Returns the estimate of how much load is added for each new queue as reported by the load probe
    * installed in this server.
    */
-  public double getLoadPerQueue();
+  double getLoadPerQueue();
 
   /**
    * Returns the rate of get requests.
    */
-  public float getGetRequestRate();
+  float getGetRequestRate();
 
   /**
    * Returns the rate of put requests.
    */
-  public float getPutRequestRate();
+  float getPutRequestRate();
 
   /**
    * Returns the total number of bytes sent to clients.
    */
-  public long getTotalSentBytes();
+  long getTotalSentBytes();
 
   /**
    * Returns the total number of bytes received from clients.
    */
-  public long getTotalReceivedBytes();
+  long getTotalReceivedBytes();
 
   /**
    * Returns a list of the host and port information for gateway senders connected to this gateway
    * receiver.
    */
-  public String[] getConnectedGatewaySenders();
+  String[] getConnectedGatewaySenders();
 
   /**
    * Returns the average batch processing time (in milliseconds).
    */
-  public long getAverageBatchProcessingTime();
+  long getAverageBatchProcessingTime();
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java b/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java
index 718a9be..f8cad87 100644
--- a/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/GatewaySenderMXBean.java
@@ -18,6 +18,7 @@ import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
 
 /**
  * MBean that provides access to information and management functionality for a
@@ -32,51 +33,51 @@ public interface GatewaySenderMXBean {
   /**
    * Returns the ID of the GatewaySender.
    */
-  public String getSenderId();
+  String getSenderId();
 
   /**
    * Returns the id of the remote <code>GatewayReceiver</code>'s DistributedSystem.
    */
-  public int getRemoteDSId();
+  int getRemoteDSId();
 
   /**
    * Returns the configured buffer size of the socket connection between this GatewaySender and its
    * receiving <code>GatewayReceiver</code>.
    */
-  public int getSocketBufferSize();
+  int getSocketBufferSize();
 
   /**
    * Returns the amount of time (in milliseconds) that a socket read between a sending GatewaySender
    * and its receiving <code>GatewayReceiver</code> is allowed to block.
    */
-  public long getSocketReadTimeout();
+  long getSocketReadTimeout();
 
   /**
    * Returns the name of the disk store that is used for persistence.
    */
-  public String getOverflowDiskStoreName();
+  String getOverflowDiskStoreName();
 
   /**
    * Returns the maximum memory after which the data needs to be overflowed to disk.
    */
-  public int getMaximumQueueMemory();
+  int getMaximumQueueMemory();
 
   /**
    * Returns the size of a batch that gets delivered by the GatewaySender.
    */
-  public int getBatchSize();
+  int getBatchSize();
 
   /**
    * Returns the interval between transmissions by the GatewaySender.
    */
-  public long getBatchTimeInterval();
+  long getBatchTimeInterval();
 
   /**
    * Returns whether batch conflation for the GatewaySender's queue is enabled
    * 
    * @return True if batch conflation is enabled, false otherwise.
    */
-  public boolean isBatchConflationEnabled();
+  boolean isBatchConflationEnabled();
 
   /**
    * Returns whether the GatewaySender is configured to be persistent or non-persistent.
@@ -84,162 +85,167 @@ public interface GatewaySenderMXBean {
    * @return True if the sender is persistent, false otherwise.
    */
 
-  public boolean isPersistenceEnabled();
+  boolean isPersistenceEnabled();
 
   /**
    * Returns the alert threshold for entries in a GatewaySender's queue.The default value is 0
    * milliseconds in which case no alert will be logged if events are delayed in Queue.
    */
-  public int getAlertThreshold();
+  int getAlertThreshold();
 
   /**
    * Returns a list of <code>GatewayEventFilter</code>s added to this GatewaySender.
    */
-  public String[] getGatewayEventFilters();
+  String[] getGatewayEventFilters();
 
   /**
    * Returns a list of <code>GatewayTransportFilter</code>s added to this GatewaySender.
    */
-  public String[] getGatewayTransportFilters();
+  String[] getGatewayTransportFilters();
 
   /**
    * Returns whether the GatewaySender is configured for manual start.
    * 
    * @return True if the GatewaySender is configured for manual start, false otherwise.
    */
-  public boolean isManualStart();
+  boolean isManualStart();
 
   /**
    * Returns whether or not this GatewaySender is running.
    * 
    * @return True if the GatewaySender is running, false otherwise.
    */
-  public boolean isRunning();
+  boolean isRunning();
 
   /**
    * Returns whether or not this GatewaySender is paused.
    * 
    * @return True of the GatewaySender is paused, false otherwise.
    */
-  public boolean isPaused();
+  boolean isPaused();
 
   /**
    * Returns the rate of events received per second by this Sender if it's a serial-wan.
    */
-  public float getEventsReceivedRate();
+  float getEventsReceivedRate();
 
   /**
    * Returns the rate of events being queued.
    */
-  public float getEventsQueuedRate();
+  float getEventsQueuedRate();
 
   /**
    * Returns the current size of the event queue.
    */
-  public int getEventQueueSize();
+  int getEventQueueSize();
 
   /**
    * Returns the number of events received, but not added to the event queue, because the queue
    * already contains an event with the same key.
    */
-  public int getTotalEventsConflated();
+  int getTotalEventsConflated();
 
 
   /**
    * Returns the average number of batches sent per second.
    */
-  public float getBatchesDispatchedRate();
+  float getBatchesDispatchedRate();
 
   /**
    * Returns the average time taken to send a batch of events.
    */
-  public long getAverageDistributionTimePerBatch();
+  long getAverageDistributionTimePerBatch();
 
   /**
    * Returns the total number of batches of events that were resent.
    */
-  public int getTotalBatchesRedistributed();
+  int getTotalBatchesRedistributed();
 
   /**
    * Starts this GatewaySender. Once the GatewaySender is running its configuration cannot be
    * changed.
    * 
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void start();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void start();
 
   /**
    * Stops this GatewaySender.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void stop();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void stop();
 
   /**
    * Pauses this GatewaySender.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void pause();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void pause();
 
   /**
    * Resumes this paused GatewaySender.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void resume();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void resume();
 
   /**
    * Rebalances this GatewaySender.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void rebalance();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.GATEWAY)
+  void rebalance();
 
   /**
    * Returns whether this GatewaySender is primary or secondary.
    * 
    * @return True if this is the primary, false otherwise.
    */
-  public boolean isPrimary();
+  boolean isPrimary();
 
   /**
    * Returns the number of dispatcher threads working for this <code>GatewaySender</code>.
    */
-  public int getDispatcherThreads();
+  int getDispatcherThreads();
 
   /**
    * Returns the order policy followed while dispatching the events to remote distributed system.
    * Order policy is only relevant when the number of dispatcher threads is greater than one.
    */
 
-  public String getOrderPolicy();
+  String getOrderPolicy();
 
   /**
    * Returns whether the isDiskSynchronous property is set for this GatewaySender.
    * 
    * @return True if the property is set, false otherwise.
    */
-  public boolean isDiskSynchronous();
+  boolean isDiskSynchronous();
 
   /**
    * Returns whether the isParallel property is set for this GatewaySender.
    * 
    * @return True if the property is set, false otherwise.
    */
-  public boolean isParallel();
+  boolean isParallel();
 
   /**
    * Returns the host and port information of GatewayReceiver to which this gateway sender is
    * connected.
    */
-  public String getGatewayReceiver();
+  String getGatewayReceiver();
 
   /**
    * Returns whether this GatewaySender is connected and sending data to a GatewayReceiver.
    */
-  public boolean isConnected();
+  boolean isConnected();
 
   /**
    * Returns number of events which have exceeded the configured alert threshold.
    */
-  public int getEventsExceedingAlertThreshold();
+  int getEventsExceedingAlertThreshold();
 
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/LockServiceMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/LockServiceMXBean.java b/geode-core/src/main/java/org/apache/geode/management/LockServiceMXBean.java
index f109022..a62ca49 100644
--- a/geode-core/src/main/java/org/apache/geode/management/LockServiceMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/LockServiceMXBean.java
@@ -35,53 +35,53 @@ public interface LockServiceMXBean {
   /**
    * Returns the name of the lock service.
    */
-  public String getName();
+  String getName();
 
   /**
    * Returns whether this is a distributed LockService.
    *
    * @return True is this is a distributed LockService, false otherwise.
    */
-  public boolean isDistributed();
+  boolean isDistributed();
 
   /**
    * Returns the number of members using this LockService.
    */
-  public int getMemberCount();
+  int getMemberCount();
 
   /**
    * Returns of the name of the member which grants the lock.
    */
-  public String fetchGrantorMember();
+  String fetchGrantorMember();
 
   /**
    * Returns a list of names of the members using this LockService.
    */
-  public String[] getMemberNames();
+  String[] getMemberNames();
 
   /**
    * Returns whether this member is the granter.
    * 
    * @return True if this member is the granter, false otherwise.
    */
-  public boolean isLockGrantor();
+  boolean isLockGrantor();
 
 
   /**
    * Requests that this member become the granter.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public void becomeLockGrantor();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
+  void becomeLockGrantor();
 
   /**
    * Returns a map of the names of the objects being locked on and the names of the threads holding
    * the locks.
    */
-  public Map<String, String> listThreadsHoldingLock();
+  Map<String, String> listThreadsHoldingLock();
 
   /**
    * Returns a list of names of the locks held by this member's threads.
    */
-  public String[] listHeldLocks();
+  String[] listHeldLocks();
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/451d12e8/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java b/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
index e2de400..ca7c2a2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
@@ -20,6 +20,7 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
 
 /**
  * MBean that provides access to information and management functionality for a
@@ -143,7 +144,7 @@ public interface MemberMXBean {
    * 
    * @param numberOfLines Number of lines to return, up to a maximum of 100.
    */
-  public String showLog(int numberOfLines);
+  String showLog(int numberOfLines);
 
   /**
    * Returns the license string for this member.
@@ -151,15 +152,16 @@ public interface MemberMXBean {
    * @deprecated Removed licensing in 8.0.
    */
   @Deprecated
-  public String viewLicense();
+  String viewLicense();
 
   /**
    * Performs compaction on all of the member's disk stores.
    * 
    * @return A list of names of the disk stores that were compacted.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public String[] compactAllDiskStores();
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
+      target = Target.DISK)
+  String[] compactAllDiskStores();
 
   /**
    * Creates a Manager MBean on this member.
@@ -167,24 +169,24 @@ public interface MemberMXBean {
    * @return True if the Manager MBean was successfully created, false otherwise.
    */
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
-  public boolean createManager();
+  boolean createManager();
 
   /**
    * Shuts down the member. This is an asynchronous call and it will return immediately without
    * waiting for a result.
    */
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
-  public void shutDownMember();
+  void shutDownMember();
 
   /**
    * Returns JVM metrics.
    */
-  public JVMMetrics showJVMMetrics();
+  JVMMetrics showJVMMetrics();
 
   /**
    * Returns operating system metrics.
    */
-  public OSMetrics showOSMetrics();
+  OSMetrics showOSMetrics();
 
   /**
    * Executes a command on the member.
@@ -222,89 +224,89 @@ public interface MemberMXBean {
    * 
    * @param includeRegionOwned Whether to include disk stores owned by a region.
    */
-  public String[] listDiskStores(boolean includeRegionOwned);
+  String[] listDiskStores(boolean includeRegionOwned);
 
   /**
    * Returns the GemFire specific properties for this member.
    */
-  public GemFireProperties listGemFireProperties();
+  GemFireProperties listGemFireProperties();
 
   /**
    * Returns the name or IP address of the host on which this member is running.
    */
-  public String getHost();
+  String getHost();
 
   /**
    * Returns the name of this member.
    */
-  public String getName();
+  String getName();
 
   /**
    * Returns the ID of this member.
    */
-  public String getId();
+  String getId();
 
   /**
    * Returns the name of the member if it's been set, otherwise the ID.
    */
-  public String getMember();
+  String getMember();
 
   /**
    * Returns the names of the groups this member belongs to.
    */
-  public String[] getGroups();
+  String[] getGroups();
 
   /**
    * Returns the operating system process ID.
    */
-  public int getProcessId();
+  int getProcessId();
 
   /**
    * Returns the status.
    */
-  public String status();
+  String status();
 
   /**
    * Returns the GemFire version.
    */
-  public String getVersion();
+  String getVersion();
 
   /**
    * Returns whether this member is attached to at least one Locator.
    * 
    * @return True if this member is attached to a Locator, false otherwise.
    */
-  public boolean isLocator();
+  boolean isLocator();
 
   /**
    * Returns the number of seconds that this member will wait for a distributed lock.
    */
-  public long getLockTimeout();
+  long getLockTimeout();
 
   /**
    * Returns the number of second that this member will lease a distributed lock.
    */
-  public long getLockLease();
+  long getLockLease();
 
   /**
    * Any long-running GemFire process that was started with "start server" command from GFSH. It
    * returns true even if that process has --disable-default-server=true.
    */
-  public boolean isServer();
+  boolean isServer();
 
   /**
    * Returns whether this member has at least one GatewaySender.
    * 
-   * @return True if this member has at least one GatwaySender, false otherwise.
+   * @return True if this member has at least one GatewaySender, false otherwise.
    */
-  public boolean hasGatewaySender();
+  boolean hasGatewaySender();
 
   /**
    * Returns whether this member is running the Manager service.
    * 
    * @return True if this member is running the Manager service, false otherwise.
    */
-  public boolean isManager();
+  boolean isManager();
 
   /**
    * Returns whether this member has created the Manager service (it may be created, but not
@@ -312,29 +314,29 @@ public interface MemberMXBean {
    * 
    * @return True if this member has created the Manager service, false otherwise.
    */
-  public boolean isManagerCreated();
+  boolean isManagerCreated();
 
   /**
    * Returns whether this member has at least one GatewayReceiver.
    * 
-   * @return True if this member has at least one GatwayReceiver, false otherwise.
+   * @return True if this member has at least one GatewayReceiver, false otherwise.
    */
-  public boolean hasGatewayReceiver();
+  boolean hasGatewayReceiver();
 
   /**
    * Returns the ClassPath.
    */
-  public String getClassPath();
+  String getClassPath();
 
   /**
    * Returns the current time on the member's host.
    */
-  public long getCurrentTime();
+  long getCurrentTime();
 
   /**
    * Returns the number of seconds that this member has been running.
    */
-  public long getMemberUpTime();
+  long getMemberUpTime();
 
   /**
    * Returns the time (as a percentage) that this member's process time with respect to Statistics
@@ -345,393 +347,402 @@ public interface MemberMXBean {
    * the platform it will be shown as -1
    * 
    */
-  public float getCpuUsage();
+  float getCpuUsage();
 
   /**
    * Returns the current size of the heap in megabytes.
    * 
    * @deprecated Please use {@link #getUsedMemory()} instead.
    */
-  public long getCurrentHeapSize();
+  long getCurrentHeapSize();
 
   /**
    * Returns the maximum size of the heap in megabytes.
    * 
    * @deprecated Please use {@link #getMaxMemory()} instead.
    */
-  public long getMaximumHeapSize();
+  long getMaximumHeapSize();
 
   /**
    * Returns the free heap size in megabytes.
    * 
    * @deprecated Please use {@link #getFreeMemory()} instead.
    */
-  public long getFreeHeapSize();
+  long getFreeHeapSize();
 
   /**
    * Returns the maximum size of the heap in megabytes.
    * 
    */
-  public long getMaxMemory();
+  long getMaxMemory();
 
   /**
    * Returns the free heap size in megabytes.
    */
-  public long getFreeMemory();
+  long getFreeMemory();
 
   /**
    * Returns the current size of the heap in megabytes.
    */
-  public long getUsedMemory();
+  long getUsedMemory();
 
   /**
    * Returns the current threads.
    */
-  public String[] fetchJvmThreads();
+  String[] fetchJvmThreads();
 
   /**
    * Returns the maximum number of open file descriptors allowed for the member's host operating
    * system.
    */
-  public long getFileDescriptorLimit();
+  long getFileDescriptorLimit();
 
   /**
    * Returns the current number of open file descriptors.
    */
-  public long getTotalFileDescriptorOpen();
+  long getTotalFileDescriptorOpen();
 
   /**
    * Returns the number of Regions present in the Cache.
    */
-  public int getTotalRegionCount();
+  int getTotalRegionCount();
 
   /**
    * Returns the number of Partition Regions present in the Cache.
    */
-  public int getPartitionRegionCount();
+  int getPartitionRegionCount();
 
   /**
    * Returns a list of all Region names.
    */
-  public String[] listRegions();
+  String[] listRegions();
 
 
   /**
    * Returns a list of all disk stores, including those owned by a Region.
    */
-  public String[] getDiskStores();
+  String[] getDiskStores();
 
   /**
    * Returns a list of all root Region names.
    */
-  public String[] getRootRegionNames();
+  String[] getRootRegionNames();
 
   /**
    * Returns the total number of entries in all regions.
    */
-  public int getTotalRegionEntryCount();
+  int getTotalRegionEntryCount();
 
   /**
    * Returns the total number of buckets.
    */
-  public int getTotalBucketCount();
+  int getTotalBucketCount();
 
   /**
    * Returns the number of buckets for which this member is the primary holder.
    */
-  public int getTotalPrimaryBucketCount();
+  int getTotalPrimaryBucketCount();
 
   /**
    * Returns the cache get average latency.
    */
-  public long getGetsAvgLatency();
+  long getGetsAvgLatency();
 
   /**
    * Returns the cache put average latency.
    */
-  public long getPutsAvgLatency();
+  long getPutsAvgLatency();
 
   /**
    * Returns the cache putAll average latency.
    */
-  public long getPutAllAvgLatency();
+  long getPutAllAvgLatency();
 
   /**
    * Returns the number of times that a cache miss occurred for all regions.
    */
-  public int getTotalMissCount();
+  int getTotalMissCount();
 
   /**
    * Returns the number of times that a hit occurred for all regions.
    */
-  public int getTotalHitCount();
+  int getTotalHitCount();
 
   /**
    * Returns the number of gets per second.
    */
-  public float getGetsRate();
+  float getGetsRate();
 
   /**
    * Returns the number of puts per second. Only includes puts done explicitly on this member's
    * cache, not those pushed from another member.
    */
-  public float getPutsRate();
+  float getPutsRate();
 
   /**
    * Returns the number of putAlls per second.
    */
-  public float getPutAllRate();
+  float getPutAllRate();
 
   /**
    * Returns the number of creates per second.
    */
-  public float getCreatesRate();
+  float getCreatesRate();
 
   /**
    * Returns the number of destroys per second.
    */
-  public float getDestroysRate();
+  float getDestroysRate();
 
   /**
    * Returns the average latency of a call to a CacheWriter.
    */
-  public long getCacheWriterCallsAvgLatency();
+  long getCacheWriterCallsAvgLatency();
 
   /**
    * Returns the average latency of a call to a CacheListener.
    */
-  public long getCacheListenerCallsAvgLatency();
+  long getCacheListenerCallsAvgLatency();
 
   /**
    * Returns the total number of times that a load on this cache has completed, as a result of
    * either a local get or a remote net load.
    */
-  public int getTotalLoadsCompleted();
+  int getTotalLoadsCompleted();
 
   /**
    * Returns the average latency of a load.
    */
-  public long getLoadsAverageLatency();
+  long getLoadsAverageLatency();
 
   /**
    * Returns the total number of times the a network load initiated by this cache has completed.
    */
-  public int getTotalNetLoadsCompleted();
+  int getTotalNetLoadsCompleted();
 
   /**
    * Returns the net load average latency.
    */
-  public long getNetLoadsAverageLatency();
+  long getNetLoadsAverageLatency();
 
   /**
    * Returns the total number of times that a network search initiated by this cache has completed.
    */
-  public int getTotalNetSearchCompleted();
+  int getTotalNetSearchCompleted();
 
   /**
    * Returns the net search average latency.
    */
-  public long getNetSearchAverageLatency();
+  long getNetSearchAverageLatency();
 
   /**
    * Returns the current number of disk tasks (op-log compaction, asynchronous recovery, etc.) that
    * are waiting for a thread to run.
    */
-  public int getTotalDiskTasksWaiting();
+  int getTotalDiskTasksWaiting();
 
   /**
    * Returns the average number of bytes per second sent.
    */
-  public float getBytesSentRate();
+  float getBytesSentRate();
 
   /**
    * Returns the average number of bytes per second received.
    */
-  public float getBytesReceivedRate();
+  float getBytesReceivedRate();
 
   /**
    * Returns a list of IDs for all connected gateway receivers.
    */
-  public String[] listConnectedGatewayReceivers();
+  String[] listConnectedGatewayReceivers();
 
   /**
    * Returns a list of IDs for all gateway senders.
    */
-  public String[] listConnectedGatewaySenders();
+  String[] listConnectedGatewaySenders();
 
   /**
    * Returns the number of currently executing functions.
    */
-  public int getNumRunningFunctions();
+  int getNumRunningFunctions();
 
   /**
    * Returns the average function execution rate.
    */
-  public float getFunctionExecutionRate();
+  float getFunctionExecutionRate();
 
   /**
-   * Returns the number of currently executing functions that will return resutls.
+   * Returns the number of currently executing functions that will return results.
    */
-  public int getNumRunningFunctionsHavingResults();
+  int getNumRunningFunctionsHavingResults();
 
   /**
    * Returns the number of current transactions.
    */
-  public int getTotalTransactionsCount();
+  int getTotalTransactionsCount();
 
   /**
    * Returns the average commit latency in nanoseconds .
    */
-  public long getTransactionCommitsAvgLatency();
+  long getTransactionCommitsAvgLatency();
 
   /**
    * Returns the number of committed transactions.
    */
-  public int getTransactionCommittedTotalCount();
+  int getTransactionCommittedTotalCount();
 
   /**
    * Returns the number of transactions that were rolled back.
    */
-  public int getTransactionRolledBackTotalCount();
+  int getTransactionRolledBackTotalCount();
 
   /**
    * Returns the average number of transactions committed per second.
    */
-  public float getTransactionCommitsRate();
+  float getTransactionCommitsRate();
 
 
   /**
    * Returns the number of bytes reads per second from all the disks of the member.
    */
-  public float getDiskReadsRate();
+  float getDiskReadsRate();
 
   /**
    * Returns the number of bytes written per second to disk to all the disks of the member.
    */
-  public float getDiskWritesRate();
+  float getDiskWritesRate();
 
   /**
    * Returns the average disk flush latency time in nanoseconds.
    */
-  public long getDiskFlushAvgLatency();
+  long getDiskFlushAvgLatency();
 
   /**
    * Returns the number of backups currently in progress for all disk stores.
    */
-  public int getTotalBackupInProgress();
+  int getTotalBackupInProgress();
 
   /**
    * Returns the number of backups that have been completed.
    */
-  public int getTotalBackupCompleted();
+  int getTotalBackupCompleted();
 
   /**
    * Returns the number of threads waiting for a lock.
    */
-  public int getLockWaitsInProgress();
+  int getLockWaitsInProgress();
 
   /**
    * Returns the amount of time (in milliseconds) spent waiting for a lock.
    */
-  public long getTotalLockWaitTime();
+  long getTotalLockWaitTime();
 
   /**
    * Returns the number of lock services in use.
    */
-  public int getTotalNumberOfLockService();
+  int getTotalNumberOfLockService();
 
   /**
    * Returns the number of locks for which this member is a granter.
    */
-  public int getTotalNumberOfGrantors();
+  int getTotalNumberOfGrantors();
 
   /**
    * Returns the number of lock request queues in use by this member.
    */
-  public int getLockRequestQueues();
+  int getLockRequestQueues();
 
   /**
    * Returns the entry eviction rate as triggered by the LRU policy.
    */
-  public float getLruEvictionRate();
+  float getLruEvictionRate();
 
   /**
    * Returns the rate of entries destroyed either by destroy cache operations or eviction.
    */
-  public float getLruDestroyRate();
+  float getLruDestroyRate();
+
+  /**
+   * Returns the number of initial images in progress.
+   * 
+   * @deprecated as typo in name has been corrected: use
+   *             {@link MemberMXBean#getInitialImagesInProgress} instead.
+   */
+  @Deprecated
+  int getInitialImagesInProgres();
 
   /**
    * Returns the number of initial images in progress.
    */
-  public int getInitialImagesInProgres();
+  int getInitialImagesInProgress();
 
   /**
    * Returns the total amount of time spent performing a "get initial image" operation when creating
    * a region.
    */
-  public long getInitialImageTime();
+  long getInitialImageTime();
 
   /**
    * Return the number of keys received while performing a "get initial image" operation when
    * creating a region.
    */
-  public int getInitialImageKeysReceived();
+  int getInitialImageKeysReceived();
 
   /**
    * Returns the average time (in nanoseconds) spent deserializing objects. Includes
    * deserializations that result in a PdxInstance.
    */
-  public long getDeserializationAvgLatency();
+  long getDeserializationAvgLatency();
 
   /**
    * Returns the average latency (in nanoseconds) spent deserializing objects. Includes
    * deserializations that result in a PdxInstance.
    */
-  public long getDeserializationLatency();
+  long getDeserializationLatency();
 
   /**
    * Returns the instantaneous rate of deserializing objects. Includes deserializations that result
    * in a PdxInstance.
    */
-  public float getDeserializationRate();
+  float getDeserializationRate();
 
   /**
    * Returns the average time (in nanoseconds) spent serializing objects. Includes serializations
    * that result in a PdxInstance.
    */
-  public long getSerializationAvgLatency();
+  long getSerializationAvgLatency();
 
   /**
    * Returns the average latency (in nanoseconds) spent serializing objects. Includes serializations
    * that result in a PdxInstance.
    */
-  public long getSerializationLatency();
+  long getSerializationLatency();
 
   /**
    * Returns the instantaneous rate of serializing objects. Includes serializations that result in a
    * PdxInstance.
    */
-  public float getSerializationRate();
+  float getSerializationRate();
 
   /**
    * Returns the instantaneous rate of PDX instance deserialization.
    */
-  public float getPDXDeserializationRate();
+  float getPDXDeserializationRate();
 
   /**
    * Returns the average time, in seconds, spent deserializing PDX instanced.
    */
-  public long getPDXDeserializationAvgLatency();
+  long getPDXDeserializationAvgLatency();
 
   /**
    * Returns the total number of bytes used on all disks.
    */
-  public long getTotalDiskUsage();
+  long getTotalDiskUsage();
 
   /**
    * Returns the number of threads in use.
    */
-  public int getNumThreads();
+  int getNumThreads();
 
   /**
    * Returns the system load average for the last minute. The system load average is the sum of the
@@ -742,99 +753,99 @@ public interface MemberMXBean {
    * 
    * @return The load average or a negative value if one is not available.
    */
-  public double getLoadAverage();
+  double getLoadAverage();
 
   /**
    * Returns the number of times garbage collection has occurred.
    */
-  public long getGarbageCollectionCount();
+  long getGarbageCollectionCount();
 
   /**
    * Returns the amount of time (in milliseconds) spent on garbage collection.
    */
-  public long getGarbageCollectionTime();
+  long getGarbageCollectionTime();
 
   /**
    * Returns the average number of reads per second.
    */
-  public float getAverageReads();
+  float getAverageReads();
 
   /**
    * Returns the average writes per second, including both put and putAll operations.
    */
-  public float getAverageWrites();
+  float getAverageWrites();
 
   /**
    * Returns the number JVM pauses (which may or may not include full garbage collection pauses)
    * detected by GemFire.
    */
-  public long getJVMPauses();
+  long getJVMPauses();
 
 
   /**
    * Returns the underlying host's current cpuActive percentage
    */
-  public int getHostCpuUsage();
+  int getHostCpuUsage();
 
   /**
    * 
    * Returns true if a cache server is running on this member and able server requests from GemFire
    * clients
    */
-  public boolean isCacheServer();
+  boolean isCacheServer();
 
   /**
    * Returns the redundancy-zone of the member;
    */
-  public String getRedundancyZone();
+  String getRedundancyZone();
 
   /**
    * Returns current number of cache rebalance operations being directed by this process.
    */
-  public int getRebalancesInProgress();
+  int getRebalancesInProgress();
 
   /**
    * Returns current number of threads waiting for a reply.
    */
-  public int getReplyWaitsInProgress();
+  int getReplyWaitsInProgress();
 
   /**
    * Returns total number of times waits for a reply have completed.
    */
-  public int getReplyWaitsCompleted();
+  int getReplyWaitsCompleted();
 
   /**
    * The current number of nodes in this distributed system visible to this member.
    */
-  public int getVisibleNodes();
+  int getVisibleNodes();
 
   /**
    * Returns the number of off heap objects.
    */
-  public int getOffHeapObjects();
+  int getOffHeapObjects();
 
   /**
    * Returns the size of the maximum configured off-heap memory in bytes.
    */
-  public long getOffHeapMaxMemory();
+  long getOffHeapMaxMemory();
 
   /**
    * Returns the size of available (or unallocated) off-heap memory in bytes.
    */
-  public long getOffHeapFreeMemory();
+  long getOffHeapFreeMemory();
 
   /**
    * Returns the size of utilized off-heap memory in bytes.
    */
-  public long getOffHeapUsedMemory();
+  long getOffHeapUsedMemory();
 
   /**
    * Returns the percentage of off-heap memory fragmentation.
    */
-  public int getOffHeapFragmentation();
+  int getOffHeapFragmentation();
 
   /**
-   * Returns the total time spent compacting in millseconds.
+   * Returns the total time spent compacting in milliseconds.
    */
-  public long getOffHeapCompactionTime();
+  long getOffHeapCompactionTime();
 }