You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ag...@apache.org on 2016/05/17 00:38:23 UTC

[14/50] [abbrv] incubator-geode git commit: GEODE-1330: Set boolean so that internal lucene regions aren't generated in xml

GEODE-1330: Set boolean so that internal lucene regions aren't generated in xml


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

Branch: refs/heads/feature/GEODE-1209
Commit: 3fa22f0d4c472d2d06bbeaaea92413e483207a05
Parents: 52ea8df
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Mon May 9 15:00:31 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Mon May 16 17:31:54 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexForPartitionedRegion.java        | 38 ++++-----
 .../LuceneIndexForReplicatedRegion.java         |  1 +
 .../cache/lucene/internal/LuceneIndexImpl.java  | 29 +++++++
 .../LuceneIndexCreationIntegrationTest.java     | 74 ++++++++++++++++++
 .../LuceneIndexForPartitionedRegionTest.java    | 82 +++++++++++---------
 .../lucene/internal/LuceneIntegrationTest.java  | 47 +++++++++++
 6 files changed, 217 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fa22f0d/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 28ff543..382f61c 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
@@ -19,6 +19,7 @@
 
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.PartitionAttributes;
@@ -37,12 +38,8 @@ import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 /* wrapper of IndexWriter */
 public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
 
-  private final Cache cache;
-
   public LuceneIndexForPartitionedRegion(String indexName, String regionPath, Cache cache) {
-    this.indexName = indexName;
-    this.regionPath = regionPath;
-    this.cache = cache;
+    super(indexName, regionPath, cache);
   }
 
   @Override
@@ -138,13 +135,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   Region createFileRegion(final RegionShortcut regionShortCut,
                                 final String fileRegionName,
                                 final PartitionAttributes partitionAttributes) {
-    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<String, File>();
-    partitionAttributesFactory.setColocatedWith(regionPath);
-    configureLuceneRegionAttributesFactory(partitionAttributesFactory, partitionAttributes);
-
-    return cache.<String, File> createRegionFactory(regionShortCut)
-        .setPartitionAttributes(partitionAttributesFactory.create())
-        .create(fileRegionName);
+    return createRegion(fileRegionName, regionShortCut, this.regionPath, partitionAttributes);
   }
 
   String createFileRegionName() {
@@ -158,13 +149,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   Region<ChunkKey, byte[]> createChunkRegion(final RegionShortcut regionShortCut,
                            final String fileRegionName,
                            final PartitionAttributes partitionAttributes, final String chunkRegionName) {
-    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<String, File>();
-    partitionAttributesFactory.setColocatedWith(fileRegionName);
-    configureLuceneRegionAttributesFactory(partitionAttributesFactory, partitionAttributes);
-
-    return cache.<ChunkKey, byte[]> createRegionFactory(regionShortCut)
-      .setPartitionAttributes(partitionAttributesFactory.create())
-      .create(chunkRegionName);
+    return createRegion(chunkRegionName, regionShortCut, fileRegionName, partitionAttributes);
   }
 
   String createChunkRegionName() {
@@ -177,6 +162,21 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return attributesFactory;
   }
 
+  protected <K, V> Region<K, V> createRegion(final String regionName, final RegionShortcut regionShortCut,
+      final String colocatedWithRegionName, final PartitionAttributes partitionAttributes) {
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<String, File>();
+    partitionAttributesFactory.setColocatedWith(colocatedWithRegionName);
+    configureLuceneRegionAttributesFactory(partitionAttributesFactory, partitionAttributes);
+
+    // Create AttributesFactory based on input RegionShortcut
+    RegionAttributes baseAttributes = this.cache.getRegionAttributes(regionShortCut.toString());
+    AttributesFactory factory = new AttributesFactory(baseAttributes);
+    factory.setPartitionAttributes(partitionAttributesFactory.create());
+    RegionAttributes<K, V> attributes = factory.create();
+
+    return createRegion(regionName, attributes);
+  }
+
   public void close() {
     // TODO Auto-generated method stub
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fa22f0d/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
index cd07672..c24cb9d 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
@@ -29,6 +29,7 @@ import com.gemstone.gemfire.cache.Cache;
 public class LuceneIndexForReplicatedRegion extends LuceneIndexImpl {
 
   public LuceneIndexForReplicatedRegion(String indexName, String regionPath, Cache cache) {
+    super(indexName, regionPath, cache);
     throw new UnsupportedOperationException("Lucene indexes on replicated regions is not yet implemented");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fa22f0d/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index 1158fd1..0b5f8fa 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -22,6 +22,12 @@ package com.gemstone.gemfire.cache.lucene.internal;
 import java.util.Collections;
 import java.util.Map;
 
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
@@ -50,6 +56,14 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   protected boolean hasInitialized = false;
   protected Map<String, Analyzer> fieldAnalyzers;
 
+  protected final Cache cache;
+  
+  protected LuceneIndexImpl(String indexName, String regionPath, Cache cache) {
+    this.indexName = indexName;
+    this.regionPath = regionPath;
+    this.cache = cache;
+  }
+
   @Override
   public String getName() {
     return this.indexName;
@@ -108,4 +122,19 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     creation.setFieldAnalyzers(this.getFieldAnalyzers());
     dataRegion.getExtensionPoint().addExtension(creation);
   }
+
+  protected <K, V> Region<K, V> createRegion(final String regionName, final RegionAttributes<K, V> attributes) {
+    // Create InternalRegionArguments to set isUsedForMetaRegion true to suppress xml generation (among other things)
+    InternalRegionArguments ira = new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
+        .setSnapshotInputStream(null).setImageTarget(null).setIsUsedForMetaRegion(true);
+
+    // Create the region
+    try {
+      return ((GemFireCacheImpl)this.cache).createVMRegion(regionName, attributes, ira);
+    } catch (Exception e) {
+      InternalGemFireError ige = new InternalGemFireError(LocalizedStrings.GemFireCache_UNEXPECTED_EXCEPTION.toLocalizedString());
+      ige.initCause(e);
+      throw ige;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fa22f0d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java
new file mode 100644
index 0000000..e04d1c7
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.junit.Assert.*;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
+
+  @Test
+  public void verifyLuceneRegionInternal() {
+    // Create cache
+    createCache();
+
+    // Create index
+    String indexName = "index";
+    String regionName = this.name.getMethodName();
+    createIndex(indexName, regionName, "text");
+
+    // Create partitioned region
+    createRegion(regionName);
+
+    // Get index
+    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) LuceneServiceProvider.get(this.cache).getIndex(indexName, regionName);
+    assertNotNull(index);
+
+    // Verify the meta regions exist and are internal
+    LocalRegion chunkRegion = (LocalRegion) this.cache.getRegion(index.createChunkRegionName());
+    assertNotNull(chunkRegion);
+    assertTrue(chunkRegion.isInternalRegion());
+    LocalRegion fileRegion = (LocalRegion) this.cache.getRegion(index.createFileRegionName());
+    assertNotNull(fileRegion);
+    assertTrue(fileRegion.isInternalRegion());
+
+    // Verify the meta regions are not contained in the root regions
+    for (Region region : cache.rootRegions()) {
+      assertNotEquals(chunkRegion.getFullPath(), region.getFullPath());
+      assertNotEquals(fileRegion.getFullPath(), region.getFullPath());
+    }
+  }
+
+  private Region createRegion(String regionName) {
+    return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(regionName);
+  }
+
+  private void createIndex(String indexName, String regionName, String fieldName) {
+    LuceneServiceProvider.get(this.cache).createIndex(indexName, regionName, fieldName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fa22f0d/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
index ff22f85..984b221 100644
--- 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
@@ -27,12 +27,15 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheListener;
 import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.MembershipAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributes;
 import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.extension.ExtensionPoint;
 import com.gemstone.gemfire.test.fake.Fakes;
@@ -113,38 +116,6 @@ public class LuceneIndexForPartitionedRegionTest {
   }
 
   @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";
@@ -181,11 +152,9 @@ public class LuceneIndexForPartitionedRegionTest {
     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);
@@ -195,6 +164,20 @@ public class LuceneIndexForPartitionedRegionTest {
     when(region.getExtensionPoint()).thenReturn(extensionPoint);
   }
 
+  private PartitionAttributes initializeAttributes(final Cache cache) {
+    PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
+    RegionAttributes attributes = mock(RegionAttributes.class);
+    when(attributes.getCacheListeners()).thenReturn(new CacheListener[0]);
+    when(attributes.getRegionTimeToLive()).thenReturn(ExpirationAttributes.DEFAULT);
+    when(attributes.getRegionIdleTimeout()).thenReturn(ExpirationAttributes.DEFAULT);
+    when(attributes.getEntryTimeToLive()).thenReturn(ExpirationAttributes.DEFAULT);
+    when(attributes.getEntryIdleTimeout()).thenReturn(ExpirationAttributes.DEFAULT);
+    when(attributes.getMembershipAttributes()).thenReturn(new MembershipAttributes());
+    when(cache.getRegionAttributes(RegionShortcut.PARTITION.toString())).thenReturn(attributes);
+    when(partitionAttributes.getTotalNumBuckets()).thenReturn(113);
+    return partitionAttributes;
+  }
+
   @Test
   public void initializeWithNoLocalMemoryThrowsException() {
     expectedExceptions.expect(IllegalStateException.class);
@@ -285,6 +268,35 @@ public class LuceneIndexForPartitionedRegionTest {
   }
 
   @Test
+  public void createFileRegionWithPartitionShortcutCreatesRegionUsingCreateVMRegion() throws Exception {
+    String name = "indexName";
+    String regionPath = "regionName";
+    GemFireCacheImpl cache = Fakes.cache();
+    PartitionAttributes partitionAttributes = initializeAttributes(cache);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    LuceneIndexForPartitionedRegion indexSpy = spy(index);
+    indexSpy.createFileRegion(RegionShortcut.PARTITION, index.createFileRegionName(), partitionAttributes);
+    String fileRegionName = index.createFileRegionName();
+    verify(indexSpy).createRegion(fileRegionName, RegionShortcut.PARTITION, regionPath, partitionAttributes);
+    verify(cache).createVMRegion(eq(fileRegionName), any(), any());
+  }
+
+  @Test
+  public void createChunkRegionWithPartitionShortcutCreatesRegionUsingCreateVMRegion() throws Exception {
+    String name = "indexName";
+    String regionPath = "regionName";
+    GemFireCacheImpl cache = Fakes.cache();
+    PartitionAttributes partitionAttributes = initializeAttributes(cache);
+    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
+    LuceneIndexForPartitionedRegion indexSpy = spy(index);
+    String chunkRegionName = index.createChunkRegionName();
+    String fileRegionName = index.createFileRegionName();
+    indexSpy.createChunkRegion(RegionShortcut.PARTITION, fileRegionName, partitionAttributes, chunkRegionName);
+    verify(indexSpy).createRegion(chunkRegionName, RegionShortcut.PARTITION, fileRegionName, partitionAttributes);
+    verify(cache).createVMRegion(eq(chunkRegionName), any(), any());
+  }
+
+  @Test
   public void initializeShouldCreatePartitionPersistentChunkRegion() {
     boolean withPersistence = true;
     String name = "indexName";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fa22f0d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java
new file mode 100644
index 0000000..4d94042
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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 com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+
+public class LuceneIntegrationTest {
+
+  protected Cache cache;
+
+  @Rule
+  public TestName name = new TestName();
+
+  @After
+  public void tearDown() {
+    if(this.cache != null) {
+      this.cache.close();
+    }
+  }
+
+  protected void createCache() {
+    CacheFactory cf = new CacheFactory();
+    cf.set("mcast-port", "0");
+    this.cache = cf.create();
+  }
+}