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

[19/50] [abbrv] incubator-geode git commit: GEODE-1220: Adding unit tests for LuceneIndexForPartitionRegion

GEODE-1220: Adding unit tests for LuceneIndexForPartitionRegion

Refactored LuceneIndexForPartitionRegion, breaking down initialize()
into smaller method calls


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 8cdeb55e4c580b9699fac3ca1f24079ace8c33e1
Parents: 760c6e2
Author: Jason Huynh <hu...@gmail.com>
Authored: Thu Apr 14 17:00:44 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Apr 19 11:33:36 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexForPartitionedRegion.java        | 117 +++++--
 .../LuceneIndexForPartitionedRegionTest.java    | 348 +++++++++++++++++++
 2 files changed, 433 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8cdeb55e/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 25b63a4..4145459 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
@@ -23,9 +23,11 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.PartitionAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
@@ -43,13 +45,13 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     this.regionPath = regionPath;
     this.cache = cache;
   }
-  
+
   @Override
   public void initialize() {
     if (!hasInitialized) {
       /* create index region */
-      PartitionedRegion dataRegion = (PartitionedRegion)cache.getRegion(regionPath);
-      assert dataRegion != null;
+      PartitionedRegion dataRegion = (PartitionedRegion) cache.getRegion(regionPath);
+      //assert dataRegion != null;
       RegionAttributes ra = dataRegion.getAttributes();
       DataPolicy dp = ra.getDataPolicy();
       final boolean withPersistence = dp.withPersistence();
@@ -73,52 +75,103 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       }
 
       // create PR fileRegion, but not to create its buckets for now
-      final String fileRegionName = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath)+".files";
-      fileRegion = cache.<String, File> getRegion(fileRegionName);
+      final String fileRegionName = createFileRegionName();
       PartitionAttributes partitionAttributes = dataRegion.getPartitionAttributes();
-      if (null == fileRegion) {
-        fileRegion = cache.<String, File> createRegionFactory(regionShortCut)
-            .setPartitionAttributes(new PartitionAttributesFactory<String, File>().setColocatedWith(regionPath)
-                .setTotalNumBuckets(partitionAttributes.getTotalNumBuckets())
-                .create())
-                .create(fileRegionName);
+      if (!fileRegionExists(fileRegionName)) {
+        fileRegion = createFileRegion(regionShortCut, fileRegionName, partitionAttributes);
       }
 
       // create PR chunkRegion, but not to create its buckets for now
-      final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
-      chunkRegion = cache.<ChunkKey, byte[]> getRegion(chunkRegionName);
-      if (null == chunkRegion) {
-        chunkRegion = cache.<ChunkKey, byte[]> createRegionFactory(regionShortCut)
-            .setPartitionAttributes(new PartitionAttributesFactory<ChunkKey, byte[]>().setColocatedWith(fileRegionName)
-                .setTotalNumBuckets(partitionAttributes.getTotalNumBuckets())
-                .create())
-                .create(chunkRegionName);
+      final String chunkRegionName = createChunkRegionName();
+      if (!chunkRegionExists(chunkRegionName)) {
+        chunkRegion = createChunkRegion(regionShortCut, fileRegionName, partitionAttributes, chunkRegionName);
       }
 
-      // we will create RegionDirectorys on the fly when data coming
+      // we will create RegionDirectories on the fly when data comes in
       HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(getFieldNames());
       repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, mapper, analyzer);
       
-      // create AEQ, AEQ listner and specify the listener to repositoryManager
-      AsyncEventQueueFactoryImpl factory = (AsyncEventQueueFactoryImpl) cache.createAsyncEventQueueFactory();
+      // create AEQ, AEQ listener and specify the listener to repositoryManager
       if (withPersistence) {
-        factory.setPersistent(true);
+        createAEQWithPersistence();
+      }
+      else {
+        createAEQ();
       }
-      factory.setParallel(true); // parallel AEQ for PR
-      factory.setMaximumQueueMemory(1000);
-      factory.setDispatcherThreads(1);
-      factory.setIsMetaQueue(true);
-      
-      LuceneEventListener listener = new LuceneEventListener(repositoryManager);
-      String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), regionPath);
-      AsyncEventQueueImpl aeq = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
-      AsyncEventQueue indexQueue = factory.create(aeqId, listener);
 
       addExtension(dataRegion);
       hasInitialized = true;
     }
   }
 
+  private AsyncEventQueueFactoryImpl createAEQFactory() {
+    AsyncEventQueueFactoryImpl factory = (AsyncEventQueueFactoryImpl) cache.createAsyncEventQueueFactory();
+    factory.setParallel(true); // parallel AEQ for PR
+    factory.setMaximumQueueMemory(1000);
+    factory.setDispatcherThreads(1);
+    factory.setIsMetaQueue(true);
+    return factory;
+  }
+
+  AsyncEventQueue createAEQWithPersistence() {
+    AsyncEventQueueFactoryImpl factory = createAEQFactory();
+    factory.setPersistent(true);
+    return createAEQ(factory);
+  }
+
+  AsyncEventQueue createAEQ() {
+    return createAEQ(createAEQFactory());
+  }
+
+  private AsyncEventQueue createAEQ(AsyncEventQueueFactoryImpl factory) {
+    LuceneEventListener listener = new LuceneEventListener(repositoryManager);
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), regionPath);
+    AsyncEventQueueImpl aeq = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
+    AsyncEventQueue indexQueue = factory.create(aeqId, listener);
+    return indexQueue;
+  }
+
+
+  boolean fileRegionExists(String fileRegionName) {
+    return cache.<String, File> getRegion(fileRegionName) != null;
+  }
+
+  Region createFileRegion(final RegionShortcut regionShortCut,
+                                final String fileRegionName,
+                                final PartitionAttributes partitionAttributes)
+  {
+    return cache.<String, File> createRegionFactory(regionShortCut)
+        .setPartitionAttributes(new PartitionAttributesFactory<String, File>().setColocatedWith(regionPath)
+            .setTotalNumBuckets(partitionAttributes.getTotalNumBuckets())
+            .create())
+            .create(fileRegionName);
+  }
+
+  String createFileRegionName() {
+    return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath)+".files";
+  }
+
+  boolean chunkRegionExists(String chunkRegionName) {
+    return cache.<ChunkKey, byte[]> getRegion(chunkRegionName) != null;
+  }
+
+  Region<ChunkKey, byte[]> createChunkRegion(final RegionShortcut regionShortCut,
+                           final String fileRegionName,
+                           final PartitionAttributes partitionAttributes, final String chunkRegionName)
+  {
+    return cache.<ChunkKey, byte[]> createRegionFactory(regionShortCut)
+      .setPartitionAttributes(new PartitionAttributesFactory<ChunkKey, byte[]>().setColocatedWith(fileRegionName)
+        .setTotalNumBuckets(partitionAttributes.getTotalNumBuckets())
+        .create())
+      .create(chunkRegionName);
+  }
+
+  String createChunkRegionName() {
+    return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
+  }
+
+
+
   public void close() {
     // TODO Auto-generated method stub
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8cdeb55e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
new file mode 100644
index 0000000..74d4e34
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import static org.mockito.Mockito.*;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.extension.ExtensionPoint;
+import com.gemstone.gemfire.test.fake.Fakes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+@Category(UnitTest.class)
+public class LuceneIndexForPartitionedRegionTest {
+
+  @Rule
+  public ExpectedException expectedExceptions = ExpectedException.none();
+
+  @Test
+  public void getIndexNameReturnsCorrectName() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = null;
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    Assert.assertEquals(name, index.getName());
+  }
+
+  @Test
+  public void getRegionPathReturnsPath() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = null;
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    Assert.assertEquals(regionPath, index.getRegionPath());
+  }
+
+  @Test
+  public void fileRegionExistsWhenFileRegionExistsShouldReturnTrue() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    PartitionedRegion region = mock(PartitionedRegion.class);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    String fileRegionName = index.createFileRegionName();
+    when(cache.getRegion(fileRegionName)).thenReturn(region);
+
+    Assert.assertTrue(index.fileRegionExists(fileRegionName));
+  }
+
+  @Test
+  public void fileRegionExistsWhenFileRegionDoesNotExistShouldReturnFalse() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    String fileRegionName = index.createFileRegionName();
+    when(cache.getRegion(fileRegionName)).thenReturn(null);
+
+    Assert.assertFalse(index.fileRegionExists(fileRegionName));
+  }
+
+  @Test
+  public void chunkRegionExistsWhenChunkRegionExistsShouldReturnTrue() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    PartitionedRegion region = mock(PartitionedRegion.class);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    String chunkRegionName = index.createChunkRegionName();
+    when(cache.getRegion(chunkRegionName)).thenReturn(region);
+
+    Assert.assertTrue(index.chunkRegionExists(chunkRegionName));
+  }
+
+  @Test
+  public void chunkRegionExistsWhenChunkRegionDoesNotExistShouldReturnFalse() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    String chunkRegionName = index.createChunkRegionName();
+    when(cache.getRegion(chunkRegionName)).thenReturn(null);
+
+    Assert.assertFalse(index.chunkRegionExists(chunkRegionName));
+  }
+
+  @Test
+  public void createChunkRegionWithPartitionShortcutCreatesRegionThroughFactory() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    RegionFactory regionFactory = mock(RegionFactory.class);
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    when(cache.createRegionFactory(RegionShortcut.PARTITION)).thenReturn(regionFactory);
+    when(regionFactory.setPartitionAttributes(any())).thenReturn(regionFactory);
+    when(partitionAttributes.getTotalNumBuckets()).thenReturn(113);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+
+    index.createChunkRegion(RegionShortcut.PARTITION, index.createFileRegionName(), partitionAttributes, index.createChunkRegionName());
+    verify(regionFactory).create(index.createChunkRegionName());
+  }
+
+  @Test
+  public void createFileRegionWithPartitionShortcutCreatesRegionThroughFactory() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    RegionFactory regionFactory = mock(RegionFactory.class);
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    when(cache.createRegionFactory(RegionShortcut.PARTITION)).thenReturn(regionFactory);
+    when(regionFactory.setPartitionAttributes(any())).thenReturn(regionFactory);
+    when(partitionAttributes.getTotalNumBuckets()).thenReturn(113);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+
+    index.createFileRegion(RegionShortcut.PARTITION, index.createFileRegionName(), partitionAttributes);
+    verify(regionFactory).create(index.createFileRegionName());
+  }
+
+  @Test
+  public void createAEQWithPersistenceCallsCreateOnAEQFactory() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
+    when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.createAEQWithPersistence();
+
+    verify(aeqFactory).create(any(), any());
+  }
+
+  @Test
+  public void createAEQCallsCreateOnAEQFactory() {
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
+    when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.createAEQ();
+
+    verify(aeqFactory).create(any(), any());
+  }
+
+  private void initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache) {
+    int defaultLocalMemory = 100;
+    initializeScenario(withPersistence, regionPath, cache, defaultLocalMemory);
+  }
+
+  private void initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache, int localMaxMemory) {
+    PartitionedRegion region = mock(PartitionedRegion.class);
+    RegionAttributes regionAttributes = mock(RegionAttributes.class);
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    RegionFactory regionFactory = mock(RegionFactory.class);
+    DataPolicy dataPolicy = mock(DataPolicy.class);
+    ExtensionPoint extensionPoint = mock(ExtensionPoint.class);
+    when(cache.getRegion(regionPath)).thenReturn(region);
+    when(cache.createRegionFactory(isA(RegionShortcut.class))).thenReturn(regionFactory);
+    when(region.getAttributes()).thenReturn(regionAttributes);
+    when(regionAttributes.getPartitionAttributes()).thenReturn(partitionAttributes);
+    when(regionAttributes.getDataPolicy()).thenReturn(dataPolicy);
+    when(partitionAttributes.getLocalMaxMemory()).thenReturn(localMaxMemory);
+    when(partitionAttributes.getTotalNumBuckets()).thenReturn(113);
+    when(dataPolicy.withPersistence()).thenReturn(withPersistence);
+    when(region.getExtensionPoint()).thenReturn(extensionPoint);
+  }
+
+  @Test
+  public void initializeWithNoLocalMemoryThrowsException() {
+    expectedExceptions.expect(IllegalStateException.class);
+    expectedExceptions.expectMessage("The data region to create lucene index should be with storage");
+    boolean withPersistence = false;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache, 0);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.initialize();
+  }
+
+  @Test
+  public void initializeWithPersistenceShouldCreateAEQWithPersistence() {
+    boolean withPersistence = true;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQWithPersistence();
+    spy.initialize();
+
+    verify(spy).createAEQWithPersistence();
+  }
+
+  @Test
+  public void initializeWithoutPersistenceShouldCreateAEQ() {
+    boolean withPersistence = false;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQ();
+    spy.initialize();
+
+    verify(spy).createAEQ();
+  }
+
+  @Test
+  public void initializeShouldCreatePartitionChunkRegion() {
+    boolean withPersistence = false;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQ();
+    spy.initialize();
+
+    verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()));
+  }
+
+  @Test
+  public void initializeShouldCreatePartitionFileRegion() {
+    boolean withPersistence = false;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQ();
+    spy.initialize();
+
+    verify(spy).createFileRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()), any());
+  }
+
+  @Test
+  public void initializeShouldCreatePartitionPersistentChunkRegion() {
+    boolean withPersistence = true;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQWithPersistence();
+    spy.initialize();
+
+    verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()));
+  }
+
+  @Test
+  public void initializeShouldCreatePartitionPersistentFileRegion() {
+    boolean withPersistence = true;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQWithPersistence();
+    spy.initialize();
+
+    verify(spy).createFileRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any());
+  }
+
+  @Test
+  public void initializeWhenCalledMultipleTimesShouldNotCreateMultipleFileRegions() {
+    boolean withPersistence = true;
+    String name = "indexName";
+    String regionPath = "regionName";
+    Cache cache = Fakes.cache();
+    initializeScenario(withPersistence, regionPath, cache);
+
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    index.setSearchableFields(new String[]{"field"});
+    LuceneIndexForPartitionedRegion spy = spy(index);
+    doReturn(null).when(spy).createFileRegion(any(), any(), any());
+    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
+    doReturn(null).when(spy).createAEQWithPersistence();
+    spy.initialize();
+    spy.initialize();
+
+    verify(spy).createFileRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any());
+  }
+
+}