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/29 22:26:52 UTC

incubator-geode git commit: GEODE-11: Cleaning up old IndexRepositoryImpls when buckets move

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


GEODE-11: Cleaning up old IndexRepositoryImpls when buckets move

When a bucket is moved, we leave the IndexRepositoryImpl open. But even
after the bucket moves back, we just dereference the old
IndexRepositoryImpl without closing it. We should make sure we always
invoke close on the IndexRepositoryImpl to clean up any resources the
IndexWriter is using.


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

Branch: refs/heads/develop
Commit: 63ce3c0f4828f4af11e6c354d228c42ea0226b49
Parents: b5a50fe
Author: Dan Smith <up...@apache.org>
Authored: Wed Jun 15 13:58:31 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed Jun 29 14:46:59 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/AbstractRegion.java  |  4 +-
 .../com/gemstone/gemfire/test/fake/Fakes.java   | 17 +++++++
 .../cache/lucene/internal/LuceneIndexStats.java |  2 +-
 .../internal/PartitionedRepositoryManager.java  | 47 +++++++++++---------
 .../internal/repository/IndexRepository.java    |  5 +++
 .../repository/IndexRepositoryImpl.java         | 15 ++++++-
 .../internal/LuceneIndexStatsJUnitTest.java     |  6 +--
 .../PartitionedRepositoryManagerJUnitTest.java  | 40 ++++++++++-------
 .../IndexRepositoryImplJUnitTest.java           |  9 +++-
 9 files changed, 98 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
index 95854ec..2d44979 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
@@ -276,13 +276,13 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
     return destroy(key, null);
   }
 
-  public final Object get(Object name) throws CacheLoaderException,
+  public Object get(Object name) throws CacheLoaderException,
       TimeoutException
   {
     return get(name, null, true, null);
   }
 
-  public final Object put(Object name, Object value) throws TimeoutException,
+  public Object put(Object name, Object value) throws TimeoutException,
       CacheWriterException
   {
     return put(name, value, null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
index 323c281..c3b1a0e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
@@ -28,10 +28,13 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.cache.AbstractRegion;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 
 import java.io.File;
 import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.mockito.Mockito.*;
 
@@ -123,6 +126,20 @@ public class Fakes {
     return region;
   }
 
+  /**
+   * Add real map behavior to a mock region. Useful for tests
+   * where you want to mock region that just behaves like a map.
+   * @param mock the mockito mock to add behavior too.
+   */
+  public static void addMapBehavior(Region mock) {
+    //Allow the region to behave like a fake map
+    Map underlyingMap = new HashMap();
+    when(mock.get(any()))
+      .then(invocation -> underlyingMap.get(invocation.getArguments()[0]));
+    when(mock.put(any(), any()))
+      .then(invocation -> underlyingMap.put(invocation.getArguments()[0], invocation.getArguments()[1]));
+  }
+
   private Fakes() {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/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 ea1f35e..8e2d97c 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
@@ -138,7 +138,7 @@ public class LuceneIndexStats {
     stats.incInt(commitsId, 1);
   }
 
-  public void addDocumentsSuppplier(IntSupplier supplier) {
+  public void addDocumentsSupplier(IntSupplier supplier) {
     this.documentsSuppliers.add(supplier);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/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 25010b0..a119157 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
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexWriter;
@@ -60,7 +61,7 @@ public class PartitionedRepositoryManager implements RepositoryManager {
    * 
    * It is weak so that the old BucketRegion will be garbage collected. 
    */
-  CopyOnWriteHashMap<Integer, IndexRepository> indexRepositories = new CopyOnWriteHashMap<Integer, IndexRepository>();
+  private final ConcurrentHashMap<Integer, IndexRepository> indexRepositories = new ConcurrentHashMap<Integer, IndexRepository>();
   
   /** The user region for this index */
   private final PartitionedRegion userRegion;
@@ -128,45 +129,49 @@ public class PartitionedRepositoryManager implements RepositoryManager {
    */
   private IndexRepository getRepository(Integer bucketId) throws BucketNotFoundException {
     IndexRepository repo = indexRepositories.get(bucketId);
-    
-    //Remove the repository if it has been destroyed (due to rebalancing)
-    if(repo != null && repo.isClosed()) {
-      indexRepositories.remove(bucketId, repo);
-      repo = null;
+    if(repo != null && !repo.isClosed()) {
+      return repo;
     }
-    
-    if(repo == null) {
+
+    repo = indexRepositories.compute(bucketId, (key, oldRepository) -> {
+      if(oldRepository != null && !oldRepository.isClosed()) {
+        return oldRepository;
+      }
+      if(oldRepository != null) {
+        oldRepository.cleanup();
+      }
+
       try {
         BucketRegion fileBucket = getMatchingBucket(fileRegion, bucketId);
         BucketRegion chunkBucket = getMatchingBucket(chunkRegion, bucketId);
+        if(fileBucket == null || chunkBucket == null) {
+          return null;
+        }
         RegionDirectory dir = new RegionDirectory(fileBucket, chunkBucket, fileSystemStats);
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         IndexWriter writer = new IndexWriter(dir, config);
-        repo = new IndexRepositoryImpl(fileBucket, writer, serializer, indexStats);
-        IndexRepository oldRepo = indexRepositories.putIfAbsent(bucketId, repo);
-        if(oldRepo != null) {
-          repo = oldRepo;
-        }
+        return new IndexRepositoryImpl(fileBucket, writer, serializer, indexStats);
+
       } catch(IOException e) {
         throw new InternalGemFireError("Unable to create index repository", e);
       }
+
+    });
+
+    if(repo == null) {
+      throw new BucketNotFoundException("Colocated index buckets not found for regions " + chunkRegion + ", " + fileRegion + " bucket id " + bucketId);
     }
-    
+
     return repo;
   }
 
   /**
    * Find the bucket in region2 that matches the bucket id from region1.
    */
-  private BucketRegion getMatchingBucket(PartitionedRegion region, Integer bucketId) throws BucketNotFoundException {
+  private BucketRegion getMatchingBucket(PartitionedRegion region, Integer bucketId) {
     //Force the bucket to be created if it is not already
     region.getOrCreateNodeForBucketWrite(bucketId, null);
     
-    BucketRegion result = region.getDataStore().getLocalBucketById(bucketId);
-    if(result == null) {
-      throw new BucketNotFoundException("Bucket not found for region " + region + " bucekt id " + bucketId);
-    }
-    
-    return result;
+    return region.getDataStore().getLocalBucketById(bucketId);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
index fab2c2a..e487884 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
@@ -81,4 +81,9 @@ public interface IndexRepository {
    * For debugging purposes, return the underlying IndexWriter
    */
   IndexWriter getWriter();
+
+  /**
+   * Clean up any resources associated with this index repository.
+   */
+  void cleanup();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/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 110f85a..563e382 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
@@ -48,6 +48,7 @@ public class IndexRepositoryImpl implements IndexRepository {
   private final SearcherManager searcherManager;
   private Region<?,?> region;
   private LuceneIndexStats stats;
+  private DocumentCountSupplier documentCountSupplier;
 
   private static final Logger logger = LogService.getLogger();
   
@@ -57,7 +58,8 @@ public class IndexRepositoryImpl implements IndexRepository {
     searcherManager = new SearcherManager(writer, APPLY_ALL_DELETES, true, null);
     this.serializer = serializer;
     this.stats = stats;
-    stats.addDocumentsSuppplier(new DocumentCountSupplier());
+    documentCountSupplier = new DocumentCountSupplier();
+    stats.addDocumentsSupplier(documentCountSupplier);
   }
 
   @Override
@@ -148,6 +150,17 @@ public class IndexRepositoryImpl implements IndexRepository {
     return region.isDestroyed();
   }
 
+  @Override
+  public void cleanup() {
+    stats.removeDocumentsSupplier(documentCountSupplier);
+    try {
+      writer.close();
+    }
+    catch (IOException e) {
+      logger.warn("Unable to clean up index repository", e);
+    }
+  }
+
   private class DocumentCountSupplier implements IntSupplier {
     @Override
     public int getAsInt() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/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 05e64af..6f2f411 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
@@ -26,8 +26,6 @@ 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;
@@ -94,8 +92,8 @@ public class LuceneIndexStatsJUnitTest {
 
   @Test
   public void shouldPollSuppliersForDocumentStat() {
-    stats.addDocumentsSuppplier(() -> 5);
-    stats.addDocumentsSuppplier(() -> 3);
+    stats.addDocumentsSupplier(() -> 5);
+    stats.addDocumentsSupplier(() -> 3);
 
     int documentsId = type.nameToId("documents");
     ArgumentCaptor<IntSupplier> documentsSupplierCaptor = ArgumentCaptor.forClass(IntSupplier.class);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/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 ec56381..2221a6d 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
@@ -20,6 +20,7 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
 import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -51,6 +52,7 @@ import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion.RetryTimeKeeper;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.execute.InternalRegionFunctionContext;
+import com.gemstone.gemfire.test.fake.Fakes;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -73,14 +75,14 @@ public class PartitionedRepositoryManagerJUnitTest {
   public void setUp() {
     userRegion = Mockito.mock(PartitionedRegion.class);
     userDataStore = Mockito.mock(PartitionedRegionDataStore.class);
-    Mockito.when(userRegion.getDataStore()).thenReturn(userDataStore);
+    when(userRegion.getDataStore()).thenReturn(userDataStore);
     
     fileRegion = Mockito.mock(PartitionedRegion.class);
     fileDataStore = Mockito.mock(PartitionedRegionDataStore.class);
-    Mockito.when(fileRegion.getDataStore()).thenReturn(fileDataStore);
+    when(fileRegion.getDataStore()).thenReturn(fileDataStore);
     chunkRegion = Mockito.mock(PartitionedRegion.class);
     chunkDataStore = Mockito.mock(PartitionedRegionDataStore.class);
-    Mockito.when(chunkRegion.getDataStore()).thenReturn(chunkDataStore);
+    when(chunkRegion.getDataStore()).thenReturn(chunkDataStore);
     serializer = new HeterogeneousLuceneSerializer(new String[] {"a", "b"} );
     indexStats = Mockito.mock(LuceneIndexStats.class);
     fileSystemStats = Mockito.mock(FileSystemStats.class);
@@ -112,7 +114,7 @@ public class PartitionedRepositoryManagerJUnitTest {
    * Test what happens when a bucket is destroyed.
    */
   @Test
-  public void destroyBucket() throws BucketNotFoundException, IOException {
+  public void destroyBucketShouldCreateNewIndexRepository() throws BucketNotFoundException, IOException {
     PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer(),
       indexStats, fileSystemStats);
     
@@ -127,12 +129,14 @@ public class PartitionedRepositoryManagerJUnitTest {
     
     //Simulate rebalancing of a bucket by marking the old bucket is destroyed
     //and creating a new bucket
-    Mockito.when(fileBucket0.isDestroyed()).thenReturn(true);
+    when(fileBucket0.isDestroyed()).thenReturn(true);
     setUpMockBucket(0);
     
     IndexRepositoryImpl newRepo0 = (IndexRepositoryImpl) repoManager.getRepository(userRegion, 0, null);
     assertNotEquals(repo0, newRepo0);
     checkRepository(newRepo0, 0);
+    assertTrue(repo0.isClosed());
+    assertFalse(repo0.getWriter().isOpen());
   }
   
   /**
@@ -151,12 +155,12 @@ public class PartitionedRepositoryManagerJUnitTest {
       indexStats, fileSystemStats);
     setUpMockBucket(0);
     
-    Mockito.when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(null);
+    when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(null);
     
-    Mockito.when(fileRegion.getOrCreateNodeForBucketWrite(eq(0), (RetryTimeKeeper) any())).then(new Answer() {
+    when(fileRegion.getOrCreateNodeForBucketWrite(eq(0), (RetryTimeKeeper) any())).then(new Answer() {
       @Override
       public Object answer(InvocationOnMock invocation) throws Throwable {
-        Mockito.when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(fileBuckets.get(0));
+        when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(fileBuckets.get(0));
         return null;
       }
     });
@@ -174,7 +178,7 @@ public class PartitionedRepositoryManagerJUnitTest {
 
     Set<Integer> buckets = new LinkedHashSet<Integer>(Arrays.asList(0, 1));
     InternalRegionFunctionContext ctx = Mockito.mock(InternalRegionFunctionContext.class);
-    Mockito.when(ctx.getLocalBucketSet((any()))).thenReturn(buckets);
+    when(ctx.getLocalBucketSet((any()))).thenReturn(buckets);
     Collection<IndexRepository> repos = repoManager.getRepositories(ctx);
     assertEquals(2, repos.size());
 
@@ -203,7 +207,7 @@ public class PartitionedRepositoryManagerJUnitTest {
     Set<Integer> buckets = new LinkedHashSet<Integer>(Arrays.asList(0, 1));
 
     InternalRegionFunctionContext ctx = Mockito.mock(InternalRegionFunctionContext.class);
-    Mockito.when(ctx.getLocalBucketSet((any()))).thenReturn(buckets);
+    when(ctx.getLocalBucketSet((any()))).thenReturn(buckets);
     repoManager.getRepositories(ctx);
   }
   
@@ -218,14 +222,16 @@ public class PartitionedRepositoryManagerJUnitTest {
   private BucketRegion setUpMockBucket(int id) {
     BucketRegion mockBucket = Mockito.mock(BucketRegion.class);
     BucketRegion fileBucket = Mockito.mock(BucketRegion.class);
+    //Allowing the fileBucket to behave like a map so that the IndexWriter operations don't fail
+    Fakes.addMapBehavior(fileBucket);
     BucketRegion chunkBucket = Mockito.mock(BucketRegion.class);
-    Mockito.when(mockBucket.getId()).thenReturn(id);
-    Mockito.when(userRegion.getBucketRegion(eq(id), eq(null))).thenReturn(mockBucket);
-    Mockito.when(userDataStore.getLocalBucketById(eq(id))).thenReturn(mockBucket);
-    Mockito.when(userRegion.getBucketRegion(eq(id + 113), eq(null))).thenReturn(mockBucket);
-    Mockito.when(userDataStore.getLocalBucketById(eq(id + 113))).thenReturn(mockBucket);
-    Mockito.when(fileDataStore.getLocalBucketById(eq(id))).thenReturn(fileBucket);
-    Mockito.when(chunkDataStore.getLocalBucketById(eq(id))).thenReturn(chunkBucket);
+    when(mockBucket.getId()).thenReturn(id);
+    when(userRegion.getBucketRegion(eq(id), eq(null))).thenReturn(mockBucket);
+    when(userDataStore.getLocalBucketById(eq(id))).thenReturn(mockBucket);
+    when(userRegion.getBucketRegion(eq(id + 113), eq(null))).thenReturn(mockBucket);
+    when(userDataStore.getLocalBucketById(eq(id + 113))).thenReturn(mockBucket);
+    when(fileDataStore.getLocalBucketById(eq(id))).thenReturn(fileBucket);
+    when(chunkDataStore.getLocalBucketById(eq(id))).thenReturn(chunkBucket);
     
     fileBuckets.put(id, fileBucket);
     chunkBuckets.put(id, chunkBucket);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/63ce3c0f/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 7d3caf8..cd67413 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
@@ -160,11 +160,18 @@ public class IndexRepositoryImplJUnitTest {
     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());
+    verify(stats).addDocumentsSupplier(captor.capture());
     IntSupplier supplier = captor.getValue();
     assertEquals(1, supplier.getAsInt());
   }
 
+  @Test
+  public void cleanupShouldCloseWriter() throws IOException {
+    repo.cleanup();
+    verify(stats).removeDocumentsSupplier(any());
+    assertFalse(writer.isOpen());
+  }
+
   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"));