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:48 UTC

[1/3] incubator-geode git commit: Changing a reference from GemFire to Geode

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-11 5dadbb4fe -> 09c9342ce


Changing a reference from GemFire to Geode


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

Branch: refs/heads/feature/GEODE-11
Commit: 62952c17c3c2e4af37d52174137b8534814faab0
Parents: 5dadbb4
Author: Dan Smith <up...@apache.org>
Authored: Fri Oct 9 10:17:56 2015 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue Oct 13 10:14:03 2015 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/62952c17/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
index a113fab..cad9095 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
@@ -8,7 +8,7 @@ import com.gemstone.gemfire.cache.query.QueryException;
 
 /**
  * The instances of this class will be used for distributing Lucene Query objects and re-constructing the Query object.
- * If necessary the implementation needs to take care of serializing and de-serializing Lucene Query object. GemFire
+ * If necessary the implementation needs to take care of serializing and de-serializing Lucene Query object. Geode
  * respects the DataSerializable contract to provide optimal object serialization. For instance,
  * {@link LuceneQueryProvider}'s toData method will be used to serialize it when it is sent to another member of the
  * distributed system. Implementation of DataSerializable can provide a zero-argument constructor that will be invoked


[2/3] incubator-geode git commit: Adding an internal listener that can modify region attributes

Posted by up...@apache.org.
Adding an internal listener that can modify region attributes

This listener is attached to the cache and gets a callback before any
region is created. This can be used by extensions to modify region
attributes to create resources need for a region before the region is
created.


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

Branch: refs/heads/feature/GEODE-11
Commit: 485ebf9512ba11a19a469078ab136f80a1a85f7b
Parents: 62952c1
Author: Dan Smith <up...@apache.org>
Authored: Wed Oct 14 10:50:45 2015 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu Oct 15 10:59:17 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/GemFireCacheImpl.java        | 27 +++++++++++
 .../gemfire/internal/cache/LocalRegion.java     |  8 ++--
 .../gemfire/internal/cache/RegionListener.java  | 30 +++++++++++++
 .../internal/cache/RegionListenerJUnitTest.java | 47 ++++++++++++++++++++
 4 files changed, 108 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index b0f0f43..e73af49 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -198,6 +198,7 @@ import com.gemstone.gemfire.internal.cache.wan.parallel.ParallelGatewaySenderQue
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlParser;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlPropertyResolver;
 import com.gemstone.gemfire.internal.cache.xmlcache.PropertyResolver;
+import com.gemstone.gemfire.internal.concurrent.ConcurrentHashSet;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.jndi.JNDIInvoker;
 import com.gemstone.gemfire.internal.jta.TransactionManagerImpl;
@@ -589,6 +590,8 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   
   private final CqService cqService;
   
+  private final Set<RegionListener> regionListeners = new ConcurrentHashSet<RegionListener>();
+  
   public static final int DEFAULT_CLIENT_FUNCTION_TIMEOUT = 0;
 
   private static int clientFunctionTimeout;
@@ -3023,6 +3026,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     stopper.checkCancelInProgress(null);
     LocalRegion.validateRegionName(name);
     RegionAttributes<K, V> attrs = p_attrs;
+    attrs = invokeRegionBefore(null, name, attrs, internalRegionArgs);
     if (attrs == null) {
       throw new IllegalArgumentException(LocalizedStrings.GemFireCache_ATTRIBUTES_MUST_NOT_BE_NULL.toLocalizedString());
     }
@@ -3158,6 +3162,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       throw ex;
     }
 
+    invokeRegionAfter(rgn);
     /**
      * Added for M&M . Putting the callback here to avoid creating RegionMBean in case of Exception
      **/
@@ -3168,6 +3173,20 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     return rgn;
   }
 
+  public RegionAttributes invokeRegionBefore(LocalRegion parent,
+      String name, RegionAttributes attrs, InternalRegionArguments internalRegionArgs) {
+    for(RegionListener listener : regionListeners) {
+      attrs = listener.beforeCreate(parent, name, attrs, internalRegionArgs);
+    }
+    return attrs;
+  }
+  
+  public void invokeRegionAfter(LocalRegion region) {
+    for(RegionListener listener : regionListeners) {
+      listener.afterCreate(region);
+    }
+  }
+
   /**
    * turn on eviction by default for HDFS regions
    */
@@ -3713,6 +3732,14 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       return cacheLifecycleListeners.remove(l);
     }
   }
+  
+  public void addRegionListener(RegionListener l ) {
+    this.regionListeners.add(l);
+  }
+  
+  public void removeRegionListener(RegionListener l ) {
+    this.regionListeners.remove(l);
+  }
 
   /**
    * Creates the single instance of the Transation Manager for this cache. Returns the existing one upon request.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 9e5bcd2..22d42a9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -8,7 +8,7 @@
 
 package com.gemstone.gemfire.internal.cache;
 
-import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.*;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -114,8 +114,8 @@ import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector;
 import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector.HdfsRegionManager;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.cache.query.FunctionDomainException;
-import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.IndexType;
 import com.gemstone.gemfire.cache.query.MultiIndexCreationException;
 import com.gemstone.gemfire.cache.query.NameResolutionException;
@@ -198,14 +198,12 @@ import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
-import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.internal.util.concurrent.FutureResult;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReadWriteLock;
-import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantReadWriteLock.StoppableWriteLock;
 import com.gemstone.org.jgroups.util.StringId;
 
 /**
@@ -941,6 +939,7 @@ public class LocalRegion extends AbstractRegion
     checkReadiness();
     LocalRegion newRegion = null;
     RegionAttributes regionAttributes = attrs;
+    attrs = cache.invokeRegionBefore(this, subregionName, attrs, internalRegionArgs);
     final InputStream snapshotInputStream = internalRegionArgs
         .getSnapshotInputStream();
     final boolean getDestroyLock = internalRegionArgs.getDestroyLockFlag();
@@ -1110,6 +1109,7 @@ public class LocalRegion extends AbstractRegion
       }
     }
 
+    cache.invokeRegionAfter(newRegion);
     return newRegion;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java
new file mode 100644
index 0000000..88b913c
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java
@@ -0,0 +1,30 @@
+package com.gemstone.gemfire.internal.cache;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+
+/**
+ * Callback on a cache that receives notifications about region creates.
+ * 
+ * If there are multiple listeners added to a cache they are invoked in a random
+ * order. See {@link GemFireCacheImpl#addRegionListener(RegionListener)}
+ * 
+ * If any of these callbacks throw an exception, that exception will get thrown
+ * out to the user creating the region and the region creation will fail.
+ */
+public interface RegionListener {
+  
+  /**
+   * Invoked before a region is created. This callback is allowed to modify the region
+   * attributes before the region is created. Note that it's generally a bad idea to modify
+   * the RegionAttributes in place; a new set of RegionAttributes should be returned that contain
+   * the modifications. InternalRegionArguments *may* be modified, but only if you are sure
+   * the caller is not going to reuse the InternalRegionArguments for something else.
+   */
+  public RegionAttributes beforeCreate(Region parent, String regionName, RegionAttributes attrs, InternalRegionArguments internalRegionArgs);
+
+  /**
+   * Invoked after a region is created.
+   */
+  public void afterCreate(Region region);
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java
new file mode 100644
index 0000000..1fc7e59
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java
@@ -0,0 +1,47 @@
+package com.gemstone.gemfire.internal.cache;
+
+import static org.junit.Assert.*;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class RegionListenerJUnitTest {
+
+  @Test
+  public void test() {
+    final AtomicBoolean afterCreateInvoked = new AtomicBoolean(); 
+    RegionListener listener = new RegionListener() {
+      
+      @Override
+      public RegionAttributes beforeCreate(Region parent, String regionName,
+          RegionAttributes attrs, InternalRegionArguments internalRegionArgs) {
+        AttributesFactory newAttrsFactory = new AttributesFactory(attrs);
+        newAttrsFactory.setDataPolicy(DataPolicy.EMPTY);
+        return newAttrsFactory.create();
+      }
+
+      @Override
+      public void afterCreate(Region region) {
+        afterCreateInvoked.set(true);
+      }
+    };
+    
+    GemFireCacheImpl cache = (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").create();
+    cache.addRegionListener(listener);
+    Region region = cache.createRegionFactory(RegionShortcut.REPLICATE).create("region");
+    assertEquals(DataPolicy.EMPTY, region.getAttributes().getDataPolicy());
+    assertTrue(afterCreateInvoked.get());
+  }
+
+}


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

Posted by up...@apache.org.
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
    */