You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2015/10/15 20:03:50 UTC

[3/3] incubator-geode git commit: Requiring lucene index to be created before the region

Requiring lucene index to be created before the region

At the moment, adding an AsyncEventQueue after a region is created has
issues - See GEODE-404.

Therefore, in order to create a lucene index programmatically, the
lucene index needs to be created before the region exists so that we can
create the AEQ ahead of time.

With this change, when an LuceneIndex is created, the actual creation is
deferred in a RegionListener callback. When the region is created, we
add the AEQ to the region attributes and create the colocated regions.


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

Branch: refs/heads/feature/GEODE-11
Commit: 09c9342ce034753618e53a93b21171594713b157
Parents: 485ebf9
Author: Dan Smith <up...@apache.org>
Authored: Wed Oct 14 13:15:04 2015 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu Oct 15 10:59:17 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneService.java     |  4 +-
 .../LuceneIndexForPartitionedRegion.java        |  7 +-
 .../lucene/internal/LuceneServiceImpl.java      | 98 ++++++++++++++------
 .../internal/xml/LuceneIndexCreation.java       | 12 ++-
 .../internal/LuceneRebalanceJUnitTest.java      | 16 +++-
 .../internal/LuceneServiceImplJUnitTest.java    | 18 +++-
 .../LuceneFunctionReadPathDUnitTest.java        |  7 +-
 ...neIndexXmlGeneratorIntegrationJUnitTest.java |  4 +-
 ...uceneIndexXmlParserIntegrationJUnitTest.java | 14 ++-
 9 files changed, 125 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
index eed80d9..e1623d6 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
@@ -64,7 +64,7 @@ public interface LuceneService {
    * @param fields
    * @return LuceneIndex object
    */
-  public LuceneIndex createIndex(String indexName, String regionPath, String... fields);
+  public void createIndex(String indexName, String regionPath, String... fields);
   
   /**
    * Create a lucene index using specified analyzer per field
@@ -75,7 +75,7 @@ public interface LuceneService {
    * @return LuceneIndex object
    *
    */
-  public LuceneIndex createIndex(String indexName, String regionPath,  
+  public void createIndex(String indexName, String regionPath,  
       Map<String, Analyzer> analyzerPerField);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index 2bf848f..d0bb328 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -98,12 +98,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       LuceneEventListener listener = new LuceneEventListener(repositoryManager);
       String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), regionPath);
       AsyncEventQueueImpl aeq = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
-      if (aeq == null) {
-        AsyncEventQueue indexQueue = factory.create(aeqId, listener);
-        dataRegion.getAttributesMutator().addAsyncEventQueueId(aeqId);
-      } else {
-        logger.info("The AEQ "+aeq+" is created at another member");
-      }
+      AsyncEventQueue indexQueue = factory.create(aeqId, listener);
 
       addExtension(dataRegion);
       hasInitialized = true;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
index 776d005..c4a2047 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
@@ -9,8 +9,10 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
+import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
@@ -26,7 +28,9 @@ import com.gemstone.gemfire.cache.lucene.internal.xml.LuceneServiceXmlGenerator;
 import com.gemstone.gemfire.internal.DSFIDFactory;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.RegionListener;
 import com.gemstone.gemfire.internal.cache.extension.Extensible;
 import com.gemstone.gemfire.internal.cache.xmlcache.XmlGenerator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -40,11 +44,11 @@ import com.gemstone.gemfire.internal.logging.LogService;
  * @since 8.5
  */
 public class LuceneServiceImpl implements InternalLuceneService {
-  private final Cache cache;
-
-  private final HashMap<String, LuceneIndex> indexMap;
-  
   private static final Logger logger = LogService.getLogger();
+  
+  private final GemFireCacheImpl cache;
+  private final HashMap<String, LuceneIndex> indexMap = new HashMap<String, LuceneIndex>();;
+  
 
   public LuceneServiceImpl(final Cache cache) {
     if (cache == null) {
@@ -57,9 +61,6 @@ public class LuceneServiceImpl implements InternalLuceneService {
 
     FunctionService.registerFunction(new LuceneFunction());
     registerDataSerializables();
-
-    // Initialize the Map which maintains indexes
-    this.indexMap = new HashMap<String, LuceneIndex>();
   }
   
   public static String getUniqueIndexName(String indexName, String regionPath) {
@@ -71,17 +72,72 @@ public class LuceneServiceImpl implements InternalLuceneService {
   }
 
   @Override
-  public LuceneIndex createIndex(String indexName, String regionPath, String... fields) {
-    LuceneIndexImpl index = createIndexRegions(indexName, regionPath);
-    if (index == null) {
-      return null;
+  public void createIndex(String indexName, String regionPath, String... fields) {
+    StandardAnalyzer analyzer = new StandardAnalyzer();
+    
+    createIndex(indexName, regionPath, analyzer, fields);
+  }
+  
+  @Override
+  public void createIndex(String indexName, String regionPath, Map<String, Analyzer> analyzerPerField) {
+    Analyzer analyzer = new PerFieldAnalyzerWrapper(new StandardAnalyzer(), analyzerPerField);
+    String[] fields = (String[])analyzerPerField.keySet().toArray(new String[analyzerPerField.keySet().size()]);
+
+    createIndex(indexName, regionPath, analyzer, fields);
+  }
+
+  private void createIndex(final String indexName, String regionPath,
+      final Analyzer analyzer, final String... fields) {
+
+    if(!regionPath.startsWith("/")) {
+      regionPath = "/" + regionPath;
+    }
+    Region region = cache.getRegion(regionPath);
+    if(region != null) {
+      throw new IllegalStateException("The lucene index must be created before region");
     }
+    
+    final String dataRegionPath = regionPath;
+    cache.addRegionListener(new RegionListener() {
+      @Override
+      public RegionAttributes beforeCreate(Region parent, String regionName,
+          RegionAttributes attrs, InternalRegionArguments internalRegionArgs) {
+        String path = parent == null ? "/" + regionName : parent.getFullPath() + "/" + regionName;
+        if(path.equals(dataRegionPath)) {
+          String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, dataRegionPath);
+          AttributesFactory af = new AttributesFactory(attrs);
+          af.addAsyncEventQueueId(aeqId);
+          return af.create();
+        } else {
+          return attrs;
+        }
+      }
+      
+      @Override
+      public void afterCreate(Region region) {
+        if(region.getFullPath().equals(dataRegionPath)) {
+          afterDataRegionCreated(indexName, analyzer, dataRegionPath, fields);
+          cache.removeRegionListener(this);
+        }
+      }
+    });
+    
+  }
+  
+  /**
+   * Finish creating the lucene index after the data region is created .
+   * 
+   * Public because this is called by the Xml parsing code
+   */
+  public void afterDataRegionCreated(final String indexName,
+      final Analyzer analyzer, final String dataRegionPath,
+      final String... fields) {
+    LuceneIndexImpl index = createIndexRegions(indexName, dataRegionPath);
     index.setSearchableFields(fields);
     // for this API, set index to use the default StandardAnalyzer for each field
-    index.setAnalyzer(null);
+    index.setAnalyzer(analyzer);
     index.initialize();
     registerIndex(index);
-    return index;
   }
   
   private LuceneIndexImpl createIndexRegions(String indexName, String regionPath) {
@@ -119,22 +175,6 @@ public class LuceneServiceImpl implements InternalLuceneService {
   }
 
   @Override
-  public LuceneIndex createIndex(String indexName, String regionPath, Map<String, Analyzer> analyzerPerField) {
-    LuceneIndexImpl index = createIndexRegions(indexName, regionPath);
-    if (index == null) {
-      return null;
-    }
-    
-    Analyzer analyzer = new PerFieldAnalyzerWrapper(new StandardAnalyzer(), analyzerPerField);
-    String[] fields = (String[])analyzerPerField.keySet().toArray(new String[analyzerPerField.keySet().size()]);
-    index.setSearchableFields(fields);
-    index.setAnalyzer(analyzer);
-    index.initialize();
-    registerIndex(index);
-    return index;
-  }
-
-  @Override
   public void destroyIndex(LuceneIndex index) {
     LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
     indexMap.remove(getUniqueIndexName(index.getName(), index.getRegionPath()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
index 5520f96..02ecd8f 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -6,12 +6,14 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 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.internal.cache.extension.Extensible;
 import com.gemstone.gemfire.internal.cache.extension.Extension;
 import com.gemstone.gemfire.internal.cache.xmlcache.XmlGenerator;
@@ -68,9 +70,13 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
       Extensible<Region<?, ?>> target) {
     target.getExtensionPoint().addExtension(LuceneIndex.class, this);
     Cache cache = target.getExtensionPoint().getTarget().getCache();
-    LuceneService service = LuceneServiceProvider.get(cache);
-    //TODO - should this be a different method than the public API here?
-    service.createIndex(getName(), getRegionPath(), getFieldNames());
+    LuceneServiceImpl service = (LuceneServiceImpl) LuceneServiceProvider.get(cache);
+    Region region = target.getExtensionPoint().getTarget();
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), getRegionPath());
+    //Here, it is safe to add the aeq with the mutator, because onCreate is
+    //fired in a special place before the region is initialized.
+    region.getAttributesMutator().addAsyncEventQueueId(aeqId);
+    service.afterDataRegionCreated(getName(), new StandardAnalyzer(), getRegionPath(), getFieldNames());
   }
 
   public void addField(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneRebalanceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneRebalanceJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneRebalanceJUnitTest.java
index 478981f..afddb4a 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneRebalanceJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneRebalanceJUnitTest.java
@@ -1,11 +1,12 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
-import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -21,10 +22,11 @@ import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class LuceneRebalanceJUnitTest {
   String[] indexedFields = new String[] { "txt" };
   HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
@@ -37,6 +39,14 @@ public class LuceneRebalanceJUnitTest {
     analyzer = new StandardAnalyzer();
     LuceneServiceImpl.registerDataSerializables();
   }
+  
+  @After
+  public void tearDown() {
+    Cache cache = GemFireCacheImpl.getInstance();
+    if(cache != null) {
+      cache.close();
+    }
+  }
 
   /**
    * Test what happens when a bucket is destroyed.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index eff2813..26487a0 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -120,12 +120,23 @@ public class LuceneServiceImplJUnitTest {
       return region;
     }
   }
+  
+  /**Test that we don't allow the user
+   * to create the region first.
+   */
+  @Test(expected = IllegalStateException.class)
+  public void createRegionFirst() throws IOException, ParseException {
+    getService();
+    LocalRegion userRegion = createPR("PR1", false);
+    service.createIndex("index1", "PR1", "field1", "field2", "field3");
+  }
 
   @Test
   public void testCreateIndexForPR() throws IOException, ParseException {
     getService();
+    service.createIndex("index1", "PR1", "field1", "field2", "field3");
     LocalRegion userRegion = createPR("PR1", false);
-    LuceneIndexImpl index1 = (LuceneIndexImpl)service.createIndex("index1", "PR1", "field1", "field2", "field3");
+    LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
     assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
     LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion)index1;
     assertEquals("index1", index1.getName());
@@ -155,7 +166,6 @@ public class LuceneServiceImplJUnitTest {
   @Test
   public void testCreateIndexForPRWithAnalyzer() throws IOException, ParseException {
     getService();
-    createPR("PR1", false);
     StandardAnalyzer sa = new StandardAnalyzer();
     KeywordAnalyzer ka = new KeywordAnalyzer();
     Map<String, Analyzer> analyzerPerField = new HashMap<String, Analyzer>();
@@ -165,7 +175,9 @@ public class LuceneServiceImplJUnitTest {
     //  field2 and field3 will use StandardAnalyzer
     PerFieldAnalyzerWrapper analyzer2 = new PerFieldAnalyzerWrapper(sa, analyzerPerField);
 
-    LuceneIndexImpl index1 = (LuceneIndexImpl)service.createIndex("index1", "PR1", 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());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
index 6e44b72..e5c1a4e 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionReadPathDUnitTest.java
@@ -60,14 +60,11 @@ public class LuceneFunctionReadPathDUnitTest extends CacheTestCase {
       public Object call() throws Exception {
         final Cache cache = getCache();
         assertNotNull(cache);
-        // TODO: we have to workarround it now: specify an AEQ id when creating data region
-        String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+        LuceneService service = LuceneServiceProvider.get(cache);
+        service.createIndex(INDEX_NAME, REGION_NAME, "text");
         RegionFactory<Object, Object> regionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
         Region<Object, Object> region = regionFactory.
-            addAsyncEventQueueId(aeqId). // TODO: we need it for the time being
             create(REGION_NAME);
-        LuceneService service = LuceneServiceProvider.get(cache);
-        InternalLuceneIndex index = (InternalLuceneIndex) service.createIndex(INDEX_NAME, REGION_NAME, "text");
         return null;
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
index 65c73f7..e991643 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
@@ -28,10 +28,10 @@ public class LuceneIndexXmlGeneratorIntegrationJUnitTest {
   @Test
   public void generateWithFields() {
     Cache cache = new CacheFactory().set("mcast-port", "0").create();
-    cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
     LuceneService service = LuceneServiceProvider.get(cache);
-    
     service.createIndex("index", "region", "a", "b", "c");
+    cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
+    
     
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintWriter pw = new PrintWriter(baos);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c9342c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.java
index 56a726f..d97e160 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.java
@@ -8,6 +8,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.junit.After;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -18,19 +19,28 @@ import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.extension.Extension;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlParser;
 import com.gemstone.gemfire.internal.cache.xmlcache.RegionCreation;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.gemstone.gemfire.util.test.TestUtil;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class LuceneIndexXmlParserIntegrationJUnitTest {
 
   @Rule
   public TestName name = new TestName();
   
+  @After
+  public void tearDown() {
+    Cache cache = GemFireCacheImpl.getInstance();
+    if(cache != null) {
+      cache.close();
+    }
+  }
+  
   /**
    * Test that we parse the index fields correctly
    */