You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2016/06/01 20:09:37 UTC

incubator-geode git commit: GEODE-11 Adding stats for the lucene file system.

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 3f356594e -> 1c8688243


GEODE-11 Adding stats for the lucene file system.

Adding stats to track changes to the lucene file system.


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

Branch: refs/heads/develop
Commit: 1c8688243ae70f8a13977abf9db52edd31fc5ab1
Parents: 3f35659
Author: Dan Smith <up...@apache.org>
Authored: Tue May 31 14:09:52 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed Jun 1 13:05:23 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexForPartitionedRegion.java        |   7 +-
 .../cache/lucene/internal/LuceneIndexImpl.java  |   8 +-
 .../cache/lucene/internal/LuceneIndexStats.java |  11 +-
 .../internal/PartitionedRepositoryManager.java  |  15 +--
 .../internal/directory/RegionDirectory.java     |   5 +-
 .../lucene/internal/filesystem/FileSystem.java  |  15 ++-
 .../internal/filesystem/FileSystemStats.java    |  92 +++++++++++++++++
 .../repository/IndexRepositoryImpl.java         |   4 +-
 .../LuceneIndexRecoveryHAIntegrationTest.java   |  23 ++---
 .../internal/LuceneIndexStatsJUnitTest.java     |   5 +-
 .../PartitionedRepositoryManagerJUnitTest.java  |  25 +++--
 .../directory/RegionDirectoryJUnitTest.java     |   6 +-
 .../DistributedScoringJUnitTest.java            |  11 +-
 .../filesystem/FileSystemJUnitTest.java         |  70 +++++++++++--
 .../filesystem/FileSystemStatsJUnitTest.java    | 100 +++++++++++++++++++
 .../IndexRepositoryImplJUnitTest.java           |  12 ++-
 .../IndexRepositoryImplPerformanceTest.java     |  14 ++-
 17 files changed, 357 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 8f2cb10..3def888 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
@@ -31,14 +31,18 @@ import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
 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.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 
 /* wrapper of IndexWriter */
 public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
+  FileSystemStats fileSystemStats;
 
   public LuceneIndexForPartitionedRegion(String indexName, String regionPath, Cache cache) {
     super(indexName, regionPath, cache);
+    final String statsName = indexName + "-" + regionPath;
+    this.fileSystemStats = new FileSystemStats(cache.getDistributedSystem(), statsName);
   }
 
   @Override
@@ -84,7 +88,8 @@ 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, this.stats);
+      repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion) fileRegion,
+        (PartitionedRegion) chunkRegion, mapper, analyzer, this.indexStats, this.fileSystemStats);
       
       // create AEQ, AEQ listener and specify the listener to repositoryManager
       createAEQ(dataRegion);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 3b58a95..c75148a 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
@@ -34,6 +34,7 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 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.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.xml.LuceneIndexCreation;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -52,8 +53,8 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   
   Region<String, File> fileRegion;
   Region<ChunkKey, byte[]> chunkRegion;
-  LuceneIndexStats stats;
-  
+  LuceneIndexStats indexStats;
+
   protected String indexName;
   protected String regionPath;
   protected boolean hasInitialized = false;
@@ -65,7 +66,8 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     this.indexName = indexName;
     this.regionPath = regionPath;
     this.cache = cache;
-    this.stats = new LuceneIndexStats(cache.getDistributedSystem(), indexName, regionPath);
+    final String statsName = indexName + "-" + regionPath;
+    this.indexStats = new LuceneIndexStats(cache.getDistributedSystem(), statsName);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 dab1b0f..8dd34a4 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
@@ -25,7 +25,6 @@ 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 {
@@ -37,6 +36,7 @@ public class LuceneIndexStats {
   private static final int queryExecutionsId;
   private static final int queryExecutionTimeId;
   private static final int queryExecutionsInProgressId;
+  private static final int queryExecutionTotalHits;
   private static final int updatesId;
   private static final int updateTimeId;
   private static final int updatesInProgressId;
@@ -56,6 +56,7 @@ public class LuceneIndexStats {
         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.createLongCounter("queryExecutionTotalHits", "Total number of documents returned by query executions", "entries"),
         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"),
@@ -69,6 +70,7 @@ public class LuceneIndexStats {
     queryExecutionsId = statsType.nameToId("queryExecutions");
     queryExecutionTimeId = statsType.nameToId("queryExecutionTime");
     queryExecutionsInProgressId = statsType.nameToId("queryExecutionsInProgress");
+    queryExecutionTotalHits = statsType.nameToId("queryExecutionTotalHits");
     updatesId = statsType.nameToId("updates");
     updateTimeId = statsType.nameToId("updateTime");
     updatesInProgressId = statsType.nameToId("updatesInProgress");
@@ -78,8 +80,8 @@ public class LuceneIndexStats {
     documentsId = statsType.nameToId("documents");
   }
 
-  public LuceneIndexStats(StatisticsFactory f, String indexName, String regionName) {
-    this.stats = f.createAtomicStatistics(statsType, indexName + "-" + regionName);
+  public LuceneIndexStats(StatisticsFactory f, String name) {
+    this.stats = f.createAtomicStatistics(statsType, name);
   }
 
   /**
@@ -92,10 +94,11 @@ public class LuceneIndexStats {
   /**
    * @param start the timestamp taken when the operation started
    */
-  public void endQuery(long start) {
+  public void endQuery(long start, final int totalHits) {
     stats.incLong(queryExecutionTimeId, getStatTime()-start);
     stats.incInt(queryExecutionsInProgressId, -1);
     stats.incInt(queryExecutionsId, 1);
+    stats.incLong(queryExecutionTotalHits, totalHits);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 1b48316..25010b0 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
@@ -27,12 +27,12 @@ import java.util.Set;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 
 import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepositoryImpl;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
@@ -69,7 +69,8 @@ public class PartitionedRepositoryManager implements RepositoryManager {
   private final PartitionedRegion chunkRegion;
   private final LuceneSerializer serializer;
   private final Analyzer analyzer;
-  private final LuceneIndexStats stats;
+  private final LuceneIndexStats indexStats;
+  private final FileSystemStats fileSystemStats;
 
   /**
    * 
@@ -83,13 +84,15 @@ public class PartitionedRepositoryManager implements RepositoryManager {
                                       PartitionedRegion chunkRegion,
                                       LuceneSerializer serializer,
                                       Analyzer analyzer,
-                                      LuceneIndexStats stats) {
+                                      LuceneIndexStats indexStats,
+                                      FileSystemStats fileSystemStats) {
     this.userRegion = userRegion;
     this.fileRegion = fileRegion;
     this.chunkRegion = chunkRegion;
     this.serializer = serializer;
     this.analyzer = analyzer;
-    this.stats = stats;
+    this.indexStats = indexStats;
+    this.fileSystemStats = fileSystemStats;
   }
 
   @Override
@@ -136,10 +139,10 @@ public class PartitionedRepositoryManager implements RepositoryManager {
       try {
         BucketRegion fileBucket = getMatchingBucket(fileRegion, bucketId);
         BucketRegion chunkBucket = getMatchingBucket(chunkRegion, bucketId);
-        RegionDirectory dir = new RegionDirectory(fileBucket, chunkBucket);
+        RegionDirectory dir = new RegionDirectory(fileBucket, chunkBucket, fileSystemStats);
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         IndexWriter writer = new IndexWriter(dir, config);
-        repo = new IndexRepositoryImpl(fileBucket, writer, serializer, stats);
+        repo = new IndexRepositoryImpl(fileBucket, writer, serializer, indexStats);
         IndexRepository oldRepo = indexRepositories.putIfAbsent(bucketId, repo);
         if(oldRepo != null) {
           repo = oldRepo;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
index 28117bd..45b9c97 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
@@ -36,6 +36,7 @@ import org.apache.lucene.store.SingleInstanceLockFactory;
 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.filesystem.FileSystem;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 
 /**
  * An implementation of Directory that stores data in geode regions.
@@ -52,9 +53,9 @@ public class RegionDirectory extends BaseDirectory {
    * Create a region directory with a given file and chunk region. These regions
    * may be bucket regions or they may be replicated regions.
    */
-  public RegionDirectory(ConcurrentMap<String, File> fileRegion, ConcurrentMap<ChunkKey, byte[]> chunkRegion) {
+  public RegionDirectory(ConcurrentMap<String, File> fileRegion, ConcurrentMap<ChunkKey, byte[]> chunkRegion, FileSystemStats stats) {
     super(new SingleInstanceLockFactory());
-    fs = new FileSystem(fileRegion, chunkRegion);
+    fs = new FileSystem(fileRegion, chunkRegion, stats);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
index 5f5218a..5f4fb77 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
@@ -41,6 +41,7 @@ public class FileSystem {
   private final ConcurrentMap<ChunkKey, byte[]> chunkRegion;
   
   static final int CHUNK_SIZE = 1024 * 1024; //1 MB
+  private final FileSystemStats stats;
 
   /**
    * Create filesystem that will store data in the two provided regions. The fileRegion contains
@@ -50,10 +51,10 @@ public class FileSystem {
    * @param fileRegion the region to store metadata about the files
    * @param chunkRegion the region to store actual file data.
    */
-  public FileSystem(ConcurrentMap<String, File> fileRegion, ConcurrentMap<ChunkKey, byte[]> chunkRegion) {
-    super();
+  public FileSystem(ConcurrentMap<String, File> fileRegion, ConcurrentMap<ChunkKey, byte[]> chunkRegion, FileSystemStats stats) {
     this.fileRegion = fileRegion;
     this.chunkRegion = chunkRegion;
+    this.stats = stats;
   }
 
   public Collection<String> listFileNames() {
@@ -66,12 +67,14 @@ public class FileSystem {
     if (null != fileRegion.putIfAbsent(name, file)) {
       throw new IOException("File exists.");
     }
+    stats.incFileCreates(1);
     // TODO unlock region ?
     return file;
   }
 
   public File createTemporaryFile(final String name) throws IOException {
     final File file = new File(this, name);
+    stats.incTemporaryFileCreates(1);
     return file;
   }
   
@@ -97,7 +100,7 @@ public class FileSystem {
     if(file == null) {
       throw new FileNotFoundException(name);
     }
-    
+
     // TODO consider removeAll with all ChunkKeys listed.
     final ChunkKey key = new ChunkKey(file.id, 0);
     while (true) {
@@ -108,6 +111,8 @@ public class FileSystem {
       }
       key.chunkId++;
     }
+
+    stats.incFileDeletes(1);
   }
   
   public void renameFile(String source, String dest) throws IOException {
@@ -131,6 +136,8 @@ public class FileSystem {
     // at the same data
 
     fileRegion.remove(source);
+
+    stats.incFileRenames(1);
   }
   
   byte[] getChunk(final File file, final int id) {
@@ -148,12 +155,14 @@ public class FileSystem {
     }
     
     final byte[] chunk = chunkRegion.get(key);
+    stats.incReadBytes(chunk.length);
     return chunk;
   }
 
   public void putChunk(final File file, final int id, final byte[] chunk) {
     final ChunkKey key = new ChunkKey(file.id, id);
     chunkRegion.put(key, chunk);
+    stats.incWrittenBytes(chunk.length);
   }
 
   void updateFile(File file) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStats.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStats.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStats.java
new file mode 100644
index 0000000..1f609fd
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStats.java
@@ -0,0 +1,92 @@
+/*
+ * 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.filesystem;
+
+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.internal.StatisticsTypeFactoryImpl;
+
+public class FileSystemStats {
+  private static final StatisticsType statsType;
+  private static final String statsTypeName = "FileSystemStats";
+  private static final String statsTypeDescription = "Statistics about in memory file system implementation";
+
+  private final Statistics stats;
+
+  private static final int readBytesId;
+  private static final int writtenBytesId;
+  private static final int fileCreatesId;
+  private static final int temporaryFileCreatesId;
+  private static final int fileDeletesId;
+  private static final int fileRenamesId;
+
+  static {
+    final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+    statsType = f.createType(
+      statsTypeName,
+      statsTypeDescription,
+      new StatisticDescriptor[] {
+        f.createLongCounter("readBytes", "Number of bytes written", "bytes"),
+        f.createLongCounter("writtenBytes", "Number of bytes read", "bytes"),
+        f.createIntCounter("fileCreates", "Number files created", "files"),
+        f.createIntCounter("temporaryFileCreates", "Number temporary files created", "files"),
+        f.createIntCounter("fileDeletes", "Number files deleted", "files"),
+        f.createIntCounter("fileRenames", "Number files renamed", "files"),
+      }
+    );
+
+    readBytesId = statsType.nameToId("readBytes");
+    writtenBytesId = statsType.nameToId("writtenBytes");
+    fileCreatesId = statsType.nameToId("fileCreates");
+    temporaryFileCreatesId = statsType.nameToId("temporaryFileCreates");
+    fileDeletesId = statsType.nameToId("fileDeletes");
+    fileRenamesId = statsType.nameToId("fileRenames");
+  }
+
+  public FileSystemStats(StatisticsFactory f, String name) {
+    this.stats = f.createAtomicStatistics(statsType, name);
+  }
+
+  public void incReadBytes(int delta) {
+    stats.incLong(readBytesId, delta);
+  }
+
+  public void incWrittenBytes(int delta) {
+    stats.incLong(writtenBytesId, delta);
+  }
+
+  public void incFileCreates(final int delta) {
+    stats.incInt(fileCreatesId,delta);
+  }
+
+  public void incTemporaryFileCreates(final int delta) {
+    stats.incInt(temporaryFileCreatesId, delta);
+  }
+
+  public void incFileDeletes(final int delta) {
+    stats.incInt(fileDeletesId,delta);
+  }
+
+  public void incFileRenames(final int delta) {
+    stats.incInt(fileRenamesId,delta);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 47e43b5..ee4e5bb 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
@@ -103,9 +103,11 @@ public class IndexRepositoryImpl implements IndexRepository {
   @Override
   public void query(Query query, int limit, IndexResultCollector collector) throws IOException {
     long start = stats.startQuery();
+    int totalHits = 0;
     IndexSearcher searcher = searcherManager.acquire();
     try {
       TopDocs docs = searcher.search(query, limit);
+      totalHits = docs.totalHits;
       for(ScoreDoc scoreDoc : docs.scoreDocs) {
         Document doc = searcher.doc(scoreDoc.doc);
         Object key = SerializerUtil.getKey(doc);
@@ -116,7 +118,7 @@ public class IndexRepositoryImpl implements IndexRepository {
       }
     } finally {
       searcherManager.release(searcher);
-      stats.endQuery(start);
+      stats.endQuery(start, totalHits);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 b4de0c5..70b39e7 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
@@ -21,7 +21,6 @@ package com.gemstone.gemfire.cache.lucene.internal;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
@@ -33,28 +32,19 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.EvictionAction;
-import com.gemstone.gemfire.cache.EvictionAlgorithm;
 import com.gemstone.gemfire.cache.PartitionAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
-import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
-import com.gemstone.gemfire.cache.lucene.LuceneQuery;
-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.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.Type1;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
-import com.gemstone.gemfire.internal.cache.EvictionAttributesImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 @Category(IntegrationTest.class)
@@ -67,7 +57,8 @@ public class LuceneIndexRecoveryHAIntegrationTest {
   Analyzer analyzer = new StandardAnalyzer();
 
   Cache cache;
-  private LuceneIndexStats stats;
+  private LuceneIndexStats indexStats;
+  private FileSystemStats fileSystemStats;
 
   @Before
   public void setup() {
@@ -77,7 +68,8 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     LuceneServiceImpl.registerDataSerializables();
 
     cache = new CacheFactory().set("mcast-port", "0").create();
-    stats = new LuceneIndexStats(cache.getDistributedSystem(), "INDEX", "REGION");
+    indexStats = new LuceneIndexStats(cache.getDistributedSystem(), "INDEX-REGION");
+    fileSystemStats = new FileSystemStats(cache.getDistributedSystem(), "INDEX-REGION");
   }
 
   @After
@@ -105,7 +97,8 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     PartitionedRegion fileRegion = (PartitionedRegion) regionfactory.create("fileRegion");
     PartitionedRegion chunkRegion = (PartitionedRegion) regionfactory.create("chunkRegion");
 
-    RepositoryManager manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer, stats);
+    RepositoryManager manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer,
+      indexStats, fileSystemStats);
     IndexRepository repo = manager.getRepository(userRegion, 0, null);
     assertNotNull(repo);
 
@@ -117,7 +110,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
 
     userRegion = (PartitionedRegion) regionfactory.create("userRegion");
     userRegion.put("rebalance", "test");
-    manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer, stats);
+    manager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, mapper, analyzer, indexStats, fileSystemStats);
     IndexRepository newRepo = manager.getRepository(userRegion, 0, null);
 
     Assert.assertNotEquals(newRepo, repo);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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
index 6633f22..3e6d44c 100644
--- 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
@@ -42,7 +42,7 @@ public class LuceneIndexStatsJUnitTest {
     StatisticsFactory statsFactory = mock(StatisticsFactory.class);
     statistics =  mock(Statistics.class);
     when(statsFactory.createAtomicStatistics(any(), anyString())).thenReturn(statistics);
-    stats = new LuceneIndexStats(statsFactory, "region", "index");
+    stats = new LuceneIndexStats(statsFactory, "region-index");
 
 
     ArgumentCaptor<StatisticsType> statsTypeCaptor = ArgumentCaptor.forClass(StatisticsType.class);
@@ -55,9 +55,10 @@ public class LuceneIndexStatsJUnitTest {
 
     stats.startQuery();
     verifyIncInt("queryExecutionsInProgress", 1);
-    stats.endQuery(5);
+    stats.endQuery(5, 2);
     verifyIncInt("queryExecutionsInProgress", -1);
     verifyIncInt("queryExecutions", 1);
+    verifyIncLong("queryExecutionTotalHits", 2);
     //Because the initial stat time is 0 and the final time is 5, the delta is -5
     verifyIncLong("queryExecutionTime", -5);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 f0ff7ca..ec56381 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
@@ -40,6 +40,7 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepositoryImpl;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
@@ -65,7 +66,8 @@ public class PartitionedRepositoryManagerJUnitTest {
   
   private Map<Integer, BucketRegion> fileBuckets = new HashMap<Integer, BucketRegion>();
   private Map<Integer, BucketRegion> chunkBuckets= new HashMap<Integer, BucketRegion>();
-  private LuceneIndexStats stats;
+  private LuceneIndexStats indexStats;
+  private FileSystemStats fileSystemStats;
 
   @Before
   public void setUp() {
@@ -80,12 +82,14 @@ 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);
+    indexStats = Mockito.mock(LuceneIndexStats.class);
+    fileSystemStats = Mockito.mock(FileSystemStats.class);
   }
   
   @Test
   public void getByKey() throws BucketNotFoundException, IOException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
+      indexStats, fileSystemStats);
     
     setUpMockBucket(0);
     setUpMockBucket(1);
@@ -109,7 +113,8 @@ public class PartitionedRepositoryManagerJUnitTest {
    */
   @Test
   public void destroyBucket() throws BucketNotFoundException, IOException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
+      indexStats, fileSystemStats);
     
     setUpMockBucket(0);
     
@@ -135,13 +140,15 @@ public class PartitionedRepositoryManagerJUnitTest {
    */
   @Test(expected = BucketNotFoundException.class)
   public void getMissingBucketByKey() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
+      indexStats, fileSystemStats);
     repoManager.getRepository(userRegion, 0, null);
   }
   
   @Test
   public void createMissingBucket() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
+      indexStats, fileSystemStats);
     setUpMockBucket(0);
     
     Mockito.when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(null);
@@ -159,7 +166,8 @@ public class PartitionedRepositoryManagerJUnitTest {
   
   @Test
   public void getByRegion() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
+      indexStats, fileSystemStats);
     
     setUpMockBucket(0);
     setUpMockBucket(1);
@@ -187,7 +195,8 @@ public class PartitionedRepositoryManagerJUnitTest {
    */
   @Test(expected = BucketNotFoundException.class)
   public void getMissingBucketByRegion() throws BucketNotFoundException {
-    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(), stats);
+    PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
+      indexStats, fileSystemStats);
     
     setUpMockBucket(0);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
index e8118c5..8c5843b 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
@@ -18,6 +18,8 @@
  */
 package com.gemstone.gemfire.cache.lucene.internal.directory;
 
+import static org.mockito.Mockito.mock;
+
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.concurrent.ConcurrentHashMap;
@@ -30,6 +32,7 @@ import org.junit.experimental.categories.Category;
 
 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.filesystem.FileSystemStats;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
@@ -47,11 +50,12 @@ public class RegionDirectoryJUnitTest extends BaseDirectoryTestCase {
 
   @Override
   protected Directory getDirectory(Path path) throws IOException {
+    final FileSystemStats stats = mock(FileSystemStats.class);
     
     //This is super lame, but log4j automatically sets the system property, and the lucene
     //test asserts that no system properties have changed. Unfortunately, there is no
     //way to control the order of rules, so we can't clear this property with a rule
     //or @After method. Instead, do it in the close method of the directory.
-    return new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
+    return new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>(), stats);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 4cc2d5c..4abc97c 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
@@ -40,6 +40,7 @@ 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;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepositoryImpl;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
@@ -52,13 +53,15 @@ public class DistributedScoringJUnitTest {
 
   private final StandardAnalyzer analyzer = new StandardAnalyzer();
   private Region<String, String> region;
-  private LuceneIndexStats stats;
+  private LuceneIndexStats indexStats;
+  private FileSystemStats fileSystemStats;
 
   @Before
   public void createMocks() {
     region = Mockito.mock(Region.class);
     Mockito.when(region.isDestroyed()).thenReturn(false);
-    stats = Mockito.mock(LuceneIndexStats.class);
+    indexStats = Mockito.mock(LuceneIndexStats.class);
+    fileSystemStats = Mockito.mock(FileSystemStats.class);
   }
 
   /**
@@ -140,12 +143,12 @@ public class DistributedScoringJUnitTest {
   private IndexRepositoryImpl createIndexRepo() throws IOException {
     ConcurrentHashMap<String, File> fileRegion = new ConcurrentHashMap<String, File>();
     ConcurrentHashMap<ChunkKey, byte[]> chunkRegion = new ConcurrentHashMap<ChunkKey, byte[]>();
-    RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
+    RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion, fileSystemStats);
 
     IndexWriterConfig config = new IndexWriterConfig(analyzer);
     IndexWriter writer = new IndexWriter(dir, config);
 
-    return new IndexRepositoryImpl(region, writer, mapper, stats);
+    return new IndexRepositoryImpl(region, writer, mapper, indexStats);
   }
 
   private static class TestType {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
index 78bf1de..a72f7d1 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
@@ -19,6 +19,7 @@
 package com.gemstone.gemfire.cache.lucene.internal.filesystem;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -39,6 +40,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -60,12 +62,14 @@ public class FileSystemJUnitTest {
 
   @Rule
   public DiskDirRule dirRule = new DiskDirRule();
+  private FileSystemStats fileSystemStats;
 
   @Before
   public void setUp() {
     fileRegion = new ConcurrentHashMap<String, File>();
     chunkRegion = new ConcurrentHashMap<ChunkKey, byte[]>();
-    system = new FileSystem(fileRegion, chunkRegion);
+    fileSystemStats = mock(FileSystemStats.class);
+    system = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
   }
   
   /**
@@ -316,7 +320,7 @@ public class FileSystemJUnitTest {
     byte[] bytes = getRandomBytes(size );
     file1.getOutputStream().write(bytes);
     
-    FileSystem system2 = new FileSystem(fileRegion, chunkRegion);
+    FileSystem system2 = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
     File file = system2.getFile(name1);
     
     assertTrue(file.getLength() <= bytes.length);
@@ -354,10 +358,10 @@ public class FileSystemJUnitTest {
     //Create a couple of mock regions where we count the operations
     //that happen to them. We will then use this to abort the rename
     //in the middle.
-    ConcurrentHashMap<String, File> spyFileRegion = Mockito.mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileRegion));
-    ConcurrentHashMap<ChunkKey, byte[]> spyChunkRegion = Mockito.mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, chunkRegion));
+    ConcurrentHashMap<String, File> spyFileRegion = mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileRegion));
+    ConcurrentHashMap<ChunkKey, byte[]> spyChunkRegion = mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, chunkRegion));
     
-    system = new FileSystem(spyFileRegion, spyChunkRegion);
+    system = new FileSystem(spyFileRegion, spyChunkRegion, fileSystemStats);
     
     String name = "file";
     File file = system.createFile(name);
@@ -393,7 +397,7 @@ public class FileSystemJUnitTest {
       
     }
     
-    system = new FileSystem(fileRegion, chunkRegion);
+    system = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
     
     //This is not the ideal behavior. We are left
     //with two duplicate files. However, we will still
@@ -421,10 +425,10 @@ public class FileSystemJUnitTest {
     //Create a couple of mock regions where we count the operations
     //that happen to them. We will then use this to abort the rename
     //in the middle.
-    ConcurrentHashMap<String, File> spyFileRegion = Mockito.mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileRegion));
-    ConcurrentHashMap<ChunkKey, byte[]> spyChunkRegion = Mockito.mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, chunkRegion));
+    ConcurrentHashMap<String, File> spyFileRegion = mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileRegion));
+    ConcurrentHashMap<ChunkKey, byte[]> spyChunkRegion = mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, chunkRegion));
     
-    system = new FileSystem(spyFileRegion, spyChunkRegion);
+    system = new FileSystem(spyFileRegion, spyChunkRegion, fileSystemStats);
     
     String name1 = "file1";
     String name2 = "file2";
@@ -461,7 +465,7 @@ public class FileSystemJUnitTest {
     } catch(CacheClosedException expectedException) {
     }
     
-    system = new FileSystem(fileRegion, chunkRegion);
+    system = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
     
     if(system.listFileNames().size() == 0) {
       //File was deleted, but shouldn't have any dangling chunks at this point
@@ -492,6 +496,52 @@ public class FileSystemJUnitTest {
     assertExportedFileContents(file2Data, new java.io.File(parentDir, "testFile2"));
   }
 
+  @Test
+  public void testIncrementFileCreates() throws IOException {
+    File file= system.createFile("file");
+    verify(fileSystemStats).incFileCreates(1);
+  }
+  @Test
+  public void testIncrementFileDeletes() throws IOException {
+    File file= system.createFile("file");
+    system.deleteFile("file");
+    verify(fileSystemStats).incFileDeletes(1);
+  }
+
+  @Test
+  public void testIncrementFileRenames() throws IOException {
+    File file= system.createFile("file");
+    system.renameFile("file", "dest");
+    verify(fileSystemStats).incFileRenames(1);
+  }
+
+  @Test
+  public void testIncrementTemporaryFileCreates() throws IOException {
+    File file= system.createTemporaryFile("file");
+    verify(fileSystemStats).incTemporaryFileCreates(1);
+  }
+
+  @Test
+  public void testIncrementWrittenBytes() throws IOException {
+    File file= system.createTemporaryFile("file");
+    final byte[] bytes = writeRandomBytes(file);
+    ArgumentCaptor<Integer> captor = ArgumentCaptor.forClass(Integer.class);
+    verify(fileSystemStats, atLeast(1)).incWrittenBytes(captor.capture());
+    final int actualByteCount = captor.getAllValues().stream().mapToInt(Integer::intValue).sum();
+    assertEquals(bytes.length, actualByteCount);
+  }
+
+  @Test
+  public void testIncrementReadBytes() throws IOException {
+    File file= system.createTemporaryFile("file");
+    final byte[] bytes = writeRandomBytes(file);
+    file.getInputStream().read(bytes);
+    ArgumentCaptor<Integer> captor = ArgumentCaptor.forClass(Integer.class);
+    verify(fileSystemStats, atLeast(1)).incReadBytes(captor.capture());
+    final int actualByteCount = captor.getAllValues().stream().mapToInt(Integer::intValue).sum();
+    assertEquals(bytes.length, actualByteCount);
+  }
+
   private void assertExportedFileContents(final byte[] expected, final java.io.File exportedFile) throws IOException {
     byte[] actual = Files.readAllBytes(exportedFile.toPath());
     assertArrayEquals(expected, actual);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStatsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStatsJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStatsJUnitTest.java
new file mode 100644
index 0000000..e8400c5
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemStatsJUnitTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.filesystem;
+
+import static org.junit.Assert.*;
+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 FileSystemStatsJUnitTest {
+
+  private Statistics statistics;
+  private FileSystemStats 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 FileSystemStats(statsFactory, "stats");
+
+
+    ArgumentCaptor<StatisticsType> statsTypeCaptor = ArgumentCaptor.forClass(StatisticsType.class);
+    verify(statsFactory).createAtomicStatistics(statsTypeCaptor.capture(), anyString());
+    type = statsTypeCaptor.getValue();
+  }
+
+  @Test
+  public void shouldIncrementReadBytes() {
+    stats.incReadBytes(5);
+    verifyIncLong("readBytes", 5);
+  }
+
+  @Test
+  public void shouldIncrementWrittenBytes() {
+    stats.incWrittenBytes(5);
+    verifyIncLong("writtenBytes", 5);
+  }
+
+  @Test
+  public void shouldIncrementFileCreates() {
+    stats.incFileCreates(5);
+    verifyIncInt("fileCreates", 5);
+  }
+
+  @Test
+  public void shouldIncrementFileDeletes() {
+    stats.incFileDeletes(5);
+    verifyIncInt("fileDeletes", 5);
+  }
+
+  @Test
+  public void shouldIncrementFileRenames() {
+    stats.incFileRenames(5);
+    verifyIncInt("fileRenames", 5);
+  }
+
+  @Test
+  public void shouldIncrementTemporyFileCreates() {
+    stats.incTemporaryFileCreates(5);
+    verifyIncInt("temporaryFileCreates", 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/1c868824/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 f2a6bc4..6004623 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
@@ -45,6 +45,7 @@ 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;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.Type2;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
@@ -63,12 +64,14 @@ public class IndexRepositoryImplJUnitTest {
   private IndexWriter writer;
   private Region region;
   private LuceneIndexStats stats;
+  private FileSystemStats fileSystemStats;
 
   @Before
   public void setUp() throws IOException {
     ConcurrentHashMap<String, File> fileRegion = new ConcurrentHashMap<String, File>();
     ConcurrentHashMap<ChunkKey, byte[]> chunkRegion = new ConcurrentHashMap<ChunkKey, byte[]>();
-    RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
+    fileSystemStats = mock(FileSystemStats.class);
+    RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion, fileSystemStats);
     IndexWriterConfig config = new IndexWriterConfig(analyzer);
     writer = new IndexWriter(dir, config);
     String[] indexedFields= new String[] {"s", "i", "l", "d", "f", "s2", "missing"};
@@ -142,9 +145,12 @@ public class IndexRepositoryImplJUnitTest {
 
   @Test
   public void queryShouldUpdateStats() throws IOException, ParseException {
-    checkQuery("NotARealWord", "s");
+    repo.create("key2", new Type2("McMinnville Cream doughnut", 1, 2L, 3.0, 4.0f, "Captain my Captain 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");
     verify(stats, times(1)).startQuery();
-    verify(stats, times(1)).endQuery(anyLong());
+    verify(stats, times(1)).endQuery(anyLong(), eq(2));
   }
 
   private void updateAndRemove(Object key1, Object key2, Object key3,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c868824/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 9349985..90c6032 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
@@ -66,8 +66,10 @@ import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
 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.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 
 /**
@@ -114,8 +116,8 @@ public class IndexRepositoryImplPerformanceTest {
         Region<String, File> fileRegion = cache.<String, File>createRegionFactory(RegionShortcut.REPLICATE).create("files");
         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");
+        RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion, new FileSystemStats(cache.getDistributedSystem(), "region-index"));
+        final LuceneIndexStats stats = new LuceneIndexStats(cache.getDistributedSystem(), "region-index");
         
         
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
@@ -216,12 +218,17 @@ public class IndexRepositoryImplPerformanceTest {
   public  void testLuceneWithRegionDirectory() throws Exception {
     doTest("RegionDirectory", new TestCallbacks() {
 
+      public Cache cache;
       private IndexWriter writer;
       private SearcherManager searcherManager;
 
       @Override
       public void init() throws Exception {
-        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
+        cache = new CacheFactory().set("mcast-port", "0")
+          .set("log-level", "warning")
+          .create();
+        final FileSystemStats stats = new FileSystemStats(cache.getDistributedSystem(), "stats");
+        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>(), stats);
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         writer = new IndexWriter(dir, config);
         searcherManager = new SearcherManager(writer, true, true, null);
@@ -244,6 +251,7 @@ public class IndexRepositoryImplPerformanceTest {
       @Override
       public void cleanup() throws Exception {
         writer.close();
+        cache.close();;
       }
       
       @Override