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 21:17:21 UTC

[2/5] incubator-geode git commit: GEODE-11: Added transaction tests for Lucene indexes

GEODE-11: Added transaction tests for Lucene indexes

* Moved tests from LuceneServiceImplIntegrationTest to better locations
* Added LuceneIndexMaintenanceIntegrationTest


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

Branch: refs/heads/feature/GEODE-835
Commit: afc5dde7a62ba55c214c40b97538ba262a8fe509
Parents: 425581c
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue May 17 14:33:38 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Fri May 20 13:47:18 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexCreationIntegrationTest.java     |  38 ++--
 ...IndexCreationPersistenceIntegrationTest.java |  53 +++++
 .../LuceneIndexMaintenanceIntegrationTest.java  | 119 +++++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |   8 +
 .../LuceneServiceImplIntegrationTest.java       | 205 +------------------
 5 files changed, 214 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/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 4c28938..d1cd8ac 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,12 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.RegionShortcut.*;
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static junitparams.JUnitParamsRunner.*;
 import static org.junit.Assert.*;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -42,11 +45,15 @@ import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 
 import junitparams.JUnitParamsRunner;
@@ -65,6 +72,8 @@ import junitparams.Parameters;
 @RunWith(JUnitParamsRunner.class)
 public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
 
   @Test
   public void shouldCreateIndexWriterWithAnalyzersWhenSettingPerFieldAnalyzers()
@@ -136,20 +145,6 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldCreateInternalRegionsForIndex() {
-    createIndex("field1", "field2");
-
-    // Create partitioned region
-    createRegion();
-
-    verifyInternalRegions(region -> {
-      region.isInternalRegion();
-      assertNotNull(region.getAttributes().getPartitionAttributes().getColocatedWith());
-      cache.rootRegions().contains(region);
-    });
-  }
-
-  @Test
   public void shouldUseFixedPartitionsForInternalRegions() {
     createIndex("text");
 
@@ -167,13 +162,26 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     });
   }
 
+  @Test(expected = IllegalStateException.class)
+  public void cannotCreateLuceneIndexAfterRegionHasBeenCreated() throws IOException, ParseException {
+    createRegion();
+    createIndex("field1", "field2", "field3");
+  }
+
+  @Test
+  public void cannotCreateLuceneIndexForReplicateRegion() throws IOException, ParseException {
+    expectedException.expect(UnsupportedOperationException.class);
+    expectedException.expectMessage("Lucene indexes on replicated regions are not supported");
+    createIndex("field1", "field2", "field3");
+    this.cache.createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+  }
 
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
     LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
 
   private Region createRegion() {
-    return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+    return createRegion(REGION_NAME, RegionShortcut.PARTITION);
   }
 
   private void createIndex(String ... fieldNames) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/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
index d6bf116..c43b592 100644
--- 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
@@ -18,7 +18,9 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.RegionShortcut.*;
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static junitparams.JUnitParamsRunner.$;
 import static org.junit.Assert.*;
 
 import java.io.File;
@@ -130,7 +132,58 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     assertEquals(1, query.search().size());
   }
 
+  @Test
+  @Parameters(method = "getRegionShortcuts")
+  public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws ParseException {
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_1", REGION_NAME, "field1");
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_2", REGION_NAME, "field2");
+    Region region = cache.createRegionFactory(shortcut).create(REGION_NAME);
+    region.put("key1", new TestObject());
+    verifyQueryResultSize(INDEX_NAME+"_1", REGION_NAME, "field1:world", 1);
+    verifyQueryResultSize(INDEX_NAME+"_2", REGION_NAME, "field2:field", 1);
+  }
+
+  @Test
+  @Parameters(method = "getRegionShortcuts")
+  public void shouldCreateInternalRegionsForIndex(RegionShortcut shortcut) {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+
+    // Create partitioned region
+    createRegion(REGION_NAME, shortcut);
+
+    verifyInternalRegions(region -> {
+      region.isInternalRegion();
+      assertTrue(region.isInternalRegion());
+
+      assertNotNull(region.getAttributes().getPartitionAttributes().getColocatedWith());
+      cache.rootRegions().contains(region);
+      assertFalse(cache.rootRegions().contains(region));
+    });
+  }
+
+  private void verifyQueryResultSize(String indexName, String regionName, String queryString, int size) throws ParseException {
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(indexName, regionName, queryString);
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      assertEquals(size, query.search().size());
+    });
+  }
+
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
     LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
+
+
+  private static final Object[] getRegionShortcuts() {
+    return $(
+      new Object[] { PARTITION },
+      new Object[] { PARTITION_REDUNDANT },
+      new Object[] { PARTITION_PERSISTENT },
+      new Object[] { PARTITION_REDUNDANT_PERSISTENT },
+      new Object[] { PARTITION_OVERFLOW },
+      new Object[] { PARTITION_REDUNDANT_OVERFLOW },
+      new Object[] { PARTITION_PERSISTENT_OVERFLOW },
+      new Object[] { PARTITION_REDUNDANT_PERSISTENT_OVERFLOW }
+    );
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/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
new file mode 100644
index 0000000..07780ca
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.Serializable;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest {
+
+  private static int WAIT_FOR_FLUSH_TIME = 10000;
+
+  @Test
+  public void indexIsNotUpdatedIfTransactionHasNotCommittedYet() 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"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"");
+    LuceneQueryResults<Integer, TestObject> results = query.search();
+    assertEquals(3, results.size());
+
+    //begin transaction
+    cache.getCacheTransactionManager().begin();
+    region.put("object-1", new TestObject("title 1", "updated"));
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    assertEquals(3, query.search().size());
+  }
+
+  @Test
+  public void indexIsUpdatedAfterTransactionHasCommitted() 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"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"");
+    LuceneQueryResults<Integer, TestObject> results = query.search();
+    assertEquals(3, results.size());
+
+    cache.getCacheTransactionManager().begin();
+    region.put("object-1", new TestObject("title 1", "updated"));
+    cache.getCacheTransactionManager().commit();
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+
+    assertEquals(2, query.search().size());
+  }
+
+  @Test
+  public void indexIsNotUpdatedAfterTransactionRollback() 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"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"");
+    LuceneQueryResults<Integer, TestObject> results = query.search();
+    assertEquals(3, results.size());
+
+    cache.getCacheTransactionManager().begin();
+    region.put("object-1", new TestObject("title 1", "updated"));
+    cache.getCacheTransactionManager().rollback();
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+
+    assertEquals(3, query.search().size());
+  }
+
+  private static class TestObject implements Serializable {
+
+    String title;
+    String description;
+
+    public TestObject(String title, String description) {
+      this.title = title;
+      this.description = description;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/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 c302460..6d8e370 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
@@ -19,10 +19,14 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+
 import java.io.File;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -60,4 +64,8 @@ public class LuceneIntegrationTest {
     cf.set("locators", "");
     return cf;
   }
+
+  protected Region createRegion(String regionName, RegionShortcut shortcut) {
+    return this.cache.createRegionFactory(shortcut).create(regionName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
index fa3392c..a078910 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
@@ -16,34 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import com.gemstone.gemfire.cache.Region;
-
-import org.apache.logging.log4j.Logger;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.core.KeywordAnalyzer;
-import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.queryparser.classic.ParseException;
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.execute.Function;
@@ -51,31 +30,26 @@ import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
-import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
 @Category(IntegrationTest.class)
 public class LuceneServiceImplIntegrationTest {
 
   Cache cache;
   ClientCache clientCache;
-  private LuceneIndexImpl repo;
-  private HeterogeneousLuceneSerializer mapper;
-  private StandardAnalyzer analyzer = new StandardAnalyzer();
-  private IndexWriter writer;
   LuceneServiceImpl service = null;
-  private static final Logger logger = LogService.getLogger();
 
   @Rule
   public ExpectedException expectedException = ExpectedException.none();
 
   @Test
-  public void luceneServiceProviderGetShouldAcceptClientCacheAsAParameter(){
+  public void luceneServiceProviderGetShouldAcceptClientCacheAsAParameter() {
     clientCache = getClientCache();
     LuceneService luceneService = LuceneServiceProvider.get(clientCache);
     assertNotNull(luceneService);
@@ -107,7 +81,7 @@ public class LuceneServiceImplIntegrationTest {
       cache.close();
       cache = null;
     }
-    if (null != clientCache  && !clientCache.isClosed()) {
+    if (null != clientCache && !clientCache.isClosed()) {
       clientCache.close();
       clientCache = null;
     }
@@ -117,7 +91,7 @@ public class LuceneServiceImplIntegrationTest {
     if (null == clientCache) {
       clientCache = new ClientCacheFactory().set("mcast-port", "0").create();
     }
-    else{
+    else {
       return clientCache;
     }
     return clientCache;
@@ -130,163 +104,6 @@ public class LuceneServiceImplIntegrationTest {
     return cache;
   }
 
-  private LuceneService getService() {
-    if (null == cache) {
-      getCache();
-    }
-    if (null == service) {
-      service = (LuceneServiceImpl)LuceneServiceProvider.get(cache);
-    }
-    return service;
-  }
-
-
-  private Region createRegion(String regionName, RegionShortcut shortcut) {
-    return cache.createRegionFactory(shortcut).create(regionName);
-  }
-
-  private LocalRegion createPR(String regionName, boolean isSubRegion) {
-    if (isSubRegion) {
-      LocalRegion root = (LocalRegion) cache.createRegionFactory(RegionShortcut.PARTITION).create("root");
-      LocalRegion region = (LocalRegion) cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
-        createSubregion(root, regionName);
-      return region;
-    }
-    else {
-      LocalRegion region = (LocalRegion) createRegion(regionName, RegionShortcut.PARTITION_PERSISTENT);
-      return region;
-    }
-  }
-
-  private LocalRegion createRR(String regionName, boolean isSubRegion) {
-    if (isSubRegion) {
-
-      LocalRegion root = (LocalRegion) cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
-      LocalRegion region = (LocalRegion) cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
-        createSubregion(root, regionName);
-      return region;
-    }
-    else {
-      LocalRegion region = (LocalRegion) createRegion(regionName, RegionShortcut.REPLICATE_PERSISTENT);
-      return region;
-    }
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void cannotCreateLuceneIndexAfterRegionHasBeenCreated() throws IOException, ParseException {
-    getService();
-
-    LocalRegion userRegion = createPR("PR1", false);
-    service.createIndex("index1", "PR1", "field1", "field2", "field3");
-  }
-
-  @Test
-  public void canCreateLuceneIndexForPRWithAnalyzer() throws IOException, ParseException {
-    getService();
-    StandardAnalyzer sa = new StandardAnalyzer();
-    KeywordAnalyzer ka = new KeywordAnalyzer();
-    Map<String, Analyzer> analyzerPerField = new HashMap<String, Analyzer>();
-    analyzerPerField.put("field1", ka);
-    analyzerPerField.put("field2", sa);
-    analyzerPerField.put("field3", sa);
-    //  field2 and field3 will use StandardAnalyzer
-    PerFieldAnalyzerWrapper analyzer2 = new PerFieldAnalyzerWrapper(sa, analyzerPerField);
-
-    service.createIndex("index1", "PR1", analyzerPerField);
-    createPR("PR1", false);
-    LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
-    assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion) index1;
-    assertEquals("index1", index1.getName());
-    assertEquals("/PR1", index1.getRegionPath());
-    String[] fields1 = index1.getFieldNames();
-    assertEquals(3, fields1.length);
-    Analyzer analyzer = index1PR.getAnalyzer();
-    assertTrue(analyzer instanceof PerFieldAnalyzerWrapper);
-    RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
-    assertTrue(RepositoryManager != null);
-
-    final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".files";
-    final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".chunks";
-    PartitionedRegion filePR = (PartitionedRegion) cache.getRegion(fileRegionName);
-    PartitionedRegion chunkPR = (PartitionedRegion) cache.getRegion(chunkRegionName);
-    assertTrue(filePR != null);
-    assertTrue(chunkPR != null);
-  }
-
-  @Test
-  public void cannotCreateLuceneIndexForReplicateRegion() throws IOException, ParseException {
-    expectedException.expect(UnsupportedOperationException.class);
-    expectedException.expectMessage("Lucene indexes on replicated regions are not supported");
-    getService();
-    service.createIndex("index1", "RR1", "field1", "field2", "field3");
-    createRR("RR1", false);
-  }
-
-  @Test
-  public void canCreateIndexForAllNonProxyPartitionRegionTypes() {
-    for (RegionShortcut shortcut : RegionShortcut.values()) {
-      String sname = shortcut.name().toLowerCase();
-      if (sname.contains("partition") && !sname.contains("proxy")) {
-        canCreateLuceneIndexForPRType(shortcut);
-        //Destroying cache and service for now because aeq's are not completely being cleaned up correctly after
-        // being destroyed.  Instead we should close the aeq and clean up any regions associated with this lucene
-        //index but only after aeq destroy works properly
-        destroyCache();
-        destroyService();
-      }
-    }
-  }
-
-  private void canCreateLuceneIndexForPRType(RegionShortcut regionShortcut) {
-    getService();
-    service.createIndex("index1", "PR1", "field1", "field2", "field3");
-    Region region = null;
-    AsyncEventQueueImpl aeq = null;
-    try {
-      region = createRegion("PR1", regionShortcut);
-      LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
-      assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-      LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion) index1;
-      assertEquals("index1", index1.getName());
-      assertEquals("/PR1", index1.getRegionPath());
-      String[] fields1 = index1.getFieldNames();
-      assertEquals(3, fields1.length);
-      Analyzer analyzer = index1PR.getAnalyzer();
-      assertTrue(analyzer instanceof StandardAnalyzer);
-      RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
-      assertTrue(RepositoryManager != null);
-
-      final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".files";
-      final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".chunks";
-      PartitionedRegion filePR = (PartitionedRegion) cache.getRegion(fileRegionName);
-      PartitionedRegion chunkPR = (PartitionedRegion) cache.getRegion(chunkRegionName);
-      assertTrue(filePR != null);
-      assertTrue(chunkPR != null);
+}
 
-      String aeqId = LuceneServiceImpl.getUniqueIndexName(index1.getName(), index1.getRegionPath());
-      aeq = (AsyncEventQueueImpl) cache.getAsyncEventQueue(aeqId);
-      assertTrue(aeq != null);
 
-      //Make sure our queue doesn't show up in the list of async event queues
-      assertEquals(Collections.emptySet(), cache.getAsyncEventQueues());
-    }
-    finally {
-      String aeqId = LuceneServiceImpl.getUniqueIndexName("index1", "PR1");
-      PartitionedRegion chunkRegion = (PartitionedRegion) cache.getRegion(aeqId + ".chunks");
-      if (chunkRegion != null) {
-        chunkRegion.destroyRegion();
-      }
-      PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
-      if (fileRegion != null) {
-        fileRegion.destroyRegion();
-      }
-      ((GemFireCacheImpl) cache).removeAsyncEventQueue(aeq);
-      if (aeq != null) {
-        aeq.destroy();
-      }
-      region.destroyRegion();
-    }
-  }
-
-}