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/04 22:58:00 UTC

[60/63] [abbrv] incubator-geode git commit: GEODE-11: Refactoring the LuceneFunctionReadPathDUnitTest

GEODE-11: Refactoring the LuceneFunctionReadPathDUnitTest

Refactoring this test into a framework for adding more tests with a
bunch of subclasses.


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

Branch: refs/heads/feature/GEODE-1276
Commit: 0481732f0223a38adf4084bc2afb977e20db364f
Parents: 4a6c779
Author: Dan Smith <up...@apache.org>
Authored: Mon May 2 13:56:10 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue May 3 16:41:29 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQueriesBase.java | 148 ++++++++++++
 .../cache/lucene/LuceneQueriesPRBase.java       |  75 ++++++
 .../lucene/LuceneQueriesPeerPRDUnitTest.java    |  36 +++
 .../LuceneQueriesPeerPROverflowDUnitTest.java   |  41 ++++
 .../LuceneFunctionReadPathDUnitTest.java        | 237 -------------------
 5 files changed, 300 insertions(+), 237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0481732f/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
new file mode 100644
index 0000000..c467a18
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -0,0 +1,148 @@
+/*
+ * 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 org.junit.Assert.assertEquals;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+
+import org.junit.Test;
+
+/**
+  * This test class is intended to contain basic integration tests
+  * of the lucene query class that should be executed against a number
+  * of different regions types and topologies.
+  *
+  */
+public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
+
+  protected static final String INDEX_NAME = "index";
+  protected static final String REGION_NAME = "index";
+  private static final long serialVersionUID = 1L;
+  protected VM dataStore1;
+  protected VM dataStore2;
+  protected VM accessor;
+
+  @Override
+  public final void postSetUp() throws Exception {
+    Host host = Host.getHost(0);
+    dataStore1 = host.getVM(0);
+    dataStore2 = host.getVM(1);
+    accessor = host.getVM(3);
+  }
+
+  protected abstract void initDataStore(SerializableRunnableIF createIndex) throws Exception;
+
+  protected abstract void initAccessor(SerializableRunnableIF createIndex) throws Exception;
+
+  @Test
+  public void returnCorrectResultsFromStringQueryWithDefaultAnalyzer() {
+    SerializableRunnableIF createIndex = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    };
+    dataStore1.invoke(() -> initDataStore(createIndex));
+    dataStore2.invoke(() -> initDataStore(createIndex));
+    accessor.invoke(() -> initAccessor(createIndex));
+
+    putDataInRegion(accessor);
+    executeTextSearch(accessor);
+  }
+
+  protected void executeTextSearch(VM vm) {
+    vm.invoke(() -> {
+      Cache cache = getCache();
+      Region<Object, Object> region = cache.getRegion(REGION_NAME);
+
+      LuceneService service = LuceneServiceProvider.get(cache);
+      LuceneQuery<Integer, TestObject> query;
+      query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "text:world");
+      LuceneQueryResults<Integer, TestObject> results = query.search();
+      assertEquals(3, results.size());
+      List<LuceneResultStruct<Integer, TestObject>> page = results.getNextPage();
+
+      Map<Integer, TestObject> data = new HashMap<Integer, TestObject>();
+      for (LuceneResultStruct<Integer, TestObject> row : page) {
+        data.put(row.getKey(), row.getValue());
+      }
+
+      assertEquals(new HashMap(region),data);
+      return null;
+    });
+  }
+
+  protected void putDataInRegion(VM vm) {
+    vm.invoke(() -> {
+      final Cache cache = getCache();
+      Region<Object, Object> region = cache.getRegion(REGION_NAME);
+      region.put(1, new TestObject("hello world"));
+      region.put(113, new TestObject("hi world"));
+      region.put(2, new TestObject("goodbye world"));
+    });
+  }
+
+  private static class TestObject implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String text;
+
+    public TestObject(String text) {
+      this.text = text;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((text == null) ? 0 : text.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      TestObject other = (TestObject) obj;
+      if (text == null) {
+        if (other.text != null)
+          return false;
+      } else if (!text.equals(other.text))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "TestObject[" + text + "]";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0481732f/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
new file mode 100644
index 0000000..fbd101e
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.junit.Assert.*;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.control.RebalanceOperation;
+import com.gemstone.gemfire.cache.control.RebalanceResults;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+/**
+ * This test class adds more basic tests of lucene functionality
+ * for partitioned regions. These tests should work across all types
+ * of PRs and topologies.
+ *
+ */
+public abstract class LuceneQueriesPRBase extends LuceneQueriesBase {
+
+  @Test
+  public void returnCorrectResultsAfterRebalance() {
+    SerializableRunnableIF createIndex = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    };
+    dataStore1.invoke(() -> initDataStore(createIndex));
+    accessor.invoke(() -> initAccessor(createIndex));
+    putDataInRegion(accessor);
+    dataStore2.invoke(() -> initDataStore(createIndex));
+
+    rebalanceRegion(dataStore1);
+    executeTextSearch(accessor);
+  }
+
+  private void rebalanceRegion(VM vm) {
+    // Do a rebalance
+    vm.invoke(() -> {
+        RebalanceOperation op = getCache().getResourceManager().createRebalanceFactory().start();
+        RebalanceResults results = op.getResults();
+        assertTrue("Transferred " + results.getTotalBucketTransfersCompleted(), 1 < results.getTotalBucketTransfersCompleted());
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0481732f/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
new file mode 100644
index 0000000..51d0a33
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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 com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class LuceneQueriesPeerPRDUnitTest extends LuceneQueriesPRBase {
+
+  @Override protected void initDataStore(final SerializableRunnableIF createIndex) throws Exception {
+    createIndex.run();
+    getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+  }
+
+  @Override protected void initAccessor(final SerializableRunnableIF createIndex) throws Exception {
+    initDataStore(createIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0481732f/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
new file mode 100644
index 0000000..cf2bac7
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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 com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class LuceneQueriesPeerPROverflowDUnitTest extends LuceneQueriesPRBase {
+
+  @Override protected void initDataStore(final SerializableRunnableIF createIndex) throws Exception {
+    createIndex.run();
+    EvictionAttributes evicAttr = EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK);
+    getCache().createRegionFactory(RegionShortcut.PARTITION_OVERFLOW)
+      .setEvictionAttributes(evicAttr)
+      .create(REGION_NAME);
+  }
+
+  @Override protected void initAccessor(final SerializableRunnableIF createIndex) throws Exception {
+    initDataStore(createIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0481732f/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
deleted file mode 100644
index bc62578..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
+++ /dev/null
@@ -1,237 +0,0 @@
-/*
- * 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.distributed;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.EvictionAction;
-import com.gemstone.gemfire.cache.EvictionAlgorithm;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.control.RebalanceOperation;
-import com.gemstone.gemfire.cache.control.RebalanceResults;
-import com.gemstone.gemfire.cache.lucene.LuceneQuery;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
-import com.gemstone.gemfire.cache.lucene.LuceneService;
-import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
-import com.gemstone.gemfire.cache30.CacheTestCase;
-import com.gemstone.gemfire.internal.cache.EvictionAttributesImpl;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.SerializableCallable;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Assert;
-import org.junit.experimental.categories.Category;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CancellationException;
-
-@Category(DistributedTest.class)
-public class LuceneFunctionReadPathDUnitTest extends CacheTestCase {
-  private static final String INDEX_NAME = "index";
-
-  private static final long serialVersionUID = 1L;
-
-  private VM server1;
-  private VM server2;
-
-  public LuceneFunctionReadPathDUnitTest(String name) {
-    super(name);
-  }
-
-  @Override
-  public final void postSetUp() throws Exception {
-    Host host = Host.getHost(0);
-    server1 = host.getVM(0);
-    server2 = host.getVM(1);
-  }
-
-  public void testEnd2EndFunctionExecution() {
-    e2eTextSearchForRegionType(RegionShortcut.PARTITION);
-    e2eTextSearchForRegionType(RegionShortcut.PARTITION_PERSISTENT);
-    e2eTextSearchForRegionType(RegionShortcut.PARTITION_OVERFLOW);
-    e2eTextSearchForRegionType(RegionShortcut.PARTITION_PERSISTENT_OVERFLOW);
-  }
-
-  private void e2eTextSearchForRegionType(RegionShortcut type) {
-    final String regionName = type.toString();
-    createRegionAndIndex(server1, regionName, type);
-    putDataInRegion(server1, regionName);
-    createRegionAndIndex(server2, regionName, type);
-    // Make sure we can search from both members
-    executeTextSearch(server1, regionName);
-    executeTextSearch(server2, regionName);
-
-    rebalanceRegion(server1);
-    // Make sure the search still works
-    executeTextSearch(server1, regionName);
-    executeTextSearch(server2, regionName);
-    destroyPartitionRegion(server2, regionName);
-  }
-
-  private void rebalanceRegion(VM vm) {
-    // Do a rebalance
-    vm.invoke(new SerializableCallable<Object>() {
-      private static final long serialVersionUID = 1L;
-
-      @Override
-      public Object call() throws CancellationException, InterruptedException {
-        RebalanceOperation op = getCache().getResourceManager().createRebalanceFactory().start();
-        RebalanceResults results = op.getResults();
-        assertTrue(1 < results.getTotalBucketTransfersCompleted());
-        return null;
-      }
-    });
-  }
-
-  private void executeTextSearch(VM vm, final String regionName) {
-    SerializableCallable<Object> executeSearch = new SerializableCallable<Object>("executeSearch") {
-      private static final long serialVersionUID = 1L;
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        assertNotNull(cache);
-        Region<Object, Object> region = cache.getRegion(regionName);
-        Assert.assertNotNull(region);
-
-        LuceneService service = LuceneServiceProvider.get(cache);
-        LuceneQuery<Integer, TestObject> query;
-        query = service.createLuceneQueryFactory().create(INDEX_NAME, regionName, "text:world");
-        LuceneQueryResults<Integer, TestObject> results = query.search();
-        assertEquals(3, results.size());
-        List<LuceneResultStruct<Integer, TestObject>> page = results.getNextPage();
-
-        Map<Integer, TestObject> data = new HashMap<Integer, TestObject>();
-        for (LuceneResultStruct<Integer, TestObject> row : page) {
-          data.put(row.getKey(), row.getValue());
-        }
-
-        assertEquals(data, region);
-        return null;
-      }
-    };
-
-    vm.invoke(executeSearch);
-  }
-
-  private void putDataInRegion(VM vm, final String regionName) {
-    SerializableCallable<Object> createSomeData = new SerializableCallable<Object>("putDataInRegion") {
-      private static final long serialVersionUID = 1L;
-
-      public Object call() throws Exception {
-        final Cache cache = getCache();
-        Region<Object, Object> region = cache.getRegion(regionName);
-        assertNotNull(region);
-        region.put(1, new TestObject("hello world"));
-        region.put(113, new TestObject("hi world"));
-        region.put(2, new TestObject("goodbye world"));
-
-        return null;
-      }
-    };
-
-    vm.invoke(createSomeData);
-  }
-
-  private void createRegionAndIndex(VM vm, final String regionName, final RegionShortcut type) {
-    SerializableCallable<Object> createRegion = new SerializableCallable<Object>("createRegionAndIndex") {
-      private static final long serialVersionUID = 1L;
-
-      public Object call() throws Exception {
-        final Cache cache = getCache();
-        assertNotNull(cache);
-        LuceneService service = LuceneServiceProvider.get(cache);
-        service.createIndex(INDEX_NAME, regionName, "text");
-        RegionFactory<Object, Object> regionFactory = cache.createRegionFactory(type);
-        if (regionName.contains("OVERFLOW")) {
-          EvictionAttributesImpl evicAttr = new EvictionAttributesImpl().setAction(EvictionAction.OVERFLOW_TO_DISK);
-          evicAttr.setAlgorithm(EvictionAlgorithm.LRU_ENTRY).setMaximum(1);
-          regionFactory.setEvictionAttributes(evicAttr);
-        }
-        regionFactory.create(regionName);
-        return null;
-      }
-    };
-    vm.invoke(createRegion);
-  }
-
-  private void destroyPartitionRegion(VM vm, final String regionName) {
-    SerializableCallable<Object> createPartitionRegion = new SerializableCallable<Object>("destroyPartitionRegion") {
-      private static final long serialVersionUID = 1L;
-
-      public Object call() throws Exception {
-        final Cache cache = getCache();
-        assertNotNull(cache);
-        String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, regionName);
-        PartitionedRegion chunkRegion = (PartitionedRegion) cache.getRegion(aeqId + ".chunks");
-        assertNotNull(chunkRegion);
-        chunkRegion.destroyRegion();
-        PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
-        assertNotNull(fileRegion);
-        fileRegion.destroyRegion();
-        Region<Object, Object> region = cache.getRegion(regionName);
-        assertNotNull(region);
-        region.destroyRegion();
-        return null;
-      }
-    };
-    vm.invoke(createPartitionRegion);
-  }
-
-  private static class TestObject implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private String text;
-
-    public TestObject(String text) {
-      this.text = text;
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      int result = 1;
-      result = prime * result + ((text == null) ? 0 : text.hashCode());
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj)
-        return true;
-      if (obj == null)
-        return false;
-      if (getClass() != obj.getClass())
-        return false;
-      TestObject other = (TestObject) obj;
-      if (text == null) {
-        if (other.text != null)
-          return false;
-      } else if (!text.equals(other.text))
-        return false;
-      return true;
-    }
-  }
-}