You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bo...@apache.org on 2016/06/30 17:10:49 UTC

incubator-geode git commit: GEODE-11: Added JMX MBean API providing raw stat values

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 63ce3c0f4 -> ee057745c


GEODE-11: Added JMX MBean API providing raw stat values


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

Branch: refs/heads/develop
Commit: ee057745cfb13a9f1bbea6ca41580afba28633e6
Parents: 63ce3c0
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Tue Jun 28 16:01:03 2016 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Thu Jun 30 09:59:10 2016 -0700

----------------------------------------------------------------------
 .../distributed/internal/ResourceEvent.java     |   4 +-
 .../gemfire/internal/cache/CacheService.java    |   7 +
 .../internal/cache/GemFireCacheImpl.java        |   1 +
 .../gemfire/management/JMXNotificationType.java |   7 +
 .../management/internal/MBeanJMXAdapter.java    |   5 +
 .../internal/ManagementConstants.java           |   8 +-
 .../internal/beans/CacheServiceMBeanBase.java   |  24 +++
 .../internal/beans/ManagementAdapter.java       |  30 +++-
 .../internal/beans/ManagementListener.java      |   7 +-
 .../internal/cache/MockCacheServiceImpl.java    |   6 +
 .../cache/lucene/internal/IndexListener.java    |  35 ++++
 .../cache/lucene/internal/LuceneIndexStats.java |  39 +++++
 .../lucene/internal/LuceneServiceImpl.java      |   9 +-
 .../internal/management/LuceneIndexMetrics.java | 161 +++++++++++++++++++
 .../management/LuceneServiceBridge.java         |  82 ++++++++++
 .../internal/management/LuceneServiceMBean.java |  57 +++++++
 .../management/LuceneServiceMXBean.java         |  58 +++++++
 17 files changed, 534 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ResourceEvent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ResourceEvent.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ResourceEvent.java
index 2250bf4..584975e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ResourceEvent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ResourceEvent.java
@@ -45,7 +45,9 @@ public enum ResourceEvent {
     GATEWAYSENDER_START, 
     GATEWAYSENDER_STOP, 
     GATEWAYSENDER_PAUSE, 
-    GATEWAYSENDER_RESUME 
+    GATEWAYSENDER_RESUME,
+    CACHE_SERVICE_CREATE,
+    CACHE_SERVICE_REMOVE
     
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheService.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheService.java
index a8c4cb8..b1d3b62 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheService.java
@@ -17,6 +17,7 @@
 package com.gemstone.gemfire.internal.cache;
 
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.management.internal.beans.CacheServiceMBeanBase;
 
 /**
  * Interface for a service that is linked to a cache.
@@ -39,4 +40,10 @@ public interface CacheService {
    * this service. 
    */
   public Class<? extends CacheService> getInterface();
+
+  /**
+   * Returns the MBean associated with this server
+   * @return the MBean associated with this server
+   */
+  public CacheServiceMBeanBase getMBean();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 186ebbc..13e0602 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -1101,6 +1101,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     for(CacheService service : loader) {
       service.init(this);
       this.services.put(service.getInterface(), service);
+      system.handleResourceEvent(ResourceEvent.CACHE_SERVICE_CREATE, service);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/management/JMXNotificationType.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/JMXNotificationType.java b/geode-core/src/main/java/com/gemstone/gemfire/management/JMXNotificationType.java
index 4726583..199f9ae 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/JMXNotificationType.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/JMXNotificationType.java
@@ -206,4 +206,11 @@ public interface JMXNotificationType {
    * <CODE>gemfire.distributedsystem.locator.started</CODE>.
    */
   public static final String LOCATOR_STARTED = DistributionConfig.GEMFIRE_PREFIX + "distributedsystem.locator.started";
+
+  /**
+   * Notification type which indicates that a cache service is created <BR>
+   * The value of this type string is
+   * <CODE>gemfire.distributedsystem.cache.service.created</CODE>.
+   */
+  public static final String CACHE_SERVICE_CREATED = DistributionConfig.GEMFIRE_PREFIX + "distributedsystem.cache.service.created";
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/management/internal/MBeanJMXAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/MBeanJMXAdapter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/MBeanJMXAdapter.java
index 345c452..bf116d0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/MBeanJMXAdapter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/MBeanJMXAdapter.java
@@ -574,6 +574,11 @@ public class MBeanJMXAdapter implements ManagementConstants {
     return getObjectName((MessageFormat.format(OBJECTNAME__LOCATOR_MXBEAN, new Object[] { makeCompliantName(member) })));
   }
 
+  public static ObjectName getCacheServiceMBeanName(DistributedMember member, String cacheServiceId) {
+    return getObjectName((MessageFormat.format(OBJECTNAME__CACHESERVICE_MXBEAN, new Object[] { cacheServiceId,
+        getMemberNameOrId(member) })));
+  }
+
   public Map<ObjectName, Object> getLocalGemFireMBean() {
     return this.localGemFireMBean;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementConstants.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementConstants.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementConstants.java
index 8598104..ca04573 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementConstants.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementConstants.java
@@ -119,7 +119,9 @@ public interface ManagementConstants {
   public static final String OBJECTNAME__DISKSTORE_MXBEAN              = OBJECTNAME__PREFIX + "service=DiskStore,name={0},type=Member,member={1}";
   
   public static final String OBJECTNAME__LOCATOR_MXBEAN                = OBJECTNAME__PREFIX + "service=Locator,type=Member,member={0}";
-  
+
+  public static final String OBJECTNAME__CACHESERVICE_MXBEAN           = OBJECTNAME__PREFIX + "service=CacheService,name={0},type=Member,member={1}";
+
   public static final String AGGREGATE_MBEAN_PATTERN                   = OBJECTNAME__PREFIX + "*,type=Distributed";
   // Object Name keys 
   
@@ -171,7 +173,9 @@ public interface ManagementConstants {
   public static final String GATEWAY_RECEIVER_STOPPED_PREFIX= "GatewayReceiver Stopped in the VM ";
   
   public static final String ASYNC_EVENT_QUEUE_CREATED_PREFIX= "Async Event Queue is Created  in the VM ";
-  
+
+  public static final String CACHE_SERVICE_CREATED_PREFIX= "Cache Service Created With Name ";
+
   public static final String CACHE_SERVER_STARTED_PREFIX= "Cache Server is Started in the VM ";
   public static final String CACHE_SERVER_STOPPED_PREFIX= "Cache Server is stopped in the VM ";
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServiceMBeanBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServiceMBeanBase.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServiceMBeanBase.java
new file mode 100644
index 0000000..3e15a28
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServiceMBeanBase.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.beans;
+
+public interface CacheServiceMBeanBase {
+
+  public String getId();
+
+  public Class getInterfaceClass();
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
index b1cc9fc..a288338 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
@@ -32,6 +32,8 @@ import javax.management.NotificationBroadcasterSupport;
 import javax.management.ObjectInstance;
 import javax.management.ObjectName;
 
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.internal.cache.CacheService;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.cache.Cache;
@@ -1062,7 +1064,33 @@ public class ManagementAdapter {
 
   }
 
-  
+  public void handleCacheServiceCreation(CacheService cacheService)
+      throws ManagementException {
+    if (!isServiceInitialised("handleCacheServiceCreation")) {
+      return;
+    }
+    // Don't register the CacheServices in the Locator
+    InternalDistributedMember member = cacheImpl.getDistributedSystem().getDistributedMember();
+    if (member.getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+      return;
+    }
+    CacheServiceMBeanBase mbean = cacheService.getMBean();
+    if (mbean != null) {
+      String id = mbean.getId();
+      ObjectName cacheServiceObjectName = MBeanJMXAdapter.getCacheServiceMBeanName(member, id);
+
+      ObjectName changedMBeanName = service.registerInternalMBean(mbean, cacheServiceObjectName);
+
+      service.federate(changedMBeanName, mbean.getInterfaceClass(), true);
+
+      Notification notification = new Notification(
+          JMXNotificationType.CACHE_SERVICE_CREATED, memberSource,
+          SequenceNumber.next(), System.currentTimeMillis(),
+          ManagementConstants.CACHE_SERVICE_CREATED_PREFIX + id);
+      memberLevelNotifEmitter.sendNotification(notification);
+    }
+  }
+
   /**
    * Private class which acts as a ClientMembershipListener to propagate client
    * joined/left notifications

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementListener.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementListener.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementListener.java
index 5947727..24dc94d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementListener.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementListener.java
@@ -29,6 +29,7 @@ import com.gemstone.gemfire.distributed.internal.ResourceEvent;
 import com.gemstone.gemfire.distributed.internal.ResourceEventsListener;
 import com.gemstone.gemfire.distributed.internal.locks.DLockService;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
+import com.gemstone.gemfire.internal.cache.CacheService;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.management.ManagementException;
 import com.gemstone.gemfire.management.internal.AlertDetails;
@@ -205,7 +206,11 @@ public class ManagementListener implements ResourceEventsListener{
       case LOCATOR_START:
         Locator loc = (Locator) resource;
         adapter.handleLocatorStart(loc);
-        break; 
+        break;
+      case CACHE_SERVICE_CREATE:
+        CacheService service = (CacheService) resource;
+        adapter.handleCacheServiceCreation(service);
+        break;
       default:
       break;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/MockCacheServiceImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/MockCacheServiceImpl.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/MockCacheServiceImpl.java
index 4653ddf..1ba82f1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/MockCacheServiceImpl.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/MockCacheServiceImpl.java
@@ -17,6 +17,7 @@
 package com.gemstone.gemfire.internal.cache;
 
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.management.internal.beans.CacheServiceMBeanBase;
 
 public class MockCacheServiceImpl implements MockCacheService {
   
@@ -33,6 +34,11 @@ public class MockCacheServiceImpl implements MockCacheService {
   }
 
   @Override
+  public CacheServiceMBeanBase getMBean() {
+    return null;
+  }
+
+  @Override
   public Cache getCache() {
     return cache;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/IndexListener.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/IndexListener.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/IndexListener.java
new file mode 100644
index 0000000..c9c03a5
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/IndexListener.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import org.apache.lucene.analysis.Analyzer;
+
+import java.util.Map;
+
+public interface IndexListener {
+
+  public void beforeIndexCreated(final String indexName, String regionPath,
+      final Analyzer analyzer, final Map<String, Analyzer> fieldAnalyzers,
+      final String... fields);
+
+  public void afterIndexCreated(LuceneIndex index);
+
+  public void beforeIndexDestroyed(LuceneIndex index);
+
+  public void afterIndexDestroyed(LuceneIndex index);
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStats.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStats.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStats.java
index 8e2d97c..703f9da 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStats.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStats.java
@@ -156,4 +156,43 @@ public class LuceneIndexStats {
       .sum();
   }
 
+  public int getQueryExecutions() {
+    return stats.getInt(queryExecutionsId);
+  }
+
+  public long getQueryExecutionTime() {
+    return stats.getLong(queryExecutionTimeId);
+  }
+
+  public int getQueryExecutionsInProgress() {
+    return stats.getInt(queryExecutionsInProgressId);
+  }
+
+  public long getQueryExecutionTotalHits() {
+    return stats.getLong(queryExecutionTotalHits);
+  }
+
+  public int getUpdates() {
+    return stats.getInt(updatesId);
+  }
+
+  public long getUpdateTime() {
+    return stats.getLong(updateTimeId);
+  }
+
+  public int getUpdatesInProgress() {
+    return stats.getInt(updatesInProgressId);
+  }
+
+  public int getCommits() {
+    return stats.getInt(commitsId);
+  }
+
+  public long getCommitTime() {
+    return stats.getLong(commitTimeId);
+  }
+
+  public int getCommitsInProgress() {
+    return stats.getInt(commitsInProgressId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
index 6bfaa78..410b27b 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
@@ -21,6 +21,8 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import java.util.*;
 
+import com.gemstone.gemfire.cache.lucene.internal.management.LuceneServiceMBean;
+import com.gemstone.gemfire.management.internal.beans.CacheServiceMBeanBase;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
@@ -86,7 +88,12 @@ public class LuceneServiceImpl implements InternalLuceneService {
     FunctionService.registerFunction(new DumpDirectoryFiles());
     registerDataSerializables();
   }
-  
+
+  @Override
+  public CacheServiceMBeanBase getMBean() {
+    return new LuceneServiceMBean(this);
+  }
+
   @Override
   public Class<? extends CacheService> getInterface() {
     return InternalLuceneService.class;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneIndexMetrics.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneIndexMetrics.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneIndexMetrics.java
new file mode 100644
index 0000000..10c08a7
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneIndexMetrics.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.management;
+
+import java.beans.ConstructorProperties;
+
+public class LuceneIndexMetrics {
+
+  private final String regionPath;
+
+  private final String indexName;
+
+  private final int queryExecutions;
+
+  private final long queryExecutionTime;
+
+  private final int queryExecutionsInProgress;
+
+  private final long queryExecutionTotalHits;
+
+  private final int updates;
+
+  private final long updateTime;
+
+  private final int updatesInProgress;
+
+  private final int commits;
+
+  private final long commitTime;
+
+  private final int commitsInProgress;
+
+  private final int documents;
+
+  /**
+   * This constructor is to be used by internal JMX framework only. A user should
+   * not try to create an instance of this class.
+   */
+  @ConstructorProperties( { "regionPath", "indexName", "queryExecutions", "queryExecutionTime",
+      "queryExecutionsInProgress", "queryExecutionTotalHits", "updates", "updateTime", "updatesInProgress",
+      "commits", "commitTime", "commitsInProgress", "documents"
+  })
+  public LuceneIndexMetrics(String regionPath, String indexName, int queryExecutions, long queryExecutionTime,
+      int queryExecutionsInProgress, long queryExecutionTotalHits, int updates, long updateTime,
+      int updatesInProgress, int commits, long commitTime, int commitsInProgress, int documents) {
+    this.regionPath = regionPath;
+    this.indexName = indexName;
+    this.queryExecutions = queryExecutions;
+    this.queryExecutionTime = queryExecutionTime;
+    this.queryExecutionsInProgress = queryExecutionsInProgress;
+    this.queryExecutionTotalHits = queryExecutionTotalHits;
+    this.updates = updates;
+    this.updateTime = updateTime;
+    this.updatesInProgress = updatesInProgress;
+    this.commits = commits;
+    this.commitTime = commitTime;
+    this.commitsInProgress = commitsInProgress;
+    this.documents = documents;
+  }
+
+  public String getRegionPath() {
+    return this.regionPath;
+  }
+
+  public String getIndexName() {
+    return this.indexName;
+  }
+
+  public int getQueryExecutions() {
+    return this.queryExecutions;
+  }
+
+  public long getQueryExecutionTime() {
+    return this.queryExecutionTime;
+  }
+
+  public int getQueryExecutionsInProgress() {
+    return this.queryExecutionsInProgress;
+  }
+
+  public long getQueryExecutionTotalHits() {
+    return this.queryExecutionTotalHits;
+  }
+
+  public int getUpdates() {
+    return this.updates;
+  }
+
+  public long getUpdateTime() {
+    return this.updateTime;
+  }
+
+  public int getUpdatesInProgress() {
+    return this.updatesInProgress;
+  }
+
+  public int getCommits() {
+    return this.commits;
+  }
+
+  public long getCommitTime() {
+    return this.commitTime;
+  }
+
+  public int getCommitsInProgress() {
+    return this.commitsInProgress;
+  }
+
+  public int getDocuments() {
+    return documents;
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder()
+        .append(getClass().getSimpleName())
+        .append("[")
+        .append("regionPath=")
+        .append(this.regionPath)
+        .append("; indexName=")
+        .append(this.indexName)
+        .append("; queryExecutions=")
+        .append(this.queryExecutions)
+        .append("; queryExecutionTime=")
+        .append(this.queryExecutionTime)
+        .append("; queryExecutionsInProgress=")
+        .append(this.queryExecutionsInProgress)
+        .append("; queryExecutionTotalHits=")
+        .append(this.queryExecutionTotalHits)
+        .append("; updates=")
+        .append(this.updates)
+        .append("; updateTime=")
+        .append(this.updateTime)
+        .append("; updatesInProgress=")
+        .append(this.updatesInProgress)
+        .append("; commits=")
+        .append(this.commits)
+        .append("; commitTime=")
+        .append(this.commitTime)
+        .append("; commitsInProgress=")
+        .append(this.commitsInProgress)
+        .append("; documents=")
+        .append(this.documents)
+        .append("]")
+        .toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceBridge.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceBridge.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceBridge.java
new file mode 100644
index 0000000..b851b2c
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceBridge.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.management;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexImpl;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+public class LuceneServiceBridge {
+
+  private LuceneService service;
+
+  public LuceneServiceBridge(LuceneService service) {
+    this.service = service;
+  }
+
+  public LuceneIndexMetrics[] listIndexMetrics() {
+    Collection<LuceneIndex> indexes = this.service.getAllIndexes();
+    LuceneIndexMetrics[] indexMetrics = new LuceneIndexMetrics[indexes.size()];
+    int i=0;
+    for (LuceneIndex index : this.service.getAllIndexes()) {
+      indexMetrics[i++] = getIndexMetrics((LuceneIndexImpl)index);
+    }
+    return indexMetrics;
+  }
+
+  public LuceneIndexMetrics[] listIndexMetrics(String regionPath) {
+    if(!regionPath.startsWith(Region.SEPARATOR)) {
+      regionPath = Region.SEPARATOR + regionPath;
+    }
+    List<LuceneIndexMetrics> indexMetrics = new ArrayList();
+    for (LuceneIndex index : this.service.getAllIndexes()) {
+      if (index.getRegionPath().equals(regionPath)) {
+        indexMetrics.add(getIndexMetrics((LuceneIndexImpl) index));
+      }
+    }
+    return indexMetrics.toArray(new LuceneIndexMetrics[indexMetrics.size()]);
+  }
+
+  public LuceneIndexMetrics listIndexMetrics(String regionPath, String indexName) {
+    LuceneIndexImpl index = (LuceneIndexImpl) this.service.getIndex(indexName, regionPath);
+    return index == null ? null : getIndexMetrics(index);
+  }
+
+  private LuceneIndexMetrics getIndexMetrics(LuceneIndexImpl index) {
+    LuceneIndexStats indexStats = index.getIndexStats();
+    int queryExecutions = indexStats.getQueryExecutions();
+    long queryExecutionTime = indexStats.getQueryExecutionTime();
+    int queryExecutionsInProgress = indexStats.getQueryExecutionsInProgress();
+    long queryExecutionTotalHits = indexStats.getQueryExecutionTotalHits();
+    int updates = indexStats.getUpdates();
+    long updateTime = indexStats.getUpdateTime();
+    int updatesInProgress = indexStats.getUpdatesInProgress();
+    int commits = indexStats.getCommits();
+    long commitTime = indexStats.getCommitTime();
+    int commitsInProgress = indexStats.getCommitsInProgress();
+    int documents = indexStats.getDocuments();
+    return new LuceneIndexMetrics(index.getRegionPath(), index.getName(), queryExecutions, queryExecutionTime,
+        queryExecutionsInProgress, queryExecutionTotalHits, updates, updateTime, updatesInProgress, commits,
+        commitTime, commitsInProgress, documents);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMBean.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMBean.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMBean.java
new file mode 100644
index 0000000..6eb0b69
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMBean.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.management;
+
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.management.internal.beans.CacheServiceMBeanBase;
+
+import javax.management.NotificationBroadcasterSupport;
+
+public class LuceneServiceMBean extends NotificationBroadcasterSupport
+    implements LuceneServiceMXBean, CacheServiceMBeanBase {
+
+  private LuceneServiceBridge bridge;
+
+  public LuceneServiceMBean(LuceneService service) {
+    this.bridge = new LuceneServiceBridge(service);
+  }
+
+  @Override
+  public LuceneIndexMetrics[] listIndexMetrics() {
+    return this.bridge.listIndexMetrics();
+  }
+
+  @Override
+  public LuceneIndexMetrics[] listIndexMetrics(String regionPath) {
+    return this.bridge.listIndexMetrics(regionPath);
+  }
+
+  @Override
+  public LuceneIndexMetrics listIndexMetrics(String regionPath, String indexName) {
+    return this.bridge.listIndexMetrics(regionPath, indexName);
+  }
+
+  @Override
+  public String getId() {
+    return "LuceneService";
+  }
+
+  @Override
+  public Class getInterfaceClass() {
+    return LuceneServiceMXBean.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ee057745/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMXBean.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMXBean.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMXBean.java
new file mode 100644
index 0000000..7911a25
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/management/LuceneServiceMXBean.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.management;
+
+import com.gemstone.gemfire.cache.operations.OperationContext;
+import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+
+/**
+ * MBean that provides access to the {@link com.gemstone.gemfire.cache.lucene.LuceneService}.
+ */
+@ResourceOperation(resource = OperationContext.Resource.CLUSTER, operation = OperationContext.OperationCode.READ)
+public interface LuceneServiceMXBean {
+
+  /**
+   * Returns an array of {@link LuceneIndexMetrics} for the {@link com.gemstone.gemfire.cache.lucene.LuceneIndex}
+   * instances defined in this member
+   *
+   * @return an array of LuceneIndexMetrics for the LuceneIndexes defined in this member
+   */
+  public LuceneIndexMetrics[] listIndexMetrics();
+
+  /**
+   * Returns an array of {@link LuceneIndexMetrics} for the {@link com.gemstone.gemfire.cache.lucene.LuceneIndex}
+   * instances defined on the input region in this member
+   *
+   * @param regionPath The full path of the region to retrieve
+   *
+   * @return an array of LuceneIndexMetrics for the LuceneIndex instances defined on the input region
+   * in this member
+   */
+  public LuceneIndexMetrics[] listIndexMetrics(String regionPath);
+
+  /**
+   * Returns a {@link LuceneIndexMetrics} for the {@link com.gemstone.gemfire.cache.lucene.LuceneIndex}
+   * with the input index name defined on the input region in this member.
+   *
+   * @param regionPath The full path of the region to retrieve
+   * @param indexName The name of the index to retrieve
+   *
+   * @return a LuceneIndexMetrics for the LuceneIndex with the input index name defined on the input region
+   * in this member.
+   */
+  public LuceneIndexMetrics listIndexMetrics(String regionPath, String indexName);
+}