Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 76DC8200AEE for ; Wed, 4 May 2016 01:59:10 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 755D31609F6; Wed, 4 May 2016 01:59:10 +0200 (CEST) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 275241609F5 for ; Wed, 4 May 2016 01:59:09 +0200 (CEST) Received: (qmail 52058 invoked by uid 500); 3 May 2016 23:59:08 -0000 Mailing-List: contact commits-help@geode.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@geode.incubator.apache.org Delivered-To: mailing list commits@geode.incubator.apache.org Received: (qmail 52049 invoked by uid 99); 3 May 2016 23:59:08 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 03 May 2016 23:59:08 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id E331EC04B9 for ; Tue, 3 May 2016 23:59:07 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -3.221 X-Spam-Level: X-Spam-Status: No, score=-3.221 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id cKbsjMH9g2f7 for ; Tue, 3 May 2016 23:59:04 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 621AA5F39D for ; Tue, 3 May 2016 23:59:03 +0000 (UTC) Received: (qmail 51873 invoked by uid 99); 3 May 2016 23:59:02 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 03 May 2016 23:59:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 75774DFB79; Tue, 3 May 2016 23:59:02 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: upthewaterspout@apache.org To: commits@geode.incubator.apache.org Date: Tue, 03 May 2016 23:59:02 -0000 Message-Id: <7ba16f01399c465c9c4f4316817f416a@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/3] incubator-geode git commit: GEODE-11: Refactoring the LuceneFunctionReadPathDUnitTest archived-at: Tue, 03 May 2016 23:59:10 -0000 Repository: incubator-geode Updated Branches: refs/heads/develop 4a6c779d3 -> 8dc2d3031 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/develop Commit: 0481732f0223a38adf4084bc2afb977e20db364f Parents: 4a6c779 Author: Dan Smith Authored: Mon May 2 13:56:10 2016 -0700 Committer: Dan Smith 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 region = cache.getRegion(REGION_NAME); + + LuceneService service = LuceneServiceProvider.get(cache); + LuceneQuery query; + query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "text:world"); + LuceneQueryResults results = query.search(); + assertEquals(3, results.size()); + List> page = results.getNextPage(); + + Map data = new HashMap(); + for (LuceneResultStruct 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 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() { - 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 executeSearch = new SerializableCallable("executeSearch") { - private static final long serialVersionUID = 1L; - - public Object call() throws Exception { - Cache cache = getCache(); - assertNotNull(cache); - Region region = cache.getRegion(regionName); - Assert.assertNotNull(region); - - LuceneService service = LuceneServiceProvider.get(cache); - LuceneQuery query; - query = service.createLuceneQueryFactory().create(INDEX_NAME, regionName, "text:world"); - LuceneQueryResults results = query.search(); - assertEquals(3, results.size()); - List> page = results.getNextPage(); - - Map data = new HashMap(); - for (LuceneResultStruct 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 createSomeData = new SerializableCallable("putDataInRegion") { - private static final long serialVersionUID = 1L; - - public Object call() throws Exception { - final Cache cache = getCache(); - Region 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 createRegion = new SerializableCallable("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 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 createPartitionRegion = new SerializableCallable("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 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; - } - } -}