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/05/20 17:10:48 UTC

[23/50] [abbrv] incubator-geode git commit: GEODE-11: Split out lucene tests of persistence and off heap

GEODE-11: Split out lucene tests of persistence and off heap

Persistence and offheap require some special before and after code to
create and clean up the resources. Splitting these to separate
integration tests.


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

Branch: refs/heads/feature/GEODE-835
Commit: 8024f33d5687ebbbfda7a0e20375620ff58015d9
Parents: 09b3c03
Author: Dan Smith <up...@apache.org>
Authored: Fri May 13 09:51:43 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue May 17 12:11:40 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexCreationIntegrationTest.java     | 159 ++++---------------
 ...ceneIndexCreationOffHeapIntegrationTest.java |  72 +++++++++
 ...IndexCreationPersistenceIntegrationTest.java | 141 ++++++++++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |  20 ++-
 .../cache/lucene/test/LuceneTestUtilities.java  |  64 ++++++++
 .../gemfire/cache/lucene/test/TestObject.java   |  27 ++++
 .../gemfire/cache/lucene/test/package-info.java |  23 +++
 7 files changed, 365 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index ef7f808..4c28938 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -19,9 +19,9 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -36,10 +36,8 @@ import com.gemstone.gemfire.cache.FixedPartitionAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
@@ -47,14 +45,26 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+/**
+ * Tests of creating lucene indexes on regions. All tests of index creation
+ * use cases should be in classes starting with LuceneIndexCreation*. Most
+ * tests belong in this class, except for:
+ * <ul>
+ * <li> Tests that use persistence are in {@link LuceneIndexCreationPersistenceIntegrationTest}  </li>
+ * <li> Tests that use offheap are in {@link LuceneIndexCreationOffHeapIntegrationTest}  </li>
+ * </ul>
+ */
 @Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
-  public static final String INDEX_NAME = "index";
-  public static final String REGION_NAME = "region";
+
 
   @Test
   public void shouldCreateIndexWriterWithAnalyzersWhenSettingPerFieldAnalyzers()
@@ -70,18 +80,22 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     Region region = createRegion();
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     region.put("key1", new TestObject());
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
     assertEquals(analyzers, index.getFieldAnalyzers());
     assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
     assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
   }
 
   @Test
-  public void shouldUseRedundancyForInternalRegionsWhenUserRegionHasRedundancy() {
+  @Parameters({"0", "1", "2"})
+  public void shouldUseRedundancyForInternalRegionsWhenUserRegionHasRedundancy(int redundancy) {
     createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_REDUNDANT).create(REGION_NAME);
+    PartitionAttributesFactory paf = new PartitionAttributesFactory();
+    paf.setRedundantCopies(redundancy);
+
+    cache.createRegionFactory(RegionShortcut.PARTITION_REDUNDANT).setPartitionAttributes(paf.create()).create(REGION_NAME);
     verifyInternalRegions(region -> {
-      assertEquals(1, region.getAttributes().getPartitionAttributes().getRedundantCopies());
+      assertEquals(redundancy, region.getAttributes().getPartitionAttributes().getRedundantCopies());
     });
   }
 
@@ -122,103 +136,6 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION)
-      .setOffHeap(true)
-      .create(REGION_NAME);
-
-    verifyInternalRegions(region -> {
-      assertEquals(false, region.getOffHeap());
-    });
-  }
-
-  @Test
-  public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getAttributes().getEvictionAttributes().getAction().isNone());
-    });
-  }
-
-  @Test
-  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .setDiskSynchronous(true)
-      .create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getDataPolicy().withPersistence());
-      assertTrue(region.isDiskSynchronous());
-    });
-    AsyncEventQueue queue = getIndexQueue();
-    assertEquals(true, queue.isDiskSynchronous());
-    assertEquals(true, queue.isPersistent());
-  }
-
-  @Test
-  public void shouldUseDiskSyncFalseOnQueueWhenUserRegionHasDiskSynchronousFalse() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .setDiskSynchronous(false)
-      .create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getDataPolicy().withPersistence());
-      assertTrue(region.isDiskSynchronous());
-    });
-    AsyncEventQueue queue = getIndexQueue();
-    assertEquals(false, queue.isDiskSynchronous());
-    assertEquals(true, queue.isPersistent());
-  }
-
-  @Test
-  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
-    createIndex("field1", "field2");
-    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    //Pause the sender so that the entry stays in the queue
-    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
-    queue.getSender().pause();
-
-    dataRegion.put("A", new TestObject());
-    cache.close();
-    createCache();
-    createIndex("field1", "field2");
-    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
-    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
-      .create(INDEX_NAME, REGION_NAME,
-        "field1:world");
-    assertEquals(1, query.search().size());
-  }
-
-  private void verifyIndexFinishFlushing(String indexName, String regionName) {
-    LuceneIndex index = luceneService.getIndex(indexName, regionName);
-    boolean flushed = index.waitUntilFlushed(60000);
-    assertTrue(flushed);
-  }
-  
-  @Test
-  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
-    createIndex("field1", "field2");
-    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    dataRegion.put("A", new TestObject());
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
-    cache.close();
-    createCache();
-    createIndex("text");
-    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
-      .create(INDEX_NAME, REGION_NAME,
-      "field1:world");
-    assertEquals(1, query.search().size());
-  }
-
-  @Test
   public void shouldCreateInternalRegionsForIndex() {
     createIndex("field1", "field2");
 
@@ -252,19 +169,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
 
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
-    // Get index
-    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
-
-    // Verify the meta regions exist and are internal
-    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
-    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
-    verify.accept(chunkRegion);
-    verify.accept(fileRegion);
-  }
-
-  private AsyncEventQueue getIndexQueue() {
-    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
-    return cache.getAsyncEventQueue(aeqId);
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
 
   private Region createRegion() {
@@ -272,13 +177,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   private void createIndex(String ... fieldNames) {
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
-  }
-
-  private static class TestObject implements Serializable {
-
-    String field1 = "hello world";
-    String field2 = "this is a field";
+    LuceneTestUtilities.createIndex(cache, fieldNames);
   }
 
   private static class RecordingAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
new file mode 100644
index 0000000..8f10c26
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.assertEquals;
+
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests of lucene index creation that use off heap memory
+ */
+@Category(IntegrationTest.class)
+public class LuceneIndexCreationOffHeapIntegrationTest extends LuceneIntegrationTest {
+
+  @Override
+  public void closeCache() {
+    super.closeCache();
+    MemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Override
+  protected CacheFactory getCacheFactory() {
+    CacheFactory factory = super.getCacheFactory();
+    factory.set("off-heap-memory-size", "100m");
+    return factory;
+  }
+
+  @Test
+  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setOffHeap(true)
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      assertEquals(false, region.getOffHeap());
+    });
+  }
+
+  private void verifyInternalRegions(Consumer<LocalRegion> verify) {
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
new file mode 100644
index 0000000..23983cb
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
+import com.jayway.awaitility.Awaitility;
+
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+/**
+ * Tests of lucene index creation that use persistence
+ */
+@Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegrationTest {
+
+  @Rule
+  public DiskDirRule diskDirRule = new DiskDirRule();
+
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  @Override
+  public void createCache() {
+    super.createCache();
+    cache.createDiskStoreFactory()
+      .setDiskDirs(new File[] {diskDirRule.get()})
+      .setMaxOplogSize(1)
+      .create(GemFireCacheImpl.getDefaultDiskStoreName());
+  }
+
+  @Test
+  public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getAttributes().getEvictionAttributes().getAction().isNone());
+    });
+  }
+
+  @Test
+  @Parameters({"true", "false"})
+  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous(boolean synchronous) {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(synchronous)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      //Underlying region should always be synchronous
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue(cache);
+    assertEquals(synchronous, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+    createIndex(cache, "field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    //Pause the sender so that the entry stays in the queue
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue(cache);
+    queue.getSender().pause();
+
+    dataRegion.put("A", new TestObject());
+    cache.close();
+    createCache();
+    createIndex(cache, "field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+        "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+    createIndex(cache, "field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    dataRegion.put("A", new TestObject());
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
+    cache.close();
+    createCache();
+    createIndex(cache, "field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+      "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  private void verifyInternalRegions(Consumer<LocalRegion> verify) {
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index 67775d2..c302460 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -38,28 +38,26 @@ public class LuceneIntegrationTest {
 
   protected Cache cache;
   protected LuceneService luceneService;
-  @Rule
-  public DiskDirRule diskDirRule = new DiskDirRule();
 
   @After
-  public void tearDown() {
+  public void closeCache() {
     if(this.cache != null) {
       this.cache.close();
     }
-    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Before
   public void createCache() {
+    CacheFactory cf = getCacheFactory();
+    this.cache = cf.create();
+
+    luceneService = LuceneServiceProvider.get(this.cache);
+  }
+
+  protected CacheFactory getCacheFactory() {
     CacheFactory cf = new CacheFactory();
     cf.set("mcast-port", "0");
     cf.set("locators", "");
-    cf.set("off-heap-memory-size", "100m");
-    this.cache = cf.create();
-    cache.createDiskStoreFactory()
-      .setDiskDirs(new File[] {diskDirRule.get()})
-      .setMaxOplogSize(1)
-      .create(GemFireCacheImpl.getDefaultDiskStoreName());
-    luceneService = LuceneServiceProvider.get(this.cache);
+    return cf;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
new file mode 100644
index 0000000..61355c4
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -0,0 +1,64 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+
+public class LuceneTestUtilities {
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  public static void verifyInternalRegions(LuceneService luceneService, Cache cache, Consumer<LocalRegion> verify) {
+    // Get index
+    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    // Verify the meta regions exist and are internal
+    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
+    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
+    verify.accept(chunkRegion);
+    verify.accept(fileRegion);
+  }
+
+  public static AsyncEventQueue getIndexQueue(Cache cache) {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+    return cache.getAsyncEventQueue(aeqId);
+  }
+
+  public static void createIndex(Cache cache, String... fieldNames) {
+    LuceneServiceProvider.get(cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
+  }
+
+  public static void verifyIndexFinishFlushing(Cache cache, String indexName, String regionName) {
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    LuceneIndex index = luceneService.getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
new file mode 100644
index 0000000..3287b9f
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
@@ -0,0 +1,27 @@
+/*
+ * 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.test;
+
+import java.io.Serializable;
+
+public class TestObject implements Serializable {
+
+  String field1 = "hello world";
+  String field2 = "this is a field";
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
new file mode 100644
index 0000000..6345804
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * Utility classes for testing the lucene functionality.
+ */
+
+package com.gemstone.gemfire.cache.lucene.test;
\ No newline at end of file