You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by es...@apache.org on 2016/05/31 21:44:19 UTC

[17/50] [abbrv] incubator-geode git commit: GEODE-11: Adding stats for lucene index updates and queries

GEODE-11: Adding stats for lucene index updates and queries

Adding some initial stats for lucene indexes tracking updates and
queries.


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

Branch: refs/heads/feature/GEODE-1400
Commit: 3e8fe7af0ced64d2a15b6c3ce52eb9560a83d097
Parents: 1339930
Author: Dan Smith <up...@apache.org>
Authored: Thu May 26 11:18:43 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri May 27 13:43:58 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexForPartitionedRegion.java        |   2 +-
 .../cache/lucene/internal/LuceneIndexImpl.java  |   3 +-
 .../cache/lucene/internal/LuceneIndexStats.java | 136 +++++++++++++++++++
 .../internal/PartitionedRepositoryManager.java  |  19 ++-
 .../repository/IndexRepositoryImpl.java         |  43 ++++--
 .../LuceneIndexForPartitionedRegionTest.java    |   4 +-
 .../LuceneIndexRecoveryHAIntegrationTest.java   |   6 +-
 .../internal/LuceneIndexStatsJUnitTest.java     | 105 ++++++++++++++
 .../PartitionedRepositoryManagerJUnitTest.java  |  14 +-
 .../DistributedScoringJUnitTest.java            |   5 +-
 .../IndexRepositoryImplJUnitTest.java           |  43 +++++-
 .../IndexRepositoryImplPerformanceTest.java     |   4 +-
 12 files changed, 352 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index d22ca4a..8f2cb10 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -84,7 +84,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
 
       // we will create RegionDirectories on the fly when data comes in
       HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(getFieldNames());
-      repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, mapper, analyzer);
+      repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, mapper, analyzer, this.stats);
       
       // create AEQ, AEQ listener and specify the listener to repositoryManager
       createAEQ(dataRegion);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index c165085..3b58a95 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -32,7 +32,6 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
@@ -53,6 +52,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   
   Region<String, File> fileRegion;
   Region<ChunkKey, byte[]> chunkRegion;
+  LuceneIndexStats stats;
   
   protected String indexName;
   protected String regionPath;
@@ -65,6 +65,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     this.indexName = indexName;
     this.regionPath = regionPath;
     this.cache = cache;
+    this.stats = new LuceneIndexStats(cache.getDistributedSystem(), indexName, regionPath);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/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
new file mode 100644
index 0000000..dab1b0f
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStats.java
@@ -0,0 +1,136 @@
+/*
+ * 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 static com.gemstone.gemfire.distributed.internal.DistributionStats.getStatTime;
+
+import com.gemstone.gemfire.StatisticDescriptor;
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.StatisticsFactory;
+import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.StatisticsTypeFactory;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
+
+public class LuceneIndexStats {
+  // statistics type
+  private static final StatisticsType statsType;
+  private static final String statsTypeName = "LuceneIndexStats";
+  private static final String statsTypeDescription = "Statistics about lucene indexes";
+
+  private static final int queryExecutionsId;
+  private static final int queryExecutionTimeId;
+  private static final int queryExecutionsInProgressId;
+  private static final int updatesId;
+  private static final int updateTimeId;
+  private static final int updatesInProgressId;
+  private static final int commitsId;
+  private static final int commitTimeId;
+  private static final int commitsInProgressId;
+  private static final int documentsId;
+
+  private final Statistics stats;
+
+  static {
+    final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+    statsType = f.createType(
+      statsTypeName,
+      statsTypeDescription,
+      new StatisticDescriptor[] {
+        f.createIntCounter("queryExecutions", "Number of lucene queries executed on this member", "operations"),
+        f.createLongCounter("queryExecutionTime", "Amount of time spent executing lucene queries", "nanoseconds"),
+        f.createIntGauge("queryExecutionsInProgress", "Number of query executions currently in progress", "operations"),
+        f.createIntCounter("updates", "Number of lucene index documents added/removed on this member", "operations"),
+        f.createLongCounter("updateTime", "Amount of time spent adding or removing documents from the index", "nanoseconds"),
+        f.createIntGauge("updatesInProgress", "Number of index updates in progress", "operations"),
+        f.createIntCounter("commits", "Number of lucene index commits on this member", "operations"),
+        f.createLongCounter("commitTime", "Amount of time spent in lucene index commits", "nanoseconds"),
+        f.createIntGauge("commitsInProgress", "Number of lucene index commits in progress", "operations"),
+        f.createIntGauge("documents", "Number of documents in the index", "documents"),
+      }
+    );
+
+    queryExecutionsId = statsType.nameToId("queryExecutions");
+    queryExecutionTimeId = statsType.nameToId("queryExecutionTime");
+    queryExecutionsInProgressId = statsType.nameToId("queryExecutionsInProgress");
+    updatesId = statsType.nameToId("updates");
+    updateTimeId = statsType.nameToId("updateTime");
+    updatesInProgressId = statsType.nameToId("updatesInProgress");
+    commitsId = statsType.nameToId("commits");
+    commitTimeId = statsType.nameToId("commitTime");
+    commitsInProgressId = statsType.nameToId("commitsInProgress");
+    documentsId = statsType.nameToId("documents");
+  }
+
+  public LuceneIndexStats(StatisticsFactory f, String indexName, String regionName) {
+    this.stats = f.createAtomicStatistics(statsType, indexName + "-" + regionName);
+  }
+
+  /**
+   * @return the timestamp that marks the start of the operation
+   */
+  public long startQuery() {
+    stats.incInt(queryExecutionsInProgressId, 1);
+    return getStatTime();
+  }
+  /**
+   * @param start the timestamp taken when the operation started
+   */
+  public void endQuery(long start) {
+    stats.incLong(queryExecutionTimeId, getStatTime()-start);
+    stats.incInt(queryExecutionsInProgressId, -1);
+    stats.incInt(queryExecutionsId, 1);
+  }
+
+  /**
+   * @return the timestamp that marks the start of the operation
+   */
+  public long startUpdate() {
+    stats.incInt(updatesInProgressId, 1);
+    return getStatTime();
+  }
+  /**
+   * @param start the timestamp taken when the operation started
+   */
+  public void endUpdate(long start) {
+    stats.incLong(updateTimeId, getStatTime()-start);
+    stats.incInt(updatesInProgressId, -1);
+    stats.incInt(updatesId, 1);
+  }
+
+  /**
+   * @return the timestamp that marks the start of the operation
+   */
+  public long startCommit() {
+    stats.incInt(commitsInProgressId, 1);
+    return getStatTime();
+  }
+  /**
+   * @param start the timestamp taken when the operation started
+   */
+  public void endCommit(long start) {
+    stats.incLong(commitTimeId, getStatTime()-start);
+    stats.incInt(commitsInProgressId, -1);
+    stats.incInt(commitsId, 1);
+  }
+
+  public void incDocuments(int delta) {
+    stats.incInt(documentsId, delta);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
index 57b8862..1b48316 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
@@ -51,7 +51,8 @@ import com.gemstone.gemfire.internal.util.concurrent.CopyOnWriteHashMap;
  * index repository when the bucket returns to this node.
  */
 public class PartitionedRepositoryManager implements RepositoryManager {
-  /** map of the parent bucket region to the index repository 
+
+  /** map of the parent bucket region to the index repository
    * 
    * This is based on the BucketRegion in case a bucket is rebalanced, we don't want to 
    * return a stale index repository. If a bucket moves off of this node and
@@ -68,7 +69,8 @@ public class PartitionedRepositoryManager implements RepositoryManager {
   private final PartitionedRegion chunkRegion;
   private final LuceneSerializer serializer;
   private final Analyzer analyzer;
-  
+  private final LuceneIndexStats stats;
+
   /**
    * 
    * @param userRegion The user partition region
@@ -76,15 +78,18 @@ public class PartitionedRepositoryManager implements RepositoryManager {
    * @param chunkRegion The partition region users for chunk metadata.
    * @param serializer The serializer that should be used for converting objects to lucene docs.
    */
-  public PartitionedRepositoryManager(PartitionedRegion userRegion, PartitionedRegion fileRegion,
-      PartitionedRegion chunkRegion,
-      LuceneSerializer serializer,
-      Analyzer analyzer) {
+  public PartitionedRepositoryManager(PartitionedRegion userRegion,
+                                      PartitionedRegion fileRegion,
+                                      PartitionedRegion chunkRegion,
+                                      LuceneSerializer serializer,
+                                      Analyzer analyzer,
+                                      LuceneIndexStats stats) {
     this.userRegion = userRegion;
     this.fileRegion = fileRegion;
     this.chunkRegion = chunkRegion;
     this.serializer = serializer;
     this.analyzer = analyzer;
+    this.stats = stats;
   }
 
   @Override
@@ -134,7 +139,7 @@ public class PartitionedRepositoryManager implements RepositoryManager {
         RegionDirectory dir = new RegionDirectory(fileBucket, chunkBucket);
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         IndexWriter writer = new IndexWriter(dir, config);
-        repo = new IndexRepositoryImpl(fileBucket, writer, serializer);
+        repo = new IndexRepositoryImpl(fileBucket, writer, serializer, stats);
         IndexRepository oldRepo = indexRepositories.putIfAbsent(bucketId, repo);
         if(oldRepo != null) {
           repo = oldRepo;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
index 065cc6a..3dbbc94 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
@@ -33,6 +33,7 @@ import org.apache.lucene.search.SearcherManager;
 import org.apache.lucene.search.TopDocs;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.LuceneSerializer;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.SerializerUtil;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -50,40 +51,58 @@ public class IndexRepositoryImpl implements IndexRepository {
   private final LuceneSerializer serializer;
   private final SearcherManager searcherManager;
   private Region<?,?> region;
+  private LuceneIndexStats stats;
   
   private static final Logger logger = LogService.getLogger();
   
-  public IndexRepositoryImpl(Region<?,?> region, IndexWriter writer, LuceneSerializer serializer) throws IOException {
+  public IndexRepositoryImpl(Region<?,?> region, IndexWriter writer, LuceneSerializer serializer, LuceneIndexStats stats) throws IOException {
     this.region = region;
     this.writer = writer;
     searcherManager = new SearcherManager(writer, APPLY_ALL_DELETES, true, null);
     this.serializer = serializer;
+    this.stats = stats;
   }
 
   @Override
   public void create(Object key, Object value) throws IOException {
+    long start = stats.startUpdate();
+    try {
       Document doc = new Document();
       SerializerUtil.addKey(key, doc);
       serializer.toDocument(value, doc);
       writer.addDocument(doc);
+    } finally {
+      stats.endUpdate(start);
+    }
   }
 
   @Override
   public void update(Object key, Object value) throws IOException {
-    Document doc = new Document();
-    SerializerUtil.addKey(key, doc);
-    serializer.toDocument(value, doc);
-    writer.updateDocument(SerializerUtil.getKeyTerm(doc), doc);
+    long start = stats.startUpdate();
+    try {
+      Document doc = new Document();
+      SerializerUtil.addKey(key, doc);
+      serializer.toDocument(value, doc);
+      writer.updateDocument(SerializerUtil.getKeyTerm(doc), doc);
+    } finally {
+      stats.endUpdate(start);
+    }
   }
 
   @Override
   public void delete(Object key) throws IOException {
-    Term keyTerm = SerializerUtil.toKeyTerm(key);
-    writer.deleteDocuments(keyTerm);
+    long start = stats.startUpdate();
+    try {
+      Term keyTerm = SerializerUtil.toKeyTerm(key);
+      writer.deleteDocuments(keyTerm);
+    } finally {
+      stats.endUpdate(start);
+    }
   }
 
   @Override
   public void query(Query query, int limit, IndexResultCollector collector) throws IOException {
+    long start = stats.startQuery();
     IndexSearcher searcher = searcherManager.acquire();
     try {
       TopDocs docs = searcher.search(query, limit);
@@ -97,13 +116,19 @@ public class IndexRepositoryImpl implements IndexRepository {
       }
     } finally {
       searcherManager.release(searcher);
+      stats.endQuery(start);
     }
   }
 
   @Override
   public synchronized void commit() throws IOException {
-    writer.commit();
-    searcherManager.maybeRefresh();
+    long start = stats.startCommit();
+    try {
+      writer.commit();
+      searcherManager.maybeRefresh();
+    } finally {
+      stats.endCommit(start);
+    }
   }
 
   public IndexWriter getWriter() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
index e35de5e..4ae1efe 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
@@ -52,7 +52,7 @@ public class LuceneIndexForPartitionedRegionTest {
   public void getIndexNameReturnsCorrectName() {
     String name = "indexName";
     String regionPath = "regionName";
-    Cache cache = null;
+    Cache cache = Fakes.cache();
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     assertEquals(name, index.getName());
   }
@@ -61,7 +61,7 @@ public class LuceneIndexForPartitionedRegionTest {
   public void getRegionPathReturnsPath() {
     String name = "indexName";
     String regionPath = "regionName";
-    Cache cache = null;
+    Cache cache = Fakes.cache();
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     assertEquals(regionPath, index.getRegionPath());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
index 194f3c7..b4de0c5 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
@@ -67,6 +67,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
   Analyzer analyzer = new StandardAnalyzer();
 
   Cache cache;
+  private LuceneIndexStats stats;
 
   @Before
   public void setup() {
@@ -76,6 +77,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     LuceneServiceImpl.registerDataSerializables();
 
     cache = new CacheFactory().set("mcast-port", "0").create();
+    stats = new LuceneIndexStats(cache.getDistributedSystem(), "INDEX", "REGION");
   }
 
   @After
@@ -103,7 +105,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     PartitionedRegion fileRegion = (PartitionedRegion) regionfactory.create("fileRegion");
     PartitionedRegion chunkRegion = (PartitionedRegion) regionfactory.create("chunkRegion");
 
-    RepositoryManager manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer);
+    RepositoryManager manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer, stats);
     IndexRepository repo = manager.getRepository(userRegion, 0, null);
     assertNotNull(repo);
 
@@ -115,7 +117,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
 
     userRegion = (PartitionedRegion) regionfactory.create("userRegion");
     userRegion.put("rebalance", "test");
-    manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer);
+    manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer, stats);
     IndexRepository newRepo = manager.getRepository(userRegion, 0, null);
 
     Assert.assertNotEquals(newRepo, repo);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStatsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStatsJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStatsJUnitTest.java
new file mode 100644
index 0000000..6633f22
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexStatsJUnitTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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 static org.mockito.Mockito.*;
+
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.StatisticsFactory;
+import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+@Category(UnitTest.class)
+public class LuceneIndexStatsJUnitTest {
+
+  private Statistics statistics;
+  private LuceneIndexStats stats;
+  private StatisticsType type;
+
+  @Before
+  public void createStats() {
+    StatisticsFactory statsFactory = mock(StatisticsFactory.class);
+    statistics =  mock(Statistics.class);
+    when(statsFactory.createAtomicStatistics(any(), anyString())).thenReturn(statistics);
+    stats = new LuceneIndexStats(statsFactory, "region", "index");
+
+
+    ArgumentCaptor<StatisticsType> statsTypeCaptor = ArgumentCaptor.forClass(StatisticsType.class);
+    verify(statsFactory).createAtomicStatistics(statsTypeCaptor.capture(), anyString());
+    type = statsTypeCaptor.getValue();
+  }
+
+  @Test
+  public void shouldIncrementQueryStats() {
+
+    stats.startQuery();
+    verifyIncInt("queryExecutionsInProgress", 1);
+    stats.endQuery(5);
+    verifyIncInt("queryExecutionsInProgress", -1);
+    verifyIncInt("queryExecutions", 1);
+    //Because the initial stat time is 0 and the final time is 5, the delta is -5
+    verifyIncLong("queryExecutionTime", -5);
+  }
+
+  @Test
+  public void shouldIncrementUpdateStats() {
+
+    stats.startUpdate();
+    verifyIncInt("updatesInProgress", 1);
+    stats.endUpdate(5);
+    verifyIncInt("updatesInProgress", -1);
+    verifyIncInt("updates", 1);
+    //Because the initial stat time is 0 and the final time is 5, the delta is -5
+    verifyIncLong("updateTime", -5);
+  }
+
+  @Test
+  public void shouldIncrementCommitStats() {
+
+    stats.startCommit();
+    verifyIncInt("commitsInProgress", 1);
+    stats.endCommit(5);
+    verifyIncInt("commitsInProgress", -1);
+    verifyIncInt("commits", 1);
+    //Because the initial stat time is 0 and the final time is 5, the delta is -5
+    verifyIncLong("commitTime", -5);
+  }
+
+  @Test
+  public void shouldIncrementDocumentStat() {
+    stats.incDocuments(5);
+    verifyIncInt("documents", 5);
+  }
+
+  private void verifyIncInt(final String statName, final int value) {
+    final int statId = type.nameToId(statName);
+    verify(statistics).incInt(eq(statId), eq(value));
+  }
+
+  private void verifyIncLong(final String statName, final long value) {
+    final int statId = type.nameToId(statName);
+    verify(statistics).incLong(eq(statId), eq(value));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
index 4532d16..f0ff7ca 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
@@ -65,6 +65,7 @@ public class PartitionedRepositoryManagerJUnitTest {
   
   private Map<Integer, BucketRegion> fileBuckets = new HashMap<Integer, BucketRegion>();
   private Map<Integer, BucketRegion> chunkBuckets= new HashMap<Integer, BucketRegion>();
+  private LuceneIndexStats stats;
 
   @Before
   public void setUp() {
@@ -79,11 +80,12 @@ public class PartitionedRepositoryManagerJUnitTest {
     chunkDataStore = Mockito.mock(PartitionedRegionDataStore.class);
     Mockito.when(chunkRegion.getDataStore()).thenReturn(chunkDataStore);
     serializer = new HeterogeneousLuceneSerializer(new String[] {"a", "b"} );
+    stats = Mockito.mock(LuceneIndexStats.class);
   }
   
   @Test
   public void getByKey() throws BucketNotFoundException, IOException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
     
     setUpMockBucket(0);
     setUpMockBucket(1);
@@ -107,7 +109,7 @@ public class PartitionedRepositoryManagerJUnitTest {
    */
   @Test
   public void destroyBucket() throws BucketNotFoundException, IOException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
     
     setUpMockBucket(0);
     
@@ -133,13 +135,13 @@ public class PartitionedRepositoryManagerJUnitTest {
    */
   @Test(expected = BucketNotFoundException.class)
   public void getMissingBucketByKey() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
     repoManager.getRepository(userRegion, 0, null);
   }
   
   @Test
   public void createMissingBucket() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
     setUpMockBucket(0);
     
     Mockito.when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(null);
@@ -157,7 +159,7 @@ public class PartitionedRepositoryManagerJUnitTest {
   
   @Test
   public void getByRegion() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
     
     setUpMockBucket(0);
     setUpMockBucket(1);
@@ -185,7 +187,7 @@ public class PartitionedRepositoryManagerJUnitTest {
    */
   @Test(expected = BucketNotFoundException.class)
   public void getMissingBucketByRegion() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
     
     setUpMockBucket(0);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
index 7bcc761..4cc2d5c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
@@ -36,6 +36,7 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
@@ -51,11 +52,13 @@ public class DistributedScoringJUnitTest {
 
   private final StandardAnalyzer analyzer = new StandardAnalyzer();
   private Region<String, String> region;
+  private LuceneIndexStats stats;
 
   @Before
   public void createMocks() {
     region = Mockito.mock(Region.class);
     Mockito.when(region.isDestroyed()).thenReturn(false);
+    stats = Mockito.mock(LuceneIndexStats.class);
   }
 
   /**
@@ -142,7 +145,7 @@ public class DistributedScoringJUnitTest {
     IndexWriterConfig config = new IndexWriterConfig(analyzer);
     IndexWriter writer = new IndexWriter(dir, config);
 
-    return new IndexRepositoryImpl(region, writer, mapper);
+    return new IndexRepositoryImpl(region, writer, mapper, stats);
   }
 
   private static class TestType {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
index 53c4161..f2a6bc4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
@@ -20,6 +20,7 @@
 package com.gemstone.gemfire.cache.lucene.internal.repository;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -40,6 +41,7 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
@@ -60,6 +62,7 @@ public class IndexRepositoryImplJUnitTest {
   private StandardAnalyzer analyzer = new StandardAnalyzer();
   private IndexWriter writer;
   private Region region;
+  private LuceneIndexStats stats;
 
   @Before
   public void setUp() throws IOException {
@@ -71,8 +74,9 @@ public class IndexRepositoryImplJUnitTest {
     String[] indexedFields= new String[] {"s", "i", "l", "d", "f", "s2", "missing"};
     mapper = new HeterogeneousLuceneSerializer(indexedFields);
     region = Mockito.mock(Region.class);
+    stats = Mockito.mock(LuceneIndexStats.class);
     Mockito.when(region.isDestroyed()).thenReturn(false);
-    repo = new IndexRepositoryImpl(region, writer, mapper);
+    repo = new IndexRepositoryImpl(region, writer, mapper, stats);
   }
   
   @Test
@@ -82,7 +86,7 @@ public class IndexRepositoryImplJUnitTest {
     repo.create("key3", new Type2("Voodoo Doll doughnut", 1, 2L, 3.0, 4.0f, "Toasted coconut doughnut"));
     repo.create("key4", new Type2("Portland Cream doughnut", 1, 2L, 3.0, 4.0f, "Captain my Captain doughnut"));
     repo.commit();
-    
+
     checkQuery("Cream", "s", "key2", "key4");
     checkQuery("NotARealWord", "s");
   }
@@ -108,6 +112,41 @@ public class IndexRepositoryImplJUnitTest {
     updateAndRemove(key1, key2, key3, key4);
   }
 
+  @Test
+  public void createShouldUpdateStats() throws IOException {
+    repo.create("key1", new Type2("bar", 1, 2L, 3.0, 4.0f, "Grape Ape doughnut"));
+    verify(stats, times(1)).startUpdate();
+    verify(stats, times(1)).endUpdate(anyLong());
+  }
+
+  @Test
+  public void updateShouldUpdateStats() throws IOException {
+    repo.update("key1", new Type2("bacon maple bar", 1, 2L, 3.0, 4.0f, "Grape Ape doughnut"));
+    verify(stats, times(1)).startUpdate();
+    verify(stats, times(1)).endUpdate(anyLong());
+  }
+
+  @Test
+  public void deleteShouldUpdateStats() throws IOException {
+    repo.delete("key1");
+    verify(stats, times(1)).startUpdate();
+    verify(stats, times(1)).endUpdate(anyLong());
+  }
+
+  @Test
+  public void commitShouldUpdateStats() throws IOException {
+    repo.commit();
+    verify(stats, times(1)).startCommit();
+    verify(stats, times(1)).endCommit(anyLong());
+  }
+
+  @Test
+  public void queryShouldUpdateStats() throws IOException, ParseException {
+    checkQuery("NotARealWord", "s");
+    verify(stats, times(1)).startQuery();
+    verify(stats, times(1)).endQuery(anyLong());
+  }
+
   private void updateAndRemove(Object key1, Object key2, Object key3,
       Object key4) throws IOException, ParseException {
     repo.create(key1, new Type2("bacon maple bar", 1, 2L, 3.0, 4.0f, "Grape Ape doughnut"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8fe7af/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index cec76ba..9349985 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -60,6 +60,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
@@ -114,13 +115,14 @@ public class IndexRepositoryImplPerformanceTest {
         Region<ChunkKey, byte[]> chunkRegion = cache.<ChunkKey, byte[]>createRegionFactory(RegionShortcut.REPLICATE).create("chunks");
 
         RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
+        final LuceneIndexStats stats = new LuceneIndexStats(cache.getDistributedSystem(), "region", "index");
         
         
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         writer = new IndexWriter(dir, config);
         String[] indexedFields= new String[] {"text"};
         HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(indexedFields);
-        repo = new IndexRepositoryImpl(fileRegion, writer, mapper);
+        repo = new IndexRepositoryImpl(fileRegion, writer, mapper, stats);
       }
 
       @Override