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

[03/23] incubator-geode git commit: GEODE-11: Adding stats for lucene documents, files, and chunks

GEODE-11: Adding stats for lucene documents, files, and chunks

Using the new stat suppliers added with GEODE-1494 to sample useful
lucene statistics:
 * Number of documents
 * Number of files
 * Number of chunks
 * Number of bytes


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

Branch: refs/heads/feature/GEODE-835
Commit: 615e643cf989ab709aa3880d89ad04dfffa530f0
Parents: 01848b6
Author: Dan Smith <up...@apache.org>
Authored: Wed Jun 8 14:33:15 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Mon Jun 13 10:45:13 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexForPartitionedRegion.java        | 14 +++++--
 .../cache/lucene/internal/LuceneIndexImpl.java  | 32 ++++++++------
 .../cache/lucene/internal/LuceneIndexStats.java | 24 ++++++++++-
 .../internal/filesystem/FileSystemStats.java    | 44 ++++++++++++++++++--
 .../repository/IndexRepositoryImpl.java         | 13 ++++++
 .../LuceneIndexMaintenanceIntegrationTest.java  | 30 +++++++++++++
 .../internal/LuceneIndexStatsJUnitTest.java     | 17 ++++++--
 .../IndexRepositoryImplJUnitTest.java           | 12 ++++++
 8 files changed, 162 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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 8fe5fac..d283e29 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
@@ -40,12 +40,9 @@ 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
@@ -88,6 +85,9 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       if (!chunkRegionExists(chunkRegionName)) {
         chunkRegion = createChunkRegion(regionShortCut, fileRegionName, partitionAttributes, chunkRegionName);
       }
+      fileSystemStats.setFileSupplier(() -> (int) getFileRegion().getLocalSize());
+      fileSystemStats.setChunkSupplier(() -> (int) getChunkRegion().getLocalSize());
+      fileSystemStats.setBytesSupplier(() -> getChunkRegion().getPrStats().getDataStoreBytesInUse());
 
       // we will create RegionDirectories on the fly when data comes in
       HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(getFieldNames());
@@ -106,6 +106,14 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return (PartitionedRegion) cache.getRegion(regionPath);
   }
 
+  private PartitionedRegion getFileRegion() {
+    return (PartitionedRegion) fileRegion;
+  }
+
+  private PartitionedRegion getChunkRegion() {
+    return (PartitionedRegion) chunkRegion;
+  }
+
   private AsyncEventQueueFactoryImpl createAEQFactory(final Region dataRegion) {
     AsyncEventQueueFactoryImpl factory = (AsyncEventQueueFactoryImpl) cache.createAsyncEventQueueFactory();
     factory.setParallel(true); // parallel AEQ for PR

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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 c75148a..acd3765 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
@@ -46,28 +46,28 @@ import com.gemstone.gemfire.internal.logging.LogService;
 public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   protected static final Logger logger = LogService.getLogger();
   
-//  protected HashSet<String> searchableFieldNames = new HashSet<String>();
-  String[] searchableFieldNames;
-  protected RepositoryManager repositoryManager;
-  protected Analyzer analyzer;
-  
-  Region<String, File> fileRegion;
-  Region<ChunkKey, byte[]> chunkRegion;
-  LuceneIndexStats indexStats;
+  protected final String indexName;
+  protected final String regionPath;
+  protected final Cache cache;
+  protected final LuceneIndexStats indexStats;
+  protected final FileSystemStats fileSystemStats;
 
-  protected String indexName;
-  protected String regionPath;
   protected boolean hasInitialized = false;
   protected Map<String, Analyzer> fieldAnalyzers;
+  protected String[] searchableFieldNames;
+  protected RepositoryManager repositoryManager;
+  protected Analyzer analyzer;
+  protected Region<String, File> fileRegion;
+  protected Region<ChunkKey, byte[]> chunkRegion;
+
 
-  protected final Cache cache;
-  
   protected LuceneIndexImpl(String indexName, String regionPath, Cache cache) {
     this.indexName = indexName;
     this.regionPath = regionPath;
     this.cache = cache;
     final String statsName = indexName + "-" + regionPath;
     this.indexStats = new LuceneIndexStats(cache.getDistributedSystem(), statsName);
+    this.fileSystemStats = new FileSystemStats(cache.getDistributedSystem(), statsName);
   }
 
   @Override
@@ -139,6 +139,14 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     this.fieldAnalyzers = fieldAnalyzers == null ? null : Collections.unmodifiableMap(fieldAnalyzers);
   }
 
+  public LuceneIndexStats getIndexStats() {
+    return indexStats;
+  }
+
+  public FileSystemStats getFileSystemStats() {
+    return fileSystemStats;
+  }
+
   protected abstract void initialize();
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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 8dd34a4..ea1f35e 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
@@ -20,11 +20,14 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import static com.gemstone.gemfire.distributed.internal.DistributionStats.getStatTime;
 
+import java.util.function.IntSupplier;
+
 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.CopyOnWriteHashSet;
 import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
 
 public class LuceneIndexStats {
@@ -46,6 +49,7 @@ public class LuceneIndexStats {
   private static final int documentsId;
 
   private final Statistics stats;
+  private final CopyOnWriteHashSet<IntSupplier> documentsSuppliers = new CopyOnWriteHashSet<>();
 
   static {
     final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
@@ -82,6 +86,7 @@ public class LuceneIndexStats {
 
   public LuceneIndexStats(StatisticsFactory f, String name) {
     this.stats = f.createAtomicStatistics(statsType, name);
+    stats.setIntSupplier(documentsId, this::computeDocumentCount);
   }
 
   /**
@@ -133,7 +138,22 @@ public class LuceneIndexStats {
     stats.incInt(commitsId, 1);
   }
 
-  public void incDocuments(int delta) {
-    stats.incInt(documentsId, delta);
+  public void addDocumentsSuppplier(IntSupplier supplier) {
+    this.documentsSuppliers.add(supplier);
+  }
+
+  public void removeDocumentsSupplier(IntSupplier supplier) {
+    this.documentsSuppliers.remove(supplier);
+  }
+
+  public int getDocuments() {
+    return this.stats.getInt(documentsId);
+  }
+
+  private int computeDocumentCount() {
+    return this.documentsSuppliers.stream()
+      .mapToInt(IntSupplier::getAsInt)
+      .sum();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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
index 1f609fd..ee66850 100644
--- 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
@@ -18,6 +18,9 @@
  */
 package com.gemstone.gemfire.cache.lucene.internal.filesystem;
 
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+
 import com.gemstone.gemfire.StatisticDescriptor;
 import com.gemstone.gemfire.Statistics;
 import com.gemstone.gemfire.StatisticsFactory;
@@ -38,6 +41,9 @@ public class FileSystemStats {
   private static final int temporaryFileCreatesId;
   private static final int fileDeletesId;
   private static final int fileRenamesId;
+  private static final int filesId;
+  private static final int chunksId;
+  private static final int bytesId;
 
   static {
     final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
@@ -47,10 +53,13 @@ public class FileSystemStats {
       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"),
+        f.createIntCounter("fileCreates", "Number of files created", "files"),
+        f.createIntCounter("temporaryFileCreates", "Number of temporary files created", "files"),
+        f.createIntCounter("fileDeletes", "Number of files deleted", "files"),
+        f.createIntCounter("fileRenames", "Number of files renamed", "files"),
+        f.createIntGauge("files", "Number of files on this member", "files"),
+        f.createIntGauge("chunks", "Number of file chunks on this member", "chunks"),
+        f.createLongGauge("bytes", "Number of bytes on this member", "bytes"),
       }
     );
 
@@ -60,6 +69,9 @@ public class FileSystemStats {
     temporaryFileCreatesId = statsType.nameToId("temporaryFileCreates");
     fileDeletesId = statsType.nameToId("fileDeletes");
     fileRenamesId = statsType.nameToId("fileRenames");
+    filesId = statsType.nameToId("files");
+    chunksId = statsType.nameToId("chunks");
+    bytesId = statsType.nameToId("bytes");
   }
 
   public FileSystemStats(StatisticsFactory f, String name) {
@@ -89,4 +101,28 @@ public class FileSystemStats {
   public void incFileRenames(final int delta) {
     stats.incInt(fileRenamesId,delta);
   }
+
+  public void setFileSupplier(IntSupplier supplier) {
+    stats.setIntSupplier(filesId, supplier);
+  }
+
+  public int getFiles() {
+    return stats.getInt(filesId);
+  }
+
+  public void setChunkSupplier(IntSupplier supplier) {
+    stats.setIntSupplier(chunksId, supplier);
+  }
+
+  public int getChunks() {
+    return stats.getInt(chunksId);
+  }
+
+  public void setBytesSupplier(LongSupplier supplier) {
+    stats.setLongSupplier(bytesId, supplier);
+  }
+
+  public long getBytes() {
+    return stats.getLong(bytesId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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 88631b8..110f85a 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
@@ -32,6 +32,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 
 import java.io.IOException;
+import java.util.function.IntSupplier;
 
 /**
  * A repository that writes to a single lucene index writer
@@ -56,6 +57,7 @@ public class IndexRepositoryImpl implements IndexRepository {
     searcherManager = new SearcherManager(writer, APPLY_ALL_DELETES, true, null);
     this.serializer = serializer;
     this.stats = stats;
+    stats.addDocumentsSuppplier(new DocumentCountSupplier());
   }
 
   @Override
@@ -145,4 +147,15 @@ public class IndexRepositoryImpl implements IndexRepository {
   public boolean isClosed() {
     return region.isDestroyed();
   }
+
+  private class DocumentCountSupplier implements IntSupplier {
+    @Override
+    public int getAsInt() {
+      if(isClosed()) {
+        stats.removeDocumentsSupplier(this);
+        return 0;
+      }
+      return writer.numDocs();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
index 07780ca..af8c51f 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -22,10 +22,15 @@ import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
 import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.jayway.awaitility.Awaitility;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -106,6 +111,31 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     assertEquals(3, query.search().size());
   }
 
+  @Test
+  public void statsAreUpdatedAfterACommit() throws Exception {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    region.put("object-1", new TestObject("title 1", "hello world"));
+    region.put("object-2", new TestObject("title 2", "this will not match"));
+    region.put("object-3", new TestObject("title 3", "hello world"));
+    region.put("object-4", new TestObject("hello world", "hello world"));
+
+    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+
+    FileSystemStats fileSystemStats = index.getFileSystemStats();
+    LuceneIndexStats indexStats = index.getIndexStats();
+    await(() -> assertEquals(4, indexStats.getDocuments()));
+    await(() -> assertTrue(fileSystemStats.getFiles() > 0));
+    await(() -> assertTrue(fileSystemStats.getChunks() > 0));
+    await(() -> assertTrue(fileSystemStats.getBytes() > 0));
+  }
+
+  private void await(Runnable runnable) {
+    Awaitility.await().atMost(30, TimeUnit.SECONDS).until(runnable);
+  }
+
   private static class TestObject implements Serializable {
 
     String title;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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 3e6d44c..05e64af 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
@@ -18,11 +18,16 @@
  */
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.*;
 
+import java.util.function.IntSupplier;
+
 import com.gemstone.gemfire.Statistics;
 import com.gemstone.gemfire.StatisticsFactory;
 import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystemJUnitTest;
+import com.gemstone.gemfire.management.internal.cli.parser.Argument;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 import org.junit.Before;
@@ -88,9 +93,15 @@ public class LuceneIndexStatsJUnitTest {
   }
 
   @Test
-  public void shouldIncrementDocumentStat() {
-    stats.incDocuments(5);
-    verifyIncInt("documents", 5);
+  public void shouldPollSuppliersForDocumentStat() {
+    stats.addDocumentsSuppplier(() -> 5);
+    stats.addDocumentsSuppplier(() -> 3);
+
+    int documentsId = type.nameToId("documents");
+    ArgumentCaptor<IntSupplier> documentsSupplierCaptor = ArgumentCaptor.forClass(IntSupplier.class);
+    verify(statistics).setIntSupplier(eq(documentsId), documentsSupplierCaptor.capture());
+    IntSupplier documentsSuppler = documentsSupplierCaptor.getValue();
+    assertEquals(8, documentsSuppler.getAsInt());
   }
 
   private void verifyIncInt(final String statName, final int value) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/615e643c/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 6004623..7d3caf8 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
@@ -29,6 +29,7 @@ import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.IntSupplier;
 
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.index.IndexWriter;
@@ -38,6 +39,7 @@ import org.apache.lucene.queryparser.classic.QueryParser;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.cache.Region;
@@ -153,6 +155,16 @@ public class IndexRepositoryImplJUnitTest {
     verify(stats, times(1)).endQuery(anyLong(), eq(2));
   }
 
+  @Test
+  public void addingDocumentsShouldUpdateDocumentsStat() throws IOException {
+    repo.create("key1", new Type2("bar", 1, 2L, 3.0, 4.0f, "Grape Ape doughnut"));
+    repo.commit();
+    ArgumentCaptor<IntSupplier> captor = ArgumentCaptor.forClass(IntSupplier.class);
+    verify(stats).addDocumentsSuppplier(captor.capture());
+    IntSupplier supplier = captor.getValue();
+    assertEquals(1, supplier.getAsInt());
+  }
+
   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"));